Merge pull request #15299 from jasontedor/collections-shuffle-reproducibility

Remove and forbid use of Collections#shuffle(List) and Random#<init>()
This commit is contained in:
Jason Tedor 2015-12-11 11:24:56 -05:00
commit cd40cd78ba
37 changed files with 251 additions and 312 deletions

View File

@ -123,3 +123,6 @@ java.lang.System#getProperties() @ Use BootstrapInfo.getSystemProperties for a r
java.util.Collections#EMPTY_LIST
java.util.Collections#EMPTY_MAP
java.util.Collections#EMPTY_SET
java.util.Collections#shuffle(java.util.List) @ Use java.util.Collections#shuffle(java.util.List, java.util.Random) with a reproducible source of randomness
java.util.Random#<init>() @ Use org.elasticsearch.common.random.Randomness#create for reproducible sources of randomness

View File

@ -27,18 +27,11 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.index.shard.ShardId;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.*;
import java.util.function.Predicate;
/**
@ -671,7 +664,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
public void shuffle() {
Collections.shuffle(unassigned);
Randomness.shuffle(unassigned);
}
/**

View File

@ -39,15 +39,12 @@ import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.*;
import java.util.function.Function;
import java.util.stream.Collectors;

View File

@ -0,0 +1,132 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.common;
import org.elasticsearch.common.settings.Settings;
import java.lang.reflect.Method;
import java.security.SecureRandom;
import java.util.Collections;
import java.util.List;
import java.util.Random;
/**
* Provides factory methods for producing reproducible sources of
* randomness. Reproducible sources of randomness contribute to
* reproducible tests. When running the Elasticsearch test suite, the
* test runner will establish a global random seed accessible via the
* system property "tests.seed". By seeding a random number generator
* with this global seed, we ensure that instances of Random produced
* with this class produce reproducible sources of randomness under
* when running under the Elasticsearch test suite. Alternatively,
* a reproducible source of randomness can be produced by providing a
* setting a reproducible seed. When running the Elasticsearch server
* process, non-reproducible sources of randomness are provided (unless
* a setting is provided for a module that exposes a seed setting (e.g.,
* DiscoveryService#SETTING_DISCOVERY_SEED)).
*/
public final class Randomness {
private static final SecureRandom SR = new SecureRandom();
private static final Method currentMethod;
private static final Method getRandomMethod;
static {
Method maybeCurrentMethod;
Method maybeGetRandomMethod;
try {
Class<?> clazz = Class.forName("com.carrotsearch.randomizedtesting.RandomizedContext");
maybeCurrentMethod = clazz.getMethod("current");
maybeGetRandomMethod = clazz.getMethod("getRandom");
} catch (Throwable t) {
maybeCurrentMethod = null;
maybeGetRandomMethod = null;
}
currentMethod = maybeCurrentMethod;
getRandomMethod = maybeGetRandomMethod;
}
private Randomness() {}
/**
* Provides a reproducible source of randomness seeded by a long
* seed in the settings with the key setting.
*
* @param settings the settings containing the seed
* @param setting the key to access the seed
* @return a reproducible source of randomness
*/
public static Random get(Settings settings, String setting) {
Long maybeSeed = settings.getAsLong(setting, null);
if (maybeSeed != null) {
return new Random(maybeSeed);
} else {
return get();
}
}
/**
* Provides a source of randomness that is reproducible when
* running under the Elasticsearch test suite, and otherwise
* produces a non-reproducible source of randomness. Reproducible
* sources of randomness are created when the system property
* "tests.seed" is set and the security policy allows reading this
* system property. Otherwise, non-reproducible sources of
* randomness are created.
*
* @return a source of randomness
* @throws IllegalStateException if running tests but was not able
* to acquire an instance of Random from
* RandomizedContext or tests are
* running but tests.seed is not set
*/
public static Random get() {
if (currentMethod != null && getRandomMethod != null) {
try {
Object randomizedContext = currentMethod.invoke(null);
return (Random) getRandomMethod.invoke(randomizedContext);
} catch (ReflectiveOperationException e) {
// unexpected, bail
throw new IllegalStateException("running tests but failed to invoke RandomizedContext#getRandom", e);
}
} else {
return getWithoutSeed();
}
}
private static ThreadLocal<Random> LOCAL;
private static Random getWithoutSeed() {
assert currentMethod == null && getRandomMethod == null : "running under tests but tried to create non-reproducible random";
if (LOCAL.get() == null) {
byte[] bytes = SR.generateSeed(8);
long accumulator = 0;
for (int i = 0; i < bytes.length; i++) {
accumulator = (accumulator << 8) + bytes[i] & 0xFFL;
}
final long seed = accumulator;
LOCAL = ThreadLocal.withInitial(() -> new Random(seed));
}
return LOCAL.get();
}
public static void shuffle(List<?> list) {
Collections.shuffle(list, get());
}
}

View File

@ -23,6 +23,7 @@ import org.elasticsearch.ElasticsearchTimeoutException;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
@ -41,7 +42,6 @@ public class DiscoveryService extends AbstractLifecycleComponent<DiscoveryServic
public static final String SETTING_INITIAL_STATE_TIMEOUT = "discovery.initial_state_timeout";
public static final String SETTING_DISCOVERY_SEED = "discovery.id.seed";
private static class InitialStateListener implements InitialStateDiscoveryListener {
private final CountDownLatch latch = new CountDownLatch(1);
@ -132,10 +132,7 @@ public class DiscoveryService extends AbstractLifecycleComponent<DiscoveryServic
}
public static String generateNodeId(Settings settings) {
String seed = settings.get(DiscoveryService.SETTING_DISCOVERY_SEED);
if (seed != null) {
return Strings.randomBase64UUID(new Random(Long.parseLong(seed)));
}
return Strings.randomBase64UUID();
Random random = Randomness.get(settings, DiscoveryService.SETTING_DISCOVERY_SEED);
return Strings.randomBase64UUID(random);
}
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.plugins;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.*;
import org.elasticsearch.bootstrap.JarHell;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.cli.Terminal;
import org.elasticsearch.common.collect.Tuple;
@ -66,7 +67,7 @@ public class PluginManager {
"plugin",
"plugin.bat",
"service.bat"));
static final Set<String> MODULES = unmodifiableSet(newHashSet(
"lang-expression",
"lang-groovy"));
@ -124,7 +125,7 @@ public class PluginManager {
checkForForbiddenName(pluginHandle.name);
} else {
// if we have no name but url, use temporary name that will be overwritten later
pluginHandle = new PluginHandle("temp_name" + new Random().nextInt(), null, null);
pluginHandle = new PluginHandle("temp_name" + Randomness.get().nextInt(), null, null);
}
Path pluginFile = download(pluginHandle, terminal);
@ -224,7 +225,7 @@ public class PluginManager {
PluginInfo info = PluginInfo.readFromProperties(root);
terminal.println(VERBOSE, "%s", info);
// don't let luser install plugin as a module...
// don't let luser install plugin as a module...
// they might be unavoidably in maven central and are packaged up the same way)
if (MODULES.contains(info.getName())) {
throw new IOException("plugin '" + info.getName() + "' cannot be installed like this, it is a system module");

View File

@ -23,19 +23,8 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.DisjunctionMaxQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.QueryUtils;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.index.*;
import org.apache.lucene.search.*;
import org.apache.lucene.search.similarities.BM25Similarity;
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.search.similarities.Similarity;
@ -44,11 +33,7 @@ import org.apache.lucene.util.TestUtil;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.*;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.equalTo;

View File

@ -26,20 +26,12 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.collect.ImmutableOpenIntMap;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.*;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.transport.NodeDisconnectedException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.*;
import static org.hamcrest.Matchers.equalTo;
@ -119,7 +111,7 @@ public class IndicesShardStoreResponseTests extends ESTestCase {
orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 3, IndicesShardStoresResponse.StoreStatus.Allocation.REPLICA, new IOException("corrupted")));
List<IndicesShardStoresResponse.StoreStatus> storeStatuses = new ArrayList<>(orderedStoreStatuses);
Collections.shuffle(storeStatuses);
Collections.shuffle(storeStatuses, random());
CollectionUtil.timSort(storeStatuses);
assertThat(storeStatuses, equalTo(orderedStoreStatuses));
}

View File

@ -32,8 +32,6 @@ import org.elasticsearch.action.admin.indices.upgrade.UpgradeIT;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
@ -62,23 +60,12 @@ import org.junit.Before;
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.DirectoryStream;
import java.nio.file.FileVisitResult;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.SimpleFileVisitor;
import java.nio.file.*;
import java.nio.file.attribute.BasicFileAttributes;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.*;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
// needs at least 2 nodes since it bumps replicas to 1
@ -271,7 +258,7 @@ public class OldIndexBackwardsCompatibilityIT extends ESIntegTestCase {
public void testOldIndexes() throws Exception {
setupCluster();
Collections.shuffle(indexes, getRandom());
Collections.shuffle(indexes, random());
for (String index : indexes) {
long startTime = System.currentTimeMillis();
logger.info("--> Testing old index " + index);

View File

@ -29,11 +29,7 @@ import org.junit.BeforeClass;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.*;
import static java.util.Collections.emptyMap;
import static java.util.Collections.singletonMap;
@ -246,7 +242,7 @@ public class DiscoveryNodeFiltersTests extends ESTestCase {
private Settings shuffleSettings(Settings source) {
Settings.Builder settings = Settings.settingsBuilder();
List<String> keys = new ArrayList<>(source.getAsMap().keySet());
Collections.shuffle(keys, getRandom());
Collections.shuffle(keys, random());
for (String o : keys) {
settings.put(o, source.getAsMap().get(o));
}

View File

@ -170,7 +170,7 @@ public class DelayedAllocationIT extends ESIntegTestCase {
private String findNodeWithShard() {
ClusterState state = client().admin().cluster().prepareState().get().getState();
List<ShardRouting> startedShards = state.routingTable().shardsWithState(ShardRoutingState.STARTED);
Collections.shuffle(startedShards, getRandom());
Collections.shuffle(startedShards,random());
return state.nodes().get(startedShards.get(0).currentNodeId()).getName();
}
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.cluster.routing.allocation;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -388,7 +387,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
logger.info("Removing [{}] nodes", numNodes);
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder(clusterState.nodes());
ArrayList<DiscoveryNode> discoveryNodes = CollectionUtils.iterableAsArrayList(clusterState.nodes());
Collections.shuffle(discoveryNodes, getRandom());
Collections.shuffle(discoveryNodes, random());
for (DiscoveryNode node : discoveryNodes) {
nodes.remove(node.id());
numNodes--;

View File

@ -39,15 +39,10 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED;
import static org.elasticsearch.cluster.routing.ShardRoutingState.*;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.test.VersionUtils.randomVersion;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.*;
/**
*
@ -199,7 +194,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder();
int numNodes = between(1, 20);
if (nodes.size() > numNodes) {
Collections.shuffle(nodes, getRandom());
Collections.shuffle(nodes, random());
nodes = nodes.subList(0, numNodes);
} else {
for (int j = nodes.size(); j < numNodes; j++) {

View File

@ -17,20 +17,14 @@
* under the License.
*/
package org.elasticsearch.common.lucene;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.Field.Store;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.NoDeletionPolicy;
import org.apache.lucene.index.NoMergePolicy;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.*;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.TermQuery;
@ -41,11 +35,7 @@ import org.apache.lucene.util.Version;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.*;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;

View File

@ -24,13 +24,7 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.NoMergePolicy;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.*;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.Directory;
@ -42,14 +36,7 @@ import org.elasticsearch.test.ESTestCase;
import org.junit.After;
import org.junit.Before;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.*;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
@ -201,7 +188,7 @@ public class FreqTermsEnumTests extends ESTestCase {
for (int i = 0; i < cycles; i++) {
List<String> terms = new ArrayList<>(Arrays.asList(this.terms));
Collections.shuffle(terms, getRandom());
Collections.shuffle(terms, random());
for (String term : terms) {
if (!termsEnum.seekExact(new BytesRef(term))) {
assertThat("term : " + term, reference.get(term).docFreq, is(0));

View File

@ -25,14 +25,7 @@ import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.Counter;
import org.elasticsearch.test.ESTestCase;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.*;
import static org.elasticsearch.common.util.CollectionUtils.eagerPartition;
import static org.hamcrest.Matchers.equalTo;
@ -80,7 +73,7 @@ public class CollectionUtilsTests extends ESTestCase {
array.append(new BytesRef(s));
}
if (randomBoolean()) {
Collections.shuffle(tmpList, getRandom());
Collections.shuffle(tmpList, random());
for (BytesRef ref : tmpList) {
array.append(ref);
}
@ -111,7 +104,7 @@ public class CollectionUtilsTests extends ESTestCase {
array.append(new BytesRef(s));
}
if (randomBoolean()) {
Collections.shuffle(values, getRandom());
Collections.shuffle(values, random());
}
int[] indices = new int[array.size()];
for (int i = 0; i < indices.length; i++) {

View File

@ -27,13 +27,7 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.PriorityBlockingQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.hamcrest.Matchers.equalTo;
@ -46,7 +40,7 @@ public class PrioritizedExecutorsTests extends ESTestCase {
public void testPriorityQueue() throws Exception {
PriorityBlockingQueue<Priority> queue = new PriorityBlockingQueue<>();
List<Priority> priorities = Arrays.asList(Priority.values());
Collections.shuffle(priorities);
Collections.shuffle(priorities, random());
for (Priority priority : priorities) {
queue.add(priority);

View File

@ -473,7 +473,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
docsPerIndexer = 1 + randomInt(5);
logger.info("indexing " + docsPerIndexer + " docs per indexer during partition");
countDownLatchRef.set(new CountDownLatch(docsPerIndexer * indexers.size()));
Collections.shuffle(semaphores);
Collections.shuffle(semaphores, random());
for (Semaphore semaphore : semaphores) {
assertThat(semaphore.availablePermits(), equalTo(0));
semaphore.release(docsPerIndexer);
@ -683,7 +683,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
ensureGreen("test");
nodes = new ArrayList<>(nodes);
Collections.shuffle(nodes, getRandom());
Collections.shuffle(nodes, random());
String isolatedNode = nodes.get(0);
String notIsolatedNode = nodes.get(1);
@ -1038,7 +1038,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
new NetworkDisconnectPartition(getRandom()),
new SlowClusterStateProcessing(getRandom())
);
Collections.shuffle(list);
Collections.shuffle(list, random());
setDisruptionScheme(list.get(0));
return list.get(0);
}

View File

@ -26,11 +26,7 @@ import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
import org.elasticsearch.test.ESTestCase;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.*;
public class ElectMasterServiceTests extends ESTestCase {
@ -53,7 +49,7 @@ public class ElectMasterServiceTests extends ESTestCase {
nodes.add(node);
}
Collections.shuffle(nodes, getRandom());
Collections.shuffle(nodes, random());
return nodes;
}

View File

@ -244,7 +244,7 @@ public class NodeJoinControllerTests extends ESTestCase {
// add
Collections.shuffle(nodesToJoin);
Collections.shuffle(nodesToJoin, random());
logger.debug("--> joining [{}] unique master nodes. Total of [{}] join requests", initialJoins, nodesToJoin.size());
for (DiscoveryNode node : nodesToJoin) {
pendingJoins.add(joinNodeAsync(node));
@ -269,7 +269,7 @@ public class NodeJoinControllerTests extends ESTestCase {
}
}
Collections.shuffle(nodesToJoin);
Collections.shuffle(nodesToJoin, random());
logger.debug("--> joining [{}] nodes, with repetition a total of [{}]", finalJoins, nodesToJoin.size());
for (DiscoveryNode node : nodesToJoin) {
pendingJoins.add(joinNodeAsync(node));
@ -316,7 +316,7 @@ public class NodeJoinControllerTests extends ESTestCase {
nodesToJoin.add(node);
}
}
Collections.shuffle(nodesToJoin);
Collections.shuffle(nodesToJoin, random());
logger.debug("--> joining [{}] nodes, with repetition a total of [{}]", initialJoins, nodesToJoin.size());
for (DiscoveryNode node : nodesToJoin) {
pendingJoins.add(joinNodeAsync(node));

View File

@ -62,7 +62,7 @@ public class ZenPingTests extends ESTestCase {
}
// shuffle
Collections.shuffle(pings);
Collections.shuffle(pings, random());
ZenPing.PingCollection collection = new ZenPing.PingCollection();
collection.addPings(pings.toArray(new ZenPing.PingResponse[pings.size()]));

View File

@ -21,17 +21,14 @@ package org.elasticsearch.discovery.zen.publish;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.cluster.Diff;
import org.elasticsearch.cluster.*;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
@ -47,35 +44,20 @@ import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BytesTransportRequest;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportConnectionListener;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportResponseOptions;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.*;
import org.elasticsearch.transport.local.LocalTransport;
import org.junit.After;
import org.junit.Before;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.*;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.emptyIterable;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.*;
@TestLogging("discovery.zen.publish:TRACE")
public class PublishClusterStateActionTests extends ESTestCase {
@ -675,7 +657,7 @@ public class PublishClusterStateActionTests extends ESTestCase {
logger.info("--> committing states");
Collections.shuffle(states, random());
Randomness.shuffle(states);
for (ClusterState state : states) {
node.action.handleCommitRequest(new PublishClusterStateAction.CommitClusterStateRequest(state.stateUUID()), channel);
assertThat(channel.response.get(), equalTo((TransportResponse) TransportResponse.Empty.INSTANCE));

View File

@ -19,12 +19,7 @@
package org.elasticsearch.gateway;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.SimpleFSDirectory;
import org.apache.lucene.store.*;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
@ -33,11 +28,7 @@ import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.common.xcontent.*;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
@ -50,20 +41,10 @@ import java.nio.file.DirectoryStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.*;
import java.util.stream.StreamSupport;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.startsWith;
import static org.hamcrest.Matchers.*;
@LuceneTestCase.SuppressFileSystems("ExtrasFS") // TODO: fix test to work with ExtrasFS
public class MetaDataStateFormatTests extends ESTestCase {
@ -349,7 +330,7 @@ public class MetaDataStateFormatTests extends ESTestCase {
}
List<Path> dirList = Arrays.asList(dirs);
Collections.shuffle(dirList, getRandom());
Collections.shuffle(dirList, random());
MetaData loadedMetaData = format.loadLatestState(logger, dirList.toArray(new Path[0]));
MetaData latestMetaData = meta.get(numStates-1);
assertThat(loadedMetaData.clusterUUID(), not(equalTo("_na_")));

View File

@ -49,21 +49,12 @@ import org.elasticsearch.test.ESSingleNodeTestCase;
import org.hamcrest.Matchers;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.*;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.StreamsUtils.copyToBytesFromClasspath;
import static org.elasticsearch.test.StreamsUtils.copyToStringFromClasspath;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasItem;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.*;
public class SimpleAllMapperTests extends ESSingleNodeTestCase {
@ -251,7 +242,7 @@ public class SimpleAllMapperTests extends ESSingleNodeTestCase {
if (randomBoolean()) {
booleanOptionList.add(new Tuple<>("store_term_vector_payloads", tv_payloads = randomBoolean()));
}
Collections.shuffle(booleanOptionList, getRandom());
Collections.shuffle(booleanOptionList, random());
for (Tuple<String, Boolean> option : booleanOptionList) {
mappingBuilder.field(option.v1(), option.v2().booleanValue());
}

View File

@ -19,11 +19,7 @@
package org.elasticsearch.index.mapper.core;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.CannedTokenStream;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.*;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
@ -85,7 +81,7 @@ public class TokenCountFieldMapperTests extends ESSingleNodeTestCase {
t2.setPositionIncrement(2); // Count funny tokens with more than one increment
int finalTokenIncrement = 4; // Count the final token increment on the rare token streams that have them
Token[] tokens = new Token[] {t1, t2, t3};
Collections.shuffle(Arrays.asList(tokens), getRandom());
Collections.shuffle(Arrays.asList(tokens), random());
final TokenStream tokenStream = new CannedTokenStream(finalTokenIncrement, 0, tokens);
// TODO: we have no CannedAnalyzer?
Analyzer analyzer = new Analyzer() {

View File

@ -479,7 +479,7 @@ public class MultiFieldTests extends ESSingleNodeTestCase {
.startObject("my_field").field("type", "string").startObject("fields").startObject(MY_MULTI_FIELD)
.field("type", "string").startObject("fielddata");
String[] keys = possibleSettings.keySet().toArray(new String[]{});
Collections.shuffle(Arrays.asList(keys));
Collections.shuffle(Arrays.asList(keys), random());
for(int i = randomIntBetween(0, possibleSettings.size()-1); i >= 0; --i)
builder.field(keys[i], possibleSettings.get(keys[i]));
builder.endObject().endObject().endObject().endObject().endObject().endObject().endObject();

View File

@ -19,26 +19,21 @@
package org.elasticsearch.index.store;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.IndexFileNames;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.client.Requests;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.*;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
import org.elasticsearch.common.Nullable;
@ -48,7 +43,6 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.gateway.PrimaryShardAllocator;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest;
@ -75,14 +69,7 @@ import java.nio.charset.StandardCharsets;
import java.nio.file.DirectoryStream;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.*;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
@ -91,16 +78,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.util.CollectionUtils.iterableAsArrayList;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllSuccessful;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notNullValue;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.hamcrest.Matchers.*;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE)
public class CorruptedFileIT extends ESIntegTestCase {
@ -320,7 +299,7 @@ public class CorruptedFileIT extends ESIntegTestCase {
}
assertThat(dataNodeStats.size(), greaterThanOrEqualTo(2));
Collections.shuffle(dataNodeStats, getRandom());
Collections.shuffle(dataNodeStats, random());
NodeStats primariesNode = dataNodeStats.get(0);
NodeStats unluckyNode = dataNodeStats.get(1);
assertAcked(prepareCreate("test").setSettings(Settings.builder()
@ -380,7 +359,7 @@ public class CorruptedFileIT extends ESIntegTestCase {
}
assertThat(dataNodeStats.size(), greaterThanOrEqualTo(2));
Collections.shuffle(dataNodeStats, getRandom());
Collections.shuffle(dataNodeStats, random());
NodeStats primariesNode = dataNodeStats.get(0);
NodeStats unluckyNode = dataNodeStats.get(1);

View File

@ -26,13 +26,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.recovery.RecoveryState.File;
import org.elasticsearch.indices.recovery.RecoveryState.Index;
import org.elasticsearch.indices.recovery.RecoveryState.Stage;
import org.elasticsearch.indices.recovery.RecoveryState.Timer;
import org.elasticsearch.indices.recovery.RecoveryState.Translog;
import org.elasticsearch.indices.recovery.RecoveryState.Type;
import org.elasticsearch.indices.recovery.RecoveryState.VerifyIndex;
import org.elasticsearch.indices.recovery.RecoveryState.*;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
@ -43,14 +37,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.test.VersionUtils.randomVersion;
import static org.hamcrest.Matchers.arrayContainingInAnyOrder;
import static org.hamcrest.Matchers.closeTo;
import static org.hamcrest.Matchers.either;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThan;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.Matchers.*;
public class RecoveryStateTests extends ESTestCase {
abstract class Streamer<T extends Streamable> extends Thread {
@ -201,7 +188,7 @@ public class RecoveryStateTests extends ESTestCase {
}
}
Collections.shuffle(Arrays.asList(files));
Collections.shuffle(Arrays.asList(files), random());
final RecoveryState.Index index = new RecoveryState.Index();
if (randomBoolean()) {

View File

@ -29,7 +29,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.indices.recovery.IndexRecoveryIT;
import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest;
@ -84,7 +83,7 @@ public class TruncatedRecoveryIT extends ESIntegTestCase {
}
}
assertThat(dataNodeStats.size(), greaterThanOrEqualTo(2));
Collections.shuffle(dataNodeStats, getRandom());
Collections.shuffle(dataNodeStats, random());
// we use 2 nodes a lucky and unlucky one
// the lucky one holds the primary
// the unlucky one gets the replica and the truncated leftovers

View File

@ -942,13 +942,13 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase {
}
public void assertSuggestions(String suggestion, String... suggestions) {
String suggestionName = RandomStrings.randomAsciiOfLength(new Random(), 10);
String suggestionName = RandomStrings.randomAsciiOfLength(random(), 10);
CompletionSuggestionBuilder suggestionBuilder = SuggestBuilders.completionSuggestion(suggestionName).field(FIELD).text(suggestion).size(10);
assertSuggestions(suggestionName, suggestionBuilder, suggestions);
}
public void assertSuggestionsNotInOrder(String suggestString, String... suggestions) {
String suggestionName = RandomStrings.randomAsciiOfLength(new Random(), 10);
String suggestionName = RandomStrings.randomAsciiOfLength(random(), 10);
SuggestResponse suggestResponse = client().prepareSuggest(INDEX).addSuggestion(
SuggestBuilders.completionSuggestion(suggestionName).field(FIELD).text(suggestString).size(10)
).execute().actionGet();

View File

@ -48,58 +48,24 @@ import org.elasticsearch.script.groovy.GroovyPlugin;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHitField;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.sort.FieldSortBuilder;
import org.elasticsearch.search.sort.GeoDistanceSortBuilder;
import org.elasticsearch.search.sort.ScriptSortBuilder;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.search.sort.*;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.VersionUtils;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.hamcrest.Matchers;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.*;
import java.util.Map.Entry;
import java.util.Random;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ExecutionException;
import static org.apache.lucene.util.GeoUtils.TOLERANCE;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.fieldValueFactorFunction;
import static org.elasticsearch.search.sort.SortBuilders.fieldSort;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFirstHit;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertOrderedSearchHits;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSecondHit;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSortValues;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.hasId;
import static org.hamcrest.Matchers.closeTo;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.lessThan;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.apache.lucene.util.GeoUtils.TOLERANCE;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.hamcrest.Matchers.*;
/**
*
@ -530,7 +496,7 @@ public class SimpleSortTests extends ESIntegTestCase {
.endObject());
builders.add(builder);
}
Collections.shuffle(builders, random);
Randomness.shuffle(builders);
for (IndexRequestBuilder builder : builders) {
builder.execute().actionGet();
if (random.nextBoolean()) {

View File

@ -32,6 +32,7 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cloud.aws.network.Ec2NameResolver;
import org.elasticsearch.cloud.aws.node.Ec2CustomNodeAttributes;
import org.elasticsearch.cluster.node.DiscoveryNodeService;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.network.NetworkService;
@ -119,7 +120,7 @@ public class AwsEc2ServiceImpl extends AbstractLifecycleComponent<AwsEc2Service>
}
// Increase the number of retries in case of 5xx API responses
final Random rand = new Random();
final Random rand = Randomness.get();
RetryPolicy retryPolicy = new RetryPolicy(
RetryPolicy.RetryCondition.NO_RETRY_CONDITION,
new RetryPolicy.BackoffStrategy() {

View File

@ -24,6 +24,7 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.DiscoveryService;
import org.elasticsearch.node.Node;
import org.elasticsearch.node.internal.InternalSettingsPreparer;
import org.elasticsearch.test.ESIntegTestCase;
@ -57,8 +58,20 @@ public class TribeUnitTests extends ESTestCase {
.put("node.mode", NODE_MODE)
.put("path.home", createTempDir()).build();
tribe1 = new TribeClientNode(Settings.builder().put(baseSettings).put("cluster.name", "tribe1").put("name", "tribe1_node").build()).start();
tribe2 = new TribeClientNode(Settings.builder().put(baseSettings).put("cluster.name", "tribe2").put("name", "tribe2_node").build()).start();
tribe1 = new TribeClientNode(
Settings.builder()
.put(baseSettings)
.put("cluster.name", "tribe1")
.put("node.name", "tribe1_node")
.put(DiscoveryService.SETTING_DISCOVERY_SEED, random().nextLong())
.build()).start();
tribe2 = new TribeClientNode(
Settings.builder()
.put(baseSettings)
.put("cluster.name", "tribe2")
.put("node.name", "tribe2_node")
.put(DiscoveryService.SETTING_DISCOVERY_SEED, random().nextLong())
.build()).start();
}
@AfterClass
@ -73,6 +86,8 @@ public class TribeUnitTests extends ESTestCase {
System.setProperty("es.cluster.name", "tribe_node_cluster");
System.setProperty("es.tribe.t1.cluster.name", "tribe1");
System.setProperty("es.tribe.t2.cluster.name", "tribe2");
System.setProperty("es.tribe.t1.discovery.id.seed", Long.toString(random().nextLong()));
System.setProperty("es.tribe.t2.discovery.id.seed", Long.toString(random().nextLong()));
try {
assertTribeNodeSuccesfullyCreated(Settings.EMPTY);
@ -80,6 +95,8 @@ public class TribeUnitTests extends ESTestCase {
System.clearProperty("es.cluster.name");
System.clearProperty("es.tribe.t1.cluster.name");
System.clearProperty("es.tribe.t2.cluster.name");
System.clearProperty("es.tribe.t1.discovery.id.seed");
System.clearProperty("es.tribe.t2.discovery.id.seed");
}
}

View File

@ -1,3 +1,5 @@
cluster.name: tribe_node_cluster
tribe.t1.cluster.name: tribe1
tribe.t2.cluster.name: tribe2
tribe.t2.cluster.name: tribe2
tribe.t1.discovery.id.seed: 1
tribe.t2.discovery.id.seed: 2

View File

@ -36,6 +36,7 @@ import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.common.transport.TransportAddress;
@ -111,7 +112,7 @@ public abstract class ESAllocationTestCase extends ESTestCase {
for (AllocationDecider d : list) {
assertThat(defaultAllocationDeciders.contains(d.getClass()), is(true));
}
Collections.shuffle(list, random);
Randomness.shuffle(list);
return new AllocationDeciders(settings, list.toArray(new AllocationDecider[0]));
}

View File

@ -533,7 +533,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
}
private TestCluster buildWithPrivateContext(final Scope scope, final long seed) throws Exception {
return RandomizedContext.current().runWithPrivateRandomness(new Randomness(seed), new Callable<TestCluster>() {
return RandomizedContext.current().runWithPrivateRandomness(new com.carrotsearch.randomizedtesting.Randomness(seed), new Callable<TestCluster>() {
@Override
public TestCluster call() throws Exception {
return buildTestCluster(scope, seed);
@ -1388,7 +1388,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
}
}
final String[] indices = indicesSet.toArray(new String[indicesSet.size()]);
Collections.shuffle(builders, random);
Collections.shuffle(builders, random());
final CopyOnWriteArrayList<Tuple<IndexRequestBuilder, Throwable>> errors = new CopyOnWriteArrayList<>();
List<CountDownLatch> inFlightAsyncOperations = new ArrayList<>();
// If you are indexing just a few documents then frequently do it one at a time. If many then frequently in bulk.

View File

@ -566,7 +566,7 @@ public abstract class ESTestCase extends LuceneTestCase {
throw new IllegalArgumentException("Can\'t pick " + size + " random objects from a list of " + values.length + " objects");
}
List<T> list = arrayAsArrayList(values);
Collections.shuffle(list);
Collections.shuffle(list, random());
return list.subList(0, size);
}
@ -615,7 +615,7 @@ public abstract class ESTestCase extends LuceneTestCase {
sb.append("]");
assertThat(count + " files exist that should have been cleaned:\n" + sb.toString(), count, equalTo(0));
}
/** Returns the suite failure marker: internal use only! */
public static TestRuleMarkFailure getSuiteFailureMarker() {
return suiteFailureMarker;