mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-24 17:09:48 +00:00
Tests: Revamp static bwc test framework to use dangling indexes
The static old index tests currently take a long time to run because each index version essentially recreates the cluster, and spins up new nodes. This PR instead loads each old version into the existing cluster as a dangling index. It also removes the intermediate "StaticIndexBackwardCompatibilityTest" which was an extra layer with no purpose, and moves a shared version of a commonly found function to get an http client. The test now takes between 40 and 60 seconds for me. I also ran it "under stress" by running all ES tests in one shell, while simultaneously running 10 iterations of the old index tests. Each iteration took on average about 90 seconds, which is much better than the 20+ minutes we see in master on jenkins. closes #10247
This commit is contained in:
parent
d379b3618e
commit
c3011cead4
@ -158,7 +158,8 @@ def create_client(http_port, timeout=30):
|
||||
assert False, 'Timed out waiting for node for %s seconds' % timeout
|
||||
|
||||
def generate_index(client, version):
|
||||
client.indices.delete(index='test', ignore=404)
|
||||
name = 'index-%s' % version.lower()
|
||||
client.indices.delete(index=name, ignore=404)
|
||||
num_shards = random.randint(1, 10)
|
||||
num_replicas = random.randint(0, 1)
|
||||
logging.info('Create single shard test index')
|
||||
@ -228,7 +229,7 @@ def generate_index(client, version):
|
||||
}
|
||||
}
|
||||
|
||||
client.indices.create(index='test', body={
|
||||
client.indices.create(index=name, body={
|
||||
'settings': {
|
||||
'number_of_shards': 1,
|
||||
'number_of_replicas': 0
|
||||
@ -244,9 +245,9 @@ def generate_index(client, version):
|
||||
# lighter index for it to keep bw tests reasonable
|
||||
# see https://github.com/elastic/elasticsearch/issues/5817
|
||||
num_docs = num_docs / 10
|
||||
index_documents(client, 'test', 'doc', num_docs)
|
||||
index_documents(client, name, 'doc', num_docs)
|
||||
logging.info('Running basic asserts on the data added')
|
||||
run_basic_asserts(client, 'test', 'doc', num_docs)
|
||||
run_basic_asserts(client, name, 'doc', num_docs)
|
||||
|
||||
def snapshot_index(client, cfg):
|
||||
# Add bogus persistent settings to make sure they can be restored
|
||||
|
@ -19,16 +19,20 @@
|
||||
|
||||
package org.elasticsearch.bwcompat;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
||||
|
||||
import org.apache.lucene.util.TimeUnits;
|
||||
import com.carrotsearch.randomizedtesting.LifecycleScope;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
|
||||
import org.elasticsearch.action.get.GetResponse;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.common.io.FileSystemUtils;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.env.NodeEnvironment;
|
||||
import org.elasticsearch.index.engine.EngineConfig;
|
||||
import org.elasticsearch.index.merge.policy.MergePolicyModule;
|
||||
import org.elasticsearch.index.query.FilterBuilders;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
@ -39,12 +43,15 @@ import org.elasticsearch.search.aggregations.AggregationBuilders;
|
||||
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
|
||||
import org.elasticsearch.search.sort.SortOrder;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.elasticsearch.test.hamcrest.ElasticsearchAssertions;
|
||||
import org.elasticsearch.test.index.merge.NoMergePolicyProvider;
|
||||
import org.elasticsearch.test.rest.client.http.HttpRequestBuilder;
|
||||
import org.hamcrest.Matchers;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.lang.reflect.Modifier;
|
||||
import java.net.URL;
|
||||
import java.nio.file.DirectoryStream;
|
||||
@ -54,22 +61,26 @@ import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
|
||||
@TimeoutSuite(millis = 40 * TimeUnits.MINUTE)
|
||||
public class OldIndexBackwardsCompatibilityTests extends StaticIndexBackwardCompatibilityTest {
|
||||
@LuceneTestCase.SuppressCodecs({"Lucene3x", "MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41", "Appending", "Lucene42", "Lucene45", "Lucene46", "Lucene49"})
|
||||
@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.TEST, numDataNodes = 0)
|
||||
public class OldIndexBackwardsCompatibilityTests extends ElasticsearchIntegrationTest {
|
||||
// TODO: test for proper exception on unsupported indexes (maybe via separate test?)
|
||||
// We have a 0.20.6.zip etc for this.
|
||||
|
||||
static List<String> indexes;
|
||||
static Path indicesDir;
|
||||
|
||||
@BeforeClass
|
||||
public static void initIndexes() throws Exception {
|
||||
public static void initIndexesList() throws Exception {
|
||||
indexes = new ArrayList<>();
|
||||
URL dirUrl = OldIndexBackwardsCompatibilityTests.class.getResource(".");
|
||||
Path dir = Paths.get(dirUrl.toURI());
|
||||
@ -81,6 +92,77 @@ public class OldIndexBackwardsCompatibilityTests extends StaticIndexBackwardComp
|
||||
Collections.sort(indexes);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownStatics() {
|
||||
indexes = null;
|
||||
indicesDir = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Settings nodeSettings(int ord) {
|
||||
return ImmutableSettings.builder()
|
||||
.put(Node.HTTP_ENABLED, true) // for _upgrade
|
||||
.put(MergePolicyModule.MERGE_POLICY_TYPE_KEY, NoMergePolicyProvider.class) // disable merging so no segments will be upgraded
|
||||
.build();
|
||||
}
|
||||
|
||||
void setupCluster() throws Exception {
|
||||
ListenableFuture<List<String>> replicas = internalCluster().startNodesAsync(2); // for replicas
|
||||
|
||||
Path dataDir = newTempDirPath(LifecycleScope.SUITE);
|
||||
ImmutableSettings.Builder nodeSettings = ImmutableSettings.builder()
|
||||
.put("path.data", dataDir.toAbsolutePath())
|
||||
.put("node.master", false); // workaround for dangling index loading issue when node is master
|
||||
String loadingNode = internalCluster().startNode(nodeSettings.build());
|
||||
|
||||
Path[] nodePaths = internalCluster().getInstance(NodeEnvironment.class, loadingNode).nodeDataPaths();
|
||||
assertEquals(1, nodePaths.length);
|
||||
indicesDir = nodePaths[0].resolve(NodeEnvironment.INDICES_FOLDER);
|
||||
assertFalse(Files.exists(indicesDir));
|
||||
Files.createDirectories(indicesDir);
|
||||
|
||||
replicas.get(); // wait for replicas
|
||||
}
|
||||
|
||||
String loadIndex(String indexFile) throws Exception {
|
||||
Path unzipDir = newTempDirPath();
|
||||
Path unzipDataDir = unzipDir.resolve("data");
|
||||
String indexName = indexFile.replace(".zip", "").toLowerCase(Locale.ROOT);
|
||||
|
||||
// decompress the index
|
||||
Path backwardsIndex = Paths.get(getClass().getResource(indexFile).toURI());
|
||||
try (InputStream stream = Files.newInputStream(backwardsIndex)) {
|
||||
TestUtil.unzip(stream, unzipDir);
|
||||
}
|
||||
|
||||
// check it is unique
|
||||
assertTrue(Files.exists(unzipDataDir));
|
||||
Path[] list = FileSystemUtils.files(unzipDataDir);
|
||||
if (list.length != 1) {
|
||||
throw new IllegalStateException("Backwards index must contain exactly one cluster");
|
||||
}
|
||||
|
||||
// the bwc scripts packs the indices under this path
|
||||
Path src = list[0].resolve("nodes/0/indices/" + indexName);
|
||||
Path dest = indicesDir.resolve(indexName);
|
||||
assertTrue("[" + indexFile + "] missing index dir: " + src.toString(), Files.exists(src));
|
||||
|
||||
logger.info("--> injecting index [{}] into path [{}]", indexName, dest);
|
||||
Files.move(src, dest);
|
||||
assertFalse(Files.exists(src));
|
||||
assertTrue(Files.exists(dest));
|
||||
|
||||
// force reloading dangling indices with a cluster state republish
|
||||
client().admin().cluster().prepareReroute().get();
|
||||
ensureGreen(indexName);
|
||||
return indexName;
|
||||
}
|
||||
|
||||
void unloadIndex(String indexName) throws Exception {
|
||||
ElasticsearchAssertions.assertAcked(client().admin().indices().prepareDelete(indexName).get());
|
||||
ElasticsearchAssertions.assertAllFilesClosed();
|
||||
}
|
||||
|
||||
public void testAllVersionsTested() throws Exception {
|
||||
SortedSet<String> expectedVersions = new TreeSet<>();
|
||||
for (java.lang.reflect.Field field : Version.class.getDeclaredFields()) {
|
||||
@ -109,28 +191,28 @@ public class OldIndexBackwardsCompatibilityTests extends StaticIndexBackwardComp
|
||||
}
|
||||
|
||||
public void testOldIndexes() throws Exception {
|
||||
setupCluster();
|
||||
|
||||
Collections.shuffle(indexes, getRandom());
|
||||
for (String index : indexes) {
|
||||
logger.info("Testing old index " + index);
|
||||
long startTime = System.currentTimeMillis();
|
||||
logger.info("--> Testing old index " + index);
|
||||
assertOldIndexWorks(index);
|
||||
logger.info("--> Done testing " + index + ", took " + ((System.currentTimeMillis() - startTime)/1000.0) + " seconds");
|
||||
}
|
||||
}
|
||||
|
||||
void assertOldIndexWorks(String index) throws Exception {
|
||||
Settings settings = ImmutableSettings.builder()
|
||||
.put(Node.HTTP_ENABLED, true) // for _upgrade
|
||||
.put(MergePolicyModule.MERGE_POLICY_TYPE_KEY, NoMergePolicyProvider.class) // disable merging so no segments will be upgraded
|
||||
.build();
|
||||
loadIndex(index, settings);
|
||||
logMemoryStats();
|
||||
assertBasicSearchWorks();
|
||||
assertBasicAggregationWorks();
|
||||
assertRealtimeGetWorks();
|
||||
assertNewReplicasWork();
|
||||
String indexName = loadIndex(index);
|
||||
assertIndexSanity(indexName);
|
||||
assertBasicSearchWorks(indexName);
|
||||
assertBasicAggregationWorks(indexName);
|
||||
assertRealtimeGetWorks(indexName);
|
||||
assertNewReplicasWork(indexName);
|
||||
Version version = extractVersion(index);
|
||||
assertUpgradeWorks(isLatestLuceneVersion(version));
|
||||
assertDeleteByQueryWorked(version);
|
||||
unloadIndex();
|
||||
assertUpgradeWorks(indexName, isLatestLuceneVersion(version));
|
||||
assertDeleteByQueryWorked(indexName, version);
|
||||
unloadIndex(indexName);
|
||||
}
|
||||
|
||||
Version extractVersion(String index) {
|
||||
@ -142,25 +224,38 @@ public class OldIndexBackwardsCompatibilityTests extends StaticIndexBackwardComp
|
||||
version.luceneVersion.minor == Version.CURRENT.luceneVersion.minor;
|
||||
}
|
||||
|
||||
void assertBasicSearchWorks() {
|
||||
SearchRequestBuilder searchReq = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery());
|
||||
|
||||
void assertIndexSanity(String indexName) {
|
||||
GetIndexResponse getIndexResponse = client().admin().indices().prepareGetIndex().addIndices(indexName).get();
|
||||
assertEquals(1, getIndexResponse.indices().length);
|
||||
assertEquals(indexName, getIndexResponse.indices()[0]);
|
||||
ensureYellow(indexName);
|
||||
SearchResponse test = client().prepareSearch(indexName).get();
|
||||
assertThat(test.getHits().getTotalHits(), greaterThanOrEqualTo(1l));
|
||||
}
|
||||
|
||||
void assertBasicSearchWorks(String indexName) {
|
||||
logger.info("--> testing basic search");
|
||||
SearchRequestBuilder searchReq = client().prepareSearch(indexName).setQuery(QueryBuilders.matchAllQuery());
|
||||
SearchResponse searchRsp = searchReq.get();
|
||||
ElasticsearchAssertions.assertNoFailures(searchRsp);
|
||||
long numDocs = searchRsp.getHits().getTotalHits();
|
||||
logger.info("Found " + numDocs + " in old index");
|
||||
|
||||
logger.info("--> testing basic search with sort");
|
||||
searchReq.addSort("long_sort", SortOrder.ASC);
|
||||
ElasticsearchAssertions.assertNoFailures(searchReq.get());
|
||||
|
||||
searchReq = client().prepareSearch("test").setQuery(QueryBuilders.filteredQuery(QueryBuilders.matchAllQuery(), FilterBuilders.existsFilter("string")));
|
||||
logger.info("--> testing exists filter");
|
||||
searchReq = client().prepareSearch(indexName).setQuery(QueryBuilders.filteredQuery(QueryBuilders.matchAllQuery(), FilterBuilders.existsFilter("string")));
|
||||
searchRsp = searchReq.get();
|
||||
ElasticsearchAssertions.assertNoFailures(searchRsp);
|
||||
assertThat(numDocs, equalTo(searchRsp.getHits().getTotalHits()));
|
||||
}
|
||||
|
||||
void assertBasicAggregationWorks() {
|
||||
void assertBasicAggregationWorks(String indexName) {
|
||||
// histogram on a long
|
||||
SearchResponse searchRsp = client().prepareSearch("test").addAggregation(AggregationBuilders.histogram("histo").field("long_sort").interval(10)).get();
|
||||
SearchResponse searchRsp = client().prepareSearch(indexName).addAggregation(AggregationBuilders.histogram("histo").field("long_sort").interval(10)).get();
|
||||
ElasticsearchAssertions.assertSearchResponse(searchRsp);
|
||||
Histogram histo = searchRsp.getAggregations().get("histo");
|
||||
assertNotNull(histo);
|
||||
@ -171,7 +266,7 @@ public class OldIndexBackwardsCompatibilityTests extends StaticIndexBackwardComp
|
||||
assertEquals(totalCount, searchRsp.getHits().getTotalHits());
|
||||
|
||||
// terms on a boolean
|
||||
searchRsp = client().prepareSearch("test").addAggregation(AggregationBuilders.terms("bool_terms").field("bool")).get();
|
||||
searchRsp = client().prepareSearch(indexName).addAggregation(AggregationBuilders.terms("bool_terms").field("bool")).get();
|
||||
Terms terms = searchRsp.getAggregations().get("bool_terms");
|
||||
totalCount = 0;
|
||||
for (Terms.Bucket bucket : terms.getBuckets()) {
|
||||
@ -180,64 +275,52 @@ public class OldIndexBackwardsCompatibilityTests extends StaticIndexBackwardComp
|
||||
assertEquals(totalCount, searchRsp.getHits().getTotalHits());
|
||||
}
|
||||
|
||||
void assertRealtimeGetWorks() {
|
||||
assertAcked(client().admin().indices().prepareUpdateSettings("test").setSettings(ImmutableSettings.builder()
|
||||
void assertRealtimeGetWorks(String indexName) {
|
||||
assertAcked(client().admin().indices().prepareUpdateSettings(indexName).setSettings(ImmutableSettings.builder()
|
||||
.put("refresh_interval", -1)
|
||||
.build()));
|
||||
SearchRequestBuilder searchReq = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery());
|
||||
SearchRequestBuilder searchReq = client().prepareSearch(indexName).setQuery(QueryBuilders.matchAllQuery());
|
||||
SearchHit hit = searchReq.get().getHits().getAt(0);
|
||||
String docId = hit.getId();
|
||||
// foo is new, it is not a field in the generated index
|
||||
client().prepareUpdate("test", "doc", docId).setDoc("foo", "bar").get();
|
||||
GetResponse getRsp = client().prepareGet("test", "doc", docId).get();
|
||||
client().prepareUpdate(indexName, "doc", docId).setDoc("foo", "bar").get();
|
||||
GetResponse getRsp = client().prepareGet(indexName, "doc", docId).get();
|
||||
Map<String, Object> source = getRsp.getSourceAsMap();
|
||||
assertThat(source, Matchers.hasKey("foo"));
|
||||
|
||||
assertAcked(client().admin().indices().prepareUpdateSettings("test").setSettings(ImmutableSettings.builder()
|
||||
.put("refresh_interval", "1s")
|
||||
assertAcked(client().admin().indices().prepareUpdateSettings(indexName).setSettings(ImmutableSettings.builder()
|
||||
.put("refresh_interval", EngineConfig.DEFAULT_REFRESH_INTERVAL)
|
||||
.build()));
|
||||
}
|
||||
|
||||
void assertNewReplicasWork() throws Exception {
|
||||
void assertNewReplicasWork(String indexName) throws Exception {
|
||||
final int numReplicas = randomIntBetween(1, 2);
|
||||
logger.debug("Creating [{}] nodes for replicas", numReplicas);
|
||||
internalCluster().startNodesAsync(numReplicas, ImmutableSettings.builder()
|
||||
.put("data.node", true)
|
||||
.put("master.node", false)
|
||||
.put(Node.HTTP_ENABLED, true) // for _upgrade
|
||||
.build()).get();
|
||||
logger.debug("Creating [{}] replicas for index [{}]", numReplicas, indexName);
|
||||
assertAcked(client().admin().indices().prepareUpdateSettings(indexName).setSettings(ImmutableSettings.builder()
|
||||
.put("number_of_replicas", numReplicas)
|
||||
).execute().actionGet());
|
||||
ensureGreen(indexName);
|
||||
|
||||
client().admin().cluster().prepareHealth("test").setWaitForNodes("" + (numReplicas + 1));
|
||||
assertAcked(client().admin().indices().prepareUpdateSettings("test").setSettings(ImmutableSettings.builder()
|
||||
.put("number_of_replicas", numReplicas)).execute().actionGet());
|
||||
// This can take a while when the number of replicas is greater than cluster.routing.allocation.node_concurrent_recoveries
|
||||
// (which defaults to 2). We could override that setting, but running this test on a busy box could
|
||||
// still result in taking a long time to finish starting replicas, so instead we have an increased timeout
|
||||
ensureGreen(TimeValue.timeValueMinutes(1), "test");
|
||||
|
||||
assertAcked(client().admin().indices().prepareUpdateSettings("test").setSettings(ImmutableSettings.builder()
|
||||
.put("number_of_replicas", 0))
|
||||
.execute().actionGet());
|
||||
waitNoPendingTasksOnAll(); // make sure the replicas are removed before going on
|
||||
// TODO: do something with the replicas! query? index?
|
||||
}
|
||||
|
||||
// #10067: create-bwc-index.py deleted any doc with long_sort:[10-20]
|
||||
void assertDeleteByQueryWorked(Version version) throws Exception {
|
||||
void assertDeleteByQueryWorked(String indexName, Version version) throws Exception {
|
||||
if (version.onOrBefore(Version.V_1_0_0_Beta2)) {
|
||||
// TODO: remove this once #10262 is fixed
|
||||
return;
|
||||
}
|
||||
SearchRequestBuilder searchReq = client().prepareSearch("test").setQuery(QueryBuilders.queryStringQuery("long_sort:[10 TO 20]"));
|
||||
SearchRequestBuilder searchReq = client().prepareSearch(indexName).setQuery(QueryBuilders.queryStringQuery("long_sort:[10 TO 20]"));
|
||||
assertEquals(0, searchReq.get().getHits().getTotalHits());
|
||||
}
|
||||
|
||||
void assertUpgradeWorks(boolean alreadyLatest) throws Exception {
|
||||
void assertUpgradeWorks(String indexName, boolean alreadyLatest) throws Exception {
|
||||
HttpRequestBuilder httpClient = httpClient();
|
||||
|
||||
if (alreadyLatest == false) {
|
||||
UpgradeTest.assertNotUpgraded(httpClient, "test");
|
||||
UpgradeTest.assertNotUpgraded(httpClient, indexName);
|
||||
}
|
||||
UpgradeTest.runUpgrade(httpClient, "test", "wait_for_completion", "true");
|
||||
UpgradeTest.assertUpgraded(httpClient, "test");
|
||||
UpgradeTest.runUpgrade(httpClient, indexName);
|
||||
UpgradeTest.assertUpgraded(httpClient, indexName);
|
||||
}
|
||||
}
|
||||
|
@ -1,87 +0,0 @@
|
||||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.bwcompat;
|
||||
|
||||
import org.apache.http.impl.client.HttpClients;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
|
||||
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
|
||||
import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.transport.InetSocketTransportAddress;
|
||||
import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.elasticsearch.test.hamcrest.ElasticsearchAssertions;
|
||||
import org.elasticsearch.test.rest.client.http.HttpRequestBuilder;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
import java.nio.file.Paths;
|
||||
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
|
||||
/**
|
||||
* These tests are against static indexes, built from versions of ES that cannot be upgraded without
|
||||
* a full cluster restart (ie no wire format compatibility).
|
||||
*/
|
||||
@LuceneTestCase.SuppressCodecs({"Lucene3x", "MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom", "Lucene40", "Lucene41", "Appending", "Lucene42", "Lucene45", "Lucene46", "Lucene49"})
|
||||
@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.TEST, numDataNodes = 0, minNumDataNodes = 0, maxNumDataNodes = 0)
|
||||
public class StaticIndexBackwardCompatibilityTest extends ElasticsearchIntegrationTest {
|
||||
|
||||
public void loadIndex(String index, Object... settings) throws Exception {
|
||||
logger.info("Checking static index " + index);
|
||||
Settings nodeSettings = prepareBackwardsDataDir(Paths.get(getClass().getResource(index).toURI()), settings);
|
||||
internalCluster().startNode(nodeSettings);
|
||||
ensureGreen("test");
|
||||
assertIndexSanity();
|
||||
}
|
||||
|
||||
public void unloadIndex() throws Exception {
|
||||
ElasticsearchAssertions.assertAcked(client().admin().indices().prepareDelete("test").get());
|
||||
while (internalCluster().stopRandomDataNode()) {} // stop all data nodes
|
||||
ElasticsearchAssertions.assertAllFilesClosed();
|
||||
}
|
||||
|
||||
void assertIndexSanity() {
|
||||
GetIndexResponse getIndexResponse = client().admin().indices().prepareGetIndex().get();
|
||||
assertEquals(1, getIndexResponse.indices().length);
|
||||
assertEquals("test", getIndexResponse.indices()[0]);
|
||||
ensureYellow("test");
|
||||
SearchResponse test = client().prepareSearch("test").get();
|
||||
assertThat(test.getHits().getTotalHits(), greaterThanOrEqualTo(1l));
|
||||
}
|
||||
|
||||
protected static HttpRequestBuilder httpClient() {
|
||||
NodeInfo info = nodeInfo(client());
|
||||
info.getHttp().address().boundAddress();
|
||||
TransportAddress publishAddress = info.getHttp().address().publishAddress();
|
||||
assertEquals(1, publishAddress.uniqueAddressTypeId());
|
||||
InetSocketAddress address = ((InetSocketTransportAddress) publishAddress).address();
|
||||
return new HttpRequestBuilder(HttpClients.createDefault()).host(address.getHostName()).port(address.getPort());
|
||||
}
|
||||
|
||||
static NodeInfo nodeInfo(final Client client) {
|
||||
final NodesInfoResponse nodeInfos = client.admin().cluster().prepareNodesInfo().get();
|
||||
final NodeInfo[] nodes = nodeInfos.getNodes();
|
||||
assertTrue(nodes.length > 0);
|
||||
return nodes[0];
|
||||
}
|
||||
}
|
@ -93,13 +93,7 @@ public class DirectBufferNetworkTests extends ElasticsearchIntegrationTest {
|
||||
logger.info("validating large direct buffer not allocated");
|
||||
validateNoLargeDirectBufferAllocated();
|
||||
}
|
||||
|
||||
private static HttpRequestBuilder httpClient() {
|
||||
HttpServerTransport httpServerTransport = internalCluster().getDataNodeInstance(HttpServerTransport.class);
|
||||
InetSocketAddress address = ((InetSocketTransportAddress) httpServerTransport.boundAddress().publishAddress()).address();
|
||||
return new HttpRequestBuilder(HttpClients.createDefault()).host(address.getHostName()).port(address.getPort());
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Validates that all the thread local allocated ByteBuffer in sun.nio under the Util$BufferCache
|
||||
* are not greater than 1mb.
|
||||
|
@ -59,8 +59,5 @@ public class ResponseHeaderPluginTests extends ElasticsearchIntegrationTest {
|
||||
assertThat(authResponse.getStatusCode(), equalTo(RestStatus.OK.getStatus()));
|
||||
assertThat(authResponse.getHeaders().get("Secret"), equalTo("granted"));
|
||||
}
|
||||
|
||||
private HttpRequestBuilder httpClient() {
|
||||
return new HttpRequestBuilder(HttpClients.createDefault()).httpTransport(internalCluster().getDataNodeInstance(HttpServerTransport.class));
|
||||
}
|
||||
|
||||
}
|
@ -25,7 +25,6 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.elasticsearch.test.rest.client.http.HttpResponse;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.elasticsearch.rest.CorsRegexTests.httpClient;
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
/**
|
||||
|
@ -106,12 +106,6 @@ public class CorsRegexTests extends ElasticsearchIntegrationTest {
|
||||
assertResponseWithOriginheader(response, "null");
|
||||
}
|
||||
|
||||
public static HttpRequestBuilder httpClient() {
|
||||
HttpServerTransport httpServerTransport = internalCluster().getDataNodeInstance(HttpServerTransport.class);
|
||||
InetSocketAddress address = ((InetSocketTransportAddress) httpServerTransport.boundAddress().publishAddress()).address();
|
||||
return new HttpRequestBuilder(HttpClients.createDefault()).host(address.getHostName()).port(address.getPort());
|
||||
}
|
||||
|
||||
public static void assertResponseWithOriginheader(HttpResponse response, String expectedCorsHeader) {
|
||||
assertThat(response.getStatusCode(), is(200));
|
||||
assertThat(response.getHeaders(), hasKey("Access-Control-Allow-Origin"));
|
||||
|
@ -268,12 +268,6 @@ public class UpgradeTest extends ElasticsearchBackwardsCompatIntegrationTest {
|
||||
assertTrue(rsp.hasBody());
|
||||
return (Map<String,Object>)new JsonPath(rsp.getBody()).evaluate("");
|
||||
}
|
||||
|
||||
HttpRequestBuilder httpClient() {
|
||||
InetSocketAddress[] addresses = cluster().httpAddresses();
|
||||
InetSocketAddress address = addresses[randomInt(addresses.length - 1)];
|
||||
return new HttpRequestBuilder(HttpClients.createDefault()).host(address.getHostName()).port(address.getPort());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Settings nodeSettings(int nodeOrdinal) {
|
||||
|
@ -26,6 +26,7 @@ import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import org.apache.http.impl.client.HttpClients;
|
||||
import org.apache.lucene.store.StoreRateLimiting;
|
||||
import org.apache.lucene.util.AbstractRandomizedTest;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
@ -38,6 +39,8 @@ 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.info.NodeInfo;
|
||||
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
|
||||
import org.elasticsearch.action.admin.cluster.tasks.PendingClusterTasksResponse;
|
||||
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
|
||||
import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsResponse;
|
||||
@ -89,6 +92,7 @@ import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
|
||||
import org.elasticsearch.http.HttpServerTransport;
|
||||
import org.elasticsearch.index.fielddata.FieldDataType;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
@ -114,6 +118,7 @@ import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.search.SearchService;
|
||||
import org.elasticsearch.test.client.RandomizingClient;
|
||||
import org.elasticsearch.test.disruption.ServiceDisruptionScheme;
|
||||
import org.elasticsearch.test.rest.client.http.HttpRequestBuilder;
|
||||
import org.elasticsearch.transport.netty.NettyTransport;
|
||||
import org.hamcrest.Matchers;
|
||||
import org.joda.time.DateTimeZone;
|
||||
@ -126,6 +131,7 @@ import java.lang.annotation.Inherited;
|
||||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.RetentionPolicy;
|
||||
import java.lang.annotation.Target;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.nio.file.DirectoryStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
@ -1940,6 +1946,10 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
protected HttpRequestBuilder httpClient() {
|
||||
return new HttpRequestBuilder(HttpClients.createDefault()).httpTransport(internalCluster().getDataNodeInstance(HttpServerTransport.class));
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is executed iff the test is annotated with {@link SuiteScopeTest}
|
||||
* before the first test of this class is executed.
|
||||
|
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Loading…
x
Reference in New Issue
Block a user