|
@@ -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();
|
|
|
-
|
|
|
- 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
|
|
|
+ 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);
|
|
|
+
|
|
|
+ // 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);
|
|
|
}
|
|
|
}
|