Internal: only ack index store deletion on data nodes
When an index is deleted we wait on all nodes to ack the delete. Data nodes are expected to both ack the remove of the index from their IndicesService and also the deletion of the store from disk. At the moment all nodes sends this ack which causes wrong counting on the master side. On top of this, we currently have an unneeded WARN message in the logs when client nodes try to acquire locks but do not have a data folder. Relates to #9605 Closes #9672
This commit is contained in:
parent
8dad914a1c
commit
51ba120b54
|
@ -33,11 +33,9 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
|||
import org.elasticsearch.env.NodeEnvironment;
|
||||
import org.elasticsearch.env.ShardLock;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.*;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
|
@ -87,6 +85,10 @@ public class NodeIndexDeletedAction extends AbstractComponent {
|
|||
@Override
|
||||
protected void doRun() throws Exception {
|
||||
innerNodeIndexDeleted(index, nodeId);
|
||||
if (nodes.localNode().isDataNode() == false) {
|
||||
logger.trace("[{}] not acking store deletion (not a data node)");
|
||||
return;
|
||||
}
|
||||
lockIndexAndAck(index, nodes, nodeId, clusterState);
|
||||
|
||||
}
|
||||
|
@ -94,6 +96,10 @@ public class NodeIndexDeletedAction extends AbstractComponent {
|
|||
} else {
|
||||
transportService.sendRequest(clusterState.nodes().masterNode(),
|
||||
INDEX_DELETED_ACTION_NAME, new NodeIndexDeletedMessage(index, nodeId), EmptyTransportResponseHandler.INSTANCE_SAME);
|
||||
if (nodes.localNode().isDataNode() == false) {
|
||||
logger.trace("[{}] not acking store deletion (not a data node)");
|
||||
return;
|
||||
}
|
||||
threadPool.generic().execute(new AbstractRunnable() {
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
|
|
Loading…
Reference in New Issue