Skip to content

feat: Add METRICS_INFO command#141667

Merged
leontyevdv merged 43 commits intoelastic:mainfrom
leontyevdv:feat/metrics-info-information-retrieval-command
Feb 19, 2026
Merged

feat: Add METRICS_INFO command#141667
leontyevdv merged 43 commits intoelastic:mainfrom
leontyevdv:feat/metrics-info-information-retrieval-command

Conversation

@leontyevdv
Copy link
Copy Markdown
Contributor

Add METRICS_INFO command to retrieve metrics-related metadata.

The METRICS_INFO command returns one row per metric that matches all conditions and attaches metadata to it. In cases where the same metric is defined with different properties in different indices, the same metric may appear twice in the output.

Part of #139296

Add METRICS_INFO command to retrieve metrics-related metadata.

Part of elastic#139296
@leontyevdv leontyevdv requested review from a team, dnhatn and felixbarny February 2, 2026 15:28
@leontyevdv leontyevdv self-assigned this Feb 2, 2026
@leontyevdv leontyevdv added Team:Analytics Meta label for analytical engine team (ESQL/Aggs/Geo) :Analytics/ES|QL AKA ESQL Team:StorageEngine :StorageEngine/ES|QL Timeseries / metrics / PromQL / logsdb capabilities in ES|QL v9.4.0 labels Feb 2, 2026
@elasticsearchmachine
Copy link
Copy Markdown
Collaborator

Pinging @elastic/es-analytical-engine (Team:Analytics)

@elasticsearchmachine
Copy link
Copy Markdown
Collaborator

Pinging @elastic/es-storage-engine (Team:StorageEngine)

Add mata.unit to the k8s-mappings.json

Part of elastic#139296
info.metricTypes.add(fieldInfo.metricType());
}

info.dimensionFieldKeys.addAll(dimensionFields);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

So we're blindly adding all available dimensions to each metric? I thought we want to get the ones stored along each metric, per tsid.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

For METRICS_INFO, we collapse the per tsid result into a row per metric. If a metric has different sets of attributes for different tsids, we'll return the union of all attributes. But we still need to start with extracting the _tsids, and getting the metric name and dimensions for it. Not all metrics in a data stream will have the same set of dimensions.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Right, we discussed that with Kostas and I made the corresponding changes. Now we scan the docs (one doc per _tsid) and gather metadata from them.

Copy link
Copy Markdown
Member

@kkrik-es kkrik-es left a comment

Choose a reason for hiding this comment

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

What we want METRICS_INFO to do is scan documents based on the filters provided (if any) and retrieve one document per time series. From there, we check the fields and break them down to metrics and dimensions. These are used to populate the MetricFieldInfo that's included in the output.

Maybe I misunderstood it, but I think this PR just retrieves static information around metric and dimension fields. We already have this through fieldcaps; what we want is to see what dimensions out of the 100s (1000s) available are associated with each metric name, for the time series that are present based on the query filters.

# Conflicts:
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java
@leontyevdv leontyevdv marked this pull request as draft February 3, 2026 14:50
Regenerate ESQL parser to fix conflicts

Part of elastic#139296
info.metricTypes.add(fieldInfo.metricType());
}

info.dimensionFieldKeys.addAll(dimensionFields);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

For METRICS_INFO, we collapse the per tsid result into a row per metric. If a metric has different sets of attributes for different tsids, we'll return the union of all attributes. But we still need to start with extracting the _tsids, and getting the metric name and dimensions for it. Not all metrics in a data stream will have the same set of dimensions.

Change METRICS_INFO to scan documents with unique _tsid to retrieve
metrics and dimensions metadata from them.

Part of elastic#139296
* The operator maintains a hash of seen key values and only passes through rows
* where the key value hasn't been seen before.
*/
public class DistinctByOperator extends AbstractPageMappingOperator {
Copy link
Copy Markdown
Contributor Author

@leontyevdv leontyevdv Feb 4, 2026

Choose a reason for hiding this comment

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

We needed a way to deduplicate rows by _tsid. The existing aggregation operators (FirstDocIdGroupingAggregatorFunction) operate within STATS and produce aggregated results. Here we need filtering only. Influenced by FilterOperator.

boolean hasTimeSeries = input.anyMatch(p -> p instanceof UnresolvedRelation ur && ur.indexMode() == IndexMode.TIME_SERIES);
boolean hasInfoCommand = input.anyMatch(p -> p instanceof MetricsInfo);

if (hasAggregate == false && hasTimeSeries && hasInfoCommand == false) {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

The query TS k8s | METRICS_INFO | STATS metric_count = COUNT_DISTINCT(metric_name) was failing with:

[STATS metric_count = COUNT_DISTINCT(metric_name)] requires the [@timestamp] field, 
which was either not present in the source index, or has been dropped or renamed

After METRICS_INFO, the data is no longer time series data but metadata with a fixed set of output columns. The @timestamp field doesn't exist in the METRICS_INFO output so we are defaulting to Aggregate instead of TimeSeriesAggregate.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Ah, now I know where to fix this for PROMQL, where I'm seeing the same issue, thanks!

# Conflicts:
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java
@leontyevdv leontyevdv marked this pull request as ready for review February 4, 2026 16:17
# Conflicts:
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.interp
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/PlanWritables.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java
Add INTERMEDIATE reduction
Copy link
Copy Markdown

@kpatticha kpatticha left a comment

Choose a reason for hiding this comment

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

Thanks for working on this, it will definitely help us improve the UI metrics experience within Discover.

While testing the PR locally across different scenarios, I noticed that when using METRICS_INFO with histograms, no data was returned. Is this expected behavior?

Here are the responses for reference:
https://gist.github.com/kpatticha/e3e747b72d6c487a7c2f242e98887a35

@leontyevdv
Copy link
Copy Markdown
Contributor Author

Thanks for working on this, it will definitely help us improve the UI metrics experience within Discover.

While testing the PR locally across different scenarios, I noticed that when using METRICS_INFO with histograms, no data was returned. Is this expected behavior?

Here are the responses for reference: https://gist.github.com/kpatticha/e3e747b72d6c487a7c2f242e98887a35

Hi @kpatticha! Thank you for testing this! Yes, this is not currently supported. Histograms are never leaf nodes in the synthetic source JSON that I use to collect metadata. I'll fix that. Thanks for spotting this!

Add histogram support
# Conflicts:
#	x-pack/plugin/esql/src/main/antlr/EsqlBaseLexer.tokens
#	x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.tokens
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.interp
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java
#	x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/approximation/ApproximationSupportTests.java
Merge main and resolve conflicts
@leontyevdv
Copy link
Copy Markdown
Contributor Author

Hey @kpatticha ! I've added support for all types of histograms. Thanks again for testing the command.

* <li>{@code metric_name} – keyword (single-valued)</li>
* <li>{@code data_stream} – keyword (multi-valued); data stream names that have this metric
* with the same signature (backing index names are resolved to their parent data stream)</li>
* <li>{@code unit} – keyword (multi-valued when backing indices differ; may be null)</li>
Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Q: If I understand correctly, the unit is part of the signature metric. In what case would this become multi-valued, if the signature metric is expected to generate two rows?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Unit is multi-valued only if there are conflicting definitions within a data stream. If there are conflicting units for backing indices within the same data stream, the unit will be multi-valued.

Copy link
Copy Markdown
Contributor Author

@leontyevdv leontyevdv Feb 17, 2026

Choose a reason for hiding this comment

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

In the 2nd example here if the datastream was the same (k8s), the result would be a single line with multi-valued unit.

Here is another example:
Input: Data stream k8s, one metric request.size. Backing index .ds-k8s-2026.01.01-000001 has unit: "bytes"; backing index .ds-k8s-2026.02.01-000001 has unit: "kilobytes" (e.g. mapping changed over time).

Result: Same (metric_name, data_stream) → one row. The signature’s units set is {bytes, kilobytes}, so the unit column is multi-valued.

metric_name data_stream unit metric_type field_type dimension_fields
request.size k8s [bytes, kilobytes] counter long [host]

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

Thanks for the explanation, I might doing something wrong but for the following

{
  ".ds-metrics-unit-conflict-2026.02.17-000002": {
    "mappings": {
      "_data_stream_timestamp": {
        "enabled": true
      },
      "properties": {
        "@timestamp": {
          "type": "date"
        },
        "metric": {
          "properties": {
            "name": {
              "type": "keyword",
              "time_series_dimension": true
            }
          }
        },
        "total_cost": {
          "type": "double",
          "meta": {
            "unit": "eur"
          },
          "time_series_metric": "counter"
        }
      }
    }
  },
  ".ds-metrics-unit-conflict-2026.02.17-000003": {
    "mappings": {
      "_data_stream_timestamp": {
        "enabled": true
      },
      "properties": {
        "@timestamp": {
          "type": "date"
        },
        "metric": {
          "properties": {
            "name": {
              "type": "keyword",
              "time_series_dimension": true
            }
          }
        },
        "total_cost": {
          "type": "double",
          "meta": {
            "unit": "eur"
          },
          "time_series_metric": "counter"
        }
      }
    }
  },
  ".ds-metrics-unit-conflict-2026.02.17-000001": {
    "mappings": {
      "_data_stream_timestamp": {
        "enabled": true
      },
      "properties": {
        "@timestamp": {
          "type": "date"
        },
        "metric": {
          "properties": {
            "name": {
              "type": "keyword",
              "time_series_dimension": true
            }
          }
        },
        "total_cost": {
          "type": "double",
          "meta": {
            "unit": "usd"
          },
          "time_series_metric": "counter"
        }
      }
    }
  }
}

Response

POST _query
{
    "query": "TS metrics-unit-conflict | METRICS_INFO"
}

I get two rows

  "values": [
    [
      "total_cost",
      "metrics-unit-conflict",
      "eur",
      "counter",
      "double",
      "metric.name"
    ],
    [
      "total_cost",
      "metrics-unit-conflict",
      "usd",
      "counter",
      "double",
      "metric.name"
    ]
  ]

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Yeah, this should produce a single row with the multi-valued unit. I'm checking this now, thank you!

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

@kpatticha , it was a bug with the final grouping of data. The initial grouping takes place on data nodes and the final one on a coordinator node. That final reduction was implemented wrongly and I've already pushed the fix. Thanks for finding it!

Copy link
Copy Markdown

@kpatticha kpatticha Feb 18, 2026

Choose a reason for hiding this comment

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

I pulled the latest changes and now I only get one row but with one unit

  "values": [
    [
      "total_cost",
      "metrics-unit-conflict",
      "usd",
      "counter",
      "double",
      "metric.name"
    ]
  ]

Here is the gist of what I used: https://gist.github.com/kpatticha/745d4bce942f04c6bdb35bdbf549387e

Copy link
Copy Markdown

Choose a reason for hiding this comment

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

same applies for the time_series_metric

Copy link
Copy Markdown
Contributor Author

@leontyevdv leontyevdv Feb 18, 2026

Choose a reason for hiding this comment

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

Hey @kpatticha ! Could you execute the following query, please, to check where your data had landed? I was grappling with the same issue yesterday and it turned out that all the data, even after _rollover, appeared in the 00001 backing index with USD. Check my setup: https://gist.github.com/leontyevdv/3222cf2b0996494b5706cc6afe49ddb6

It's not very convinient to wait 1m between rollover and the next POST operation but 1m is a minimum value allowed and I didn't find the other way to write data to the new backing index so far.

GET {{baseUrl}}/metrics-unit-conflict/_search
Authorization: {{auth}}
Content-Type: application/json

{
  "query": {
    "match_all": {}
  }
}

Copy link
Copy Markdown
Member

@dnhatn dnhatn left a comment

Choose a reason for hiding this comment

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

I've left some comments around null checks and error handling. The PR looks good - thanks, Dima!


public DistinctByOperator(int keyChannel) {
this.keyChannel = keyChannel;
this.seenKeys = new BytesRefHash();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we change to HashImplFactory.newBytesRefHash to use BytesRefSwissHash when available.

int rowCount = 0;
int[] positions = new int[page.getPositionCount()];

for (int p = 0; p < page.getPositionCount(); p++) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we also handle the ordinals block, but we should do it in a follow-up.

}
}

private final Map<MetricInfoKey, MetricInfo> metricsByKey = new LinkedHashMap<>();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

We should track memory usage for these maps, but we can address that in a follow-up.


/** INITIAL mode: extract metric metadata from _timeseries_metadata and _index blocks. */
private void addInputInitial(Page page) {
BytesRefBlock metadataSource = metadataSourceChannel >= 0 ? (BytesRefBlock) page.getBlock(metadataSourceChannel) : null;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

why do we need these channel checks?

/**
* Execution mode, mirroring the three-phase pattern used by aggregations.
*/
public enum Mode {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we add input/output attributes like we do in aggregate mode?

}
int start = block.getFirstValueIndex(position);
int count = block.getValueCount(position);
BytesRef scratch = new BytesRef();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we pass the scratch?

page.releaseBlocks();
}

private static String readSingleValue(BytesRefBlock block, int position) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we pass the scratch?

BytesRef indexScratch = new BytesRef();

for (int p = 0; p < page.getPositionCount(); p++) {
if (metadataSource == null || metadataSource.isNull(p)) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I think metadataSource should have values for every row?

}

private final Map<MetricInfoKey, MetricInfo> metricsByKey = new LinkedHashMap<>();
private final Map<MetricInfoKey, MetricInfo> finalMetricsByKey;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we use one map for all the modes?

}

private boolean isFinalMode() {
return finalChannels != null;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Can we pass the mode to the operator and use it instead? I think this covers both final and intermediate.

@kpatticha
Copy link
Copy Markdown

Hey @kpatticha ! I've added support for all types of histograms. Thanks again for testing the command.

Thanks for the quick fix. I was able to see the histograms :D

# Conflicts:
#	x-pack/plugin/esql/src/main/antlr/EsqlBaseLexer.tokens
#	x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.g4
#	x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.tokens
#	x-pack/plugin/esql/src/main/antlr/lexer/Expression.g4
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.interp
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/PlanWritables.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/telemetry/FeatureMetric.java
@leontyevdv
Copy link
Copy Markdown
Contributor Author

Hi @dnhatn ! Thanks for your review! I addressed all the comments including the "follow-up" ones:

DistinctByOperator

  1. Replaced new BytesRefHash() with HashImplFactory.newBytesRefHash(blockFactory)
  2. Added ordinals handling

MetricsInfoOperator

  1. Added circuit breaker tracking
  2. Replaced catch with try/finally and a success flag
  3. Added using Block.Builder.buildAll instead of building one by one
  4. Added passing BytesRef scratch to the methods instead of creating a new one every time
  5. Removed finalMetricsByKey and use only metricsByKey for or both INITIAL and FINAL
  6. Added MetricsInfoOperator.Mode (INITIAL, FINAL) and a mode field set in the constructors

MetricsInfoExec

  1. Added Input/output attributes as it is done in AggregateExec

@leontyevdv leontyevdv requested a review from dnhatn February 18, 2026 16:29
Copy link
Copy Markdown
Member

@dnhatn dnhatn left a comment

Choose a reason for hiding this comment

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

Great work - thanks Dima!

}

private void trackNewEntry() {
breaker.addEstimateBytesAndMaybeBreak(SHALLOW_SIZE, "MetricsInfoOperator");
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

We should track more than just the shallow size, but we can handle that in follow-up PRs since this one is already quite large.

# Conflicts:
#	x-pack/plugin/esql/src/main/antlr/EsqlBaseLexer.tokens
#	x-pack/plugin/esql/src/main/antlr/EsqlBaseParser.tokens
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.interp
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseLexer.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java
#	x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java
#	x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/approximation/ApproximationSupportTests.java
#	x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml
@leontyevdv leontyevdv merged commit be9ac27 into elastic:main Feb 19, 2026
35 checks passed
@felixbarny felixbarny linked an issue Feb 24, 2026 that may be closed by this pull request
leontyevdv added a commit that referenced this pull request Mar 2, 2026
Add a TS_INFO command - a time-series metadata introspection command built on top of METRICS_INFO (#141667). While METRICS_INFO returns one row per distinct metric across all time series, TS_INFO returns one row per (metric, time-series) combination, providing finer-grained visibility into individual time series.

TS_INFO produces the same 6 columns as METRICS_INFO plus an additional dimensions column.

---------

Co-authored-by: elasticsearchmachine <infra-root+elasticsearchmachine@elastic.co>
Co-authored-by: Liam Thompson <leemthompo@gmail.com>
tballison pushed a commit to tballison/elasticsearch that referenced this pull request Mar 3, 2026
Add a TS_INFO command - a time-series metadata introspection command built on top of METRICS_INFO (elastic#141667). While METRICS_INFO returns one row per distinct metric across all time series, TS_INFO returns one row per (metric, time-series) combination, providing finer-grained visibility into individual time series.

TS_INFO produces the same 6 columns as METRICS_INFO plus an additional dimensions column.

---------

Co-authored-by: elasticsearchmachine <infra-root+elasticsearchmachine@elastic.co>
Co-authored-by: Liam Thompson <leemthompo@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

:Analytics/ES|QL AKA ESQL >enhancement :StorageEngine/ES|QL Timeseries / metrics / PromQL / logsdb capabilities in ES|QL Team:Analytics Meta label for analytical engine team (ESQL/Aggs/Geo) Team:StorageEngine v9.4.0

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Time series information retrieval

6 participants