[TEST] Moved wipe* methods, randomIndexTemplate & ensureEstimatedStats from ElasticsearchIntegrationTest to TestCluster

This is the first to make it possible to have a different impl of TestCluster (e.g. based on an external cluster) that has the same methods but a different impl for them (e.g. it might use the REST API to do the same instead of the Java API)

Closes #5542
This commit is contained in:
javanna 2014-03-26 10:00:06 +01:00 committed by Luca Cavanna
parent c2e38fbf78
commit 89dd722340
25 changed files with 315 additions and 293 deletions

View File

@ -37,6 +37,7 @@ import org.junit.Test;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope.TEST;
import static org.elasticsearch.test.TestCluster.DEFAULT_MAX_NUM_SHARDS;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;

View File

@ -53,6 +53,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.*;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope.SUITE;
import static org.elasticsearch.test.TestCluster.DEFAULT_MAX_NUM_SHARDS;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.*;

View File

@ -53,7 +53,7 @@ import static org.hamcrest.Matchers.nullValue;
public class DocumentActionsTests extends ElasticsearchIntegrationTest {
protected void createIndex() {
wipeIndices(getConcreteIndexName());
cluster().wipeIndices(getConcreteIndexName());
createIndex(getConcreteIndexName());
}

View File

@ -36,6 +36,7 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.TestCluster;
import org.elasticsearch.test.engine.MockInternalEngine;
import org.elasticsearch.test.engine.ThrowingAtomicReaderWrapper;
import org.junit.Test;
@ -199,7 +200,7 @@ public class RandomExceptionCircuitBreakerTests extends ElasticsearchIntegration
private final double lowLevelRatio;
ThrowingSubReaderWrapper(Settings settings) {
final long seed = settings.getAsLong(ElasticsearchIntegrationTest.INDEX_SEED_SETTING, 0l);
final long seed = settings.getAsLong(TestCluster.SETTING_INDEX_SEED, 0l);
this.topLevelRatio = settings.getAsDouble(EXCEPTION_TOP_LEVEL_RATIO_KEY, 0.1d);
this.lowLevelRatio = settings.getAsDouble(EXCEPTION_LOW_LEVEL_RATIO_KEY, 0.1d);
this.random = new Random(seed);

View File

@ -52,7 +52,7 @@ public class ConcurrentDynamicTemplateTests extends ElasticsearchIntegrationTest
int iters = scaledRandomIntBetween(5, 15);
for (int i = 0; i < iters; i++) {
wipeIndices("test");
cluster().wipeIndices("test");
assertAcked(prepareCreate("test")
.addMapping(mappingType, mapping));
ensureYellow();

View File

@ -108,7 +108,7 @@ public class SimpleDistributorTests extends ElasticsearchIntegrationTest {
}
private void createIndexWithStoreType(String index, String storeType, String distributor) {
wipeIndices(index);
cluster().wipeIndices(index);
client().admin().indices().prepareCreate(index)
.setSettings(settingsBuilder()
.put("index.store.distributor", distributor)
@ -121,7 +121,7 @@ public class SimpleDistributorTests extends ElasticsearchIntegrationTest {
}
private void createIndexWithoutRateLimitingStoreType(String index, String storeType, String distributor) {
wipeIndices(index);
cluster().wipeIndices(index);
client().admin().indices().prepareCreate(index)
.setSettings(settingsBuilder()
.put("index.store.distributor", distributor)

View File

@ -33,6 +33,7 @@ import java.io.IOException;
import java.util.Map;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.TestCluster.DEFAULT_MAX_NUM_SHARDS;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.*;

View File

@ -46,6 +46,7 @@ import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilde
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.FilterBuilders.*;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.test.TestCluster.DEFAULT_MAX_NUM_SHARDS;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.hamcrest.Matchers.*;

View File

@ -287,7 +287,7 @@ public class PercolatorTests extends ElasticsearchIntegrationTest {
.setRefresh(true)
.execute().actionGet();
wipeIndices("test");
cluster().wipeIndices("test");
createIndex("test");
ensureGreen();

View File

@ -146,7 +146,7 @@ public class RandomTests extends ElasticsearchIntegrationTest {
final int maxNumTerms = randomIntBetween(10, 100000);
final IntOpenHashSet valuesSet = new IntOpenHashSet();
wipeIndices("idx");
cluster().wipeIndices("idx");
prepareCreate("idx").addMapping("type", jsonBuilder().startObject()
.startObject("type")
.startObject("properties")

View File

@ -101,7 +101,7 @@ public class SearchWhileCreatingIndexTests extends ElasticsearchIntegrationTest
status = client().admin().cluster().prepareHealth("test").get().getStatus();
cluster().ensureAtLeastNumNodes(numberOfReplicas + 1);
}
wipeIndices("test");
cluster().wipeIndices("test");
}
}
}

View File

@ -35,6 +35,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.TestCluster;
import org.elasticsearch.test.engine.MockInternalEngine;
import org.elasticsearch.test.engine.ThrowingAtomicReaderWrapper;
import org.elasticsearch.test.store.MockDirectoryHelper;
@ -287,7 +288,7 @@ public class SearchWithRandomExceptionsTests extends ElasticsearchIntegrationTes
private final double lowLevelRatio;
ThrowingSubReaderWrapper(Settings settings) {
final long seed = settings.getAsLong(ElasticsearchIntegrationTest.INDEX_SEED_SETTING, 0l);
final long seed = settings.getAsLong(TestCluster.SETTING_INDEX_SEED, 0l);
this.topLevelRatio = settings.getAsDouble(EXCEPTION_TOP_LEVEL_RATIO_KEY, 0.1d);
this.lowLevelRatio = settings.getAsDouble(EXCEPTION_LOW_LEVEL_RATIO_KEY, 0.1d);
this.random = new Random(seed);

View File

@ -58,8 +58,8 @@ import static org.elasticsearch.index.query.FilterBuilders.*;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.scriptFunction;
import static org.elasticsearch.search.facet.FacetBuilders.termsFacet;
import static org.elasticsearch.test.TestCluster.DEFAULT_MAX_NUM_SHARDS;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.hamcrest.Matchers.*;
/**

View File

@ -58,6 +58,7 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.FilterBuilders.*;
import static org.elasticsearch.index.query.QueryBuilders.*;
import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.scriptFunction;
import static org.elasticsearch.test.TestCluster.DEFAULT_MAX_NUM_SHARDS;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
import static org.hamcrest.Matchers.*;
@ -81,7 +82,7 @@ public class SimpleQueryTests extends ElasticsearchIntegrationTest {
SearchHit[] hits = searchResponse.getHits().hits();
assertThat(hits.length, equalTo(3));
assertThat(hits[0].score(), allOf(equalTo(hits[1].getScore()), equalTo(hits[2].getScore())));
wipeIndices("test");
cluster().wipeIndices("test");
createIndex("test");
indexRandom(true, client().prepareIndex("test", "type1", "1").setSource("field1", "the quick brown fox jumps"),
@ -375,7 +376,7 @@ public class SimpleQueryTests extends ElasticsearchIntegrationTest {
} catch (SearchPhaseExecutionException e) {
assertTrue(e.getMessage().endsWith("IllegalStateException[field \"field1\" was indexed without position data; cannot run PhraseQuery (term=quick)]; }"));
}
wipeIndices("test");
cluster().wipeIndices("test");
} catch (MapperParsingException ex) {
assertThat(version.toString(), version.onOrAfter(Version.V_1_0_0_RC2), equalTo(true));
assertThat(ex.getCause().getMessage(), equalTo("'omit_term_freq_and_positions' is not supported anymore - use ['index_options' : 'DOCS_ONLY'] instead"));

View File

@ -137,7 +137,7 @@ public class DedicatedClusterSnapshotRestoreTests extends AbstractSnapshotTests
logger.info("--> start 2 nodes");
cluster().startNode(settingsBuilder().put("gateway.type", "local"));
cluster().startNode(settingsBuilder().put("gateway.type", "local"));
wipeIndices("_all");
cluster().wipeIndices("_all");
assertAcked(prepareCreate("test-idx-1", 2, settingsBuilder().put("number_of_shards", 6)
.put("number_of_replicas", 0)

View File

@ -130,7 +130,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
// Test restore after index deletion
logger.info("--> delete indices");
wipeIndices("test-idx-1", "test-idx-2");
cluster().wipeIndices("test-idx-1", "test-idx-2");
logger.info("--> restore one index after deletion");
restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("test-idx-*", "-test-idx-2").execute().actionGet();
assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
@ -167,7 +167,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
logger.info("--> delete the index and recreate it with bar type");
wipeIndices("test-idx");
cluster().wipeIndices("test-idx");
assertAcked(prepareCreate("test-idx", 2, ImmutableSettings.builder()
.put(SETTING_NUMBER_OF_SHARDS, numShards.numPrimaries).put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)).put("refresh_interval", 5)));
assertAcked(client().admin().indices().preparePutMapping("test-idx").setType("bar").setSource("baz", "type=string"));
@ -265,7 +265,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
assertThat(client.admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap-with-global-state").get().getSnapshots().get(0).state(), equalTo(SnapshotState.SUCCESS));
logger.info("--> delete test template");
wipeTemplates("test-template");
cluster().wipeTemplates("test-template");
ClusterStateResponse clusterStateResponse = client.admin().cluster().prepareState().setRoutingTable(false).setNodes(false).setIndexTemplates("test-template").setIndices().get();
assertThat(clusterStateResponse.getState().getMetaData().templates().containsKey("test-template"), equalTo(false));
@ -302,8 +302,8 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
assertThat(client.admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap-no-global-state-with-index").get().getSnapshots().get(0).state(), equalTo(SnapshotState.SUCCESS));
logger.info("--> delete test template and index ");
wipeIndices("test-idx");
wipeTemplates("test-template");
cluster().wipeIndices("test-idx");
cluster().wipeTemplates("test-template");
clusterStateResponse = client.admin().cluster().prepareState().setRoutingTable(false).setNodes(false).setIndexTemplates("test-template").setIndices().get();
assertThat(clusterStateResponse.getState().getMetaData().templates().containsKey("test-template"), equalTo(false));
@ -469,7 +469,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
// Test restore after index deletion
logger.info("--> delete index");
wipeIndices("test-idx");
cluster().wipeIndices("test-idx");
logger.info("--> restore index after deletion");
RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute().actionGet();
assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
@ -514,7 +514,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
// Test restore after index deletion
logger.info("--> delete index");
wipeIndices("test-idx");
cluster().wipeIndices("test-idx");
logger.info("--> restore index after deletion");
ListenableActionFuture<RestoreSnapshotResponse> restoreSnapshotResponseFuture =
client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute();
@ -524,7 +524,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
assertThat(waitForIndex("test-idx", TimeValue.timeValueSeconds(10)), equalTo(true));
logger.info("--> delete index");
wipeIndices("test-idx");
cluster().wipeIndices("test-idx");
logger.info("--> get restore results");
// Now read restore results and make sure it failed
RestoreSnapshotResponse restoreSnapshotResponse = restoreSnapshotResponseFuture.actionGet(TimeValue.timeValueSeconds(10));
@ -608,7 +608,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
assertThat(numberOfFilesAfterDeletion, lessThan(numberOfFilesBeforeDeletion));
logger.info("--> delete index");
wipeIndices("test-idx");
cluster().wipeIndices("test-idx");
logger.info("--> restore index");
String lastSnapshot = "test-snap-" + (numberOfSnapshots - 1);
@ -782,7 +782,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
assertThat(snapshotInfos.get(0).shardFailures().size(), equalTo(0));
logger.info("--> delete index");
wipeIndices("test-idx");
cluster().wipeIndices("test-idx");
logger.info("--> replace mock repository with real one at the same location");
assertAcked(client.admin().cluster().preparePutRepository("test-repo")
@ -867,7 +867,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
assertThat(snapshotInfos.get(0).shardFailures().size(), equalTo(0));
logger.info("--> delete index");
wipeIndices("test-idx");
cluster().wipeIndices("test-idx");
logger.info("--> replace mock repository with real one at the same location");
assertAcked(client.admin().cluster().preparePutRepository("test-repo")
@ -911,7 +911,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
assertThat(client.admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap").get().getSnapshots().get(0).state(), equalTo(SnapshotState.SUCCESS));
logger.info("--> delete index");
wipeIndices("test-idx");
cluster().wipeIndices("test-idx");
logger.info("--> create read-only URL repository");
assertAcked(client.admin().cluster().preparePutRepository("url-repo")
@ -971,7 +971,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
logger.info("--> delete index");
wipeIndices("test-idx");
cluster().wipeIndices("test-idx");
logger.info("--> restore index");
RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute().actionGet();

View File

@ -18,22 +18,17 @@
*/
package org.elasticsearch.test;
import com.carrotsearch.hppc.ObjectArrayList;
import com.carrotsearch.randomizedtesting.RandomizedContext;
import com.carrotsearch.randomizedtesting.SeedUtils;
import com.google.common.base.Joiner;
import com.google.common.collect.Lists;
import org.apache.lucene.util.AbstractRandomizedTest;
import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsResponse;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
@ -51,7 +46,6 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.client.Requests;
import org.elasticsearch.client.internal.InternalClient;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.collect.Tuple;
@ -60,17 +54,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
import org.elasticsearch.index.mapper.FieldMapper.Loading;
import org.elasticsearch.index.merge.policy.*;
import org.elasticsearch.index.merge.scheduler.ConcurrentMergeSchedulerProvider;
import org.elasticsearch.index.merge.scheduler.MergeSchedulerModule;
import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider;
import org.elasticsearch.index.merge.scheduler.SerialMergeSchedulerProvider;
import org.elasticsearch.indices.IndexMissingException;
import org.elasticsearch.indices.IndexTemplateMissingException;
import org.elasticsearch.repositories.RepositoryMissingException;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.test.client.RandomizingClient;
import org.junit.After;
import org.junit.Before;
@ -146,7 +130,7 @@ import static org.hamcrest.Matchers.equalTo;
* <li>-D{@value #TESTS_CLIENT_RATIO} - a double value in the interval [0..1] which defines the ration between node and transport clients used</li>
* <li>-D{@value TestCluster#TESTS_ENABLE_MOCK_MODULES} - a boolean value to enable or disable mock modules. This is
* useful to test the system without asserting modules that to make sure they don't hide any bugs in production.</li>
* <li>-D{@value #INDEX_SEED_SETTING} - a random seed used to initialize the index random context.
* <li>-D{@value org.elasticsearch.test.TestCluster#SETTING_INDEX_SEED} - a random seed used to initialize the index random context.
* </ul>
* </p>
*/
@ -160,13 +144,6 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
*/
public static final String TESTS_CLIENT_RATIO = "tests.client.ratio";
/**
* Key used to retrieve the index random seed from the index settings on a running node.
* The value of this seed can be used to initialize a random context for a specific index.
* It's set once per test via a generic index template.
*/
public static final String INDEX_SEED_SETTING = "index.tests.seed";
/**
* Threshold at which indexing switches from frequently async to frequently bulk.
*/
@ -192,7 +169,7 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
private static final Map<Class<?>, TestCluster> clusters = new IdentityHashMap<Class<?>, TestCluster>();
@BeforeClass
public final static void beforeClass() throws Exception {
public static void beforeClass() throws Exception {
// Initialize lazily. No need for volatiles/ CASs since each JVM runs at most one test
// suite at any given moment.
if (GLOBAL_CLUSTER == null) {
@ -221,10 +198,8 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
fail("Unknown Scope: [" + currentClusterScope + "]");
}
currentCluster.beforeTest(getRandom(), getPerTestTransportClientRatio());
wipeIndices("_all");
wipeTemplates();
cluster().wipe();
randomIndexTemplate();
wipeRepositories();
logger.info("[{}#{}]: before test", getTestClass().getSimpleName(), getTestName());
} catch (OutOfMemoryError e) {
if (e.getMessage().contains("unable to create new native thread")) {
@ -268,12 +243,8 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
assertThat("test leaves transient cluster metadata behind: " + metaData.transientSettings().getAsMap(), metaData
.transientSettings().getAsMap().size(), equalTo(0));
}
wipeIndices("_all"); // wipe after to make sure we fail in the test that didn't ack the delete
wipeTemplates();
wipeRepositories();
ensureAllSearchersClosed();
ensureAllFilesClosed();
ensureEstimatedStats();
cluster().wipe(); // wipe after to make sure we fail in the test that didn't ack the delete
cluster().assertAfterTest();
} finally {
if (currentClusterScope == Scope.TEST) {
clearClusters(); // it is ok to leave persistent / transient cluster state behind if scope is TEST
@ -312,75 +283,19 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
* per index basis.
*/
private static void randomIndexTemplate() {
// TODO move settings for random directory etc here into the index based randomized settings.
if (cluster().size() > 0) {
ImmutableSettings.Builder builder = setRandomNormsLoading(setRandomMerge(getRandom(), ImmutableSettings.builder())
.put(INDEX_SEED_SETTING, randomLong()))
.put(SETTING_NUMBER_OF_SHARDS, between(DEFAULT_MIN_NUM_SHARDS, DEFAULT_MAX_NUM_SHARDS))
.put(SETTING_NUMBER_OF_REPLICAS, between(0, 1));
client().admin().indices().preparePutTemplate("random_index_template")
.setTemplate("*")
.setOrder(0)
.setSettings(builder)
.execute().actionGet();
}
}
private static ImmutableSettings.Builder setRandomNormsLoading(ImmutableSettings.Builder builder) {
if (randomBoolean()) {
builder.put(SearchService.NORMS_LOADING_KEY, randomFrom(Arrays.asList(Loading.EAGER, Loading.LAZY)));
}
return builder;
}
private static ImmutableSettings.Builder setRandomMerge(Random random, ImmutableSettings.Builder builder) {
if (random.nextBoolean()) {
builder.put(AbstractMergePolicyProvider.INDEX_COMPOUND_FORMAT,
random.nextBoolean() ? random.nextDouble() : random.nextBoolean());
}
Class<? extends MergePolicyProvider<?>> mergePolicy = TieredMergePolicyProvider.class;
switch (random.nextInt(5)) {
case 4:
mergePolicy = LogByteSizeMergePolicyProvider.class;
break;
case 3:
mergePolicy = LogDocMergePolicyProvider.class;
break;
case 0:
mergePolicy = null;
}
if (mergePolicy != null) {
builder.put(MergePolicyModule.MERGE_POLICY_TYPE_KEY, mergePolicy.getName());
}
if (random.nextBoolean()) {
builder.put(MergeSchedulerProvider.FORCE_ASYNC_MERGE, random.nextBoolean());
}
switch (random.nextInt(5)) {
case 4:
builder.put(MergeSchedulerModule.MERGE_SCHEDULER_TYPE_KEY, SerialMergeSchedulerProvider.class.getName());
break;
case 3:
builder.put(MergeSchedulerModule.MERGE_SCHEDULER_TYPE_KEY, ConcurrentMergeSchedulerProvider.class.getName());
break;
}
return builder;
cluster().randomIndexTemplate();
}
public static Iterable<Client> clients() {
return cluster();
}
protected static final int DEFAULT_MIN_NUM_SHARDS = 1;
protected static final int DEFAULT_MAX_NUM_SHARDS = 10;
protected int minimumNumberOfShards() {
return DEFAULT_MIN_NUM_SHARDS;
return TestCluster.DEFAULT_MIN_NUM_SHARDS;
}
protected int maximumNumberOfShards() {
return DEFAULT_MAX_NUM_SHARDS;
return TestCluster.DEFAULT_MAX_NUM_SHARDS;
}
protected int numberOfShards() {
@ -418,89 +333,6 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
return builder.build();
}
/**
* Ensures that the breaker statistics are reset to 0 since we wiped all indices and that
* means all stats should be set to 0 otherwise something is wrong with the field data
* calculation.
*/
private static void ensureEstimatedStats() {
if (cluster().size() > 0) {
NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats()
.clear().setBreaker(true).execute().actionGet();
for (NodeStats stats : nodeStats.getNodes()) {
assertThat("Breaker not reset to 0 on node: " + stats.getNode(),
stats.getBreaker().getEstimated(), equalTo(0L));
}
}
}
/**
* Deletes the given indices from the tests cluster. If no index name is passed to this method
* all indices are removed.
*/
public static void wipeIndices(String... indices) {
assert indices != null && indices.length > 0;
if (cluster().size() > 0) {
try {
assertAcked(client().admin().indices().prepareDelete(indices));
} catch (IndexMissingException e) {
// ignore
} catch (ElasticsearchIllegalArgumentException e) {
// Happens if `action.destructive_requires_name` is set to true
// which is the case in the CloseIndexDisableCloseAllTests
if ("_all".equals(indices[0])) {
ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().execute().actionGet();
ObjectArrayList<String> concreteIndices = new ObjectArrayList<String>();
for (IndexMetaData indexMetaData : clusterStateResponse.getState().metaData()) {
concreteIndices.add(indexMetaData.getIndex());
}
if (!concreteIndices.isEmpty()) {
assertAcked(client().admin().indices().prepareDelete(concreteIndices.toArray(String.class)));
}
}
}
}
}
/**
* Deletes index templates, support wildcard notation.
* If no template name is passed to this method all templates are removed.
*/
public static void wipeTemplates(String... templates) {
if (cluster().size() > 0) {
// if nothing is provided, delete all
if (templates.length == 0) {
templates = new String[]{"*"};
}
for (String template : templates) {
try {
client().admin().indices().prepareDeleteTemplate(template).execute().actionGet();
} catch (IndexTemplateMissingException e) {
// ignore
}
}
}
}
/**
* Deletes repositories, supports wildcard notation.
*/
public static void wipeRepositories(String... repositories) {
if (cluster().size() > 0) {
// if nothing is provided, delete all
if (repositories.length == 0) {
repositories = new String[]{"*"};
}
for (String repository : repositories) {
try {
client().admin().cluster().prepareDeleteRepository(repository).execute().actionGet();
} catch (RepositoryMissingException ex) {
// ignore
}
}
}
}
/**
* Creates one or more indices and asserts that the indices are acknowledged. If one of the indices
* already exists this method will fail and wipe all the indices created so far.
@ -516,7 +348,7 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
success = true;
} finally {
if (!success && !created.isEmpty()) {
wipeIndices(created.toArray(new String[created.size()]));
cluster().wipeIndices(created.toArray(new String[created.size()]));
}
}
}

View File

@ -39,7 +39,6 @@ import org.elasticsearch.common.util.concurrent.EsAbortPolicy;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.test.cache.recycler.MockBigArrays;
import org.elasticsearch.test.engine.MockInternalEngine;
import org.elasticsearch.test.junit.listeners.LoggingListener;
import org.elasticsearch.test.store.MockDirectoryHelper;
import org.junit.After;
@ -54,10 +53,10 @@ import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
import java.net.URI;
import java.util.*;
import java.util.Map.Entry;
import java.util.concurrent.TimeUnit;
import static org.hamcrest.Matchers.is;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllFilesClosed;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllSearchersClosed;
/**
* Base testcase for randomized unit testing with Elasticsearch
@ -150,71 +149,6 @@ public abstract class ElasticsearchTestCase extends AbstractRandomizedTest {
MockBigArrays.ensureAllArraysAreReleased();
}
public static void ensureAllFilesClosed() throws IOException {
try {
for (final MockDirectoryHelper.ElasticsearchMockDirectoryWrapper w : MockDirectoryHelper.wrappers) {
try {
awaitBusy(new Predicate<Object>() {
@Override
public boolean apply(Object input) {
return !w.isOpen();
}
});
} catch (InterruptedException e) {
Thread.interrupted();
}
if (!w.successfullyClosed()) {
if (w.closeException() == null) {
w.close();
if (w.closeException() != null) {
throw w.closeException();
}
} else {
throw w.closeException();
}
}
assertThat(w.isOpen(), is(false));
}
} finally {
forceClearMockWrappers();
}
}
public static void ensureAllSearchersClosed() {
/* in some cases we finish a test faster than the freeContext calls make it to the
* shards. Let's wait for some time if there are still searchers. If the are really
* pending we will fail anyway.*/
try {
if (awaitBusy(new Predicate<Object>() {
public boolean apply(Object o) {
return MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.isEmpty();
}
}, 5, TimeUnit.SECONDS)) {
return;
}
} catch (InterruptedException ex) {
if (MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.isEmpty()) {
return;
}
}
try {
RuntimeException ex = null;
StringBuilder builder = new StringBuilder("Unclosed Searchers instance for shards: [");
for (Entry<MockInternalEngine.AssertingSearcher, RuntimeException> entry : MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.entrySet()) {
ex = entry.getValue();
builder.append(entry.getKey().shardId()).append(",");
}
builder.append("]");
throw new RuntimeException(builder.toString(), ex);
} finally {
MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.clear();
}
}
public static void forceClearMockWrappers() {
MockDirectoryHelper.wrappers.clear();
}
public static boolean hasUnclosedWrapper() {
for (MockDirectoryWrapper w : MockDirectoryHelper.wrappers) {
if (w.isOpen()) {
@ -229,13 +163,13 @@ public abstract class ElasticsearchTestCase extends AbstractRandomizedTest {
closeAfterSuite(new Closeable() {
@Override
public void close() throws IOException {
ensureAllFilesClosed();
assertAllFilesClosed();
}
});
closeAfterSuite(new Closeable() {
@Override
public void close() throws IOException {
ensureAllSearchersClosed();
assertAllSearchersClosed();
}
});
defaultHandler = Thread.getDefaultUncaughtExceptionHandler();
@ -366,6 +300,4 @@ public abstract class ElasticsearchTestCase extends AbstractRandomizedTest {
return threadGroup.getName();
}
}
}

View File

@ -18,7 +18,9 @@
*/
package org.elasticsearch.test;
import com.carrotsearch.hppc.ObjectArrayList;
import com.carrotsearch.randomizedtesting.SeedUtils;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import com.google.common.base.Predicate;
import com.google.common.base.Predicates;
@ -26,8 +28,12 @@ import com.google.common.collect.Collections2;
import com.google.common.collect.Iterators;
import com.google.common.collect.Sets;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.cache.recycler.CacheRecycler;
import org.elasticsearch.cache.recycler.PageCacheRecyclerModule;
import org.elasticsearch.client.Client;
@ -35,6 +41,7 @@ import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.client.transport.TransportClient;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.ShardRouting;
@ -50,8 +57,17 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArraysModule;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.engine.IndexEngineModule;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.merge.policy.*;
import org.elasticsearch.index.merge.scheduler.ConcurrentMergeSchedulerProvider;
import org.elasticsearch.index.merge.scheduler.MergeSchedulerModule;
import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider;
import org.elasticsearch.index.merge.scheduler.SerialMergeSchedulerProvider;
import org.elasticsearch.indices.IndexMissingException;
import org.elasticsearch.indices.IndexTemplateMissingException;
import org.elasticsearch.node.Node;
import org.elasticsearch.node.internal.InternalNode;
import org.elasticsearch.repositories.RepositoryMissingException;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.test.cache.recycler.MockBigArraysModule;
import org.elasticsearch.test.cache.recycler.MockPageCacheRecyclerModule;
@ -66,6 +82,7 @@ import org.junit.Assert;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.util.*;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@ -74,8 +91,15 @@ import static com.carrotsearch.randomizedtesting.RandomizedTest.systemPropertyAs
import static com.google.common.collect.Maps.newTreeMap;
import static org.apache.lucene.util.LuceneTestCase.rarely;
import static org.apache.lucene.util.LuceneTestCase.usually;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllFilesClosed;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAllSearchersClosed;
import static org.hamcrest.Matchers.equalTo;
import static org.junit.Assert.assertThat;
/**
* TestCluster manages a set of JVM private nodes and allows convenient access to them.
@ -106,14 +130,23 @@ public final class TestCluster implements Iterable<Client> {
*/
public static final String SETTING_CLUSTER_NODE_SEED = "test.cluster.node.seed";
/**
* Key used to retrieve the index random seed from the index settings on a running node.
* The value of this seed can be used to initialize a random context for a specific index.
* It's set once per test via a generic index template.
*/
public static final String SETTING_INDEX_SEED = "index.tests.seed";
private static final String CLUSTER_NAME_KEY = "cluster.name";
private static final boolean ENABLE_MOCK_MODULES = systemPropertyAsBoolean(TESTS_ENABLE_MOCK_MODULES, true);
static final int DEFAULT_MIN_NUM_NODES = 2;
static final int DEFAULT_MAX_NUM_NODES = 6;
public static final int DEFAULT_MIN_NUM_SHARDS = 1;
public static final int DEFAULT_MAX_NUM_SHARDS = 10;
/* sorted map to make traverse order reproducible */
private final TreeMap<String, NodeAndClient> nodes = newTreeMap();
@ -626,7 +659,7 @@ public final class TestCluster implements Iterable<Client> {
public static TransportClientFactory NO_SNIFF_CLIENT_FACTORY = new TransportClientFactory(false);
public static TransportClientFactory SNIFF_CLIENT_FACTORY = new TransportClientFactory(true);
public TransportClientFactory(boolean sniff) {
private TransportClientFactory(boolean sniff) {
this.sniff = sniff;
}
@ -725,13 +758,168 @@ public final class TestCluster implements Iterable<Client> {
logger.debug("Cluster is consistent again - nodes: [{}] nextNodeId: [{}] numSharedNodes: [{}]", nodes.keySet(), nextNodeId.get(), sharedNodesSeeds.length);
}
public void wipe() {
wipeIndices("_all");
wipeTemplates();
wipeRepositories();
}
/**
* Deletes the given indices from the tests cluster. If no index name is passed to this method
* all indices are removed.
*/
public void wipeIndices(String... indices) {
assert indices != null && indices.length > 0;
if (size() > 0) {
try {
assertAcked(client().admin().indices().prepareDelete(indices));
} catch (IndexMissingException e) {
// ignore
} catch (ElasticsearchIllegalArgumentException e) {
// Happens if `action.destructive_requires_name` is set to true
// which is the case in the CloseIndexDisableCloseAllTests
if ("_all".equals(indices[0])) {
ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().execute().actionGet();
ObjectArrayList<String> concreteIndices = new ObjectArrayList<String>();
for (IndexMetaData indexMetaData : clusterStateResponse.getState().metaData()) {
concreteIndices.add(indexMetaData.getIndex());
}
if (!concreteIndices.isEmpty()) {
assertAcked(client().admin().indices().prepareDelete(concreteIndices.toArray(String.class)));
}
}
}
}
}
/**
* Deletes index templates, support wildcard notation.
* If no template name is passed to this method all templates are removed.
*/
public void wipeTemplates(String... templates) {
if (size() > 0) {
// if nothing is provided, delete all
if (templates.length == 0) {
templates = new String[]{"*"};
}
for (String template : templates) {
try {
client().admin().indices().prepareDeleteTemplate(template).execute().actionGet();
} catch (IndexTemplateMissingException e) {
// ignore
}
}
}
}
/**
* Deletes repositories, supports wildcard notation.
*/
public void wipeRepositories(String... repositories) {
if (size() > 0) {
// if nothing is provided, delete all
if (repositories.length == 0) {
repositories = new String[]{"*"};
}
for (String repository : repositories) {
try {
client().admin().cluster().prepareDeleteRepository(repository).execute().actionGet();
} catch (RepositoryMissingException ex) {
// ignore
}
}
}
}
/**
* Ensures that the breaker statistics are reset to 0 since we wiped all indices and that
* means all stats should be set to 0 otherwise something is wrong with the field data
* calculation.
*/
public void ensureEstimatedStats() {
if (size() > 0) {
NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats()
.clear().setBreaker(true).execute().actionGet();
for (NodeStats stats : nodeStats.getNodes()) {
assertThat("Breaker not reset to 0 on node: " + stats.getNode(),
stats.getBreaker().getEstimated(), equalTo(0L));
}
}
}
/**
* Creates a randomized index template. This template is used to pass in randomized settings on a
* per index basis.
*/
public void randomIndexTemplate() {
// TODO move settings for random directory etc here into the index based randomized settings.
if (size() > 0) {
ImmutableSettings.Builder builder = setRandomNormsLoading(setRandomMerge(random, ImmutableSettings.builder())
.put(SETTING_INDEX_SEED, random.nextLong()))
.put(SETTING_NUMBER_OF_SHARDS, RandomInts.randomIntBetween(random, DEFAULT_MIN_NUM_SHARDS, DEFAULT_MAX_NUM_SHARDS))
.put(SETTING_NUMBER_OF_REPLICAS, RandomInts.randomIntBetween(random, 0, 1));
client().admin().indices().preparePutTemplate("random_index_template")
.setTemplate("*")
.setOrder(0)
.setSettings(builder)
.execute().actionGet();
}
}
private ImmutableSettings.Builder setRandomNormsLoading(ImmutableSettings.Builder builder) {
if (random.nextBoolean()) {
builder.put(SearchService.NORMS_LOADING_KEY, RandomPicks.randomFrom(random, Arrays.asList(FieldMapper.Loading.EAGER, FieldMapper.Loading.LAZY)));
}
return builder;
}
private static ImmutableSettings.Builder setRandomMerge(Random random, ImmutableSettings.Builder builder) {
if (random.nextBoolean()) {
builder.put(AbstractMergePolicyProvider.INDEX_COMPOUND_FORMAT,
random.nextBoolean() ? random.nextDouble() : random.nextBoolean());
}
Class<? extends MergePolicyProvider<?>> mergePolicy = TieredMergePolicyProvider.class;
switch (random.nextInt(5)) {
case 4:
mergePolicy = LogByteSizeMergePolicyProvider.class;
break;
case 3:
mergePolicy = LogDocMergePolicyProvider.class;
break;
case 0:
mergePolicy = null;
}
if (mergePolicy != null) {
builder.put(MergePolicyModule.MERGE_POLICY_TYPE_KEY, mergePolicy.getName());
}
if (random.nextBoolean()) {
builder.put(MergeSchedulerProvider.FORCE_ASYNC_MERGE, random.nextBoolean());
}
switch (random.nextInt(5)) {
case 4:
builder.put(MergeSchedulerModule.MERGE_SCHEDULER_TYPE_KEY, SerialMergeSchedulerProvider.class.getName());
break;
case 3:
builder.put(MergeSchedulerModule.MERGE_SCHEDULER_TYPE_KEY, ConcurrentMergeSchedulerProvider.class.getName());
break;
}
return builder;
}
/**
* This method should be executed during tearDown
*/
public synchronized void afterTest() {
wipeDataDirectories();
resetClients(); /* reset all clients - each test gets its own client based on the Random instance created above. */
}
public void assertAfterTest() throws IOException {
assertAllSearchersClosed();
assertAllFilesClosed();
ensureEstimatedStats();
}
private void resetClients() {
@ -760,7 +948,7 @@ public final class TestCluster implements Iterable<Client> {
}
/**
* Returns an Iterabel to all instances for the given class &gt;T&lt; across all nodes in the cluster.
* Returns an Iterable to all instances for the given class &gt;T&lt; across all nodes in the cluster.
*/
public synchronized <T> Iterable<T> getInstances(Class<T> clazz) {
List<T> instances = new ArrayList<T>(nodes.size());

View File

@ -46,7 +46,7 @@ import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.indices.warmer.IndicesWarmer;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.TestCluster;
import org.elasticsearch.threadpool.ThreadPool;
import java.lang.reflect.Constructor;
@ -72,7 +72,7 @@ public final class MockInternalEngine extends InternalEngine implements Engine {
CodecService codecService) throws EngineException {
super(shardId, indexSettings, threadPool, indexSettingsService, indexingService, warmer, store,
deletionPolicy, translog, mergePolicyProvider, mergeScheduler, analysisService, similarityService, codecService);
final long seed = indexSettings.getAsLong(ElasticsearchIntegrationTest.INDEX_SEED_SETTING, 0l);
final long seed = indexSettings.getAsLong(TestCluster.SETTING_INDEX_SEED, 0l);
random = new Random(seed);
final double ratio = indexSettings.getAsDouble(WRAP_READER_RATIO, 0.0d); // DISABLED by default - AssertingDR is crazy slow
wrapper = indexSettings.getAsClass(READER_WRAPPER_TYPE, AssertingDirectoryReader.class);

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.test.hamcrest;
import com.google.common.base.Predicate;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Query;
import org.elasticsearch.ElasticsearchException;
@ -46,18 +47,18 @@ import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.elasticsearch.test.engine.MockInternalEngine;
import org.elasticsearch.test.store.MockDirectoryHelper;
import org.hamcrest.Matcher;
import org.hamcrest.Matchers;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Locale;
import java.util.Set;
import java.util.*;
import java.util.concurrent.TimeUnit;
import static org.elasticsearch.test.ElasticsearchTestCase.*;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.*;
import static org.junit.Assert.assertTrue;
@ -377,8 +378,8 @@ public class ElasticsearchAssertions {
}
public static void assertVersionSerializable(Streamable streamable) {
assertTrue(Version.CURRENT.after(ElasticsearchTestCase.getPreviousVersion()));
assertVersionSerializable(ElasticsearchTestCase.randomVersion(), streamable);
assertTrue(Version.CURRENT.after(getPreviousVersion()));
assertVersionSerializable(randomVersion(), streamable);
}
public static void assertVersionSerializable(Version version, Streamable streamable) {
@ -434,4 +435,65 @@ public class ElasticsearchAssertions {
assertThat("One or more shards were not successful but didn't trigger a failure", response.getSuccessfulShards(), equalTo(response.getTotalShards()));
return response;
}
public static void assertAllSearchersClosed() {
/* in some cases we finish a test faster than the freeContext calls make it to the
* shards. Let's wait for some time if there are still searchers. If the are really
* pending we will fail anyway.*/
try {
if (awaitBusy(new Predicate<Object>() {
public boolean apply(Object o) {
return MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.isEmpty();
}
}, 5, TimeUnit.SECONDS)) {
return;
}
} catch (InterruptedException ex) {
if (MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.isEmpty()) {
return;
}
}
try {
RuntimeException ex = null;
StringBuilder builder = new StringBuilder("Unclosed Searchers instance for shards: [");
for (Map.Entry<MockInternalEngine.AssertingSearcher, RuntimeException> entry : MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.entrySet()) {
ex = entry.getValue();
builder.append(entry.getKey().shardId()).append(",");
}
builder.append("]");
throw new RuntimeException(builder.toString(), ex);
} finally {
MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.clear();
}
}
public static void assertAllFilesClosed() throws IOException {
try {
for (final MockDirectoryHelper.ElasticsearchMockDirectoryWrapper w : MockDirectoryHelper.wrappers) {
try {
awaitBusy(new Predicate<Object>() {
@Override
public boolean apply(Object input) {
return !w.isOpen();
}
});
} catch (InterruptedException e) {
Thread.interrupted();
}
if (!w.successfullyClosed()) {
if (w.closeException() == null) {
w.close();
if (w.closeException() != null) {
throw w.closeException();
}
} else {
throw w.closeException();
}
}
assertThat(w.isOpen(), is(false));
}
} finally {
MockDirectoryHelper.wrappers.clear();
}
}
}

View File

@ -36,6 +36,7 @@ import org.elasticsearch.index.store.fs.MmapFsDirectoryService;
import org.elasticsearch.index.store.fs.NioFsDirectoryService;
import org.elasticsearch.index.store.fs.SimpleFsDirectoryService;
import org.elasticsearch.index.store.ram.RamDirectoryService;
import org.elasticsearch.test.TestCluster;
import java.io.IOException;
import java.util.Random;

View File

@ -39,7 +39,7 @@ import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.fs.FsDirectoryService;
import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.TestCluster;
import java.io.File;
import java.io.IOException;
@ -56,7 +56,7 @@ public class MockFSDirectoryService extends FsDirectoryService {
@Inject
public MockFSDirectoryService(final ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore, final IndicesService service) {
super(shardId, indexSettings, indexStore);
final long seed = indexSettings.getAsLong(ElasticsearchIntegrationTest.INDEX_SEED_SETTING, 0l);
final long seed = indexSettings.getAsLong(TestCluster.SETTING_INDEX_SEED, 0l);
Random random = new Random(seed);
helper = new MockDirectoryHelper(shardId, indexSettings, logger, random, seed);
checkIndexOnClose = indexSettings.getAsBoolean(CHECK_INDEX_ON_CLOSE, random.nextDouble() < 0.1);

View File

@ -25,7 +25,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.TestCluster;
import java.io.IOException;
import java.util.Random;
@ -38,7 +38,7 @@ public class MockRamDirectoryService extends AbstractIndexShardComponent impleme
@Inject
public MockRamDirectoryService(ShardId shardId, Settings indexSettings) {
super(shardId, indexSettings);
final long seed = indexSettings.getAsLong(ElasticsearchIntegrationTest.INDEX_SEED_SETTING, 0l);
final long seed = indexSettings.getAsLong(TestCluster.SETTING_INDEX_SEED, 0l);
Random random = new Random(seed);
helper = new MockDirectoryHelper(shardId, indexSettings, logger, random, seed);
delegateService = helper.randomRamDirectoryService();

View File

@ -23,8 +23,8 @@ import org.elasticsearch.Version;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.elasticsearch.test.TestCluster;
import org.elasticsearch.test.hamcrest.ElasticsearchAssertions;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*;
@ -42,7 +42,7 @@ public class AssertingLocalTransport extends LocalTransport {
@Inject
public AssertingLocalTransport(Settings settings, ThreadPool threadPool, Version version) {
super(settings, threadPool, version);
final long seed = settings.getAsLong(ElasticsearchIntegrationTest.INDEX_SEED_SETTING, 0l);
final long seed = settings.getAsLong(TestCluster.SETTING_INDEX_SEED, 0l);
random = new Random(seed);
}