Browse Source

Update data stream lifecycle telemetry to track global retention (#112451)

Currently, the data stream lifecycle telemetry has the following
structure:

```
{
....
  "data_lifecycle" : {
    "available": true,
    "enabled": true,
    "count": 0,
    "default_rollover_used": true,
    "retention": {
        "minimum_millis": 0,
        "maximum_millis": 0,
        "average_millis": 0.0
    }
  }....
```

In the snippet above you can see that we track:

- The amount of data streams managed by the data stream lifecycle by `count`
- If the default rollover has been overwritten by `default_rollover_used`
- The min, max and average of the `data_retention` configured on a data stream level.

In this PR we propose the following extention:

```
....
  "data_lifecycle" : {
    "available": true,
    "enabled": true,
    "count": 0,
    "default_rollover_used": true,
    "effective_retention": { #https://github.com/elastic/dev/issues/2537
        "retained_data_streams": 5,
        "minimum_millis": 0, # Only if retained data streams > 1
        "maximum_millis": 0,
        "average_millis": 0.0
    },
    "data_retention": {
        "configured_data_streams": 5,
        "minimum_millis": 0, # Only if retained data streams > 1
        "maximum_millis": 0,
        "average_millis": 0.0
    },
    "global_retention": {
      "default": {
         "defined": true/false,
	  "affected_data_streams": 0,
         "millis": 0 
      },
      "max": {
         "defined": true/false,
	  "affected_data_streams": 0,
         "millis": 0 
      }
    }
```

With this extension we are tracking:

- The amount of data streams managed by the data stream lifecycle by `count`
- If the default rollover has been overwritten by `default_rollover_used`
- The min, max and average of the `data_retention` configured on a data stream level and the number of data streams that have it configured. We add the min, max and avg only if there are data streams with data retention configuration to avoid messing with the stats in a dashboard.
- The min, max and average of the `effective_retention` and the number of data streams that are retained. We add the min, max and avg only if there are retained data streams to avoid messing with the stats in a dashboard.
- Global retention stats, if they are defined, if the number of the affected data streams and the actual value.

The above metrics allow us to answer questions like:

- How many data streams are affected by global retention.
- How big is the difference between the longest data retention compared to max global retention.
- How much does the effective retention diverging from the data retention, this will show the impact of the global retention.
Mary Gouseti 1 year ago
parent
commit
c1a2d390ef

+ 9 - 7
docs/changelog/111972.yaml

@@ -5,11 +5,13 @@ type: feature
 issues: []
 highlight:
   title: Add global retention in data stream lifecycle
-  body: "Data stream lifecycle now supports configuring retention on a cluster level,\
-    \ namely global retention. Global retention \nallows us to configure two different\
-    \ retentions:\n\n- `data_streams.lifecycle.retention.default` is applied to all\
-    \ data streams managed by the data stream lifecycle that do not have retention\n\
-    defined on the data stream level.\n- `data_streams.lifecycle.retention.max` is\
-    \ applied to all data streams managed by the data stream lifecycle and it allows\
-    \ any data stream \ndata to be deleted after the `max_retention` has passed."
+  body: |-
+    Data stream lifecycle now supports configuring retention on a cluster level,
+    namely global retention. Global retention \nallows us to configure two different
+    retentions:
+
+    - `data_streams.lifecycle.retention.default` is applied to all data streams managed
+    by the data stream lifecycle that do not have retention defined on the data stream level.
+    - `data_streams.lifecycle.retention.max` is applied to all data streams managed by the
+    data stream lifecycle and it allows any data stream \ndata to be deleted after the `max_retention` has passed.
   notable: true

+ 29 - 0
docs/changelog/112451.yaml

@@ -0,0 +1,29 @@
+pr: 112451
+summary: Update data stream lifecycle telemetry to track global retention
+area: Data streams
+type: breaking
+issues: []
+breaking:
+  title: Update data stream lifecycle telemetry to track global retention
+  area: REST API
+  details: |-
+    In this release we introduced global retention settings that fulfil the following criteria:
+
+    - a data stream managed by the data stream lifecycle,
+    - a data stream that is not an internal data stream.
+
+    As a result, we defined different types of retention:
+
+    - **data retention**: the retention configured on data stream level by the data stream user or owner
+    - **default global retention:** the retention configured by an admin on a cluster level and applied to any
+    data stream that doesn't have data retention and fulfils the criteria.
+    - **max global retention:** the retention configured by an admin to guard against having long retention periods.
+    Any data stream that fulfills the criteria will adhere to the data retention unless it exceeds the max retention,
+    in which case the max global retention applies.
+    - **effective retention:** the retention that applies on the data stream that fulfill the criteria at a given moment
+    in time. It takes into consideration all the retention above and resolves it to the retention that will take effect.
+
+    Considering the above changes, having a field named `retention` in the usage API was confusing. For this reason, we
+    renamed it to `data_retention` and added telemetry about the other configurations too.
+  impact: Users that use the field `data_lifecycle.retention` should use the `data_lifecycle.data_retention`
+  notable: false

+ 13 - 4
docs/reference/rest-api/usage.asciidoc

@@ -377,10 +377,19 @@ GET /_xpack/usage
     "enabled": true,
     "count": 0,
     "default_rollover_used": true,
-    "retention": {
-        "minimum_millis": 0,
-        "maximum_millis": 0,
-        "average_millis": 0.0
+    "data_retention": {
+      "configured_data_streams": 0
+    },
+    "effective_retention": {
+      "retained_data_streams": 0
+    },
+    "global_retention": {
+      "default": {
+        "defined": false
+      },
+      "max": {
+        "defined": false
+      }
     }
   },
   "data_tiers" : {

+ 2 - 1
server/src/main/java/org/elasticsearch/TransportVersions.java

@@ -206,8 +206,9 @@ public class TransportVersions {
     public static final TransportVersion ESQL_ADD_INDEX_MODE_CONCRETE_INDICES = def(8_736_00_0);
     public static final TransportVersion UNASSIGNED_PRIMARY_COUNT_ON_CLUSTER_HEALTH = def(8_737_00_0);
     public static final TransportVersion ESQL_AGGREGATE_EXEC_TRACKS_INTERMEDIATE_ATTRS = def(8_738_00_0);
-
     public static final TransportVersion CCS_TELEMETRY_STATS = def(8_739_00_0);
+    public static final TransportVersion GLOBAL_RETENTION_TELEMETRY = def(8_740_00_0);
+
     /*
      * STOP! READ THIS FIRST! No, really,
      *        ____ _____ ___  ____  _        ____  _____    _    ____    _____ _   _ ___ ____    _____ ___ ____  ____ _____ _

+ 106 - 41
x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/core/action/DataStreamLifecycleUsageTransportActionIT.java

@@ -14,12 +14,14 @@ import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.ClusterStateUpdateTask;
 import org.elasticsearch.cluster.metadata.DataStream;
 import org.elasticsearch.cluster.metadata.DataStreamAlias;
+import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionSettings;
 import org.elasticsearch.cluster.metadata.DataStreamLifecycle;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.XContentHelper;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.IndexMode;
@@ -34,11 +36,13 @@ import org.elasticsearch.xcontent.XContentType;
 import org.elasticsearch.xpack.core.XPackClientPlugin;
 import org.junit.After;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Function;
 
@@ -64,24 +68,58 @@ public class DataStreamLifecycleUsageTransportActionIT extends ESIntegTestCase {
             return clusterStateBuilder.build();
         });
         updateClusterSettings(
-            Settings.builder().put(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING.getKey(), (String) null)
+            Settings.builder()
+                .putNull(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING.getKey())
+                .putNull(DataStreamGlobalRetentionSettings.DATA_STREAMS_DEFAULT_RETENTION_SETTING.getKey())
         );
     }
 
     @SuppressWarnings("unchecked")
     public void testAction() throws Exception {
-        assertUsageResults(0, 0, 0, 0.0, true);
-        AtomicLong totalCount = new AtomicLong(0);
-        AtomicLong countLifecycleWithRetention = new AtomicLong(0);
+        // test empty results
+        {
+            Map<String, Object> map = getLifecycleUsage();
+            assertThat(map.get("available"), equalTo(true));
+            assertThat(map.get("enabled"), equalTo(true));
+            assertThat(map.get("count"), equalTo(0));
+            assertThat(map.get("default_rollover_used"), equalTo(true));
+
+            Map<String, Object> dataRetentionMap = (Map<String, Object>) map.get("data_retention");
+            assertThat(dataRetentionMap.size(), equalTo(1));
+            assertThat(dataRetentionMap.get("configured_data_streams"), equalTo(0));
+
+            Map<String, Object> effectiveRetentionMap = (Map<String, Object>) map.get("effective_retention");
+            assertThat(effectiveRetentionMap.size(), equalTo(1));
+            assertThat(effectiveRetentionMap.get("retained_data_streams"), equalTo(0));
+
+            Map<String, Object> globalRetentionMap = (Map<String, Object>) map.get("global_retention");
+            assertThat(globalRetentionMap.get("max"), equalTo(Map.of("defined", false)));
+            assertThat(globalRetentionMap.get("default"), equalTo(Map.of("defined", false)));
+        }
+
+        // Keep track of the data streams created
+        AtomicInteger dataStreamsWithLifecycleCount = new AtomicInteger(0);
+        AtomicInteger dataStreamsWithRetentionCount = new AtomicInteger(0);
+        AtomicInteger dataStreamsWithDefaultRetentionCount = new AtomicInteger(0);
+
         AtomicLong totalRetentionTimes = new AtomicLong(0);
         AtomicLong minRetention = new AtomicLong(Long.MAX_VALUE);
         AtomicLong maxRetention = new AtomicLong(Long.MIN_VALUE);
+
         boolean useDefaultRolloverConfig = randomBoolean();
         if (useDefaultRolloverConfig == false) {
             updateClusterSettings(
                 Settings.builder().put(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING.getKey(), "min_docs=33")
             );
         }
+        TimeValue defaultRetention = TimeValue.timeValueDays(10);
+        boolean useDefaultRetention = randomBoolean();
+        if (useDefaultRetention) {
+            updateClusterSettings(
+                Settings.builder()
+                    .put(DataStreamGlobalRetentionSettings.DATA_STREAMS_DEFAULT_RETENTION_SETTING.getKey(), defaultRetention.getStringRep())
+            );
+        }
         /*
          * We now add a number of simulated data streams to the cluster state. Some have lifecycles, some don't. The ones with lifecycles
          * have varying retention periods. After adding them, we make sure the numbers add up.
@@ -89,19 +127,25 @@ public class DataStreamLifecycleUsageTransportActionIT extends ESIntegTestCase {
         updateClusterState(clusterState -> {
             Metadata.Builder metadataBuilder = Metadata.builder(clusterState.metadata());
             Map<String, DataStream> dataStreamMap = new HashMap<>();
-            for (int dataStreamCount = 0; dataStreamCount < randomInt(200); dataStreamCount++) {
-                boolean hasLifecycle = randomBoolean();
+            boolean atLeastOne = false;
+            for (int dataStreamCount = 0; dataStreamCount < randomIntBetween(1, 200); dataStreamCount++) {
+                boolean hasLifecycle = randomBoolean() || atLeastOne == false;
                 DataStreamLifecycle lifecycle;
+                boolean systemDataStream = rarely();
                 if (hasLifecycle) {
                     if (randomBoolean()) {
                         lifecycle = new DataStreamLifecycle(null, null, null);
-                        totalCount.incrementAndGet();
+                        dataStreamsWithLifecycleCount.incrementAndGet();
+                        if (useDefaultRetention && systemDataStream == false) {
+                            dataStreamsWithDefaultRetentionCount.incrementAndGet();
+                        }
+                        atLeastOne = true;
                     } else {
                         long retentionMillis = randomLongBetween(1000, 100000);
-                        boolean isEnabled = randomBoolean();
+                        boolean isEnabled = randomBoolean() || atLeastOne == false;
                         if (isEnabled) {
-                            totalCount.incrementAndGet();
-                            countLifecycleWithRetention.incrementAndGet();
+                            dataStreamsWithLifecycleCount.incrementAndGet();
+                            dataStreamsWithRetentionCount.incrementAndGet();
                             totalRetentionTimes.addAndGet(retentionMillis);
 
                             if (retentionMillis < minRetention.get()) {
@@ -110,6 +154,7 @@ public class DataStreamLifecycleUsageTransportActionIT extends ESIntegTestCase {
                             if (retentionMillis > maxRetention.get()) {
                                 maxRetention.set(retentionMillis);
                             }
+                            atLeastOne = true;
                         }
                         lifecycle = DataStreamLifecycle.newBuilder().dataRetention(retentionMillis).enabled(isEnabled).build();
                     }
@@ -121,7 +166,6 @@ public class DataStreamLifecycleUsageTransportActionIT extends ESIntegTestCase {
                     Index index = new Index(randomAlphaOfLength(60), randomAlphaOfLength(60));
                     indices.add(index);
                 }
-                boolean systemDataStream = randomBoolean();
                 boolean replicated = randomBoolean();
                 DataStream dataStream = new DataStream(
                     randomAlphaOfLength(50),
@@ -147,28 +191,59 @@ public class DataStreamLifecycleUsageTransportActionIT extends ESIntegTestCase {
             clusterStateBuilder.metadata(metadataBuilder);
             return clusterStateBuilder.build();
         });
-        int expectedMinimumRetention = minRetention.get() == Long.MAX_VALUE ? 0 : minRetention.intValue();
-        int expectedMaximumRetention = maxRetention.get() == Long.MIN_VALUE ? 0 : maxRetention.intValue();
-        double expectedAverageRetention = countLifecycleWithRetention.get() == 0
+
+        int retainedDataStreams = dataStreamsWithRetentionCount.get() + dataStreamsWithDefaultRetentionCount.get();
+
+        int expectedMinimumDataRetention = minRetention.get() == Long.MAX_VALUE ? 0 : minRetention.intValue();
+        int expectedMinimumEffectiveRetention = dataStreamsWithDefaultRetentionCount.get() > 0
+            ? (int) Math.min(minRetention.get(), defaultRetention.getMillis())
+            : expectedMinimumDataRetention;
+
+        int expectedMaximumDataRetention = maxRetention.get() == Long.MIN_VALUE ? 0 : maxRetention.intValue();
+        int expectedMaximumEffectiveRetention = dataStreamsWithDefaultRetentionCount.get() > 0
+            ? (int) Math.max(maxRetention.get(), defaultRetention.getMillis())
+            : expectedMaximumDataRetention;
+
+        double expectedAverageDataRetention = dataStreamsWithRetentionCount.get() == 0
             ? 0.0
-            : totalRetentionTimes.doubleValue() / countLifecycleWithRetention.get();
-        assertUsageResults(
-            totalCount.intValue(),
-            expectedMinimumRetention,
-            expectedMaximumRetention,
-            expectedAverageRetention,
-            useDefaultRolloverConfig
-        );
+            : totalRetentionTimes.doubleValue() / dataStreamsWithRetentionCount.get();
+        double expectedAverageEffectiveRetention = dataStreamsWithDefaultRetentionCount.get() > 0
+            ? (totalRetentionTimes.doubleValue() + dataStreamsWithDefaultRetentionCount.get() * defaultRetention.getMillis())
+                / retainedDataStreams
+            : expectedAverageDataRetention;
+
+        Map<String, Object> map = getLifecycleUsage();
+        assertThat(map.get("available"), equalTo(true));
+        assertThat(map.get("enabled"), equalTo(true));
+        assertThat(map.get("count"), equalTo(dataStreamsWithLifecycleCount.intValue()));
+        assertThat(map.get("default_rollover_used"), equalTo(useDefaultRolloverConfig));
+
+        Map<String, Object> dataRetentionMap = (Map<String, Object>) map.get("data_retention");
+        assertThat(dataRetentionMap.get("configured_data_streams"), equalTo(dataStreamsWithRetentionCount.get()));
+        if (dataStreamsWithRetentionCount.get() > 0) {
+            assertThat(dataRetentionMap.get("minimum_millis"), equalTo(expectedMinimumDataRetention));
+            assertThat(dataRetentionMap.get("maximum_millis"), equalTo(expectedMaximumDataRetention));
+            assertThat(dataRetentionMap.get("average_millis"), equalTo(expectedAverageDataRetention));
+        }
+
+        Map<String, Object> effectieRetentionMap = (Map<String, Object>) map.get("effective_retention");
+        assertThat(effectieRetentionMap.get("retained_data_streams"), equalTo(retainedDataStreams));
+        if (retainedDataStreams > 0) {
+            assertThat(effectieRetentionMap.get("minimum_millis"), equalTo(expectedMinimumEffectiveRetention));
+            assertThat(effectieRetentionMap.get("maximum_millis"), equalTo(expectedMaximumEffectiveRetention));
+            assertThat(effectieRetentionMap.get("average_millis"), equalTo(expectedAverageEffectiveRetention));
+        }
+
+        Map<String, Map<String, Object>> globalRetentionMap = (Map<String, Map<String, Object>>) map.get("global_retention");
+        assertThat(globalRetentionMap.get("max").get("defined"), equalTo(false));
+        assertThat(globalRetentionMap.get("default").get("defined"), equalTo(useDefaultRetention));
+        if (useDefaultRetention) {
+            assertThat(globalRetentionMap.get("default").get("affected_data_streams"), equalTo(dataStreamsWithDefaultRetentionCount.get()));
+            assertThat(globalRetentionMap.get("default").get("retention_millis"), equalTo((int) defaultRetention.getMillis()));
+        }
     }
 
-    @SuppressWarnings("unchecked")
-    private void assertUsageResults(
-        int count,
-        int minimumRetention,
-        int maximumRetention,
-        double averageRetention,
-        boolean defaultRolloverUsed
-    ) throws Exception {
+    private Map<String, Object> getLifecycleUsage() throws IOException {
         XPackUsageFeatureResponse response = safeGet(client().execute(DATA_STREAM_LIFECYCLE, new XPackUsageRequest(SAFE_AWAIT_TIMEOUT)));
         XContentBuilder builder = XContentFactory.jsonBuilder();
         builder = response.getUsage().toXContent(builder, ToXContent.EMPTY_PARAMS);
@@ -177,17 +252,7 @@ public class DataStreamLifecycleUsageTransportActionIT extends ESIntegTestCase {
             true,
             XContentType.JSON
         );
-
-        Map<String, Object> map = tuple.v2();
-        assertThat(map.get("available"), equalTo(true));
-        assertThat(map.get("enabled"), equalTo(true));
-        assertThat(map.get("count"), equalTo(count));
-        assertThat(map.get("default_rollover_used"), equalTo(defaultRolloverUsed));
-        Map<String, Object> retentionMap = (Map<String, Object>) map.get("retention");
-        assertThat(retentionMap.size(), equalTo(3));
-        assertThat(retentionMap.get("minimum_millis"), equalTo(minimumRetention));
-        assertThat(retentionMap.get("maximum_millis"), equalTo(maximumRetention));
-        assertThat(retentionMap.get("average_millis"), equalTo(averageRetention));
+        return tuple.v2();
     }
 
     /*

+ 84 - 25
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/DataStreamLifecycleUsageTransportAction.java

@@ -12,9 +12,12 @@ import org.elasticsearch.action.admin.indices.rollover.RolloverConfiguration;
 import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.DataStream;
+import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention;
+import org.elasticsearch.cluster.metadata.DataStreamGlobalRetentionSettings;
 import org.elasticsearch.cluster.metadata.DataStreamLifecycle;
 import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.core.Tuple;
 import org.elasticsearch.injection.guice.Inject;
 import org.elasticsearch.protocol.xpack.XPackUsageRequest;
@@ -24,17 +27,22 @@ import org.elasticsearch.transport.TransportService;
 import org.elasticsearch.xpack.core.datastreams.DataStreamLifecycleFeatureSetUsage;
 
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.LongSummaryStatistics;
+import java.util.Map;
 
 public class DataStreamLifecycleUsageTransportAction extends XPackUsageFeatureTransportAction {
 
+    private final DataStreamGlobalRetentionSettings globalRetentionSettings;
+
     @Inject
     public DataStreamLifecycleUsageTransportAction(
         TransportService transportService,
         ClusterService clusterService,
         ThreadPool threadPool,
         ActionFilters actionFilters,
-        IndexNameExpressionResolver indexNameExpressionResolver
+        IndexNameExpressionResolver indexNameExpressionResolver,
+        DataStreamGlobalRetentionSettings globalRetentionSettings
     ) {
         super(
             XPackUsageFeatureAction.DATA_STREAM_LIFECYCLE.name(),
@@ -44,6 +52,7 @@ public class DataStreamLifecycleUsageTransportAction extends XPackUsageFeatureTr
             actionFilters,
             indexNameExpressionResolver
         );
+        this.globalRetentionSettings = globalRetentionSettings;
     }
 
     @Override
@@ -54,42 +63,92 @@ public class DataStreamLifecycleUsageTransportAction extends XPackUsageFeatureTr
         ActionListener<XPackUsageFeatureResponse> listener
     ) {
         final Collection<DataStream> dataStreams = state.metadata().dataStreams().values();
-        Tuple<Long, LongSummaryStatistics> stats = calculateStats(dataStreams);
-
-        long minRetention = stats.v2().getCount() == 0 ? 0 : stats.v2().getMin();
-        long maxRetention = stats.v2().getCount() == 0 ? 0 : stats.v2().getMax();
-        double averageRetention = stats.v2().getAverage();
-        RolloverConfiguration rolloverConfiguration = clusterService.getClusterSettings()
-            .get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING);
-        String rolloverConfigString = rolloverConfiguration.toString();
-        final DataStreamLifecycleFeatureSetUsage.LifecycleStats lifecycleStats = new DataStreamLifecycleFeatureSetUsage.LifecycleStats(
-            stats.v1(),
-            minRetention,
-            maxRetention,
-            averageRetention,
-            DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING.getDefault(null).toString().equals(rolloverConfigString)
+        DataStreamLifecycleFeatureSetUsage.LifecycleStats lifecycleStats = calculateStats(
+            dataStreams,
+            clusterService.getClusterSettings().get(DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING),
+            globalRetentionSettings.get()
         );
-
-        final DataStreamLifecycleFeatureSetUsage usage = new DataStreamLifecycleFeatureSetUsage(lifecycleStats);
-        listener.onResponse(new XPackUsageFeatureResponse(usage));
+        listener.onResponse(new XPackUsageFeatureResponse(new DataStreamLifecycleFeatureSetUsage(lifecycleStats)));
     }
 
     /**
-     * Counts the number of data streams that have a lifecycle configured (and enabled) and for
-     * the data streams that have a lifecycle it computes the min/max/average summary of the effective
-     * configured retention.
+     * Counts the number of data streams that have a lifecycle configured (and enabled),
+     * computes the min/max/average summary of the data and effective retention and tracks the usage of global retention.
      */
-    public static Tuple<Long, LongSummaryStatistics> calculateStats(Collection<DataStream> dataStreams) {
+    public static DataStreamLifecycleFeatureSetUsage.LifecycleStats calculateStats(
+        Collection<DataStream> dataStreams,
+        RolloverConfiguration rolloverConfiguration,
+        DataStreamGlobalRetention globalRetention
+    ) {
+        // Initialise counters of associated data streams
         long dataStreamsWithLifecycles = 0;
-        LongSummaryStatistics retentionStats = new LongSummaryStatistics();
+        long dataStreamsWithDefaultRetention = 0;
+        long dataStreamsWithMaxRetention = 0;
+
+        LongSummaryStatistics dataRetentionStats = new LongSummaryStatistics();
+        LongSummaryStatistics effectiveRetentionStats = new LongSummaryStatistics();
+
         for (DataStream dataStream : dataStreams) {
             if (dataStream.getLifecycle() != null && dataStream.getLifecycle().isEnabled()) {
                 dataStreamsWithLifecycles++;
+                // Track data retention
                 if (dataStream.getLifecycle().getDataStreamRetention() != null) {
-                    retentionStats.accept(dataStream.getLifecycle().getDataStreamRetention().getMillis());
+                    dataRetentionStats.accept(dataStream.getLifecycle().getDataStreamRetention().getMillis());
+                }
+                // Track effective retention
+                Tuple<TimeValue, DataStreamLifecycle.RetentionSource> effectiveDataRetentionWithSource = dataStream.getLifecycle()
+                    .getEffectiveDataRetentionWithSource(globalRetention, dataStream.isInternal());
+
+                // Track global retention usage
+                if (effectiveDataRetentionWithSource.v1() != null) {
+                    effectiveRetentionStats.accept(effectiveDataRetentionWithSource.v1().getMillis());
+                    if (effectiveDataRetentionWithSource.v2().equals(DataStreamLifecycle.RetentionSource.DEFAULT_GLOBAL_RETENTION)) {
+                        dataStreamsWithDefaultRetention++;
+                    }
+                    if (effectiveDataRetentionWithSource.v2().equals(DataStreamLifecycle.RetentionSource.MAX_GLOBAL_RETENTION)) {
+                        dataStreamsWithMaxRetention++;
+                    }
                 }
             }
         }
-        return new Tuple<>(dataStreamsWithLifecycles, retentionStats);
+        Map<String, DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats> globalRetentionStats = getGlobalRetentionStats(
+            globalRetention,
+            dataStreamsWithDefaultRetention,
+            dataStreamsWithMaxRetention
+        );
+        return new DataStreamLifecycleFeatureSetUsage.LifecycleStats(
+            dataStreamsWithLifecycles,
+            DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING.getDefault(null).equals(rolloverConfiguration),
+            DataStreamLifecycleFeatureSetUsage.RetentionStats.create(dataRetentionStats),
+            DataStreamLifecycleFeatureSetUsage.RetentionStats.create(effectiveRetentionStats),
+            globalRetentionStats
+        );
+    }
+
+    private static Map<String, DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats> getGlobalRetentionStats(
+        DataStreamGlobalRetention globalRetention,
+        long dataStreamsWithDefaultRetention,
+        long dataStreamsWithMaxRetention
+    ) {
+        if (globalRetention == null) {
+            return Map.of();
+        }
+        Map<String, DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats> globalRetentionStats = new HashMap<>();
+        if (globalRetention.defaultRetention() != null) {
+            globalRetentionStats.put(
+                DataStreamLifecycleFeatureSetUsage.LifecycleStats.DEFAULT_RETENTION_FIELD_NAME,
+                new DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats(
+                    dataStreamsWithDefaultRetention,
+                    globalRetention.defaultRetention()
+                )
+            );
+        }
+        if (globalRetention.maxRetention() != null) {
+            globalRetentionStats.put(
+                DataStreamLifecycleFeatureSetUsage.LifecycleStats.MAX_RETENTION_FIELD_NAME,
+                new DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats(dataStreamsWithMaxRetention, globalRetention.maxRetention())
+            );
+        }
+        return globalRetentionStats;
     }
 }

+ 149 - 34
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/datastreams/DataStreamLifecycleFeatureSetUsage.java

@@ -13,11 +13,15 @@ import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.xcontent.ToXContentFragment;
 import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.xpack.core.XPackFeatureSet;
 import org.elasticsearch.xpack.core.XPackField;
 
 import java.io.IOException;
+import java.util.LongSummaryStatistics;
+import java.util.Map;
 import java.util.Objects;
 
 public class DataStreamLifecycleFeatureSetUsage extends XPackFeatureSet.Usage {
@@ -55,13 +59,7 @@ public class DataStreamLifecycleFeatureSetUsage extends XPackFeatureSet.Usage {
     protected void innerXContent(XContentBuilder builder, Params params) throws IOException {
         super.innerXContent(builder, params);
         if (enabled) {
-            builder.field("count", lifecycleStats.dataStreamsWithLifecyclesCount);
-            builder.field("default_rollover_used", lifecycleStats.defaultRolloverUsed);
-            builder.startObject("retention");
-            builder.field("minimum_millis", lifecycleStats.minRetentionMillis);
-            builder.field("maximum_millis", lifecycleStats.maxRetentionMillis);
-            builder.field("average_millis", lifecycleStats.averageRetentionMillis);
-            builder.endObject();
+            lifecycleStats.toXContent(builder, params);
         }
     }
 
@@ -87,33 +85,53 @@ public class DataStreamLifecycleFeatureSetUsage extends XPackFeatureSet.Usage {
         return available == other.available && enabled == other.enabled && Objects.equals(lifecycleStats, other.lifecycleStats);
     }
 
-    public static class LifecycleStats implements Writeable {
-
-        public static final LifecycleStats INITIAL = new LifecycleStats(0, 0, 0, 0, true);
+    public static class LifecycleStats implements Writeable, ToXContentFragment {
 
+        public static final LifecycleStats INITIAL = new LifecycleStats(0, true, RetentionStats.NO_DATA, RetentionStats.NO_DATA, Map.of());
+        public static final String DEFAULT_RETENTION_FIELD_NAME = "default";
+        public static final String MAX_RETENTION_FIELD_NAME = "max";
         final long dataStreamsWithLifecyclesCount;
-        final long minRetentionMillis;
-        final long maxRetentionMillis;
-        final double averageRetentionMillis;
         final boolean defaultRolloverUsed;
+        final RetentionStats dataRetentionStats;
+        final RetentionStats effectiveRetentionStats;
+        final Map<String, GlobalRetentionStats> globalRetentionStats;
 
         public LifecycleStats(
             long dataStreamsWithLifecyclesCount,
-            long minRetention,
-            long maxRetention,
-            double averageRetention,
-            boolean defaultRolloverUsed
+            boolean defaultRolloverUsed,
+            RetentionStats dataRetentionStats,
+            RetentionStats effectiveRetentionStats,
+            Map<String, GlobalRetentionStats> globalRetentionStats
         ) {
             this.dataStreamsWithLifecyclesCount = dataStreamsWithLifecyclesCount;
-            this.minRetentionMillis = minRetention;
-            this.maxRetentionMillis = maxRetention;
-            this.averageRetentionMillis = averageRetention;
             this.defaultRolloverUsed = defaultRolloverUsed;
+            this.dataRetentionStats = dataRetentionStats;
+            this.effectiveRetentionStats = effectiveRetentionStats;
+            this.globalRetentionStats = globalRetentionStats;
         }
 
         public static LifecycleStats read(StreamInput in) throws IOException {
-            if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_9_X)) {
-                return new LifecycleStats(in.readVLong(), in.readVLong(), in.readVLong(), in.readDouble(), in.readBoolean());
+            if (in.getTransportVersion().onOrAfter(TransportVersions.GLOBAL_RETENTION_TELEMETRY)) {
+                return new LifecycleStats(
+                    in.readVLong(),
+                    in.readBoolean(),
+                    RetentionStats.read(in),
+                    RetentionStats.read(in),
+                    in.readMap(GlobalRetentionStats::new)
+                );
+            } else if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_9_X)) {
+                var dataStreamsWithLifecyclesCount = in.readVLong();
+                var minDataRetention = in.readVLong();
+                var maxDataRetention = in.readVLong();
+                var avgDataRetention = in.readDouble();
+                var defaultRolledOverUsed = in.readBoolean();
+                return new LifecycleStats(
+                    dataStreamsWithLifecyclesCount,
+                    defaultRolledOverUsed,
+                    new RetentionStats(dataStreamsWithLifecyclesCount, avgDataRetention, minDataRetention, maxDataRetention),
+                    RetentionStats.NO_DATA,
+                    Map.of()
+                );
             } else {
                 return INITIAL;
             }
@@ -121,11 +139,17 @@ public class DataStreamLifecycleFeatureSetUsage extends XPackFeatureSet.Usage {
 
         @Override
         public void writeTo(StreamOutput out) throws IOException {
-            if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_9_X)) {
+            if (out.getTransportVersion().onOrAfter(TransportVersions.GLOBAL_RETENTION_TELEMETRY)) {
+                out.writeVLong(dataStreamsWithLifecyclesCount);
+                out.writeBoolean(defaultRolloverUsed);
+                dataRetentionStats.writeTo(out);
+                effectiveRetentionStats.writeTo(out);
+                out.writeMap(globalRetentionStats, (o, v) -> v.writeTo(o));
+            } else if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_9_X)) {
                 out.writeVLong(dataStreamsWithLifecyclesCount);
-                out.writeVLong(minRetentionMillis);
-                out.writeVLong(maxRetentionMillis);
-                out.writeDouble(averageRetentionMillis);
+                out.writeVLong(dataRetentionStats.minMillis() == null ? 0 : dataRetentionStats.minMillis());
+                out.writeVLong(dataRetentionStats.maxMillis() == null ? 0 : dataRetentionStats.maxMillis());
+                out.writeDouble(dataRetentionStats.avgMillis() == null ? 0 : dataRetentionStats.avgMillis());
                 out.writeBoolean(defaultRolloverUsed);
             }
         }
@@ -134,10 +158,10 @@ public class DataStreamLifecycleFeatureSetUsage extends XPackFeatureSet.Usage {
         public int hashCode() {
             return Objects.hash(
                 dataStreamsWithLifecyclesCount,
-                minRetentionMillis,
-                maxRetentionMillis,
-                averageRetentionMillis,
-                defaultRolloverUsed
+                defaultRolloverUsed,
+                dataRetentionStats,
+                effectiveRetentionStats,
+                globalRetentionStats
             );
         }
 
@@ -148,10 +172,101 @@ public class DataStreamLifecycleFeatureSetUsage extends XPackFeatureSet.Usage {
             }
             LifecycleStats other = (LifecycleStats) obj;
             return dataStreamsWithLifecyclesCount == other.dataStreamsWithLifecyclesCount
-                && minRetentionMillis == other.minRetentionMillis
-                && maxRetentionMillis == other.maxRetentionMillis
-                && averageRetentionMillis == other.averageRetentionMillis
-                && defaultRolloverUsed == other.defaultRolloverUsed;
+                && defaultRolloverUsed == other.defaultRolloverUsed
+                && Objects.equals(dataRetentionStats, other.dataRetentionStats)
+                && Objects.equals(effectiveRetentionStats, other.effectiveRetentionStats)
+                && Objects.equals(globalRetentionStats, other.globalRetentionStats);
+        }
+
+        @Override
+        public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+            builder.field("count", dataStreamsWithLifecyclesCount);
+            builder.field("default_rollover_used", defaultRolloverUsed);
+
+            builder.startObject("data_retention");
+            builder.field("configured_data_streams", dataRetentionStats.dataStreamCount());
+            if (dataRetentionStats.dataStreamCount() > 0) {
+                builder.field("minimum_millis", dataRetentionStats.minMillis);
+                builder.field("maximum_millis", dataRetentionStats.maxMillis);
+                builder.field("average_millis", dataRetentionStats.avgMillis);
+            }
+            builder.endObject();
+
+            builder.startObject("effective_retention");
+            builder.field("retained_data_streams", effectiveRetentionStats.dataStreamCount());
+            if (effectiveRetentionStats.dataStreamCount() > 0) {
+                builder.field("minimum_millis", effectiveRetentionStats.minMillis);
+                builder.field("maximum_millis", effectiveRetentionStats.maxMillis);
+                builder.field("average_millis", effectiveRetentionStats.avgMillis);
+            }
+            builder.endObject();
+
+            builder.startObject("global_retention");
+            globalRetentionStatsToXContent(builder, params, LifecycleStats.DEFAULT_RETENTION_FIELD_NAME);
+            globalRetentionStatsToXContent(builder, params, LifecycleStats.MAX_RETENTION_FIELD_NAME);
+            builder.endObject();
+            return builder;
+        }
+
+        private void globalRetentionStatsToXContent(XContentBuilder builder, Params params, String retentionType) throws IOException {
+            builder.startObject(retentionType);
+            GlobalRetentionStats stats = globalRetentionStats.get(retentionType);
+            builder.field("defined", stats != null);
+            if (stats != null) {
+                builder.field("affected_data_streams", stats.dataStreamCount());
+                builder.field("retention_millis", stats.retention());
+            }
+            builder.endObject();
+        }
+    }
+
+    public record RetentionStats(long dataStreamCount, Double avgMillis, Long minMillis, Long maxMillis) implements Writeable {
+
+        static final RetentionStats NO_DATA = new RetentionStats(0, null, null, null);
+
+        public static RetentionStats create(LongSummaryStatistics statistics) {
+            if (statistics.getCount() == 0) {
+                return NO_DATA;
+            }
+            return new RetentionStats(statistics.getCount(), statistics.getAverage(), statistics.getMin(), statistics.getMax());
+        }
+
+        public static RetentionStats read(StreamInput in) throws IOException {
+            long dataStreamCount = in.readVLong();
+            if (dataStreamCount == 0) {
+                return NO_DATA;
+            }
+            double avgMillis = in.readDouble();
+            long minMillis = in.readVLong();
+            long maxMillis = in.readVLong();
+            return new RetentionStats(dataStreamCount, avgMillis, minMillis, maxMillis);
+        }
+
+        @Override
+        public void writeTo(StreamOutput out) throws IOException {
+            out.writeVLong(dataStreamCount);
+            if (dataStreamCount > 0) {
+                out.writeDouble(avgMillis);
+                out.writeVLong(minMillis);
+                out.writeVLong(maxMillis);
+            }
+        }
+    }
+
+    public record GlobalRetentionStats(long dataStreamCount, long retention) implements Writeable {
+
+        public GlobalRetentionStats(long dataStreamCount, TimeValue retention) {
+            this(dataStreamCount, retention.getMillis());
+        }
+
+        public GlobalRetentionStats(StreamInput in) throws IOException {
+            this(in.readVLong(), in.readVLong());
+        }
+
+        @Override
+        public void writeTo(StreamOutput out) throws IOException {
+            out.writeVLong(dataStreamCount);
+            out.writeVLong(retention);
         }
     }
 }

+ 7 - 0
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/rest/action/RestXPackUsageAction.java

@@ -7,6 +7,7 @@
 package org.elasticsearch.xpack.core.rest.action;
 
 import org.elasticsearch.client.internal.node.NodeClient;
+import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.protocol.xpack.XPackUsageRequest;
 import org.elasticsearch.rest.BaseRestHandler;
 import org.elasticsearch.rest.RestRequest;
@@ -23,6 +24,7 @@ import org.elasticsearch.xpack.core.action.XPackUsageResponse;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Set;
 
 import static org.elasticsearch.rest.RestRequest.Method.GET;
 import static org.elasticsearch.rest.RestStatus.OK;
@@ -59,4 +61,9 @@ public class RestXPackUsageAction extends BaseRestHandler {
             }
         );
     }
+
+    @Override
+    public Set<String> supportedCapabilities() {
+        return Sets.union(super.supportedCapabilities(), Set.of("global_retention_telemetry"));
+    }
 }

+ 123 - 63
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/datastreams/DataStreamLifecycleFeatureSetUsageTests.java

@@ -7,21 +7,25 @@
 
 package org.elasticsearch.xpack.core.datastreams;
 
+import org.elasticsearch.action.admin.indices.rollover.RolloverConditions;
+import org.elasticsearch.action.admin.indices.rollover.RolloverConfiguration;
 import org.elasticsearch.cluster.metadata.DataStream;
+import org.elasticsearch.cluster.metadata.DataStreamGlobalRetention;
 import org.elasticsearch.cluster.metadata.DataStreamLifecycle;
 import org.elasticsearch.cluster.metadata.DataStreamTestHelper;
 import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.TimeValue;
-import org.elasticsearch.core.Tuple;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.test.AbstractWireSerializingTestCase;
-import org.elasticsearch.test.ESTestCase;
 
+import java.util.HashMap;
 import java.util.List;
-import java.util.LongSummaryStatistics;
+import java.util.Map;
 import java.util.UUID;
 
 import static org.elasticsearch.xpack.core.action.DataStreamLifecycleUsageTransportAction.calculateStats;
+import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
 
 public class DataStreamLifecycleFeatureSetUsageTests extends AbstractWireSerializingTestCase<DataStreamLifecycleFeatureSetUsage> {
@@ -32,68 +36,69 @@ public class DataStreamLifecycleFeatureSetUsageTests extends AbstractWireSeriali
             ? new DataStreamLifecycleFeatureSetUsage(
                 new DataStreamLifecycleFeatureSetUsage.LifecycleStats(
                     randomNonNegativeLong(),
-                    randomNonNegativeLong(),
-                    randomNonNegativeLong(),
-                    randomDouble(),
-                    randomBoolean()
+                    randomBoolean(),
+                    generateRetentionStats(),
+                    generateRetentionStats(),
+                    randomBoolean() ? Map.of() : Map.of("default", generateGlobalRetention(), "max", generateGlobalRetention())
                 )
             )
             : DataStreamLifecycleFeatureSetUsage.DISABLED;
     }
 
+    private static DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats generateGlobalRetention() {
+        return new DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats(randomNonNegativeLong(), randomNonNegativeLong());
+    }
+
+    private static DataStreamLifecycleFeatureSetUsage.RetentionStats generateRetentionStats() {
+        return randomBoolean()
+            ? DataStreamLifecycleFeatureSetUsage.RetentionStats.NO_DATA
+            : new DataStreamLifecycleFeatureSetUsage.RetentionStats(
+                randomNonNegativeLong(),
+                randomDoubleBetween(0.0, 110.0, false),
+                randomNonNegativeLong(),
+                randomNonNegativeLong()
+            );
+    }
+
     @Override
     protected DataStreamLifecycleFeatureSetUsage mutateInstance(DataStreamLifecycleFeatureSetUsage instance) {
         if (instance.equals(DataStreamLifecycleFeatureSetUsage.DISABLED)) {
             return new DataStreamLifecycleFeatureSetUsage(DataStreamLifecycleFeatureSetUsage.LifecycleStats.INITIAL);
         }
-        return switch (randomInt(4)) {
-            case 0 -> new DataStreamLifecycleFeatureSetUsage(
-                new DataStreamLifecycleFeatureSetUsage.LifecycleStats(
-                    randomValueOtherThan(instance.lifecycleStats.dataStreamsWithLifecyclesCount, ESTestCase::randomLong),
-                    instance.lifecycleStats.minRetentionMillis,
-                    instance.lifecycleStats.maxRetentionMillis,
-                    instance.lifecycleStats.averageRetentionMillis,
-                    instance.lifecycleStats.defaultRolloverUsed
-                )
+        var count = instance.lifecycleStats.dataStreamsWithLifecyclesCount;
+        var defaultRollover = instance.lifecycleStats.defaultRolloverUsed;
+        var dataRetentionStats = instance.lifecycleStats.dataRetentionStats;
+        var effectiveRetentionStats = instance.lifecycleStats.effectiveRetentionStats;
+        var maxRetention = instance.lifecycleStats.globalRetentionStats.get("max");
+        var defaultRetention = instance.lifecycleStats.globalRetentionStats.get("default");
+        switch (randomInt(5)) {
+            case 0 -> count += (count > 0 ? -1 : 1);
+            case 1 -> defaultRollover = defaultRollover == false;
+            case 2 -> dataRetentionStats = randomValueOtherThan(
+                dataRetentionStats,
+                DataStreamLifecycleFeatureSetUsageTests::generateRetentionStats
             );
-            case 1 -> new DataStreamLifecycleFeatureSetUsage(
-                new DataStreamLifecycleFeatureSetUsage.LifecycleStats(
-                    instance.lifecycleStats.dataStreamsWithLifecyclesCount,
-                    randomValueOtherThan(instance.lifecycleStats.minRetentionMillis, ESTestCase::randomLong),
-                    instance.lifecycleStats.maxRetentionMillis,
-                    instance.lifecycleStats.averageRetentionMillis,
-                    instance.lifecycleStats.defaultRolloverUsed
-                )
-            );
-            case 2 -> new DataStreamLifecycleFeatureSetUsage(
-                new DataStreamLifecycleFeatureSetUsage.LifecycleStats(
-                    instance.lifecycleStats.dataStreamsWithLifecyclesCount,
-                    instance.lifecycleStats.minRetentionMillis,
-                    randomValueOtherThan(instance.lifecycleStats.maxRetentionMillis, ESTestCase::randomLong),
-                    instance.lifecycleStats.averageRetentionMillis,
-                    instance.lifecycleStats.defaultRolloverUsed
-                )
-            );
-            case 3 -> new DataStreamLifecycleFeatureSetUsage(
-                new DataStreamLifecycleFeatureSetUsage.LifecycleStats(
-                    instance.lifecycleStats.dataStreamsWithLifecyclesCount,
-                    instance.lifecycleStats.minRetentionMillis,
-                    instance.lifecycleStats.maxRetentionMillis,
-                    randomValueOtherThan(instance.lifecycleStats.averageRetentionMillis, ESTestCase::randomDouble),
-                    instance.lifecycleStats.defaultRolloverUsed
-                )
+            case 3 -> effectiveRetentionStats = randomValueOtherThan(
+                effectiveRetentionStats,
+                DataStreamLifecycleFeatureSetUsageTests::generateRetentionStats
             );
-            case 4 -> new DataStreamLifecycleFeatureSetUsage(
-                new DataStreamLifecycleFeatureSetUsage.LifecycleStats(
-                    instance.lifecycleStats.dataStreamsWithLifecyclesCount,
-                    instance.lifecycleStats.minRetentionMillis,
-                    instance.lifecycleStats.maxRetentionMillis,
-                    instance.lifecycleStats.averageRetentionMillis,
-                    instance.lifecycleStats.defaultRolloverUsed == false
-                )
+            case 4 -> maxRetention = randomValueOtherThan(maxRetention, DataStreamLifecycleFeatureSetUsageTests::generateGlobalRetention);
+            case 5 -> defaultRetention = randomValueOtherThan(
+                defaultRetention,
+                DataStreamLifecycleFeatureSetUsageTests::generateGlobalRetention
             );
             default -> throw new RuntimeException("unreachable");
-        };
+        }
+        Map<String, DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats> map = new HashMap<>();
+        if (defaultRetention != null) {
+            map.put("default", defaultRetention);
+        }
+        if (maxRetention != null) {
+            map.put("max", maxRetention);
+        }
+        return new DataStreamLifecycleFeatureSetUsage(
+            new DataStreamLifecycleFeatureSetUsage.LifecycleStats(count, defaultRollover, dataRetentionStats, effectiveRetentionStats, map)
+        );
     }
 
     public void testLifecycleStats() {
@@ -112,7 +117,7 @@ public class DataStreamLifecycleFeatureSetUsageTests extends AbstractWireSeriali
                 1L,
                 null,
                 false,
-                new DataStreamLifecycle(new DataStreamLifecycle.Retention(TimeValue.timeValueMillis(1000)), null, true)
+                new DataStreamLifecycle(new DataStreamLifecycle.Retention(TimeValue.timeValueSeconds(50)), null, true)
             ),
             DataStreamTestHelper.newInstance(
                 randomAlphaOfLength(10),
@@ -120,7 +125,7 @@ public class DataStreamLifecycleFeatureSetUsageTests extends AbstractWireSeriali
                 1L,
                 null,
                 false,
-                new DataStreamLifecycle(new DataStreamLifecycle.Retention(TimeValue.timeValueMillis(100)), null, true)
+                new DataStreamLifecycle(new DataStreamLifecycle.Retention(TimeValue.timeValueMillis(150)), null, true)
             ),
             DataStreamTestHelper.newInstance(
                 randomAlphaOfLength(10),
@@ -128,7 +133,7 @@ public class DataStreamLifecycleFeatureSetUsageTests extends AbstractWireSeriali
                 1L,
                 null,
                 false,
-                new DataStreamLifecycle(new DataStreamLifecycle.Retention(TimeValue.timeValueMillis(5000)), null, false)
+                new DataStreamLifecycle(new DataStreamLifecycle.Retention(TimeValue.timeValueSeconds(5)), null, false)
             ),
             DataStreamTestHelper.newInstance(
                 randomAlphaOfLength(10),
@@ -140,15 +145,70 @@ public class DataStreamLifecycleFeatureSetUsageTests extends AbstractWireSeriali
             )
         );
 
-        Tuple<Long, LongSummaryStatistics> stats = calculateStats(dataStreams);
-        // 3 data streams with an enabled lifecycle
-        assertThat(stats.v1(), is(3L));
-        LongSummaryStatistics longSummaryStatistics = stats.v2();
-        assertThat(longSummaryStatistics.getMax(), is(1000L));
-        assertThat(longSummaryStatistics.getMin(), is(100L));
-        // only counting the ones with an effective retention in the summary statistics
-        assertThat(longSummaryStatistics.getCount(), is(2L));
-        assertThat(longSummaryStatistics.getAverage(), is(550.0));
+        // Test empty global retention
+        {
+            boolean useDefault = randomBoolean();
+            RolloverConfiguration rollover = useDefault
+                ? DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING.getDefault(Settings.EMPTY)
+                : new RolloverConfiguration(new RolloverConditions());
+            DataStreamLifecycleFeatureSetUsage.LifecycleStats stats = calculateStats(dataStreams, rollover, null);
+
+            assertThat(stats.dataStreamsWithLifecyclesCount, is(3L));
+            assertThat(stats.defaultRolloverUsed, is(useDefault));
+            // Data retention
+            assertThat(stats.dataRetentionStats.dataStreamCount(), is(2L));
+            assertThat(stats.dataRetentionStats.maxMillis(), is(50_000L));
+            assertThat(stats.dataRetentionStats.minMillis(), is(150L));
+            assertThat(stats.dataRetentionStats.avgMillis(), is(25_075.0));
+
+            assertThat(stats.effectiveRetentionStats.dataStreamCount(), is(2L));
+            assertThat(stats.effectiveRetentionStats.maxMillis(), is(50_000L));
+            assertThat(stats.effectiveRetentionStats.minMillis(), is(150L));
+            assertThat(stats.effectiveRetentionStats.avgMillis(), is(25_075.0));
+
+            assertThat(stats.globalRetentionStats, equalTo(Map.of()));
+        }
+
+        // Test with global retention
+        {
+            boolean useDefault = randomBoolean();
+            RolloverConfiguration rollover = useDefault
+                ? DataStreamLifecycle.CLUSTER_LIFECYCLE_DEFAULT_ROLLOVER_SETTING.getDefault(Settings.EMPTY)
+                : new RolloverConfiguration(new RolloverConditions());
+            TimeValue defaultRetention = TimeValue.timeValueSeconds(10);
+            TimeValue maxRetention = TimeValue.timeValueSeconds(20);
+            DataStreamLifecycleFeatureSetUsage.LifecycleStats stats = calculateStats(
+                dataStreams,
+                rollover,
+                new DataStreamGlobalRetention(defaultRetention, maxRetention)
+            );
+
+            assertThat(stats.dataStreamsWithLifecyclesCount, is(3L));
+            assertThat(stats.defaultRolloverUsed, is(useDefault));
+            // Data retention
+            assertThat(stats.dataRetentionStats.dataStreamCount(), is(2L));
+            assertThat(stats.dataRetentionStats.maxMillis(), is(50_000L));
+            assertThat(stats.dataRetentionStats.minMillis(), is(150L));
+            assertThat(stats.dataRetentionStats.avgMillis(), is(25_075.0));
+
+            // Effective retention
+            assertThat(stats.effectiveRetentionStats.dataStreamCount(), is(3L));
+            assertThat(stats.effectiveRetentionStats.maxMillis(), is(20_000L));
+            assertThat(stats.effectiveRetentionStats.minMillis(), is(150L));
+            assertThat(stats.effectiveRetentionStats.avgMillis(), is(10_050.0));
+
+            assertThat(
+                stats.globalRetentionStats,
+                equalTo(
+                    Map.of(
+                        "default",
+                        new DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats(1L, 10_000L),
+                        "max",
+                        new DataStreamLifecycleFeatureSetUsage.GlobalRetentionStats(1L, 20_000L)
+                    )
+                )
+            );
+        }
     }
 
     @Override

+ 38 - 12
x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/dlm/10_usage.yml

@@ -1,9 +1,12 @@
 ---
 "Test data stream lifecycle usage stats":
   - requires:
-      cluster_features: "gte_v8.11.0"
-      reason: "the data stream lifecycle stats were updated to the usage api in 8.11"
-      test_runner_features: "allowed_warnings"
+      reason: "Global retention telemetry was added in 8.16+"
+      test_runner_features: [ capabilities, allowed_warnings ]
+      capabilities:
+        - method: GET
+          path: /_xpack/usage
+          capabilities: [ 'global_retention_telemetry' ]
 
   - do:
       xpack.usage: {}
@@ -12,9 +15,19 @@
   - match: { data_lifecycle.enabled: true }
   - match: { data_lifecycle.count: 0 }
   - match: { data_lifecycle.default_rollover_used: true }
-  - match: { data_lifecycle.retention.minimum_millis: 0 }
-  - match: { data_lifecycle.retention.maximum_millis: 0 }
-  - match: { data_lifecycle.retention.average_millis: 0 }
+
+  - match: { data_lifecycle.data_retention.configured_data_streams: 0 }
+  - is_false: data_lifecycle.data_retention.minimum_millis
+  - is_false: data_lifecycle.data_retention.maximum_millis
+  - is_false: data_lifecycle.data_retention.average_millis
+
+  - match: { data_lifecycle.effective_retention.retained_data_streams: 0 }
+  - is_false: data_lifecycle.effective_retention.minimum_millis
+  - is_false: data_lifecycle.effective_retention.maximum_millis
+  - is_false: data_lifecycle.effective_retention.average_millis
+
+  - match: { data_lifecycle.global_retention.max.defined: false }
+  - match: { data_lifecycle.global_retention.default.defined: false }
 
   - do:
       allowed_warnings:
@@ -89,9 +102,15 @@
   - match: { data_lifecycle.enabled: true }
   - match: { data_lifecycle.count: 2 }
   - match: { data_lifecycle.default_rollover_used: true }
-  - match: { data_lifecycle.retention.minimum_millis: 432000000 }
-  - match: { data_lifecycle.retention.maximum_millis: 864000000 }
-  - match: { data_lifecycle.retention.average_millis: 648000000 }
+  - match: { data_lifecycle.data_retention.configured_data_streams: 2 }
+  - match: { data_lifecycle.data_retention.minimum_millis: 432000000 }
+  - match: { data_lifecycle.data_retention.maximum_millis: 864000000 }
+  - match: { data_lifecycle.data_retention.average_millis: 648000000 }
+
+  - match: { data_lifecycle.effective_retention.retained_data_streams: 2 }
+  - match: { data_lifecycle.effective_retention.minimum_millis: 432000000 }
+  - match: { data_lifecycle.data_retention.maximum_millis: 864000000 }
+  - match: { data_lifecycle.data_retention.average_millis: 648000000 }
 
   - do:
       indices.delete_data_stream:
@@ -106,6 +125,13 @@
   - match: { data_lifecycle.enabled: true }
   - match: { data_lifecycle.count: 1 }
   - match: { data_lifecycle.default_rollover_used: true }
-  - match: { data_lifecycle.retention.minimum_millis: 432000000 }
-  - match: { data_lifecycle.retention.maximum_millis: 432000000 }
-  - match: { data_lifecycle.retention.average_millis: 432000000 }
+
+  - match: { data_lifecycle.data_retention.configured_data_streams: 1 }
+  - match: { data_lifecycle.data_retention.minimum_millis: 432000000 }
+  - match: { data_lifecycle.data_retention.maximum_millis: 432000000 }
+  - match: { data_lifecycle.data_retention.average_millis: 432000000 }
+
+  - match: { data_lifecycle.effective_retention.retained_data_streams: 1 }
+  - match: { data_lifecycle.effective_retention.minimum_millis: 432000000 }
+  - match: { data_lifecycle.effective_retention.maximum_millis: 432000000 }
+  - match: { data_lifecycle.effective_retention.average_millis: 432000000 }