×
Community Blog Elasticsearch Distributed Consistency Principles Analysis (1) - Node

Elasticsearch Distributed Consistency Principles Analysis (1) - Node

The "Elasticsearch Distribution Consistency Principle Analysis" article series describes the implementation method, principles, and existing problems of consistency models based on Elasticsearch v6.

By Alibaba Cloud Table Store Development Team

Elasticsearch (ES) is the most common open-source distributed search engine. It's based on Lucene, an information-retrieval library, and provides powerful search and query capabilities. To learn its search principles, you must understand Lucene. To learn the ES architecture, you must know how to implement the distributed system. Consistency is at the core of the distributed system.

This article describes the ES cluster composition, node discovery, master election, error detection, and scaling. In terms of node discovery and master election, ES uses its own implementation instead of external components such as ZooKeeper. We will describe how this mechanism works, and the problems with it. This article covers:

  1. ES cluster composition
  2. Node discovery
  3. Master election
  4. Error detection
  5. Cluster scaling
  6. Comparison with the implementation method, such as Zookeeper and raft
  7. Summary

ES Cluster Composition

First, an Elasticsearch cluster (ES cluster) is composed of multiple nodes, which have different types. Through the configuration below, four types of nodes can be generated:

conf/elasticsearch.yml:
    node.master: true/false
    node.data: true/false

The four types of nodes are combinations of the true/false node.master and node.data. Other types of nodes, such as IngestNode used for data pre-processing, are not within the scope of this document.

When node.master is true, the node is a master node candidate and can participate in the election. It is often referred to as a master-eligible node in ES documentation, which is similar to MasterCandidate. The ES can only have one master (that is, leader) during normal operation, as having more than one master would cause a split-brain.

When node.data is true, the node acts as a data node, stores the shard data assigned to the node, and is responsible for the write and query of the shard data.

In addition, a node in any cluster can perform any request. The cluster forwards the request to the corresponding node for processing. For example, when node.master and node.data are both false, this node acts as a proxy-like node, accepts requests and forwards aggregated results.

1

The figure above is a diagram of an ES cluster, where Node_A is the master of the current cluster, and Node_ B and Node_C are the master node candidates; Node_A and Node_ B are also DataNodes; in addition, Node_D is a simple DataNode; and Node_E is a proxy node.

Here are some questions to consider: how many master-eligible nodes should be configured for an ES cluster? When there are insufficient storage or computing resources for the cluster, and scaling is needed, what type should the added nodes be set to?

Node Discovery

After a node is started, it needs to be added to the cluster through node discovery. ZenDiscovery is an ES module providing functionality, like node discovery and master election, without having to rely on tools such as ZooKeeper. See the official documentation:

https://www.elastic.co/guide/en/elasticsearch/reference/6.8/modules-discovery-zen.html

In short, node discovery relies on the following configuration:

conf/elasticsearch.yml:
    discovery.zen.ping.unicast.hosts: [1.1.1.1, 1.1.1.2, 1.1.1.3]

This configuration creates an edge from each node to every other host. When all nodes in the cluster form a connectivity map, each node can see other nodes in the cluster, preventing silos.

The official recommendation is that the unicast hosts list is maintained as the master-eligible nodes list in a cluster. Therefore, it is recommended that the unicast hosts list is maintained as the master-eligible nodes list in a cluster.

Master Election

As mentioned above, there may be more than one master-eligible node in a cluster, and master election ensures that there is only one elected master node. If more than one node is elected master, a split-brain will occur, which can affect data consistency and lead to chaos in the cluster with varying unexpected results.

To avoid a split-brain, ES uses a common distributed system concept, ensuring that the elected master is recognized by the master-eligible node of the quorum, resulting in only one master. This quorum is configured as follows:

conf/elasticsearch.yml:
    discovery.zen.minimum_master_nodes: 2

This configuration is critical for the cluster.

1. Who initiated the master election and when did it start?

The master election is initiated by a master-eligible node when the following conditions are met:

  1. The current state of the master-eligible node is not master.
  2. The master-eligible node queries other known nodes in the cluster using ZenDiscovery's ping operation, and confirms that no nodes are connected to the master.
  3. There are currently more than minimum_master_nodes nodes (including this node) that are not connected to the master.

In short, when a node determines that the master-eligible nodes within the quorum, including itself, think that the cluster has no master, then master election can be initiated.

2. When master election is required, which node should be elected?

The first question is, which node should be elected? As shown in the following source code, the first MasterCandidate (that is, master-eligible node) after sorting is elected.

    public MasterCandidate electMaster(Collection<MasterCandidate> candidates) {
        assert hasEnoughCandidates(candidates);
        List<MasterCandidate> sortedCandidates = new ArrayList<>(candidates);
        sortedCandidates.sort(MasterCandidate::compare);
        return sortedCandidates.get(0);
    }

Then, how are they sorted?

public static int compare(MasterCandidate c1, MasterCandidate c2) {
    // we explicitly swap c1 and c2 here. The code expects "better" to be lower in a sorted
    // list, so if c2 has a higher cluster state version, it needs to come first.
    int ret = Long.compare(c2.clusterStateVersion, c1.clusterStateVersion);
    if (ret == 0) {
        ret = compareNodes(c1.getNode(), c2.getNode());
    }
    return ret;
}

As shown in the source code above, the clusterStateVersion of the nodes is compared, with higher clusterStateVersion taking priority. When nodes have the same clusterStateVersion, the program goes to compareNodes, in which the IDs of the nodes are compared (IDs are randomly generated when the node initially starts).

In summary:

  1. Higher clusterStateVersion takes priority. This ensures that the new master has the latest clusterState (that is, the meta of the cluster), avoiding loss of committed meta changes. When the master is elected, it is updated based on the clusterState of this version. ( One exception is when the cluster restarts, and none of the nodes have meta. In this case, a master needs to be elected first, then the master uses persistent data for meta recovery, and then performs meta synchronization).
  2. When nodes have the same clusterStateVersion, the node with a lower ID takes priority. That is, a node with a low ID tends to be selected. The ID is a random string generated when the node initially starts. This is designed to ensure the stability of the election results, avoiding election failure due to too many master candidates.

3. What is a successful election?

When a master-eligible node (Node_A) initiates an election, it chooses an approved master according to the sorting strategy above. The process varies depending on whether Node_A selects itself or Node_B as master.

Assuming Node_A selects Node_B as the master:

Node_A sends a join request to Node_ B, then:

  1. If Node_ B has become master, it adds Node_A to the cluster, and publishes the latest cluster_state, which contains the information for Node_A. It is similar to adding a new node under normal circumstances. Node_A completes the join when a new cluster_state is published for Node_A.
  2. If Node_ B is running for master, it will take this join as a vote. In this case, Node_A waits until timeout to see whether Node_ B becomes the master, or another node is elected as master.
  3. If Node_ B thinks it is not the master (at any time), it will reject this join. In this case, Node_A initiates the next election.

Assuming that Node_A selects itself as master:

Node_A waits for other nodes to join, that is, waits for votes from other nodes. When more than half of the votes are collected, it regards itself as master, changes the master node in the cluster_state to itself, and sends a message to the cluster.

For more information, see the following source code:

        if (transportService.getLocalNode().equals(masterNode)) {
            final int requiredJoins = Math.max(0, electMaster.minimumMasterNodes() - 1); // we count as one
            logger.debug("elected as master, waiting for incoming joins ([{}] needed)", requiredJoins);
            nodeJoinController.waitToBeElectedAsMaster(requiredJoins, masterElectionWaitForJoinsTimeout,
                    new NodeJoinController.ElectionCallback() {
                        @Override
                        public void onElectedAsMaster(ClusterState state) {
                            synchronized (stateMutex) {
                                joinThreadControl.markThreadAsDone(currentThread);
                            }
                        }

                        @Override
                        public void onFailure(Throwable t) {
                            logger.trace("failed while waiting for nodes to join, rejoining", t);
                            synchronized (stateMutex) {
                                joinThreadControl.markThreadAsDoneAndStartNew(currentThread);
                            }
                        }
                    }

            );
        } else {
            // process any incoming joins (they will fail because we are not the master)
            nodeJoinController.stopElectionContext(masterNode + " elected");

            // send join request
            final boolean success = joinElectedMaster(masterNode);

            synchronized (stateMutex) {
                if (success) {
                    DiscoveryNode currentMasterNode = this.clusterState().getNodes().getMasterNode();
                    if (currentMasterNode == null) {
                        // Post 1.3.0, the master should publish a new cluster state before acknowledging our join request. We now should have
                        // a valid master.
                        logger.debug("no master node is set, despite the join request completing. Retrying pings.") ;
                        joinThreadControl.markThreadAsDoneAndStartNew(currentThread);
                    } else if (currentMasterNode.equals(masterNode) == false) {
                        // update cluster state
                        joinThreadControl.stopRunningThreadAndRejoin("master_switched_while_finalizing_join");
                    }

                    joinThreadControl.markThreadAsDone(currentThread);
                } else {
                    // failed to join. Try again...
                    joinThreadControl.markThreadAsDoneAndStartNew(currentThread);
                }
            }
        }

Following the process above, here is a simple scenario to make it clearer:

Assuming that a cluster has 3 master-eligible nodes, Node_A, Node_ B, and Node_C, and the election priority order is Node_A, Node_ B, Node_C. Each of the three nodes determines that there is no current master. Each node initiates an election, and based on the priority order, all nodes elect Node_A. So Node_A waits for joins. Node_B and Node_C send join requests to Node_A. When Node_A receives the first join request, along with its own vote, it has two votes in total (more than half), and becomes master. At this point, the cluster_state contains two nodes. When Node_A receives a join request from the remaining node, the cluster_state contains all three nodes.

4. How can the election avoid split-brain?

The basic principle lies in the quorum strategy. If only the node approved through quorum becomes master, it is impossible for two nodes to be approved by the quorum.

In the process above, the master candidate needs to wait for nodes that submitted approval in the quorum to join before becoming master. This ensures that this node was approved by the quorum. While the process above looks reasonable and works well in most scenarios, there is a problem.

This process has no restriction on how many times a node can vote in the election process. Under what circumstances would a node be allowed to vote twice? For example, Node_B votes for Node_A once, but Node_A hasn't become master after a certain period of time. Node_ B can't wait, and initiates the next election. At this point, it determines that the cluster contains Node_0, which has a higher priority than Node_A, so Node_B votes for Node_0. Assuming that both Node_0 and Node_A are waiting for votes, then Node_B has voted twice, each time for different candidates.

How can we solve this problem? For example, the Raft algorithm introduces the concept of election term, ensuring that each node can vote only once during each election term. Additional votes would be counted in term+1. If both the last two nodes think they are the master, one term must be greater than the other. Because quorum votes are received for both terms, the quorum node has a greater term, ensuring that the node with the smaller term cannot commit any status changes (commits require the quorum node for successful log persistence, and quorum persistence conditions cannot be met due to the term check). This ensures that status changes within the cluster are always consistent.

ES (v6.2) has not solved this problem yet. In test cases in similar scenarios, sometimes two masters are elected, and both nodes consider themselves master and publish a status change to the cluster. Publishing includes two phases. First, it ensures that the quorum node "accepts" this change, then all nodes are required to commit this change. Unfortunately, the two masters may both complete the first phase, and enter the commit phase. This causes inter-node status inconsistency, which isn't an issue in Raft. How can both masters complete the first phase? Because in the first phase, ES puts the new cluster_state into the memory queue after a simple check. If the master of the current cluster_state is empty, it will not be checked. In other words, after accepting the cluster_state where Node_A becomes master (before committing), Node_B also can be accepted as master in the cluster_state. This allows both Node_A and Node_B to meet the commit condition and initiate the commit command, which leads to inconsistent cluster status. Of course, split-brain situations like this will be automatically recovered quickly, because when a master publishes cluster_state again after the inconsistency occurs, the quorum condition will no longer be met, or it is automatically downgraded to a candidate because its followers no longer constitute quorum.

When compared with mature consistency solutions, ES's ZenDiscovery modules have issues handling some specific scenarios. We will analyze other scenarios where ES consistency has issues in the following description of the meta change process.

Error Detection

1. MasterFaultDetection and NodesFaultDetection

The fault detection can be described as a heartbeat-like mechanism. There are two types of fault detection, one is for the master to regularly detect the other nodes in the cluster, and the other is for the other nodes in the cluster to regularly detect the cluster's current master. The detection method performs regular ping requests.

According to the ES documentation:

There are two fault detection processes running. The first is by the master, to ping all the other nodes in the cluster and verify that they are alive. And on the other end, each node pings to master to verify if its still alive or an election process needs to be initiated.

If the master detects that a node is not connected, the removeNode operation is performed to remove the node from the cluster_state, and a new cluster_state is published. When a new cluster_state is applied to each module, a number of recovery operations are performed, for example, to select a new primaryShard or replica, or to perform data replication.

If a node detects that the master is not connected, the pending cluster_state which has not yet been committed to memory is cleared, and a rejoin is initiated to rejoin the cluster (a new master election is triggered if the election conditions are met).

2. Rejoin

In addition to the two cases above, there is another case, in which the master finds that it doesn't meet the quorum condition (>=minimumMasterNodes), and needs to actively exit the master status (and perform rejoin) to avoid split-brain. So, how does the master find out that it needs to rejoin?

As mentioned above, removeNode is executed when a node isn't connected. When executing removeNode, it checks whether the remaining nodes meet the quorum condition. If they don't, rejoin is performed.

            if (electMasterService.hasEnoughMasterNodes(remainingNodesClusterState.nodes()) == false) {
                final int masterNodes = electMasterService.countMasterNodes(remainingNodesClusterState.nodes());
                rejoin.accept(LoggerMessageFormat.format("not enough master nodes (has [{}], but needed [{}])",
                                                         masterNodes, electMasterService.minimumMasterNodes()));
                return resultBuilder.build(currentState);
            } else {
                return resultBuilder.build(allocationService.deassociateDeadNodes(remainingNodesClusterState, true, describeTasks(tasks)));
            }

Publishing the new cluster_state is divided into the send phase and the commit phase. The send phase requires the quorum to succeed before commit. If a successful quorum return is not achieved in the send phase, there may be a new master or the quorum node isn't connected, in which case the master needs to perform a rejoin.

        try {
            publishClusterState.publish(clusterChangedEvent, electMaster.minimumMasterNodes(), ackListener);
        } catch (FailedToCommitClusterStateException t) {
            // cluster service logs a WARN message
            logger.debug("failed to publish cluster state version [{}](not enough nodes acknowledged, min master nodes [{}])",
                newState.version(), electMaster.minimumMasterNodes());

            synchronized (stateMutex) {
                pendingStatesQueue.failAllStatesAndClear(
                    new ElasticsearchException("failed to publish cluster state"));

                rejoin("zen-disco-failed-to-publish");
            }
            throw t;
        }

During periodic pings to other nodes, it is discovered that another node is also master. In this case, the cluster_state version of this node is compared with the other master node. The node with the latest version becomes master, and the node with an earlier version performs rejoin.

        if (otherClusterStateVersion > localClusterState.version()) {
            rejoin("zen-disco-discovered another master with a new cluster_state [" + otherMaster + "][" + reason + "]");
        } else {
            // TODO: do this outside mutex
            logger.warn("discovered [{}] which is also master but with an older cluster_state, telling [{}] to rejoin the cluster ([{}])", otherMaster, otherMaster, reason);
            try {
                // make sure we're connected to this node (connect to node does nothing if we're already connected)
                // since the network connections are asymmetric, it may be that we received a state but have disconnected from the node
                // in the past (after a master failure, for example)
                transportService.connectToNode(otherMaster);
                transportService.sendRequest(otherMaster, DISCOVERY_REJOIN_ACTION_NAME, new RejoinClusterRequest(localClusterState.nodes().getLocalNodeId()), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {

                    @Override
                    public void handleException(TransportException exp) {
                        logger.warn((Supplier<? >) () -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), exp);
                    }
                });
            } catch (Exception e) {
                logger.warn((Supplier<? >) () -> new ParameterizedMessage("failed to send rejoin request to [{}]", otherMaster), e);
            }
        }

Cluster Scaling

With mechanisms such as node discovery, master election, and fault detection described above, we can now look at how to scale the cluster.

1. Scale Up DataNode

Assuming that an ES cluster does not have sufficient storage or computing resources, we need to scale the capacity. Here we focus on DataNode, which is configured as follows:

conf/elasticsearch.yml:
    node.master: false
    node.data: true

Then we need other configurations, such as cluster name and node name. To add the node to the cluster, we configure discovery.zen.ping.unicast.hosts as a master-eligible node in the cluster.

conf/elasticsearch.yml:
    cluster.name: es-cluster
    node.name: node_Z
    discovery.zen.ping.unicast.hosts: ["x.x.x.x", "x.x.x.y", "x.x.x.z"]

Then we start the node, and the node is automatically added to the cluster. The cluster rebalances automatically or manually through the reroute API.

https://www.elastic.co/guide/en/elasticsearch/reference/current/cluster-reroute.html

https://www.elastic.co/guide/en/elasticsearch/reference/current/shards-allocation.html

2. Scale Down DataNode

Assuming that an ES cluster uses too many machines and needs to scale down the capacity, how do we safely perform this operation to ensure data security without compromising availability?

First, we select the nodes that need to be scaled down. This section relates to scaling down DataNode. Scaling down MasterNode is a more complex process, which will be described later.

Then, we migrate shards on this node to another node. We set the allocation rules to prevent shards from being allocated to machines selected to be scaled down, and then rebalance the cluster.

PUT _cluster/settings
{
  "transient" : {
    "cluster.routing.allocation.exclude._ip" : "10.0.0.1"
  }
}

After all the data on this node has been migrated, the node can be securely released.

For more detailed operations, see the official documentation:

https://www.elastic.co/guide/en/elasticsearch/reference/current/allocation-filtering.html

3. Scaling up MasterNode

To scale up a MasterNode (master-eligible node), we must consider the ES quorum strategy to avoid split-brain, as mentioned above. We must, therefore, configure a quorum number:

conf/elasticsearch.yml:
    discovery.zen.minimum_master_nodes: 2

Assuming there are already three master-eligible nodes, we can configure quorum to 2; if scaling up to four master-eligible nodes, the quorum should be increased to 3.

First, we must change the configuration of discovery.zen.minimum_master_nodes to 3, then scale up the master. Do this using the following API:

curl -XPUT localhost:9200/_cluster/settings -d '{
    "persistent" : {
        "discovery.zen.minimum_master_nodes" : 3
    }
}'

Once this API request is sent to the master of the current cluster, the new value takes effect immediately. The master maintains this configuration in the cluster meta, and all future nodes are based on this configuration.

This approach, however, can result in consistency issues between the configuration file values and those in the cluster meta, which may result in some unusual problems. For example, after the cluster reboots, master election is required before the cluster meta is restored. At this point, only the configuration values can be used rather than the values in the cluster meta. After the cluster meta is recovered, election is based on the cluster meta values, and some integrity-related boundary cases may arise.

In summary, any operations or configurations regarding the master node must be carefully thought out, as master configuration errors can lead to split-brain, bad data writes, data loss, and other unwanted situations.

4. Scaling down MasterNode

Scaling down MasterNode follows the opposite process of scaling up. First, we scale down the node then reduce the number of the quorum. We do not provide further details here.

Comparison of Implementation Methods

1. Comparison with ZooKeeper

This section describes several methods of implementing major node-related functions in an ES cluster:

  1. Node discovery
  2. Master election
  3. Error detection
  4. Cluster scaling

Imagine what would be different if we use ZooKeeper to implement that functionality.

About ZooKeeper

Let us begin with a brief introduction to ZooKeeper. If you are already familiar with it, you can skip this part.

The ZooKeeper distributed service framework is a subproject of Apache Hadoop. It is mainly used to solve some common data management problems that are frequently encountered in distributed applications, such as unified naming service, state synchronization service, cluster management and distributed application configuration item management.

In short, ZooKeeper is used to manage the nodes, configurations, and states in the distributed system and complete the configurations and state synchronization among individual nodes. Many distributed systems rely on ZooKeeper or similar components.

ZooKeeper manages data in the form of a directory tree; each node is referred to as a znode, and each znode consists of three parts:

  1. This is the state information that describes the znode version, permissions, and other information.
  2. The date associated with the znode.
  3. The child nodes under the znode.

One of the items in stat is ephemeralOwner; if it has a value, it represents a temporary node. This temporary node is deleted after the session ends, and it can be used to assist the application in master election and error detection.

ZooKeeper provides watch functionality that can be used to listen to corresponding events, such as the increase/decrease of a child node under a znode, the increase/decrease of a znode, and the update of a znode.

Implementing the ES functionality Above Using ZooKeeper

  1. Node discovery: Configure the ZooKeeper server address in the configuration file of each node. Once the node starts, it tries to register a temporary znode in a ZooKeeper directory. The master of the current cluster listens to increase/decrease child node events in this directory. Whenever a new node is discovered, it adds the new node to the cluster.
  2. Master election: When a master-eligible node starts, it tries to register a temporary znode named master in a fixed location. If the registration succeeds, it becomes master; if the registration fails, it listens to changes to this znode. When the master fails, it is automatically deleted because it is a temporary znode; meanwhile, the other master-eligible nodes try to register again. When you use ZooKeeper, you turn the master election into the master.
  3. Error detection: Because the znode of the node and the znode of the master are both temporary znodes, if the node fails, the session disconnects from ZooKeeper and the znode is automatically deleted. The master of the cluster only needs to listen to znode change events. If the master fails, other candidate masters listen to the master znode deletion event and try to become the new master.
  4. Cluster scaling: The minimum_master_nodes configuration no longer matters when scaling the cluster, which makes scaling easier.

Advantages and Disadvantages of using ZooKeeper

ZooKeeper handles some complex distributed consistency issues, simplifying ES operation substantially and helping guarantee data integrity. This is also the common implementation for most distributed systems. While the ES' Zen Discovery module has undergone many bug fixes, there remain critical bugs, and operation and maintenance is difficult.

So, why doesn't ES use ZooKeeper? Perhaps the official developers believe that adding ZooKeeper dependency means relying on one more component, adding complexity to cluster deployment and forcing users to manage one more service during regular operation and maintenance.

Are there any other algorithms available for self-implementation? Of course, there is raft, for example.

2. Comparison with Raft

The raft algorithm is a very popular distributed consensus algorithm. It is easier to implement than paxos, and it has been used in a wide variety of distributed systems. Instead of describing the details of this algorithm here, we focus on the master election algorithm to compare the similarities and differences between raft and the ES' current election algorithm:

Similarities

  1. Quorum principle: Only the node that gets more than half of the votes can become master.
  2. The selected leader must have the latest submitted data: In raft, the nodes with newer data do not vote for nodes with older data, and because getting elected requires a majority of votes, the leader-elect must have the latest submitted data. In ES, the sort priority is higher for nodes with up-to-date versions to ensure this as well.

Differences

  1. Proof of correctness: Raft is an algorithm whose correctness has been proved. The correctness of the ES' algorithm is unproven, and any issues will only be found in practice, at which point bugs can be fixed. This is the major difference.
  2. Election Cycle term: Raft introduces the concept of Election Cycle. The term plus one for each election round ensures that, within the same term, each participant can only have one vote. ES does not have a term concept during election and is unable to guarantee that each node can only have one vote every round.
  3. Election tendency: In raft, if a node has the latest submitted data, there is an opportunity for it to be elected master. In ES, nodes with the same version are sorted by NodeId, and nodes with a lower NodeId always take priority.

Recommendations

In terms of correctness, raft is definitely a better choice; however, after fixing several bugs, the ES' election algorithm is becoming more like raft. Of course, raft was not around during early ES development. If ES continues in this direction, it might eventually become as capable as a raft implementation.

Raft does not just handle election. We will continue to compare the similarities and differences between current ES implementation and raft in the next section when introducing meta data consistency.

Summary

This section introduces the implementations of composition, node discovery, master election, fault detection, scaling, and other aspects of the Elasticsearch cluster. Unlike more general articles, this one analyzes cluster principles and issues and compares them with other implementation methods.

As the first section of the Elasticsearch Distributed Consistency Principles Analysis series, this article begins with the node. In the next section, we discuss the consistency of meta data changes and analyze the distributed principles of ES in more depth.

To learn more about Elasticsearch on Alibaba Cloud, visit https://www.alibabacloud.com/product/elasticsearch

0 0 0
Share on

Whybert

3 posts | 0 followers

You may also like

Comments

Whybert

3 posts | 0 followers

Related Products