×
Community Blog Elasticsearch Distributed Consistency Principles Analysis (3) - Data

Elasticsearch Distributed Consistency Principles Analysis (3) - Data

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

The previous two articles described the composition of the ES clusters, master election algorithm, master update meta process, and analyzed the consistency issues of the election and Meta update. This article analyzes the data flow in ES, including its write process, PacificA algorithm model, SequenceNumber, Checkpoint and compares the similarities and differences between ES implementation and the standard PacificA algorithm. We will be covering:

  1. Current issues
  2. Data write process
  3. PacificA algorithm
  4. SequenceNumber, Checkpoint, and failure recovery
  5. Comparing ES and PacificA
  6. Summary

Current Issues

Anyone who has ever used ES knows that each ES Index is divided into multiple Shards. Shards are distributed on different nodes to enable distributed storage and queries and support large-scale datasets. Each Shard has multiple copies, one of which is the Primary node, and the others are Replica nodes. Data is written to the Primary node first then synchronized with Replica nodes from the Primary node. When reading data, to improve read capability, both Primary node and Replica nodes accept read requests.

1

With this model, we can see that ES has some of the following characteristics:

  1. High data reliability: The data has multiple copies.
  2. High service availability: If the Primary node crashes, a new Primary node can be chosen from the Replica nodes to continue offering services.
  3. Extended read capability: The Primary node and Replica nodes can take read requests.
  4. Failure recovery capability: If the Primary node or Replica nodes crash, there are not enough copies. New copies can be generated by copying the data from the new Primary node.

Some questions may come to mind, for example:

  1. How is data copied from Primary node to Replica nodes?
  2. Does it need to write to all copies to be successful?
  3. Do Primary node crashes cause data loss?
  4. Is the latest data always read when reading from Replica nodes?
  5. Do I need to copy all Shard data when performing failure recovery?

As you can see, although we can easily understand the general principles of ES data consistency, many details remain unclear. This article focuses on the ES write process, the consistency algorithm used, SequenceId and Checkpoint design, and other aspects to describe how ES works and address the questions above. It is important to note that the analysis in this article is based on ES version 6.2. Much of the content does not apply to previous ES versions, such as version 2. X version.

Data Write Process

First, let us take a look at the data write process.

From the Replication Perspective: Primary -> Replica

From the macro perspective, the ES write process involves writing data to the Primary node first, then concurrently writing it to Replica nodes and finally returning it to the Client. The process is as follows:

Check the Active Shard count.

String activeShardCountFailure = checkActiveShardCount();

Write to the Primary.

primaryResult = primary.perform(request);

Concurrently initiate write requests to all Replicates.

performOnReplicas(replicaRequest, globalCheckpoint, replicationGroup.getRoutingTable());

After all Replicates are returned or fail, they are returned to the Client.

private void decPendingAndFinishIfNeeded() {
  assert pendingActions.get() > 0 : "pending action count goes below 0 for request [" + request + "]";
  if (pendingActions.decrementAndGet() == 0) {
      finish();
  }
}

The procedure above is an execute function of the ReplicationOperation class, and the complete code is as follows:

    public void execute() throws Exception {
        final String activeShardCountFailure = checkActiveShardCount();
        final ShardRouting primaryRouting = primary.routingEntry();
        final ShardId primaryId = primaryRouting.shardId();
        if (activeShardCountFailure ! = null) {
            finishAsFailed(new UnavailableShardsException(primaryId,
                "{} Timeout: [{}], request: [{}]", activeShardCountFailure, request.timeout(), request));
            return;
        }

        totalShards.incrementAndGet();
        pendingActions.incrementAndGet(); // increase by 1 until we finish all primary coordination
        primaryResult = primary.perform(request);
        primary.updateLocalCheckpointForShard(primaryRouting.allocationId().getId(), primary.localCheckpoint());
        final ReplicaRequest replicaRequest = primaryResult.replicaRequest();
        if (replicaRequest ! = null) {
            if (logger.isTraceEnabled()) {
                logger.trace("[{}] op [{}] completed on primary for request [{}]", primaryId, opType, request);
            }

            // We must obtain the replication group after successfully indexing into the primary to follow recovery semantics.
            // We must make sure that every operation indexed into the primary after recovery start is also replicated
            // to the recovery target. If we used an old replication group, we may miss a recovery that has started since then.
            // We also must make sure to obtain the global checkpoint before the replication group to ensure that the global checkpoint
            // is valid for this replication group. If we sampled in the reverse direction, the global checkpoint might be based on a subset
            // of the sampled replication group and advanced further than what the given replication group would allow.
            // This would mean that some shards could learn about a global checkpoint that would be higher than its local checkpoint.
            final long globalCheckpoint = primary.globalCheckpoint();
            final ReplicationGroup replicationGroup = primary.getReplicationGroup();
            markUnavailableShardsAsStale(replicaRequest, replicationGroup.getInSyncAllocationIds(), replicationGroup.getRoutingTable());
            performOnReplicas(replicaRequest, globalCheckpoint, replicationGroup.getRoutingTable());
        }

        successfulShards.incrementAndGet();  // mark primary as successful
        decPendingAndFinishIfNeeded();
    }

Next, we analyze some questions about this process:

1. Why must the Active Shard count be checked in the first step?

There is a parameter called wait_for_active_shards in ES. It is an Index setting and can be attached to the request. This parameter indicates the minimum numbers of Active copies that the Shard should have before each write operation. Assume that we have an Index in which each Shard has three Replica nodes, totaling four copies (plus Primary node). If wait_for_active_shards is configured to 3, a maximum of one Replica node is allowed to crash; if two Replica nodes crash, the number Active copies is less than three and, at that point, the write operation is not allowed.

This parameter is set to 1 by default, which means that the write operation is allowed if the Primary node exists, meaning this parameter is not use at this point. If it is set to a number greater than 1, it can have a protective effect, ensuring that the written data has higher reliability. However, this parameter only carries out the check before the write operation, which cannot guarantee that the data is written successfully to the copies; thus, the minimum number of copies to which the data is written is not strictly guaranteed.

2. After writing to the Primary node finishes, why is it not returned until all Replica nodes respond (or the connection fails)?

In earlier versions of ES, asynchronous replication was allowed between the Primary node and Replica nodes, which meant that the Primary node returned once writing was successful. But, in this mode, if the Primary node crashes, there is a risk of data loss, and it is difficult to guarantee that the data read from Replica nodes is up to date. So, ES stopped using asynchronous mode. Now, the Primary node is not returned to the Client until the Replica nodes are returned.

Because the Primary node is not returned to the Client until all Replica nodes are returned, the latency is affected by the slowest Replica node, which is a clear disadvantage of the current ES architecture. Originally, we thought that the result would be returned once writing to wait_for_active_shards copies was successful, but, later, after reading the source code, we realized that the result was not returned until all Replica nodes were returned.

If writing to a Replica node fails, ES executes retry logic; however, the number of nodes that need to be successfully written to is not explicitly specified. The returned result includes the number of Shards in which the data write was successful or failed:

{
    "_shards" : {
        "total" : 2,
        "failed" : 0,
        "successful" : 2
    }
}

3. If writing to a Replica node continuously fails, do user lookups see legacy data?

In other words, assuming writing to a Replica node continuously fails, the data in the Replica node could be much older than that in the Primary node. We know that, in ES, Replicas can also handle read requests, so does the user read the legacy data in this Replica node?

The answer is that, if writing to a Replica node fails, the Primary node reports the issue to the Master, and the Master then updates the InSyncAllocations configuration of the Index in Meta and removes the Replica node. After that, it no longer handles read requests. Users can still read the data on this Replica node before Meta update reaches every Node, but this does not happen after Meta update completes. This solution is not strict. Considering that ES is a near real-time system, after data is written, refresh is required for it to be visible. So, in general, it should be acceptable that legacy data can be read for a short time.

ReplicationOperation.java, OnFailure function for failure to write to Replica nodes:

            public void onFailure(Exception replicaException) {
                logger.trace(
                    (org.apache.logging.log4j.util.Supplier<? >) () -> new ParameterizedMessage(
                        "[{}] failure while performing [{}] on replica {}, request [{}]",
                        shard.shardId(),
                        opType,
                        shard,
                        replicaRequest),
                    replicaException);
                if (TransportActions.isShardNotAvailableException(replicaException)) {
                    decPendingAndFinishIfNeeded();
                } else {
                    RestStatus restStatus = ExceptionsHelper.status(replicaException);
                    shardReplicaFailures.add(new ReplicationResponse.ShardInfo.Failure(
                        shard.shardId(), shard.currentNodeId(), replicaException, restStatus, false));
                    String message = String.format(Locale.ROOT, "failed to perform %s on replica %s", opType, shard);
                    replicasProxy.failShardIfNeeded(shard, message,
                            replicaException, ReplicationOperation.this::decPendingAndFinishIfNeeded,
                            ReplicationOperation.this::onPrimaryDemoted, throwable -> decPendingAndFinishIfNeeded());
                }
            }

call failShardIfNeeded:

        public void failShardIfNeeded(ShardRouting replica, String message, Exception exception,
                                      Runnable onSuccess, Consumer<Exception> onPrimaryDemoted, Consumer<Exception> onIgnoredFailure) {

            logger.warn((org.apache.logging.log4j.util.Supplier<? >)
                    () -> new ParameterizedMessage("[{}] {}", replica.shardId(), message), exception);
            shardStateAction.remoteShardFailed(replica.shardId(), replica.allocationId().getId(), primaryTerm, message, exception,
                    createListener(onSuccess, onPrimaryDemoted, onIgnoredFailure));
        }

shardStateAction.remoteShardFailed sends the request to the Master, executes the ShardFailed logic of the Replica, and removes the Shard from InSyncAllocation.

    public void shardFailed(ShardRouting failedShard, UnassignedInfo unassignedInfo) {
        if (failedShard.active() && unassignedInfo.getReason() ! = UnassignedInfo.Reason.NODE_LEFT) {
            removeAllocationId(failedShard);

            if (failedShard.primary()) {
                Updates updates = changes(failedShard.shardId());
                if (updates.firstFailedPrimary == null) {
                    // more than one primary can be failed (because of batching, primary can be failed, replica promoted and then failed...)
                    updates.firstFailedPrimary = failedShard;
                }
            }
        }

        if (failedShard.active() && failedShard.primary()) {
            increasePrimaryTerm(failedShard.shardId());
        }
    }

Maintaining InSyncAllocation in ES uses the PacificA algorithm, which is detailed in the next section.

From the Perspective of the Primary

From the perspective of Primary, a write request is written to Lucene before it is written to translog.

1. Why is translog write required?

Translog is similar to commitlog in a database, or binlog. Once translog write is successful and flushed, the data is flushed directly to the disk, which guarantees data security, so that Segment can be flushed to the disk later. Because translog is written using append, write performance is better than using random write.

In addition, because translog records every data change and the order in which the data changes, it can be used for data recovery. Data recovery consists of two parts: First, after the node reboots, the Segment data that has not been flushed to the disk before reboot is recovered from translog; second, it is used for data synchronization between the Primary node and the new Replica node, which is the process by which the Replica tries to keep up with the Primary data.

2. Why is Lucene write required before translog write?

Lucene write writes the data to memory. After the write operation is finished, the data can be read immediately on refresh; translog write flushes data to the disk for data persistence and recovery. Normally, in distributed systems, commitLog is written for data persistence first, then this change is applied to the memory. So, why does ES work in exactly the opposite way? It is likely that the main reason is that, when writing to Lucene, Lucene runs various data checks, and the Lucene write operation may fail. If translog is written first, you may have to deal with the issue of Lucene write continuously failing while the translog write operation is successful. So, ES adopted the process of writing to Lucene first.

PacificA Algorithm

Proposed by Microsoft Research Asia, PacificA is a distributed consistency algorithm used for log replication systems. The paper defining it was published in 2008 (PacificA paper). ES has officially stated that its Replication model is based on this algorithm.

The Elasticsearch data replication model is based on the primary-backup model and is described very well in the PacificA paper of Microsoft Research. That model is based on having a single copy from the replication group that acts as the primary shard. The other copies are called replica shards. The primary serves as the main entry point for all indexing operations. It is in charge of validating them and making sure they are correct. Once an index operation has been accepted by the primary, the primary is also responsible for replicating the operation to the other copies.

There are few articles on the Internet that provide details about this algorithm, so, in this article, we give a brief introduction to the algorithm based on the PacificA paper. The algorithm has the following features:

  1. Has strong consistency.
  2. Synchronizes the data from a single Primary node with multiple Secondary nodes.
  3. Uses additional consistency components for Configuration maintenance.
  4. Supports writes even when a minority of Replica nodes are available.

Glossary Terms

First, let us take a look at some terms used by this algorithm:

  1. Replica Group: A dataset in which each piece of data is a copy of another, and each copy is a Replica node. Only one copy in a Replica Group is the Primary node; the rest are Secondary nodes.
  2. Configuration: Configuration of a Replica Group describes which copies are included in the Replica Group and which one is the Primary.
  3. Configuration Version: The version number of the Configuration. The version number increments by 1 whenever Configuration changes occur.
  4. Configuration Manager: This manages global Configuration components, which ensures the consistency of Configuration data. Configuration change requests are initiated by a Replica node and are then sent to Configuration Manager along with the Version. Configuration Manager verifies that the Version is correct. If not, the change request is rejected.
  5. Query & Update: There are two types of Replica Group operations, Query and Update. Query does not change the data, while Update does.
  6. Serial Number(sn): This represents the order of each Update operation execution. It increments by 1 for every Update operation, and it is a consecutive number.
  7. Prepared List: This is the preparation sequence for Update operations.
  8. Committed List: This is the commit sequence for Update operations. The operations in the commit sequence definitely take effect (unless all copies crash). On the same Replica node, Committed List must come before the Prepared List.

Primary Invariant

With the PacificA algorithm, an error detection mechanism is required to satisfy the following invariant.

When a Replica node deems itself the Primary node at any time, Configuration maintained in Configuration Manager also considers it to be the current Primary. At any time, only one Replica node deems itself the Primary node in this Replica Group.

Primary Invariant can ensure that, when a node deems itself the Primary, it must be the current Primary node. If Primary Invariant cannot be satisfied, Query requests would likely be sent to the Old Primary, which would result in legacy data being read.

How do you ensure Primary Invariant is satisfied? According to the paper, this can be achieved by adopting a Lease mechanism, which is a common method used in distributed systems. Specifically, the Primary node periodically obtains a Lease, and once successfully obtained, it deems itself to be the only Primary node for a set period. It loses Primary status if it has not obtained a new Lease once the period has expired. As long as the CPU in each machine does not have significant clock skew, the effectiveness of the lease mechanism is guaranteed.

As described in the paper, the Lease mechanism has the Primary node send a heartbeat to all Secondary nodes to obtain a Lease, instead of having all nodes obtain a Lease from a centralized component. Using this decentralized model ensures that there is no centralized component that, if it fails, causes all nodes to lose their leases.

Query

The Query process is relatively simple. Queries can only be sent to the Primary node, and the Primary node returns the corresponding values based on the latest committed data. Since this algorithm requires the Primary Invariant condition to be met, Queries always read the latest committed data.

Update

The update process is as follows:

  1. Primary node assigns a Serial Number (sn) to an UpdateRequest.
  2. The Primary node adds this UpdateRequest to its own Prepared List. Meanwhile, it sends the Prepare request to all Secondary nodes, requiring them to add this UpdateRequest to their Prepared Lists.
  3. When all Replica nodes complete Prepare, that is, when the Prepared Lists of all Replica nodes contain the Update request, the Primary node starts to commit the request, adding the UpdateRequest to Committed List and applying the Update. Note that, on the same Replica node, Committed List always comes before the Prepared List, so the Primary node increases the Committed Point when including the Update Request.
  4. The result is returned to the Client, and the Update operation is successful.

When the Primary node sends the next request to a Secondary node, the current Committed Point of the Primary is attached to the request, and the Secondary node increases its Committed Point.

We can derive the following invariant from the Update process:

Committed Invariant

We mark the Committed List of a Secondary node as SecondaryCommittedList, the Prepared List as SecondaryPreparedList, and the Committed List of the Primary as PrimaryCommittedList. SecondaryCommittedList must come before PrimaryCommittedList, and PrimaryCommittedList must come before SecondaryPreparedList.

Reconfiguration: Secondary Failure, Primary Failure, Newly Added Node

1. Secondary failure

When a Secondary node fails, the Primary node sends a Reconfiguration request to Configuration Manager, removing the failed node from Replica Group. Once the Replica node is removed, it no longer belongs to the Replica Group, and requests are no longer sent to it.

Assume that a network fault occurs between a Primary node and Secondary node. In this case, both can nonetheless connect to Configuration Manager. At this time, the Primary node detects that there is no response from the Secondary node, and, likewise, the Secondary node detects that there is no response from the Primary node. Both try to send a Reconfiguration request to have the other removed from Replica Group. The strategy applied here is the First Win principle: The first request received by Configuration Manager takes effect, and the sender remains in the Replica Group; because the other node no longer belongs to the Replica Group, it can no longer update Configuration. Because the Primary node requests a Lease from the Secondary node, the Secondary node does not execute Reconfiguration while the Lease is valid, and the probe interval of the Primary node must be less than the Lease probing interval. In this situation, it appears that the tendency is always that the Primary node executes Reconfiguration to have the Secondary node removed.

2. Primary failure

When a Primary node fails, the Secondary node stops receiving heartbeats from the Primary node. If the Lease is expired, the Secondary node sends a Reconfiguration request to have the Primary removed, which also follows the First Win principle: the Secondary node sending the successful request becomes the new Primary node.

After a Secondary node becomes a Primary node, it must go through a phase called Reconciliation before providing service. Because of the Committed Invariant mentioned above, the Committed List of the previous Primary node must come before the Prepared List of the new Primary node. This means that, when we align the Prepared List content of the new Primary node with other nodes in the current Replica Group, which is equivalent of recommitting the uncommitted records of this node on all nodes, all previous Commit records must be included. That leads to the next invariant:

Reconfiguration Invariant: When a new Primary node completes Reconciliation at T time, the Committed List of any node before T time (including the original Primary node) takes precedence over the current Committed List of the new Primary node.

Reconfiguration Invariant indicates that the committed data is not lost during the Reconfiguration process.

3. Newly added node

The newly added node must become a Secondary Candidate first, then the Primary node starts to send it Prepare requests. Meanwhile, this node tries to catch up with records that were not previously synchronized. Once it catches up with the records, it sends a request to be a Secondary node, after which the Primary node sends a configuration change request to Configuration Manager to add the node to the Replica Group.

There is another scenario: A node was in the Replica Group and removed due to temporary failure, and now needs to be re-added to the Replica Group. At this time, the data in Committed List on this node must have been committed, while the data in Prepared List may not have been committed. So, the uncommitted data should be removed, and the data should be requested from the Primary beginning at the Committed Point.

PacificA Algorithm Summary

PacificA is an algorithm with strong consistency that meets both read and write requirements. It separates data consistency from Configuration consistency and uses additional consistency components (Configuration Manager) to maintain configuration consistency. This way, when less than half of the copies of data are available, new data can still be written and strong consistency can be ensured.

The ES design refers to the PacificA algorithm. It maintains Index Meta through the Master, which is similar to Configuration maintenance by the Configuration Manager, as discussed in the paper. In IndexMeta, InSyncAllocationIds represents the currently available Shards, which is similar to Replica Group maintenance in the paper. Next, we introduce the SequenceNumber and Checkpoint in ES. These two classes are similar to the Serial Number and Committed Point in the PacificA algorithm. Afterward, we compare the similarities and differences between ES implementation and PacificA.

SequenceNumber, Checkpoint, and Failure Discovery

PacificA, a consistency algorithm model used by ES, is described above. It is important to note that each PacificA Update operation has a corresponding Serial Number, which indicates the order of execution. In the previous versions of ES, some functionality was limited because each write operation lacked a Serial Number or similar mechanism. In 2015, ES officials began planning to add SequenceNumber for each write operation and assumed there would be many application scenarios.

Further details are available at the following two links:

Add Sequence Numbers to write operations #10708

Sequence IDs: Coming Soon to an Elasticsearch Cluster Near You

Next, we give a brief introduction to the definitions of Sequence and Checkpoint, and discuss their application scenarios.

Term and SequenceNumber

Each write operation is assigned two values: Term and SequenceNumber. Term increments by 1 whenever the Primary changes, which is similar to Configuration Version in the PacificA paper. SequenceNumber increments by 1 after each operation, which is similar to Serial Number in the PacificA paper.

Because the read request is always sent to the Primary node, it assigns the Term and SequenceNumber. When the synchronization request is sent to the Replica node, the two values are attached.

LocalCheckpoint and GlobalCheckpoint

LocalCheckpoint indicates that all requests in this Shard with values less than this value have been processed.

GlobalCheckpoint indicates that all requests with values less than this value have been processed on all Replica nodes. GlobalCheckpoint is maintained by the Primary node. Each Replica node reports its LocalCheckpoint to the Primary node, and then Primary increases the GlobalCheckpoint based on that information.

GlobalCheckpoint is a global safety point indicating that all requests before it have been processed properly by the Replica node and can be used to repopulate data after recovering from a node failure. GlobalCheckpoint can also be used for the Translog GC because there is no longer a need to save the previous operation records. However, the Translog GC strategy in ES is applied based on size or time, while GlobalCheckpoint does not seem to be used.

Fast Failure Rcovery

When a Replica node fails, ES removes it. When the failure exceeds a specific period, ES assigns a new Replica node to the new Node. At this point, full data synchronization is needed. But, if the previously failed Replica node returns, simply repopulating the data after the failure recovery and adding the node back once catching up with the records result in fast failure recovery. There are two conditions that must be met to enable fast failure recovery: First, all the operations and their orders during the failure can be saved; second, the node that started data synchronization must be determined. The first condition can be met by saving Translog for a specific amount of time; the second condition can be met using Checkpoint, ultimately achieving fast failure recovery. This is the first important application scenario using SequenceNumber and Checkpoint.

Comparison between Elasticsearch and PacificA

Similarities

  1. Meta consistency and Data consistency are handled separately: In PacificA, Configuration consistency is maintained through Configuration Manager; in ES, Meta consistency is maintained through Master.
  2. Maintain the copies collection in synchronization: In PacificA, Replica Group is maintained; in ES, InSyncAllocationIds is maintained.
  3. SequenceNumber: In both PacificA and ES, write operations use SequenceNumber to record the operation order.

Differences

The main difference is that ES complies with PacificA; however, its implementation still does not meet all the requirements of the algorithm, meaning strict strong consistency is not guaranteed. The key points are as follows:

  1. Meta consistency: We analyzed the Meta consistency issue in ES in the previous section, and we can see that ES cannot guarantee Meta consistency, so it certainly cannot strictly guarantee Data consistency.
  2. Prepare phase: PacificA has the Prepare phase, which ensure that the data is not committed until it is prepared successfully on all nodes and that the committed data is not lost. In ES, the data is written directly, as it lacks this phase.
  3. Read consistency: In ES, all InSync Replica nodes can be read, which improves data readability; however, legacy data may also be read. On the other hand, even if only the Primary node can be read, ES also needs a mechanism like Lease, so that the Old Primary is not read. Given that ES is a near real-time system, the requirement for read consistency may not be very strict.

Summary

This article analyzed the consistency issues of the data flow in Elasticsearch. While ES has made substantial progress addressing these issues recently, many issues remain. This article is the last of the Elasticsearch Distributed Consistency Principles Analysis series. This series covers the research, analysis, and summary for ES, with step-by-step details covering node discovery, Master election, Meta consistency, Data consistency, and other aspects.

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

References

  1. Index API | Elasticsearch Reference 6.2
  2. Reading and Writing documents | Elasticsearch Reference 6.2
  3. PacificA: Replication in Log-Based Distributed Storage Systems
  4. Add Sequence Numbers to write operations #10708
  5. Sequence IDs: Coming Soon to an Elasticsearch Cluster Near You
0 0 0
Share on

Whybert

3 posts | 0 followers

You may also like

Comments

Whybert

3 posts | 0 followers

Related Products