SOLR-12729: SplitShardCmd should lock the parent shard to prevent parallel splitting requests.

This commit is contained in:
Andrzej Bialecki 2018-10-09 21:29:38 +02:00
parent a52d47a1a5
commit 23e22e6460
10 changed files with 243 additions and 33 deletions

View File

@ -182,6 +182,8 @@ Bug Fixes
* SOLR-12836: ZkController creates a cloud solr client with no connection or read timeouts. Now the http client * SOLR-12836: ZkController creates a cloud solr client with no connection or read timeouts. Now the http client
created by the update shard handler is used instead. (shalin) created by the update shard handler is used instead. (shalin)
* SOLR-12729: SplitShardCmd should lock the parent shard to prevent parallel splitting requests. (ab)
Improvements Improvements
---------------------- ----------------------

View File

@ -61,6 +61,7 @@ import org.apache.solr.handler.component.ShardHandler;
import org.apache.solr.update.SolrIndexSplitter; import org.apache.solr.update.SolrIndexSplitter;
import org.apache.solr.util.RTimerTree; import org.apache.solr.util.RTimerTree;
import org.apache.solr.util.TestInjection; import org.apache.solr.util.TestInjection;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.data.Stat;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -116,7 +117,8 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
Slice parentSlice = getParentSlice(clusterState, collectionName, slice, splitKey); Slice parentSlice = getParentSlice(clusterState, collectionName, slice, splitKey);
if (parentSlice.getState() != Slice.State.ACTIVE) { if (parentSlice.getState() != Slice.State.ACTIVE) {
throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Parent slice is not active: " + parentSlice.getState()); throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Parent slice is not active: " +
collectionName + "/ " + parentSlice.getName() + ", state=" + parentSlice.getState());
} }
// find the leader for the shard // find the leader for the shard
@ -172,6 +174,14 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
parentShardLeader.getType()); parentShardLeader.getType());
} }
// check for the lock
if (!lockForSplit(ocmh.cloudManager, 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?): " +
collectionName + "/" + parentSlice.getName());
}
List<Map<String, Object>> replicas = new ArrayList<>((repFactor - 1) * 2); List<Map<String, Object>> replicas = new ArrayList<>((repFactor - 1) * 2);
t = timings.sub("fillRanges"); t = timings.sub("fillRanges");
@ -502,6 +512,8 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
results.add(CommonParams.TIMING, timings.asNamedList()); results.add(CommonParams.TIMING, timings.asNamedList());
} }
success = true; success = true;
// don't unlock the shard yet - only do this if the final switch-over in
// ReplicaMutator succeeds (or fails)
return true; return true;
} catch (SolrException e) { } catch (SolrException e) {
throw e; throw e;
@ -512,6 +524,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
if (sessionWrapper != null) sessionWrapper.release(); if (sessionWrapper != null) sessionWrapper.release();
if (!success) { if (!success) {
cleanupAfterFailure(zkStateReader, collectionName, parentSlice.getName(), subSlices, offlineSlices); cleanupAfterFailure(zkStateReader, collectionName, parentSlice.getName(), subSlices, offlineSlices);
unlockForSplit(ocmh.cloudManager, collectionName, parentSlice.getName());
} }
} }
} }
@ -740,4 +753,26 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
} }
return rangesStr; return rangesStr;
} }
public static boolean lockForSplit(SolrCloudManager cloudManager, String collection, String shard) throws Exception {
String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + "/" + shard + "-splitting";
if (cloudManager.getDistribStateManager().hasData(path)) {
return false;
}
Map<String, Object> map = new HashMap<>();
map.put(ZkStateReader.STATE_TIMESTAMP_PROP, String.valueOf(cloudManager.getTimeSource().getEpochTimeNs()));
byte[] data = Utils.toJSON(map);
try {
cloudManager.getDistribStateManager().makePath(path, data, CreateMode.EPHEMERAL, true);
} catch (Exception e) {
throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Can't lock parent slice for splitting (another split operation running?): " +
collection + "/" + shard, e);
}
return true;
}
public static void unlockForSplit(SolrCloudManager cloudManager, String collection, String shard) throws Exception {
String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + "/" + shard + "-splitting";
cloudManager.getDistribStateManager().removeRecursively(path, true, true);
}
} }

View File

@ -21,7 +21,9 @@ import java.util.ArrayList;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.NoSuchElementException;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.cloud.SolrCloudManager; import org.apache.solr.client.solrj.cloud.SolrCloudManager;
@ -29,6 +31,7 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Slice; import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.core.SolrResourceLoader;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -74,6 +77,7 @@ public class InactiveShardPlanAction extends TriggerActionBase {
ClusterState state = cloudManager.getClusterStateProvider().getClusterState(); ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
Map<String, List<String>> cleanup = new LinkedHashMap<>(); Map<String, List<String>> cleanup = new LinkedHashMap<>();
Map<String, List<String>> inactive = new LinkedHashMap<>(); Map<String, List<String>> inactive = new LinkedHashMap<>();
Map<String, Map<String, Object>> staleLocks = new LinkedHashMap<>();
state.forEachCollection(coll -> state.forEachCollection(coll ->
coll.getSlices().forEach(s -> { coll.getSlices().forEach(s -> {
if (Slice.State.INACTIVE.equals(s.getState())) { if (Slice.State.INACTIVE.equals(s.getState())) {
@ -94,12 +98,54 @@ public class InactiveShardPlanAction extends TriggerActionBase {
cleanup.computeIfAbsent(coll.getName(), c -> new ArrayList<>()).add(s.getName()); cleanup.computeIfAbsent(coll.getName(), c -> new ArrayList<>()).add(s.getName());
} }
} }
// check for stale shard split locks
String parentPath = ZkStateReader.COLLECTIONS_ZKNODE + "/" + coll.getName();
List<String> locks;
try {
locks = cloudManager.getDistribStateManager().listData(parentPath).stream()
.filter(name -> name.endsWith("-splitting"))
.collect(Collectors.toList());
for (String lock : locks) {
try {
String lockPath = parentPath + "/" + lock;
Map<String, Object> lockData = Utils.getJson(cloudManager.getDistribStateManager(), lockPath);
String tstampStr = (String)lockData.get(ZkStateReader.STATE_TIMESTAMP_PROP);
if (tstampStr == null || tstampStr.isEmpty()) {
return;
}
long timestamp = Long.parseLong(tstampStr);
// this timestamp uses epoch time
long currentTime = cloudManager.getTimeSource().getEpochTimeNs();
long delta = TimeUnit.NANOSECONDS.toSeconds(currentTime - timestamp);
log.debug("{}/{}: locktstamp={}, time={}, delta={}", coll.getName(), lock, timestamp, currentTime, delta);
if (delta > cleanupTTL) {
log.debug("-- delete inactive split lock for {}/{}, delta={}", coll.getName(), lock, delta);
cloudManager.getDistribStateManager().removeData(lockPath, -1);
lockData.put("currentTimeNs", currentTime);
lockData.put("deltaSec", delta);
lockData.put("ttlSec", cleanupTTL);
staleLocks.put(coll.getName() + "/" + lock, lockData);
} else {
log.debug("-- lock " + coll.getName() + "/" + lock + " still active (delta=" + delta + ")");
}
} catch (NoSuchElementException nse) {
// already removed by someone else - ignore
}
}
} catch (Exception e) {
log.warn("Exception checking for inactive shard split locks in " + parentPath, e);
}
}) })
); );
Map<String, Object> results = new LinkedHashMap<>();
if (!cleanup.isEmpty()) { if (!cleanup.isEmpty()) {
Map<String, Object> results = new LinkedHashMap<>();
results.put("inactive", inactive); results.put("inactive", inactive);
results.put("cleanup", cleanup); results.put("cleanup", cleanup);
}
if (!staleLocks.isEmpty()) {
results.put("staleLocks", staleLocks);
}
if (!results.isEmpty()) {
context.getProperties().put(getName(), results); context.getProperties().put(getName(), results);
} }
} }

View File

@ -35,6 +35,7 @@ import org.apache.solr.cloud.CloudUtil;
import org.apache.solr.cloud.Overseer; import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.Assign; import org.apache.solr.cloud.api.collections.Assign;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler; import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
import org.apache.solr.cloud.api.collections.SplitShardCmd;
import org.apache.solr.common.SolrException; import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.cloud.DocCollection;
@ -43,6 +44,7 @@ import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps; import org.apache.solr.common.cloud.ZkNodeProps;
import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.Utils; import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TestInjection;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -476,6 +478,13 @@ public class ReplicaMutator {
propMap.put(subShardSlice.getName(), Slice.State.RECOVERY_FAILED.toString()); propMap.put(subShardSlice.getName(), Slice.State.RECOVERY_FAILED.toString());
} }
} }
TestInjection.injectSplitLatch();
try {
SplitShardCmd.unlockForSplit(cloudManager, collection.getName(), parentSliceName);
} catch (Exception e) {
log.warn("Failed to unlock shard after " + (isLeaderSame ? "" : "un") + "successful split: {} / {}",
collection.getName(), parentSliceName);
}
ZkNodeProps m = new ZkNodeProps(propMap); ZkNodeProps m = new ZkNodeProps(propMap);
return new SliceMutator(cloudManager).updateShardState(prevState, m).collection; return new SliceMutator(cloudManager).updateShardState(prevState, m).collection;
} }

View File

@ -49,12 +49,12 @@ public class SliceMutator {
public static final Set<String> SLICE_UNIQUE_BOOLEAN_PROPERTIES = ImmutableSet.of(PREFERRED_LEADER_PROP); public static final Set<String> SLICE_UNIQUE_BOOLEAN_PROPERTIES = ImmutableSet.of(PREFERRED_LEADER_PROP);
protected final SolrCloudManager dataProvider; protected final SolrCloudManager cloudManager;
protected final DistribStateManager stateManager; protected final DistribStateManager stateManager;
public SliceMutator(SolrCloudManager dataProvider) { public SliceMutator(SolrCloudManager cloudManager) {
this.dataProvider = dataProvider; this.cloudManager = cloudManager;
this.stateManager = dataProvider.getDistribStateManager(); this.stateManager = cloudManager.getDistribStateManager();
} }
public ZkWriteCommand addReplica(ClusterState clusterState, ZkNodeProps message) { public ZkWriteCommand addReplica(ClusterState clusterState, ZkNodeProps message) {
@ -140,9 +140,9 @@ public class SliceMutator {
String coreURL = ZkCoreNodeProps.getCoreUrl(replica.getStr(ZkStateReader.BASE_URL_PROP), replica.getStr(ZkStateReader.CORE_NAME_PROP)); String coreURL = ZkCoreNodeProps.getCoreUrl(replica.getStr(ZkStateReader.BASE_URL_PROP), replica.getStr(ZkStateReader.CORE_NAME_PROP));
if (replica == oldLeader && !coreURL.equals(leaderUrl)) { if (replica == oldLeader && !coreURL.equals(leaderUrl)) {
replica = new ReplicaMutator(dataProvider).unsetLeader(replica); replica = new ReplicaMutator(cloudManager).unsetLeader(replica);
} else if (coreURL.equals(leaderUrl)) { } else if (coreURL.equals(leaderUrl)) {
replica = new ReplicaMutator(dataProvider).setLeader(replica); replica = new ReplicaMutator(cloudManager).setLeader(replica);
} }
newReplicas.put(replica.getName(), replica); newReplicas.put(replica.getName(), replica);
@ -179,7 +179,7 @@ public class SliceMutator {
} }
props.put(ZkStateReader.STATE_PROP, message.getStr(key)); props.put(ZkStateReader.STATE_PROP, message.getStr(key));
// we need to use epoch time so that it's comparable across Overseer restarts // we need to use epoch time so that it's comparable across Overseer restarts
props.put(ZkStateReader.STATE_TIMESTAMP_PROP, String.valueOf(dataProvider.getTimeSource().getEpochTimeNs())); props.put(ZkStateReader.STATE_TIMESTAMP_PROP, String.valueOf(cloudManager.getTimeSource().getEpochTimeNs()));
Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props); Slice newSlice = new Slice(slice.getName(), slice.getReplicasCopy(), props);
slicesCopy.put(slice.getName(), newSlice); slicesCopy.put(slice.getName(), newSlice);
} }

View File

@ -24,6 +24,8 @@ import java.util.Random;
import java.util.Set; import java.util.Set;
import java.util.Timer; import java.util.Timer;
import java.util.TimerTask; import java.util.TimerTask;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; import java.util.regex.Pattern;
@ -135,6 +137,8 @@ public class TestInjection {
public static String splitFailureAfterReplicaCreation = null; public static String splitFailureAfterReplicaCreation = null;
public static CountDownLatch splitLatch = null;
public static String waitForReplicasInSync = "true:60"; public static String waitForReplicasInSync = "true:60";
public static String failIndexFingerprintRequests = null; public static String failIndexFingerprintRequests = null;
@ -159,6 +163,7 @@ public class TestInjection {
randomDelayInCoreCreation = null; randomDelayInCoreCreation = null;
splitFailureBeforeReplicaCreation = null; splitFailureBeforeReplicaCreation = null;
splitFailureAfterReplicaCreation = null; splitFailureAfterReplicaCreation = null;
splitLatch = null;
prepRecoveryOpPauseForever = null; prepRecoveryOpPauseForever = null;
countPrepRecoveryOpPauseForever = new AtomicInteger(0); countPrepRecoveryOpPauseForever = new AtomicInteger(0);
waitForReplicasInSync = "true:60"; waitForReplicasInSync = "true:60";
@ -413,6 +418,18 @@ public class TestInjection {
return injectSplitFailure(splitFailureAfterReplicaCreation, "after creating replica for sub-shard"); return injectSplitFailure(splitFailureAfterReplicaCreation, "after creating replica for sub-shard");
} }
public static boolean injectSplitLatch() {
if (splitLatch != null) {
try {
log.info("Waiting in ReplicaMutator for up to 60s");
return splitLatch.await(60, TimeUnit.SECONDS);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
return true;
}
@SuppressForbidden(reason = "Need currentTimeMillis, because COMMIT_TIME_MSEC_KEY use currentTimeMillis as value") @SuppressForbidden(reason = "Need currentTimeMillis, because COMMIT_TIME_MSEC_KEY use currentTimeMillis as value")
public static boolean waitForInSyncWithLeader(SolrCore core, ZkController zkController, String collection, String shardId) { public static boolean waitForInSyncWithLeader(SolrCore core, ZkController zkController, String collection, String shardId) {
if (waitForReplicasInSync == null) return true; if (waitForReplicasInSync == null) return true;

View File

@ -30,6 +30,7 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.lucene.util.LuceneTestCase.Slow; import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.client.solrj.SolrQuery; import org.apache.solr.client.solrj.SolrQuery;
@ -60,10 +61,12 @@ import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams; import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.ModifiableSolrParams; import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList; 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.common.util.Utils;
import org.apache.solr.update.SolrIndexSplitter; import org.apache.solr.update.SolrIndexSplitter;
import org.apache.solr.util.LogLevel; import org.apache.solr.util.LogLevel;
import org.apache.solr.util.TestInjection; import org.apache.solr.util.TestInjection;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.junit.Test; import org.junit.Test;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -540,6 +543,64 @@ public class ShardSplitTest extends AbstractFullDistribZkTestBase {
} }
} }
@Test
public void testSplitLocking() throws Exception {
waitForThingsToLevelOut(15);
String collectionName = "testSplitLocking";
CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1", 1, 2);
create.setMaxShardsPerNode(5); // some high number so we can create replicas without hindrance
create.process(cloudClient);
waitForRecoveriesToFinish(collectionName, false);
TestInjection.splitLatch = new CountDownLatch(1); // simulate a long split operation
String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/" + SHARD1 + "-splitting";
final AtomicReference<Exception> exc = new AtomicReference<>();
try {
Runnable r = () -> {
try {
trySplit(collectionName, null, SHARD1, 1);
} catch (Exception e) {
exc.set(e);
}
};
Thread t = new Thread(r);
t.start();
// wait for the split to start executing
TimeOut timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
timeOut.sleep(500);
if (cloudClient.getZkStateReader().getZkClient().exists(path, true)) {
log.info("=== found lock node");
break;
}
}
assertFalse("timed out waiting for the lock znode to appear", timeOut.hasTimedOut());
assertNull("unexpected exception: " + exc.get(), exc.get());
log.info("=== trying second split");
try {
trySplit(collectionName, null, SHARD1, 1);
fail("expected to fail due to locking but succeeded");
} catch (Exception e) {
log.info("Expected failure: " + e.toString());
}
// make sure the lock still exists
assertTrue("lock znode expected but missing", cloudClient.getZkStateReader().getZkClient().exists(path, true));
// let the first split proceed
TestInjection.splitLatch.countDown();
timeOut = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
while (!timeOut.hasTimedOut()) {
timeOut.sleep(500);
if (!cloudClient.getZkStateReader().getZkClient().exists(path, true)) {
break;
}
}
assertFalse("timed out waiting for the lock znode to disappear", timeOut.hasTimedOut());
} finally {
TestInjection.reset();
}
}
@Test @Test
public void testSplitShardWithRule() throws Exception { public void testSplitShardWithRule() throws Exception {
doSplitShardWithRule(SolrIndexSplitter.SplitMethod.REWRITE); doSplitShardWithRule(SolrIndexSplitter.SplitMethod.REWRITE);

View File

@ -19,6 +19,7 @@ package org.apache.solr.cloud.autoscaling;
import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
@ -35,10 +36,13 @@ import org.apache.solr.cloud.CloudTestUtils;
import org.apache.solr.cloud.SolrCloudTestCase; import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.cloud.autoscaling.sim.SimCloudManager; import org.apache.solr.cloud.autoscaling.sim.SimCloudManager;
import org.apache.solr.common.cloud.ClusterState; import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.NamedList; import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.TimeSource; import org.apache.solr.common.util.TimeSource;
import org.apache.solr.common.util.Utils;
import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.core.SolrResourceLoader;
import org.apache.solr.util.LogLevel; import org.apache.solr.util.LogLevel;
import org.apache.zookeeper.CreateMode;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
@ -167,6 +171,17 @@ public class ScheduledMaintenanceTriggerTest extends SolrCloudTestCase {
CloudTestUtils.waitForState(cloudManager, "failed to create " + collection1, collection1, CloudTestUtils.waitForState(cloudManager, "failed to create " + collection1, collection1,
CloudTestUtils.clusterShape(1, 1)); CloudTestUtils.clusterShape(1, 1));
// also create a very stale lock
Map<String, Object> lockData = new HashMap<>();
lockData.put(ZkStateReader.STATE_TIMESTAMP_PROP, String.valueOf(cloudManager.getTimeSource().getEpochTimeNs() -
TimeUnit.NANOSECONDS.convert(48, TimeUnit.HOURS)));
String staleLockName = collection1 + "/staleShard-splitting";
cloudManager.getDistribStateManager().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" +
staleLockName, Utils.toJSON(lockData), CreateMode.EPHEMERAL, true);
// expect two events - one for a very stale lock, one for the cleanup
triggerFired = new CountDownLatch(2);
String setListenerCommand = "{" + String setListenerCommand = "{" +
"'set-listener' : " + "'set-listener' : " +
"{" + "{" +
@ -186,10 +201,10 @@ public class ScheduledMaintenanceTriggerTest extends SolrCloudTestCase {
"'set-trigger' : {" + "'set-trigger' : {" +
"'name' : '" + AutoScaling.SCHEDULED_MAINTENANCE_TRIGGER_NAME + "'," + "'name' : '" + AutoScaling.SCHEDULED_MAINTENANCE_TRIGGER_NAME + "'," +
"'event' : 'scheduled'," + "'event' : 'scheduled'," +
"'startTime' : 'NOW+3SECONDS'," + "'startTime' : 'NOW+10SECONDS'," +
"'every' : '+2SECONDS'," + "'every' : '+2SECONDS'," +
"'enabled' : true," + "'enabled' : true," +
"'actions' : [{'name' : 'inactive_shard_plan', 'class' : 'solr.InactiveShardPlanAction', 'ttl' : '10'}," + "'actions' : [{'name' : 'inactive_shard_plan', 'class' : 'solr.InactiveShardPlanAction', 'ttl' : '20'}," +
"{'name' : 'execute_plan', 'class' : '" + ExecutePlanAction.class.getName() + "'}," + "{'name' : 'execute_plan', 'class' : '" + ExecutePlanAction.class.getName() + "'}," +
"{'name' : 'test', 'class' : '" + TestTriggerAction.class.getName() + "'}]" + "{'name' : 'test', 'class' : '" + TestTriggerAction.class.getName() + "'}]" +
"}}"; "}}";
@ -208,7 +223,7 @@ public class ScheduledMaintenanceTriggerTest extends SolrCloudTestCase {
CloudTestUtils.clusterShape(3, 1, true, true)); CloudTestUtils.clusterShape(3, 1, true, true));
await = triggerFired.await(60, TimeUnit.SECONDS); await = triggerFired.await(90, TimeUnit.SECONDS);
assertTrue("cleanup action didn't run", await); assertTrue("cleanup action didn't run", await);
// cleanup should have occurred // cleanup should have occurred
@ -217,21 +232,27 @@ public class ScheduledMaintenanceTriggerTest extends SolrCloudTestCase {
listenerEvents.clear(); listenerEvents.clear();
assertFalse(events.isEmpty()); assertFalse(events.isEmpty());
int inactiveEvents = 0;
CapturedEvent ce = null; CapturedEvent ce = null;
CapturedEvent staleLock = null;
for (CapturedEvent e : events) { for (CapturedEvent e : events) {
if (e.stage != TriggerEventProcessorStage.AFTER_ACTION) { if (e.stage != TriggerEventProcessorStage.AFTER_ACTION) {
continue; continue;
} }
if (e.context.containsKey("properties.inactive_shard_plan")) { Map<String, Object> plan = (Map<String, Object>)e.context.get("properties.inactive_shard_plan");
if (plan == null) {
continue;
}
if (plan.containsKey("cleanup")) {
ce = e; ce = e;
break; }
} else { // capture only the first
inactiveEvents++; if (plan.containsKey("staleLocks") && staleLock == null) {
staleLock = e;
} }
} }
assertTrue("should be at least one inactive event", inactiveEvents > 0); assertNotNull("missing cleanup event: " + events, ce);
assertNotNull("missing cleanup event", ce); assertNotNull("missing staleLocks event: " + events, staleLock);
Map<String, Object> map = (Map<String, Object>)ce.context.get("properties.inactive_shard_plan"); Map<String, Object> map = (Map<String, Object>)ce.context.get("properties.inactive_shard_plan");
assertNotNull(map); assertNotNull(map);
@ -242,6 +263,12 @@ public class ScheduledMaintenanceTriggerTest extends SolrCloudTestCase {
assertEquals(1, cleanup.size()); assertEquals(1, cleanup.size());
assertNotNull(cleanup.get(collection1)); assertNotNull(cleanup.get(collection1));
map = (Map<String, Object>)staleLock.context.get("properties.inactive_shard_plan");
assertNotNull(map);
Map<String, Map<String, Object>> locks = (Map<String, Map<String, Object>>)map.get("staleLocks");
assertNotNull(locks);
assertTrue("missing stale lock data: " + locks + "\nevents: " + events, locks.containsKey(staleLockName));
ClusterState state = cloudManager.getClusterStateProvider().getClusterState(); ClusterState state = cloudManager.getClusterStateProvider().getClusterState();
CloudTestUtils.clusterShape(2, 1).matches(state.getLiveNodes(), state.getCollection(collection1)); CloudTestUtils.clusterShape(2, 1).matches(state.getLiveNodes(), state.getCollection(collection1));

View File

@ -1151,12 +1151,12 @@ public class SimClusterStateProvider implements ClusterStateProvider {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Shard " + collectionName + throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Shard " + collectionName +
" / " + sliceName.get() + " has no leader and can't be split"); " / " + sliceName.get() + " has no leader and can't be split");
} }
SplitShardCmd.lockForSplit(cloudManager, collectionName, sliceName.get());
// start counting buffered updates // start counting buffered updates
Map<String, Object> props = sliceProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>()) Map<String, Object> props = sliceProperties.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>())
.computeIfAbsent(sliceName.get(), ss -> new ConcurrentHashMap<>()); .computeIfAbsent(sliceName.get(), ss -> new ConcurrentHashMap<>());
if (props.containsKey(BUFFERED_UPDATES)) { if (props.containsKey(BUFFERED_UPDATES)) {
log.debug("--- SOLR-12729: Overlapping splitShard commands for {} / {}", collectionName, sliceName.get()); throw new Exception("--- SOLR-12729: Overlapping splitShard commands for " + collectionName + "/" + sliceName.get());
return;
} }
props.put(BUFFERED_UPDATES, new AtomicLong()); props.put(BUFFERED_UPDATES, new AtomicLong());
@ -1240,20 +1240,28 @@ public class SimClusterStateProvider implements ClusterStateProvider {
// delay it once again to better simulate replica recoveries // delay it once again to better simulate replica recoveries
//opDelay(collectionName, CollectionParams.CollectionAction.SPLITSHARD.name()); //opDelay(collectionName, CollectionParams.CollectionAction.SPLITSHARD.name());
CloudTestUtils.waitForState(cloudManager, collectionName, 30, TimeUnit.SECONDS, (liveNodes, state) -> { boolean success = false;
for (String subSlice : subSlices) { try {
Slice s = state.getSlice(subSlice); CloudTestUtils.waitForState(cloudManager, collectionName, 30, TimeUnit.SECONDS, (liveNodes, state) -> {
if (s.getLeader() == null) { for (String subSlice : subSlices) {
log.debug("** no leader in {} / {}", collectionName, s); Slice s = state.getSlice(subSlice);
return false; if (s.getLeader() == null) {
} log.debug("** no leader in {} / {}", collectionName, s);
if (s.getReplicas().size() < repFactor) { return false;
log.debug("** expected {} repFactor but there are {} replicas", repFactor, s.getReplicas().size()); }
return false; if (s.getReplicas().size() < repFactor) {
log.debug("** expected {} repFactor but there are {} replicas", repFactor, s.getReplicas().size());
return false;
}
} }
return true;
});
success = true;
} finally {
if (!success) {
SplitShardCmd.unlockForSplit(cloudManager, collectionName, sliceName.get());
} }
return true; }
});
// mark the new slices as active and the old slice as inactive // mark the new slices as active and the old slice as inactive
log.trace("-- switching slice states after split shard: collection={}, parent={}, subSlices={}", collectionName, log.trace("-- switching slice states after split shard: collection={}, parent={}, subSlices={}", collectionName,
sliceName.get(), subSlices); sliceName.get(), subSlices);
@ -1292,6 +1300,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
// invalidate cached state // invalidate cached state
collectionsStatesRef.set(null); collectionsStatesRef.set(null);
} finally { } finally {
SplitShardCmd.unlockForSplit(cloudManager, collectionName, sliceName.get());
lock.unlock(); lock.unlock();
} }
results.add("success", ""); results.add("success", "");

View File

@ -66,6 +66,9 @@ public class TestSimExtremeIndexing extends SimSolrCloudTestCase {
// tweak this threshold to test the number of splits // tweak this threshold to test the number of splits
private static final long ABOVE_SIZE = 20000000; private static final long ABOVE_SIZE = 20000000;
// tweak this to allow more operations in one event
private static final int MAX_OPS = 100;
private static TimeSource timeSource; private static TimeSource timeSource;
private static SolrClient solrClient; private static SolrClient solrClient;
@ -100,6 +103,7 @@ public class TestSimExtremeIndexing extends SimSolrCloudTestCase {
"'event' : 'indexSize'," + "'event' : 'indexSize'," +
"'waitFor' : '" + waitForSeconds + "s'," + "'waitFor' : '" + waitForSeconds + "s'," +
"'aboveDocs' : " + ABOVE_SIZE + "," + "'aboveDocs' : " + ABOVE_SIZE + "," +
"'maxOps' : " + MAX_OPS + "," +
"'enabled' : true," + "'enabled' : true," +
"'actions' : [{'name' : 'compute_plan', 'class' : 'solr.ComputePlanAction'}," + "'actions' : [{'name' : 'compute_plan', 'class' : 'solr.ComputePlanAction'}," +
"{'name' : 'execute_plan', 'class' : '" + ExecutePlanAction.class.getName() + "'}]" + "{'name' : 'execute_plan', 'class' : '" + ExecutePlanAction.class.getName() + "'}]" +