Browse Source

ESQL: Update status every second or so (#99556)

This changes `Driver` to report status after at least a second has
passed. Between each iteration we check if a second has passed since we
last reported status. If more than a second has passed we'll report
status.

That second is a pragma that can be overridden. And we override it in
the EsqlActionTaskIT so we can test the status again.
Nik Everett 2 years ago
parent
commit
f0a6596c17

+ 6 - 3
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java

@@ -152,11 +152,14 @@ public abstract class LuceneOperator extends SourceOperator {
     public String toString() {
         StringBuilder sb = new StringBuilder();
         sb.append(this.getClass().getSimpleName()).append("[");
-        sb.append(", maxPageSize=").append(maxPageSize);
+        sb.append("maxPageSize=").append(maxPageSize);
+        describe(sb);
         sb.append("]");
         return sb.toString();
     }
 
+    protected abstract void describe(StringBuilder sb);
+
     @Override
     public Operator.Status status() {
         return new Status(this);
@@ -234,11 +237,11 @@ public abstract class LuceneOperator extends SourceOperator {
             return pagesEmitted;
         }
 
-        public int leafPosition() {
+        public int slicePosition() {
             return slicePosition;
         }
 
-        public int leafSize() {
+        public int sliceSize() {
             return sliceSize;
         }
 

+ 5 - 0
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java

@@ -160,4 +160,9 @@ public class LuceneSourceOperator extends LuceneOperator {
             throw new UncheckedIOException(e);
         }
     }
+
+    @Override
+    protected void describe(StringBuilder sb) {
+        sb.append(", remainingDocs=").append(remainingDocs);
+    }
 }

+ 6 - 0
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java

@@ -212,6 +212,12 @@ public final class LuceneTopNSourceOperator extends LuceneOperator {
         );
     }
 
+    @Override
+    protected void describe(StringBuilder sb) {
+        sb.append(", limit=").append(limit);
+        sb.append(", sorts=").append(sorts);
+    }
+
     static final class PerShardCollector {
         private final int shardIndex;
         private final SearchContext searchContext;

+ 25 - 5
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java

@@ -40,15 +40,25 @@ import java.util.stream.Collectors;
 public class Driver implements Releasable, Describable {
     public static final TimeValue DEFAULT_TIME_BEFORE_YIELDING = TimeValue.timeValueMinutes(5);
     public static final int DEFAULT_MAX_ITERATIONS = 10_000;
+    /**
+     * Minimum time between updating status.
+     */
+    public static final TimeValue DEFAULT_STATUS_INTERVAL = TimeValue.timeValueSeconds(1);
 
     private final String sessionId;
     private final DriverContext driverContext;
     private final Supplier<String> description;
     private final List<Operator> activeOperators;
     private final Releasable releasable;
+    private final long statusNanos;
 
     private final AtomicReference<String> cancelReason = new AtomicReference<>();
     private final AtomicReference<ListenableActionFuture<Void>> blocked = new AtomicReference<>();
+    /**
+     * Status reported to the tasks API. We write the status at most once every
+     * {@link #statusNanos}, as soon as loop has finished and after {@link #statusNanos}
+     * have passed.
+     */
     private final AtomicReference<DriverStatus> status;
 
     /**
@@ -58,6 +68,7 @@ public class Driver implements Releasable, Describable {
      * @param source source operator
      * @param intermediateOperators  the chain of operators to execute
      * @param sink sink operator
+     * @param statusInterval minimum status reporting interval
      * @param releasable a {@link Releasable} to invoked once the chain of operators has run to completion
      */
     public Driver(
@@ -67,6 +78,7 @@ public class Driver implements Releasable, Describable {
         SourceOperator source,
         List<Operator> intermediateOperators,
         SinkOperator sink,
+        TimeValue statusInterval,
         Releasable releasable
     ) {
         this.sessionId = sessionId;
@@ -76,6 +88,7 @@ public class Driver implements Releasable, Describable {
         this.activeOperators.add(source);
         this.activeOperators.addAll(intermediateOperators);
         this.activeOperators.add(sink);
+        this.statusNanos = statusInterval.nanos();
         this.releasable = releasable;
         this.status = new AtomicReference<>(new DriverStatus(sessionId, System.currentTimeMillis(), DriverStatus.Status.QUEUED, List.of()));
     }
@@ -95,7 +108,7 @@ public class Driver implements Releasable, Describable {
         SinkOperator sink,
         Releasable releasable
     ) {
-        this("unset", driverContext, () -> null, source, intermediateOperators, sink, releasable);
+        this("unset", driverContext, () -> null, source, intermediateOperators, sink, DEFAULT_STATUS_INTERVAL, releasable);
     }
 
     public DriverContext driverContext() {
@@ -110,26 +123,33 @@ public class Driver implements Releasable, Describable {
     private ListenableActionFuture<Void> run(TimeValue maxTime, int maxIterations) {
         long maxTimeNanos = maxTime.nanos();
         long startTime = System.nanoTime();
+        long nextStatus = startTime + statusNanos;
         int iter = 0;
         while (isFinished() == false) {
             ListenableActionFuture<Void> fut = runSingleLoopIteration();
             if (fut.isDone() == false) {
+                status.set(updateStatus(DriverStatus.Status.ASYNC));
                 return fut;
             }
-            if (++iter >= maxIterations) {
+            if (iter >= maxIterations) {
                 break;
             }
             long now = System.nanoTime();
+            if (now > nextStatus) {
+                status.set(updateStatus(DriverStatus.Status.RUNNING));
+                nextStatus = now + statusNanos;
+            }
+            iter++;
             if (now - startTime > maxTimeNanos) {
                 break;
             }
         }
         if (isFinished()) {
-            status.set(updateStatus(DriverStatus.Status.DONE));  // Report status for the tasks API
+            status.set(updateStatus(DriverStatus.Status.DONE));
             driverContext.finish();
             releasable.close();
         } else {
-            status.set(updateStatus(DriverStatus.Status.RUNNING));  // Report status for the tasks API
+            status.set(updateStatus(DriverStatus.Status.WAITING));
         }
         return Operator.NOT_BLOCKED;
     }
@@ -227,7 +247,7 @@ public class Driver implements Releasable, Describable {
     }
 
     public static void start(Executor executor, Driver driver, int maxIterations, ActionListener<Void> listener) {
-        driver.status.set(driver.updateStatus(DriverStatus.Status.STARTING));  // Report status for the tasks API
+        driver.status.set(driver.updateStatus(DriverStatus.Status.STARTING));
         schedule(DEFAULT_TIME_BEFORE_YIELDING, maxIterations, executor, driver, listener);
     }
 

+ 2 - 0
x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/DriverStatus.java

@@ -210,6 +210,8 @@ public class DriverStatus implements Task.Status {
         QUEUED,
         STARTING,
         RUNNING,
+        ASYNC,
+        WAITING,
         DONE;
 
         @Override

+ 10 - 10
x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorStatusTests.java

@@ -51,36 +51,36 @@ public class LuceneSourceOperatorStatusTests extends AbstractWireSerializingTest
                 randomValueOtherThan(instance.currentLeaf(), ESTestCase::randomNonNegativeInt),
                 instance.totalLeaves(),
                 instance.pagesEmitted(),
-                instance.leafPosition(),
-                instance.leafSize()
+                instance.slicePosition(),
+                instance.sliceSize()
             );
             case 1 -> new LuceneSourceOperator.Status(
                 instance.currentLeaf(),
                 randomValueOtherThan(instance.totalLeaves(), ESTestCase::randomNonNegativeInt),
                 instance.pagesEmitted(),
-                instance.leafPosition(),
-                instance.leafSize()
+                instance.slicePosition(),
+                instance.sliceSize()
             );
             case 2 -> new LuceneSourceOperator.Status(
                 instance.currentLeaf(),
                 instance.totalLeaves(),
                 randomValueOtherThan(instance.pagesEmitted(), ESTestCase::randomNonNegativeInt),
-                instance.leafPosition(),
-                instance.leafSize()
+                instance.slicePosition(),
+                instance.sliceSize()
             );
             case 3 -> new LuceneSourceOperator.Status(
                 instance.currentLeaf(),
                 instance.totalLeaves(),
                 instance.pagesEmitted(),
-                randomValueOtherThan(instance.leafPosition(), ESTestCase::randomNonNegativeInt),
-                instance.leafSize()
+                randomValueOtherThan(instance.slicePosition(), ESTestCase::randomNonNegativeInt),
+                instance.sliceSize()
             );
             case 4 -> new LuceneSourceOperator.Status(
                 instance.currentLeaf(),
                 instance.totalLeaves(),
                 instance.pagesEmitted(),
-                instance.leafPosition(),
-                randomValueOtherThan(instance.leafSize(), ESTestCase::randomNonNegativeInt)
+                instance.slicePosition(),
+                randomValueOtherThan(instance.sliceSize(), ESTestCase::randomNonNegativeInt)
             );
             default -> throw new UnsupportedOperationException();
         };

+ 1 - 0
x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/OperatorTestCase.java

@@ -171,6 +171,7 @@ public abstract class OperatorTestCase extends AnyOperatorTestCase {
                     new SequenceLongBlockSourceOperator(LongStream.range(0, between(1, 100)), between(1, 100)),
                     List.of(),
                     new PageConsumerOperator(page -> {}),
+                    Driver.DEFAULT_STATUS_INTERVAL,
                     () -> {}
                 )
             );

+ 20 - 2
x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeServiceTests.java

@@ -268,14 +268,32 @@ public class ExchangeServiceTests extends ESTestCase {
             String description = "sink-" + i;
             ExchangeSinkOperator sinkOperator = new ExchangeSinkOperator(exchangeSink.get(), Function.identity());
             DriverContext dc = new DriverContext();
-            Driver d = new Driver("test-session:1", dc, () -> description, seqNoGenerator.get(dc), List.of(), sinkOperator, () -> {});
+            Driver d = new Driver(
+                "test-session:1",
+                dc,
+                () -> description,
+                seqNoGenerator.get(dc),
+                List.of(),
+                sinkOperator,
+                Driver.DEFAULT_STATUS_INTERVAL,
+                () -> {}
+            );
             drivers.add(d);
         }
         for (int i = 0; i < numSources; i++) {
             String description = "source-" + i;
             ExchangeSourceOperator sourceOperator = new ExchangeSourceOperator(exchangeSource.get());
             DriverContext dc = new DriverContext();
-            Driver d = new Driver("test-session:2", dc, () -> description, sourceOperator, List.of(), seqNoCollector.get(dc), () -> {});
+            Driver d = new Driver(
+                "test-session:2",
+                dc,
+                () -> description,
+                sourceOperator,
+                List.of(),
+                seqNoCollector.get(dc),
+                Driver.DEFAULT_STATUS_INTERVAL,
+                () -> {}
+            );
             drivers.add(d);
         }
         PlainActionFuture<Void> future = new PlainActionFuture<>();

+ 73 - 49
x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java

@@ -13,6 +13,7 @@ import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksActio
 import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
 import org.elasticsearch.action.bulk.BulkRequestBuilder;
 import org.elasticsearch.action.support.WriteRequest;
+import org.elasticsearch.common.collect.Iterators;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.CollectionUtils;
 import org.elasticsearch.compute.lucene.LuceneSourceOperator;
@@ -112,54 +113,59 @@ public class EsqlActionTaskIT extends AbstractEsqlIntegTestCase {
         bulk.get();
     }
 
-    @AwaitsFix(bugUrl = "the task status is only updated after max_iterations")
     public void testTaskContents() throws Exception {
         ActionFuture<EsqlQueryResponse> response = startEsql();
-        getTasksStarting();
-        List<TaskInfo> foundTasks = getTasksRunning();
-        int luceneSources = 0;
-        int valuesSourceReaders = 0;
-        int exchangeSources = 0;
-        int exchangeSinks = 0;
-        for (TaskInfo task : foundTasks) {
-            DriverStatus status = (DriverStatus) task.status();
-            assertThat(status.sessionId(), not(emptyOrNullString()));
-            for (DriverStatus.OperatorStatus o : status.activeOperators()) {
-                if (o.operator().equals("LuceneSourceOperator[shardId=0, maxPageSize=" + PAGE_SIZE + "]")) {
-                    LuceneSourceOperator.Status oStatus = (LuceneSourceOperator.Status) o.status();
-                    assertThat(oStatus.currentLeaf(), lessThanOrEqualTo(oStatus.totalLeaves()));
-                    assertThat(oStatus.leafPosition(), lessThanOrEqualTo(oStatus.leafSize()));
-                    luceneSources++;
-                    continue;
-                }
-                if (o.operator().equals("ValuesSourceReaderOperator[field = pause_me]")) {
-                    ValuesSourceReaderOperator.Status oStatus = (ValuesSourceReaderOperator.Status) o.status();
-                    assertThat(oStatus.readersBuilt(), equalTo(Map.of("LongValuesReader", 1)));
-                    assertThat(oStatus.pagesProcessed(), greaterThanOrEqualTo(1));
-                    valuesSourceReaders++;
-                    continue;
-                }
-                if (o.operator().equals("ExchangeSourceOperator")) {
-                    ExchangeSourceOperator.Status oStatus = (ExchangeSourceOperator.Status) o.status();
-                    assertThat(oStatus.pagesWaiting(), greaterThanOrEqualTo(0));
-                    assertThat(oStatus.pagesEmitted(), greaterThanOrEqualTo(0));
-                    exchangeSources++;
-                    continue;
-                }
-                if (o.operator().equals("ExchangeSinkOperator")) {
-                    ExchangeSinkOperator.Status oStatus = (ExchangeSinkOperator.Status) o.status();
-                    assertThat(oStatus.pagesAccepted(), greaterThanOrEqualTo(0));
-                    exchangeSinks++;
+        try {
+            getTasksStarting();
+            scriptPermits.release(PAGE_SIZE);
+            List<TaskInfo> foundTasks = getTasksRunning();
+            int luceneSources = 0;
+            int valuesSourceReaders = 0;
+            int exchangeSources = 0;
+            int exchangeSinks = 0;
+            for (TaskInfo task : foundTasks) {
+                DriverStatus status = (DriverStatus) task.status();
+                assertThat(status.sessionId(), not(emptyOrNullString()));
+                for (DriverStatus.OperatorStatus o : status.activeOperators()) {
+                    if (o.operator().startsWith("LuceneSourceOperator[maxPageSize=" + PAGE_SIZE)) {
+                        LuceneSourceOperator.Status oStatus = (LuceneSourceOperator.Status) o.status();
+                        assertThat(oStatus.currentLeaf(), lessThanOrEqualTo(oStatus.totalLeaves()));
+                        assertThat(oStatus.slicePosition(), greaterThanOrEqualTo(0));
+                        if (oStatus.sliceSize() != 0) {
+                            assertThat(oStatus.slicePosition(), lessThanOrEqualTo(oStatus.sliceSize()));
+                        }
+                        luceneSources++;
+                        continue;
+                    }
+                    if (o.operator().equals("ValuesSourceReaderOperator[field = pause_me]")) {
+                        ValuesSourceReaderOperator.Status oStatus = (ValuesSourceReaderOperator.Status) o.status();
+                        assertThat(oStatus.readersBuilt(), equalTo(Map.of("LongValuesReader", 1)));
+                        assertThat(oStatus.pagesProcessed(), greaterThanOrEqualTo(1));
+                        valuesSourceReaders++;
+                        continue;
+                    }
+                    if (o.operator().equals("ExchangeSourceOperator")) {
+                        ExchangeSourceOperator.Status oStatus = (ExchangeSourceOperator.Status) o.status();
+                        assertThat(oStatus.pagesWaiting(), greaterThanOrEqualTo(0));
+                        assertThat(oStatus.pagesEmitted(), greaterThanOrEqualTo(0));
+                        exchangeSources++;
+                        continue;
+                    }
+                    if (o.operator().equals("ExchangeSinkOperator")) {
+                        ExchangeSinkOperator.Status oStatus = (ExchangeSinkOperator.Status) o.status();
+                        assertThat(oStatus.pagesAccepted(), greaterThanOrEqualTo(0));
+                        exchangeSinks++;
+                    }
                 }
             }
+            assertThat(luceneSources, greaterThanOrEqualTo(1));
+            assertThat(valuesSourceReaders, equalTo(1));
+            assertThat(exchangeSinks, greaterThanOrEqualTo(1));
+            assertThat(exchangeSources, equalTo(1));
+        } finally {
+            scriptPermits.release(Integer.MAX_VALUE);
+            assertThat(Iterators.flatMap(response.get().values(), i -> i).next(), equalTo((long) NUM_DOCS));
         }
-        assertThat(luceneSources, greaterThanOrEqualTo(1));
-        assertThat(valuesSourceReaders, equalTo(1));
-        assertThat(exchangeSinks, greaterThanOrEqualTo(1));
-        assertThat(exchangeSources, equalTo(1));
-
-        scriptPermits.release(Integer.MAX_VALUE);
-        assertThat(response.get().values(), equalTo(List.of(List.of((long) NUM_DOCS))));
     }
 
     public void testCancelRead() throws Exception {
@@ -194,8 +200,17 @@ public class EsqlActionTaskIT extends AbstractEsqlIntegTestCase {
 
     private ActionFuture<EsqlQueryResponse> startEsql() {
         scriptPermits.drainPermits();
-        scriptPermits.release(between(1, 10));
-        var pragmas = new QueryPragmas(Settings.builder().put("data_partitioning", "shard").put("page_size", PAGE_SIZE).build());
+        scriptPermits.release(between(1, 5));
+        var pragmas = new QueryPragmas(
+            Settings.builder()
+                // Force shard partitioning because that's all the tests know how to match. It is easier to reason about too.
+                .put("data_partitioning", "shard")
+                // Limit the page size to something small so we do more than one page worth of work, so we get more status updates.
+                .put("page_size", PAGE_SIZE)
+                // Report the status after every action
+                .put("status_interval", "0ms")
+                .build()
+        );
         return new EsqlQueryRequestBuilder(client(), EsqlQueryAction.INSTANCE).query("from test | stats sum(pause_me)")
             .pragmas(pragmas)
             .execute();
@@ -233,7 +248,13 @@ public class EsqlActionTaskIT extends AbstractEsqlIntegTestCase {
                 assertThat(task.description(), either(equalTo(READ_DESCRIPTION)).or(equalTo(MERGE_DESCRIPTION)));
                 DriverStatus status = (DriverStatus) task.status();
                 logger.info("{}", status.status());
-                assertThat(status.status(), equalTo(DriverStatus.Status.STARTING));
+                /*
+                 * Accept tasks that are either starting or have gone
+                 * immediately async. The coordinating task is likely
+                 * to have done the latter and the reading task should
+                 * have done the former.
+                 */
+                assertThat(status.status(), either(equalTo(DriverStatus.Status.STARTING)).or(equalTo(DriverStatus.Status.ASYNC)));
             }
             foundTasks.addAll(tasks);
         });
@@ -256,10 +277,13 @@ public class EsqlActionTaskIT extends AbstractEsqlIntegTestCase {
             assertThat(tasks, hasSize(equalTo(2)));
             for (TaskInfo task : tasks) {
                 assertThat(task.action(), equalTo(DriverTaskRunner.ACTION_NAME));
-                assertThat(task.description(), either(equalTo(READ_DESCRIPTION)).or(equalTo(MERGE_DESCRIPTION)));
                 DriverStatus status = (DriverStatus) task.status();
-                // TODO: Running is not after one iteration?
-                assertThat(status.status(), equalTo(DriverStatus.Status.STARTING));
+                assertThat(task.description(), either(equalTo(READ_DESCRIPTION)).or(equalTo(MERGE_DESCRIPTION)));
+                if (task.description().equals(READ_DESCRIPTION)) {
+                    assertThat(status.status(), equalTo(DriverStatus.Status.RUNNING));
+                } else {
+                    assertThat(status.status(), equalTo(DriverStatus.Status.ASYNC));
+                }
             }
             foundTasks.addAll(tasks);
         });

+ 1 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java

@@ -205,6 +205,7 @@ public class EnrichLookupService {
                 queryOperator,
                 intermediateOperators,
                 outputOperator,
+                Driver.DEFAULT_STATUS_INTERVAL,
                 searchContext
             );
             task.addListener(() -> {

+ 10 - 4
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java

@@ -41,6 +41,7 @@ import org.elasticsearch.compute.operator.topn.TopNEncoder;
 import org.elasticsearch.compute.operator.topn.TopNOperator;
 import org.elasticsearch.compute.operator.topn.TopNOperator.TopNOperatorFactory;
 import org.elasticsearch.core.Releasables;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.index.query.MatchAllQueryBuilder;
 import org.elasticsearch.tasks.CancellableTask;
 import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException;
@@ -149,7 +150,10 @@ public class LocalExecutionPlanner {
         PhysicalOperation physicalOperation = plan(node, context);
 
         context.addDriverFactory(
-            new DriverFactory(new DriverSupplier(context.bigArrays, physicalOperation), context.driverParallelism().get())
+            new DriverFactory(
+                new DriverSupplier(context.bigArrays, physicalOperation, configuration.pragmas().statusInterval()),
+                context.driverParallelism().get()
+            )
         );
 
         return new LocalExecutionPlan(context.driverFactories);
@@ -679,8 +683,10 @@ public class LocalExecutionPlanner {
         }
     }
 
-    record DriverSupplier(BigArrays bigArrays, PhysicalOperation physicalOperation) implements Function<String, Driver>, Describable {
-
+    record DriverSupplier(BigArrays bigArrays, PhysicalOperation physicalOperation, TimeValue statusInterval)
+        implements
+            Function<String, Driver>,
+            Describable {
         @Override
         public Driver apply(String sessionId) {
             SourceOperator source = null;
@@ -693,7 +699,7 @@ public class LocalExecutionPlanner {
                 physicalOperation.operators(operators, driverContext);
                 sink = physicalOperation.sink(driverContext);
                 success = true;
-                return new Driver(sessionId, driverContext, physicalOperation::describe, source, operators, sink, () -> {});
+                return new Driver(sessionId, driverContext, physicalOperation::describe, source, operators, sink, statusInterval, () -> {});
             } finally {
                 if (false == success) {
                     Releasables.close(source, () -> Releasables.close(operators), sink);

+ 17 - 0
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java

@@ -14,6 +14,9 @@ import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.concurrent.EsExecutors;
 import org.elasticsearch.compute.lucene.DataPartitioning;
+import org.elasticsearch.compute.operator.Driver;
+import org.elasticsearch.compute.operator.DriverStatus;
+import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.threadpool.ThreadPool;
 
 import java.io.IOException;
@@ -42,6 +45,12 @@ public final class QueryPragmas implements Writeable {
      */
     public static final Setting<Integer> PAGE_SIZE = Setting.intSetting("page_size", 0, 0);
 
+    /**
+     * The minimum interval between syncs of the {@link DriverStatus}, making
+     * the status available to task API.
+     */
+    public static final Setting<TimeValue> STATUS_INTERVAL = Setting.timeSetting("status_interval", Driver.DEFAULT_STATUS_INTERVAL);
+
     public static final QueryPragmas EMPTY = new QueryPragmas(Settings.EMPTY);
 
     private final Settings settings;
@@ -87,6 +96,14 @@ public final class QueryPragmas implements Writeable {
         return PAGE_SIZE.get(settings);
     }
 
+    /**
+     * The minimum interval between syncs of the {@link DriverStatus}, making
+     * the status available to task API.
+     */
+    public TimeValue statusInterval() {
+        return STATUS_INTERVAL.get(settings);
+    }
+
     public boolean isEmpty() {
         return settings.isEmpty();
     }