Make TestCluster based integration tests more repoducible

While testing an async system providing reproducible tests that
use randomized components is a hard task we should at least try to
reestablish the enviroment of a failing test as much as possible.
This commit allows to re-establish the shared 'TestCluster' by
resetting the cluster to a predefined shared state before each test.

Before this commit a tests that is executed in isolation was likely
using a entirely different node enviroment as the failing test since
the 'TestCluster' kept intermediate nodes started by other tests around.
This commit is contained in:
Simon Willnauer 2013-09-13 19:33:18 +02:00
parent e110d53b0c
commit cabbf7805b
34 changed files with 254 additions and 365 deletions

View File

@ -388,6 +388,7 @@
<tests.timeoutSuite>${tests.timeoutSuite}</tests.timeoutSuite>
<tests.showSuccess>${tests.showSuccess}</tests.showSuccess>
<tests.integration>${tests.integration}</tests.integration>
<tests.cluster_seed>${tests.cluster_seed}</tests.cluster_seed>
<es.node.local>${env.ES_TEST_LOCAL}</es.node.local>
<es.node.mode>${es.node.mode}</es.node.mode>
<es.logger.level>${es.logger.level}</es.logger.level>

View File

@ -57,8 +57,10 @@ import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.indices.IndexMissingException;
import org.elasticsearch.indices.IndexTemplateMissingException;
import org.elasticsearch.rest.RestStatus;
import org.hamcrest.Matchers;
import org.junit.*;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.Ignore;
import java.io.IOException;
import java.util.*;
@ -68,6 +70,7 @@ import java.util.concurrent.ExecutionException;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.hamcrest.Matchers.emptyIterable;
import static org.hamcrest.Matchers.equalTo;
/**
@ -93,16 +96,11 @@ import static org.hamcrest.Matchers.equalTo;
@IntegrationTests
public abstract class AbstractSharedClusterTest extends ElasticsearchTestCase {
private static TestCluster cluster;
@BeforeClass
public static void beforeClass() throws Exception {
cluster();
}
private static final TestCluster cluster = new TestCluster(SHARED_CLUSTER_SEED, TestCluster.clusterName("shared", ElasticsearchTestCase.CHILD_VM_ID, SHARED_CLUSTER_SEED));
@Before
public final void before() {
cluster.ensureAtLeastNumNodes(numberOfNodes());
cluster.beforeTest(getRandom());
wipeIndices();
wipeTemplates();
}
@ -121,9 +119,6 @@ public abstract class AbstractSharedClusterTest extends ElasticsearchTestCase {
}
public static TestCluster cluster() {
if (cluster == null) {
cluster = ClusterManager.accquireCluster(getRandom());
}
return cluster;
}
@ -133,8 +128,6 @@ public abstract class AbstractSharedClusterTest extends ElasticsearchTestCase {
@AfterClass
public static void afterClass() {
cluster = null;
ClusterManager.releaseCluster();
}
public static Client client() {
@ -292,10 +285,6 @@ public abstract class AbstractSharedClusterTest extends ElasticsearchTestCase {
return Joiner.on(',').join(strings);
}
protected int numberOfNodes() {
return 2;
}
// utils
protected IndexResponse index(String index, String type, XContentBuilder source) {
return client().prepareIndex(index, type).setSource(source).execute().actionGet();
@ -446,7 +435,7 @@ public abstract class AbstractSharedClusterTest extends ElasticsearchTestCase {
for (CountDownLatch countDownLatch : latches) {
countDownLatch.await();
}
assertThat(errors, Matchers.emptyIterable());
assertThat(errors, emptyIterable());
if (forceRefresh) {
assertNoFailures(client().admin().indices().prepareRefresh(indices).setIgnoreIndices(IgnoreIndices.MISSING).execute().get());
}

View File

@ -1,40 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon 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;
import java.util.Random;
public class ClusterManager {
private static TestCluster cluster;
public synchronized static TestCluster accquireCluster(Random random) {
if (cluster == null) {
cluster = new TestCluster(random);
}
cluster.reset(random);
return cluster;
}
public static synchronized void releaseCluster() {
// doNothing
}
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch;
import com.carrotsearch.randomizedtesting.SeedUtils;
import com.carrotsearch.randomizedtesting.annotations.*;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
import com.google.common.base.Predicate;
@ -47,6 +48,17 @@ public abstract class ElasticsearchTestCase extends AbstractRandomizedTest {
public static final String CHILD_VM_ID = System.getProperty("junit4.childvm.id", "" + System.currentTimeMillis());
public static final long SHARED_CLUSTER_SEED = clusterSeed();
private static long clusterSeed() {
String property = System.getProperty("tests.cluster_seed");
if (property == null || property.isEmpty()) {
return System.nanoTime();
}
return SeedUtils.parseSeed(property);
}
public void awaitBusy(Predicate<?> breakPredicate) throws InterruptedException {
awaitBusy(breakPredicate, 10, TimeUnit.SECONDS);
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch;
import com.carrotsearch.randomizedtesting.SeedUtils;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableSet;
@ -40,30 +41,26 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.store.mock.MockFSIndexStoreModule;
import org.elasticsearch.index.store.mock.MockRamIndexStoreModule;
import org.elasticsearch.node.Node;
import org.elasticsearch.node.internal.InternalNode;
import org.elasticsearch.transport.TransportService;
import java.io.Closeable;
import java.util.*;
import java.util.Map.Entry;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import static com.google.common.collect.Maps.newHashMap;
import static com.google.common.collect.Maps.newTreeMap;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.elasticsearch.common.settings.ImmutableSettings.Builder.EMPTY_SETTINGS;
import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
public class TestCluster {
/* some random options to consider
* "action.auto_create_index"
* "node.local"
*/
protected final ESLogger logger = Loggers.getLogger(getClass());
private Map<String, NodeAndClient> nodes = newHashMap();
/* sorted map to make traverse order reproducible */
private final TreeMap<String, NodeAndClient> nodes = newTreeMap();
private final String clusterName;
@ -71,38 +68,65 @@ public class TestCluster {
private final Settings defaultSettings;
private NodeAndClient clientNode;
private NodeAndClient clientNode; // currently unused
private Random random;
private ClientFactory clientFactory;
private AtomicInteger nextNodeId = new AtomicInteger(0);
/* We have a fixed number of shared nodes that we keep around across tests */
private final int numSharedNodes;
public TestCluster(Random random) {
/* Each shared node has a node seed that is used to start up the node and get default settings
* this is important if a node is randomly shut down in a test since the next test relies on a
* fully shared cluster to be more reproducible */
private final long[] sharedNodesSeeds;
this(random, "shared-test-cluster-" + NetworkUtils.getLocalAddress().getHostName() + "CHILD_VM=[" + ElasticsearchTestCase.CHILD_VM_ID + "]" + "_" + System.currentTimeMillis(), ImmutableSettings.settingsBuilder().build());
public TestCluster(long clusterSeed, String clusterName) {
this(clusterSeed, clusterName, ImmutableSettings.EMPTY);
}
private TestCluster(Random random, String clusterName, Settings defaultSettings) {
this.random = new Random(random.nextLong());
clientFactory = new RandomClientFactory(random);
private TestCluster(long clusterSeed, String clusterName, Settings defaultSettings) {
this.clusterName = clusterName;
Random random = new Random(clusterSeed);
numSharedNodes = 2 + random.nextInt(4); // at least 2 nodes
/*
* TODO
* - we might want start some master only nodes?
* - we could add a flag that returns a client to the master all the time?
* - we could add a flag that never returns a client to the master
* - along those lines use a dedicated node that is master eligible and let all other nodes be only data nodes
*/
sharedNodesSeeds = new long[numSharedNodes];
for (int i = 0; i < sharedNodesSeeds.length; i++) {
sharedNodesSeeds[i] = random.nextLong();
}
logger.info("Started TestCluster with seed [{}] using [{}] nodes" , SeedUtils.formatSeed(clusterSeed), numSharedNodes);
if (defaultSettings.get("gateway.type") == null) {
// default to non gateway
defaultSettings = settingsBuilder().put(defaultSettings).put("gateway.type", "none").build();
}
if (defaultSettings.get("cluster.routing.schedule") != null) {
// decrease the routing schedule so new nodes will be added quickly
defaultSettings = settingsBuilder().put(defaultSettings).put("cluster.routing.schedule", "50ms").build();
// decrease the routing schedule so new nodes will be added quickly - some random value between 30 and 80 ms
defaultSettings = settingsBuilder().put(defaultSettings).put("cluster.routing.schedule", (30 + random.nextInt(50)) + "ms").build();
}
// TODO once we are reproducible here use MockRamIndexStoreModule
this.defaultSettings = ImmutableSettings.settingsBuilder()
.put("index.store.type", MockFSIndexStoreModule.class.getName())
.put(defaultSettings).put("cluster.name", clusterName).build();
/* use RAM directories in 10% of the runs */
.put("index.store.type", random.nextInt(10) == 0 ? MockRamIndexStoreModule.class.getName() : MockFSIndexStoreModule.class.getName())
.put(defaultSettings)
.put("cluster.name", clusterName).build();
}
public static String clusterName(String prefix, String childVMId, long clusterSeed) {
StringBuilder builder = new StringBuilder(prefix);
builder.append('-').append(NetworkUtils.getLocalAddress().getHostName());
builder.append("-CHILD_VM=[").append(childVMId).append(']');
builder.append("-CLUSTER_SEED=[").append(clusterSeed).append(']');
// if multiple maven task run on a single host we better have an identifier that doesn't rely on input params
builder.append("-HASH=[").append(SeedUtils.formatSeed(System.nanoTime())).append(']');
return builder.toString();
}
private void ensureOpen() {
if (!open.get()) {
@ -110,31 +134,40 @@ public class TestCluster {
}
}
public Node getOneNode() {
private synchronized Node getOrBuildRandomNode() {
ensureOpen();
Collection<NodeAndClient> values = nodes.values();
for (NodeAndClient nodeAndClient : values) {
return nodeAndClient.node();
NodeAndClient randomNodeAndClient = getRandomNodeAndClient();
if (randomNodeAndClient != null) {
return randomNodeAndClient.node();
}
return buildNode().start();
NodeAndClient buildNode = buildNode();
nodes.put(buildNode.name, buildNode);
return buildNode.node().start();
}
public void ensureAtLeastNumNodes(int num) {
private synchronized NodeAndClient getRandomNodeAndClient() {
ensureOpen();
Collection<NodeAndClient> values = nodes.values();
int whichOne = random.nextInt(values.size());
for (NodeAndClient nodeAndClient : values) {
if (whichOne-- == 0) {
return nodeAndClient;
}
}
return null;
}
public synchronized void ensureAtLeastNumNodes(int num) {
int size = nodes.size();
for (int i = size; i < num; i++) {
logger.info("increasing cluster size from {} to {}", size, num);
buildNode().start();
NodeAndClient buildNode = buildNode();
buildNode.node().start();
nodes.put(buildNode.name, buildNode);
}
}
public void ensureAtLeastNumNodes(Settings settings, int num) {
int size = nodes.size();
for (int i = size; i < num; i++) {
buildNode(settings).start();
}
}
public void ensureAtMostNumNodes(int num) {
public synchronized void ensureAtMostNumNodes(int num) {
if (nodes.size() <= num) {
return;
}
@ -148,58 +181,31 @@ public class TestCluster {
}
}
public Node startNode(Settings.Builder settings) {
ensureOpen();
return startNode(settings.build());
private NodeAndClient buildNode() {
return buildNode(nextNodeId.getAndIncrement(), random.nextLong());
}
public Node startNode(Settings settings) {
private NodeAndClient buildNode(int nodeId, long seed) {
ensureOpen();
return buildNode(settings).start();
}
public Node buildNode() {
ensureOpen();
return buildNode(EMPTY_SETTINGS);
}
public Node buildNode(Settings.Builder settings) {
ensureOpen();
return buildNode(settings.build());
}
public Node buildNode(Settings settings) {
ensureOpen();
String name = buildNodeName();
String settingsSource = getClass().getName().replace('.', '/') + ".yml";
String name = buildNodeName(nodeId);
assert !nodes.containsKey(name);
Settings finalSettings = settingsBuilder()
.loadFromClasspath(settingsSource)
.put(defaultSettings).put(settings)
.put(defaultSettings)
.put("name", name)
.put("discovery.id.seed", random.nextLong())
.put("discovery.id.seed", seed)
.build();
Node node = nodeBuilder().settings(finalSettings).build();
nodes.put(name, new NodeAndClient(name, node, clientFactory));
return node;
return new NodeAndClient(name, node, new RandomClientFactory());
}
private String buildNodeName() {
return "node_" + nextNodeId.getAndIncrement();
private String buildNodeName(int id) {
return "node_" + id;
}
public void setClientFactory(ClientFactory factory) {
this.clientFactory = factory;
}
public void closeNode(Node node) {
public synchronized Client client() {
ensureOpen();
NodeAndClient remove = nodes.remove(node.settings().get("name"));
IOUtils.closeWhileHandlingException(remove); // quiet
}
public Client client() {
ensureOpen();
return getOneNode().client();
/* Randomly return a client to one of the nodes in the cluster */
return getOrBuildRandomNode().client();
}
public void close() {
@ -213,7 +219,7 @@ public class TestCluster {
}
}
public ImmutableSet<ClusterBlock> waitForNoBlocks(TimeValue timeout, Node node) throws InterruptedException {
public synchronized ImmutableSet<ClusterBlock> waitForNoBlocks(TimeValue timeout, Node node) throws InterruptedException {
ensureOpen();
long start = System.currentTimeMillis();
ImmutableSet<ClusterBlock> blocks;
@ -224,34 +230,44 @@ public class TestCluster {
return blocks;
}
public class NodeAndClient implements Closeable {
final Node node;
Client client;
final AtomicBoolean closed = new AtomicBoolean(false);
final ClientFactory clientFactory;
final String name;
private final class NodeAndClient implements Closeable {
private final Node node;
private Client client;
private final AtomicBoolean closed = new AtomicBoolean(false);
private final ClientFactory clientFactory;
private final String name;
public NodeAndClient(String name, Node node, ClientFactory factory) {
NodeAndClient(String name, Node node, ClientFactory factory) {
this.node = node;
this.name = name;
this.clientFactory = factory;
}
public Node node() {
Node node() {
if (closed.get()) {
throw new RuntimeException("already closed");
}
return node;
}
public Client client() {
Client client(Random random) {
if (closed.get()) {
throw new RuntimeException("already closed");
}
if (client != null) {
return client;
}
return client = clientFactory.client(node, clusterName);
return client = clientFactory.client(node, clusterName, random);
}
void resetClient() {
if (closed.get()) {
throw new RuntimeException("already closed");
}
if (client != null) {
client.close();
client = null;
}
}
@Override
@ -259,6 +275,7 @@ public class TestCluster {
closed.set(true);
if (client != null) {
client.close();
client = null;
}
node.close();
@ -267,7 +284,7 @@ public class TestCluster {
public static class ClientFactory {
public Client client(Node node, String clusterName) {
public Client client(Node node, String clusterName, Random random) {
return node.client();
}
}
@ -283,7 +300,7 @@ public class TestCluster {
}
@Override
public Client client(Node node, String clusterName) {
public Client client(Node node, String clusterName, Random random) {
TransportAddress addr = ((InternalNode) node).injector().getInstance(TransportService.class).boundAddress().publishAddress();
TransportClient client = new TransportClient(settingsBuilder().put("client.transport.nodes_sampler_interval", "30s")
.put("cluster.name", clusterName).put("client.transport.sniff", sniff).build());
@ -293,52 +310,86 @@ public class TestCluster {
}
public static class RandomClientFactory extends ClientFactory {
private final Random random;
public RandomClientFactory(Random random) {
this.random = random;
}
@Override
public Client client(Node node, String clusterName) {
public Client client(Node node, String clusterName, Random random) {
switch (random.nextInt(10)) {
case 5:
return TransportClientFactory.NO_SNIFF_CLIENT_FACTORY.client(node, clusterName);
return TransportClientFactory.NO_SNIFF_CLIENT_FACTORY.client(node, clusterName, random);
case 3:
return TransportClientFactory.SNIFF_CLIENT_FACTORY.client(node, clusterName);
return TransportClientFactory.SNIFF_CLIENT_FACTORY.client(node, clusterName, random);
default:
return node.client();
}
}
}
void reset(Random random) {
public synchronized void beforeTest(Random random) {
this.random = new Random(random.nextLong());
this.clientFactory = new RandomClientFactory(this.random);
resetClients(); /* reset all clients - each test gets it's own client based on the Random instance created above. */
if (nextNodeId.get() == sharedNodesSeeds.length) {
return;
}
if (nodes.size() > 0) {
client().admin().cluster().prepareHealth().setWaitForNodes(""+nodes.size()).get();
}
Set<NodeAndClient> sharedNodes = new HashSet<NodeAndClient>();
boolean changed = false;
for (int i = 0; i < sharedNodesSeeds.length; i++) {
String buildNodeName = buildNodeName(i);
NodeAndClient nodeAndClient = nodes.get(buildNodeName);
if (nodeAndClient == null) {
changed = true;
nodeAndClient = buildNode(i, sharedNodesSeeds[i]);
nodeAndClient.node.start();
}
sharedNodes.add(nodeAndClient);
}
if (!changed && sharedNodes.size() == nodes.size()) {
return; // we are consistent - return
}
for (NodeAndClient nodeAndClient : sharedNodes) {
nodes.remove(nodeAndClient.name);
}
// trash the remaining nodes
final Collection<NodeAndClient> toShutDown = nodes.values();
for (NodeAndClient nodeAndClient : toShutDown) {
nodeAndClient.close();
}
nodes.clear();
for (NodeAndClient nodeAndClient : sharedNodes) {
nodes.put(nodeAndClient.name, nodeAndClient);
}
nextNodeId.set(sharedNodesSeeds.length);
assert numNodes() == sharedNodesSeeds.length;
}
public ClusterService clusterService() {
return ((InternalNode) getOneNode()).injector().getInstance(ClusterService.class);
private void resetClients() {
final Collection<NodeAndClient> nodesAndClients = nodes.values();
for (NodeAndClient nodeAndClient : nodesAndClients) {
nodeAndClient.resetClient();
}
}
public int numNodes() {
public synchronized ClusterService clusterService() {
return ((InternalNode) getOrBuildRandomNode()).injector().getInstance(ClusterService.class);
}
public synchronized int numNodes() {
return this.nodes.size();
}
public void stopRandomNode() {
public synchronized void stopRandomNode() {
ensureOpen();
// TODO randomize
Set<Entry<String, NodeAndClient>> entrySet = nodes.entrySet();
if (entrySet.isEmpty()) {
return;
NodeAndClient nodeAndClient = getRandomNodeAndClient();
if (nodeAndClient != null) {
nodes.remove(nodeAndClient.name);
nodeAndClient.close();
}
Entry<String, NodeAndClient> next = entrySet.iterator().next();
nodes.remove(next.getKey());
next.getValue().close();
}
public Iterable<Client> clients() {
public synchronized Iterable<Client> clients() {
final Map<String, NodeAndClient> nodes = this.nodes;
return new Iterable<Client>() {
@ -355,7 +406,7 @@ public class TestCluster {
@Override
public Client next() {
return iterator.next().client();
return iterator.next().client(random);
}
@Override
@ -369,31 +420,31 @@ public class TestCluster {
}
public Set<String> allButN(int numNodes) {
public synchronized Set<String> allButN(int numNodes) {
return nRandomNodes(numNodes() - numNodes);
}
public Set<String> nRandomNodes(int numNodes) {
public synchronized Set<String> nRandomNodes(int numNodes) {
assert numNodes() >= numNodes;
return Sets.newHashSet(Iterators.limit(this.nodes.keySet().iterator(), numNodes));
}
public Client nodeClient() {
ensureOpen();
private synchronized Client nodeClient() {
ensureOpen(); // currently unused
if (clientNode == null) {
String name = "client_" + buildNodeName();
String settingsSource = getClass().getName().replace('.', '/') + ".yml";
Settings finalSettings = settingsBuilder().loadFromClasspath(settingsSource).put(defaultSettings).put("node.client", true).put("name", name)
String name = "client_node";
Settings finalSettings = settingsBuilder().put(defaultSettings).put("name", name)
.build();
Node node = nodeBuilder().settings(finalSettings).build();
Node node = nodeBuilder().settings(finalSettings).client(true).build();
node.start();
this.clientNode = new NodeAndClient(name, node, clientFactory);
this.clientNode = new NodeAndClient(name, node, new ClientFactory());
}
return clientNode.client();
return clientNode.client(random);
}
public Set<String> nodesInclude(String index) {
public synchronized Set<String> nodesInclude(String index) {
if (clusterService().state().routingTable().hasIndex(index)) {
List<ShardRouting> allShards = clusterService().state().routingTable().allShards(index);
DiscoveryNodes discoveryNodes = clusterService().state().getNodes();
@ -410,7 +461,7 @@ public class TestCluster {
}
public Set<String> nodeExclude(String index) {
public synchronized Set<String> nodeExclude(String index) {
final Set<String> nodesInclude = nodesInclude(index);
return Sets.newHashSet(Iterators.transform(Iterators.filter(nodes.values().iterator(), new Predicate<NodeAndClient>() {
@Override

View File

@ -38,7 +38,8 @@ public class DiskUsageTests extends ElasticsearchTestCase {
@Test
public void randomDiskUsageTest() {
for (int i=1;i<1000000;i++) {
int iters = atLeast(1000);
for (int i = 1; i < iters; i++) {
long total = between(Integer.MIN_VALUE, Integer.MAX_VALUE);
long free = between(Integer.MIN_VALUE, Integer.MAX_VALUE);
if (free > total || total <= 0) {

View File

@ -19,6 +19,7 @@
package org.elasticsearch.deleteByQuery;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.deletebyquery.DeleteByQueryRequestBuilder;
import org.elasticsearch.action.deletebyquery.DeleteByQueryResponse;
import org.elasticsearch.action.search.SearchResponse;
@ -26,23 +27,14 @@ import org.elasticsearch.action.support.IgnoreIndices;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.indices.IndexMissingException;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.notNullValue;
public class DeleteByQueryTests extends AbstractSharedClusterTest {
@Before
public void createNodes() throws Exception {
cluster().ensureAtLeastNumNodes(2);
}
@Test
public void testDeleteAllNoIndices() {
client().admin().indices().prepareDelete().execute().actionGet();

View File

@ -20,6 +20,7 @@
package org.elasticsearch.document;
import com.google.common.base.Charsets;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse;
@ -36,7 +37,6 @@ import org.elasticsearch.action.support.broadcast.BroadcastOperationThreading;
import org.elasticsearch.action.support.replication.ReplicationType;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import java.io.IOException;
@ -53,11 +53,6 @@ import static org.hamcrest.Matchers.nullValue;
*/
public class DocumentActionsTests extends AbstractSharedClusterTest {
@Override
protected int numberOfNodes() {
return 2;
}
protected void createIndex() {
wipeIndex(getConcreteIndexName());
createIndex(getConcreteIndexName());

View File

@ -19,18 +19,16 @@
package org.elasticsearch.explain;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.explain.ExplainResponse;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.index.query.FilterBuilders;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.indices.IndexMissingException;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import java.util.Map;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
/**
@ -40,11 +38,6 @@ public class ExplainActionTests extends AbstractSharedClusterTest {
@Test
public void testSimple() throws Exception {
cluster().ensureAtLeastNumNodes(2);
try {
client().admin().indices().prepareDelete("test").execute().actionGet();
} catch (IndexMissingException e) {
}
client().admin().indices().prepareCreate("test").setSettings(
ImmutableSettings.settingsBuilder().put("index.refresh_interval", -1)
).execute().actionGet();
@ -106,11 +99,6 @@ public class ExplainActionTests extends AbstractSharedClusterTest {
@SuppressWarnings("unchecked")
@Test
public void testExplainWithFields() throws Exception {
cluster().ensureAtLeastNumNodes(2);
try {
client().admin().indices().prepareDelete("test").execute().actionGet();
} catch (IndexMissingException e) {
}
client().admin().indices().prepareCreate("test").execute().actionGet();
client().admin().cluster().prepareHealth("test").setWaitForGreenStatus().execute().actionGet();
@ -173,11 +161,6 @@ public class ExplainActionTests extends AbstractSharedClusterTest {
@SuppressWarnings("unchecked")
@Test
public void testExplainWitSource() throws Exception {
cluster().ensureAtLeastNumNodes(2);
try {
client().admin().indices().prepareDelete("test").execute().actionGet();
} catch (IndexMissingException e) {
}
client().admin().indices().prepareCreate("test").execute().actionGet();
client().admin().cluster().prepareHealth("test").setWaitForGreenStatus().execute().actionGet();
@ -218,11 +201,6 @@ public class ExplainActionTests extends AbstractSharedClusterTest {
@Test
public void testExplainWithAlias() throws Exception {
cluster().ensureAtLeastNumNodes(2);
try {
client().admin().indices().prepareDelete("test").execute().actionGet();
} catch (IndexMissingException e) {
}
client().admin().indices().prepareCreate("test")
.execute().actionGet();
client().admin().cluster().prepareHealth("test").setWaitForGreenStatus().execute().actionGet();

View File

@ -21,6 +21,7 @@ package org.elasticsearch.gateway.fs;
import com.carrotsearch.randomizedtesting.annotations.Nightly;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.elasticsearch.AbstractNodesTests;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
@ -39,7 +40,6 @@ import org.elasticsearch.env.Environment;
import org.elasticsearch.gateway.Gateway;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.node.internal.InternalNode;
import org.elasticsearch.AbstractNodesTests;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -364,7 +364,7 @@ public class IndexGatewayTests extends AbstractNodesTests {
@Test
@Slow
public void testRandom() {
testLoad(getRandom().nextBoolean());
testLoad(randomBoolean());
}
@Test

View File

@ -19,12 +19,12 @@
package org.elasticsearch.indices;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.admin.indices.close.CloseIndexResponse;
import org.elasticsearch.action.search.MultiSearchResponse;
import org.elasticsearch.action.support.IgnoreIndices;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import static org.hamcrest.Matchers.*;
@ -172,7 +172,6 @@ public class IgnoreIndicesTests extends AbstractSharedClusterTest {
@Test
// For now don't handle closed indices
public void testClosed() throws Exception {
cluster().ensureAtLeastNumNodes(2);
createIndex("test1", "test2");
ensureGreen();
client().prepareSearch("test1", "test2").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet();

View File

@ -1,5 +1,6 @@
package org.elasticsearch.indices.mapping;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
@ -13,7 +14,6 @@ import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MergeMappingException;
import org.elasticsearch.AbstractSharedClusterTest;
import org.hamcrest.Matchers;
import org.junit.Test;
@ -285,7 +285,7 @@ public class UpdateMappingTests extends AbstractSharedClusterTest {
@Test
public void updateMappingConcurrently() throws Throwable {
// Test that we can concurrently update different indexes and types.
int shardNo = Math.max(5, numberOfNodes());
int shardNo = Math.max(5, cluster().numNodes());
prepareCreate("test1").setSettings("index.number_of_shards", shardNo).execute().actionGet();
prepareCreate("test2").setSettings("index.number_of_shards", shardNo).execute().actionGet();

View File

@ -2,7 +2,9 @@ package org.elasticsearch.junit.listeners;
import com.carrotsearch.randomizedtesting.RandomizedContext;
import com.carrotsearch.randomizedtesting.ReproduceErrorMessageBuilder;
import com.carrotsearch.randomizedtesting.SeedUtils;
import com.carrotsearch.randomizedtesting.TraceFormatting;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.ElasticsearchTestCase;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
@ -42,7 +44,10 @@ public class ReproduceInfoPrinter extends RunListener {
final StringBuilder b = new StringBuilder();
b.append("FAILURE : ").append(d.getDisplayName()).append("\n");
b.append("REPRODUCE WITH : mvn test");
new MavenMessageBuilder(b).appendAllOpts(failure.getDescription());
ReproduceErrorMessageBuilder builder = new MavenMessageBuilder(b).appendAllOpts(failure.getDescription());
if (AbstractSharedClusterTest.class.isAssignableFrom(failure.getDescription().getTestClass())) {
builder.appendOpt("tests.cluster_seed", SeedUtils.formatSeed(ElasticsearchTestCase.SHARED_CLUSTER_SEED));
}
b.append("\n");
b.append("Throwable:\n");
@ -77,7 +82,10 @@ public class ReproduceInfoPrinter extends RunListener {
if (sysPropName.equals("tests.iters")) { // we don't want the iters to be in there!
return this;
}
return super.appendOpt(sysPropName, value);
if (value != null && !value.isEmpty()) {
return super.appendOpt(sysPropName, value);
}
return this;
}
public ReproduceErrorMessageBuilder appendESProperties() {

View File

@ -19,13 +19,13 @@
package org.elasticsearch.mlt;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import static org.elasticsearch.client.Requests.*;
@ -123,7 +123,7 @@ public class MoreLikeThisActionTests extends AbstractSharedClusterTest {
@Test
public void testMoreLikeThisIssue2197() throws Exception {
Client client = cluster().nodeClient();
Client client = client();
String mapping = XContentFactory.jsonBuilder().startObject().startObject("bar")
.startObject("properties")
.endObject()
@ -167,9 +167,13 @@ public class MoreLikeThisActionTests extends AbstractSharedClusterTest {
@Test
// See issue: https://github.com/elasticsearch/elasticsearch/issues/3039
public void testMoreLikeThisIssueRoutingNotSerialized() throws Exception {
cluster().ensureAtLeastNumNodes(3);
String mapping = XContentFactory.jsonBuilder().startObject().startObject("bar")
.startObject("properties")
.endObject()
.endObject().endObject().string();
prepareCreate("foo", 2, ImmutableSettings.builder().put("index.number_of_replicas", 0)
.put("index.number_of_shards", 2))
.addMapping("bar", mapping)
.execute().actionGet();
ensureGreen();

View File

@ -19,6 +19,7 @@
package org.elasticsearch.percolator;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesResponse;
@ -42,7 +43,6 @@ import org.elasticsearch.index.query.FilterBuilders;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.query.functionscore.factor.FactorBuilder;
import org.elasticsearch.search.highlight.HighlightBuilder;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import java.util.*;
@ -61,11 +61,6 @@ import static org.hamcrest.Matchers.*;
*/
public class PercolatorTests extends AbstractSharedClusterTest {
@Override
protected int numberOfNodes() {
return 2;
}
@Test
public void testSimple1() throws Exception {
client().admin().indices().prepareCreate("test").execute().actionGet();
@ -1300,7 +1295,7 @@ public class PercolatorTests extends AbstractSharedClusterTest {
@Test
public void testPercolatorWithHighlighting() throws Exception {
Client client = cluster().nodeClient();
Client client = client();
client.admin().indices().prepareCreate("test")
.setSettings(ImmutableSettings.settingsBuilder().put("index.number_of_shards", 2))
.execute().actionGet();

View File

@ -19,6 +19,7 @@
package org.elasticsearch.recovery;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
@ -26,11 +27,9 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import static org.elasticsearch.client.Requests.*;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
/**
@ -38,10 +37,6 @@ import static org.hamcrest.Matchers.equalTo;
*/
public class SimpleRecoveryTests extends AbstractSharedClusterTest {
@Override
protected int numberOfNodes() {
return 3;
}
@Override
public Settings getSettings() {
return recoverySettings();

View File

@ -19,9 +19,9 @@
package org.elasticsearch.routing;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.common.Priority;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import java.util.Map;
@ -30,7 +30,6 @@ import java.util.Set;
import static com.google.common.collect.Maps.newHashMap;
import static com.google.common.collect.Sets.newHashSet;
import static org.elasticsearch.cluster.metadata.AliasAction.newAddAliasAction;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.nullValue;
@ -39,11 +38,6 @@ import static org.hamcrest.Matchers.nullValue;
*/
public class AliasResolveRoutingTests extends AbstractSharedClusterTest {
@Override
protected int numberOfNodes() {
return 2;
}
@Test
public void testResolveIndexRouting() throws Exception {
createIndex("test1");

View File

@ -19,6 +19,7 @@
package org.elasticsearch.routing;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.action.RoutingMissingException;
import org.elasticsearch.client.Requests;
@ -26,11 +27,9 @@ import org.elasticsearch.common.Priority;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
@ -40,11 +39,6 @@ import static org.hamcrest.Matchers.instanceOf;
public class SimpleRoutingTests extends AbstractSharedClusterTest {
@Override
protected int numberOfNodes() {
return 2;
}
@Test
public void testSimpleCrudRouting() throws Exception {
createIndex("test");
@ -105,11 +99,6 @@ public class SimpleRoutingTests extends AbstractSharedClusterTest {
@Test
public void testSimpleSearchRouting() {
try {
client().admin().indices().prepareDelete("test").execute().actionGet();
} catch (Exception e) {
// ignore
}
client().admin().indices().prepareCreate("test").execute().actionGet();
client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet();
@ -177,11 +166,6 @@ public class SimpleRoutingTests extends AbstractSharedClusterTest {
@Test
public void testRequiredRoutingMapping() throws Exception {
try {
client().admin().indices().prepareDelete("test").execute().actionGet();
} catch (Exception e) {
// ignore
}
client().admin().indices().prepareCreate("test")
.addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("_routing").field("required", true).endObject().endObject().endObject())
.execute().actionGet();
@ -226,11 +210,6 @@ public class SimpleRoutingTests extends AbstractSharedClusterTest {
@Test
public void testRequiredRoutingWithPathMapping() throws Exception {
try {
client().admin().indices().prepareDelete("test").execute().actionGet();
} catch (Exception e) {
// ignore
}
client().admin().indices().prepareCreate("test")
.addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1")
.startObject("_routing").field("required", true).field("path", "routing_field").endObject()
@ -262,11 +241,6 @@ public class SimpleRoutingTests extends AbstractSharedClusterTest {
@Test
public void testRequiredRoutingWithPathMappingBulk() throws Exception {
try {
client().admin().indices().prepareDelete("test").execute().actionGet();
} catch (Exception e) {
// ignore
}
client().admin().indices().prepareCreate("test")
.addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1")
.startObject("_routing").field("required", true).field("path", "routing_field").endObject()

View File

@ -20,12 +20,12 @@
package org.elasticsearch.search.basic;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
@ -40,10 +40,6 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo;
*/
public class SearchWhileCreatingIndexTests extends AbstractSharedClusterTest {
protected int numberOfNodes() {
return 1;
}
@Test
@Slow
public void testIndexCausesIndexCreation() throws Exception {

View File

@ -19,6 +19,7 @@
package org.elasticsearch.search.basic;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
@ -26,7 +27,6 @@ import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.common.lucene.search.function.CombineFunction;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.AbstractSharedClusterTest;
import org.hamcrest.Matchers;
import org.junit.Test;
@ -47,11 +47,6 @@ import static org.hamcrest.Matchers.equalTo;
public class SearchWhileRelocatingTests extends AbstractSharedClusterTest {
@Override
protected int numberOfNodes() {
return 3;
}
@Test
public void testSearchAndRelocateConcurrently() throws Exception {
final int numShards = between(10, 20);

View File

@ -23,6 +23,7 @@ package org.elasticsearch.search.basic;
import com.google.common.base.Charsets;
import com.google.common.collect.Sets;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.action.search.MultiSearchResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
@ -38,7 +39,6 @@ import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.facet.FacetBuilders;
import org.elasticsearch.search.facet.query.QueryFacet;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import java.io.IOException;
@ -62,7 +62,6 @@ public class TransportTwoNodesSearchTests extends AbstractSharedClusterTest {
private Set<String> prepareData() throws Exception {
Set<String> fullExpectedIds = Sets.newHashSet();
cluster().ensureAtLeastNumNodes(2);
client().admin().indices().create(createIndexRequest("test")
.settings(settingsBuilder().put("index.number_of_shards", 3)
.put("index.number_of_replicas", 0)

View File

@ -1,5 +1,7 @@
package org.elasticsearch.search.facet;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.regex.Regex;
@ -9,7 +11,6 @@ import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.search.facet.terms.TermsFacet;
import org.elasticsearch.search.facet.terms.TermsFacetBuilder;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import java.util.*;
@ -34,17 +35,13 @@ public class ExtendedFacetsTests extends AbstractSharedClusterTest {
return 1;
}
@Override
protected int numberOfNodes() {
return 1;
}
protected int numDocs() {
return 2500;
}
@Test
@Slow
public void testTermFacet_stringFields() throws Throwable {
prepareCreate("test")
.addMapping("type1", jsonBuilder().startObject()

View File

@ -9,11 +9,6 @@ public class ExtendedFacetsTestsMultiShardMultiNodeTests extends ExtendedFacetsT
return 8;
}
@Override
protected int numberOfNodes() {
return 4;
}
@Override
protected int numDocs() {
return 10000;

View File

@ -31,8 +31,4 @@ public class SimpleFacetsMultiShardMultiNodeTests extends SimpleFacetsTests {
return 3;
}
@Override
protected int numberOfNodes() {
return 2;
}
}

View File

@ -76,11 +76,6 @@ public class SimpleFacetsTests extends AbstractSharedClusterTest {
return 1;
}
@Override
protected int numberOfNodes() {
return 1;
}
protected int numberOfRuns() {
return 5;
}

View File

@ -52,11 +52,6 @@ public class UnmappedFieldsTermsFacetsTests extends AbstractSharedClusterTest {
return 5;
}
@Override
protected int numberOfNodes() {
return 1;
}
/**
* Tests the terms facet when faceting on unmapped field
*/

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.highlight;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse;
@ -36,7 +37,6 @@ import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.test.hamcrest.ElasticsearchAssertions;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import java.io.IOException;
@ -58,11 +58,6 @@ import static org.hamcrest.Matchers.startsWith;
*/
public class HighlighterSearchTests extends AbstractSharedClusterTest {
@Override
protected int numberOfNodes() {
return 4; // why 4?
}
@Test
// see #3486
public void testHighTermFrequencyDoc() throws ElasticSearchException, IOException {

View File

@ -19,25 +19,23 @@
package org.elasticsearch.search.preference;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.Priority;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.*;
public class SearchPreferenceTests extends AbstractSharedClusterTest {
@Test // see #2896
public void testStopOneNodePreferenceWithRedState() throws InterruptedException {
cluster().ensureAtLeastNumNodes(3);
client().admin().indices().prepareDelete().execute().actionGet();
client().admin().indices().prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", cluster().numNodes()+2).put("index.number_of_replicas", 0)).execute().actionGet();
client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet();
for (int i = 0; i < 10; i++) {
@ -60,18 +58,16 @@ public class SearchPreferenceTests extends AbstractSharedClusterTest {
@Test
public void noPreferenceRandom() throws Exception {
client().admin().indices().prepareDelete().execute().actionGet();
client().admin().indices().prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 1).put("index.number_of_replicas", 1)).execute().actionGet();
client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet();
client().prepareIndex("test", "type1").setSource("field1", "value1").execute().actionGet();
client().admin().indices().prepareRefresh().execute().actionGet();
SearchResponse searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).execute().actionGet();
final Client client = client();
SearchResponse searchResponse = client.prepareSearch("test").setQuery(matchAllQuery()).execute().actionGet();
String firstNodeId = searchResponse.getHits().getAt(0).shard().nodeId();
searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).execute().actionGet();
searchResponse = client.prepareSearch("test").setQuery(matchAllQuery()).execute().actionGet();
String secondNodeId = searchResponse.getHits().getAt(0).shard().nodeId();
assertThat(firstNodeId, not(equalTo(secondNodeId)));
@ -79,8 +75,6 @@ public class SearchPreferenceTests extends AbstractSharedClusterTest {
@Test
public void simplePreferenceTests() throws Exception {
client().admin().indices().prepareDelete().execute().actionGet();
client().admin().indices().prepareCreate("test").execute().actionGet();
client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet();

View File

@ -20,6 +20,7 @@
package org.elasticsearch.search.query;
import org.apache.lucene.util.English;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
@ -34,7 +35,6 @@ import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits;
import org.elasticsearch.search.facet.FacetBuilders;
import org.elasticsearch.AbstractSharedClusterTest;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.format.ISODateTimeFormat;
@ -56,10 +56,6 @@ import static org.hamcrest.Matchers.*;
public class SimpleQueryTests extends AbstractSharedClusterTest {
public int numberOfNodes() {
return 4;
}
@Test // see https://github.com/elasticsearch/elasticsearch/issues/3177
public void testIssue3177() {
run(prepareCreate("test").setSettings(ImmutableSettings.settingsBuilder().put("index.number_of_shards", 1)));
@ -1512,7 +1508,6 @@ public class SimpleQueryTests extends AbstractSharedClusterTest {
@Test
public void testSimpleDFSQuery() throws ElasticSearchException, IOException {
cluster().ensureAtLeastNumNodes(5);
prepareCreate("test", -1,
ImmutableSettings.settingsBuilder()
.put("index.number_of_shards", 5)

View File

@ -19,12 +19,12 @@
package org.elasticsearch.search.simple;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.junit.annotations.TestLogging;
import org.junit.Test;

View File

@ -641,7 +641,7 @@ public class CompletionSuggestSearchTests extends AbstractSharedClusterTest {
private ImmutableSettings.Builder createDefaultSettings() {
int randomShardNumber = between(1, 5);
int randomReplicaNumber = between(0, numberOfNodes() - 1);
int randomReplicaNumber = between(0, cluster().numNodes() - 1);
return settingsBuilder().put(SETTING_NUMBER_OF_SHARDS, randomShardNumber).put(SETTING_NUMBER_OF_REPLICAS, randomReplicaNumber);
}

View File

@ -142,7 +142,7 @@ public class SuggestSearchTests extends AbstractSharedClusterTest {
public void testUnmappedField() throws IOException, InterruptedException, ExecutionException {
CreateIndexRequestBuilder builder = prepareCreate("test").setSettings(settingsBuilder()
.put(SETTING_NUMBER_OF_SHARDS, between(1,5))
.put(SETTING_NUMBER_OF_REPLICAS, between(0, numberOfNodes() - 1))
.put(SETTING_NUMBER_OF_REPLICAS, between(0, cluster().numNodes() - 1))
.put("index.analysis.analyzer.biword.tokenizer", "standard")
.putArray("index.analysis.analyzer.biword.filter", "shingler", "lowercase")
.put("index.analysis.filter.shingler.type", "shingle")
@ -690,7 +690,7 @@ public class SuggestSearchTests extends AbstractSharedClusterTest {
public void testShardFailures() throws IOException, InterruptedException {
CreateIndexRequestBuilder builder = prepareCreate("test").setSettings(settingsBuilder()
.put(SETTING_NUMBER_OF_SHARDS, between(1, 5))
.put(SETTING_NUMBER_OF_REPLICAS, between(0, numberOfNodes() - 1))
.put(SETTING_NUMBER_OF_REPLICAS, between(0, cluster().numNodes() - 1))
.put("index.analysis.analyzer.suggest.tokenizer", "standard")
.putArray("index.analysis.analyzer.suggest.filter", "standard", "lowercase", "shingler")
.put("index.analysis.filter.shingler.type", "shingle")
@ -798,7 +798,7 @@ public class SuggestSearchTests extends AbstractSharedClusterTest {
CreateIndexRequestBuilder builder = prepareCreate("test").setSettings(settingsBuilder()
.put(SETTING_NUMBER_OF_SHARDS, numberOfShards)
.put(SETTING_NUMBER_OF_REPLICAS, between(0, numberOfNodes() - 1))
.put(SETTING_NUMBER_OF_REPLICAS, between(0, cluster().numNodes() - 1))
.put("index.analysis.analyzer.body.tokenizer", "standard")
.putArray("index.analysis.analyzer.body.filter", "lowercase", "my_shingle")
.put("index.analysis.filter.my_shingle.type", "shingle")
@ -854,11 +854,6 @@ public class SuggestSearchTests extends AbstractSharedClusterTest {
assertSuggestion(searchSuggest, 0, 0, "simple_phrase", "nobel prize");
}
@Override
protected int numberOfNodes() {
return 3;
}
protected Suggest searchSuggest(Client client, SuggestionBuilder<?>... suggestion) {
return searchSuggest(client(), null, suggestion);
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.validate;
import com.google.common.base.Charsets;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.Priority;
@ -30,13 +31,11 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.FilterBuilders;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.AbstractSharedClusterTest;
import org.hamcrest.Matcher;
import org.junit.Test;
import java.io.IOException;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.*;
/**
@ -217,7 +216,7 @@ public class SimpleValidateQueryTests extends AbstractSharedClusterTest {
}
for (Client client : cluster().clients()) {
ValidateQueryResponse response = client().admin().indices().prepareValidateQuery("test")
ValidateQueryResponse response = client.admin().indices().prepareValidateQuery("test")
.setQuery(QueryBuilders.queryString("foo"))
.setExplain(true)
.execute().actionGet();

View File

@ -1,8 +1,8 @@
package org.elasticsearch.versioning;
import org.elasticsearch.AbstractSharedClusterTest;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.AbstractSharedClusterTest;
import org.junit.Test;
import java.util.Map;
@ -10,7 +10,6 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.nullValue;
@ -19,10 +18,6 @@ import static org.hamcrest.Matchers.nullValue;
*/
public class ConcurrentDocumentOperationTests extends AbstractSharedClusterTest {
protected int numberOfNodes() {
return 3;
}
@Test
public void concurrentOperationOnSameDocTest() throws Exception {