We have various ways of copying between two streams and handling thread-local
buffers throughout the codebase. This commit unifies a number of them and
removes buffer allocations in many spots.
Previously if an inner_hits block required _ source, we would reload and parse
the root document's source for every hit. This PR adds a shared SourceLookup to
the inner hits context that allows inner hits to reuse parsed source if it's
already available. This matches our approach for sharing the root document ID.
Relates to #32818.
This PR simplifies the hierarchy for ordinals field data classes:
* Remove `AbstractIndexFieldData`, since only `AbstractIndexOrdinalsFieldData`
inherits directly from it.
* Make `SortedSetOrdinalsIndexFieldData` extend
`AbstractIndexOrdinalsFieldData`. This lets us remove some redundant code.
Allows nanosecond resolution in search_after (#60328)
This fixes `search_after` to properly parse string formatted dates that
have nanosecond resolution.
Closes#52424
Today when a node fails to properly deserialize a transport message with
a parent task we log the following relatively uninformative message:
java.lang.IllegalStateException: Message not fully read (response) for requestId [9999], handler [org.elasticsearch.transport.TransportService$ContextRestoreResponseHandler/org.elasticsearch.transport.TransportService$ContextRestoreResponseHandler/org.elasticsearch.transport.TransportService$6@abcdefgh], error [false]; resetting
In particular, the wrapping of the listener in the `TransportService`
obscures all clues as to the source of the problem, e.g. the action name
or the identity of the underlying listener. This commit exposes the
inner listener to the logs.
Also if the listener is wrapped with `ContextPreservingActionListener`
then its identity is similarly hidden. This commit also exposes the
wrapped listener in this case.
Relates #38939
Same as #59905 but for shard level metadata. Since we wnat to retain
the ability to do safe+atomic writes for non-uuid shard generations
this PR has to create two separate write paths for both kinds of
shard generations.
Writing the `index.latest` blob is unnecessary unless the contents of the repository
are to be used as a URL-repository. Also, in some edge cases, the fact that `index.latest` is the only
blob in the repository that regularly gets overwritten was causing compatibility issues with
some backing blobstores (Azure no-overwrite policy, Hitachy S3 equivalent).
=> this commit changes behavior to make snapshots not fail if writing `index.latest` fails
and adds a setting to disable writing `index.latest`.
For consistency reasons (and reducing the overload of IllegalArgumentException)
this changes the exception thrown when trying to create a data stream
that already exists.
(cherry picked from commit ac2184c4614bba0f3ee377da49aea0daed98bab4)
Signed-off-by: Andrei Dan <andrei.dan@elastic.co>
This change ensures that we call the listener for partial merge failure **before**
calling the completion listener in order to avoid race condition in tests.
Closes#60446
- Replace immediate task creations by using task avoidance api
- One step closer to #56610
- Still many tasks are created during configuration phase. Tackled in separate steps
The `SourceLookup` class provides access to the _source for a particular
document, specified through `SourceLookup#setSegmentAndDocument`. Previously
the search context contained a single `SourceLookup` that was shared between
different fetch subphases. It was hard to reason about its state: is
`SourceLookup` set to the expected document? Is the _source already loaded and
available?
Instead of using a global source lookup, the fetch hit context now provides
access to a lookup that is set to load from the hit document.
This refactor closes#31000, since the same `SourceLookup` is no longer shared
between the 'fetch _source phase' and script execution.
The oversize algorithm was allocating more pages than necessary to accommodate `minTargetSize`.
An example would be that a 16k page size and 15k `minTargetSize` would result in a new size of 32k (2 pages).
The difference between the minimum number of necessary pages and the estimated size then keeps growing as sizes increase.
I don't think there is much value in preemptively allocating pages by over-sizing aggressively since the behavior of
the system is quite different from that of a single array where over-sizing avoids copying
once the minimum target size is more than a single page.
Relates #60173 which lead me to this when `BytesStreamOutput` would allocate a large number of never used
pages during serialization of repository metadata.
Instances of `BlobStoreIndexShardSnapshots` can be of non-trivial size. In case of snapshotting a larger
number of shards the previous execution order would lead to memory use proportional to the number of shards
for these objects. With this change, the number of these objects on heap is bounded by the size of the snapshot
pool (except for in the BwC format path).
This PR makes it so that they are written to the repository at the earliest possible point in time
so that they can be garbage collected.
If shard generations are used, we can safely write these right at the beginning of the shard snapshot.
If shard generations are not used we can only write them at the end of the shard snapshot after all
other blobs have been written.
Closes#60173
CCR will stop functioning if the master node is on 7.8, but data nodes
are before that version because the master node considers that all data
nodes do not have the remote cluster client role. This commit allows CCR
work on data nodes with legacy roles only.
Relates #54146
Relates #59375
This fixes the behavior of the snapshot state machine in the following edge case:
1. Snapshot is running
2. Delete/abort for the snapshot is started
3. Snapshot fails to finalize
We were not removing the failed snapshot id from the list of snapshots to delete in the delete.
This lead to an error in the repository, which throws if we try to delete a non-existing snapshot.
This commmit updates the deletions in progress by removing the failed snapshot id.
The fact that this could lead to snapshot delete entries without any snapshot ids is not optimized
on purpose because it allows for another attempt at writing clean `RepositoryData` and will run basic
cleanup on the repository (root level blobs and stale indices) and thus bring the repository back into
a clean state after a failed finalization.
Closes#60274
The test assumed that the master fail-over would always work out as a single step.
This is not guaranteed however and we can randomly see master failing over twice,
in which case the transport listener will be failed on the node that stops being
leader and we have to catch an exception for the deletes as well just like we do
for the snapshot.
Closes#60262
Adds a full list of supported aggregations to the node info API. This list
will be used in transform tests and telemetry mapping tests that will be added
as follow-up PRs.
Fixes#59774
This feature adds a new `fields` parameter to the search request, which
consults both the document `_source` and the mappings to fetch fields in a
consistent way. The PR merges the `field-retrieval` feature branch.
Addresses #49028 and #55363.
Transport connections between nodes remain in place until one or other
node shuts down or the connection is disrupted by a flaky network.
Today it is very difficult to demonstrate that transient failures and
cluster instability are caused by the network even though this is often
the case. In particular, transport connections open and close without
logging anything, even at `DEBUG` level, making it very hard to quantify
the scale of the problem or to correlate the networking problems with
external events.
This commit adds the missing `DEBUG`-level logging when transport
connections open and close, and also tracks the total number of
transport connections a node has opened as a measure of the stability of
the underlying network.
* Fix Test Failure in testCorrectCountsForDoneShards
Fixing the freak edge case where the node shard status request returns before
the node was able to send the state update request to master and update the cluster state.
Without this change, the snapshot shard status would report as `DONE` once the data node
has finished updating the shard in the cluster state.
If the data node then drops out of the cluster before the state has been updated, then
the status will jump to "FAILURE" because the master updates the state once the data node
leaves the cluster.
Closes#60247
Today if a cluster state observer's listener takes a long time to
process a notification then we log the following rather useless warning
message:
[notifying listener [org.elasticsearch.cluster.ClusterStateObserver$ObserverClusterStateListener@12345678]] took [34567ms]
This commit adds a handful of simple `toString()` implementations in
order to identify the owner of the listener in question.
This change forks the execution of partial
reduces in the coordinating node to the search thread pool.
It also ensures that partial reduces are executed sequentially
and asynchronously in order to limit the memory and cpu that a
single search request can use but also to avoid blocking a
network thread.
If a partial reduce fails with an exception, the search
request is cancelled and the reporting of the error is
delayed to the start of the fetch phase (when the final
reduce is performed). This ensures that we cleanup the
in-flight search requests before returning an error to
the user.
Closes#53411
Relates #51857
We can save one round of serializing `RepositoryData` on the write path.
This also leads to somewhat better compression because we compress larger chunks
in one go potentially when compared to serializing and compressing in one go.
Also, fixed the double wrapping of collections when copying the repository
data instance via the `withGenId`.
Keepalive options are not well-documented (only in transport section, although also available at http and network level).
Co-authored-by: David Turner <david.turner@elastic.co>
Co-authored-by: James Rodewig <40268737+jrodewig@users.noreply.github.com>
keepalives tell any intermediate devices that the connection remains alive, which helps with overzealous firewalls that are
killing idle connections. keepalives are enabled by default in Elasticsearch, but use system defaults for their
configuration, which often times do not have reasonable defaults (e.g. 7200s for TCP_KEEP_IDLE) in the context of
distributed systems such as Elasticsearch.
This PR sets the socket-level keep_alive options for network.tcp.{keep_idle,keep_interval} to 5 minutes on configurations
that support it (>= Java 11 & (MacOS || Linux)) and where the system defaults are set to something higher than 5
minutes. This helps keep the connections alive while not interfering with system defaults or user-specified settings
unless they are deemed to be set too high by providing better out-of-the-box defaults.
Currently, we do not categorize the operation type in the rejection
exception messsage when we reject an indexing operation for indexing
memory limits. This commit fixes this to ensure that it is identified as
coordinating, primary, or replica.
Currently the logic to rejection replica rejections is evaluate before
adding the additional bytes of the current operation. This means that
the first replica operation which should be rejected will be allowed to
proceed. This commit fixes this logic and adds unit level test to ensure
indexing pressure behavior is correct.
When a document which is distant from existing buckets gets collected, the
`variable_width_histogram` will create a new bucket and then insert it
into the ordered list of buckets.
Currently, a new merge map array is created to move this bucket. This is
very expensive as there might be thousands of buckets.
This PR creates `mergeBuckets(UnaryOperator<Long> mergeMap)` methods in
`BucketsAggregator` and `MergingBucketsDefferingCollector`, and updates
the `variable_width_histogram` to use them. This eliminates the need to
create an entire merge map array for each new bucket and reduces the
memory overhead of the algorithm.
Co-authored-by: James Dorfman <jamesdorfman@users.noreply.github.com>
In almost all cases we write uuid named files via this method.
Preemptively deleting just wastes IO ops, we can delete after a write failed
and retry the write to cover the few cases where we actually do an overwrite.
Putting an ingest pipeline used to require that the user calling
it had permission to get nodes info as well as permission to
manage ingest. This was due to an internal implementaton detail
that was not visible to the end user.
This change alters the behaviour so that a user with the
manage_pipeline cluster privilege can put an ingest pipeline
regardless of whether they have the separate privilege to get
nodes info. The internal implementation detail now runs as
the internal _xpack user when security is enabled.
Backport of #60106
There is a very unlikely but possible test failure in this test.
The `SnapshotsService` continues iterating over queued operations after
resolving the transport listener. This can lead to a situation where the moved
repository data is not picked up when running the delete (even though we
have the concurrent modifications BwC mode activated) concurrently.
I fixed this in the test so that the test still verifies that this setting works.
Technically speaking, one could add logic to the way we queue and execute repo operations
to address this special case. Since this case only comes about with the concurrent modifications
setting enabled (and the setting is gone in master already) I don't really see a reason to improve
the logic here since we should always fail queued up repo operations on concurrent modification for
safety reasons.
Today there is a node-level `canAllocate` override which the balancer
uses to ignore certain nodes to which it is certain no more shards can
be allocated. In fact this override only ignores nodes which have hit
the rarely-used `cluster.routing.allocation.total_shards_per_node`
limit, so this optimization doesn't have a meaningful impact on real
clusters.
This commit removes this unnecessary fast path from the balancer, and
also removes all the machinery needed to support it.
The single element `close` calls go through a very inefficient path that includes creating
a one element list.
`releaseOnce` is only with a single non-null input in production in two spots so no need for
varargs and any complexity here.
`ReleasableBytesStreamOutput` does not require any `releaseOnce` wrapping because we already have
that kind of logic implemented in `org.elasticsearch.common.util.AbstractArray` (which we were
wrapping here) already.
This improves modularity and also fixes some issues when `docvalues_fields` is
used within `inner_hits` or the `top_hits` agg:
* We previously didn't resolve wildcards in field names.
* We also forgot to enforce the limit `index.max_docvalue_fields_search`.
Due to complicated access checks (reads and writes execute in their own access context) on some repositories (GCS, Azure, HDFS), using a hard coded buffer size of 4k for restores was needlessly inefficient.
By the same token, the use of stream copying with the default 8k buffer size for blob writes was inefficient as well.
We also had dedicated, undocumented buffer size settings for HDFS and FS repositories. For these two we would use a 100k buffer by default. We did not have such a setting for e.g. GCS though, which would only use an 8k read buffer which is needlessly small for reading from a raw `URLConnection`.
This commit adds an undocumented setting that sets the default buffer size to `128k` for all repositories. It removes wasteful allocation of such a large buffer for small writes and reads in case of HDFS and FS repositories (i.e. still using the smaller buffer to write metadata) but uses a large buffer for doing restores and uploading segment blobs.
This should speed up Azure and GCS restores and snapshots in a non-trivial way as well as save some memory when reading small blobs on FS and HFDS repositories.
This commit continues on the work in #59801 and makes other
implementors of the LocalNodeMasterListener interface thread safe in
that they will no longer allow the callbacks to run on different
threads and possibly race each other. This also helps address other
issues where these events could be queued to wait for execution while
the service keeps moving forward thinking it is the master even when
that is not the case.
In order to accomplish this, the LocalNodeMasterListener no longer has
the executorName() method to prevent future uses that could encounter
this surprising behavior.
Each use was inspected and if the class was also a
ClusterStateListener, the implementation of LocalNodeMasterListener
was removed in favor of a single listener that combined the logic. A
single listener is used and there is currently no guarantee on execution
order between ClusterStateListeners and LocalNodeMasterListeners,
so a future change there could cause undesired consequences. For other
classes, the implementations of the callbacks were inspected and if the
operations were lightweight, the overriden executorName method was
removed to use the default, which runs on the same thread.
Backport of #59932
Sometimes there is the need to make a field required in the mappings, and validate that a value has been provided for it. This can be done through a validator when using ParametrizedFieldMapper, but validators need to run also when a value for a field has not been specified.
Relates to #59332
There were two subtle bugs here from backporting #56911 to 7.x.
1. We passed `null` for the `shards` map which isn't nullable any longer
when creating `SnapshotsInProgress.Entry`, fixed by just passing an empty map
like the `null` handling did in the past.
2. The removal of a failed `INIT` state snapshot from the cluster state tried
removing it from the finalization loop (the set of repository names that are
currently finalizing). This will trip an assertion since the snapshot failed
before its repository was put into the set. I made the logic ignore the set
in case we remove a failed `INIT` state snapshot to restore the old logic to
exactly as it was before the concurrent snapshots backport to be on the safe
side here.
Also, added tests that explicitly call the old code paths because as can be seen
from initially missing this, the BwC tests will only run in the configuration new
version master, old version nodes ever so often and having a deterministic test
for the old state machine seems the safest bet here.
Closes#59986
For ingest node processors a per processor description
was recently added. This commit displays that description
in the verbose output of the pipeline simulation.
related #57906
In #54716 I removed pipeline aggregators from the aggregation result
tree and caused us to read them from the request. This saves a bunch of
round trip bytes, which is neat. But there was a bug in the backwards
compatibility logic. You see, we still have to give the pipeline
aggregations to nodes older than 7.8 over the wire because that is how
they know what pipelines to run. They have the pipelines in the request
but they don't read them. They use the ones in the response tree.
Anyway, we had a bug where we were never sending pipelines defined two
levels down. So while you are upgrading the pipeline wouldn't run.
Sometimes. If the data node of the "first" result was post-7.8 and the
coordinating node was pre-7.8.
This fixes the bug.
Today the `InternalClusterInfoService` uses the
`LocalNodeMasterListener` interface to start/stop its operations. Since
the `onMaster` and `offMaster` methods are called on the `MANAGEMENT`
threadpool, there's no guarantee that they run in the correct sequence,
which could result in an elected master failing to regularly update the
cluster info.
Since this service is also a `ClusterStateListener` we may as well drop
the usage of the `LocalNodeMasterListener` interface and simply update
the status of the local node on the applier thread in `clusterChanged`
to ensure consistency.
Additionally, today the `InternalClusterInfoService` uses a simple flag
to track whether the local node is the elected master or not. If the
node stops being the master and then starts again within a few seconds
then the scheduled updates from the old mastership might carry on
running in addition to the ones for the new mastership.
This commit addresses that by tracking the identity of the scheduled
update job and creating a new job for each mastership.
The TypeParser implementations of all ParametrizedFieldMapper descendant classes are
essentially the same - stateless, requiring the construction of a Builder object, and calling
parse on it before returning it. We can make this easier (and less error-prone) to
implement by wrapping the logic up into a final class, which takes a function to produce
the Builder from a name and parser context.
We never used the `IndexSettings` parameter and we only used the
`MappedFieldType` parameter to get the name of the field which we
already know everywhere where we build the `IFD.Builder`. This allows us
to drop a fair bit of ceremony from a couple of tests.
ParametrizedFieldMapper overrides `toXContent` from `FieldMapper`, yet it could override `doXContentBody` and rely on the `toXContent` from the base class. Additionally, this allows to make `doXContentBody` final. Also, toXContent is still overridden only to make it final.
With uuid named segment data blobs there is no reason to ensure no overwrites are happening
for these blobs when writing. On the contrary, at least on Azure this check can conflict with
the SDK's retrying and cause upload failures randomly.
Refactored `CheckSumBlobStoreFormat` so it can more easily be reused in
other functionality (i.e. upcoming repair logic).
Simplified away constant `failIfAlreadyExists` parameter and removed the atomic
write method and its tests.
The atomic write method was only used in a single spot and that spot has now been adjusted to
work the same way writing root level metadata works.
Follow up to #59606 using some of the new infrastructure and making similar cleanups (and due to at times better handling of size hints and empty collections also optimizations in the stream utility methods this also means speedups) in various spots in the core codebase.
Previously we constructed a GeometryFormat object and delegated point parsing to
it. This wasn't a good fit conceptually because each GeometryFormat instance
didn't represent a distinct point format.
This replaces that data structure that we use to resolve bucket ids in
bucketing aggs that are inside other bucketing aggs. This replaces the
"legoed together" data structure with a purpose built `LongLongHash`
with semantics similar to `LongHash`, except that it has two `long`s
as keys instead of one.
The microbenchmarks show a fairly substantial performance gain on the
hot path, around 30%. Rally's higher level benchmarks show anywhere
from 0 to 7% speed improvements. Not as much as I'd hoped, but nothing
to sneeze at. And, after all, we all allocating slightly less data per
owningBucketOrd, which is always nice.
Today `GET _nodes/stats/fs` includes `{least,most}_usage_estimate`
fields for some nodes. These fields have rather strange semantics. They
are only reported on the elected master and on nodes that have been the
elected master since they were last restarted; when a node stops being
the elected master these stats remain in place but we stop updating them
so they may become arbitrarily stale.
This means that these statistics are pretty meaningless and impossible
to use correctly. Even if they were kept up to date they're never
reported for data-only nodes anyway, despite the fact that data nodes
are the ones where we care most about disk usage. The information needed
to compute the path with the least/most available space is already
provided in the rest the stats output, so we can treat the inclusion of
these stats as a bug and fix it by simply removing them in this commit.
Since these stats were always optional and mostly omitted (for opaque
reasons) this is not considered a breaking change.
This cleans up a few rough edged in the `variable_width_histogram`,
mostly found by @wwang500:
1. Setting its tuning parameters in an unexpected order could cause the
request to fail.
2. We checked that the maximum number of buckets was both less than
50000 and MAX_BUCKETS. This drops the 50000.
3. Fixes a divide by 0 that can occur of the `shard_size` is 1.
4. Fixes a divide by 0 that can occur if the `shard_size * 3` overflows
a signed int.
5. Requires `shard_size * 3 / 4` to be at least `buckets`. If it is less
than `buckets` we will very consistently return fewer buckets than
requested. For the most part we expect folks to leave it at the
default. If they change it, we expect it to be much bigger than
`buckets`.
6. Allocate a smaller `mergeMap` in when initially bucketing requests
that don't use the entire `shard_size * 3 / 4`. Its just a waste.
7. Default `shard_size` to `10 * buckets` rather than `100`. It *looks*
like that was our intention the whole time. And it feels like it'd
keep the algorithm humming along more smoothly.
8. Default the `initial_buffer` to `min(10 * shard_size, 50000)` like
we've documented it rather than `5000`. Like the point above, this
feels like the right thing to do to keep the algorithm happy.
Co-authored-by: Elastic Machine <elasticmachine@users.noreply.github.com>
Co-authored-by: Elastic Machine <elasticmachine@users.noreply.github.com>
There was a bug in the geoshape circuit-breaker check where the
hash values array was being allocated before its new size was
accounted for by the circuit breaker.
Fixes#57847.
* Adding new `require_alias` option to indexing requests (#58917)
This commit adds the `require_alias` flag to requests that create new documents.
This flag, when `true` prevents the request from automatically creating an index. Instead, the destination of the request MUST be an alias.
When the flag is not set, or `false`, the behavior defaults to the `action.auto_create_index` settings.
This is useful when an alias is required instead of a concrete index.
closes https://github.com/elastic/elasticsearch/issues/55267
Currently, when we apply a cluster state change to a shard on a non-master node,
we check to see if the mappings need to be updated by comparing the decompressed
serialized mappings from the update against the serialized version of the shard's
existing mappings. However, we already have a much simpler way of checking this,
by comparing mapping versions on the index metadata of the old and new states.
This commit adds a shortcut to MapperService.updateMappings() that compares
these mapping versions, and ignores the merge if they are equal.
This commit makes DateFieldMapper extend ParametrizedFieldMapper,
declaring its parameters explicitly. As well as changes to DateFieldMapper
itself, there are some changes to dynamic mapping code to ensure that
dynamically detected date formats are passed through to new date mapper
builders.
* Fix handling of final pipelines when destination is changed (#59522)
This change fixes final pipelines if destination index is changed during pipeline run:
-final pipelines can't change destination anymore, exception is thrown if they try to
-if request/default pipeline changes destination final pipeline from old index won't be executed
-if request/default pipeline changes destination and new index has final pipeline it will be executed
-default pipeline from new index won't be executed
Additionally TransportBulkAction.resolvePipelines was moved to IngestService as it's needed for resolving pipelines from new index. Tests were moved accordingly.
Closes#57968
After the work to retry transient replication failures, the local and
global checkpoint test metadata can be incremented on a different thread
than the test thread. This appears to introduce an extremely rare
scenario where this data is not visible for later test assertions. This
commit fixes the issue by using synchronized maps.
Adds a hard_bounds parameter to explicitly limit the buckets that a histogram
can generate. This is especially useful in case of open ended ranges that can
produce a very large number of buckets.
This adds a number of new optional parameters to Parameter, including:
* custom serialization (to handle analyzers)
* deprecated parameter names
* parameter validation
* allowing default values to be based on the values of other parameters
We preserve the previous serialization format of CompletionFieldMapper,
always emitting most fields, in order to meet mapping checks in mixed
version clusters, where the mapper service will check that mappings have
been correctly parsed and updated by checking their serialized outputs.
Eclipse was confused by #59583. It can't see a the public inner
interface within the superclass. This time. Usually that is fine, but
the Eclipse gods don't like this particular code, I guess.
The MappedFieldType#updateMeta method was used for testing equality checks, but we
no longer need these after #59212 , so we can remove this method and make meta final.
Trying to queue up snapshot deletes by blocking the delete of the latest
index-N doesn't work here. The first delete will block on the delete operation
but only do so after having already written the updated repository data.
Since that repository data will contain no snapshots, the subsequent deletes for
`*` will just fall through and complete instead of queue up.
=> Fixed by simply waiting on all files on master so that we block before updating
the repository data and get to test the queueing of equivalent operations
closes#59608
Previously the concrete type parameters for the MappedFieldType didn't always
match those for the FieldMapper. This PR updates the mappers so that the type
parameters always match, which makes the design easier to follow.
A number of obvious possible simplifications that also improve efficiency
in some cases (better empty collection handling and size hint use).
Also, added a shortcut for writing and reading immutable open maps that
can be used to dry up additional spots.