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:
parent
e110d53b0c
commit
cabbf7805b
1
pom.xml
1
pom.xml
|
@ -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>
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
public ClusterService clusterService() {
|
||||
return ((InternalNode) getOneNode()).injector().getInstance(ClusterService.class);
|
||||
// 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 int numNodes() {
|
||||
private void resetClients() {
|
||||
final Collection<NodeAndClient> nodesAndClients = nodes.values();
|
||||
for (NodeAndClient nodeAndClient : nodesAndClients) {
|
||||
nodeAndClient.resetClient();
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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,8 +82,11 @@ public class ReproduceInfoPrinter extends RunListener {
|
|||
if (sysPropName.equals("tests.iters")) { // we don't want the iters to be in there!
|
||||
return this;
|
||||
}
|
||||
if (value != null && !value.isEmpty()) {
|
||||
return super.appendOpt(sysPropName, value);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public ReproduceErrorMessageBuilder appendESProperties() {
|
||||
for (String sysPropName : Arrays.asList(
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -9,11 +9,6 @@ public class ExtendedFacetsTestsMultiShardMultiNodeTests extends ExtendedFacetsT
|
|||
return 8;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int numberOfNodes() {
|
||||
return 4;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int numDocs() {
|
||||
return 10000;
|
||||
|
|
|
@ -31,8 +31,4 @@ public class SimpleFacetsMultiShardMultiNodeTests extends SimpleFacetsTests {
|
|||
return 3;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int numberOfNodes() {
|
||||
return 2;
|
||||
}
|
||||
}
|
|
@ -76,11 +76,6 @@ public class SimpleFacetsTests extends AbstractSharedClusterTest {
|
|||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int numberOfNodes() {
|
||||
return 1;
|
||||
}
|
||||
|
||||
protected int numberOfRuns() {
|
||||
return 5;
|
||||
}
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
Loading…
Reference in New Issue