Merge pull request #11666 from brwe/meta-data-zen

[TEST] remove redundant tests and move to different suite
This commit is contained in:
Britta Weber 2015-07-27 12:40:11 +02:00
commit 20facccaf6
6 changed files with 211 additions and 236 deletions

View File

@ -110,8 +110,11 @@ public class ClusterChangedEvent {
// is actually supposed to be deleted or imported as dangling instead. for example a new master might not have
// the index in its cluster state because it was started with an empty data folder and in this case we want to
// import as dangling. we check here for new master too to be on the safe side in this case.
// norelease because we are not sure this is actually a good solution
// See discussion on https://github.com/elastic/elasticsearch/pull/9952
// This means that under certain conditions deleted indices might be reimported if a master fails while the deletion
// request is issued and a node receives the cluster state that would trigger the deletion from the new master.
// See test MetaDataWriteDataNodesTests.testIndicesDeleted()
// See discussion on https://github.com/elastic/elasticsearch/pull/9952 and
// https://github.com/elastic/elasticsearch/issues/11665
if (hasNewMaster() || previousState == null) {
return ImmutableList.of();
}

View File

@ -28,7 +28,8 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.*;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
@ -98,6 +99,7 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
@Override
public void clusterChanged(ClusterChangedEvent event) {
Set<String> relevantIndices = new HashSet<>();
final ClusterState state = event.state();
if (state.blocks().disableStatePersistence()) {
@ -148,7 +150,7 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
}
Iterable<IndexMetaWriteInfo> writeInfo;
relevantIndices = getRelevantIndices(event.state(), previouslyWrittenIndices);
relevantIndices = getRelevantIndices(event.state(), event.previousState(), previouslyWrittenIndices);
writeInfo = resolveStatesToBeWritten(previouslyWrittenIndices, relevantIndices, previousMetaData, event.state().metaData());
// check and write changes in indices
for (IndexMetaWriteInfo indexMetaWrite : writeInfo) {
@ -169,10 +171,10 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
}
}
public static Set<String> getRelevantIndices(ClusterState state, ImmutableSet<String> previouslyWrittenIndices) {
public static Set<String> getRelevantIndices(ClusterState state, ClusterState previousState,ImmutableSet<String> previouslyWrittenIndices) {
Set<String> relevantIndices;
if (isDataOnlyNode(state)) {
relevantIndices = getRelevantIndicesOnDataOnlyNode(state, previouslyWrittenIndices);
relevantIndices = getRelevantIndicesOnDataOnlyNode(state, previousState, previouslyWrittenIndices);
} else if (state.nodes().localNode().masterNode() == true) {
relevantIndices = getRelevantIndicesForMasterEligibleNode(state);
} else {
@ -278,7 +280,7 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
return indicesToWrite;
}
public static Set<String> getRelevantIndicesOnDataOnlyNode(ClusterState state, ImmutableSet<String> previouslyWrittenIndices) {
public static Set<String> getRelevantIndicesOnDataOnlyNode(ClusterState state, ClusterState previousState, ImmutableSet<String> previouslyWrittenIndices) {
RoutingNode newRoutingNode = state.getRoutingNodes().node(state.nodes().localNodeId());
if (newRoutingNode == null) {
throw new IllegalStateException("cluster state does not contain this node - cannot write index meta state");
@ -289,7 +291,14 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
}
// we have to check the meta data also: closed indices will not appear in the routing table, but we must still write the state if we have it written on disk previously
for (IndexMetaData indexMetaData : state.metaData()) {
if (previouslyWrittenIndices.contains(indexMetaData.getIndex()) && state.metaData().getIndices().get(indexMetaData.getIndex()).state().equals(IndexMetaData.State.CLOSE)) {
boolean isOrWasClosed = indexMetaData.state().equals(IndexMetaData.State.CLOSE);
// if the index is open we might still have to write the state if it just transitioned from closed to open
// so we have to check for that as well.
IndexMetaData previousMetaData = previousState.metaData().getIndices().get(indexMetaData.getIndex());
if (previousMetaData != null) {
isOrWasClosed = isOrWasClosed || previousMetaData.state().equals(IndexMetaData.State.CLOSE);
}
if (previouslyWrittenIndices.contains(indexMetaData.getIndex()) && isOrWasClosed) {
indices.add(indexMetaData.getIndex());
}
}

View File

@ -56,6 +56,7 @@ import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.transport.*;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import java.io.IOException;
@ -65,6 +66,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
@ -237,7 +239,9 @@ public class DiscoveryWithServiceDisruptionsTests extends ElasticsearchIntegrati
}
/** Verify that nodes fault detection works after master (re) election */
/**
* Verify that nodes fault detection works after master (re) election
*/
@Test
public void testNodesFDAfterMasterReelection() throws Exception {
startCluster(4);
@ -414,7 +418,7 @@ public class DiscoveryWithServiceDisruptionsTests extends ElasticsearchIntegrati
/**
* Test that we do not loose document whose indexing request was successful, under a randomly selected disruption scheme
* We also collect & report the type of indexing failures that occur.
*
* <p/>
* This test is a superset of tests run in the Jepsen test suite, with the exception of versioned updates
*/
@Test
@ -948,6 +952,50 @@ public class DiscoveryWithServiceDisruptionsTests extends ElasticsearchIntegrati
ensureStableCluster(3);
}
@Test
public void testIndexImportedFromDataOnlyNodesIfMasterLostDataFolder() throws Exception {
// test for https://github.com/elastic/elasticsearch/issues/8823
configureCluster(2, 1);
String masterNode = internalCluster().startMasterOnlyNode(Settings.EMPTY);
internalCluster().startDataOnlyNode(Settings.EMPTY);
ensureStableCluster(2);
assertAcked(prepareCreate("index").setSettings(Settings.builder().put("index.number_of_replicas", 0)));
index("index", "doc", "1", jsonBuilder().startObject().field("text", "some text").endObject());
ensureGreen();
internalCluster().restartNode(masterNode, new InternalTestCluster.RestartCallback() {
public boolean clearData(String nodeName) {
return true;
}
});
ensureGreen("index");
assertTrue(client().prepareGet("index", "doc", "1").get().isExists());
}
// tests if indices are really deleted even if a master transition inbetween
@Ignore("https://github.com/elastic/elasticsearch/issues/11665")
@Test
public void testIndicesDeleted() throws Exception {
configureCluster(3, 2);
Future<List<String>> masterNodes= internalCluster().startMasterOnlyNodesAsync(2);
Future<String> dataNode = internalCluster().startDataOnlyNodeAsync();
dataNode.get();
masterNodes.get();
ensureStableCluster(3);
assertAcked(prepareCreate("test"));
ensureYellow();
String masterNode1 = internalCluster().getMasterName();
NetworkPartition networkPartition = new NetworkUnresponsivePartition(masterNode1, dataNode.get(), getRandom());
internalCluster().setDisruptionScheme(networkPartition);
networkPartition.startDisrupting();
internalCluster().client(masterNode1).admin().indices().prepareDelete("test").setTimeout("1s").get();
internalCluster().restartNode(masterNode1, InternalTestCluster.EMPTY_CALLBACK);
ensureYellow();
assertFalse(client().admin().indices().prepareExists("test").get().isExists());
}
protected NetworkPartition addRandomPartition() {
NetworkPartition partition;

View File

@ -174,9 +174,9 @@ public class GatewayMetaStateTests extends ElasticsearchAllocationTestCase {
if (stateInMemory) {
inMemoryMetaData = event.previousState().metaData();
ImmutableSet.Builder<String> relevantIndices = ImmutableSet.builder();
oldIndicesList = relevantIndices.addAll(GatewayMetaState.getRelevantIndices(event.previousState(), oldIndicesList)).build();
oldIndicesList = relevantIndices.addAll(GatewayMetaState.getRelevantIndices(event.previousState(), event.previousState(), oldIndicesList)).build();
}
Set<String> newIndicesList = GatewayMetaState.getRelevantIndices(event.state(), oldIndicesList);
Set<String> newIndicesList = GatewayMetaState.getRelevantIndices(event.state(),event.previousState(), oldIndicesList);
// third, get the actual write info
Iterator<GatewayMetaState.IndexMetaWriteInfo> indices = GatewayMetaState.resolveStatesToBeWritten(oldIndicesList, newIndicesList, inMemoryMetaData, event.state().metaData()).iterator();

View File

@ -21,6 +21,7 @@ package org.elasticsearch.gateway;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import com.google.common.base.Predicate;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -33,227 +34,81 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest.ClusterScope;
import org.elasticsearch.test.InternalTestCluster;
import org.junit.Test;
import java.io.IOException;
import java.util.LinkedHashMap;
import java.util.concurrent.Future;
import static org.elasticsearch.client.Requests.clusterHealthRequest;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope;
import static org.elasticsearch.test.InternalTestCluster.RestartCallback;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
/**
*
*/
@LuceneTestCase.Slow
@ClusterScope(scope = Scope.TEST, numDataNodes = 0)
public class MetaDataWriteDataNodesTests extends ElasticsearchIntegrationTest {
@Test
public void testMetaWrittenAlsoOnDataNode() throws Exception {
// this test checks that index state is written on data only nodes
String masterNodeName = startMasterNode();
String redNode = startDataNode("red");
assertAcked(prepareCreate("test").setSettings(Settings.builder().put("index.number_of_replicas", 0)));
// this test checks that index state is written on data only nodes if they have a shard allocated
String masterNode = internalCluster().startMasterOnlyNode(Settings.EMPTY);
String dataNode = internalCluster().startDataOnlyNode(Settings.EMPTY);
assertAcked(prepareCreate("test").setSettings("index.number_of_replicas", 0));
index("test", "doc", "1", jsonBuilder().startObject().field("text", "some text").endObject());
ensureGreen("test");
assertIndexInMetaState(redNode, "test");
assertIndexInMetaState(masterNodeName, "test");
//stop master node and start again with an empty data folder
((InternalTestCluster) cluster()).stopCurrentMasterNode();
String newMasterNode = startMasterNode();
ensureGreen("test");
// check for meta data
assertIndexInMetaState(redNode, "test");
assertIndexInMetaState(newMasterNode, "test");
// check if index and doc is still there
ensureGreen("test");
assertTrue(client().prepareGet("test", "doc", "1").get().isExists());
}
@Test
public void testMetaWrittenOnlyForIndicesOnNodesThatHaveAShard() throws Exception {
// this test checks that the index state is only written to a data only node if they have a shard of that index allocated on the node
String masterNode = startMasterNode();
String blueNode = startDataNode("blue");
String redNode = startDataNode("red");
assertAcked(prepareCreate("blue_index").setSettings(Settings.builder().put("index.number_of_replicas", 0).put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "color", "blue")));
index("blue_index", "doc", "1", jsonBuilder().startObject().field("text", "some text").endObject());
assertAcked(prepareCreate("red_index").setSettings(Settings.builder().put("index.number_of_replicas", 0).put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "color", "red")));
index("red_index", "doc", "1", jsonBuilder().startObject().field("text", "some text").endObject());
ensureGreen();
assertIndexNotInMetaState(blueNode, "red_index");
assertIndexNotInMetaState(redNode, "blue_index");
assertIndexInMetaState(blueNode, "blue_index");
assertIndexInMetaState(redNode, "red_index");
assertIndexInMetaState(masterNode, "red_index");
assertIndexInMetaState(masterNode, "blue_index");
// not the index state for blue_index should only be written on blue_node and the for red_index only on red_node
// we restart red node and master but with empty data folders
stopNode(redNode);
((InternalTestCluster) cluster()).stopCurrentMasterNode();
masterNode = startMasterNode();
redNode = startDataNode("red");
ensureGreen();
assertIndexNotInMetaState(blueNode, "red_index");
assertIndexInMetaState(blueNode, "blue_index");
assertIndexNotInMetaState(redNode, "red_index");
assertIndexNotInMetaState(redNode, "blue_index");
assertIndexNotInMetaState(masterNode, "red_index");
assertIndexInMetaState(masterNode, "blue_index");
// check that blue index is still there
assertFalse(client().admin().indices().prepareExists("red_index").get().isExists());
assertTrue(client().prepareGet("blue_index", "doc", "1").get().isExists());
// red index should be gone
// if the blue node had stored the index state then cluster health would be red and red_index would exist
assertFalse(client().admin().indices().prepareExists("red_index").get().isExists());
assertIndexInMetaState(dataNode, "test");
assertIndexInMetaState(masterNode, "test");
}
@Test
public void testMetaIsRemovedIfAllShardsFromIndexRemoved() throws Exception {
// this test checks that the index state is removed from a data only node once all shards have been allocated away from it
String masterNode = startMasterNode();
String blueNode = startDataNode("blue");
String redNode = startDataNode("red");
// create blue_index on blue_node and same for red
client().admin().cluster().health(clusterHealthRequest().waitForYellowStatus().waitForNodes("3")).get();
assertAcked(prepareCreate("blue_index").setSettings(Settings.builder().put("index.number_of_replicas", 0).put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "color", "blue")));
index("blue_index", "doc", "1", jsonBuilder().startObject().field("text", "some text").endObject());
assertAcked(prepareCreate("red_index").setSettings(Settings.builder().put("index.number_of_replicas", 0).put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "color", "red")));
index("red_index", "doc", "1", jsonBuilder().startObject().field("text", "some text").endObject());
String masterNode = internalCluster().startMasterOnlyNode(Settings.EMPTY);
Future<String> nodeName1 = internalCluster().startDataOnlyNodeAsync();
Future<String> nodeName2 = internalCluster().startDataOnlyNodeAsync();
String node1 = nodeName1.get();
String node2 = nodeName2.get();
String index = "index";
assertAcked(prepareCreate(index).setSettings(Settings.builder().put("index.number_of_replicas", 0).put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "_name", node1)));
index(index, "doc", "1", jsonBuilder().startObject().field("text", "some text").endObject());
ensureGreen();
assertIndexNotInMetaState(redNode, "blue_index");
assertIndexNotInMetaState(blueNode, "red_index");
assertIndexInMetaState(redNode, "red_index");
assertIndexInMetaState(blueNode, "blue_index");
assertIndexInMetaState(masterNode, "red_index");
assertIndexInMetaState(masterNode, "blue_index");
assertIndexInMetaState(node1, index);
assertIndexNotInMetaState(node2, index);
assertIndexInMetaState(masterNode, index);
// now relocate blue_index to red_node and red_index to blue_node
logger.debug("relocating indices...");
client().admin().indices().prepareUpdateSettings("blue_index").setSettings(Settings.builder().put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "color", "red")).get();
client().admin().indices().prepareUpdateSettings("red_index").setSettings(Settings.builder().put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "color", "blue")).get();
logger.debug("relocating index...");
client().admin().indices().prepareUpdateSettings(index).setSettings(Settings.builder().put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "_name", node2)).get();
client().admin().cluster().prepareHealth().setWaitForRelocatingShards(0).get();
ensureGreen();
assertIndexNotInMetaState(redNode, "red_index");
assertIndexNotInMetaState(blueNode, "blue_index");
assertIndexInMetaState(redNode, "blue_index");
assertIndexInMetaState(blueNode, "red_index");
assertIndexInMetaState(masterNode, "red_index");
assertIndexInMetaState(masterNode, "blue_index");
//at this point the blue_index is on red node and the red_index on blue node
// now, when we start red and master node again but without data folder, the red index should be gone but the blue index should initialize fine
stopNode(redNode);
((InternalTestCluster) cluster()).stopCurrentMasterNode();
masterNode = startMasterNode();
redNode = startDataNode("red");
ensureGreen();
assertIndexNotInMetaState(redNode, "blue_index");
assertIndexNotInMetaState(blueNode, "blue_index");
assertIndexNotInMetaState(redNode, "red_index");
assertIndexInMetaState(blueNode, "red_index");
assertIndexInMetaState(masterNode, "red_index");
assertIndexNotInMetaState(masterNode, "blue_index");
assertTrue(client().prepareGet("red_index", "doc", "1").get().isExists());
// if the red_node had stored the index state then cluster health would be red and blue_index would exist
assertFalse(client().admin().indices().prepareExists("blue_index").get().isExists());
}
@Test
public void testMetaWrittenWhenIndexIsClosed() throws Exception {
String masterNode = startMasterNode();
String redNodeDataPath = createTempDir().toString();
String redNode = startDataNode("red", redNodeDataPath);
String blueNode = startDataNode("blue");
// create red_index on red_node and same for red
client().admin().cluster().health(clusterHealthRequest().waitForYellowStatus().waitForNodes("3")).get();
assertAcked(prepareCreate("red_index").setSettings(Settings.builder().put("index.number_of_replicas", 0).put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "color", "red")));
index("red_index", "doc", "1", jsonBuilder().startObject().field("text", "some text").endObject());
ensureGreen();
assertIndexNotInMetaState(blueNode, "red_index");
assertIndexInMetaState(redNode, "red_index");
assertIndexInMetaState(masterNode, "red_index");
client().admin().indices().prepareClose("red_index").get();
// close the index
ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().get();
assertThat(clusterStateResponse.getState().getMetaData().index("red_index").getState().name(), equalTo(IndexMetaData.State.CLOSE.name()));
// restart master with empty data folder and maybe red node
boolean restartRedNode = randomBoolean();
//at this point the red_index on red node
if (restartRedNode) {
stopNode(redNode);
}
((InternalTestCluster) cluster()).stopCurrentMasterNode();
masterNode = startMasterNode();
if (restartRedNode) {
redNode = startDataNode("red", redNodeDataPath);
}
ensureGreen("red_index");
assertIndexNotInMetaState(blueNode, "red_index");
assertIndexInMetaState(redNode, "red_index");
assertIndexInMetaState(masterNode, "red_index");
clusterStateResponse = client().admin().cluster().prepareState().get();
assertThat(clusterStateResponse.getState().getMetaData().index("red_index").getState().name(), equalTo(IndexMetaData.State.CLOSE.name()));
// open the index again
client().admin().indices().prepareOpen("red_index").get();
clusterStateResponse = client().admin().cluster().prepareState().get();
assertThat(clusterStateResponse.getState().getMetaData().index("red_index").getState().name(), equalTo(IndexMetaData.State.OPEN.name()));
// restart again
ensureGreen();
if (restartRedNode) {
stopNode(redNode);
}
((InternalTestCluster) cluster()).stopCurrentMasterNode();
masterNode = startMasterNode();
if (restartRedNode) {
redNode = startDataNode("red", redNodeDataPath);
}
ensureGreen("red_index");
assertIndexNotInMetaState(blueNode, "red_index");
assertIndexInMetaState(redNode, "red_index");
assertIndexInMetaState(masterNode, "red_index");
clusterStateResponse = client().admin().cluster().prepareState().get();
assertThat(clusterStateResponse.getState().getMetaData().index("red_index").getState().name(), equalTo(IndexMetaData.State.OPEN.name()));
assertTrue(client().prepareGet("red_index", "doc", "1").get().isExists());
assertIndexNotInMetaState(node1, index);
assertIndexInMetaState(node2, index);
assertIndexInMetaState(masterNode, index);
}
@Test
public void testMetaWrittenWhenIndexIsClosedAndMetaUpdated() throws Exception {
String masterNode = startMasterNode();
String redNodeDataPath = createTempDir().toString();
String redNode = startDataNode("red", redNodeDataPath);
// create red_index on red_node and same for red
client().admin().cluster().health(clusterHealthRequest().waitForYellowStatus().waitForNodes("2")).get();
assertAcked(prepareCreate("red_index").setSettings(Settings.builder().put("index.number_of_replicas", 0).put(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "color", "red")));
index("red_index", "doc", "1", jsonBuilder().startObject().field("text", "some text").endObject());
String masterNode = internalCluster().startMasterOnlyNode(Settings.EMPTY);
final String dataNode = internalCluster().startDataOnlyNode(Settings.EMPTY);
logger.info("--> wait for green red_index");
final String index = "index";
assertAcked(prepareCreate(index).setSettings(Settings.builder().put("index.number_of_replicas", 0)));
logger.info("--> wait for green index");
ensureGreen();
logger.info("--> wait for meta state written for red_index");
assertIndexInMetaState(redNode, "red_index");
assertIndexInMetaState(masterNode, "red_index");
logger.info("--> wait for meta state written for index");
assertIndexInMetaState(dataNode, index);
assertIndexInMetaState(masterNode, index);
logger.info("--> close red_index");
client().admin().indices().prepareClose("red_index").get();
logger.info("--> close index");
client().admin().indices().prepareClose(index).get();
// close the index
ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().get();
assertThat(clusterStateResponse.getState().getMetaData().index("red_index").getState().name(), equalTo(IndexMetaData.State.CLOSE.name()));
assertThat(clusterStateResponse.getState().getMetaData().index(index).getState().name(), equalTo(IndexMetaData.State.CLOSE.name()));
logger.info("--> restart red node");
stopNode(redNode);
redNode = startDataNode("red", redNodeDataPath);
client().admin().indices().preparePutMapping("red_index").setType("doc").setSource(jsonBuilder().startObject()
// update the mapping. this should cause the new meta data to be written although index is closed
client().admin().indices().preparePutMapping(index).setType("doc").setSource(jsonBuilder().startObject()
.startObject("properties")
.startObject("integer_field")
.field("type", "integer")
@ -261,45 +116,43 @@ public class MetaDataWriteDataNodesTests extends ElasticsearchIntegrationTest {
.endObject()
.endObject()).get();
GetMappingsResponse getMappingsResponse = client().admin().indices().prepareGetMappings("red_index").addTypes("doc").get();
assertNotNull(((LinkedHashMap) (getMappingsResponse.getMappings().get("red_index").get("doc").getSourceAsMap().get("properties"))).get("integer_field"));
// restart master with empty data folder and maybe red node
((InternalTestCluster) cluster()).stopCurrentMasterNode();
masterNode = startMasterNode();
GetMappingsResponse getMappingsResponse = client().admin().indices().prepareGetMappings(index).addTypes("doc").get();
assertNotNull(((LinkedHashMap) (getMappingsResponse.getMappings().get(index).get("doc").getSourceAsMap().get("properties"))).get("integer_field"));
ensureGreen("red_index");
assertIndexInMetaState(redNode, "red_index");
assertIndexInMetaState(masterNode, "red_index");
clusterStateResponse = client().admin().cluster().prepareState().get();
assertThat(clusterStateResponse.getState().getMetaData().index("red_index").getState().name(), equalTo(IndexMetaData.State.CLOSE.name()));
getMappingsResponse = client().admin().indices().prepareGetMappings("red_index").addTypes("doc").get();
assertNotNull(((LinkedHashMap) (getMappingsResponse.getMappings().get("red_index").get("doc").getSourceAsMap().get("properties"))).get("integer_field"));
// make sure it was also written on red node although index is closed
ImmutableOpenMap<String, IndexMetaData> indicesMetaData = getIndicesMetaDataOnNode(dataNode);
assertNotNull(((LinkedHashMap) (indicesMetaData.get(index).getMappings().get("doc").getSourceAsMap().get("properties"))).get("integer_field"));
assertThat(indicesMetaData.get(index).state(), equalTo(IndexMetaData.State.CLOSE));
}
/* Try the same and see if this also works if node was just restarted.
* Each node holds an array of indices it knows of and checks if it should
* write new meta data by looking up in this array. We need it because if an
* index is closed it will not appear in the shard routing and we therefore
* need to keep track of what we wrote before. However, when the node is
* restarted this array is empty and we have to fill it before we decide
* what we write. This is why we explicitly test for it.
*/
internalCluster().restartNode(dataNode, new RestartCallback());
client().admin().indices().preparePutMapping(index).setType("doc").setSource(jsonBuilder().startObject()
.startObject("properties")
.startObject("float_field")
.field("type", "float")
.endObject()
.endObject()
.endObject()).get();
private String startDataNode(String color) {
return startDataNode(color, createTempDir().toString());
}
getMappingsResponse = client().admin().indices().prepareGetMappings(index).addTypes("doc").get();
assertNotNull(((LinkedHashMap) (getMappingsResponse.getMappings().get(index).get("doc").getSourceAsMap().get("properties"))).get("float_field"));
private String startDataNode(String color, String newDataPath) {
Settings.Builder settingsBuilder = Settings.builder()
.put("node.data", true)
.put("node.master", false)
.put("node.color", color)
.put("path.data", newDataPath);
return internalCluster().startNode(settingsBuilder.build());
}
// make sure it was also written on red node although index is closed
indicesMetaData = getIndicesMetaDataOnNode(dataNode);
assertNotNull(((LinkedHashMap) (indicesMetaData.get(index).getMappings().get("doc").getSourceAsMap().get("properties"))).get("float_field"));
assertThat(indicesMetaData.get(index).state(), equalTo(IndexMetaData.State.CLOSE));
private String startMasterNode() {
Settings.Builder settingsBuilder = Settings.builder()
.put("node.data", false)
.put("node.master", true)
.put("path.data", createTempDir().toString());
return internalCluster().startNode(settingsBuilder.build());
}
private void stopNode(String name) throws IOException {
internalCluster().stopRandomNode(InternalTestCluster.nameFilter(name));
// finally check that meta data is also written of index opened again
assertAcked(client().admin().indices().prepareOpen(index).get());
indicesMetaData = getIndicesMetaDataOnNode(dataNode);
assertThat(indicesMetaData.get(index).state(), equalTo(IndexMetaData.State.OPEN));
}
protected void assertIndexNotInMetaState(String nodeName, String indexName) throws Exception {
@ -335,14 +188,18 @@ public class MetaDataWriteDataNodesTests extends ElasticsearchIntegrationTest {
}
private boolean metaStateExists(String nodeName, String indexName) throws Exception {
GatewayMetaState nodeMetaState = ((InternalTestCluster) cluster()).getInstance(GatewayMetaState.class, nodeName);
MetaData nodeMetaData = null;
nodeMetaData = nodeMetaState.loadMetaState();
ImmutableOpenMap<String, IndexMetaData> indices = nodeMetaData.getIndices();
ImmutableOpenMap<String, IndexMetaData> indices = getIndicesMetaDataOnNode(nodeName);
boolean inMetaSate = false;
for (ObjectObjectCursor<String, IndexMetaData> index : indices) {
inMetaSate = inMetaSate || index.key.equals(indexName);
}
return inMetaSate;
}
private ImmutableOpenMap<String, IndexMetaData> getIndicesMetaDataOnNode(String nodeName) throws Exception {
GatewayMetaState nodeMetaState = ((InternalTestCluster) cluster()).getInstance(GatewayMetaState.class, nodeName);
MetaData nodeMetaData = null;
nodeMetaData = nodeMetaState.loadMetaState();
return nodeMetaData.getIndices();
}
}

View File

@ -1284,6 +1284,18 @@ public final class InternalTestCluster extends TestCluster {
}
}
/**
* Restarts a node and calls the callback during restart.
*/
public void restartNode(String nodeName, RestartCallback callback) throws Exception {
ensureOpen();
NodeAndClient nodeAndClient = nodes.get(nodeName);
if (nodeAndClient != null) {
logger.info("Restarting node [{}] ", nodeAndClient.name);
nodeAndClient.restart(callback);
}
}
private void restartAllNodes(boolean rollingRestart, RestartCallback callback) throws Exception {
ensureOpen();
List<NodeAndClient> toRemove = new ArrayList<>();
@ -1341,7 +1353,7 @@ public final class InternalTestCluster extends TestCluster {
}
private static final RestartCallback EMPTY_CALLBACK = new RestartCallback() {
public static final RestartCallback EMPTY_CALLBACK = new RestartCallback() {
@Override
public Settings onNodeStopped(String node) {
return null;
@ -1466,6 +1478,52 @@ public final class InternalTestCluster extends TestCluster {
return buildNode.name;
}
public synchronized ListenableFuture<List<String>> startMasterOnlyNodesAsync(int numNodes) {
return startMasterOnlyNodesAsync(numNodes, Settings.EMPTY);
}
public synchronized ListenableFuture<List<String>> startMasterOnlyNodesAsync(int numNodes, Settings settings) {
Settings settings1 = Settings.builder().put(settings).put("node.master", true).put("node.data", false).build();
return startNodesAsync(numNodes, settings1, Version.CURRENT);
}
public synchronized ListenableFuture<List<String>> startDataOnlyNodesAsync(int numNodes) {
return startDataOnlyNodesAsync(numNodes, Settings.EMPTY);
}
public synchronized ListenableFuture<List<String>> startDataOnlyNodesAsync(int numNodes, Settings settings) {
Settings settings1 = Settings.builder().put(settings).put("node.master", false).put("node.data", true).build();
return startNodesAsync(numNodes, settings1, Version.CURRENT);
}
public synchronized ListenableFuture<String> startMasterOnlyNodeAsync() {
return startMasterOnlyNodeAsync(Settings.EMPTY);
}
public synchronized ListenableFuture<String> startMasterOnlyNodeAsync(Settings settings) {
Settings settings1 = Settings.builder().put(settings).put("node.master", true).put("node.data", false).build();
return startNodeAsync(settings1, Version.CURRENT);
}
public synchronized String startMasterOnlyNode(Settings settings) {
Settings settings1 = Settings.builder().put(settings).put("node.master", true).put("node.data", false).build();
return startNode(settings1, Version.CURRENT);
}
public synchronized ListenableFuture<String> startDataOnlyNodeAsync() {
return startDataOnlyNodeAsync(Settings.EMPTY);
}
public synchronized ListenableFuture<String> startDataOnlyNodeAsync(Settings settings) {
Settings settings1 = Settings.builder().put(settings).put("node.master", false).put("node.data", true).build();
return startNodeAsync(settings1, Version.CURRENT);
}
public synchronized String startDataOnlyNode(Settings settings) {
Settings settings1 = Settings.builder().put(settings).put("node.master", false).put("node.data", true).build();
return startNode(settings1, Version.CURRENT);
}
/**
* Starts a node in an async manner with the given settings and returns future with its name.
*/
@ -1724,7 +1782,7 @@ public final class InternalTestCluster extends TestCluster {
* and / or {@link #fullRestart(InternalTestCluster.RestartCallback)} to execute actions at certain
* stages of the restart.
*/
public static abstract class RestartCallback {
public static class RestartCallback {
/**
* Executed once the give node name has been stopped.