Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Track Lucene field usage #74227

Merged
merged 51 commits into from
Jul 14, 2021
Merged

Track Lucene field usage #74227

merged 51 commits into from
Jul 14, 2021

Conversation

ywelsch
Copy link
Contributor

@ywelsch ywelsch commented Jun 17, 2021

Adds a field usage API that reports shard-level statistics about which Lucene fields have been accessed, and which parts of the Lucene data structures have been accessed.

Field usage statistics are automatically captured when queries are runnning on a cluster. A shard-level search request that accesses a given field, even if multiple times during that request, is counted as a single use.

$ GET /my-index-000001/_field_usage_stats

{
    "_shards": {
        "total": 1,
        "successful": 1,
        "failed": 0
    },
    "my-index-000001": {
        "shards": [
            {
                "tracking_id": "MpOl0QlTQ4SYYhEe6KgJoQ",
                "tracking_started_at_millis": 1625558985010,
                "routing": {
                    "state": "STARTED",
                    "primary": true,
                    "node": "gA6KeeVzQkGURFCUyV-e8Q",
                    "relocating_node": null
                },
                "stats" : {
                    "all_fields": {
                        "any": "6", <1>
                        "inverted_index": {
                            "terms" : 1,
                            "postings" : 1,
                            "proximity" : 1, <2>
                            "positions" : 0,
                            "term_frequencies" : 1,
                            "offsets" : 0,
                            "payloads" : 0
                        },
                        "stored_fields" : 2,
                        "doc_values" : 1,
                        "points" : 0,
                        "norms" : 1,
                        "term_vectors" : 0
                    },
                    "fields": {
                        "_id": {
                            "any" : 1,
                            "inverted_index": {
                                "terms" : 1,
                                "postings" : 1,
                                "proximity" : 1,
                                "positions" : 0,
                                "term_frequencies" : 1,
                                "offsets" : 0,
                                "payloads" : 0
                            },
                            "stored_fields" : 1,
                            "doc_values" : 0,
                            "points" : 0,
                            "norms" : 0,
                            "term_vectors" : 0
                        },
                        "_source": {...},
                        "context": {...},
                        "message.keyword": {...}
                    }
                }
            }
        ]
    }
}

I've benchmarked this change on the http_logs track and could not see any substantial changes in performance. I think we will only get full validation of this by merging this change and having our nightly benchmarks catch any changes in performance.

Relates #73944

Copy link
Contributor

@jpountz jpountz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I like how you are checking the flags that are used in TermsEnum#postings rather than wrapping the PostingsEnum, which helps avoid adding a wrapper class over something that is called in tight loops.

Do we need to actually check whether impacts are used? As a Lucene developer, I'd be curious about this, but I don't know if users would be since this isn't actionable (Lucene doesn't allow you to disable indexing impacts).

@ywelsch
Copy link
Contributor Author

ywelsch commented Jun 17, 2021

Do we need to actually check whether impacts are used? As a Lucene developer, I'd be curious about this, but I don't know if users would be since this isn't actionable (Lucene doesn't allow you to disable indexing impacts).

Good point. I wasn't sure whether those were configurable or not. I've removed them.

@ywelsch ywelsch added :Search/Search Search-related issues that do not fall into other categories >enhancement v7.14.0 v8.0.0 labels Jun 18, 2021
@ywelsch ywelsch marked this pull request as ready for review June 18, 2021 13:44
@elasticmachine elasticmachine added the Team:Search Meta label for search team label Jun 18, 2021
@elasticmachine
Copy link
Collaborator

Pinging @elastic/es-search (Team:Search)

@jpountz
Copy link
Contributor

jpountz commented Jun 18, 2021

If I read the PR correctly, it currently returns raw counts which are impacted by the number of shards. E.g. a 2-shards index would have 2x the values of a single-shard index. Should we automatically divide the counts for one index by the number of shards it has so that numbers across different indices are easier to compare?

@@ -615,8 +616,11 @@ private GetResult getFromTranslog(Get get, Translog.Index index, MappingLookup m
final Engine.Searcher searcher = new Engine.Searcher("realtime_get", ElasticsearchDirectoryReader.wrap(inMemoryReader, shardId),
config().getSimilarity(), config().getQueryCache(), config().getQueryCachingPolicy(), inMemoryReader);
final Searcher wrappedSearcher = searcherWrapper.apply(searcher);
if (wrappedSearcher == searcher) {
searcher.close();
// TODO: we always wrap with field usage tracking, so we need to find another way to determine if we can safely do this
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@dnhatn would love to have your thoughts on this ^^

I'm wondering if we could have TranslogLeafReader somehow embedded in SingleDocDirectoryReader so that we always apply the wrapper, but then lazily load the InMemoryLeafReader when there is access to more than what the TranslogLeafReader can offer.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've built a solution for this. I won't push it to this PR though as it is quite a big change to how TranslogLeafReader works. Will make a separate PR tomorrow if test show all is fine with that solution.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've opened #74722 for this.

Copy link
Contributor

@jpountz jpountz left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This looks good to me overall.

One question I wonder about is whether the response should return stats per-shard instead of merging them. If we were to dogfood this by indexing it into Elasticsearch to track field usage over time, having per-shard information would be better as we could more easily figure when resets happen due to shard relocation or node shutdowns. Said otherwise this could be treated as a source of metrics where the index name, shard number, field name and data structure are dimensions, and the number of times it's been used is a counter.

"doc_values" : 1,
"points" : 0,
"norms" : 1,
"term_vectors" : 0
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder if we should have an object for all data structures for consitency, e.g.

        "all_fields": {
            "any": "6",
            "inverted_index": {
                "terms" : 1,
                "postings" : 1,
                "proximity" : 1,
                "positions" : 0,
                "term_frequencies" : 1,
                "offsets" : 0,
                "payloads" : 0
            },
            "stored_fields" : {
                "any": 2
            },
            [...]

@ywelsch
Copy link
Contributor Author

ywelsch commented Jun 29, 2021

One question I wonder about is whether the response should return stats per-shard instead of merging them. If we were to dogfood this by indexing it into Elasticsearch to track field usage over time, having per-shard information would be better as we could more easily figure when resets happen due to shard relocation or node shutdowns. Said otherwise this could be treated as a source of metrics where the index name, shard number, field name and data structure are dimensions, and the number of times it's been used is a counter.

Good question. In that case, tracking per shard would not even be enough as we have multiple replicas for which we are already summing up the stats (those can come and go as well). It would have to be tracking per some kind of "shard copy".

We suffer here from the same issue as other existing stats, e.g., query_total in "search stats". Will think a bit more about it.

ywelsch added a commit that referenced this pull request Jul 1, 2021
Reading from translog during a realtime get requires special handling in some higher level components, e.g.
ShardGetService, where we're doing a bunch of tricks to extract other stored fields from the source. Another issue with
the current approach relates to #74227 where we introduce a new "field usage tracking" directory wrapper that's always
applied, and we want to make sure that we can still quickly do realtime gets from translog without creating an in-memory
index of the document, even when this directory wrapper exists.

This PR introduces a directory reader that contains a single translog indexing operation. This can be used during a
realtime get to access documents that haven't been refreshed yet. In the normal case, all information relevant to resolve
the realtime get is mocked out to provide fast access to _id and _source. In case where more values are requested (e.g.
access to other stored fields) etc., this reader will index the document into an in-memory Lucene segment that is
created on-demand.

Relates #64504
ywelsch added a commit that referenced this pull request Jul 1, 2021
Reading from translog during a realtime get requires special handling in some higher level components, e.g.
ShardGetService, where we're doing a bunch of tricks to extract other stored fields from the source. Another issue with
the current approach relates to #74227 where we introduce a new "field usage tracking" directory wrapper that's always
applied, and we want to make sure that we can still quickly do realtime gets from translog without creating an in-memory
index of the document, even when this directory wrapper exists.

This PR introduces a directory reader that contains a single translog indexing operation. This can be used during a
realtime get to access documents that haven't been refreshed yet. In the normal case, all information relevant to resolve
the realtime get is mocked out to provide fast access to _id and _source. In case where more values are requested (e.g.
access to other stored fields) etc., this reader will index the document into an in-memory Lucene segment that is
created on-demand.

Relates #64504
@ywelsch
Copy link
Contributor Author

ywelsch commented Jul 1, 2021

My current thinking is to change the API for now to not aggregate the data beyond the level of shard copies. As there is not a good identifier for shard copies (allocation id of a primary shard e.g. stays same after node restart), I would go with just a UUID for each shard instance, similar to how we have done it in the repository metering API (repository_ephemeral_id). This follows a similar idea s the repository metering API. Similar to the repository metering API, we can also keep a bit historical data on the node when a shard moves away to a different node, but otherwise (node restarts / crashes) will have to accept that the information is lost.

It would look like the following:

shards: [
  {
    "tracking_id" : "SOME_UUID",
    "tracking_started_at" : 123141414 // unix timestamp
    "tracking_stopped_at" : 123441414  // only available if shard stopped / moved away -> historical info
    "stats": {
      "terms" : 0,
      "proximity" : 0,
      "stored_fields" : 1,
      ...
  },
  {
    "tracking_id" : "SOME_OTHER_UUID",
    ...
  }
]

@ywelsch ywelsch requested a review from jpountz July 6, 2021 12:03
@ywelsch
Copy link
Contributor Author

ywelsch commented Jul 13, 2021

@jpountz this is ready for another review :)

docs/reference/indices/field-usage-stats.asciidoc Outdated Show resolved Hide resolved
docs/reference/indices/field-usage-stats.asciidoc Outdated Show resolved Hide resolved
docs/reference/indices/field-usage-stats.asciidoc Outdated Show resolved Hide resolved
@ywelsch ywelsch merged commit db814f4 into elastic:master Jul 14, 2021
ywelsch added a commit that referenced this pull request Jul 14, 2021
Adds a field usage API that reports shard-level statistics about which Lucene fields have been accessed, and which
parts of the Lucene data structures have been accessed.

Field usage statistics are automatically captured when queries are runnning on a cluster. A shard-level search request
that accesses a given field, even if multiple times during that request, is counted as a single use.
ywelsch added a commit that referenced this pull request Jul 14, 2021
Adapts BWC conditions after backporting #74227 to 7.x branch (targeting 7.15).
masseyke pushed a commit to masseyke/elasticsearch that referenced this pull request Jul 16, 2021
Adds a field usage API that reports shard-level statistics about which Lucene fields have been accessed, and which
parts of the Lucene data structures have been accessed.

Field usage statistics are automatically captured when queries are runnning on a cluster. A shard-level search request
that accesses a given field, even if multiple times during that request, is counted as a single use.
masseyke pushed a commit to masseyke/elasticsearch that referenced this pull request Jul 16, 2021
Adapts BWC conditions after backporting elastic#74227 to 7.x branch (targeting 7.15).
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
>enhancement :Search/Search Search-related issues that do not fall into other categories Team:Clients Meta label for clients team Team:Search Meta label for search team v7.15.0 v8.0.0-alpha1
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

6 participants