The Distributed Area contains indexing and coordination systems.
The index path stretches from the user REST command through shard routing down to each individual shard's translog and storage engine. Reindexing is effectively reading from a source index and writing to a destination index (perhaps on different nodes). The coordination side includes cluster coordination, shard allocation, cluster autoscaling stats, task management, and cross cluster replication. Less obvious coordination systems include networking, the discovery plugin system, the snapshot/restore logic, and shard recovery.
A guide to the general Elasticsearch components can be found here.
(We have many thread pools, what and why)
See the Javadocs for ActionListener
(TODO: add useful starter references and explanations for a range of Listener classes. Reference the Netty section.)
The REST and Transport layers are bound together through the ActionModule
. ActionModule#initRestHandlers
registers all the
rest actions with a RestController
that matches incoming requests to particular REST actions. RestController#registerHandler
uses each Rest*Action
's #routes()
implementation to match HTTP requests to that particular Rest*Action
. Typically, REST
actions follow the class naming convention Rest*Action
, which makes them easier to find, but not always; the #routes()
definition can also be helpful in finding a REST action. RestController#dispatchRequest
eventually calls #handleRequest
on a
RestHandler
implementation. RestHandler
is the base class for BaseRestHandler
, which most Rest*Action
instances extend to
implement a particular REST action.
BaseRestHandler#handleRequest
calls into BaseRestHandler#prepareRequest
, which children Rest*Action
classes extend to
define the behavior for a particular action. RestController#dispatchRequest
passes a RestChannel
to the Rest*Action
via
RestHandler#handleRequest
: Rest*Action#prepareRequest
implementations return a RestChannelConsumer
defining how to execute
the action and reply on the channel (usually in the form of completing an ActionListener wrapper). Rest*Action#prepareRequest
implementations are responsible for parsing the incoming request, and verifying that the structure of the request is valid.
BaseRestHandler#handleRequest
will then check that all the request parameters have been consumed: unexpected request parameters
result in an error.
The Rest layer uses an implementation of AbstractClient
. BaseRestHandler#prepareRequest
takes a NodeClient
: this client
knows how to connect to a specified TransportAction. A Rest*Action
implementation will return a RestChannelConsumer
that
most often invokes a call into a method on the NodeClient
to pass through to the TransportAction. Along the way from
BaseRestHandler#prepareRequest
through the AbstractClient
and NodeClient
code, NodeClient#executeLocally
is called: this
method calls into TaskManager#registerAndExecute
, registering the operation with the TaskManager
so it can be found in Task
API requests, before moving on to execute the specified TransportAction.
NodeClient
has a NodeClient#actions
map from ActionType
to TransportAction
. ActionModule#setupActions
registers all the
core TransportActions, as well as those defined in any plugins that are being used: plugins can override Plugin#getActions()
to
define additional TransportActions. Note that not all TransportActions will be mapped back to a REST action: many TransportActions
are only used for internode operations/communications.
(Managed by the TransportService, TransportActions must be registered there, too)
(Executing a TransportAction (either locally via NodeClient or remotely via TransportService) is where most of the authorization & other security logic runs)
(What actions, and why, are registered in TransportService but not NodeClient?)
(TransportService maps incoming requests to TransportActions)
(long running actions should be forked off of the Netty thread. Keep short operations to avoid forking costs)
The RestClient
is primarily used in testing, to send requests against cluster nodes in the same format as would users. There
are some uses of RestClient
, via RestClientBuilder
, in the production code. For example, remote reindex leverages the
RestClient
internally as the REST client to the remote elasticsearch cluster, and to take advantage of the compatibility of
RestClient
requests with much older elasticsearch versions. The RestClient
is also used externally by the Java API Client
to communicate with Elasticsearch.
(Sketch of important classes? Might inform more sections to add for details.)
(A NodeB can coordinate a search across several other nodes, when NodeB itself does not have the data, and then return a result to the caller. Explain this coordinating role)
(Quorum, terms, any eligibility limitations)
(Explain joining, and how it happens every time a new master is elected)
(Majority concensus to apply, what happens if a master-eligible node falls behind / is incommunicado.)
(Go over the two kinds of listeners -- ClusterStateApplier and ClusterStateListener?)
(Sketch ephemeral vs persisted cluster state.)
(what's the format for persisted metadata)
(More Topics: ReplicationTracker concepts / highlights.)
(How a primary shard is chosen)
(terms and such)
(How an index write replicates across shards -- TransportReplicationAction?)
(What guarantees do we give the user about persistence and readability?)
(rarely use locks)
(What does Engine mean in the distrib layer? Distinguish Engine vs Directory vs Lucene)
(High level explanation of how translog ties in with Lucene)
(contrast Lucene vs ES flush / refresh / fsync)
(internal vs external reader manager refreshes? flush vs refresh)
(Data lives beyond a high level IndexShard instance. Continue to exist until all references to the Store go away, then Lucene data is removed)
(Explain checkpointing and generations, when happens on Lucene flush / fsync)
(Concurrency control for flushing)
(VersionMap)
(copy a sketch of the files Lucene can have here and explain)
(Explain about SearchIndexInput -- IndexWriter, IndexReader -- and the shared blob cache)
(Lucene uses Directory, ES extends/overrides the Directory class to implement different forms of file storage. Lucene contains a map of where all the data is located in files and offsites, and fetches it from various files. ES doesn't just treat Lucene as a storage engine at the bottom (the end) of the stack. Rather ES has other information that works in parallel with the storage engine.)
(All shards go through a 'recovery' process. Describe high level. createShard goes through this code.)
(How is the translog involved in recovery?)
(partial shard recoveries survive server restart? reestablishRecovery
? How does that work.)
(Frozen, warm, hot, etc.)
(AllocationService runs on the master node)
(Discuss different deciders that limit allocation. Sketch / list the different deciders that we have.)
(Significant internal APIs for balancing a cluster)
(How does this command behave with the desired auto balancer.)
The Autoscaling API in ES (Elasticsearch) uses cluster and node level statistics to provide a recommendation for a cluster size to support the current cluster data and active workloads. ES Autoscaling is paired with an ES Cloud service that periodically polls the ES elected master node for suggested cluster changes. The cloud service will add more resources to the cluster based on Elasticsearch's recommendation. Elasticsearch by itself cannot automatically scale.
Autoscaling recommendations are tailored for the user based on user defined policies, composed of data roles (hot, frozen, etc.) and deciders. There's a public webinar on autoscaling, as well as the public Autoscaling APIs docs.
Autoscaling's current implementation is based primary on storage requirements, as well as memory capacity for ML and frozen tier. It does not yet support scaling related to search load. Paired with ES Cloud, autoscaling only scales upward, not downward, except for ML nodes that do get scaled up and down.
Autoscaling is a plugin. All the REST APIs can be found in autoscaling/rest/.
GetAutoscalingCapacityAction
is the capacity calculation operation REST endpoint, as opposed to the
other rest commands that get/set/delete the policies guiding the capacity calculation. The Transport
Actions can be found in autoscaling/action/, where TransportGetAutoscalingCapacityAction is the
entrypoint on the master node for calculating the optimal cluster resources based on the autoscaling
policies.
AutoscalingMetadata implements Metadata.Custom in order to persist autoscaling policies. Each Decider is an implementation of AutoscalingDeciderService. The AutoscalingCalculateCapacityService is responsible for running the calculation.
TransportGetAutoscalingCapacityAction.computeCapacity is the entry point to AutoscalingCalculateCapacityService.calculate, which creates a AutoscalingDeciderResults for each autoscaling policy. AutoscalingDeciderResults.toXContent then determines the maximum required capacity to return to the caller. AutoscalingCapacity is the base unit of a cluster resources recommendation.
The TransportGetAutoscalingCapacityAction
response is cached to prevent concurrent callers
overloading the system: the operation is expensive. TransportGetAutoscalingCapacityAction
contains
a CapacityResponseCache. TransportGetAutoscalingCapacityAction.masterOperation
calls through the CapacityResponseCache, into the AutoscalingCalculateCapacityService
, to handle
concurrent callers.
The Deciders each pull data from different sources as needed to inform their decisions. The
DiskThresholdMonitor is one such data source. The Monitor runs on the master node and maintains
lists of nodes that exceed various disk size thresholds. DiskThresholdSettings contains the
threshold settings with which the DiskThresholdMonitor
runs.
The ReactiveStorageDeciderService
tracks information that demonstrates storage limitations are causing
problems in the cluster. It uses an algorithm defined here. Some examples are
- information from the
DiskThresholdMonitor
to find out whether nodes are exceeding their storage capacity - number of unassigned shards that failed allocation because of insufficient storage
- the max shard size and minimum node size, and whether these can be satisfied with the existing infrastructure
The ProactiveStorageDeciderService
maintains a forecast window that defaults to 30 minutes. It only
runs on data streams (ILM, rollover, etc.), not regular indexes. It looks at past index changes that
took place within the forecast window to predict resources that will be needed shortly.
There are several more Decider Services, implementing the AutoscalingDeciderService
interface.
(We've got some good package level documentation that should be linked here in the intro)
(copy a sketch of the file system here, with explanation -- good reference)
(Include an overview of the coordination between data and master nodes, which writes what and when)
(Concurrency control: generation numbers, pending generation number, etc.)
(partial snapshots)
The tasks infrastructure is used to track currently executing operations in the Elasticsearch cluster. The Task management API provides an interface for querying, cancelling, and monitoring the status of tasks.
Each individual task is local to a node, but can be related to other tasks, on the same node or other nodes, via a parent-child relationship.
Note
The Task management API is experimental/beta, its status and outstanding issues can be tracked here.
Tasks are tracked in-memory on each node in the node's TaskManager, new tasks are registered via one of the TaskManager#register methods. Registration of a task creates a Task instance with a unique-for-the-node numeric identifier, populates it with some metadata and stores it in the TaskManager.
The register methods will return the registered Task instance, which can be used to interact with the task. The Task class is often sub-classed to include task-specific data and operations. Specific Task subclasses are created by overriding the createTask method on the TaskAwareRequest passed to the TaskManager#register methods.
When a task is completed, it must be unregistered via TaskManager#unregister.
The IDs given to a task are numeric, supplied by a counter that starts at zero and increments over the life of the node process. So while they are unique in the individual node process, they would collide with IDs allocated after the node restarts, or IDs allocated on other nodes.
To better identify a task in the cluster scope, a tuple of persistent node ID and task ID is used. This is represented in code using the TaskId class and serialized as the string {node-ID}:{local-task-ID}
(e.g. oTUltX4IQMOUUVeiohTt8A:124
). While TaskId is safe to use to uniquely identify tasks currently running in a cluster, it should be used with caution as it can collide with tasks that have run in the cluster in the past (i.e. tasks that ran prior to a cluster node restart).
The purpose of tasks is to provide management and visibility of the cluster workload. There is some overhead involved in tracking a task, so they are best suited to tracking non-trivial and/or long-running operations. For smaller, more trivial operations, visibility is probably better implemented using telemetry APIs.
Some examples of operations that are tracked using tasks include:
- Execution of TransportActions
- NodeClient#executeLocally invokes TaskManager#registerAndExecute
- RequestHandlerRegistry#processMessageReceived registers tasks for actions that are spawned to handle TransportRequests
- Publication of cluster state updates
All ThreadPool threads have an associated ThreadContext. The ThreadContext contains a map of headers which carry information relevant to the operation currently being executed. For example, a thread spawned to handle a REST request will include the HTTP headers received in that request.
When threads submit work to an ExecutorService from the ThreadPool, those spawned threads will inherit the ThreadContext of the thread that submitted them. When TransportRequests are dispatched, the headers from the sending ThreadContext are included and then loaded into the ThreadContext of the thread handling the request. In these ways, ThreadContext is preserved across threads involved in an operation, both locally and on remote nodes.
When a task is registered by a thread, a subset (defined by Task#HEADERS_TO_COPY and any ActionPlugins loaded on the node) of the headers from the ThreadContext are copied into the Task's set of headers.
One such header is X-Opaque-Id
. This is a string that can be submitted on REST requests, and it will be associated with all tasks created on all nodes in the course of handling that request.
Another way to track the operations of a task is by following the parent/child relationships. When registering a task it can be optionally associated with a parent task. Generally if an executing task initiates sub-tasks, the ID of the executing task will be set as the parent of any spawned tasks (see ParentTaskAssigningClient, TransportService#sendChildRequest and TaskAwareRequest#setParentTask for how this is implemented for TransportActions).
Some long-running tasks are implemented to be cancel-able. Cancellation of a task and its descendants can be done via the Cancel Task REST API or programmatically using TaskManager#cancelTaskAndDescendants. Perhaps the most common use of cancellation you will see is cancellation of TransportActions dispatched from the REST layer when the client disconnects, to facilitate this we use the RestCancellableNodeClient.
In order to support cancellation, the Task instance associated with the task must extend CancellableTask. It is the job of any workload tracked by a CancellableTask to periodically check whether it has been cancelled and, if so, finish early. We generally wait for the result of a cancelled task, so tasks can decide how they complete upon being cancelled, typically it's exceptionally with TaskCancelledException.
When a Task extends CancellableTask the TaskManager keeps track of it and any child tasks that it spawns. When the task is cancelled, requests are sent to any nodes that have had child tasks submitted to them to ban the starting of any further children of that task, and any cancellable child tasks already running are themselves cancelled (see BanParentRequestHandler).
When a cancellable task dispatches child requests through the TransportService, it registers a proxy response handler that will instruct the remote node to cancel that child and any lingering descendants in the event that it completes exceptionally (see UnregisterChildTransportResponseHandler). A typical use-case for this is when no response is received within the time-out, the sending node will cancel the remote action and complete with a timeout exception.
A list of tasks currently running in a cluster can be requested via the Task management API, or the cat task management API. The former returns each task represented using TaskResult, the latter returning a more compact CAT representation.
Some ActionRequests allow the results of the actions they spawn to be stored upon completion for later retrieval. If ActionRequest#getShouldStoreResult returns true, a TaskResultStoringActionListener will be inserted into the chain of response listeners. TaskResultStoringActionListener serializes the TaskResult of the TransportAction and persists it in the .tasks
index using the TaskResultsService.
The Task management API also exposes an endpoint where a task ID can be specified, this form of the API will return currently running tasks, or completed tasks whose results were persisted. Note that although we use TaskResult to return task information from all the JSON APIs, the error
or response
fields will only ever be populated for stored tasks that are already completed.
Up until now we have discussed only ephemeral tasks. If we want a task to survive node failures, it needs to be registered as a persistent task at the cluster level.
Plugins can register persistent tasks definitions by implementing PersistentTaskPlugin and returning one or more PersistentTasksExecutor instances. These are collated into a PersistentTasksExecutorRegistry which is provided to PersistentTasksNodeService active on each node in the cluster, and a PersistentTasksClusterService active on the master.
The PersistentTasksClusterService runs on the master to manage the set of running persistent tasks. It periodically checks that all persistent tasks are assigned to live nodes and handles the creation, completion, removal and updates-to-the-state of persistent task instances in the cluster state (see PersistentTasksCustomMetadata).
The PersistentTasksNodeService monitors the cluster state to:
- Start any tasks allocated to it (tracked in the local TaskManager by an AllocatedPersistentTask)
- Cancel any running tasks that have been removed (AllocatedPersistentTask extends CancellableTask)
If a node leaves the cluster while it has a persistent task allocated to it, the master will re-allocate that task to a surviving node. To do this, it creates a new PersistentTasksCustomMetadata.PersistentTask entry with a higher #allocationId
. The allocation ID is included any time the PersistentTasksNodeService communicates with the PersistentTasksClusterService about the task, it allows the PersistentTasksClusterService to ignore persistent task messages originating from stale allocations.
Some examples of the use of persistent tasks include:
- ShardFollowTasksExecutor: Defined by cross-cluster replication to poll a remote cluster for updates
- HealthNodeTaskExecutor: Used to schedule work related to monitoring cluster health
- SystemIndexMigrationExecutor: Manages the migration of system indices after an upgrade
Tasks are integrated with the ElasticSearch APM infrastructure. They implement the Traceable interface, and spans are published to represent the execution of each task.
(Brief explanation of the use case for CCR)
(Explain how this works at a high level, and details of any significant components / ideas.)
(Explain that the Distributed team is responsible for the write path, while the Search team owns the read path.)
(Generating document IDs. Same across shard replicas, _id field)
(Sequence number: different than ID)
(what limits write concurrency, and how do we minimize)
(explain visibility of writes, and reference the Lucene section for more details (whatever makes more sense explained there))
(this can also happen during shard reallocation, right? This might be a standalone topic, or need another section about it in allocation?...)