diff --git a/libs/arrow-spi/src/main/java/org/opensearch/arrow/spi/NativeAllocator.java b/libs/arrow-spi/src/main/java/org/opensearch/arrow/spi/NativeAllocator.java
index 2ffd1f1b132e2..425cc5795d358 100644
--- a/libs/arrow-spi/src/main/java/org/opensearch/arrow/spi/NativeAllocator.java
+++ b/libs/arrow-spi/src/main/java/org/opensearch/arrow/spi/NativeAllocator.java
@@ -36,7 +36,9 @@ public interface NativeAllocator extends Closeable {
PoolHandle getOrCreatePool(String poolName, long limit);
/**
- * Updates the limit of an existing pool.
+ * Updates the limit of an existing pool. Children of the pool allocator
+ * inherit the change automatically via Arrow's parent-cap check at
+ * allocation time — no notification SPI is needed.
*
* @param poolName logical pool name
* @param newLimit new maximum bytes for the pool
diff --git a/libs/arrow-spi/src/main/java/org/opensearch/arrow/spi/NativeAllocatorPoolConfig.java b/libs/arrow-spi/src/main/java/org/opensearch/arrow/spi/NativeAllocatorPoolConfig.java
index 53e0ae88fc595..98f991cb86704 100644
--- a/libs/arrow-spi/src/main/java/org/opensearch/arrow/spi/NativeAllocatorPoolConfig.java
+++ b/libs/arrow-spi/src/main/java/org/opensearch/arrow/spi/NativeAllocatorPoolConfig.java
@@ -30,6 +30,8 @@ public final class NativeAllocatorPoolConfig {
public static final String POOL_FLIGHT = "flight";
/** Pool name for ingest pipeline memory. */
public static final String POOL_INGEST = "ingest";
+ /** Pool name for query-execution memory (analytics-engine fragments and per-query allocators). */
+ public static final String POOL_QUERY = "query";
/** Setting key for the root allocator limit. */
public static final String SETTING_ROOT_LIMIT = "native.allocator.root.limit";
@@ -42,6 +44,10 @@ public final class NativeAllocatorPoolConfig {
public static final String SETTING_INGEST_MIN = "native.allocator.pool.ingest.min";
/** Setting key for the ingest pool maximum. */
public static final String SETTING_INGEST_MAX = "native.allocator.pool.ingest.max";
+ /** Setting key for the query pool minimum. */
+ public static final String SETTING_QUERY_MIN = "native.allocator.pool.query.min";
+ /** Setting key for the query pool maximum. */
+ public static final String SETTING_QUERY_MAX = "native.allocator.pool.query.max";
private NativeAllocatorPoolConfig() {}
}
diff --git a/libs/arrow-spi/src/test/java/org/opensearch/arrow/spi/NativeAllocatorPoolConfigTests.java b/libs/arrow-spi/src/test/java/org/opensearch/arrow/spi/NativeAllocatorPoolConfigTests.java
index f0060d47c5df9..a21ca8ff54943 100644
--- a/libs/arrow-spi/src/test/java/org/opensearch/arrow/spi/NativeAllocatorPoolConfigTests.java
+++ b/libs/arrow-spi/src/test/java/org/opensearch/arrow/spi/NativeAllocatorPoolConfigTests.java
@@ -15,6 +15,7 @@ public class NativeAllocatorPoolConfigTests extends OpenSearchTestCase {
public void testPoolConstants() {
assertEquals("flight", NativeAllocatorPoolConfig.POOL_FLIGHT);
assertEquals("ingest", NativeAllocatorPoolConfig.POOL_INGEST);
+ assertEquals("query", NativeAllocatorPoolConfig.POOL_QUERY);
}
public void testSettingKeys() {
@@ -22,6 +23,8 @@ public void testSettingKeys() {
assertEquals("native.allocator.pool.flight.max", NativeAllocatorPoolConfig.SETTING_FLIGHT_MAX);
assertEquals("native.allocator.pool.ingest.min", NativeAllocatorPoolConfig.SETTING_INGEST_MIN);
assertEquals("native.allocator.pool.ingest.max", NativeAllocatorPoolConfig.SETTING_INGEST_MAX);
+ assertEquals("native.allocator.pool.query.min", NativeAllocatorPoolConfig.SETTING_QUERY_MIN);
+ assertEquals("native.allocator.pool.query.max", NativeAllocatorPoolConfig.SETTING_QUERY_MAX);
}
public void testRootSettingKey() {
diff --git a/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/AllocationRejection.java b/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/AllocationRejection.java
new file mode 100644
index 0000000000000..c5dbb23681ed3
--- /dev/null
+++ b/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/AllocationRejection.java
@@ -0,0 +1,97 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.arrow.allocator;
+
+import org.apache.arrow.memory.OutOfMemoryException;
+import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException;
+
+import java.util.function.Supplier;
+
+/**
+ * Translates Arrow's {@link OutOfMemoryException} into OpenSearch's standard
+ * {@link OpenSearchRejectedExecutionException} at allocation boundaries.
+ *
+ *
Arrow throws {@code OutOfMemoryException} when an allocation would exceed
+ * a parent's {@code allocationLimit}. This is a per-request, recoverable
+ * condition — the caller should back off and retry — but Arrow's exception type
+ * is not recognized by OpenSearch's REST layer, so it surfaces to clients as a
+ * generic 500. {@link OpenSearchRejectedExecutionException} (a subclass of
+ * {@link java.util.concurrent.RejectedExecutionException}) maps to HTTP 429
+ * (Too Many Requests) and is the framework's standard rejection signal.
+ *
+ *
Apply this wrapper at per-request allocation sites — places where
+ * a failure represents "this request didn't fit" rather than "the framework
+ * failed to start". Examples:
+ *
+ * per-query child allocator creation in
+ * {@code DefaultPlanExecutor.executeInternal}
+ * per-VSR child allocator creation in
+ * {@code ArrowBufferPool.createChildAllocator}
+ * request-handler-side allocations in transport/RPC layers
+ *
+ *
+ * Do not wrap startup-time / lifetime-of-component
+ * allocations (e.g. transport server/client allocators created in
+ * {@code doStart}, service-level allocators created in plugin constructors).
+ * Those failing during node init is a configuration error, not a per-request
+ * back-pressure signal — they should propagate as the original exception.
+ */
+public final class AllocationRejection {
+
+ private AllocationRejection() {}
+
+ /**
+ * Runs {@code body} and translates any Arrow {@link OutOfMemoryException}
+ * into {@link OpenSearchRejectedExecutionException}.
+ *
+ * @param context short label included in the rejection message (typically the
+ * pool or operation name; e.g. "query-pool", "ingest-vsr")
+ * @param body the allocation site to invoke
+ * @param return type of {@code body}
+ * @return whatever {@code body} returns on success
+ * @throws OpenSearchRejectedExecutionException if {@code body} throws
+ * {@link OutOfMemoryException}; the original Arrow exception is
+ * attached as the cause
+ */
+ public static T wrap(String context, Supplier body) {
+ try {
+ return body.get();
+ } catch (OutOfMemoryException e) {
+ throw rejection(context, e);
+ }
+ }
+
+ /**
+ * Runs {@code body} and translates any Arrow {@link OutOfMemoryException}
+ * into {@link OpenSearchRejectedExecutionException}. Use this overload for
+ * void allocation sites.
+ *
+ * @param context short label included in the rejection message (typically the
+ * pool or operation name; e.g. "query-pool", "ingest-vsr")
+ * @param body the allocation site to invoke
+ * @throws OpenSearchRejectedExecutionException if {@code body} throws
+ * {@link OutOfMemoryException}; the original Arrow exception is
+ * attached as the cause
+ */
+ public static void wrap(String context, Runnable body) {
+ try {
+ body.run();
+ } catch (OutOfMemoryException e) {
+ throw rejection(context, e);
+ }
+ }
+
+ private static OpenSearchRejectedExecutionException rejection(String context, OutOfMemoryException cause) {
+ OpenSearchRejectedExecutionException rejection = new OpenSearchRejectedExecutionException(
+ "native memory allocation rejected at [" + context + "]: " + cause.getMessage()
+ );
+ rejection.initCause(cause);
+ return rejection;
+ }
+}
diff --git a/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/ArrowBasePlugin.java b/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/ArrowBasePlugin.java
index 73b222150b73b..bf016aa20c883 100644
--- a/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/ArrowBasePlugin.java
+++ b/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/ArrowBasePlugin.java
@@ -8,21 +8,21 @@
package org.opensearch.arrow.allocator;
-import org.opensearch.arrow.memory.ArrowAllocatorService;
-import org.opensearch.arrow.memory.DefaultArrowAllocatorService;
import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
import org.opensearch.cluster.metadata.IndexNameExpressionResolver;
import org.opensearch.cluster.service.ClusterService;
-import org.opensearch.common.inject.AbstractModule;
-import org.opensearch.common.inject.Module;
+import org.opensearch.common.settings.ClusterSettings;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
import org.opensearch.core.common.io.stream.NamedWriteableRegistry;
+import org.opensearch.core.common.unit.ByteSizeValue;
import org.opensearch.core.xcontent.NamedXContentRegistry;
import org.opensearch.env.Environment;
import org.opensearch.env.NodeEnvironment;
+import org.opensearch.node.resource.tracker.ResourceTrackerSettings;
import org.opensearch.plugins.ExtensiblePlugin;
import org.opensearch.plugins.Plugin;
+import org.opensearch.plugins.PluginNodeStats;
import org.opensearch.repositories.RepositoriesService;
import org.opensearch.script.ScriptService;
import org.opensearch.threadpool.ThreadPool;
@@ -30,7 +30,6 @@
import org.opensearch.watcher.ResourceWatcherService;
import java.io.IOException;
-import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.function.Supplier;
@@ -50,29 +49,75 @@ public class ArrowBasePlugin extends Plugin implements ExtensiblePlugin {
/** Creates the plugin. */
public ArrowBasePlugin() {}
- /** Maximum bytes for the root Arrow allocator. */
- public static final Setting ROOT_LIMIT_SETTING = Setting.longSetting(
+ /**
+ * Maximum bytes for the root Arrow allocator.
+ *
+ * When unset, the default is 20% of
+ * {@link ResourceTrackerSettings#NODE_NATIVE_MEMORY_LIMIT_SETTING}; see
+ * {@link #deriveRootLimitDefault}. The Arrow framework gets a small fraction of the
+ * native budget because the dominant consumer of native memory in analytics workloads
+ * is the DataFusion Rust runtime (~75% of {@code node.native_memory.limit}), not Arrow.
+ * If AC is unconfigured (limit = 0), the default is {@link Long#MAX_VALUE}, preserving
+ * pre-AC behaviour.
+ */
+ public static final Setting ROOT_LIMIT_SETTING = new Setting<>(
NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT,
- Long.MAX_VALUE,
- 0L,
+ ArrowBasePlugin::deriveRootLimitDefault,
+ s -> {
+ long v = Long.parseLong(s);
+ if (v < 0) {
+ throw new IllegalArgumentException("Setting [" + NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT + "] must be >= 0, got " + v);
+ }
+ return v;
+ },
Setting.Property.NodeScope,
Setting.Property.Dynamic
);
+ /**
+ * Computes the default for {@link #ROOT_LIMIT_SETTING} as 20% of
+ * {@link ResourceTrackerSettings#NODE_NATIVE_MEMORY_LIMIT_SETTING}. The Arrow framework's
+ * hard cap covers only Arrow allocations — DataFusion's Rust runtime is a sibling of
+ * Arrow root and gets the larger share of the native budget (see
+ * {@code DataFusionPlugin#deriveMemoryPoolLimitDefault}).
+ *
+ * Returns the bytes-as-string representation expected by the {@link Setting} parser.
+ * If the AC limit is unset (== 0), the default is {@link Long#MAX_VALUE} — unbounded —
+ * preserving pre-AC behaviour.
+ */
+ static String deriveRootLimitDefault(Settings settings) {
+ ByteSizeValue nativeLimit = ResourceTrackerSettings.NODE_NATIVE_MEMORY_LIMIT_SETTING.get(settings);
+ if (nativeLimit.getBytes() <= 0) {
+ return Long.toString(Long.MAX_VALUE);
+ }
+ return Long.toString(nativeLimit.getBytes() * 20 / 100);
+ }
+
/** Minimum guaranteed bytes for the Flight pool. */
public static final Setting FLIGHT_MIN_SETTING = Setting.longSetting(
NativeAllocatorPoolConfig.SETTING_FLIGHT_MIN,
- Long.MAX_VALUE,
+ 0L,
0L,
Setting.Property.NodeScope,
Setting.Property.Dynamic
);
- /** Maximum bytes the Flight pool can burst to. */
- public static final Setting FLIGHT_MAX_SETTING = Setting.longSetting(
+ /**
+ * Maximum bytes the Flight pool can burst to. Default is 5% of
+ * {@link ResourceTrackerSettings#NODE_NATIVE_MEMORY_LIMIT_SETTING}; see
+ * {@link #derivePoolMaxDefault}. Falls back to {@link Long#MAX_VALUE} when AC is
+ * unconfigured. Matches the partitioning model documented in PR #21732.
+ */
+ public static final Setting FLIGHT_MAX_SETTING = new Setting<>(
NativeAllocatorPoolConfig.SETTING_FLIGHT_MAX,
- Long.MAX_VALUE,
- 0L,
+ s -> derivePoolMaxDefault(s, 5),
+ s -> {
+ long v = Long.parseLong(s);
+ if (v < 0) {
+ throw new IllegalArgumentException("Setting [" + NativeAllocatorPoolConfig.SETTING_FLIGHT_MAX + "] must be >= 0, got " + v);
+ }
+ return v;
+ },
Setting.Property.NodeScope,
Setting.Property.Dynamic
);
@@ -80,21 +125,104 @@ public ArrowBasePlugin() {}
/** Minimum guaranteed bytes for the ingest pool. */
public static final Setting INGEST_MIN_SETTING = Setting.longSetting(
NativeAllocatorPoolConfig.SETTING_INGEST_MIN,
- Long.MAX_VALUE,
+ 0L,
0L,
Setting.Property.NodeScope,
Setting.Property.Dynamic
);
- /** Maximum bytes the ingest pool can burst to. */
- public static final Setting INGEST_MAX_SETTING = Setting.longSetting(
+ /**
+ * Maximum bytes the ingest pool can burst to. Default is 8% of
+ * {@link ResourceTrackerSettings#NODE_NATIVE_MEMORY_LIMIT_SETTING}; see
+ * {@link #derivePoolMaxDefault}. Falls back to {@link Long#MAX_VALUE} when AC is
+ * unconfigured. Ingest gets a larger fraction than Flight/Query because parquet VSR
+ * allocators dominate write-path memory usage — see partitioning model in PR #21732.
+ */
+ public static final Setting INGEST_MAX_SETTING = new Setting<>(
NativeAllocatorPoolConfig.SETTING_INGEST_MAX,
- Long.MAX_VALUE,
+ s -> derivePoolMaxDefault(s, 8),
+ s -> {
+ long v = Long.parseLong(s);
+ if (v < 0) {
+ throw new IllegalArgumentException("Setting [" + NativeAllocatorPoolConfig.SETTING_INGEST_MAX + "] must be >= 0, got " + v);
+ }
+ return v;
+ },
+ Setting.Property.NodeScope,
+ Setting.Property.Dynamic
+ );
+
+ /**
+ * Minimum guaranteed bytes for the query pool. Honored by the rebalancer (when
+ * enabled) — sets a floor below which the rebalancer will not shrink the pool.
+ * Has no effect when rebalancing is disabled.
+ */
+ public static final Setting QUERY_MIN_SETTING = Setting.longSetting(
+ NativeAllocatorPoolConfig.SETTING_QUERY_MIN,
+ 0L,
0L,
Setting.Property.NodeScope,
Setting.Property.Dynamic
);
+ /**
+ * Maximum bytes the query pool can allocate. Default is 5% of
+ * {@link ResourceTrackerSettings#NODE_NATIVE_MEMORY_LIMIT_SETTING}; see
+ * {@link #derivePoolMaxDefault}. Falls back to {@link Long#MAX_VALUE} when AC is
+ * unconfigured. Enforced by Arrow's child-allocator limit — analytics-engine's
+ * per-query allocators are children of this pool, so the sum of in-flight per-query
+ * allocations is capped here.
+ *
+ * Note: each individual analytics query is also bounded by
+ * {@code analytics.exec.QueryContext} per-query limit (currently the constant
+ * {@code DEFAULT_PER_QUERY_MEMORY_LIMIT = 256 MB}). Lowering {@code QUERY_MAX}
+ * below {@code 256 MB × concurrent-queries} can starve queries even when each
+ * individual query is within its per-query limit.
+ */
+ public static final Setting QUERY_MAX_SETTING = new Setting<>(
+ NativeAllocatorPoolConfig.SETTING_QUERY_MAX,
+ s -> derivePoolMaxDefault(s, 5),
+ s -> {
+ long v = Long.parseLong(s);
+ if (v < 0) {
+ throw new IllegalArgumentException("Setting [" + NativeAllocatorPoolConfig.SETTING_QUERY_MAX + "] must be >= 0, got " + v);
+ }
+ return v;
+ },
+ Setting.Property.NodeScope,
+ Setting.Property.Dynamic
+ );
+
+ /**
+ * Computes the default for a pool max as a percentage of
+ * {@link ResourceTrackerSettings#NODE_NATIVE_MEMORY_LIMIT_SETTING} (the operator's
+ * declared off-heap budget), falling back to {@link Long#MAX_VALUE} when AC is
+ * unconfigured. Returns the bytes-as-string representation expected by the
+ * {@link Setting} parser.
+ *
+ * Pools are anchored to {@code node.native_memory.limit} rather than to
+ * {@link #ROOT_LIMIT_SETTING} so the diagrammed partitioning (PR #21732) holds:
+ * sum of pool maxes (5+8+5 = 18% of native_memory.limit) fits within the framework
+ * root cap (20% of native_memory.limit) by default. Operator overrides of
+ * {@code root.limit} that drop it below {@code sum(pool.max)} are caught by the
+ * grouped validator.
+ *
+ *
The fraction is taken straight from {@code node.native_memory.limit}, not from
+ * {@code limit - buffer_percent}. {@code buffer_percent} is an admission-control
+ * throttle margin, not a framework budget reduction.
+ *
+ * @param settings node settings
+ * @param percent fraction of {@code node.native_memory.limit} the pool max defaults to
+ */
+ static String derivePoolMaxDefault(Settings settings, int percent) {
+ ByteSizeValue nativeLimit = ResourceTrackerSettings.NODE_NATIVE_MEMORY_LIMIT_SETTING.get(settings);
+ if (nativeLimit.getBytes() <= 0) {
+ return Long.toString(Long.MAX_VALUE);
+ }
+ long pool = Math.max(0L, nativeLimit.getBytes() * percent / 100);
+ return Long.toString(pool);
+ }
+
/** Interval in seconds between pool rebalance cycles. 0 disables rebalancing. */
public static final Setting REBALANCE_INTERVAL_SETTING = Setting.longSetting(
"native.allocator.rebalance.interval_seconds",
@@ -105,7 +233,6 @@ public ArrowBasePlugin() {}
);
private volatile ArrowNativeAllocator allocator;
- private DefaultArrowAllocatorService allocatorService;
@Override
public Collection createComponents(
@@ -122,43 +249,88 @@ public Collection createComponents(
Supplier repositoriesServiceSupplier
) {
Settings settings = environment.settings();
+ ClusterSettings cs = clusterService.getClusterSettings();
+ ArrowNativeAllocator built = buildAllocator(settings, cs);
+ this.allocator = built;
+ return List.of(built);
+ }
+ /**
+ * Constructs the allocator and wires its pools and dynamic-update consumers from
+ * a pure {@code (Settings, ClusterSettings)} pair. Package-private so unit tests
+ * can exercise the full wiring without a heavyweight {@link ClusterService}
+ * fixture — mirrors the shape of {@link #registerSettingsUpdateConsumers} which
+ * is already test-friendly for the same reason.
+ */
+ static ArrowNativeAllocator buildAllocator(Settings settings, ClusterSettings cs) {
long rootLimit = ROOT_LIMIT_SETTING.get(settings);
- allocator = new ArrowNativeAllocator(rootLimit);
+ ArrowNativeAllocator allocator = new ArrowNativeAllocator(rootLimit);
allocator.setRebalanceInterval(REBALANCE_INTERVAL_SETTING.get(settings));
- long flightMin = FLIGHT_MIN_SETTING.get(settings);
- long flightMax = FLIGHT_MAX_SETTING.get(settings);
- long ingestMin = INGEST_MIN_SETTING.get(settings);
- long ingestMax = INGEST_MAX_SETTING.get(settings);
+ // Single source of truth for cross-setting invariants — same logic runs on
+ // dynamic updates via the grouped consumer below.
+ validateUpdate(settings);
- validateMinMax(NativeAllocatorPoolConfig.POOL_FLIGHT, flightMin, flightMax);
- validateMinMax(NativeAllocatorPoolConfig.POOL_INGEST, ingestMin, ingestMax);
- validateMinSum(rootLimit, flightMin, ingestMin);
-
- allocator.getOrCreatePool(NativeAllocatorPoolConfig.POOL_FLIGHT, flightMin, flightMax);
- allocator.getOrCreatePool(NativeAllocatorPoolConfig.POOL_INGEST, ingestMin, ingestMax);
+ allocator.getOrCreatePool(
+ NativeAllocatorPoolConfig.POOL_FLIGHT,
+ FLIGHT_MIN_SETTING.get(settings),
+ FLIGHT_MAX_SETTING.get(settings)
+ );
+ allocator.getOrCreatePool(
+ NativeAllocatorPoolConfig.POOL_INGEST,
+ INGEST_MIN_SETTING.get(settings),
+ INGEST_MAX_SETTING.get(settings)
+ );
+ allocator.getOrCreatePool(NativeAllocatorPoolConfig.POOL_QUERY, QUERY_MIN_SETTING.get(settings), QUERY_MAX_SETTING.get(settings));
- clusterService.getClusterSettings().addSettingsUpdateConsumer(ROOT_LIMIT_SETTING, allocator::setRootLimit);
- clusterService.getClusterSettings().addSettingsUpdateConsumer(REBALANCE_INTERVAL_SETTING, allocator::setRebalanceInterval);
+ registerSettingsUpdateConsumers(cs, allocator);
+ return allocator;
+ }
- // ArrowBasePlugin component: node-level Arrow allocator service for cross-plugin zero-copy
- this.allocatorService = new DefaultArrowAllocatorService(allocator);
+ /**
+ * Registers cluster-settings update consumers that propagate dynamic setting changes
+ * into the live {@link ArrowNativeAllocator}. Package-private so unit tests can exercise
+ * the wiring with a real {@link ClusterSettings} instance — the test that asserts a PUT
+ * lands on the allocator is what catches a future regression where one of these lines
+ * is accidentally removed.
+ */
+ static void registerSettingsUpdateConsumers(ClusterSettings cs, ArrowNativeAllocator allocator) {
+ cs.addSettingsUpdateConsumer(ROOT_LIMIT_SETTING, allocator::setRootLimit);
+ cs.addSettingsUpdateConsumer(REBALANCE_INTERVAL_SETTING, allocator::setRebalanceInterval);
+ cs.addSettingsUpdateConsumer(FLIGHT_MAX_SETTING, v -> allocator.setPoolLimit(NativeAllocatorPoolConfig.POOL_FLIGHT, v));
+ cs.addSettingsUpdateConsumer(FLIGHT_MIN_SETTING, v -> allocator.setPoolMin(NativeAllocatorPoolConfig.POOL_FLIGHT, v));
+ cs.addSettingsUpdateConsumer(INGEST_MAX_SETTING, v -> allocator.setPoolLimit(NativeAllocatorPoolConfig.POOL_INGEST, v));
+ cs.addSettingsUpdateConsumer(INGEST_MIN_SETTING, v -> allocator.setPoolMin(NativeAllocatorPoolConfig.POOL_INGEST, v));
+ cs.addSettingsUpdateConsumer(QUERY_MAX_SETTING, v -> allocator.setPoolLimit(NativeAllocatorPoolConfig.POOL_QUERY, v));
+ cs.addSettingsUpdateConsumer(QUERY_MIN_SETTING, v -> allocator.setPoolMin(NativeAllocatorPoolConfig.POOL_QUERY, v));
- List components = new ArrayList<>();
- components.add(allocator);
- components.add(allocatorService);
- return components;
+ // Grouped validator runs across the related settings on every dynamic update so cross-setting
+ // invariants (sum of pool mins ≤ root, per-pool min ≤ max) are enforced post-startup.
+ cs.addSettingsUpdateConsumer(s -> {}, MIN_MAX_SETTINGS, ArrowBasePlugin::validateUpdate);
}
- @Override
- public Collection createGuiceModules() {
- return List.of(new AbstractModule() {
- @Override
- protected void configure() {
- bind(ArrowAllocatorService.class).toInstance(allocatorService);
- }
- });
+ private static final List> MIN_MAX_SETTINGS = List.of(
+ ROOT_LIMIT_SETTING,
+ FLIGHT_MIN_SETTING,
+ FLIGHT_MAX_SETTING,
+ INGEST_MIN_SETTING,
+ INGEST_MAX_SETTING,
+ QUERY_MIN_SETTING,
+ QUERY_MAX_SETTING
+ );
+
+ private static void validateUpdate(Settings settings) {
+ long rootLimit = ROOT_LIMIT_SETTING.get(settings);
+ long flightMin = FLIGHT_MIN_SETTING.get(settings);
+ long flightMax = FLIGHT_MAX_SETTING.get(settings);
+ long ingestMin = INGEST_MIN_SETTING.get(settings);
+ long ingestMax = INGEST_MAX_SETTING.get(settings);
+ long queryMin = QUERY_MIN_SETTING.get(settings);
+ long queryMax = QUERY_MAX_SETTING.get(settings);
+ validateMinMax(NativeAllocatorPoolConfig.POOL_FLIGHT, flightMin, flightMax);
+ validateMinMax(NativeAllocatorPoolConfig.POOL_INGEST, ingestMin, ingestMax);
+ validateMinMax(NativeAllocatorPoolConfig.POOL_QUERY, queryMin, queryMax);
+ validateMinSum(rootLimit, flightMin, ingestMin, queryMin);
}
@Override
@@ -169,10 +341,27 @@ public List> getSettings() {
FLIGHT_MAX_SETTING,
INGEST_MIN_SETTING,
INGEST_MAX_SETTING,
+ QUERY_MIN_SETTING,
+ QUERY_MAX_SETTING,
REBALANCE_INTERVAL_SETTING
);
}
+ @Override
+ public List nodeStats() {
+ if (allocator == null) {
+ return List.of();
+ }
+ return List.of(new NativeAllocatorPluginStats(allocator.stats()));
+ }
+
+ @Override
+ public List getNamedWriteables() {
+ return List.of(
+ new NamedWriteableRegistry.Entry(PluginNodeStats.class, NativeAllocatorPluginStats.NAME, NativeAllocatorPluginStats::new)
+ );
+ }
+
private static void validateMinMax(String poolName, long min, long max) {
if (min > max) {
throw new IllegalArgumentException("Pool '" + poolName + "' min (" + min + ") exceeds max (" + max + ")");
@@ -185,10 +374,10 @@ private static void validateMinSum(long rootLimit, long... mins) {
}
long sum = 0;
for (long min : mins) {
- long prev = sum;
- sum += min;
- if (sum < prev) {
- throw new IllegalArgumentException("Sum of pool minimums overflows.");
+ try {
+ sum = Math.addExact(sum, min);
+ } catch (ArithmeticException overflow) {
+ throw new IllegalArgumentException("Sum of pool minimums overflows.", overflow);
}
}
if (sum > rootLimit) {
@@ -206,9 +395,6 @@ private static void validateMinSum(long rootLimit, long... mins) {
@Override
public void close() throws IOException {
- if (allocatorService != null) {
- allocatorService.close();
- }
if (allocator != null) {
allocator.close();
allocator = null;
diff --git a/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/ArrowNativeAllocator.java b/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/ArrowNativeAllocator.java
index 6be5ba0d10cab..afab06a249d59 100644
--- a/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/ArrowNativeAllocator.java
+++ b/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/ArrowNativeAllocator.java
@@ -37,33 +37,29 @@
* the root limit. When contention rises, pools shrink back toward their guarantee.
* This prevents idle capacity from being wasted while maintaining isolation under load.
*
- * The singleton instance is accessible via {@link #instance()} after the
- * plugin creates it. This allows child plugins (in the same classloader) to
- * locate the allocator without explicit injection.
+ *
Constructed once by {@link ArrowBasePlugin#createComponents} and exposed to
+ * downstream plugins via Guice and {@code PluginComponentRegistry} so consumers
+ * receive the instance through explicit dependency injection rather than a static
+ * singleton.
*/
public class ArrowNativeAllocator implements NativeAllocator {
- private static volatile ArrowNativeAllocator INSTANCE;
-
- /**
- * Returns the singleton instance, or throws if the plugin hasn't started.
- */
- public static ArrowNativeAllocator instance() {
- ArrowNativeAllocator inst = INSTANCE;
- if (inst == null) {
- throw new IllegalStateException(
- "ArrowNativeAllocator not initialized. " + "Ensure arrow-base plugin is installed and started."
- );
- }
- return inst;
- }
-
private final RootAllocator root;
private final ConcurrentMap pools = new ConcurrentHashMap<>();
private final ConcurrentMap poolMins = new ConcurrentHashMap<>();
private final ConcurrentMap poolMaxes = new ConcurrentHashMap<>();
private final ScheduledExecutorService rebalancer;
private volatile ScheduledFuture> rebalanceTask;
+ /**
+ * True iff the rebalancer is configured to run periodically. Used by
+ * {@link #getOrCreatePool} to decide each pool's initial child-allocator
+ * limit: when rebalancing is enabled, pools start at {@code min} and grow
+ * via the next rebalance tick (preserving the original PR's
+ * "guarantee + burst" semantics); when rebalancing is disabled, pools
+ * start at {@code max} so consumers can allocate immediately without
+ * waiting for a tick that never comes.
+ */
+ private volatile boolean rebalancerEnabled = false;
/**
* Creates a new allocator with a fresh RootAllocator.
@@ -80,7 +76,6 @@ public ArrowNativeAllocator(long rootLimit) {
});
executor.setRemoveOnCancelPolicy(true);
this.rebalancer = executor;
- INSTANCE = this;
}
/**
@@ -95,7 +90,8 @@ public void setRebalanceInterval(long intervalSeconds) {
org.opensearch.common.util.concurrent.FutureUtils.cancel(existing);
rebalanceTask = null;
}
- if (intervalSeconds > 0) {
+ rebalancerEnabled = intervalSeconds > 0;
+ if (rebalancerEnabled) {
rebalanceTask = rebalancer.scheduleAtFixedRate(this::rebalance, intervalSeconds, intervalSeconds, TimeUnit.SECONDS);
}
}
@@ -117,7 +113,14 @@ public PoolHandle getOrCreatePool(String poolName, long min, long max) {
poolMins.putIfAbsent(poolName, min);
poolMaxes.putIfAbsent(poolName, max);
return pools.computeIfAbsent(poolName, name -> {
- BufferAllocator child = root.newChildAllocator(name, 0, min);
+ // Pick an initial limit that's safe for both rebalancer-on and rebalancer-off
+ // deployments. When rebalancing is enabled, start at min (the original PR's
+ // "guarantee + burst" semantics): the next rebalance tick will distribute
+ // headroom up to each pool's max. When rebalancing is disabled (the default),
+ // pools with min=0 would otherwise reject every allocation until a tick that
+ // never comes — start at max so consumers can allocate immediately.
+ long initial = rebalancerEnabled ? min : max;
+ BufferAllocator child = root.newChildAllocator(name, 0, initial);
return new ArrowPoolHandle(child);
});
}
@@ -133,16 +136,39 @@ public void setPoolLimit(String poolName, long newLimit) {
}
/**
- * Updates the minimum guaranteed bytes for a pool.
+ * Updates the minimum guaranteed bytes for a pool. The new min is recorded for the
+ * rebalancer (which honors it as a floor on the next tick) and also pushed to the
+ * live {@link BufferAllocator} so the change takes effect immediately even when
+ * the rebalancer is disabled — the alternative was a Dynamic setting that returned
+ * HTTP 200 but had no observable effect.
+ *
+ * Live propagation rules:
+ *
+ * If {@code newMin} exceeds the pool's current limit, the limit is raised to
+ * {@code newMin} (capped at the configured pool max). Children of the pool
+ * allocator inherit the change automatically via Arrow's parent-cap check at
+ * allocation time, so dynamic resizes reach in-flight workloads without an
+ * explicit notification SPI.
+ * If {@code newMin} is below the current limit, the limit is left alone —
+ * the rebalancer is the only path that shrinks live limits, so a min change
+ * on its own never reduces capacity in flight.
+ *
*
* @param poolName the pool name
* @param newMin new minimum bytes
*/
public void setPoolMin(String poolName, long newMin) {
- if (!pools.containsKey(poolName)) {
+ ArrowPoolHandle handle = pools.get(poolName);
+ if (handle == null) {
throw new IllegalStateException("Pool '" + poolName + "' does not exist");
}
poolMins.put(poolName, newMin);
+ long max = poolMaxes.getOrDefault(poolName, Long.MAX_VALUE);
+ long current = handle.allocator.getLimit();
+ long target = Math.min(newMin, max);
+ if (target > current) {
+ handle.allocator.setLimit(target);
+ }
}
@Override
@@ -188,7 +214,6 @@ public void close() {
}
}
root.close();
- INSTANCE = null;
}
/**
@@ -198,9 +223,12 @@ public void close() {
*
* Every pool is guaranteed at least its configured min
* Compute headroom = rootLimit - sum(all pool current allocations)
- * Identify active pools (allocated > 0)
- * Distribute headroom equally among active pools, capped at each pool's max
- * Inactive pools are set to their min
+ * Distribute headroom equally across all pools (not just active ones), capped
+ * at each pool's max. Distributing to all pools — including those with zero
+ * current allocation — avoids the dead-pool corner case where a pool with
+ * min = 0 starts at limit = 0, can never make its first allocation, and so
+ * never becomes "active" enough to receive a bonus. Pools that don't need the
+ * headroom stay at min naturally because their max caps the bonus.
* No pool's limit ever drops below its current allocation or its min
*
*/
@@ -209,20 +237,14 @@ void rebalance() {
long rootLimit = root.getLimit();
long totalAllocated = 0;
- List activePoolNames = new ArrayList<>();
for (Map.Entry entry : pools.entrySet()) {
- long allocated = entry.getValue().allocator.getAllocatedMemory();
- totalAllocated += allocated;
-
- if (allocated > 0) {
- activePoolNames.add(entry.getKey());
- }
+ totalAllocated += entry.getValue().allocator.getAllocatedMemory();
}
long headroom = Math.max(0, rootLimit - totalAllocated);
- int activeCount = activePoolNames.size();
- long bonusPerActive = activeCount > 0 ? headroom / activeCount : 0;
+ int poolCount = pools.size();
+ long bonusPerPool = poolCount > 0 ? headroom / poolCount : 0;
for (Map.Entry entry : pools.entrySet()) {
String name = entry.getKey();
@@ -231,12 +253,7 @@ void rebalance() {
long max = poolMaxes.getOrDefault(name, Long.MAX_VALUE);
long currentAllocation = alloc.getAllocatedMemory();
- long effectiveLimit;
- if (activeCount > 0 && activePoolNames.contains(name)) {
- effectiveLimit = min + bonusPerActive;
- } else {
- effectiveLimit = min;
- }
+ long effectiveLimit = min + bonusPerPool;
// Cap at pool's max
effectiveLimit = Math.min(effectiveLimit, max);
@@ -277,11 +294,6 @@ public Set getPoolNames() {
return Collections.unmodifiableSet(pools.keySet());
}
- /**
- * Returns the guaranteed limit for a pool.
- *
- * @param poolName name of the pool
- */
/**
* Returns the minimum guaranteed bytes for a pool.
*
diff --git a/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/NativeAllocatorPluginStats.java b/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/NativeAllocatorPluginStats.java
new file mode 100644
index 0000000000000..bfded512a6ecd
--- /dev/null
+++ b/plugins/arrow-base/src/main/java/org/opensearch/arrow/allocator/NativeAllocatorPluginStats.java
@@ -0,0 +1,93 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.arrow.allocator;
+
+import org.opensearch.arrow.spi.NativeAllocatorPoolStats;
+import org.opensearch.core.common.io.stream.StreamInput;
+import org.opensearch.core.common.io.stream.StreamOutput;
+import org.opensearch.core.xcontent.XContentBuilder;
+import org.opensearch.plugins.PluginNodeStats;
+
+import java.io.IOException;
+
+/**
+ * {@link PluginNodeStats} adapter that exposes {@link NativeAllocatorPoolStats}
+ * as a top-level {@code native_allocator} entry under {@code _nodes/stats}.
+ *
+ * Wraps the SPI-defined Arrow-agnostic snapshot so it can flow through the
+ * generic plugin stats wire path without adding a server dependency to {@code arrow-spi}.
+ */
+public class NativeAllocatorPluginStats implements PluginNodeStats {
+
+ /** NamedWriteable name and top-level field key under {@code nodes.}. */
+ public static final String NAME = "native_allocator";
+
+ private final NativeAllocatorPoolStats poolStats;
+
+ /**
+ * Wraps the given snapshot.
+ *
+ * @param poolStats the pool stats snapshot to expose under {@code _nodes/stats}
+ */
+ public NativeAllocatorPluginStats(NativeAllocatorPoolStats poolStats) {
+ this.poolStats = poolStats;
+ }
+
+ /**
+ * Reads from the wire.
+ *
+ * @param in the stream input
+ * @throws IOException if reading fails
+ */
+ public NativeAllocatorPluginStats(StreamInput in) throws IOException {
+ this.poolStats = new NativeAllocatorPoolStats(in);
+ }
+
+ @Override
+ public String getWriteableName() {
+ return NAME;
+ }
+
+ @Override
+ public void writeTo(StreamOutput out) throws IOException {
+ poolStats.writeTo(out);
+ }
+
+ @Override
+ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+ // NativeAllocatorPoolStats wraps its body in {"native_allocator":{...}}; here we are
+ // already inside a builder.startObject("native_allocator") opened by NodeStats.toXContent,
+ // so emit only the inner fields.
+ builder.startObject("root");
+ builder.humanReadableField(
+ "allocated_bytes",
+ "allocated",
+ new org.opensearch.core.common.unit.ByteSizeValue(poolStats.getRootAllocatedBytes())
+ );
+ builder.humanReadableField("peak_bytes", "peak", new org.opensearch.core.common.unit.ByteSizeValue(poolStats.getRootPeakBytes()));
+ builder.humanReadableField(
+ "limit_bytes",
+ "limit",
+ new org.opensearch.core.common.unit.ByteSizeValue(poolStats.getRootLimitBytes())
+ );
+ builder.endObject();
+
+ builder.startObject("pools");
+ for (NativeAllocatorPoolStats.PoolStats pool : poolStats.getPools()) {
+ pool.toXContent(builder, params);
+ }
+ builder.endObject();
+ return builder;
+ }
+
+ /** Returns the wrapped snapshot. */
+ public NativeAllocatorPoolStats getPoolStats() {
+ return poolStats;
+ }
+}
diff --git a/plugins/arrow-base/src/main/java/org/opensearch/arrow/memory/ArrowAllocatorService.java b/plugins/arrow-base/src/main/java/org/opensearch/arrow/memory/ArrowAllocatorService.java
deleted file mode 100644
index 38cc7138bba6f..0000000000000
--- a/plugins/arrow-base/src/main/java/org/opensearch/arrow/memory/ArrowAllocatorService.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * SPDX-License-Identifier: Apache-2.0
- *
- * The OpenSearch Contributors require contributions made to
- * this file be licensed under the Apache-2.0 license or a
- * compatible open source license.
- */
-
-package org.opensearch.arrow.memory;
-
-import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.memory.RootAllocator;
-import org.opensearch.common.annotation.ExperimentalApi;
-
-/**
- * Node-level Arrow allocator service. Plugins that need an Arrow {@link BufferAllocator}
- * obtain a child from this service instead of creating their own {@link RootAllocator},
- * so every Arrow buffer on a node shares the same root. Arrow's
- * {@link org.apache.arrow.memory.AllocationManager} associate check on cross-allocator
- * operations requires {@code source.getRoot() == target.getRoot()}, and this is what
- * makes producer-to-consumer zero-copy handoff work across plugin boundaries.
- *
- * One instance per node, bound into Guice by {@code ArrowBasePlugin}. Consumers inject
- * the interface and do not depend on a concrete implementation.
- *
- * @opensearch.experimental
- */
-@ExperimentalApi
-public interface ArrowAllocatorService {
-
- /**
- * Creates a named child allocator with its own memory limit. Callers own the returned
- * allocator and must close it.
- *
- * @param name descriptive name for debugging (e.g., "flight", "analytics-search")
- * @param limit maximum bytes this child can allocate; Arrow enforces this on every
- * {@code buffer()} call and walks the parent chain, throwing
- * {@link org.apache.arrow.memory.OutOfMemoryException} on breach
- */
- BufferAllocator newChildAllocator(String name, long limit);
-
- /** Current bytes allocated across all descendants of the root. */
- long getAllocatedMemory();
-
- /** Peak bytes allocated since this service started. */
- long getPeakMemoryAllocation();
-}
diff --git a/plugins/arrow-base/src/main/java/org/opensearch/arrow/memory/DefaultArrowAllocatorService.java b/plugins/arrow-base/src/main/java/org/opensearch/arrow/memory/DefaultArrowAllocatorService.java
deleted file mode 100644
index a9e0aada21a9d..0000000000000
--- a/plugins/arrow-base/src/main/java/org/opensearch/arrow/memory/DefaultArrowAllocatorService.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * SPDX-License-Identifier: Apache-2.0
- *
- * The OpenSearch Contributors require contributions made to
- * this file be licensed under the Apache-2.0 license or a
- * compatible open source license.
- */
-
-package org.opensearch.arrow.memory;
-
-import org.apache.arrow.memory.BufferAllocator;
-import org.opensearch.arrow.allocator.ArrowNativeAllocator;
-
-import java.io.Closeable;
-
-/**
- * Default {@link ArrowAllocatorService} that delegates to the unified
- * {@link ArrowNativeAllocator}. All child allocators share the same root,
- * ensuring a single memory domain across the node.
- */
-public final class DefaultArrowAllocatorService implements ArrowAllocatorService, Closeable {
-
- private final ArrowNativeAllocator nativeAllocator;
-
- /**
- * Creates a service backed by the singleton native allocator.
- *
- * @param nativeAllocator the unified native allocator instance
- */
- public DefaultArrowAllocatorService(ArrowNativeAllocator nativeAllocator) {
- this.nativeAllocator = nativeAllocator;
- }
-
- @Override
- public BufferAllocator newChildAllocator(String name, long limit) {
- return nativeAllocator.getRootAllocator().newChildAllocator(name, 0, limit);
- }
-
- @Override
- public long getAllocatedMemory() {
- return nativeAllocator.getRootAllocator().getAllocatedMemory();
- }
-
- @Override
- public long getPeakMemoryAllocation() {
- return nativeAllocator.getRootAllocator().getPeakMemoryAllocation();
- }
-
- @Override
- public void close() {
- // Root is owned by ArrowNativeAllocator — don't close here.
- }
-}
diff --git a/plugins/arrow-base/src/main/java/org/opensearch/arrow/memory/package-info.java b/plugins/arrow-base/src/main/java/org/opensearch/arrow/memory/package-info.java
deleted file mode 100644
index 8a6ae217d0937..0000000000000
--- a/plugins/arrow-base/src/main/java/org/opensearch/arrow/memory/package-info.java
+++ /dev/null
@@ -1,15 +0,0 @@
-/*
- * SPDX-License-Identifier: Apache-2.0
- *
- * The OpenSearch Contributors require contributions made to
- * this file be licensed under the Apache-2.0 license or a
- * compatible open source license.
- */
-
-/**
- * Node-level Arrow memory management. Plugins that produce or consume Arrow data
- * obtain child allocators from {@link ArrowAllocatorService},
- * which roots them under a single per-node {@link org.apache.arrow.memory.RootAllocator}
- * so cross-plugin buffer handoffs pass Arrow's {@code AllocationManager.associate} check.
- */
-package org.opensearch.arrow.memory;
diff --git a/plugins/arrow-base/src/main/java/org/opensearch/arrow/transport/ArrowBatchResponse.java b/plugins/arrow-base/src/main/java/org/opensearch/arrow/transport/ArrowBatchResponse.java
index f5dd7389a01e5..4418c0863f6df 100644
--- a/plugins/arrow-base/src/main/java/org/opensearch/arrow/transport/ArrowBatchResponse.java
+++ b/plugins/arrow-base/src/main/java/org/opensearch/arrow/transport/ArrowBatchResponse.java
@@ -9,7 +9,7 @@
package org.opensearch.arrow.transport;
import org.apache.arrow.vector.VectorSchemaRoot;
-import org.opensearch.arrow.memory.ArrowAllocatorService;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
import org.opensearch.common.annotation.ExperimentalApi;
import org.opensearch.core.action.ActionResponse;
import org.opensearch.core.common.io.stream.StreamInput;
@@ -47,7 +47,16 @@
*
*
Allocator rules:
*
- * Send side: Use a child of {@link ArrowAllocatorService}.
+ * Send side: Source the allocator from one of the framework's named pools via
+ * {@link ArrowNativeAllocator#getPoolAllocator(String)}. Pick the pool that matches the
+ * semantics of the producing component:
+ *
+ * {@code POOL_FLIGHT} — transport-layer producers/consumers (arrow-flight-rpc and
+ * plugins built on top of {@code StreamTransportService}).
+ * {@code POOL_INGEST} — ingest-path producers (parquet-data-format VSR allocators).
+ * {@code POOL_QUERY} — query-execution producers (analytics-engine fragments and
+ * coordinator-side intermediate batches).
+ *
* All allocators must share the same root so zero-copy transfers pass Arrow's
* {@code AllocationManager} associate check.
* Send side: Allocators must outlive the transport stream — some transports
@@ -55,10 +64,10 @@
* create and close a child allocator per request.
* Receive side: The transport transfers vectors from its own allocator into
* the response. The consumer can then transfer them into its own allocator — which
- * must also be a child of {@link ArrowAllocatorService}.
+ * must also descend from one of the framework's named pools.
*
*
- * Cross-plugin footgun: bypassing {@link ArrowAllocatorService}
+ *
Cross-plugin footgun: bypassing the framework's named pools
* (e.g. {@code new RootAllocator()} inside a plugin) does not fail fast — allocation and
* single-plugin use still work. But any zero-copy handoff to another plugin's buffers will trip
* Arrow's {@code AllocationManager.associate()} check, because roots are compared by identity,
diff --git a/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/AllocationRejectionTests.java b/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/AllocationRejectionTests.java
new file mode 100644
index 0000000000000..3d0e193656201
--- /dev/null
+++ b/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/AllocationRejectionTests.java
@@ -0,0 +1,65 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.arrow.allocator;
+
+import org.apache.arrow.memory.OutOfMemoryException;
+import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException;
+import org.opensearch.test.OpenSearchTestCase;
+
+public class AllocationRejectionTests extends OpenSearchTestCase {
+
+ public void testWrapPassesThroughOnSuccess() {
+ String result = AllocationRejection.wrap("ctx", () -> "ok");
+ assertEquals("ok", result);
+ }
+
+ public void testWrapTranslatesArrowOomToRejection() {
+ OutOfMemoryException arrowOom = new OutOfMemoryException(
+ "Unable to allocate buffer of size 1024 due to memory limit. Current allocation: 0"
+ );
+ OpenSearchRejectedExecutionException rejection = expectThrows(
+ OpenSearchRejectedExecutionException.class,
+ () -> AllocationRejection.wrap("query-pool", () -> {
+ throw arrowOom;
+ })
+ );
+ assertTrue("rejection message must include the context label", rejection.getMessage().contains("[query-pool]"));
+ assertTrue(
+ "rejection message must propagate Arrow's diagnostic detail",
+ rejection.getMessage().contains("Unable to allocate buffer of size 1024")
+ );
+ assertSame("Arrow OOM must be attached as cause", arrowOom, rejection.getCause());
+ assertFalse(
+ "rejection should not be marked as executor-shutdown — this is per-request, not lifecycle",
+ rejection.isExecutorShutdown()
+ );
+ }
+
+ public void testWrapDoesNotInterceptOtherExceptions() {
+ // Non-Arrow exceptions must propagate unchanged. The wrapper is OOM-specific.
+ IllegalStateException ise = new IllegalStateException("something else");
+ IllegalStateException caught = expectThrows(
+ IllegalStateException.class,
+ () -> AllocationRejection.wrap("ctx", () -> { throw ise; })
+ );
+ assertSame(ise, caught);
+ }
+
+ public void testRunnableOverloadAlsoTranslatesOom() {
+ OutOfMemoryException arrowOom = new OutOfMemoryException("limit exceeded");
+ OpenSearchRejectedExecutionException rejection = expectThrows(
+ OpenSearchRejectedExecutionException.class,
+ () -> AllocationRejection.wrap("ingest-vsr", (Runnable) () -> {
+ throw arrowOom;
+ })
+ );
+ assertTrue(rejection.getMessage().contains("[ingest-vsr]"));
+ assertSame(arrowOom, rejection.getCause());
+ }
+}
diff --git a/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/ArrowBasePluginTests.java b/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/ArrowBasePluginTests.java
new file mode 100644
index 0000000000000..ad72b70b8cbbb
--- /dev/null
+++ b/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/ArrowBasePluginTests.java
@@ -0,0 +1,440 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.arrow.allocator;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.OutOfMemoryException;
+import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
+import org.opensearch.common.settings.ClusterSettings;
+import org.opensearch.common.settings.Setting;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.test.OpenSearchTestCase;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class ArrowBasePluginTests extends OpenSearchTestCase {
+
+ public void testDeriveRootLimitDefaultUnsetReturnsLongMaxValue() {
+ // Explicit 0 expresses "AC unconfigured" — default is now ram - heap, so Settings.EMPTY
+ // would resolve to a real value on whatever machine the test runs on.
+ Settings s = Settings.builder().put("node.native_memory.limit", "0b").build();
+ assertEquals(Long.toString(Long.MAX_VALUE), ArrowBasePlugin.deriveRootLimitDefault(s));
+ }
+
+ public void testDeriveRootLimitDefaultUsesAcLimitWhenSet() {
+ Settings s = Settings.builder().put("node.native_memory.limit", "1gb").build();
+ // ROOT_LIMIT defaults to 20% of node.native_memory.limit — the Arrow framework gets a
+ // small fraction of native budget; DataFusion's Rust runtime takes the larger share.
+ long oneGiB = 1024L * 1024 * 1024;
+ assertEquals(Long.toString(oneGiB * 20 / 100), ArrowBasePlugin.deriveRootLimitDefault(s));
+ }
+
+ public void testDeriveRootLimitDefaultIgnoresBufferPercent() {
+ // node.native_memory.buffer_percent is admission control's throttle margin, not a
+ // framework budget reduction. The framework default takes its 20% fraction off
+ // node.native_memory.limit directly so AC's safety margin sits between AC's throttle
+ // threshold and the framework's hard cap rather than being collapsed into the cap.
+ // 1000 bytes limit, 20% buffer => root.limit still 20% of 1000 = 200.
+ Settings s = Settings.builder().put("node.native_memory.limit", "1000b").put("node.native_memory.buffer_percent", 20).build();
+ assertEquals("200", ArrowBasePlugin.deriveRootLimitDefault(s));
+ }
+
+ public void testRootLimitSettingExposesDerivedDefault() {
+ Settings s = Settings.builder().put("node.native_memory.limit", "10gb").build();
+ // 20% of 10 GiB.
+ long expected = 10L * 1024 * 1024 * 1024 * 20 / 100;
+ assertEquals(Long.valueOf(expected), ArrowBasePlugin.ROOT_LIMIT_SETTING.get(s));
+ }
+
+ public void testRootLimitSettingExplicitOverridesDerived() {
+ Settings s = Settings.builder()
+ .put("node.native_memory.limit", "8gb")
+ .put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 1024L)
+ .build();
+ assertEquals(Long.valueOf(1024L), ArrowBasePlugin.ROOT_LIMIT_SETTING.get(s));
+ }
+
+ public void testRootLimitRejectsNegative() {
+ Settings s = Settings.builder().put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, -1L).build();
+ IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> ArrowBasePlugin.ROOT_LIMIT_SETTING.get(s));
+ assertTrue(e.getMessage().contains("must be >= 0"));
+ }
+
+ public void testQuerySettingsExposeDefaults() {
+ // Explicit 0 expresses "AC unconfigured" so QUERY_MAX falls back to Long.MAX_VALUE.
+ // Settings.EMPTY would resolve via ram - heap default to a finite, machine-dependent value.
+ Settings s = Settings.builder().put("node.native_memory.limit", "0b").build();
+ assertEquals(Long.valueOf(0L), ArrowBasePlugin.QUERY_MIN_SETTING.get(s));
+ assertEquals(Long.valueOf(Long.MAX_VALUE), ArrowBasePlugin.QUERY_MAX_SETTING.get(s));
+ }
+
+ public void testFlightAndIngestMinDefaultsToZero() {
+ // The grouped validator (validateMinSum) treats per-pool mins as a guarantee
+ // floor — defaults of Long.MAX_VALUE caused the validator to reject any PUT
+ // that set a non-MAX root. Pool mins must default to zero so the baseline
+ // configuration is consistent.
+ Settings s = Settings.EMPTY;
+ assertEquals(Long.valueOf(0L), ArrowBasePlugin.FLIGHT_MIN_SETTING.get(s));
+ assertEquals(Long.valueOf(0L), ArrowBasePlugin.INGEST_MIN_SETTING.get(s));
+ }
+
+ public void testQuerySettingsAcceptValues() {
+ Settings s = Settings.builder()
+ .put(NativeAllocatorPoolConfig.SETTING_QUERY_MIN, 100L)
+ .put(NativeAllocatorPoolConfig.SETTING_QUERY_MAX, 1000L)
+ .build();
+ assertEquals(Long.valueOf(100L), ArrowBasePlugin.QUERY_MIN_SETTING.get(s));
+ assertEquals(Long.valueOf(1000L), ArrowBasePlugin.QUERY_MAX_SETTING.get(s));
+ }
+
+ // -- Pool max defaults derived from node.native_memory.limit ----------
+ // Pool maxes anchor to the operator's off-heap budget (node.native_memory.limit),
+ // not to native.allocator.root.limit. This matches the PR #21732 partitioning
+ // diagram where pool fractions (5%/8%/5%) are of native_memory.limit. Sum of
+ // pool maxes (18% of native_memory.limit) fits within root.limit (20% of
+ // native_memory.limit) by default, leaving 2 pp headroom inside the root cap.
+
+ public void testPoolMaxDefaultsAreLongMaxValueWhenAcUnset() {
+ // AC explicitly unconfigured — pool maxes default to Long.MAX_VALUE (unbounded),
+ // preserving pre-AC behaviour. The default for node.native_memory.limit is
+ // 79% of (ram - heap), so to test the "unset" branch we must explicitly set it to 0.
+ Settings s = Settings.builder().put("node.native_memory.limit", "0b").build();
+ assertEquals(Long.valueOf(Long.MAX_VALUE), ArrowBasePlugin.FLIGHT_MAX_SETTING.get(s));
+ assertEquals(Long.valueOf(Long.MAX_VALUE), ArrowBasePlugin.INGEST_MAX_SETTING.get(s));
+ assertEquals(Long.valueOf(Long.MAX_VALUE), ArrowBasePlugin.QUERY_MAX_SETTING.get(s));
+ }
+
+ public void testPoolMaxDefaultsScaleFromAcBudget() {
+ // 10 GiB native memory limit. Pool maxes per the partitioning model in PR #21732:
+ // FLIGHT_MAX = 5% INGEST_MAX = 8% QUERY_MAX = 5%
+ // Anchored to node.native_memory.limit, not to root.limit (which defaults to 20%
+ // of native_memory.limit) — see derivePoolMaxDefault Javadoc.
+ Settings s = Settings.builder().put("node.native_memory.limit", "10gb").build();
+ long limit = 10L * 1024 * 1024 * 1024;
+ assertEquals(Long.valueOf(limit * 5 / 100), ArrowBasePlugin.FLIGHT_MAX_SETTING.get(s));
+ assertEquals(Long.valueOf(limit * 8 / 100), ArrowBasePlugin.INGEST_MAX_SETTING.get(s));
+ assertEquals(Long.valueOf(limit * 5 / 100), ArrowBasePlugin.QUERY_MAX_SETTING.get(s));
+ }
+
+ public void testPoolMaxDefaultsIgnoreRootLimitOverride() {
+ // Pool maxes anchor to node.native_memory.limit, not to root.limit. An operator
+ // who overrides root.limit (e.g. to 4 GiB instead of the default 20% of
+ // native_memory.limit = 2 GiB) does not shrink pool defaults proportionally;
+ // the diagrammed partitioning of native_memory.limit holds.
+ Settings s = Settings.builder()
+ .put("node.native_memory.limit", "10gb")
+ .put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 4L * 1024 * 1024 * 1024)
+ .build();
+ long limit = 10L * 1024 * 1024 * 1024;
+ assertEquals(Long.valueOf(limit * 5 / 100), ArrowBasePlugin.FLIGHT_MAX_SETTING.get(s));
+ assertEquals(Long.valueOf(limit * 8 / 100), ArrowBasePlugin.INGEST_MAX_SETTING.get(s));
+ assertEquals(Long.valueOf(limit * 5 / 100), ArrowBasePlugin.QUERY_MAX_SETTING.get(s));
+ }
+
+ public void testPoolMaxDefaultsIgnoreBufferPercent() {
+ // node.native_memory.buffer_percent is AC's throttle margin, not a framework budget
+ // reduction. Pool maxes default off node.native_memory.limit directly so AC's safety
+ // margin sits between AC's throttle threshold and the framework's hard cap rather than
+ // being collapsed into the cap.
+ // 1000 bytes limit, 20% buffer => pool maxes are still 5/8/5% of 1000 = 50/80/50.
+ Settings s = Settings.builder().put("node.native_memory.limit", "1000b").put("node.native_memory.buffer_percent", 20).build();
+ assertEquals(Long.valueOf(50L), ArrowBasePlugin.FLIGHT_MAX_SETTING.get(s));
+ assertEquals(Long.valueOf(80L), ArrowBasePlugin.INGEST_MAX_SETTING.get(s));
+ assertEquals(Long.valueOf(50L), ArrowBasePlugin.QUERY_MAX_SETTING.get(s));
+ }
+
+ public void testPoolMaxExplicitOverridesDerived() {
+ // Operator-set values must win over derived defaults.
+ Settings s = Settings.builder()
+ .put("node.native_memory.limit", "10gb")
+ .put(NativeAllocatorPoolConfig.SETTING_FLIGHT_MAX, 7L)
+ .put(NativeAllocatorPoolConfig.SETTING_INGEST_MAX, 8L)
+ .put(NativeAllocatorPoolConfig.SETTING_QUERY_MAX, 9L)
+ .build();
+ assertEquals(Long.valueOf(7L), ArrowBasePlugin.FLIGHT_MAX_SETTING.get(s));
+ assertEquals(Long.valueOf(8L), ArrowBasePlugin.INGEST_MAX_SETTING.get(s));
+ assertEquals(Long.valueOf(9L), ArrowBasePlugin.QUERY_MAX_SETTING.get(s));
+ }
+
+ public void testPoolMaxRejectsNegative() {
+ // Negative pool max is rejected at parse time, mirroring ROOT_LIMIT_SETTING.
+ // Each pool's parser has its own message so we exercise all three to lock down
+ // the per-pool error contract (and keep coverage honest on what is otherwise
+ // boilerplate-but-distinct branches).
+ Settings flight = Settings.builder().put(NativeAllocatorPoolConfig.SETTING_FLIGHT_MAX, -1L).build();
+ IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> ArrowBasePlugin.FLIGHT_MAX_SETTING.get(flight));
+ assertTrue(e.getMessage().contains("must be >= 0"));
+ assertTrue("flight error must name its setting", e.getMessage().contains(NativeAllocatorPoolConfig.SETTING_FLIGHT_MAX));
+
+ Settings ingest = Settings.builder().put(NativeAllocatorPoolConfig.SETTING_INGEST_MAX, -1L).build();
+ IllegalArgumentException eIngest = expectThrows(
+ IllegalArgumentException.class,
+ () -> ArrowBasePlugin.INGEST_MAX_SETTING.get(ingest)
+ );
+ assertTrue(eIngest.getMessage().contains("must be >= 0"));
+ assertTrue("ingest error must name its setting", eIngest.getMessage().contains(NativeAllocatorPoolConfig.SETTING_INGEST_MAX));
+
+ Settings query = Settings.builder().put(NativeAllocatorPoolConfig.SETTING_QUERY_MAX, -1L).build();
+ IllegalArgumentException eQuery = expectThrows(IllegalArgumentException.class, () -> ArrowBasePlugin.QUERY_MAX_SETTING.get(query));
+ assertTrue(eQuery.getMessage().contains("must be >= 0"));
+ assertTrue("query error must name its setting", eQuery.getMessage().contains(NativeAllocatorPoolConfig.SETTING_QUERY_MAX));
+ }
+
+ // -----------------------------------------------------------------
+ // End-to-end wiring tests — verify that Setting.Property.Dynamic settings
+ // actually flow through to the live allocator. These guard against the
+ // "dynamic in name only" failure mode where a setting parses, the validator
+ // runs, the cluster-state update succeeds, and the runtime component
+ // silently does nothing because the addSettingsUpdateConsumer line was
+ // never registered. Bare-setter unit tests do not catch this; tests must
+ // drive a real ClusterSettings#applySettings round-trip.
+ // -----------------------------------------------------------------
+
+ /**
+ * Builds a {@link ClusterSettings} preloaded with all of {@link ArrowBasePlugin}'s
+ * settings, mirroring what {@code SettingsModule} does at node startup. Returns the
+ * fresh allocator with the framework's pools created and consumers registered
+ * — the same wiring path {@code createComponents} runs.
+ */
+ private static ArrowNativeAllocator newWiredAllocator(Settings nodeSettings, ClusterSettings cs) {
+ long rootLimit = ArrowBasePlugin.ROOT_LIMIT_SETTING.get(nodeSettings);
+ ArrowNativeAllocator allocator = new ArrowNativeAllocator(rootLimit);
+ allocator.setRebalanceInterval(ArrowBasePlugin.REBALANCE_INTERVAL_SETTING.get(nodeSettings));
+ allocator.getOrCreatePool(
+ NativeAllocatorPoolConfig.POOL_FLIGHT,
+ ArrowBasePlugin.FLIGHT_MIN_SETTING.get(nodeSettings),
+ ArrowBasePlugin.FLIGHT_MAX_SETTING.get(nodeSettings)
+ );
+ allocator.getOrCreatePool(
+ NativeAllocatorPoolConfig.POOL_INGEST,
+ ArrowBasePlugin.INGEST_MIN_SETTING.get(nodeSettings),
+ ArrowBasePlugin.INGEST_MAX_SETTING.get(nodeSettings)
+ );
+ allocator.getOrCreatePool(
+ NativeAllocatorPoolConfig.POOL_QUERY,
+ ArrowBasePlugin.QUERY_MIN_SETTING.get(nodeSettings),
+ ArrowBasePlugin.QUERY_MAX_SETTING.get(nodeSettings)
+ );
+ ArrowBasePlugin.registerSettingsUpdateConsumers(cs, allocator);
+ return allocator;
+ }
+
+ private static ClusterSettings newClusterSettings(Settings nodeSettings) {
+ Set> registered = new HashSet<>();
+ registered.addAll(new ArrowBasePlugin().getSettings());
+ return new ClusterSettings(nodeSettings, registered);
+ }
+
+ public void testBuildAllocatorWiresAllPoolsAndSettingsConsumers() {
+ // Verifies the full createComponents code path — the helper extracted from
+ // createComponents builds the allocator, creates all three pools (FLIGHT, INGEST,
+ // QUERY), and registers the cluster-settings update consumers. We bypass the
+ // heavyweight ClusterService fixture and inject a real ClusterSettings directly,
+ // which is what production wiring also passes through to buildAllocator after
+ // unpacking the createComponents arguments.
+ Settings nodeSettings = Settings.builder()
+ .put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 8L * 1024 * 1024 * 1024)
+ .put(NativeAllocatorPoolConfig.SETTING_FLIGHT_MAX, 1L * 1024 * 1024 * 1024)
+ .put(NativeAllocatorPoolConfig.SETTING_INGEST_MAX, 2L * 1024 * 1024 * 1024)
+ .put(NativeAllocatorPoolConfig.SETTING_QUERY_MAX, 1L * 1024 * 1024 * 1024)
+ .build();
+ ClusterSettings cs = newClusterSettings(nodeSettings);
+
+ ArrowNativeAllocator allocator = ArrowBasePlugin.buildAllocator(nodeSettings, cs);
+ try {
+ // All three pools created.
+ Set poolNames = allocator.getPoolNames();
+ assertEquals("buildAllocator must register exactly the framework's three pools", 3, poolNames.size());
+ assertTrue(poolNames.contains(NativeAllocatorPoolConfig.POOL_FLIGHT));
+ assertTrue(poolNames.contains(NativeAllocatorPoolConfig.POOL_INGEST));
+ assertTrue(poolNames.contains(NativeAllocatorPoolConfig.POOL_QUERY));
+
+ // Pool maxes match the operator-set values (rebalancer disabled by default,
+ // so initial limit == max).
+ assertEquals(1L * 1024 * 1024 * 1024, allocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_FLIGHT).getLimit());
+ assertEquals(2L * 1024 * 1024 * 1024, allocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_INGEST).getLimit());
+ assertEquals(1L * 1024 * 1024 * 1024, allocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_QUERY).getLimit());
+
+ // Cluster-settings update consumers are registered: a PUT to a pool max must
+ // propagate to the live allocator.
+ cs.applySettings(
+ Settings.builder()
+ .put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 8L * 1024 * 1024 * 1024)
+ .put(NativeAllocatorPoolConfig.SETTING_INGEST_MAX, 4L * 1024 * 1024 * 1024)
+ .build()
+ );
+ assertEquals(
+ "buildAllocator must wire the INGEST_MAX cluster-settings consumer",
+ 4L * 1024 * 1024 * 1024,
+ allocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_INGEST).getLimit()
+ );
+ } finally {
+ allocator.close();
+ }
+ }
+
+ public void testQueryMaxClusterSettingPropagatesToAllocator() {
+ // The full wired path: node starts at default settings, plugin registers
+ // consumers, operator PUTs a new max via _cluster/settings.
+ Settings nodeSettings = Settings.builder().put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 8L * 1024 * 1024 * 1024).build();
+ ClusterSettings cs = newClusterSettings(nodeSettings);
+ ArrowNativeAllocator allocator = newWiredAllocator(nodeSettings, cs);
+ try {
+ cs.applySettings(
+ Settings.builder()
+ .put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 8L * 1024 * 1024 * 1024)
+ .put(NativeAllocatorPoolConfig.SETTING_QUERY_MAX, 1024L * 1024 * 1024)
+ .build()
+ );
+ assertEquals(
+ "PUT to query max must update the live BufferAllocator limit",
+ 1024L * 1024 * 1024,
+ allocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_QUERY).getLimit()
+ );
+ assertEquals(1024L * 1024 * 1024, allocator.getPoolMax(NativeAllocatorPoolConfig.POOL_QUERY));
+ } finally {
+ allocator.close();
+ }
+ }
+
+ public void testFlightMinClusterSettingPropagatesToAllocator() {
+ // Min is the regression-prone path: prior to the live-propagation fix,
+ // setPoolMin only updated the poolMins map and operators got HTTP 200 with
+ // no observable behavior change.
+ Settings nodeSettings = Settings.builder().put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 8L * 1024 * 1024 * 1024).build();
+ ClusterSettings cs = newClusterSettings(nodeSettings);
+ ArrowNativeAllocator allocator = newWiredAllocator(nodeSettings, cs);
+ try {
+ // Pool starts at max (rebalancer disabled by default), so a min PUT below
+ // the current limit is a no-op on the live limit but updates poolMins.
+ // Use a min ABOVE the current limit to force the live raise path.
+ cs.applySettings(
+ Settings.builder()
+ .put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 8L * 1024 * 1024 * 1024)
+ .put(NativeAllocatorPoolConfig.SETTING_FLIGHT_MAX, 4L * 1024 * 1024 * 1024)
+ .put(NativeAllocatorPoolConfig.SETTING_FLIGHT_MIN, 2L * 1024 * 1024 * 1024)
+ .build()
+ );
+ assertEquals(
+ "PUT to flight min must update the recorded min for the rebalancer",
+ 2L * 1024 * 1024 * 1024,
+ allocator.getPoolMin(NativeAllocatorPoolConfig.POOL_FLIGHT)
+ );
+ assertTrue(
+ "PUT to flight min must raise the live BufferAllocator limit when min exceeds current",
+ allocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_FLIGHT).getLimit() >= 2L * 1024 * 1024 * 1024
+ );
+ } finally {
+ allocator.close();
+ }
+ }
+
+ public void testRootLimitClusterSettingPropagatesToAllocator() {
+ Settings nodeSettings = Settings.builder().put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 8L * 1024 * 1024 * 1024).build();
+ ClusterSettings cs = newClusterSettings(nodeSettings);
+ ArrowNativeAllocator allocator = newWiredAllocator(nodeSettings, cs);
+ try {
+ cs.applySettings(Settings.builder().put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 16L * 1024 * 1024 * 1024).build());
+ assertEquals(
+ "PUT to root limit must update the RootAllocator's limit",
+ 16L * 1024 * 1024 * 1024,
+ allocator.getRootAllocator().getLimit()
+ );
+ } finally {
+ allocator.close();
+ }
+ }
+
+ public void testValidatorRejectsSumOfMinsExceedingRoot() {
+ // The cross-setting grouped validator must reject PUTs that would over-
+ // subscribe the root. Test the rejection path end-to-end through ClusterSettings.
+ // Set node.native_memory.limit=0b explicitly so pool maxes default to Long.MAX_VALUE
+ // — minmax path firing first.
+ Settings nodeSettings = Settings.builder()
+ .put("node.native_memory.limit", "0b")
+ .put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 10L * 1024 * 1024 * 1024)
+ .build();
+ ClusterSettings cs = newClusterSettings(nodeSettings);
+ ArrowNativeAllocator allocator = newWiredAllocator(nodeSettings, cs);
+ try {
+ // root=10gb, flight_min=6gb, ingest_min=6gb => sum_mins=12gb > root=10gb.
+ IllegalArgumentException e = expectThrows(
+ IllegalArgumentException.class,
+ () -> cs.applySettings(
+ Settings.builder()
+ .put("node.native_memory.limit", "0b")
+ .put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 10L * 1024 * 1024 * 1024)
+ .put(NativeAllocatorPoolConfig.SETTING_FLIGHT_MIN, 6L * 1024 * 1024 * 1024)
+ .put(NativeAllocatorPoolConfig.SETTING_INGEST_MIN, 6L * 1024 * 1024 * 1024)
+ .build()
+ )
+ );
+ assertTrue(
+ "expected sum-exceeds-root in error, got: " + e.getMessage(),
+ e.getMessage().contains("exceeds root limit") || e.getMessage().contains("Sum of pool minimums")
+ );
+ } finally {
+ allocator.close();
+ }
+ }
+
+ public void testChildAllocatorInheritsParentCapAfterPoolLimitUpdate() {
+ // Sanity check for the AnalyticsSearchService / FlightTransport pattern:
+ // when a consumer creates a child of the framework's pool with Long.MAX_VALUE
+ // limit, a PUT to the pool's max takes effect on the child's allocations
+ // automatically via Arrow's parent-cap check at allocateBytes — no listener needed.
+ //
+ // The contract we rely on (Arrow Accountant.allocate, lines 191-203 in 18.3.0):
+ // when the child's reservation is exhausted, it calls parent.allocate(...) which
+ // checks the parent's allocationLimit on every allocation. Setting the child's own
+ // limit to Long.MAX_VALUE means the child has no own-cap on top of the parent's;
+ // setLimit on the parent is observed atomically by all subsequent allocations
+ // through any descendant.
+ Settings nodeSettings = Settings.builder().put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 8L * 1024 * 1024 * 1024).build();
+ ClusterSettings cs = newClusterSettings(nodeSettings);
+ ArrowNativeAllocator allocator = newWiredAllocator(nodeSettings, cs);
+ try {
+ BufferAllocator queryPool = allocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_QUERY);
+ BufferAllocator child = queryPool.newChildAllocator("consumer", 0, Long.MAX_VALUE);
+ try {
+ // Step 1: a small allocation through the child succeeds with the original pool max.
+ try (var buf = child.buffer(1024)) {
+ assertEquals("child accounting reflects allocation", 1024L, child.getAllocatedMemory());
+ assertEquals("parent pool sees child allocation", 1024L, queryPool.getAllocatedMemory());
+ }
+
+ // Step 2: PUT a small pool max via cluster settings.
+ cs.applySettings(
+ Settings.builder()
+ .put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, 8L * 1024 * 1024 * 1024)
+ .put(NativeAllocatorPoolConfig.SETTING_QUERY_MAX, 1L * 1024 * 1024) // 1 MB
+ .build()
+ );
+ assertEquals("pool's own limit reflects the PUT", 1L * 1024 * 1024, queryPool.getLimit());
+ assertEquals("child's own limit is intentionally uncapped", Long.MAX_VALUE, child.getLimit());
+
+ // Step 3: allocations within the new parent cap still work.
+ try (var withinCap = child.buffer(512 * 1024)) { // 512 KB, under 1 MB cap
+ assertEquals(512L * 1024, child.getAllocatedMemory());
+ }
+
+ // Step 4: allocation exceeding the new parent cap fails — this is the
+ // behavior the deleted listener pattern was emulating, now provided
+ // natively by Arrow's parent-cap check.
+ expectThrows(OutOfMemoryException.class, () -> child.buffer(2L * 1024 * 1024)); // 2 MB, over 1 MB cap
+ } finally {
+ child.close();
+ }
+ } finally {
+ allocator.close();
+ }
+ }
+}
diff --git a/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/ArrowNativeAllocatorTests.java b/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/ArrowNativeAllocatorTests.java
index 665a8c7d6a743..880ea445c3ea7 100644
--- a/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/ArrowNativeAllocatorTests.java
+++ b/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/ArrowNativeAllocatorTests.java
@@ -106,12 +106,12 @@ public void testGetPoolNames() {
assertEquals(2, allocator.getPoolNames().size());
}
- public void testRebalanceGivesHeadroomToActivePool() {
+ public void testRebalanceDistributesHeadroomToAllPools() {
allocator.setRootLimit(100 * 1024 * 1024);
allocator.getOrCreatePool("active", 10 * 1024 * 1024, 100 * 1024 * 1024);
allocator.getOrCreatePool("idle", 10 * 1024 * 1024, 100 * 1024 * 1024);
- // Simulate activity: allocate in "active" pool (allocated > 0 = active)
+ // Simulate activity: allocate in "active" pool.
BufferAllocator activeAlloc = allocator.getPoolAllocator("active");
BufferAllocator child = activeAlloc.newChildAllocator("worker", 0, 100 * 1024 * 1024);
var buf = child.buffer(5 * 1024 * 1024);
@@ -119,19 +119,40 @@ public void testRebalanceGivesHeadroomToActivePool() {
try {
allocator.rebalance();
- // Active pool should get bonus headroom beyond its min
+ // Active pool gets bonus headroom on top of its min.
long activeLimit = activeAlloc.getLimit();
assertTrue("Active pool limit should exceed min after rebalance, got " + activeLimit, activeLimit > 10 * 1024 * 1024);
- // Idle pool should be at its min
+ // Idle pool also receives headroom: distributing to all pools (not just
+ // currently-active ones) avoids the dead-pool corner case where a pool
+ // with min = 0 starts at limit = 0 and can never make a first allocation.
+ // Idle pools that don't end up needing the headroom return it on the next
+ // tick once they remain at zero allocation.
long idleLimit = allocator.getPoolAllocator("idle").getLimit();
- assertEquals(10 * 1024 * 1024, idleLimit);
+ assertTrue("Idle pool should also receive headroom, got " + idleLimit, idleLimit > 10 * 1024 * 1024);
} finally {
buf.close();
child.close();
}
}
+ public void testRebalanceLetsZeroMinPoolAllocate() {
+ // Regression test: under the previous "active pools only" rebalance algorithm,
+ // a pool with min = 0 would start at limit = 0 (rebalancer-on path), be unable
+ // to allocate, never become "active", and so never receive a bonus — permanently
+ // dead. Distributing headroom across all pools fixes the chicken-and-egg.
+ allocator.setRebalanceInterval(60);
+ allocator.setRootLimit(100 * 1024 * 1024);
+ allocator.getOrCreatePool("zero-min", 0L, 100 * 1024 * 1024);
+ try {
+ allocator.rebalance();
+ BufferAllocator pool = allocator.getPoolAllocator("zero-min");
+ assertTrue("Zero-min pool should receive headroom, got " + pool.getLimit(), pool.getLimit() > 0);
+ } finally {
+ allocator.setRebalanceInterval(0);
+ }
+ }
+
public void testRebalanceNeverDropsBelowCurrentAllocation() {
allocator.setRootLimit(50 * 1024 * 1024);
allocator.getOrCreatePool("busy", 10 * 1024 * 1024);
@@ -154,6 +175,24 @@ public void testRebalanceWithNoPools() {
allocator.rebalance();
}
+ public void testInitialLimitIsMaxWhenRebalancerDisabled() {
+ // Default tearDown allocator has rebalancer disabled (interval=0).
+ NativeAllocator.PoolHandle handle = allocator.getOrCreatePool("burst", 10 * 1024 * 1024, 100 * 1024 * 1024);
+ // With the rebalancer off, pools must start at their max so consumers can allocate
+ // immediately. Otherwise default-configured pools (min=0) would reject everything.
+ assertEquals(100 * 1024 * 1024, handle.limit());
+ }
+
+ public void testInitialLimitIsMinWhenRebalancerEnabled() {
+ // Enabling the rebalancer reverts to the original "guarantee + burst" semantics:
+ // pools start at min and grow via the next rebalance tick.
+ allocator.setRebalanceInterval(60); // any positive value enables the flag
+ NativeAllocator.PoolHandle handle = allocator.getOrCreatePool("guaranteed", 10 * 1024 * 1024, 100 * 1024 * 1024);
+ assertEquals(10 * 1024 * 1024, handle.limit());
+ // Disable so subsequent tests aren't affected by the scheduled task.
+ allocator.setRebalanceInterval(0);
+ }
+
public void testCloseReleasesAllPools() {
allocator.getOrCreatePool("close-test", 10 * 1024 * 1024);
allocator.close();
@@ -162,4 +201,35 @@ public void testCloseReleasesAllPools() {
// Recreate for tearDown
allocator = new ArrowNativeAllocator(1024L * 1024 * 1024);
}
+
+ public void testSetPoolMinRaisesLiveLimitWhenRebalancerOff() {
+ // setPoolMin must affect the live BufferAllocator immediately, not just the
+ // poolMins map. Otherwise it's a Dynamic setting that returns HTTP 200 and
+ // does nothing observable until the operator also enables the rebalancer.
+ allocator.setRootLimit(100 * 1024 * 1024);
+ allocator.getOrCreatePool("p", 0L, 100 * 1024 * 1024);
+ BufferAllocator pool = allocator.getPoolAllocator("p");
+
+ long startLimit = pool.getLimit();
+ allocator.setPoolMin("p", 50 * 1024 * 1024);
+
+ long afterMinUpdate = pool.getLimit();
+ assertTrue(
+ "setPoolMin should raise live limit to at least the new min (was " + startLimit + ", now " + afterMinUpdate + ")",
+ afterMinUpdate >= 50 * 1024 * 1024
+ );
+ }
+
+ public void testSetPoolMinDoesNotShrinkLiveLimit() {
+ // Dropping the min must not shrink an in-flight pool — the rebalancer is the
+ // only path that reduces limits, so a min change on its own should never
+ // reclaim capacity.
+ allocator.setRootLimit(100 * 1024 * 1024);
+ allocator.getOrCreatePool("p", 0L, 100 * 1024 * 1024);
+ BufferAllocator pool = allocator.getPoolAllocator("p");
+ long startLimit = pool.getLimit();
+
+ allocator.setPoolMin("p", 1L);
+ assertEquals("dropping min must not shrink live limit", startLimit, pool.getLimit());
+ }
}
diff --git a/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/NativeAllocatorPluginStatsTests.java b/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/NativeAllocatorPluginStatsTests.java
new file mode 100644
index 0000000000000..66aa186cf6fcb
--- /dev/null
+++ b/plugins/arrow-base/src/test/java/org/opensearch/arrow/allocator/NativeAllocatorPluginStatsTests.java
@@ -0,0 +1,66 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.arrow.allocator;
+
+import org.opensearch.arrow.spi.NativeAllocatorPoolStats;
+import org.opensearch.common.io.stream.BytesStreamOutput;
+import org.opensearch.common.xcontent.json.JsonXContent;
+import org.opensearch.core.common.bytes.BytesReference;
+import org.opensearch.core.common.io.stream.StreamInput;
+import org.opensearch.core.xcontent.ToXContent;
+import org.opensearch.core.xcontent.XContentBuilder;
+import org.opensearch.test.OpenSearchTestCase;
+
+import java.util.List;
+
+public class NativeAllocatorPluginStatsTests extends OpenSearchTestCase {
+
+ public void testWriteableNameIsStable() {
+ NativeAllocatorPluginStats stats = new NativeAllocatorPluginStats(new NativeAllocatorPoolStats(0L, 0L, 0L, List.of()));
+ assertEquals("native_allocator", stats.getWriteableName());
+ }
+
+ public void testStreamRoundTrip() throws Exception {
+ List pools = List.of(
+ new NativeAllocatorPoolStats.PoolStats("flight", 100L, 200L, 1000L, 1),
+ new NativeAllocatorPoolStats.PoolStats("query", 300L, 400L, 2000L, 2)
+ );
+ NativeAllocatorPluginStats original = new NativeAllocatorPluginStats(new NativeAllocatorPoolStats(400L, 600L, 8000L, pools));
+
+ BytesStreamOutput out = new BytesStreamOutput();
+ original.writeTo(out);
+ try (StreamInput in = out.bytes().streamInput()) {
+ NativeAllocatorPluginStats deser = new NativeAllocatorPluginStats(in);
+ assertEquals(original.getPoolStats().getRootAllocatedBytes(), deser.getPoolStats().getRootAllocatedBytes());
+ assertEquals(original.getPoolStats().getRootPeakBytes(), deser.getPoolStats().getRootPeakBytes());
+ assertEquals(original.getPoolStats().getRootLimitBytes(), deser.getPoolStats().getRootLimitBytes());
+ assertEquals(original.getPoolStats().getPools().size(), deser.getPoolStats().getPools().size());
+ }
+ }
+
+ public void testToXContentEmitsInnerFieldsOnly() throws Exception {
+ NativeAllocatorPluginStats stats = new NativeAllocatorPluginStats(
+ new NativeAllocatorPoolStats(100L, 200L, 1000L, List.of(new NativeAllocatorPoolStats.PoolStats("flight", 50L, 75L, 500L, 1)))
+ );
+
+ XContentBuilder builder = JsonXContent.contentBuilder();
+ builder.startObject();
+ // Caller (NodeStats) is responsible for opening the named object.
+ builder.startObject(stats.getWriteableName());
+ stats.toXContent(builder, ToXContent.EMPTY_PARAMS);
+ builder.endObject();
+ builder.endObject();
+
+ String json = BytesReference.bytes(builder).utf8ToString();
+ assertTrue("expected native_allocator block at top level", json.contains("\"native_allocator\":"));
+ assertTrue("expected inner root block", json.contains("\"root\""));
+ assertTrue("expected inner pools block", json.contains("\"pools\""));
+ assertTrue("expected pool name 'flight'", json.contains("\"flight\""));
+ }
+}
diff --git a/plugins/arrow-flight-rpc/build.gradle b/plugins/arrow-flight-rpc/build.gradle
index f638ffc007e55..291f25e6ed992 100644
--- a/plugins/arrow-flight-rpc/build.gradle
+++ b/plugins/arrow-flight-rpc/build.gradle
@@ -22,6 +22,8 @@ dependencies {
// arrow-base exposes Arrow + Netty core / Jackson / flatbuffers / slf4j / commons-codec
// as api deps, so a compileOnly project dep pulls them onto the compile classpath
// while keeping them out of the bundled plugin zip (arrow-base ships them at runtime).
+ // The framework also ships the SPI types this plugin calls into (NativeAllocatorPoolConfig).
+ compileOnly project(':libs:opensearch-arrow-spi')
compileOnly project(':plugins:arrow-base')
// Netty codec libs used directly by grpc-netty / flight-core.
diff --git a/plugins/arrow-flight-rpc/src/internalClusterTest/java/org/opensearch/arrow/flight/NativeAllocatorBoundaryIT.java b/plugins/arrow-flight-rpc/src/internalClusterTest/java/org/opensearch/arrow/flight/NativeAllocatorBoundaryIT.java
new file mode 100644
index 0000000000000..5633cfa429c5d
--- /dev/null
+++ b/plugins/arrow-flight-rpc/src/internalClusterTest/java/org/opensearch/arrow/flight/NativeAllocatorBoundaryIT.java
@@ -0,0 +1,189 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.arrow.flight;
+
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.OutOfMemoryException;
+import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse;
+import org.opensearch.arrow.allocator.ArrowBasePlugin;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
+import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
+import org.opensearch.common.settings.Settings;
+import org.opensearch.plugins.Plugin;
+import org.opensearch.test.OpenSearchIntegTestCase;
+
+import java.util.Collection;
+import java.util.List;
+
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.is;
+
+/**
+ * Integration tests for the native-allocator framework's cap enforcement.
+ *
+ * Boots a single-node cluster with tight memory settings, then exercises
+ * the actual Arrow allocation path to verify that the framework's
+ * configured caps are enforced at allocation time (not just at config-parse
+ * time). Complements unit-level tests in {@code ArrowBasePluginTests} by
+ * verifying that the production wiring (Guice -> ArrowNativeAllocator ->
+ * Arrow's RootAllocator chain) honors the caps end-to-end.
+ *
+ *
Each test sets explicit byte limits and allocates real
+ * {@link org.apache.arrow.memory.ArrowBuf} buffers, asserting either
+ * successful allocation or {@link OutOfMemoryException} based on whether
+ * the request fits within the configured cap.
+ */
+@ThreadLeakScope(ThreadLeakScope.Scope.NONE)
+@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, minNumDataNodes = 1, maxNumDataNodes = 1)
+public class NativeAllocatorBoundaryIT extends OpenSearchIntegTestCase {
+
+ /** 1 MiB. Chosen small enough that tests run fast but large enough that
+ * Arrow's internal accounting doesn't round it away. */
+ private static final long MB = 1024L * 1024;
+
+ /** Cap large enough for the framework's own bookkeeping but small enough
+ * to trigger OOM well before exhausting host memory. */
+ private static final long ROOT_CAP_BYTES = 16 * MB;
+
+ @Override
+ protected Collection> nodePlugins() {
+ return List.of(ArrowBasePlugin.class);
+ }
+
+ @Override
+ protected Settings nodeSettings(int nodeOrdinal) {
+ // Set node.native_memory.limit explicitly so framework defaults derive
+ // from a known value rather than the (machine-dependent) ram-heap default.
+ // ROOT_LIMIT and pool maxes are then overridden per-test via cluster
+ // settings PUT or directly via this node-settings layer.
+ return Settings.builder()
+ .put(super.nodeSettings(nodeOrdinal))
+ .put("node.native_memory.limit", "256mb")
+ // Tight root cap: 16 MiB total Arrow framework budget.
+ .put(NativeAllocatorPoolConfig.SETTING_ROOT_LIMIT, ROOT_CAP_BYTES)
+ // Per-pool maxes set generously so per-pool caps don't trip
+ // before root.limit. Tests targeting per-pool caps override below.
+ .put(NativeAllocatorPoolConfig.SETTING_FLIGHT_MAX, ROOT_CAP_BYTES)
+ .put(NativeAllocatorPoolConfig.SETTING_INGEST_MAX, ROOT_CAP_BYTES)
+ .put(NativeAllocatorPoolConfig.SETTING_QUERY_MAX, ROOT_CAP_BYTES)
+ .build();
+ }
+
+ /**
+ * Verifies that {@code parquet.native.pool.query.max} caps allocations
+ * through the QUERY pool: a buffer request exceeding the per-pool cap
+ * throws {@link OutOfMemoryException} even when root has headroom.
+ */
+ public void testPoolMaxRejectsAllocationsBeyondCap() {
+ // Tighten QUERY pool to 4 MiB while leaving root at 16 MiB.
+ long poolCap = 4 * MB;
+ ClusterUpdateSettingsResponse resp = client().admin()
+ .cluster()
+ .prepareUpdateSettings()
+ .setTransientSettings(Settings.builder().put(NativeAllocatorPoolConfig.SETTING_QUERY_MAX, poolCap))
+ .get();
+ assertTrue("PUT to query.max must succeed", resp.isAcknowledged());
+
+ ArrowNativeAllocator allocator = internalCluster().getInstance(ArrowNativeAllocator.class);
+ BufferAllocator queryPool = allocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_QUERY);
+ assertThat("pool's live limit reflects the PUT", queryPool.getLimit(), is(poolCap));
+
+ // Sub-cap allocation succeeds.
+ try (var withinCap = queryPool.buffer(2 * MB)) {
+ assertThat(queryPool.getAllocatedMemory(), greaterThanOrEqualTo(2 * MB));
+ }
+
+ // Cap+1 allocation fails — Arrow's parent-cap check at allocateBytes
+ // walks queryPool's allocationLimit and rejects.
+ expectThrows(OutOfMemoryException.class, () -> queryPool.buffer(8 * MB));
+ }
+
+ /**
+ * Verifies that {@code native.allocator.root.limit} caps allocations
+ * across all pools combined: when the sum of in-flight pool allocations
+ * approaches the root cap, the next allocation is rejected at the root
+ * level even if each individual pool's max would allow it.
+ */
+ public void testRootLimitRejectsAllocationsBeyondCap() {
+ ArrowNativeAllocator allocator = internalCluster().getInstance(ArrowNativeAllocator.class);
+ BufferAllocator flightPool = allocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_FLIGHT);
+ BufferAllocator queryPool = allocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_QUERY);
+ BufferAllocator root = allocator.getRootAllocator();
+
+ // Sanity-check setup: confirm the live limits match nodeSettings.
+ // If these fail, the test setup is wrong and the body's expectations are
+ // meaningless — surface the misconfiguration instead of misleading failures below.
+ assertThat("root.limit must match nodeSettings", root.getLimit(), is(ROOT_CAP_BYTES));
+ assertThat("flight.max must match nodeSettings", flightPool.getLimit(), is(ROOT_CAP_BYTES));
+ assertThat("query.max must match nodeSettings", queryPool.getLimit(), is(ROOT_CAP_BYTES));
+
+ // Hold 8 MiB through the FLIGHT pool. With root at 16 MiB this leaves 8 MiB
+ // headroom across the root. (Power-of-2 sizes avoid Arrow's chunked-allocation
+ // rounding surprises; e.g. a 12 MiB request actually consumes 16 MiB.)
+ try (var flightHold = flightPool.buffer(8 * MB)) {
+ assertThat("FLIGHT pool reflects 8MB allocation", flightPool.getAllocatedMemory(), is(8L * MB));
+ assertThat("root reflects 8MB allocation", root.getAllocatedMemory(), is(8L * MB));
+
+ // A 4 MiB allocation through QUERY succeeds (within remaining root headroom).
+ try (var queryFit = queryPool.buffer(4 * MB)) {
+ assertThat(allocator.getRootAllocator().getAllocatedMemory(), is(12L * MB));
+ }
+
+ // An 8 MiB allocation through QUERY would push the root past the 16 MiB cap
+ // (8 MiB FLIGHT + 8 MiB QUERY). Arrow's parent-cap check at allocateBytes
+ // walks queryPool -> root and rejects with OOM, even though QUERY's own
+ // (16 MiB) max would individually allow it.
+ expectThrows(OutOfMemoryException.class, () -> queryPool.buffer(16 * MB));
+ }
+ }
+
+ /**
+ * Verifies that a dynamic PUT to a pool's max takes effect on
+ * subsequent allocations through descendants of that pool. This is the
+ * behavior the deleted {@code NativeAllocatorListener} SPI was emulating;
+ * it is now provided natively by Arrow's parent-cap check at allocateBytes.
+ */
+ public void testDynamicPoolResizeAffectsInFlightAllocations() {
+ ArrowNativeAllocator allocator = internalCluster().getInstance(ArrowNativeAllocator.class);
+ BufferAllocator queryPool = allocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_QUERY);
+
+ // Step 1: create a child allocator at Long.MAX_VALUE — the AnalyticsSearchService /
+ // DefaultPlanExecutor pattern. The child intentionally has no own-cap; it relies
+ // on the parent pool's allocationLimit at allocation time.
+ try (BufferAllocator child = queryPool.newChildAllocator("boundary-it-child", 0, Long.MAX_VALUE)) {
+ // Step 2: a small buffer through the child succeeds with the initial pool max.
+ try (var buf = child.buffer(2 * MB)) {
+ assertThat(child.getAllocatedMemory(), greaterThanOrEqualTo(2 * MB));
+ }
+
+ // Step 3: PUT a tighter pool max via cluster settings.
+ long newPoolCap = 1 * MB;
+ ClusterUpdateSettingsResponse resp = client().admin()
+ .cluster()
+ .prepareUpdateSettings()
+ .setTransientSettings(Settings.builder().put(NativeAllocatorPoolConfig.SETTING_QUERY_MAX, newPoolCap))
+ .get();
+ assertTrue("PUT to query.max must succeed", resp.isAcknowledged());
+ assertThat("pool's own limit reflects the PUT", queryPool.getLimit(), is(newPoolCap));
+ assertThat("child's own limit is intentionally uncapped", child.getLimit(), is(Long.MAX_VALUE));
+
+ // Step 4: an allocation that fit before the resize now exceeds the parent cap.
+ // Arrow's parent-cap check at allocateBytes walks queryPool.allocationLimit
+ // and rejects — no listener machinery needed.
+ expectThrows(OutOfMemoryException.class, () -> child.buffer(2 * MB));
+
+ // Step 5: an allocation under the new cap still succeeds.
+ try (var smallBuf = child.buffer(512 * 1024)) {
+ assertThat(child.getAllocatedMemory(), greaterThanOrEqualTo(512L * 1024));
+ }
+ }
+ }
+}
diff --git a/plugins/arrow-flight-rpc/src/internalClusterTest/java/org/opensearch/arrow/flight/NativeArrowTransportIT.java b/plugins/arrow-flight-rpc/src/internalClusterTest/java/org/opensearch/arrow/flight/NativeArrowTransportIT.java
index 41f401099fb5b..7e1652292e2ab 100644
--- a/plugins/arrow-flight-rpc/src/internalClusterTest/java/org/opensearch/arrow/flight/NativeArrowTransportIT.java
+++ b/plugins/arrow-flight-rpc/src/internalClusterTest/java/org/opensearch/arrow/flight/NativeArrowTransportIT.java
@@ -25,8 +25,9 @@
import org.opensearch.action.support.ActionFilters;
import org.opensearch.action.support.TransportAction;
import org.opensearch.arrow.allocator.ArrowBasePlugin;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
import org.opensearch.arrow.flight.transport.FlightStreamPlugin;
-import org.opensearch.arrow.memory.ArrowAllocatorService;
+import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
import org.opensearch.arrow.transport.ArrowBatchResponse;
import org.opensearch.arrow.transport.ArrowBatchResponseHandler;
import org.opensearch.cluster.node.DiscoveryNode;
@@ -376,10 +377,11 @@ public static class TransportTestArrowAction extends TransportAction createComponents(
return Collections.emptyList();
}
- this.allocatorService = pluginComponentRegistry.getComponent(ArrowAllocatorService.class)
- .orElseThrow(() -> new IllegalStateException("ArrowAllocatorService not available; arrow-base plugin must be installed"));
+ this.nativeAllocator = pluginComponentRegistry.getComponent(ArrowNativeAllocator.class)
+ .orElseThrow(() -> new IllegalStateException("ArrowNativeAllocator not available; arrow-base plugin must be installed"));
statsCollector = new FlightStatsCollector();
return List.of(statsCollector);
@@ -148,7 +148,7 @@ public Map> getSecureTransports(
tracer,
sslContextProvider,
statsCollector,
- allocatorService
+ nativeAllocator
)
);
}
@@ -190,7 +190,7 @@ public Map> getTransports(
tracer,
null,
statsCollector,
- allocatorService
+ nativeAllocator
)
);
}
diff --git a/plugins/arrow-flight-rpc/src/main/java/org/opensearch/arrow/flight/transport/FlightTransport.java b/plugins/arrow-flight-rpc/src/main/java/org/opensearch/arrow/flight/transport/FlightTransport.java
index 1534a734ab391..513b63e227717 100644
--- a/plugins/arrow-flight-rpc/src/main/java/org/opensearch/arrow/flight/transport/FlightTransport.java
+++ b/plugins/arrow-flight-rpc/src/main/java/org/opensearch/arrow/flight/transport/FlightTransport.java
@@ -19,10 +19,11 @@
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.opensearch.Version;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
import org.opensearch.arrow.flight.bootstrap.ServerConfig;
import org.opensearch.arrow.flight.bootstrap.tls.SslContextProvider;
import org.opensearch.arrow.flight.stats.FlightStatsCollector;
-import org.opensearch.arrow.memory.ArrowAllocatorService;
+import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.network.NetworkAddress;
import org.opensearch.common.network.NetworkService;
@@ -95,13 +96,12 @@ class FlightTransport extends TcpTransport {
private final AtomicInteger nextExecutorIndex = new AtomicInteger(0);
private final ThreadPool threadPool;
- private BufferAllocator flightAllocator;
private BufferAllocator serverAllocator;
private BufferAllocator clientAllocator;
private final NamedWriteableRegistry namedWriteableRegistry;
private final FlightStatsCollector statsCollector;
- private final ArrowAllocatorService allocatorService;
+ private final ArrowNativeAllocator nativeAllocator;
private final FlightTransportConfig config = new FlightTransportConfig();
final FlightServerMiddleware.Key SERVER_HEADER_KEY = FlightServerMiddleware.Key.of(
@@ -119,7 +119,7 @@ public FlightTransport(
Tracer tracer,
SslContextProvider sslContextProvider,
FlightStatsCollector statsCollector,
- ArrowAllocatorService allocatorService
+ ArrowNativeAllocator nativeAllocator
) {
super(settings, version, threadPool, pageCacheRecycler, circuitBreakerService, namedWriteableRegistry, networkService, tracer);
this.portRange = SETTING_FLIGHT_PORTS.get(settings);
@@ -127,7 +127,7 @@ public FlightTransport(
this.publishHosts = SETTING_FLIGHT_PUBLISH_HOST.get(settings).toArray(new String[0]);
this.sslContextProvider = sslContextProvider;
this.statsCollector = statsCollector;
- this.allocatorService = allocatorService;
+ this.nativeAllocator = nativeAllocator;
this.bossEventLoopGroup = createEventLoopGroup("os-grpc-boss-ELG", 1);
this.workerEventLoopGroup = createEventLoopGroup("os-grpc-worker-ELG", Runtime.getRuntime().availableProcessors());
this.serverExecutor = threadPool.executor(ServerConfig.GRPC_EXECUTOR_THREAD_POOL_NAME);
@@ -147,14 +147,23 @@ public FlightTransport(
protected void doStart() {
boolean success = false;
try {
- flightAllocator = allocatorService.newChildAllocator("flight", Integer.MAX_VALUE);
- serverAllocator = flightAllocator.newChildAllocator("server", 0, flightAllocator.getLimit());
- clientAllocator = flightAllocator.newChildAllocator("client", 0, flightAllocator.getLimit());
+ // Use the unified native allocator's flight pool directly as the parent for
+ // server/client child allocators. Allocations are tracked and capped by the
+ // framework alongside ingest, query, and datafusion. Hard-fail if the framework
+ // plugin is missing — silently falling back to a separate root would break the
+ // same-root invariant for cross-plugin Arrow handoff.
+ //
+ // Server/client are children of the pool with Long.MAX_VALUE limits so dynamic
+ // resizes of parquet.native.pool.flight.max take effect immediately via Arrow's
+ // parent-cap check at allocateBytes — no listener needed.
+ BufferAllocator flightPool = nativeAllocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_FLIGHT);
+ serverAllocator = flightPool.newChildAllocator("server", 0, Long.MAX_VALUE);
+ clientAllocator = flightPool.newChildAllocator("client", 0, Long.MAX_VALUE);
if (statsCollector != null) {
- statsCollector.setBufferAllocator(flightAllocator);
+ statsCollector.setBufferAllocator(flightPool);
statsCollector.setThreadPool(threadPool);
}
- flightProducer = new ArrowFlightProducer(this, flightAllocator, SERVER_HEADER_KEY, statsCollector);
+ flightProducer = new ArrowFlightProducer(this, flightPool, SERVER_HEADER_KEY, statsCollector);
bindServer();
success = true;
if (statsCollector != null) {
@@ -261,6 +270,7 @@ private List bindToPort(InetAddress[] hostAddresses) {
@Override
protected void stopInternal() {
try {
+
if (flightServer != null) {
flightServer.shutdown();
flightServer.awaitTermination();
@@ -269,7 +279,6 @@ protected void stopInternal() {
}
serverAllocator.close();
clientAllocator.close();
- flightAllocator.close();
gracefullyShutdownELG(bossEventLoopGroup, "os-grpc-boss-ELG");
gracefullyShutdownELG(workerEventLoopGroup, "os-grpc-worker-ELG");
diff --git a/plugins/arrow-flight-rpc/src/test/java/org/opensearch/arrow/flight/transport/FlightTransportTestBase.java b/plugins/arrow-flight-rpc/src/test/java/org/opensearch/arrow/flight/transport/FlightTransportTestBase.java
index 9e5ce92b5bd2d..ba5efa42df567 100644
--- a/plugins/arrow-flight-rpc/src/test/java/org/opensearch/arrow/flight/transport/FlightTransportTestBase.java
+++ b/plugins/arrow-flight-rpc/src/test/java/org/opensearch/arrow/flight/transport/FlightTransportTestBase.java
@@ -10,12 +10,11 @@
import org.apache.arrow.flight.FlightClient;
import org.apache.arrow.flight.Location;
-import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.memory.RootAllocator;
import org.opensearch.Version;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
import org.opensearch.arrow.flight.bootstrap.ServerConfig;
import org.opensearch.arrow.flight.stats.FlightStatsCollector;
-import org.opensearch.arrow.memory.ArrowAllocatorService;
+import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
import org.opensearch.cluster.node.DiscoveryNode;
import org.opensearch.common.network.NetworkService;
import org.opensearch.common.settings.Settings;
@@ -63,6 +62,7 @@ public abstract class FlightTransportTestBase extends OpenSearchTestCase {
protected BoundTransportAddress boundAddress;
protected FlightTransport flightTransport;
protected StreamTransportService streamTransportService;
+ protected ArrowNativeAllocator nativeAllocator;
@Before
@Override
@@ -94,23 +94,11 @@ public void setUp() throws Exception {
namedWriteableRegistry = new NamedWriteableRegistry(Collections.emptyList());
statsCollector = new FlightStatsCollector();
- RootAllocator testRoot = new RootAllocator(Long.MAX_VALUE);
- ArrowAllocatorService testAllocatorService = new ArrowAllocatorService() {
- @Override
- public BufferAllocator newChildAllocator(String name, long limit) {
- return testRoot.newChildAllocator(name, 0, limit);
- }
-
- @Override
- public long getAllocatedMemory() {
- return testRoot.getAllocatedMemory();
- }
-
- @Override
- public long getPeakMemoryAllocation() {
- return testRoot.getPeakMemoryAllocation();
- }
- };
+ // FlightTransport sources its allocator from the framework's FLIGHT pool. Construct one
+ // here so the test has a usable allocator; tearDown closes it.
+ nativeAllocator = new ArrowNativeAllocator(Long.MAX_VALUE);
+ nativeAllocator.getOrCreatePool(NativeAllocatorPoolConfig.POOL_FLIGHT, 0L, Long.MAX_VALUE);
+
flightTransport = new FlightTransport(
settings,
Version.CURRENT,
@@ -122,7 +110,7 @@ public long getPeakMemoryAllocation() {
mock(Tracer.class),
null,
statsCollector,
- testAllocatorService
+ nativeAllocator
);
flightTransport.start();
TransportService transportService = mock(TransportService.class);
@@ -157,6 +145,10 @@ public void tearDown() throws Exception {
if (threadPool != null) {
threadPool.shutdown();
}
+ if (nativeAllocator != null) {
+ nativeAllocator.close();
+ nativeAllocator = null;
+ }
super.tearDown();
}
diff --git a/plugins/examples/stream-transport-example/src/main/java/org/opensearch/example/stream/TransportNativeArrowStreamDataAction.java b/plugins/examples/stream-transport-example/src/main/java/org/opensearch/example/stream/TransportNativeArrowStreamDataAction.java
index 4822edb430615..3a970e38b8a2f 100644
--- a/plugins/examples/stream-transport-example/src/main/java/org/opensearch/example/stream/TransportNativeArrowStreamDataAction.java
+++ b/plugins/examples/stream-transport-example/src/main/java/org/opensearch/example/stream/TransportNativeArrowStreamDataAction.java
@@ -18,7 +18,8 @@
import org.apache.arrow.vector.types.pojo.Schema;
import org.opensearch.action.support.ActionFilters;
import org.opensearch.action.support.TransportAction;
-import org.opensearch.arrow.memory.ArrowAllocatorService;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
+import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
import org.opensearch.common.inject.Inject;
import org.opensearch.core.action.ActionListener;
import org.opensearch.tasks.Task;
@@ -37,12 +38,18 @@
*
* Demonstrates the pipelined producer pattern:
*
- * Receive an allocator owned by the plugin (closed in {@link StreamTransportExamplePlugin#close()})
+ * Receive an allocator sourced from the framework's FLIGHT pool
* For each batch, create a {@link VectorSchemaRoot}, populate it, and wrap it in a response
* Send via {@code sendResponseBatch()} — the framework zero-copy transfers
* the vectors into the Flight stream on the executor thread
* Call {@code completeStream()} when done
*
+ *
+ * Known leak: the action's allocator is not closed on plugin teardown
+ * (the action is Guice-managed and not held by {@link StreamTransportExamplePlugin}).
+ * On node shutdown, ArrowNativeAllocator.close() will warn about this outstanding child.
+ * A clean fix would make this action {@link java.io.Closeable} and have the plugin track
+ * and close it from {@code Plugin#close()}.
*/
public class TransportNativeArrowStreamDataAction extends TransportAction {
@@ -53,10 +60,14 @@ public class TransportNativeArrowStreamDataAction extends TransportAction arrow-base
+ // extendedPlugins chain.
+ compileOnly project(':libs:opensearch-arrow-spi')
+ compileOnly project(':plugins:arrow-base')
compileOnly "org.apache.logging.log4j:log4j-api:${versions.log4j}"
compileOnly "org.apache.logging.log4j:log4j-core:${versions.log4j}"
@@ -85,6 +90,9 @@ dependencies {
// Planner + Lucene backend for end-to-end delegation unit tests
testImplementation project(':sandbox:plugins:analytics-engine')
testImplementation project(':sandbox:plugins:analytics-backend-lucene')
+ // Tests run on a flat classpath; pull arrow-base in so ArrowNativeAllocator
+ // is reachable for unit tests that exercise the listener mirror.
+ testImplementation project(':plugins:arrow-base')
testCompileOnly 'org.immutables:value-annotations:2.8.8'
}
diff --git a/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/DataFusionPlugin.java b/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/DataFusionPlugin.java
index 0d160096d6ad9..9ef33cf62a877 100644
--- a/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/DataFusionPlugin.java
+++ b/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/DataFusionPlugin.java
@@ -22,13 +22,16 @@
import org.opensearch.common.settings.Settings;
import org.opensearch.common.settings.SettingsFilter;
import org.opensearch.core.common.io.stream.NamedWriteableRegistry;
+import org.opensearch.core.common.unit.ByteSizeValue;
import org.opensearch.core.xcontent.NamedXContentRegistry;
import org.opensearch.env.Environment;
import org.opensearch.env.NodeEnvironment;
import org.opensearch.index.engine.dataformat.DataFormatRegistry;
import org.opensearch.index.engine.dataformat.ReaderManagerConfig;
import org.opensearch.index.engine.exec.EngineReaderManager;
+import org.opensearch.monitor.os.OsProbe;
import org.opensearch.nativebridge.spi.NativeMemoryFetcher;
+import org.opensearch.node.resource.tracker.ResourceTrackerSettings;
import org.opensearch.plugin.stats.AnalyticsBackendNativeMemoryStats;
import org.opensearch.plugin.stats.AnalyticsBackendTaskCancellationStats;
import org.opensearch.plugins.ActionPlugin;
@@ -65,26 +68,110 @@ public class DataFusionPlugin extends Plugin implements SearchBackEndPlugin
- * Dynamic: changes take effect for new allocations only. Existing reservations
+ *
+ * When unset, the default is derived from the admission-control native-memory budget
+ * ({@link ResourceTrackerSettings#NODE_NATIVE_MEMORY_LIMIT_SETTING}), which is
+ * the same off-heap budget admission control throttles against. The DataFusion Rust
+ * runtime is the dominant native-memory consumer for analytics workloads (see PR #21732
+ * partitioning model), so the default takes 75% of {@code node.native_memory.limit}.
+ * If the AC limit is unset (== 0), the default is {@link Long#MAX_VALUE} — unbounded — to
+ * preserve pre-AC behaviour rather than make up a number from JVM heap (which is a
+ * separate, already-allocated region with no relation to native-memory sizing).
+ *
+ *
Dynamic: changes take effect for new allocations only. Existing reservations
* that exceed the new limit are not reclaimed — they drain naturally as queries complete.
*/
- public static final Setting DATAFUSION_MEMORY_POOL_LIMIT = Setting.longSetting(
+ public static final Setting DATAFUSION_MEMORY_POOL_LIMIT = new Setting<>(
"datafusion.memory_pool_limit_bytes",
- Runtime.getRuntime().maxMemory() / 4,
- 0L,
+ DataFusionPlugin::deriveMemoryPoolLimitDefault,
+ s -> {
+ long v = Long.parseLong(s);
+ if (v < 0) {
+ throw new IllegalArgumentException("Setting [datafusion.memory_pool_limit_bytes] must be >= 0, got " + v);
+ }
+ return v;
+ },
Setting.Property.NodeScope,
Setting.Property.Dynamic
);
- /** Spill memory limit — when exceeded, DataFusion spills to disk. */
- public static final Setting DATAFUSION_SPILL_MEMORY_LIMIT = Setting.longSetting(
+ /**
+ * Computes the default for {@link #DATAFUSION_MEMORY_POOL_LIMIT} as 75% of
+ * {@link ResourceTrackerSettings#NODE_NATIVE_MEMORY_LIMIT_SETTING}, falling back to
+ * {@link Long#MAX_VALUE} when AC is unconfigured.
+ *
+ * The fraction is taken straight from {@code node.native_memory.limit}, not from
+ * {@code limit - buffer_percent}. {@code buffer_percent} is an admission-control throttle
+ * margin, not a framework budget reduction; subtracting it here would collapse AC's safety
+ * margin into the framework's hard cap.
+ *
+ *
Returns the bytes-as-string representation expected by the {@link Setting} parser.
+ */
+ static String deriveMemoryPoolLimitDefault(Settings settings) {
+ ByteSizeValue nativeLimit = ResourceTrackerSettings.NODE_NATIVE_MEMORY_LIMIT_SETTING.get(settings);
+ if (nativeLimit.getBytes() <= 0) {
+ return Long.toString(Long.MAX_VALUE);
+ }
+ // 75% of node.native_memory.limit. DataFusion is the dominant native consumer for
+ // analytics workloads; operators tune via the dynamic setting once they characterize
+ // their workload.
+ long pool = Math.max(0L, nativeLimit.getBytes() * 75 / 100);
+ return Long.toString(pool);
+ }
+
+ /**
+ * Disk-staging budget for DataFusion spill. When in-memory operations (HashAggregate, Sort,
+ * TopK) exceed {@link #DATAFUSION_MEMORY_POOL_LIMIT}, DataFusion writes working state to disk;
+ * this setting caps how much disk space that staging can consume.
+ *
+ *
Default: 50% of physical RAM. Spill is a disk budget, not a memory budget,
+ * so it is intentionally not derived from {@link ResourceTrackerSettings#NODE_NATIVE_MEMORY_LIMIT_SETTING}
+ * — the operator-declared off-heap budget bounds working memory, but the spill ceiling needs
+ * to scale with how much state could plausibly need to spill across all concurrent queries,
+ * which tracks physical RAM rather than the off-heap carve-out. 50% is a conservative upper
+ * bound that leaves room for page cache, JVM heap, and OS overhead.
+ *
+ *
Falls back to {@link Long#MAX_VALUE} when {@link OsProbe#getTotalPhysicalMemorySize()}
+ * returns 0 (containerized environments where {@code /proc/meminfo} is restricted), preserving
+ * pre-AC unbounded behaviour.
+ *
+ *
Dynamic only when the loaded native library exports {@code df_set_spill_limit}
+ * (see {@link org.opensearch.be.datafusion.nativelib.NativeBridge#isSpillLimitDynamic()}).
+ * When the symbol is absent the setting can still be updated at the cluster level,
+ * but the new value only takes effect after a node restart — the live update consumer
+ * logs a warning in that case.
+ */
+ public static final Setting DATAFUSION_SPILL_MEMORY_LIMIT = new Setting<>(
"datafusion.spill_memory_limit_bytes",
- Runtime.getRuntime().maxMemory() / 8,
- 0L,
- Setting.Property.NodeScope
+ s -> deriveSpillLimitDefault(),
+ s -> {
+ long v = Long.parseLong(s);
+ if (v < 0) {
+ throw new IllegalArgumentException("Setting [datafusion.spill_memory_limit_bytes] must be >= 0, got " + v);
+ }
+ return v;
+ },
+ Setting.Property.NodeScope,
+ Setting.Property.Dynamic
);
+ /**
+ * Computes the default for {@link #DATAFUSION_SPILL_MEMORY_LIMIT} as 50% of physical RAM.
+ * Returns the bytes-as-string representation expected by the {@link Setting} parser.
+ *
+ * Falls back to {@link Long#MAX_VALUE} when the OS probe cannot read total physical memory
+ * (returns 0 or negative), which happens in some containerized environments. Preserving the
+ * unbounded fallback matches the pattern used by {@link #DATAFUSION_MEMORY_POOL_LIMIT} and
+ * {@code ArrowBasePlugin}'s pool-max defaults when AC is unconfigured.
+ */
+ static String deriveSpillLimitDefault() {
+ long totalRam = OsProbe.getInstance().getTotalPhysicalMemorySize();
+ if (totalRam <= 0) {
+ return Long.toString(Long.MAX_VALUE);
+ }
+ return Long.toString(totalRam / 2);
+ }
+
/**
* Selects how the coordinator-reduce sink hands shard responses to the native runtime.
*
@@ -152,8 +239,13 @@ public Collection createComponents(
logger.debug("DataFusion plugin initialized — memory pool {}B, spill limit {}B", memoryPoolLimit, spillMemoryLimit);
// Wire the dynamic memory pool limit setting to the native runtime so updates via the
- // cluster settings API take effect without restarting the node.
+ // cluster settings API take effect without restarting the node. The framework's
+ // parquet.native.pool.datafusion.{min,max} controls the Java-side Arrow pool that
+ // sources the per-query allocators handed to DataFusion; this setting controls the
+ // Rust runtime's internal MemoryPool used by query execution. They're separate
+ // accounting layers — operators tune them independently.
clusterService.getClusterSettings().addSettingsUpdateConsumer(DATAFUSION_MEMORY_POOL_LIMIT, this::updateMemoryPoolLimit);
+ clusterService.getClusterSettings().addSettingsUpdateConsumer(DATAFUSION_SPILL_MEMORY_LIMIT, this::updateSpillMemoryLimit);
this.datafusionSettings = new DatafusionSettings(clusterService);
@@ -250,6 +342,38 @@ void updateMemoryPoolLimit(long newLimitBytes) {
}
}
+ /**
+ * Applies a new spill memory limit to the running DataFusion runtime when the loaded
+ * native library exports {@code df_set_spill_limit}; otherwise emits a warning. The
+ * cluster-settings update is accepted unconditionally because the value is read at next
+ * node startup. Package-private for testing.
+ */
+ void updateSpillMemoryLimit(long newLimitBytes) {
+ DataFusionService service = dataFusionService;
+ if (service == null) {
+ logger.debug("DataFusion service not yet initialized; ignoring spill limit update to {}B", newLimitBytes);
+ return;
+ }
+ if (!service.isSpillLimitDynamic()) {
+ logger.warn(
+ "Updated DataFusion spill memory limit to {}B at the cluster level; the loaded native library does not "
+ + "support runtime spill resize, so the new value will only take effect after a node restart",
+ newLimitBytes
+ );
+ return;
+ }
+ try {
+ service.setSpillMemoryLimit(newLimitBytes);
+ logger.info("Updated DataFusion spill memory limit to {}B", newLimitBytes);
+ } catch (IllegalStateException e) {
+ logger.warn("Ignoring spill memory limit update to {}B; service is not running", newLimitBytes);
+ } catch (UnsupportedOperationException e) {
+ // isSpillLimitDynamic() guard above should make this unreachable, but defend
+ // against a race between probe and call.
+ logger.warn("Ignoring spill memory limit update to {}B; native runtime does not support live updates", newLimitBytes);
+ }
+ }
+
@Override
public String name() {
return "datafusion";
diff --git a/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/DataFusionService.java b/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/DataFusionService.java
index 6db0cdcefe7c6..21a1b99f5edd9 100644
--- a/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/DataFusionService.java
+++ b/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/DataFusionService.java
@@ -136,6 +136,23 @@ public void setMemoryPoolLimit(long newLimitBytes) {
NativeBridge.setMemoryPoolLimit(getNativeRuntime().get(), newLimitBytes);
}
+ /**
+ * Returns true if the loaded native library can update the spill cap at runtime.
+ * When false, {@link #setSpillMemoryLimit(long)} will throw and cluster-state
+ * updates of {@code datafusion.spill_memory_limit_bytes} have no live effect.
+ */
+ public boolean isSpillLimitDynamic() {
+ return NativeBridge.isSpillLimitDynamic();
+ }
+
+ /**
+ * Sets the spill memory limit at runtime. Requires {@link #isSpillLimitDynamic()};
+ * otherwise throws {@link UnsupportedOperationException}.
+ */
+ public void setSpillMemoryLimit(long newLimitBytes) {
+ NativeBridge.setSpillLimit(getNativeRuntime().get(), newLimitBytes);
+ }
+
/**
* Returns the latest native executor stats, collected fresh from JNI on every call.
*
diff --git a/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/nativelib/NativeBridge.java b/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/nativelib/NativeBridge.java
index c9c22df6fb434..357a0e2c4c00d 100644
--- a/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/nativelib/NativeBridge.java
+++ b/sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/nativelib/NativeBridge.java
@@ -54,6 +54,17 @@ public final class NativeBridge {
private static final MethodHandle GET_MEMORY_POOL_USAGE;
private static final MethodHandle GET_MEMORY_POOL_LIMIT;
private static final MethodHandle SET_MEMORY_POOL_LIMIT;
+ /**
+ * Forward-compat probe for runtime spill-cap updates. Today the upstream
+ * DataFusion {@code DiskManager.max_temp_directory_size} is a plain {@code u64}
+ * that cannot be safely mutated through {@code Arc} once the
+ * runtime is in use, so we do not ship this symbol from our Rust crate.
+ * When the upstream PR converting that field to {@code Arc}
+ * lands, our crate exports {@code df_set_spill_limit} and this handle
+ * becomes non-null automatically — the same Java JAR works against both
+ * versions of the native library.
+ */
+ private static final MethodHandle SET_SPILL_LIMIT;
private static final MethodHandle CREATE_READER;
private static final MethodHandle CLOSE_READER;
private static final MethodHandle EXECUTE_QUERY;
@@ -136,6 +147,19 @@ public final class NativeBridge {
FunctionDescriptor.of(ValueLayout.JAVA_LONG, ValueLayout.JAVA_LONG, ValueLayout.JAVA_LONG)
);
+ // Optional spill-limit setter. Not yet shipped by our Rust crate (upstream
+ // DataFusion 53.1.0 does not support runtime spill resize through the public
+ // Arc API). Bind only if the symbol is present so the same
+ // Java JAR is forward-compatible with a future native library that ships it.
+ SET_SPILL_LIMIT = lib.find("df_set_spill_limit")
+ .map(
+ addr -> linker.downcallHandle(
+ addr,
+ FunctionDescriptor.of(ValueLayout.JAVA_LONG, ValueLayout.JAVA_LONG, ValueLayout.JAVA_LONG)
+ )
+ )
+ .orElse(null);
+
CREATE_READER = linker.downcallHandle(
lib.find("df_create_reader").orElseThrow(),
FunctionDescriptor.of(
@@ -585,6 +609,33 @@ public static void setMemoryPoolLimit(long runtimePtr, long newLimitBytes) {
}
}
+ /**
+ * Returns true if the loaded native library exports {@code df_set_spill_limit}.
+ * When false, spill-cap updates require a node restart — the public
+ * {@link org.opensearch.be.datafusion.DataFusionPlugin#DATAFUSION_SPILL_MEMORY_LIMIT}
+ * setting stays {@code NodeScope}-only.
+ */
+ public static boolean isSpillLimitDynamic() {
+ return SET_SPILL_LIMIT != null;
+ }
+
+ /**
+ * Updates the spill-temp-directory cap at runtime. Throws
+ * {@link UnsupportedOperationException} when the loaded native library does not
+ * export {@code df_set_spill_limit} — callers should gate on
+ * {@link #isSpillLimitDynamic()} before invoking.
+ */
+ public static void setSpillLimit(long runtimePtr, long newLimitBytes) {
+ if (SET_SPILL_LIMIT == null) {
+ throw new UnsupportedOperationException(
+ "df_set_spill_limit not available in the loaded native library; spill cap is fixed at startup"
+ );
+ }
+ try (var call = new NativeCall()) {
+ call.invoke(SET_SPILL_LIMIT, runtimePtr, newLimitBytes);
+ }
+ }
+
// ---- Reader management (confined Arena for path + file strings) ----
/**
diff --git a/sandbox/plugins/analytics-backend-datafusion/src/test/java/org/opensearch/be/datafusion/DataFusionPluginSettingsTests.java b/sandbox/plugins/analytics-backend-datafusion/src/test/java/org/opensearch/be/datafusion/DataFusionPluginSettingsTests.java
index 0e2120293c000..d4d23d7003049 100644
--- a/sandbox/plugins/analytics-backend-datafusion/src/test/java/org/opensearch/be/datafusion/DataFusionPluginSettingsTests.java
+++ b/sandbox/plugins/analytics-backend-datafusion/src/test/java/org/opensearch/be/datafusion/DataFusionPluginSettingsTests.java
@@ -9,6 +9,7 @@
package org.opensearch.be.datafusion;
import org.opensearch.common.settings.Setting;
+import org.opensearch.common.settings.Settings;
import org.opensearch.test.OpenSearchTestCase;
import java.util.List;
@@ -29,6 +30,32 @@ public void testMemoryPoolLimitIsDynamic() {
);
}
+ public void testSpillMemoryLimitIsDynamic() {
+ assertTrue(
+ "datafusion.spill_memory_limit_bytes must be dynamic so cluster-state updates are accepted; "
+ + "live propagation depends on the loaded native library",
+ DataFusionPlugin.DATAFUSION_SPILL_MEMORY_LIMIT.isDynamic()
+ );
+ assertTrue(
+ "datafusion.spill_memory_limit_bytes must have node scope",
+ DataFusionPlugin.DATAFUSION_SPILL_MEMORY_LIMIT.hasNodeScope()
+ );
+ }
+
+ /**
+ * The cluster-settings listener can fire before {@link DataFusionPlugin#createComponents}
+ * is called (service field still null). {@code updateSpillMemoryLimit} must swallow this
+ * quietly to mirror {@link DataFusionPlugin#updateMemoryPoolLimit} so cluster-state
+ * application does not log a spurious failure during node startup.
+ */
+ public void testUpdateSpillMemoryLimitBeforeServiceStartDoesNotThrow() {
+ try (DataFusionPlugin plugin = new DataFusionPlugin()) {
+ plugin.updateSpillMemoryLimit(32L * 1024 * 1024);
+ } catch (Exception e) {
+ throw new AssertionError(e);
+ }
+ }
+
public void testMemoryPoolLimitHasNodeScope() {
assertTrue("datafusion.memory_pool_limit_bytes must have node scope", DataFusionPlugin.DATAFUSION_MEMORY_POOL_LIMIT.hasNodeScope());
}
@@ -96,4 +123,50 @@ public void testDatafusionSettingsIsNullBeforeCreateComponents() {
throw new AssertionError(e);
}
}
+
+ public void testDeriveMemoryPoolLimitDefaultUnsetReturnsLongMaxValue() {
+ // AC explicitly unconfigured — the default must be Long.MAX_VALUE (unbounded),
+ // preserving pre-AC behaviour. The default for node.native_memory.limit is now
+ // ram - heap, so to test the "unset" branch we must explicitly set it to 0.
+ Settings s = Settings.builder().put("node.native_memory.limit", "0b").build();
+ assertEquals(Long.toString(Long.MAX_VALUE), DataFusionPlugin.deriveMemoryPoolLimitDefault(s));
+ }
+
+ public void testDeriveMemoryPoolLimitDefaultUsesNativeMemoryLimit() {
+ // 10 GiB native memory limit — default takes 75% straight from limit, not
+ // from limit - buffer_percent (which is AC's throttle margin, not a framework
+ // budget reduction). 75% of 10 GiB.
+ Settings s = Settings.builder().put("node.native_memory.limit", "10gb").build();
+ long expected = (10L * 1024 * 1024 * 1024) * 75 / 100;
+ assertEquals(Long.toString(expected), DataFusionPlugin.deriveMemoryPoolLimitDefault(s));
+ }
+
+ public void testDeriveMemoryPoolLimitDefaultIgnoresBufferPercent() {
+ // node.native_memory.buffer_percent is AC's throttle margin. The framework default
+ // takes its fraction off node.native_memory.limit directly so the buffer can sit
+ // between AC's throttle threshold and the framework's hard cap.
+ // 1000 bytes limit, 20% buffer => pool max still 75% of 1000 = 750.
+ Settings s = Settings.builder().put("node.native_memory.limit", "1000b").put("node.native_memory.buffer_percent", 20).build();
+ assertEquals("750", DataFusionPlugin.deriveMemoryPoolLimitDefault(s));
+ }
+
+ public void testMemoryPoolLimitSettingExposesDerivedDefault() {
+ Settings s = Settings.builder().put("node.native_memory.limit", "10gb").build();
+ long expected = (10L * 1024 * 1024 * 1024) * 75 / 100;
+ assertEquals(Long.valueOf(expected), DataFusionPlugin.DATAFUSION_MEMORY_POOL_LIMIT.get(s));
+ }
+
+ public void testMemoryPoolLimitSettingExplicitOverridesDerived() {
+ Settings s = Settings.builder().put("node.native_memory.limit", "10gb").put("datafusion.memory_pool_limit_bytes", 1024L).build();
+ assertEquals(Long.valueOf(1024L), DataFusionPlugin.DATAFUSION_MEMORY_POOL_LIMIT.get(s));
+ }
+
+ public void testMemoryPoolLimitRejectsNegative() {
+ Settings s = Settings.builder().put("datafusion.memory_pool_limit_bytes", -1L).build();
+ IllegalArgumentException e = expectThrows(
+ IllegalArgumentException.class,
+ () -> DataFusionPlugin.DATAFUSION_MEMORY_POOL_LIMIT.get(s)
+ );
+ assertTrue(e.getMessage().contains("must be >= 0"));
+ }
}
diff --git a/sandbox/plugins/analytics-backend-datafusion/src/test/java/org/opensearch/be/datafusion/NativeBridgeSpillProbeTests.java b/sandbox/plugins/analytics-backend-datafusion/src/test/java/org/opensearch/be/datafusion/NativeBridgeSpillProbeTests.java
new file mode 100644
index 0000000000000..cce8fcffff10c
--- /dev/null
+++ b/sandbox/plugins/analytics-backend-datafusion/src/test/java/org/opensearch/be/datafusion/NativeBridgeSpillProbeTests.java
@@ -0,0 +1,67 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.be.datafusion;
+
+import org.opensearch.be.datafusion.nativelib.NativeBridge;
+import org.opensearch.test.OpenSearchTestCase;
+
+/**
+ * Tests for the runtime spill-limit probe in {@link NativeBridge}. The probe is
+ * forward-compatible: today the upstream DataFusion 53.1.0 {@code DiskManager} does
+ * not expose a safe runtime resize through {@code Arc}, so our Rust
+ * crate does not export {@code df_set_spill_limit} and the probe reports
+ * {@code false}. When the upstream patch converting {@code max_temp_directory_size}
+ * to {@code Arc} lands and we ship the symbol, the probe flips to
+ * {@code true} automatically.
+ *
+ * Each test below branches on {@link NativeBridge#isSpillLimitDynamic()} so the
+ * suite remains green in both regimes. When the symbol ships, the negative-path
+ * assertions become unreachable but the positive-path assertions activate — the
+ * test class continues to lock in the expected behaviour without needing rewrites
+ * (only verification when the change is made).
+ */
+public class NativeBridgeSpillProbeTests extends OpenSearchTestCase {
+
+ public void testProbeReportsAvailability() {
+ // The probe should return a stable answer for the loaded native library.
+ // Today this is false (we do not ship the symbol). When the upstream
+ // Arc patch lands and our crate exports df_set_spill_limit,
+ // this assertion will start passing the other branch.
+ boolean dynamic = NativeBridge.isSpillLimitDynamic();
+ // Whatever the current value, the probe must be deterministic.
+ assertEquals("isSpillLimitDynamic must be deterministic across calls", dynamic, NativeBridge.isSpillLimitDynamic());
+ }
+
+ public void testSetSpillLimitContractMatchesProbe() {
+ if (NativeBridge.isSpillLimitDynamic()) {
+ // Symbol is present — setSpillLimit must not throw the
+ // UnsupportedOperationException sentinel. We can't assert on the
+ // actual side effect without a live runtime pointer, but invoking the
+ // method with runtimePtr=0 must surface a different failure mode
+ // (typically a downcall error) rather than the "symbol absent" path.
+ try {
+ NativeBridge.setSpillLimit(0L, 1024L * 1024 * 1024);
+ } catch (UnsupportedOperationException e) {
+ fail("isSpillLimitDynamic() returned true but setSpillLimit threw the symbol-absent sentinel: " + e.getMessage());
+ } catch (Exception expected) {
+ // Any other failure is acceptable here; we're only asserting that
+ // the symbol-absent path is not taken when the probe says present.
+ }
+ } else {
+ UnsupportedOperationException e = expectThrows(
+ UnsupportedOperationException.class,
+ () -> NativeBridge.setSpillLimit(0L, 1024L * 1024 * 1024)
+ );
+ assertTrue(
+ "expected error message to mention the missing symbol, got: " + e.getMessage(),
+ e.getMessage().contains("df_set_spill_limit")
+ );
+ }
+ }
+}
diff --git a/sandbox/plugins/analytics-engine/build.gradle b/sandbox/plugins/analytics-engine/build.gradle
index 928450005e5e4..3fa93a5b3d832 100644
--- a/sandbox/plugins/analytics-engine/build.gradle
+++ b/sandbox/plugins/analytics-engine/build.gradle
@@ -19,7 +19,8 @@ opensearchplugin {
classname = 'org.opensearch.analytics.AnalyticsPlugin'
// Extend arrow-base so analytics-engine shares one Arrow classloader with any peer plugin
// (arrow-flight-rpc, analytics-backend-datafusion, parquet-data-format, …). Cross-plugin
- // Arrow types (VectorSchemaRoot, ArrowBatchResponse) only work when loaded by the same classloader
+ // Arrow types (VectorSchemaRoot, ArrowBatchResponse) only work when loaded by the same classloader.
+ // arrow-base exposes ArrowNativeAllocator + the SPI used to source the query pool.
extendedPlugins = ['arrow-base']
}
@@ -62,6 +63,9 @@ dependencies {
// as api deps, so a compileOnly project dep pulls them onto the compile classpath while
// keeping them out of the bundled plugin zip (arrow-base ships them at runtime via the
// extendedPlugins classloader chain, giving us a shared Arrow class identity for zero-copy).
+ // arrow-base ships the ArrowNativeAllocator + SPI types this plugin calls into.
+ // compileOnly because at runtime the parent plugin (extendedPlugins above) exposes them.
+ compileOnly project(':libs:opensearch-arrow-spi')
compileOnly project(':plugins:arrow-base')
// Guava — required at compile time because Calcite base classes expose guava types.
@@ -100,7 +104,7 @@ dependencies {
// Unit tests run on a flat classpath (no plugin classloader), so pull arrow-base's runtime
// jars in via the project dep — tests then use the same Arrow + Netty allocator as production.
- testRuntimeOnly project(':plugins:arrow-base')
+ testImplementation project(':plugins:arrow-base')
// Arrow Flight streaming transport for ITs
internalClusterTestImplementation project(':plugins:arrow-flight-rpc')
diff --git a/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/AnalyticsPlugin.java b/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/AnalyticsPlugin.java
index 6a69b7906fe37..0725ed4334055 100644
--- a/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/AnalyticsPlugin.java
+++ b/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/AnalyticsPlugin.java
@@ -25,7 +25,7 @@
import org.opensearch.analytics.planner.FieldStorageResolver;
import org.opensearch.analytics.schema.OpenSearchSchemaBuilder;
import org.opensearch.analytics.spi.AnalyticsSearchBackendPlugin;
-import org.opensearch.arrow.memory.ArrowAllocatorService;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
import org.opensearch.cluster.metadata.IndexNameExpressionResolver;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.inject.Module;
@@ -107,8 +107,8 @@ public Collection createComponents(
Supplier repositoriesServiceSupplier,
PluginComponentRegistry pluginComponentRegistry
) {
- ArrowAllocatorService allocatorService = pluginComponentRegistry.getComponent(ArrowAllocatorService.class)
- .orElseThrow(() -> new IllegalStateException("ArrowAllocatorService not available; arrow-base plugin must be installed"));
+ ArrowNativeAllocator nativeAllocator = pluginComponentRegistry.getComponent(ArrowNativeAllocator.class)
+ .orElseThrow(() -> new IllegalStateException("ArrowNativeAllocator not available; arrow-base plugin must be installed"));
operatorTable = aggregateOperatorTables();
DefaultEngineContext ctx = new DefaultEngineContext(clusterService, operatorTable);
@@ -118,7 +118,7 @@ public Collection createComponents(
for (AnalyticsSearchBackendPlugin be : backEnds) {
backEndsByName.put(be.name(), be);
}
- searchService = new AnalyticsSearchService(backEndsByName, allocatorService, namedWriteableRegistry);
+ searchService = new AnalyticsSearchService(backEndsByName, nativeAllocator, namedWriteableRegistry);
return List.of(searchService, ctx, capabilityRegistry);
}
diff --git a/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/exec/AnalyticsSearchService.java b/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/exec/AnalyticsSearchService.java
index bffc1f430ec23..e212c7f9c9f6a 100644
--- a/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/exec/AnalyticsSearchService.java
+++ b/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/exec/AnalyticsSearchService.java
@@ -26,7 +26,8 @@
import org.opensearch.analytics.spi.FragmentInstructionHandler;
import org.opensearch.analytics.spi.FragmentInstructionHandlerFactory;
import org.opensearch.analytics.spi.InstructionNode;
-import org.opensearch.arrow.memory.ArrowAllocatorService;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
+import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
import org.opensearch.common.concurrent.GatedCloseable;
import org.opensearch.core.common.io.stream.NamedWriteableRegistry;
import org.opensearch.core.tasks.TaskCancelledException;
@@ -51,11 +52,12 @@
* Does NOT hold {@code IndicesService} — receives an already-resolved
* {@link IndexShard} from the transport action.
*
- *
Owns a service-lifetime {@link BufferAllocator} shared by every fragment, obtained as a child of the
- * node-level root via {@link ArrowAllocatorService}. One allocator per service means memory accounting is
- * reported at the service level. For the streaming path, Arrow Flight's outbound handler co-locates its
- * transfer target on the same root (see {@code FlightOutboundHandler#processBatchTask}), keeping transfers
- * same-root and avoiding the known cross-allocator bug with foreign-backed buffers from the C Data Interface.
+ *
Owns a service-lifetime {@link BufferAllocator} shared by every fragment, obtained as a child of
+ * the framework's QUERY pool via {@link ArrowNativeAllocator#getPoolAllocator(String)}. One allocator
+ * per service means memory accounting is reported at the service level. For the streaming path, Arrow
+ * Flight's outbound handler co-locates its transfer target on the same root (see
+ * {@code FlightOutboundHandler#processBatchTask}), keeping transfers same-root and avoiding the known
+ * cross-allocator bug with foreign-backed buffers from the C Data Interface.
*
* @opensearch.internal
*/
@@ -68,28 +70,38 @@ public class AnalyticsSearchService implements AutoCloseable {
private final NamedWriteableRegistry namedWriteableRegistry;
private TaskResourceTrackingService taskResourceTrackingService;
private final BufferAllocator allocator;
+ private final ArrowNativeAllocator nativeAllocator;
- public AnalyticsSearchService(Map backends, ArrowAllocatorService allocatorService) {
- this(backends, List.of(), allocatorService, null);
+ public AnalyticsSearchService(Map backends, ArrowNativeAllocator nativeAllocator) {
+ this(backends, List.of(), nativeAllocator, null);
}
public AnalyticsSearchService(
Map backends,
- ArrowAllocatorService allocatorService,
+ ArrowNativeAllocator nativeAllocator,
NamedWriteableRegistry namedWriteableRegistry
) {
- this(backends, List.of(), allocatorService, namedWriteableRegistry);
+ this(backends, List.of(), nativeAllocator, namedWriteableRegistry);
}
public AnalyticsSearchService(
Map backends,
List listeners,
- ArrowAllocatorService allocatorService,
+ ArrowNativeAllocator nativeAllocator,
NamedWriteableRegistry namedWriteableRegistry
) {
this.backends = backends;
this.listener = new AnalyticsOperationListener.CompositeListener(listeners);
- this.allocator = allocatorService.newChildAllocator("analytics-search-service", Long.MAX_VALUE);
+ this.nativeAllocator = nativeAllocator;
+ // Source the service-level allocator from the unified framework's query pool so all
+ // analytics-engine allocations are tracked and capped by the framework. Hard-fail if
+ // the framework is missing — silently falling back to a separate root would break
+ // Arrow's same-root invariant for cross-plugin handoff.
+ //
+ // Child uses Long.MAX_VALUE so dynamic resizes of parquet.native.pool.query.max take
+ // effect immediately via Arrow's parent-cap check at allocateBytes — no listener needed.
+ BufferAllocator queryPool = nativeAllocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_QUERY);
+ this.allocator = queryPool.newChildAllocator("analytics-search-service", 0, Long.MAX_VALUE);
this.namedWriteableRegistry = namedWriteableRegistry;
}
diff --git a/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/exec/DefaultPlanExecutor.java b/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/exec/DefaultPlanExecutor.java
index 633c13b00d8a8..57a62e1aa567a 100644
--- a/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/exec/DefaultPlanExecutor.java
+++ b/sandbox/plugins/analytics-engine/src/main/java/org/opensearch/analytics/exec/DefaultPlanExecutor.java
@@ -32,7 +32,9 @@
import org.opensearch.analytics.planner.dag.FragmentConversionDriver;
import org.opensearch.analytics.planner.dag.PlanForker;
import org.opensearch.analytics.planner.dag.QueryDAG;
-import org.opensearch.arrow.memory.ArrowAllocatorService;
+import org.opensearch.arrow.allocator.AllocationRejection;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
+import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.inject.Inject;
import org.opensearch.common.unit.TimeValue;
@@ -76,8 +78,13 @@ public class DefaultPlanExecutor extends HandledTransportAction {
throw new UnsupportedOperationException("Transport path not implemented yet");
@@ -103,7 +110,17 @@ public DefaultPlanExecutor(
this.taskManager = transportService.getTaskManager();
this.client = client;
this.scheduler = scheduler;
- this.coordinatorAllocator = allocatorService.newChildAllocator("coordinator", Long.MAX_VALUE);
+ this.nativeAllocator = nativeAllocator;
+ // Source the coordinator allocator from the framework's QUERY pool so coordinator-side
+ // allocations (held shard responses, intermediate batches during reduce) count against
+ // parquet.native.pool.query.max alongside the data-node-side per-fragment allocators in
+ // AnalyticsSearchService. Before this, coordinator allocations were root-siblings outside
+ // any pool and invisible to _nodes/stats.native_allocator.pools.query.allocated.
+ //
+ // Child uses Long.MAX_VALUE so dynamic resizes of parquet.native.pool.query.max take
+ // effect immediately via Arrow's parent-cap check at allocateBytes — no listener needed.
+ BufferAllocator queryPool = nativeAllocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_QUERY);
+ this.coordinatorAllocator = queryPool.newChildAllocator("coordinator", 0, Long.MAX_VALUE);
this.perQueryBufferLimit = AnalyticsPlugin.COORDINATOR_BUFFER_LIMIT.get(clusterService.getSettings());
clusterService.getClusterSettings()
.addSettingsUpdateConsumer(AnalyticsPlugin.COORDINATOR_BUFFER_LIMIT, v -> perQueryBufferLimit = v);
@@ -168,7 +185,14 @@ private void executeInternal(RelNode logicalFragment, ActionListener coordinatorAllocator.newChildAllocator("query-" + dag.queryId(), 0, perQueryBufferLimit)
+ );
ownsAllocator = true;
}
logger.debug("[query-{}] Arrow allocator created, limit={}B", dag.queryId(), perQueryBufferLimit);
diff --git a/sandbox/plugins/composite-engine/src/internalClusterTest/java/org/opensearch/composite/CompositeParquetIndexIT.java b/sandbox/plugins/composite-engine/src/internalClusterTest/java/org/opensearch/composite/CompositeParquetIndexIT.java
index 1e95dc64e79be..ef3432b7d0240 100644
--- a/sandbox/plugins/composite-engine/src/internalClusterTest/java/org/opensearch/composite/CompositeParquetIndexIT.java
+++ b/sandbox/plugins/composite-engine/src/internalClusterTest/java/org/opensearch/composite/CompositeParquetIndexIT.java
@@ -15,6 +15,7 @@
import org.opensearch.action.admin.indices.stats.IndicesStatsResponse;
import org.opensearch.action.admin.indices.stats.ShardStats;
import org.opensearch.action.index.IndexResponse;
+import org.opensearch.arrow.allocator.ArrowBasePlugin;
import org.opensearch.be.datafusion.DataFusionPlugin;
import org.opensearch.be.lucene.LucenePlugin;
import org.opensearch.cluster.metadata.IndexMetadata;
@@ -50,7 +51,15 @@ public class CompositeParquetIndexIT extends OpenSearchIntegTestCase {
@Override
protected Collection> nodePlugins() {
- return Arrays.asList(ParquetDataFormatPlugin.class, CompositeDataFormatPlugin.class, LucenePlugin.class, DataFusionPlugin.class);
+ // ParquetDataFormatPlugin sources its allocator from the unified native-allocator
+ // framework's ingest pool, so the framework plugin must be installed.
+ return Arrays.asList(
+ ArrowBasePlugin.class,
+ ParquetDataFormatPlugin.class,
+ CompositeDataFormatPlugin.class,
+ LucenePlugin.class,
+ DataFusionPlugin.class
+ );
}
@Override
diff --git a/sandbox/plugins/parquet-data-format/benchmarks/src/main/java/org/opensearch/parquet/benchmark/VSRRotationBenchmark.java b/sandbox/plugins/parquet-data-format/benchmarks/src/main/java/org/opensearch/parquet/benchmark/VSRRotationBenchmark.java
index 088c47e16e32d..8a65d5fe52734 100644
--- a/sandbox/plugins/parquet-data-format/benchmarks/src/main/java/org/opensearch/parquet/benchmark/VSRRotationBenchmark.java
+++ b/sandbox/plugins/parquet-data-format/benchmarks/src/main/java/org/opensearch/parquet/benchmark/VSRRotationBenchmark.java
@@ -79,6 +79,7 @@ public class VSRRotationBenchmark {
private ThreadPool threadPool;
private ArrowBufferPool bufferPool;
+ private org.opensearch.arrow.allocator.ArrowNativeAllocator nativeAllocator;
private Schema schema;
private List fieldTypes;
private VSRManager vsrManager;
@@ -125,7 +126,9 @@ public void setupTrial() {
@Setup(Level.Invocation)
public void setup() throws IOException {
- bufferPool = new ArrowBufferPool(Settings.EMPTY);
+ nativeAllocator = new org.opensearch.arrow.allocator.ArrowNativeAllocator(Long.MAX_VALUE);
+ nativeAllocator.getOrCreatePool(org.opensearch.arrow.spi.NativeAllocatorPoolConfig.POOL_INGEST, 0L, Long.MAX_VALUE);
+ bufferPool = new ArrowBufferPool(Settings.EMPTY, nativeAllocator);
filePath = Path.of(System.getProperty("java.io.tmpdir"), "benchmark_vsr_" + System.nanoTime() + ".parquet").toString();
Settings idxSettings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).build();
IndexMetadata indexMetadata = IndexMetadata.builder("benchmark-index").settings(idxSettings).build();
@@ -167,6 +170,10 @@ public void tearDown() {
Files.deleteIfExists(Path.of(filePath));
} catch (Exception ignored) {}
bufferPool.close();
+ if (nativeAllocator != null) {
+ nativeAllocator.close();
+ nativeAllocator = null;
+ }
}
@TearDown(Level.Trial)
diff --git a/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/ParquetDataFormatPlugin.java b/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/ParquetDataFormatPlugin.java
index c44dac2a792e9..675b345983550 100644
--- a/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/ParquetDataFormatPlugin.java
+++ b/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/ParquetDataFormatPlugin.java
@@ -8,6 +8,7 @@
package org.opensearch.parquet;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
import org.opensearch.cluster.metadata.IndexNameExpressionResolver;
import org.opensearch.cluster.service.ClusterService;
import org.opensearch.common.settings.Setting;
@@ -31,6 +32,7 @@
import org.opensearch.parquet.fields.ArrowSchemaBuilder;
import org.opensearch.parquet.store.ParquetStoreStrategy;
import org.opensearch.plugins.Plugin;
+import org.opensearch.plugins.PluginComponentRegistry;
import org.opensearch.repositories.RepositoriesService;
import org.opensearch.script.ScriptService;
import org.opensearch.threadpool.ExecutorBuilder;
@@ -75,6 +77,14 @@ public class ParquetDataFormatPlugin extends Plugin implements DataFormatPlugin
/** Initialized to EMPTY to avoid NPE if indexingEngine() is called before createComponents(). */
private Settings settings = Settings.EMPTY;
private ThreadPool threadPool;
+ private ArrowNativeAllocator nativeAllocator;
+ /**
+ * Live value of {@link ParquetSettings#MAX_PER_VSR_ALLOCATION_DIVISOR}. Updated by the
+ * cluster-settings consumer registered in {@link #createComponents}. Read by every
+ * {@link org.opensearch.parquet.memory.ArrowBufferPool#createChildAllocator(String)}
+ * call so dynamic updates take effect for new child allocators without restart.
+ */
+ private volatile int maxPerVsrAllocationDivisor = ParquetSettings.MAX_PER_VSR_ALLOCATION_DIVISOR.get(Settings.EMPTY);
/** Creates a new ParquetDataFormatPlugin. */
public ParquetDataFormatPlugin() {}
@@ -91,10 +101,16 @@ public Collection createComponents(
NodeEnvironment nodeEnvironment,
NamedWriteableRegistry namedWriteableRegistry,
IndexNameExpressionResolver indexNameExpressionResolver,
- Supplier repositoriesServiceSupplier
+ Supplier repositoriesServiceSupplier,
+ PluginComponentRegistry pluginComponentRegistry
) {
this.settings = clusterService.getSettings();
this.threadPool = threadPool;
+ this.maxPerVsrAllocationDivisor = ParquetSettings.MAX_PER_VSR_ALLOCATION_DIVISOR.get(this.settings);
+ clusterService.getClusterSettings()
+ .addSettingsUpdateConsumer(ParquetSettings.MAX_PER_VSR_ALLOCATION_DIVISOR, v -> this.maxPerVsrAllocationDivisor = v);
+ this.nativeAllocator = pluginComponentRegistry.getComponent(ArrowNativeAllocator.class)
+ .orElseThrow(() -> new IllegalStateException("ArrowNativeAllocator not available; arrow-base plugin must be installed"));
return Collections.emptyList();
}
@@ -113,7 +129,9 @@ public DataFormat getDataFormat() {
() -> engineConfig.mapperService().getIndexSettings().getIndexMetadata().getMappingVersion(),
engineConfig.indexSettings(),
threadPool,
- engineConfig.checksumStrategies().get(ParquetDataFormat.PARQUET_DATA_FORMAT_NAME)
+ engineConfig.checksumStrategies().get(ParquetDataFormat.PARQUET_DATA_FORMAT_NAME),
+ () -> maxPerVsrAllocationDivisor,
+ nativeAllocator
);
}
diff --git a/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/ParquetSettings.java b/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/ParquetSettings.java
index ab58d0bfdf11c..ce9be28804537 100644
--- a/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/ParquetSettings.java
+++ b/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/ParquetSettings.java
@@ -105,6 +105,25 @@ private ParquetSettings() {}
Setting.Property.NodeScope
);
+ /**
+ * Per-VSR allocation as a divisor of the ingest pool's limit. Each child allocator
+ * created by {@link org.opensearch.parquet.memory.ArrowBufferPool#createChildAllocator(String)}
+ * is capped at {@code poolLimit / divisor}. Default 10 preserves the historical
+ * sizing. Dynamic — updates take effect on the next child-allocator creation.
+ *
+ * The setting is named {@code *_divisor} (not {@code *_ratio}) because larger
+ * values yield smaller per-VSR caps. A divisor of 10 means "each VSR may use up
+ * to 1/10 of the pool"; a divisor of 2 means "each VSR may use up to 1/2".
+ */
+ public static final Setting MAX_PER_VSR_ALLOCATION_DIVISOR = Setting.intSetting(
+ "parquet.max_per_vsr_allocation_divisor",
+ 10,
+ 1,
+ 100,
+ Setting.Property.NodeScope,
+ Setting.Property.Dynamic
+ );
+
/** File size threshold for in-memory sort vs streaming merge sort (default 32MB). */
public static final Setting SORT_IN_MEMORY_THRESHOLD = Setting.byteSizeSetting(
"index.parquet.sort_in_memory_threshold",
@@ -166,6 +185,7 @@ public static List> getSettings() {
BLOOM_FILTER_NDV,
MAX_NATIVE_ALLOCATION,
MAX_ROWS_PER_VSR,
+ MAX_PER_VSR_ALLOCATION_DIVISOR,
SORT_IN_MEMORY_THRESHOLD,
SORT_BATCH_SIZE,
ROW_GROUP_MAX_ROWS,
diff --git a/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/engine/ParquetIndexingEngine.java b/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/engine/ParquetIndexingEngine.java
index 91644145e12ea..3cabff3e236bf 100644
--- a/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/engine/ParquetIndexingEngine.java
+++ b/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/engine/ParquetIndexingEngine.java
@@ -11,6 +11,7 @@
import org.apache.arrow.vector.types.pojo.Schema;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
import org.opensearch.common.settings.Settings;
import org.opensearch.index.IndexSettings;
import org.opensearch.index.engine.dataformat.IndexingExecutionEngine;
@@ -43,6 +44,7 @@
import java.util.Collection;
import java.util.List;
import java.util.Map;
+import java.util.function.IntSupplier;
import java.util.function.Supplier;
import static org.opensearch.parquet.ParquetDataFormatPlugin.PARQUET_DATA_FORMAT;
@@ -102,7 +104,8 @@ public ParquetIndexingEngine(
Supplier schemaSupplier,
Supplier mappingVersionSupplier,
IndexSettings indexSettings,
- ThreadPool threadPool
+ ThreadPool threadPool,
+ ArrowNativeAllocator nativeAllocator
) {
this(
settings,
@@ -112,7 +115,9 @@ public ParquetIndexingEngine(
mappingVersionSupplier,
indexSettings,
threadPool,
- new PrecomputedChecksumStrategy()
+ new PrecomputedChecksumStrategy(),
+ () -> ParquetSettings.MAX_PER_VSR_ALLOCATION_DIVISOR.get(settings),
+ nativeAllocator
);
}
@@ -136,13 +141,55 @@ public ParquetIndexingEngine(
Supplier mappingVersionSupplier,
IndexSettings indexSettings,
ThreadPool threadPool,
- FormatChecksumStrategy checksumStrategy
+ FormatChecksumStrategy checksumStrategy,
+ ArrowNativeAllocator nativeAllocator
+ ) {
+ this(
+ settings,
+ dataFormat,
+ shardPath,
+ schemaSupplier,
+ mappingVersionSupplier,
+ indexSettings,
+ threadPool,
+ checksumStrategy,
+ () -> ParquetSettings.MAX_PER_VSR_ALLOCATION_DIVISOR.get(settings),
+ nativeAllocator
+ );
+ }
+
+ /**
+ * Creates a new ParquetIndexingEngine with a live divisor supplier.
+ *
+ * @param settings the node-level settings
+ * @param dataFormat the Parquet data format descriptor
+ * @param shardPath the shard path for file storage
+ * @param schemaSupplier the supplier for schema resolution
+ * @param mappingVersionSupplier the supplier for mapping version resolution
+ * @param indexSettings the index-level settings
+ * @param threadPool the thread pool for background native writes
+ * @param checksumStrategy the checksum strategy to use (shared with the directory)
+ * @param divisorSupplier live source for {@link ParquetSettings#MAX_PER_VSR_ALLOCATION_DIVISOR};
+ * read on every {@link org.opensearch.parquet.memory.ArrowBufferPool#createChildAllocator(String)}
+ * call so dynamic cluster-settings updates take effect
+ */
+ public ParquetIndexingEngine(
+ Settings settings,
+ ParquetDataFormat dataFormat,
+ ShardPath shardPath,
+ Supplier schemaSupplier,
+ Supplier mappingVersionSupplier,
+ IndexSettings indexSettings,
+ ThreadPool threadPool,
+ FormatChecksumStrategy checksumStrategy,
+ IntSupplier divisorSupplier,
+ ArrowNativeAllocator nativeAllocator
) {
this.dataFormat = dataFormat;
this.shardPath = shardPath;
this.schemaSupplier = schemaSupplier;
this.mappingVersionSupplier = mappingVersionSupplier;
- this.bufferPool = new ArrowBufferPool();
+ this.bufferPool = new ArrowBufferPool(settings, divisorSupplier, nativeAllocator);
this.indexSettings = indexSettings;
this.nodeSettings = settings;
this.threadPool = threadPool;
diff --git a/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/memory/ArrowBufferPool.java b/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/memory/ArrowBufferPool.java
index c6d1313c92f83..0ed0f711155ec 100644
--- a/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/memory/ArrowBufferPool.java
+++ b/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/memory/ArrowBufferPool.java
@@ -9,18 +9,15 @@
package org.opensearch.parquet.memory;
import org.apache.arrow.memory.BufferAllocator;
-import org.apache.arrow.memory.RootAllocator;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.opensearch.arrow.allocator.ArrowNativeAllocator;
import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
import org.opensearch.common.settings.Settings;
-import org.opensearch.common.unit.RatioValue;
-import org.opensearch.monitor.jvm.JvmInfo;
-import org.opensearch.monitor.os.OsProbe;
import org.opensearch.parquet.ParquetSettings;
import java.io.Closeable;
+import java.util.function.IntSupplier;
/**
* Arrow memory allocator pool for Parquet ingest operations.
@@ -34,49 +31,52 @@ public class ArrowBufferPool implements Closeable {
private static final Logger logger = LogManager.getLogger(ArrowBufferPool.class);
private final BufferAllocator poolAllocator;
- private final long maxChildAllocation;
- private final boolean ownsAllocator;
+ private final IntSupplier divisorSupplier;
/**
* Creates a new ArrowBufferPool backed by the unified native allocator's ingest pool.
+ *
+ * @param settings node settings (used by the {@link #ArrowBufferPool(Settings, ArrowNativeAllocator)}
+ * convenience ctor to derive a static divisor supplier; ignored
+ * here)
+ * @param divisorSupplier supplies the current value of
+ * {@link ParquetSettings#MAX_PER_VSR_ALLOCATION_DIVISOR}; read on
+ * every {@link #createChildAllocator(String)} call so dynamic
+ * cluster-settings updates take effect for new child allocators
+ * @param nativeAllocator the framework's unified native allocator, injected by
+ * {@code ParquetDataFormatPlugin#createComponents}
*/
- public ArrowBufferPool() {
- BufferAllocator alloc;
- boolean owns = false;
- try {
- alloc = ArrowNativeAllocator.instance().getPoolAllocator(NativeAllocatorPoolConfig.POOL_INGEST);
- } catch (IllegalStateException e) {
- alloc = new RootAllocator(Long.MAX_VALUE);
- owns = true;
- logger.warn("NativeAllocator not available, using standalone RootAllocator");
- }
- this.poolAllocator = alloc;
- this.ownsAllocator = owns;
- long limit = poolAllocator.getLimit();
- this.maxChildAllocation = limit == Long.MAX_VALUE ? Long.MAX_VALUE : limit / 10;
- logger.debug("ArrowBufferPool: limit={}, maxChildAllocation={}", limit, maxChildAllocation);
+ public ArrowBufferPool(Settings settings, IntSupplier divisorSupplier, ArrowNativeAllocator nativeAllocator) {
+ this.poolAllocator = nativeAllocator.getPoolAllocator(NativeAllocatorPoolConfig.POOL_INGEST);
+ this.divisorSupplier = divisorSupplier;
+ logger.debug("ArrowBufferPool: poolLimit={}", poolAllocator.getLimit());
}
/**
- * Creates a new ArrowBufferPool with a standalone allocator (legacy/test path).
+ * Test-only convenience constructor that derives the divisor statically from
+ * {@code settings}. Production callers go through
+ * {@link #ArrowBufferPool(Settings, IntSupplier, ArrowNativeAllocator)} so the divisor
+ * follows dynamic cluster-settings updates.
*
- * @param settings node settings used to derive the maximum native allocation
+ * @param settings node settings
+ * @param nativeAllocator the framework's unified native allocator (typically a fixture)
*/
- public ArrowBufferPool(Settings settings) {
- long maxAllocationInBytes = getMaxAllocationInBytes(settings);
- this.poolAllocator = new RootAllocator(maxAllocationInBytes);
- this.ownsAllocator = true;
- this.maxChildAllocation = maxAllocationInBytes / 10;
- logger.debug("ArrowBufferPool standalone: limit={}, maxChildAllocation={}", maxAllocationInBytes, maxChildAllocation);
+ public ArrowBufferPool(Settings settings, ArrowNativeAllocator nativeAllocator) {
+ this(settings, () -> ParquetSettings.MAX_PER_VSR_ALLOCATION_DIVISOR.get(settings), nativeAllocator);
}
/**
- * Creates a child allocator with the given name.
+ * Creates a child allocator with the given name. The cap is computed lazily from the
+ * pool's current limit and the latest divisor, so cluster-settings updates to
+ * {@code parquet.max_per_vsr_allocation_divisor} are picked up here.
*
* @param name the allocator name
* @return a new child buffer allocator
*/
public BufferAllocator createChildAllocator(String name) {
+ long limit = poolAllocator.getLimit();
+ int divisor = divisorSupplier.getAsInt();
+ long maxChildAllocation = limit == Long.MAX_VALUE ? Long.MAX_VALUE : limit / divisor;
return poolAllocator.newChildAllocator(name, 0, maxChildAllocation);
}
@@ -87,14 +87,8 @@ public long getTotalAllocatedBytes() {
@Override
public void close() {
- if (ownsAllocator) {
- poolAllocator.close();
- }
- }
-
- private static long getMaxAllocationInBytes(Settings settings) {
- long totalAvailableMemory = OsProbe.getInstance().getTotalPhysicalMemorySize() - JvmInfo.jvmInfo().getConfiguredMaxHeapSize();
- RatioValue ratio = RatioValue.parseRatioValue(ParquetSettings.MAX_NATIVE_ALLOCATION.get(settings));
- return (long) (totalAvailableMemory * ratio.getAsRatio());
+ // The framework owns the ingest pool's BufferAllocator; nothing to free here.
+ // Child allocators created via createChildAllocator are owned by their callers
+ // (VSRPool / ManagedVSR) and closed when those resources release.
}
}
diff --git a/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/vsr/VSRPool.java b/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/vsr/VSRPool.java
index c6c29f40e43b3..90430bf5091f5 100644
--- a/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/vsr/VSRPool.java
+++ b/sandbox/plugins/parquet-data-format/src/main/java/org/opensearch/parquet/vsr/VSRPool.java
@@ -12,6 +12,7 @@
import org.apache.arrow.vector.types.pojo.Schema;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import org.opensearch.arrow.allocator.AllocationRejection;
import org.opensearch.parquet.memory.ArrowBufferPool;
import java.io.IOException;
@@ -98,7 +99,12 @@ public boolean maybeRotateActiveVSR() throws IOException {
if (current.getRowCount() > 0) {
freezeVSR(current);
}
- ManagedVSR newVSR = createNewVSR();
+ // VSR rotation is a per-request operation: the active VSR filled up during ingest
+ // and we need a new one. If the INGEST pool is exhausted, surface the failure as
+ // OpenSearchRejectedExecutionException (HTTP 429) instead of Arrow's raw OOM.
+ // Startup-time VSR creation in initializeActiveVSR() is intentionally not wrapped:
+ // a failure there is a framework misconfiguration, not a backpressure signal.
+ ManagedVSR newVSR = AllocationRejection.wrap("vsr-rotation-" + poolId, this::createNewVSR);
if (activeVSR.compareAndSet(current, newVSR) == false) {
throw new IOException("Failed to set new active VSR during rotation");
}
diff --git a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/engine/ParquetDataFormatAwareEngineTests.java b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/engine/ParquetDataFormatAwareEngineTests.java
index 41ee376516de6..5efb4fede3711 100644
--- a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/engine/ParquetDataFormatAwareEngineTests.java
+++ b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/engine/ParquetDataFormatAwareEngineTests.java
@@ -115,14 +115,26 @@ public Query termQuery(Object value, QueryShardContext context) {
};
private Schema schema;
+ private org.opensearch.arrow.allocator.ArrowNativeAllocator nativeAllocator;
@Override
public void setUp() throws Exception {
super.setUp();
RustBridge.initLogger();
+ nativeAllocator = new org.opensearch.arrow.allocator.ArrowNativeAllocator(Long.MAX_VALUE);
+ nativeAllocator.getOrCreatePool(org.opensearch.arrow.spi.NativeAllocatorPoolConfig.POOL_INGEST, 0L, Long.MAX_VALUE);
schema = buildSchema();
}
+ @Override
+ public void tearDown() throws Exception {
+ if (nativeAllocator != null) {
+ nativeAllocator.close();
+ nativeAllocator = null;
+ }
+ super.tearDown();
+ }
+
@Override
protected Store createStore() throws IOException {
// Parquet engine needs a real filesystem path for writing .parquet files
@@ -183,7 +195,8 @@ public org.opensearch.index.engine.dataformat.DataFormat getDataFormat() {
() -> 1L,
engineConfig.indexSettings(),
threadPool,
- new PrecomputedChecksumStrategy()
+ new PrecomputedChecksumStrategy(),
+ nativeAllocator
);
}
};
diff --git a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/engine/ParquetIndexingEngineTests.java b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/engine/ParquetIndexingEngineTests.java
index aaacae8c30fa0..12a58da86d80b 100644
--- a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/engine/ParquetIndexingEngineTests.java
+++ b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/engine/ParquetIndexingEngineTests.java
@@ -13,6 +13,8 @@
import org.apache.arrow.vector.types.pojo.FieldType;
import org.apache.arrow.vector.types.pojo.Schema;
import org.opensearch.Version;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
+import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.common.settings.Settings;
import org.opensearch.core.index.shard.ShardId;
@@ -56,6 +58,7 @@
public class ParquetIndexingEngineTests extends OpenSearchTestCase {
+ private org.opensearch.arrow.allocator.ArrowNativeAllocator nativeAllocator;
private MappedFieldType idField;
private MappedFieldType nameField;
private MappedFieldType scoreField;
@@ -68,6 +71,8 @@ public class ParquetIndexingEngineTests extends OpenSearchTestCase {
public void setUp() throws Exception {
super.setUp();
RustBridge.initLogger();
+ nativeAllocator = new ArrowNativeAllocator(Long.MAX_VALUE);
+ nativeAllocator.getOrCreatePool(NativeAllocatorPoolConfig.POOL_INGEST, 0L, Long.MAX_VALUE);
idField = new NumberFieldMapper.NumberFieldType("id", NumberFieldMapper.NumberType.INTEGER);
nameField = new KeywordFieldMapper.KeywordFieldType("name");
scoreField = new NumberFieldMapper.NumberFieldType("score", NumberFieldMapper.NumberType.LONG);
@@ -90,6 +95,10 @@ public void setUp() throws Exception {
@Override
public void tearDown() throws Exception {
terminate(threadPool);
+ if (nativeAllocator != null) {
+ nativeAllocator.close();
+ nativeAllocator = null;
+ }
super.tearDown();
}
@@ -205,7 +214,8 @@ private ParquetIndexingEngine createEngine() {
() -> ArrowSchemaBuilder.getSchema(mapperService),
() -> mapperService.getIndexSettings().getIndexMetadata().getMappingVersion(),
indexSettings,
- threadPool
+ threadPool,
+ nativeAllocator
);
} catch (Exception e) {
throw new RuntimeException(e);
diff --git a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/memory/ArrowBufferPoolTests.java b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/memory/ArrowBufferPoolTests.java
index 7f7c4b074c10a..1bbfe369e5d2a 100644
--- a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/memory/ArrowBufferPoolTests.java
+++ b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/memory/ArrowBufferPoolTests.java
@@ -10,57 +10,134 @@
import org.apache.arrow.memory.ArrowBuf;
import org.apache.arrow.memory.BufferAllocator;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
+import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
import org.opensearch.common.settings.Settings;
import org.opensearch.test.OpenSearchTestCase;
public class ArrowBufferPoolTests extends OpenSearchTestCase {
+ private ArrowNativeAllocator nativeAllocator;
+
+ @Override
+ public void setUp() throws Exception {
+ super.setUp();
+ // Each test gets its own allocator with the standard pools pre-created.
+ // Production code receives this via dependency injection; tests build it explicitly.
+ nativeAllocator = new ArrowNativeAllocator(Long.MAX_VALUE);
+ nativeAllocator.getOrCreatePool(NativeAllocatorPoolConfig.POOL_INGEST, 0L, Long.MAX_VALUE);
+ }
+
+ @Override
+ public void tearDown() throws Exception {
+ if (nativeAllocator != null) {
+ nativeAllocator.close();
+ nativeAllocator = null;
+ }
+ super.tearDown();
+ }
+
public void testAllocatedBytesIncreasesOnAllocation() {
- try (ArrowBufferPool pool = new ArrowBufferPool(Settings.EMPTY)) {
- BufferAllocator child = pool.createChildAllocator("alloc-test");
+ ArrowBufferPool pool = new ArrowBufferPool(Settings.EMPTY, nativeAllocator);
+ BufferAllocator child = pool.createChildAllocator("alloc-test");
+ try {
assertNotNull(child);
assertEquals(0, pool.getTotalAllocatedBytes());
ArrowBuf buf = child.buffer(1024);
assertTrue(pool.getTotalAllocatedBytes() > 0);
buf.close();
+ } finally {
child.close();
+ pool.close();
}
}
public void testMultipleChildAllocators() {
- try (ArrowBufferPool pool = new ArrowBufferPool(Settings.EMPTY)) {
- BufferAllocator c1 = pool.createChildAllocator("c1");
- BufferAllocator c2 = pool.createChildAllocator("c2");
+ ArrowBufferPool pool = new ArrowBufferPool(Settings.EMPTY, nativeAllocator);
+ BufferAllocator c1 = pool.createChildAllocator("c1");
+ BufferAllocator c2 = pool.createChildAllocator("c2");
+ try {
ArrowBuf b1 = c1.buffer(512);
ArrowBuf b2 = c2.buffer(512);
assertTrue(pool.getTotalAllocatedBytes() >= 1024);
b1.close();
b2.close();
+ } finally {
c1.close();
c2.close();
+ pool.close();
}
}
public void testAllocatedBytesDecreasesAfterFree() {
- try (ArrowBufferPool pool = new ArrowBufferPool(Settings.EMPTY)) {
- BufferAllocator child = pool.createChildAllocator("free-test");
+ ArrowBufferPool pool = new ArrowBufferPool(Settings.EMPTY, nativeAllocator);
+ BufferAllocator child = pool.createChildAllocator("free-test");
+ try {
ArrowBuf buf = child.buffer(1024);
assertTrue(pool.getTotalAllocatedBytes() > 0);
buf.close();
+ } finally {
child.close();
- assertEquals(0, pool.getTotalAllocatedBytes());
+ pool.close();
}
+ // Pool's BufferAllocator is owned by the framework; the assertion now confirms
+ // the *child* that we closed returned its bytes to the parent.
+ assertEquals(0, pool.getTotalAllocatedBytes());
+ }
+
+ public void testChildAllocatorLimitScalesWithDivisor() {
+ // Both pools share the same framework ingest pool; divisor distinguishes the children.
+ Settings strict = Settings.builder().put("parquet.max_per_vsr_allocation_divisor", 10).build();
+ Settings loose = Settings.builder().put("parquet.max_per_vsr_allocation_divisor", 2).build();
+ ArrowBufferPool poolStrict = new ArrowBufferPool(strict, nativeAllocator);
+ ArrowBufferPool poolLoose = new ArrowBufferPool(loose, nativeAllocator);
+ BufferAllocator childStrict = poolStrict.createChildAllocator("c-strict");
+ BufferAllocator childLoose = poolLoose.createChildAllocator("c-loose");
+ try {
+ long limitStrict = childStrict.getLimit();
+ long limitLoose = childLoose.getLimit();
+ // The setUp ingest pool has limit Long.MAX_VALUE → both children clamp at
+ // Long.MAX_VALUE. Validate the dynamic-divisor path separately in
+ // testDivisorIsReadDynamicallyOnEachChildCreation.
+ if (limitStrict != Long.MAX_VALUE && limitLoose != Long.MAX_VALUE) {
+ assertEquals(5L * limitStrict, limitLoose, 1);
+ }
+ } finally {
+ childStrict.close();
+ childLoose.close();
+ poolStrict.close();
+ poolLoose.close();
+ }
+ }
+
+ public void testDivisorIsReadDynamicallyOnEachChildCreation() {
+ // Constrain the framework's ingest pool to a finite limit so the divisor matters.
+ nativeAllocator.setPoolLimit(NativeAllocatorPoolConfig.POOL_INGEST, 1024L * 1024 * 1024);
+
+ // Mutable supplier emulates a dynamic cluster-settings update.
+ int[] divisor = new int[] { 10 };
+ ArrowBufferPool pool = new ArrowBufferPool(Settings.EMPTY, () -> divisor[0], nativeAllocator);
+ BufferAllocator beforeUpdate = pool.createChildAllocator("c-before");
+ long limitBefore = beforeUpdate.getLimit();
+ beforeUpdate.close();
+
+ divisor[0] = 2; // simulate dynamic settings update
+ BufferAllocator afterUpdate = pool.createChildAllocator("c-after");
+ long limitAfter = afterUpdate.getLimit();
+ afterUpdate.close();
+ pool.close();
+
+ // divisor went from 10 → 2; new child should be ~5x the old.
+ assertEquals(5L * limitBefore, limitAfter, 1);
+ }
+
+ public void testDivisorSettingRejectsZero() {
+ Settings s = Settings.builder().put("parquet.max_per_vsr_allocation_divisor", 0).build();
+ expectThrows(IllegalArgumentException.class, () -> org.opensearch.parquet.ParquetSettings.MAX_PER_VSR_ALLOCATION_DIVISOR.get(s));
}
- public void testCloseWithOpenChildAllocatorThrows() {
- ArrowBufferPool pool = new ArrowBufferPool(Settings.EMPTY);
- BufferAllocator child = pool.createChildAllocator("leaked-child");
- ArrowBuf buf = child.buffer(1024);
- // Closing pool with outstanding child allocations throws
- IllegalStateException e = expectThrows(IllegalStateException.class, pool::close);
- assertTrue(e.getMessage().contains("Memory was leaked"));
- // First close marked root as closed, so child.close() can't notify parent.
- // Release buffer directly — Arrow frees the underlying memory even if allocator is closed.
- buf.getReferenceManager().release();
+ public void testDivisorSettingRejectsNegative() {
+ Settings s = Settings.builder().put("parquet.max_per_vsr_allocation_divisor", -1).build();
+ expectThrows(IllegalArgumentException.class, () -> org.opensearch.parquet.ParquetSettings.MAX_PER_VSR_ALLOCATION_DIVISOR.get(s));
}
}
diff --git a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/vsr/VSRManagerTests.java b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/vsr/VSRManagerTests.java
index a98a9fbd31973..df07efd3c2172 100644
--- a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/vsr/VSRManagerTests.java
+++ b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/vsr/VSRManagerTests.java
@@ -14,6 +14,7 @@
import org.apache.arrow.vector.types.pojo.FieldType;
import org.apache.arrow.vector.types.pojo.Schema;
import org.opensearch.Version;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.common.settings.Settings;
import org.opensearch.index.IndexSettings;
@@ -37,6 +38,7 @@
public class VSRManagerTests extends OpenSearchTestCase {
+ private ArrowNativeAllocator nativeAllocator;
private ArrowBufferPool bufferPool;
private Schema schema;
private ThreadPool threadPool;
@@ -46,7 +48,9 @@ public class VSRManagerTests extends OpenSearchTestCase {
public void setUp() throws Exception {
super.setUp();
RustBridge.initLogger();
- bufferPool = new ArrowBufferPool(Settings.EMPTY);
+ nativeAllocator = new ArrowNativeAllocator(Long.MAX_VALUE);
+ nativeAllocator.getOrCreatePool(org.opensearch.arrow.spi.NativeAllocatorPoolConfig.POOL_INGEST, 0L, Long.MAX_VALUE);
+ bufferPool = new ArrowBufferPool(Settings.EMPTY, nativeAllocator);
schema = new Schema(List.of(new Field("val", FieldType.nullable(new ArrowType.Int(32, true)), null)));
Settings indexSettingsBuilder = Settings.builder()
.put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT)
@@ -72,6 +76,10 @@ public void setUp() throws Exception {
public void tearDown() throws Exception {
terminate(threadPool);
bufferPool.close();
+ if (nativeAllocator != null) {
+ nativeAllocator.close();
+ nativeAllocator = null;
+ }
super.tearDown();
}
diff --git a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/vsr/VSRPoolTests.java b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/vsr/VSRPoolTests.java
index 26231488d7ecd..8cb2626921fad 100644
--- a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/vsr/VSRPoolTests.java
+++ b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/vsr/VSRPoolTests.java
@@ -12,6 +12,8 @@
import org.apache.arrow.vector.types.pojo.Field;
import org.apache.arrow.vector.types.pojo.FieldType;
import org.apache.arrow.vector.types.pojo.Schema;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
+import org.opensearch.arrow.spi.NativeAllocatorPoolConfig;
import org.opensearch.common.settings.Settings;
import org.opensearch.parquet.memory.ArrowBufferPool;
import org.opensearch.test.OpenSearchTestCase;
@@ -21,19 +23,26 @@
public class VSRPoolTests extends OpenSearchTestCase {
+ private ArrowNativeAllocator nativeAllocator;
private ArrowBufferPool bufferPool;
private Schema schema;
@Override
public void setUp() throws Exception {
super.setUp();
- bufferPool = new ArrowBufferPool(Settings.EMPTY);
+ nativeAllocator = new ArrowNativeAllocator(Long.MAX_VALUE);
+ nativeAllocator.getOrCreatePool(NativeAllocatorPoolConfig.POOL_INGEST, 0L, Long.MAX_VALUE);
+ bufferPool = new ArrowBufferPool(Settings.EMPTY, nativeAllocator);
schema = new Schema(List.of(new Field("val", FieldType.nullable(new ArrowType.Int(32, true)), null)));
}
@Override
public void tearDown() throws Exception {
bufferPool.close();
+ if (nativeAllocator != null) {
+ nativeAllocator.close();
+ nativeAllocator = null;
+ }
super.tearDown();
}
diff --git a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/writer/ParquetWriterTests.java b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/writer/ParquetWriterTests.java
index bc203b177e11a..6c5dd20dce46c 100644
--- a/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/writer/ParquetWriterTests.java
+++ b/sandbox/plugins/parquet-data-format/src/test/java/org/opensearch/parquet/writer/ParquetWriterTests.java
@@ -11,6 +11,7 @@
import org.apache.arrow.vector.types.pojo.Field;
import org.apache.arrow.vector.types.pojo.Schema;
import org.opensearch.Version;
+import org.opensearch.arrow.allocator.ArrowNativeAllocator;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.common.settings.Settings;
import org.opensearch.index.IndexSettings;
@@ -40,6 +41,7 @@
public class ParquetWriterTests extends OpenSearchTestCase {
+ private ArrowNativeAllocator nativeAllocator;
private ArrowBufferPool bufferPool;
private MappedFieldType idField;
private MappedFieldType nameField;
@@ -52,7 +54,9 @@ public class ParquetWriterTests extends OpenSearchTestCase {
public void setUp() throws Exception {
super.setUp();
RustBridge.initLogger();
- bufferPool = new ArrowBufferPool(Settings.EMPTY);
+ nativeAllocator = new ArrowNativeAllocator(Long.MAX_VALUE);
+ nativeAllocator.getOrCreatePool(org.opensearch.arrow.spi.NativeAllocatorPoolConfig.POOL_INGEST, 0L, Long.MAX_VALUE);
+ bufferPool = new ArrowBufferPool(Settings.EMPTY, nativeAllocator);
idField = new NumberFieldMapper.NumberFieldType("id", NumberFieldMapper.NumberType.INTEGER);
nameField = new KeywordFieldMapper.KeywordFieldType("name");
scoreField = new NumberFieldMapper.NumberFieldType("score", NumberFieldMapper.NumberType.LONG);
@@ -81,6 +85,10 @@ public void setUp() throws Exception {
public void tearDown() throws Exception {
terminate(threadPool);
bufferPool.close();
+ if (nativeAllocator != null) {
+ nativeAllocator.close();
+ nativeAllocator = null;
+ }
super.tearDown();
}
diff --git a/sandbox/qa/analytics-engine-coordinator/build.gradle b/sandbox/qa/analytics-engine-coordinator/build.gradle
index 03b15b0bf1ea3..6290abd8ea451 100644
--- a/sandbox/qa/analytics-engine-coordinator/build.gradle
+++ b/sandbox/qa/analytics-engine-coordinator/build.gradle
@@ -36,7 +36,8 @@ dependencies {
// these directly without going through analytics-engine's internal classes.
internalClusterTestImplementation project(':sandbox:libs:analytics-api')
- // Arrow base — provides Arrow memory/vector classes.
+ // Arrow base — provides Arrow memory/vector classes and the unified native-allocator
+ // framework required by FlightStreamPlugin and the analytics path.
internalClusterTestImplementation project(':plugins:arrow-base')
// Arrow Flight streaming transport — provides the streaming TransportService.
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java
index 46ee4b5985f9b..6709eb44498e9 100644
--- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java
@@ -40,6 +40,10 @@
import org.opensearch.cluster.service.ClusterManagerThrottlingStats;
import org.opensearch.common.Nullable;
import org.opensearch.common.cache.service.NodeCacheStats;
+import org.opensearch.common.io.stream.BytesStreamOutput;
+import org.opensearch.core.common.bytes.BytesReference;
+import org.opensearch.core.common.io.stream.NamedWriteableAwareStreamInput;
+import org.opensearch.core.common.io.stream.NamedWriteableRegistry;
import org.opensearch.core.common.io.stream.StreamInput;
import org.opensearch.core.common.io.stream.StreamOutput;
import org.opensearch.core.indices.breaker.AllCircuitBreakerStats;
@@ -61,6 +65,7 @@
import org.opensearch.node.NodesResourceUsageStats;
import org.opensearch.node.remotestore.RemoteStoreNodeStats;
import org.opensearch.plugin.stats.AnalyticsBackendNativeMemoryStats;
+import org.opensearch.plugins.PluginNodeStats;
import org.opensearch.ratelimitting.admissioncontrol.stats.AdmissionControlStats;
import org.opensearch.repositories.RepositoriesStats;
import org.opensearch.script.ScriptCacheStats;
@@ -72,6 +77,8 @@
import org.opensearch.transport.TransportStats;
import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.Map;
/**
@@ -167,6 +174,8 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment {
@Nullable
private RemoteStoreNodeStats remoteStoreNodeStats;
+ private Map pluginStats;
+
@Nullable
private AnalyticsBackendNativeMemoryStats nativeMemoryStats;
@@ -256,6 +265,11 @@ public NodeStats(StreamInput in) throws IOException {
} else {
remoteStoreNodeStats = null;
}
+ if (in.getVersion().onOrAfter(Version.V_3_7_0)) {
+ pluginStats = readPluginStats(in);
+ } else {
+ pluginStats = Collections.emptyMap();
+ }
if (in.getVersion().onOrAfter(Version.V_3_7_0)) {
nativeMemoryStats = in.readOptionalWriteable(AnalyticsBackendNativeMemoryStats::new);
} else {
@@ -263,6 +277,46 @@ public NodeStats(StreamInput in) throws IOException {
}
}
+ /**
+ * Reads the plugin nodeStats map from the wire. Each entry is framed as
+ * {@code (name, length-prefixed bytes)} so a receiver can skip any entry whose
+ * {@link PluginNodeStats} type is not registered locally — for example a
+ * coordinator that lacks a plugin a data node has installed. Unknown entries
+ * are dropped silently (counted in the debug log) instead of failing the entire
+ * NodeStats deserialization for that node.
+ */
+ private static Map readPluginStats(StreamInput in) throws IOException {
+ int size = in.readVInt();
+ if (size == 0) {
+ return Collections.emptyMap();
+ }
+ NamedWriteableRegistry registry = in.namedWriteableRegistry();
+ Map result = new HashMap<>(size);
+ for (int i = 0; i < size; i++) {
+ String name = in.readString();
+ BytesReference payload = in.readBytesReference();
+ if (registry == null) {
+ // No registry attached to the parent stream — we can't deserialize any
+ // entry. Drop the whole map. This branch is defensive; production paths
+ // always set a registry when they expect named writeables.
+ continue;
+ }
+ try (
+ StreamInput rawIn = payload.streamInput();
+ NamedWriteableAwareStreamInput payloadIn = new NamedWriteableAwareStreamInput(rawIn, registry)
+ ) {
+ payloadIn.setVersion(in.getVersion());
+ PluginNodeStats stats = payloadIn.readNamedWriteable(PluginNodeStats.class);
+ result.put(name, stats);
+ } catch (IOException | IllegalArgumentException e) {
+ // Receiver doesn't have the plugin's NamedWriteable registered (typical
+ // during rolling upgrades or in a non-uniform plugin install). Drop the
+ // entry; the rest of NodeStats remains decodable.
+ }
+ }
+ return result;
+ }
+
public NodeStats(
DiscoveryNode node,
long timestamp,
@@ -294,6 +348,7 @@ public NodeStats(
@Nullable AdmissionControlStats admissionControlStats,
@Nullable NodeCacheStats nodeCacheStats,
@Nullable RemoteStoreNodeStats remoteStoreNodeStats,
+ @Nullable Map pluginStats,
@Nullable AnalyticsBackendNativeMemoryStats nativeMemoryStats
) {
super(node);
@@ -326,6 +381,7 @@ public NodeStats(
this.admissionControlStats = admissionControlStats;
this.nodeCacheStats = nodeCacheStats;
this.remoteStoreNodeStats = remoteStoreNodeStats;
+ this.pluginStats = pluginStats == null ? Collections.emptyMap() : pluginStats;
this.nativeMemoryStats = nativeMemoryStats;
}
@@ -494,6 +550,10 @@ public RemoteStoreNodeStats getRemoteStoreNodeStats() {
return remoteStoreNodeStats;
}
+ public Map getPluginStats() {
+ return pluginStats == null ? Collections.emptyMap() : pluginStats;
+ }
+
/**
* Returns the analytics backend native memory stats, or {@code null} if not available.
*/
@@ -563,11 +623,32 @@ public void writeTo(StreamOutput out) throws IOException {
if (out.getVersion().onOrAfter(Version.V_2_18_0)) {
out.writeOptionalWriteable(remoteStoreNodeStats);
}
+ if (out.getVersion().onOrAfter(Version.V_3_7_0)) {
+ writePluginStats(out, pluginStats == null ? Collections.emptyMap() : pluginStats);
+ }
if (out.getVersion().onOrAfter(Version.V_3_7_0)) {
out.writeOptionalWriteable(nativeMemoryStats);
}
}
+ /**
+ * Writes the plugin nodeStats map with each entry framed as
+ * {@code (name, length-prefixed bytes)} so a receiver can skip entries whose
+ * {@link PluginNodeStats} type is not registered locally. Symmetric with
+ * {@link #readPluginStats(StreamInput)}.
+ */
+ private static void writePluginStats(StreamOutput out, Map stats) throws IOException {
+ out.writeVInt(stats.size());
+ for (Map.Entry entry : stats.entrySet()) {
+ out.writeString(entry.getKey());
+ try (BytesStreamOutput payloadOut = new BytesStreamOutput()) {
+ payloadOut.setVersion(out.getVersion());
+ payloadOut.writeNamedWriteable(entry.getValue());
+ out.writeBytesReference(payloadOut.bytes());
+ }
+ }
+ }
+
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
@@ -675,6 +756,11 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws
if (getRemoteStoreNodeStats() != null) {
getRemoteStoreNodeStats().toXContent(builder, params);
}
+ for (Map.Entry e : getPluginStats().entrySet()) {
+ builder.startObject(e.getKey());
+ e.getValue().toXContent(builder, params);
+ builder.endObject();
+ }
if (getAnalyticsBackendNativeMemoryStats() != null) {
getAnalyticsBackendNativeMemoryStats().toXContent(builder, params);
}
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java
index 2947efa832eda..badb9596612ae 100644
--- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java
@@ -211,6 +211,7 @@ public enum Metric {
ADMISSION_CONTROL("admission_control"),
CACHE_STATS("caches"),
REMOTE_STORE("remote_store"),
+ PLUGIN_STATS("plugin_stats"),
NATIVE_MEMORY("native_memory");
private String metricName;
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java
index a62d046fcb6d3..08adb985742a9 100644
--- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java
@@ -131,6 +131,7 @@ protected NodeStats nodeOperation(NodeStatsRequest nodeStatsRequest) {
NodesStatsRequest.Metric.ADMISSION_CONTROL.containedIn(metrics),
NodesStatsRequest.Metric.CACHE_STATS.containedIn(metrics),
NodesStatsRequest.Metric.REMOTE_STORE.containedIn(metrics),
+ NodesStatsRequest.Metric.PLUGIN_STATS.containedIn(metrics),
NodesStatsRequest.Metric.NATIVE_MEMORY.containedIn(metrics)
);
}
diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java
index a83cecfd21821..9bdfddd140ca5 100644
--- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java
+++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java
@@ -200,6 +200,7 @@ protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeReq
false,
false,
false,
+ false,
false
);
List shardsStats = new ArrayList<>();
diff --git a/server/src/main/java/org/opensearch/node/NodeService.java b/server/src/main/java/org/opensearch/node/NodeService.java
index 4d249c3ba72a7..eed5b35905f66 100644
--- a/server/src/main/java/org/opensearch/node/NodeService.java
+++ b/server/src/main/java/org/opensearch/node/NodeService.java
@@ -55,6 +55,8 @@
import org.opensearch.ingest.IngestService;
import org.opensearch.monitor.MonitorService;
import org.opensearch.node.remotestore.RemoteStoreNodeStats;
+import org.opensearch.plugins.Plugin;
+import org.opensearch.plugins.PluginNodeStats;
import org.opensearch.plugins.PluginsService;
import org.opensearch.ratelimitting.admissioncontrol.AdmissionControlService;
import org.opensearch.repositories.RepositoriesService;
@@ -68,6 +70,9 @@
import java.io.Closeable;
import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
import java.util.concurrent.TimeUnit;
/**
@@ -245,6 +250,7 @@ public NodeStats stats(
boolean admissionControl,
boolean cacheService,
boolean remoteStoreNodeStats,
+ boolean pluginStats,
boolean nativeMemory
) {
// for indices stats we want to include previous allocated shards stats as well (it will
@@ -280,10 +286,21 @@ public NodeStats stats(
admissionControl ? this.admissionControlService.stats() : null,
cacheService ? this.cacheService.stats(indices) : null,
remoteStoreNodeStats ? new RemoteStoreNodeStats() : null,
+ pluginStats ? collectPluginStats() : Collections.emptyMap(),
nativeMemory ? monitorService.memoryReportingService().nativeStats() : null
);
}
+ private Map collectPluginStats() {
+ Map result = new HashMap<>();
+ for (Plugin plugin : pluginService.filterPlugins(Plugin.class)) {
+ for (PluginNodeStats stats : plugin.nodeStats()) {
+ result.put(stats.getWriteableName(), stats);
+ }
+ }
+ return result;
+ }
+
public IngestService getIngestService() {
return ingestService;
}
diff --git a/server/src/main/java/org/opensearch/node/resource/tracker/ResourceTrackerSettings.java b/server/src/main/java/org/opensearch/node/resource/tracker/ResourceTrackerSettings.java
index d6bc2db709d41..baacf68a8e4f1 100644
--- a/server/src/main/java/org/opensearch/node/resource/tracker/ResourceTrackerSettings.java
+++ b/server/src/main/java/org/opensearch/node/resource/tracker/ResourceTrackerSettings.java
@@ -12,6 +12,7 @@
import org.opensearch.common.settings.Settings;
import org.opensearch.common.unit.TimeValue;
import org.opensearch.core.common.unit.ByteSizeValue;
+import org.opensearch.monitor.os.OsProbe;
/**
* Settings related to resource usage trackers such as polling interval, window duration etc
@@ -88,16 +89,68 @@ private static class Defaults {
);
/**
- * Absolute native-memory budget for this node, in bytes. When the value is {@link ByteSizeValue#ZERO}
- * (default) the tracker treats the budget as unconfigured and reports {@code 0%}.
+ * Absolute native-memory budget for this node, in bytes.
+ *
+ * Default: 79% of {@code RAM - JVM heap}. The JVM-aware
+ * {@link OsProbe#getTotalPhysicalMemorySize()} returns the cgroup-limited RAM in
+ * containers and total physical RAM on bare metal; subtracting {@link Runtime#maxMemory()}
+ * (the configured {@code -Xmx}) gives the off-heap budget the operator implicitly has
+ * available, and 79% of that is allocated to AC's tracker. The remaining ~21% is left
+ * unmanaged for Lucene mmap, Liquid cache, OS page cache, and other native consumers
+ * not bounded by the framework. Operators with predominantly analytics workloads can
+ * raise this toward 100% of off-heap; search-heavy workloads needing more page cache
+ * should lower it explicitly.
+ *
+ *
Falls back to {@link ByteSizeValue#ZERO} (unconfigured) when either probe is unreliable
+ * — RAM probe returns 0 in some restricted containers, or {@code maxMemory()} is somehow
+ * larger than RAM. With ZERO, the tracker reports {@code 0%} and admission control treats the
+ * budget as unset; downstream framework defaults (root limit, pool maxes, DataFusion pool)
+ * fall back to {@link Long#MAX_VALUE} unbounded mode, preserving pre-AC behaviour.
*/
public static final Setting NODE_NATIVE_MEMORY_LIMIT_SETTING = Setting.byteSizeSetting(
"node.native_memory.limit",
- ByteSizeValue.ZERO,
+ s -> deriveNativeMemoryLimitDefault(),
Setting.Property.Dynamic,
Setting.Property.NodeScope
);
+ /**
+ * Computes the default for {@link #NODE_NATIVE_MEMORY_LIMIT_SETTING} as {@code RAM - JVM heap}.
+ * Returns the bytes-as-string representation expected by the {@link Setting} parser.
+ *
+ * Returns {@code "0b"} (unset, AC unconfigured) when:
+ *
+ * {@link OsProbe#getTotalPhysicalMemorySize()} returns {@code 0} (e.g. restricted
+ * containers where {@code /proc/meminfo} cgroup files are not readable)
+ * {@code -Xmx} (heap) is somehow {@code >=} reported physical RAM (degenerate config
+ * or container-aware probe disagreeing with the JVM heap)
+ *
+ * Both fall-back paths preserve the pre-default opt-in semantics: AC stays unconfigured,
+ * downstream framework caps stay unbounded.
+ */
+ static String deriveNativeMemoryLimitDefault() {
+ long ram = OsProbe.getInstance().getTotalPhysicalMemorySize();
+ long heap = Runtime.getRuntime().maxMemory();
+ return deriveNativeMemoryLimitDefault(ram, heap);
+ }
+
+ /**
+ * Pure overload of {@link #deriveNativeMemoryLimitDefault()} with the OS- and
+ * JVM-derived inputs lifted to parameters so unit tests can exercise the fallback
+ * branches deterministically without mocking {@link OsProbe} or {@link Runtime}.
+ */
+ static String deriveNativeMemoryLimitDefault(long ram, long heap) {
+ if (ram <= 0 || heap <= 0 || heap >= ram) {
+ return "0b";
+ }
+ // 79% of off-heap (RAM - heap) leaves ~21% headroom for unmanaged native consumers:
+ // Lucene mmap, Liquid cache, OS page cache, sidecar processes. Matches the partitioning
+ // model in PR #21732. Operators with predominantly analytics workloads can raise this
+ // toward 100%; search-heavy workloads needing more page cache should lower it.
+ long offHeap = ram - heap;
+ return Long.toString(offHeap * 79 / 100) + "b";
+ }
+
/**
* Percentage of the native-memory limit that is reserved as buffer (not usable). The effective
* native memory the tracker divides against is {@code limit - (limit * bufferPercent / 100)}.
diff --git a/server/src/main/java/org/opensearch/plugins/Plugin.java b/server/src/main/java/org/opensearch/plugins/Plugin.java
index e5aee0e780776..3220d80d9fe30 100644
--- a/server/src/main/java/org/opensearch/plugins/Plugin.java
+++ b/server/src/main/java/org/opensearch/plugins/Plugin.java
@@ -39,6 +39,7 @@
import org.opensearch.cluster.metadata.Metadata;
import org.opensearch.cluster.node.DiscoveryNodeRole;
import org.opensearch.cluster.service.ClusterService;
+import org.opensearch.common.annotation.ExperimentalApi;
import org.opensearch.common.annotation.PublicApi;
import org.opensearch.common.inject.Module;
import org.opensearch.common.lifecycle.LifecycleComponent;
@@ -249,6 +250,23 @@ public List> getSettings() {
return Collections.emptyList();
}
+ /**
+ * Returns plugin-contributed node statistics that surface under {@code _nodes/stats}.
+ * Each entry renders at top-level under {@code nodes..}.
+ *
+ * Plugins that override this method must also register the concrete
+ * {@link PluginNodeStats} subclass via {@link #getNamedWriteables()} so the
+ * coordinator can deserialize per-node payloads received over transport.
+ *
+ *
Default: empty.
+ *
+ * @opensearch.experimental
+ */
+ @ExperimentalApi
+ public List nodeStats() {
+ return Collections.emptyList();
+ }
+
/**
* Returns a list of additional settings filter for this plugin
*/
diff --git a/server/src/main/java/org/opensearch/plugins/PluginNodeStats.java b/server/src/main/java/org/opensearch/plugins/PluginNodeStats.java
new file mode 100644
index 0000000000000..8b50a155fa536
--- /dev/null
+++ b/server/src/main/java/org/opensearch/plugins/PluginNodeStats.java
@@ -0,0 +1,30 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ *
+ * The OpenSearch Contributors require contributions made to
+ * this file be licensed under the Apache-2.0 license or a
+ * compatible open source license.
+ */
+
+package org.opensearch.plugins;
+
+import org.opensearch.common.annotation.ExperimentalApi;
+import org.opensearch.core.common.io.stream.NamedWriteable;
+import org.opensearch.core.xcontent.ToXContentFragment;
+
+/**
+ * Plugin-contributed payload that surfaces in {@code _nodes/stats} under
+ * {@code nodes..}. Combines the wire-protocol
+ * contract ({@link NamedWriteable}) with the JSON-rendering contract
+ * ({@link ToXContentFragment}) that {@link org.opensearch.action.admin.cluster.node.stats.NodeStats}
+ * needs to merge the contribution into the standard {@code _nodes/stats} response shape.
+ *
+ * Plugins emit instances of their concrete type from
+ * {@link Plugin#nodeStats()} and register the type with
+ * {@link Plugin#getNamedWriteables()} so the coordinator can deserialize
+ * per-node payloads it received over transport.
+ *
+ * @opensearch.experimental
+ */
+@ExperimentalApi
+public interface PluginNodeStats extends NamedWriteable, ToXContentFragment {}
diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/AnalyticsBackendNativeMemoryStatsVersionGateTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/AnalyticsBackendNativeMemoryStatsVersionGateTests.java
index 3eb34abf49ad7..1ef711a0ea286 100644
--- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/AnalyticsBackendNativeMemoryStatsVersionGateTests.java
+++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/AnalyticsBackendNativeMemoryStatsVersionGateTests.java
@@ -156,6 +156,7 @@ private NodeStats createNodeStatsWithNativeMemory(AnalyticsBackendNativeMemorySt
null, // admissionControlStats
null, // nodeCacheStats
null, // remoteStoreNodeStats
+ null, // pluginStats
nativeMemoryStats
);
}
diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java
index 6fda6fbb44efa..cc0cc23335417 100644
--- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java
+++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java
@@ -62,7 +62,10 @@
import org.opensearch.common.xcontent.XContentFactory;
import org.opensearch.common.xcontent.XContentHelper;
import org.opensearch.core.common.bytes.BytesReference;
+import org.opensearch.core.common.io.stream.NamedWriteableAwareStreamInput;
+import org.opensearch.core.common.io.stream.NamedWriteableRegistry;
import org.opensearch.core.common.io.stream.StreamInput;
+import org.opensearch.core.common.io.stream.StreamOutput;
import org.opensearch.core.index.Index;
import org.opensearch.core.index.shard.ShardId;
import org.opensearch.core.indices.breaker.AllCircuitBreakerStats;
@@ -97,6 +100,8 @@
import org.opensearch.node.NodesResourceUsageStats;
import org.opensearch.node.ResponseCollectorService;
import org.opensearch.node.remotestore.RemoteStoreNodeStats;
+import org.opensearch.plugins.Plugin;
+import org.opensearch.plugins.PluginNodeStats;
import org.opensearch.ratelimitting.admissioncontrol.controllers.AdmissionController;
import org.opensearch.ratelimitting.admissioncontrol.controllers.CpuBasedAdmissionController;
import org.opensearch.ratelimitting.admissioncontrol.enums.AdmissionControlActionType;
@@ -1052,6 +1057,7 @@ public long getLastSuccessfulFetchOfPinnedTimestamps() {
admissionControlStats,
nodeCacheStats,
remoteStoreNodeStats,
+ null,
null
);
}
@@ -1510,4 +1516,181 @@ public MockNodeIndicesStats generateMockNodeIndicesStats(
);
}
}
+
+ /**
+ * Older nodes (pre-V_3_7_0) don't write the plugin nodeStats map, and this version
+ * gate must keep them round-tripping cleanly: the receiver should see an empty map
+ * rather than a deserialization failure.
+ */
+ public void testPluginStatsBwcEmptyOnOldVersion() throws IOException {
+ Map stats = new HashMap<>();
+ stats.put("native_allocator", new TestPluginStats("native_allocator", 42L));
+ DiscoveryNode node = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
+ NodeStats original = newNodeStatsWithPluginStats(node, stats);
+
+ try (BytesStreamOutput out = new BytesStreamOutput()) {
+ out.setVersion(Version.V_3_6_0);
+ original.writeTo(out);
+ try (StreamInput in = out.bytes().streamInput()) {
+ in.setVersion(Version.V_3_6_0);
+ NodeStats roundtripped = new NodeStats(in);
+ assertTrue("plugin stats must be empty when written by an older node", roundtripped.getPluginStats().isEmpty());
+ }
+ }
+ }
+
+ /**
+ * A receiver missing the {@link PluginNodeStats} subtype's NamedWriteable registration
+ * must drop the unknown entry rather than failing the entire NodeStats deserialization.
+ * This is the rolling-upgrade path: data nodes send stats from a plugin the
+ * coordinator doesn't have loaded.
+ */
+ public void testPluginStatsSkipsUnknownNamedWriteableOnReceiver() throws IOException {
+ Map stats = new HashMap<>();
+ stats.put("test_plugin", new TestPluginStats("test_plugin", 1234L));
+ DiscoveryNode node = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
+ NodeStats original = newNodeStatsWithPluginStats(node, stats);
+
+ // Sender side knows about TestPluginStats.
+ NamedWriteableRegistry senderRegistry = new NamedWriteableRegistry(
+ List.of(new NamedWriteableRegistry.Entry(PluginNodeStats.class, "test_plugin", TestPluginStats::new))
+ );
+ // Receiver side does not.
+ NamedWriteableRegistry receiverRegistry = new NamedWriteableRegistry(List.of());
+
+ try (BytesStreamOutput out = new BytesStreamOutput()) {
+ out.setVersion(Version.CURRENT);
+ original.writeTo(out);
+ try (
+ StreamInput rawIn = out.bytes().streamInput();
+ StreamInput in = new NamedWriteableAwareStreamInput(rawIn, receiverRegistry)
+ ) {
+ in.setVersion(Version.CURRENT);
+ // Must not throw — the unknown entry is dropped, the rest of NodeStats decodes.
+ NodeStats roundtripped = new NodeStats(in);
+ assertTrue(
+ "unknown plugin stats entry must be dropped on the receiver, not propagated",
+ roundtripped.getPluginStats().isEmpty()
+ );
+ }
+ // sanity: the same payload, when decoded with the sender's registry, contains the entry.
+ try (
+ StreamInput rawIn = out.bytes().streamInput();
+ StreamInput in = new NamedWriteableAwareStreamInput(rawIn, senderRegistry)
+ ) {
+ in.setVersion(Version.CURRENT);
+ NodeStats roundtripped = new NodeStats(in);
+ assertEquals(1, roundtripped.getPluginStats().size());
+ assertTrue(roundtripped.getPluginStats().containsKey("test_plugin"));
+ }
+ }
+ }
+
+ private static NodeStats newNodeStatsWithPluginStats(DiscoveryNode node, Map pluginStats) {
+ return new NodeStats(
+ node,
+ 0L,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ pluginStats,
+ null
+ );
+ }
+
+ /**
+ * When the parent stream has no NamedWriteableRegistry attached (a defensive path —
+ * production callers always attach one), the deserializer must drop all plugin-stats
+ * entries cleanly rather than fail. This exercises the {@code registry == null} branch
+ * in {@link NodeStats#readPluginStats(StreamInput)}.
+ */
+ public void testPluginStatsDropsAllEntriesWhenReceiverHasNoRegistry() throws IOException {
+ Map stats = new HashMap<>();
+ stats.put("test_plugin_a", new TestPluginStats("test_plugin_a", 1L));
+ stats.put("test_plugin_b", new TestPluginStats("test_plugin_b", 2L));
+ DiscoveryNode node = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
+ NodeStats original = newNodeStatsWithPluginStats(node, stats);
+
+ try (BytesStreamOutput out = new BytesStreamOutput()) {
+ out.setVersion(Version.CURRENT);
+ original.writeTo(out);
+ // No NamedWriteableAwareStreamInput wrapper — the raw StreamInput has no registry.
+ try (StreamInput rawIn = out.bytes().streamInput()) {
+ rawIn.setVersion(Version.CURRENT);
+ NodeStats roundtripped = new NodeStats(rawIn);
+ assertTrue(
+ "all plugin-stats entries must be dropped when receiver has no registry attached",
+ roundtripped.getPluginStats().isEmpty()
+ );
+ }
+ }
+ }
+
+ /**
+ * The {@link Plugin#nodeStats()} default returns an empty list — plugins that don't
+ * override it must not contribute any stats. Anonymous subclass exercises the default
+ * impl directly.
+ */
+ public void testPluginNodeStatsDefaultReturnsEmpty() {
+ Plugin plugin = new Plugin() {
+ };
+ assertTrue("Plugin#nodeStats() default must return empty list", plugin.nodeStats().isEmpty());
+ }
+
+ /** Minimal PluginNodeStats implementation for the tests above. */
+ static final class TestPluginStats implements PluginNodeStats {
+ private final String name;
+ private final long value;
+
+ TestPluginStats(String name, long value) {
+ this.name = name;
+ this.value = value;
+ }
+
+ TestPluginStats(StreamInput in) throws IOException {
+ this.name = in.readString();
+ this.value = in.readLong();
+ }
+
+ @Override
+ public String getWriteableName() {
+ return name;
+ }
+
+ @Override
+ public void writeTo(StreamOutput out) throws IOException {
+ out.writeString(name);
+ out.writeLong(value);
+ }
+
+ @Override
+ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
+ return builder.field("value", value);
+ }
+ }
}
diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java
index fd0dc2f29fb94..7d4e8d4d04386 100644
--- a/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java
+++ b/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java
@@ -351,6 +351,7 @@ private ClusterStatsNodeResponse createClusterStatsNodeResponse(
null,
null,
null,
+ null,
null
);
if (defaultBehavior) {
diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsResponseTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsResponseTests.java
index c150245306df6..6e04a6f984bb4 100644
--- a/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsResponseTests.java
+++ b/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsResponseTests.java
@@ -223,6 +223,7 @@ private ClusterStatsNodeResponse createClusterStatsNodeResponse(DiscoveryNode no
null,
null,
null,
+ null,
null
);
return new ClusterStatsNodeResponse(node, null, nodeInfo, nodeStats, shardStats);
diff --git a/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java b/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java
index 0373250a88ed2..c27c486585087 100644
--- a/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java
+++ b/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java
@@ -213,6 +213,7 @@ public void testFillDiskUsage() {
null,
null,
null,
+ null,
null
),
new NodeStats(
@@ -246,6 +247,7 @@ public void testFillDiskUsage() {
null,
null,
null,
+ null,
null
),
new NodeStats(
@@ -279,6 +281,7 @@ public void testFillDiskUsage() {
null,
null,
null,
+ null,
null
)
);
@@ -343,6 +346,7 @@ public void testFillDiskUsageSomeInvalidValues() {
null,
null,
null,
+ null,
null
),
new NodeStats(
@@ -376,6 +380,7 @@ public void testFillDiskUsageSomeInvalidValues() {
null,
null,
null,
+ null,
null
),
new NodeStats(
@@ -409,6 +414,7 @@ public void testFillDiskUsageSomeInvalidValues() {
null,
null,
null,
+ null,
null
)
);
@@ -501,6 +507,7 @@ private NodeStats makeNodeStatsWithResourceUsage(DiscoveryNode node, NodesResour
null,
null,
null,
+ null,
null
);
}
diff --git a/server/src/test/java/org/opensearch/node/NodeServiceNativeMemoryTests.java b/server/src/test/java/org/opensearch/node/NodeServiceNativeMemoryTests.java
index 98371278965ee..8548787b453e6 100644
--- a/server/src/test/java/org/opensearch/node/NodeServiceNativeMemoryTests.java
+++ b/server/src/test/java/org/opensearch/node/NodeServiceNativeMemoryTests.java
@@ -140,6 +140,7 @@ public void testStatsWithNativeMemoryTrueAndServicePresent() {
false, // admissionControl
false, // cacheService
false, // remoteStoreNodeStats
+ false, // pluginStats
true // nativeMemory
);
@@ -185,6 +186,7 @@ public void testStatsWithNativeMemoryTrueAndNullService() {
false, // admissionControl
false, // cacheService
false, // remoteStoreNodeStats
+ false, // pluginStats
true // nativeMemory
);
@@ -229,6 +231,7 @@ public void testStatsWithNativeMemoryFalse() {
false, // admissionControl
false, // cacheService
false, // remoteStoreNodeStats
+ false, // pluginStats
false // nativeMemory
);
@@ -275,6 +278,7 @@ public void testNativeMemoryResponseFormatUnchanged() throws Exception {
false,
false,
false,
+ false, // pluginStats
true // nativeMemory
);
@@ -340,6 +344,7 @@ public void testNativeMemoryOmittedWhenUnavailable() throws Exception {
false,
false,
false,
+ false, // pluginStats
true // nativeMemory
);
diff --git a/server/src/test/java/org/opensearch/node/resource/tracker/NodeResourceUsageTrackerTests.java b/server/src/test/java/org/opensearch/node/resource/tracker/NodeResourceUsageTrackerTests.java
index 4b194c3aab482..5ddc8ebf0ff47 100644
--- a/server/src/test/java/org/opensearch/node/resource/tracker/NodeResourceUsageTrackerTests.java
+++ b/server/src/test/java/org/opensearch/node/resource/tracker/NodeResourceUsageTrackerTests.java
@@ -119,14 +119,17 @@ public void testUpdateSettings() {
}
public void testNativeMemoryLimitAndBufferDynamicUpdate() throws Exception {
+ // Default for node.native_memory.limit is now ram - heap (machine-dependent), so the
+ // "limit=0 bytes" baseline this test wants to start from must be set explicitly.
+ Settings unconfiguredAc = Settings.builder().put(ResourceTrackerSettings.NODE_NATIVE_MEMORY_LIMIT_SETTING.getKey(), "0b").build();
NodeResourceUsageTracker tracker = new NodeResourceUsageTracker(
mock(FsService.class),
threadPool,
- Settings.EMPTY,
- new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
+ unconfiguredAc,
+ new ClusterSettings(unconfiguredAc, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
);
try {
- // Defaults: limit=0 bytes, buffer=0 percent.
+ // Explicitly unconfigured: limit=0 bytes, buffer=0 percent.
assertThat(tracker.getResourceTrackerSettings().getNativeMemoryLimitBytes(), equalTo(0L));
assertThat(tracker.getResourceTrackerSettings().getNativeMemoryBufferPercent(), equalTo(0));
@@ -157,4 +160,30 @@ public void testNativeMemoryLimitAndBufferDynamicUpdate() throws Exception {
tracker.close();
}
}
+
+ /**
+ * Exercises {@link ResourceTrackerSettings#deriveNativeMemoryLimitDefault(long, long)}
+ * fallback paths so the {@code "0b"} branch is covered without depending on the
+ * test machine's RAM/heap configuration. The single-arg overload that reads from
+ * {@link org.opensearch.monitor.os.OsProbe} delegates to this one.
+ */
+ public void testDeriveNativeMemoryLimitDefaultFallbackPaths() {
+ // Probe returned 0 (restricted container, /proc/meminfo unreadable).
+ assertEquals("unknown ram → unconfigured", "0b", ResourceTrackerSettings.deriveNativeMemoryLimitDefault(0, 1L << 30));
+ // Negative ram (defensive against probe quirks).
+ assertEquals("negative ram → unconfigured", "0b", ResourceTrackerSettings.deriveNativeMemoryLimitDefault(-1, 1L << 30));
+ // Heap >= ram (degenerate config, e.g. -Xmx larger than container limit).
+ long ram = 8L * 1024 * 1024 * 1024;
+ assertEquals("heap >= ram → unconfigured", "0b", ResourceTrackerSettings.deriveNativeMemoryLimitDefault(ram, ram));
+ assertEquals("heap > ram → unconfigured", "0b", ResourceTrackerSettings.deriveNativeMemoryLimitDefault(ram, ram + 1));
+ // Happy path: 64 GB / 16 GB heap → 79% of (RAM - heap) = 79% of 48 GB.
+ long sixtyFourGB = 64L * 1024 * 1024 * 1024;
+ long sixteenGB = 16L * 1024 * 1024 * 1024;
+ long expected = (sixtyFourGB - sixteenGB) * 79 / 100;
+ assertEquals(
+ "happy path → 79% of (ram - heap)",
+ expected + "b",
+ ResourceTrackerSettings.deriveNativeMemoryLimitDefault(sixtyFourGB, sixteenGB)
+ );
+ }
}
diff --git a/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java b/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java
index d5b1e865e6bd6..d274deac803c9 100644
--- a/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java
+++ b/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java
@@ -151,6 +151,7 @@ List adjustNodesStats(List nodesStats) {
nodeStats.getAdmissionControlStats(),
nodeStats.getNodeCacheStats(),
nodeStats.getRemoteStoreNodeStats(),
+ nodeStats.getPluginStats(),
nodeStats.getAnalyticsBackendNativeMemoryStats()
);
}).collect(Collectors.toList());
diff --git a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java
index 32c26ee8610db..9a20531884b6a 100644
--- a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java
+++ b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java
@@ -2701,6 +2701,7 @@ public void ensureEstimatedStats() {
false,
false,
false,
+ false,
false
);
assertThat(