Commit Graph

13603 Commits

Author SHA1 Message Date
Pranav 45b30dc07d
Revert "Change default inSubQueryThreshold (#15336)" (#15722)
A low value of inSubQueryThreshold can cause queries with IN filter to plan as joins more commonly. However, some of these join queries may not get planned as IN filter on data nodes and causes significant perf regression.
2024-01-22 11:34:39 +05:30
zachjsh 9d4e8053a4
Kinesis adaptive memory management (#15360)
### Description

Our Kinesis consumer works by using the [GetRecords API](https://docs.aws.amazon.com/kinesis/latest/APIReference/API_GetRecords.html) in some number of `fetchThreads`, each fetching some number of records (`recordsPerFetch`) and each inserting into a shared buffer that can hold a `recordBufferSize` number of records. The logic is described in our documentation at: https://druid.apache.org/docs/27.0.0/development/extensions-core/kinesis-ingestion/#determine-fetch-settings 

There is a problem with the logic that this pr fixes: the memory limits rely on a hard-coded “estimated record size” that is `10 KB` if `deaggregate: false` and `1 MB` if `deaggregate: true`. There have been cases where a supervisor had `deaggregate: true` set even though it wasn’t needed, leading to under-utilization of memory and poor ingestion performance.

Users don’t always know if their records are aggregated or not. Also, even if they could figure it out, it’s better to not have to. So we’d like to eliminate the `deaggregate` parameter, which means we need to do memory management more adaptively based on the actual record sizes.

We take advantage of the fact that GetRecords doesn’t return more than 10MB (https://docs.aws.amazon.com/streams/latest/dev/service-sizes-and-limits.html ):

This pr: 

eliminates `recordsPerFetch`, always use the max limit of 10000 records (the default limit if not set)

eliminate `deaggregate`, always have it true

cap `fetchThreads` to ensure that if each fetch returns the max (`10MB`) then we don't exceed our budget (`100MB` or `5% of heap`). In practice this means `fetchThreads` will never be more than `10`. Tasks usually don't have that many processors available to them anyway, so in practice I don't think this will change the number of threads for too many deployments

add `recordBufferSizeBytes` as a bytes-based limit rather than records-based limit for the shared queue. We do know the byte size of kinesis records by at this point. Default should be `100MB` or `10% of heap`, whichever is smaller.

add `maxBytesPerPoll` as a bytes-based limit for how much data we poll from shared buffer at a time. Default is `1000000` bytes.

deprecate `recordBufferSize`, use `recordBufferSizeBytes` instead. Warning is logged if `recordBufferSize` is specified

deprecate `maxRecordsPerPoll`, use `maxBytesPerPoll` instead. Warning is logged if maxRecordsPerPoll` is specified

Fixed issue that when the record buffer is full, the fetchRecords logic throws away the rest of the GetRecords result after `recordBufferOfferTimeout` and starts a new shard iterator. This seems excessively churny. Instead,  wait an unbounded amount of time for queue to stop being full. If the queue remains full, we’ll end up right back waiting for it after the restarted fetch.

There was also a call to `newQ::offer` without check in `filterBufferAndResetBackgroundFetch`, which seemed like it could cause data loss. Now checking return value here, and failing if false.

### Release Note

Kinesis ingestion memory tuning config has been greatly simplified, and a more adaptive approach is now taken for the configuration. Here is a summary of the changes made:

eliminates `recordsPerFetch`, always use the max limit of 10000 records (the default limit if not set)

eliminate `deaggregate`, always have it true

cap `fetchThreads` to ensure that if each fetch returns the max (`10MB`) then we don't exceed our budget (`100MB` or `5% of heap`). In practice this means `fetchThreads` will never be more than `10`. Tasks usually don't have that many processors available to them anyway, so in practice I don't think this will change the number of threads for too many deployments

add `recordBufferSizeBytes` as a bytes-based limit rather than records-based limit for the shared queue. We do know the byte size of kinesis records by at this point. Default should be `100MB` or `10% of heap`, whichever is smaller.

add `maxBytesPerPoll` as a bytes-based limit for how much data we poll from shared buffer at a time. Default is `1000000` bytes.

deprecate `recordBufferSize`, use `recordBufferSizeBytes` instead. Warning is logged if `recordBufferSize` is specified

deprecate `maxRecordsPerPoll`, use `maxBytesPerPoll` instead. Warning is logged if maxRecordsPerPoll` is specified
2024-01-19 14:30:21 -05:00
Abhishek Radhakrishnan 38c1def95a
Kill tasks honor the buffer period of unused segments (#15710)
* Kill tasks should honor the buffer period of unused segments.

- The coordinator duty KillUnusedSegments determines an umbrella interval
 for each datasource to determine the kill interval. There can be multiple unused
segments in an umbrella interval with different used_status_last_updated timestamps.
For example, consider an unused segment that is 30 days old and one that is 1 hour old. Currently
the kill task after the 30-day mark would kill both the unused segments and not retain the 1-hour
old one.

- However, when a kill task is instantiated with this umbrella interval, it’d kill
all the unused segments regardless of the last updated timestamp. We need kill
tasks and RetrieveUnusedSegmentsAction to honor the bufferPeriod to avoid killing
unused segments in the kill interval prematurely.

* Clarify default behavior in docs.

* test comments

* fix canDutyRun()

* small updates.

* checkstyle

* forbidden api fix

* doc fix, unused import, codeql scan error, and cleanup logs.

* Address review comments

* Rename maxUsedFlagLastUpdatedTime to maxUsedStatusLastUpdatedTime

This is consistent with the column name `used_status_last_updated`.

* Apply suggestions from code review

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>

* Make period Duration type

* Remove older variants of runKilLTask() in OverlordClient interface

* Test can now run without waiting for canDutyRun().

* Remove previous variants of retrieveUnusedSegments from internal metadata storage coordinator interface.

Removes the following interface methods in favor of a new method added:
- retrieveUnusedSegmentsForInterval(String, Interval)
- retrieveUnusedSegmentsForInterval(String, Interval, Integer)

* Chain stream operations

* cleanup

* Pass in the lastUpdatedTime to markUnused test function and remove sleep.

---------

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
2024-01-18 22:23:50 -08:00
Benedict Jin 96b4abc8e9
Add @VisibleForTesting annotation for the backingArray() method (#15690) 2024-01-18 19:30:10 -08:00
Abhishek Radhakrishnan f51f0e07e2
Remove documentation for unused segments retrieval API (#15721)
* Undocument unused segments retrieval API.

* Mark API deprecated and unstable. Note that it'll be removed.

* Cleanup .spelling entries

* Remove the Unstable annotation
2024-01-18 18:25:48 -08:00
Tom 18d42cae3f
Kafka emitter wasn't given the correct number of threads. It should be 1 thread per scheduled task. (#15719)
This change intelligently provisions the correct number of threads per scheduled task. 1 for each event type, and 1 for logging the lost events.
This is a change to make this work. But in the future it would be worthwhile to make each task not be greedy and share threads so there isn't a need of a thread per task.
2024-01-18 15:27:40 -06:00
Vadim Ogievetsky c8c04f1032
fix worker disable rendering (#15712) 2024-01-18 13:21:32 -08:00
George Shiqi Wu dd03f7061a
Implement task payload management with azure storage (#15695)
* Implement task payload manageent with azure storage

* Remove unnecessary parameter
2024-01-18 13:21:36 -05:00
Gian Merlino 792e5c58e4
IncrementalIndex#add is no longer thread-safe. (#15697)
* IncrementalIndex#add is no longer thread-safe.

Following #14866, there is no longer a reason for IncrementalIndex#add
to be thread-safe.

It turns out it already was not using its selectors in a thread-safe way,
as exposed by #15615 making `testMultithreadAddFactsUsingExpressionAndJavaScript`
in `IncrementalIndexIngestionTest` flaky. Note that this problem isn't
new: Strings have been stored in the dimension selectors for some time,
but we didn't have a test that checked for that case; we only have
this test that checks for concurrent adds involving numeric selectors.

At any rate, this patch changes OnheapIncrementalIndex to no longer try
to offer a thread-safe "add" method. It also improves performance a bit
by adding a row ID supplier to the selectors it uses to read InputRows,
meaning that it can get the benefit of caching values inside the selectors.

This patch also:

1) Adds synchronization to HyperUniquesAggregator and CardinalityAggregator,
   which the similar datasketches versions already have. This is done to
   help them adhere to the contract of Aggregator: concurrent calls to
   "aggregate" and "get" must be thread-safe.

2) Updates OnHeapIncrementalIndexBenchmark to use JMH and moves it to the
   druid-benchmarks module.

* Spelling.

* Changes from static analysis.

* Fix javadoc.
2024-01-18 03:45:22 -08:00
Gian Merlino 764f41d959
Clear "lineSplittable" for JSON when using KafkaInputFormat. (#15692)
* Clear "lineSplittable" for JSON when using KafkaInputFormat.

JsonInputFormat has a "withLineSplittable" method that can be used to
control whether JSON is read line-by-line, or as a whole. The intent
is that in streaming ingestion, "lineSplittable" is false (although it
can be overridden by "assumeNewlineDelimited"), and in batch ingestion,
lineSplittable is true.

When a "json" format is wrapped by a "kafka" format, this isn't set
properly. This patch updates KafkaInputFormat to set this on an
underlying "json" format.

The tests for KafkaInputFormat were overriding the "lineSplittable"
parameter explicitly, which wasn't really fair, because that made them
unrealistic to what happens in production. Now they omit the parameter
and get the production behavior.

* Add test.

* Fix test coverage.
2024-01-18 03:22:41 -08:00
Gian Merlino d3d0c1c91e
Faster parsing: reduce String usage, list-based input rows. (#15681)
* Faster parsing: reduce String usage, list-based input rows.

Three changes:

1) Reworked FastLineIterator to optionally avoid generating Strings
   entirely, and reduce copying somewhat. Benefits the line-oriented
   JSON, CSV, delimited (TSV), and regex formats.

2) In the delimited (TSV) format, when the delimiter is a single byte,
   split on UTF-8 bytes directly.

3) In CSV and delimited (TSV) formats, use list-based input rows when
   the column list is provided upfront by the user.

* Fix style.

* Fix inspections.

* Restore validation.

* Remove fastutil-extra.

* Exception type.

* Fixes for error messages.

* Fixes for null handling.
2024-01-18 19:18:46 +08:00
Maytas Monsereenusorn 55acf2e2ff
Fix incorrect scale when reading decimal from parquet (#15715)
* Fix incorrect scale when reading decimal from parquet

* add comments

* fix test
2024-01-18 02:10:27 -08:00
Maytas Monsereenusorn a3b32fbd26
Fix comparator and remove deprecated methods from spectatorHistogram extension (#15698)
* Remove deprecated methods from SpectatorHistogram

* Remove deprecated methods from SpectatorHistogram

* Remove deprecated methods from SpectatorHistogram
2024-01-17 21:59:53 -08:00
AmatyaAvadhanula a26defd64b
Clean up stale entries from upgradeSegments table (#15637)
* Clean up stale entries from upgradeSegments table
2024-01-17 20:49:52 +05:30
Laksh Singla fc06f2d075
Fix summary iterator in grouping engine(#15658)
This PR fixes the summary iterator to add aggregators in the correct position. The summary iterator is used when dims are not present, therefore the new change is identical to the old one, but seems more correct while reading.
2024-01-17 20:43:45 +05:30
Abhishek Radhakrishnan c27f5bf52f
Report zero values instead of unknown for empty ingest queries (#15674)
MSQ now allows empty ingest queries by default. For such queries that don't generate any output rows, the query counters in the async status result object/task report don't contain numTotalRows and totalSizeInBytes. These properties when not set/undefined can be confusing to API clients. For example, the web-console treats it as unknown values.

This patch fixes the counters by explicitly reporting them as 0 instead of null for empty ingest queries.
2024-01-17 16:26:10 +05:30
Zoltan Haindrich 8a43db9395
Range support in window expressions (support them as groups) (#15365)
* support groups windowing mode; which is a close relative of ranges (but not in the standard)
* all windows with range expressions will be executed wit it groups
* it will be 100% correct in case for both bounds its true that: isCurrentRow() || isUnBounded()
  * this covers OVER ( ORDER BY COL )
* for other cases it will have some chances of getting correct results...
2024-01-17 00:05:21 -06:00
Laksh Singla 8ba06cf723
account for null values in the stddev post aggregator (#15660) 2024-01-16 19:57:33 +05:30
AmatyaAvadhanula 6b951b94c0
Add new context parameter for using concurrent locks (#15684)
Changes:
- Add new task context flag useConcurrentLocks.
- This can be set for an individual task or at a cluster level using `druid.indexer.task.default.context`.
- When set to true, any appending task would use an APPEND lock and any other
ingestion task would use a REPLACE lock when using time chunk locking.
- If false (default), we fall back on the context flag taskLockType and then useSharedLock.
2024-01-16 12:43:39 +05:30
AmatyaAvadhanula 11dbfb6e3f
Better error message when partition space is exhausted (#15685)
* Better error message when partition space is exhausted
2024-01-16 12:32:40 +05:30
AmatyaAvadhanula 67720b60ae
Skip compaction for intervals without data (#15676)
* Skip compaction for intervals with a single tombstone and no data
2024-01-16 12:31:36 +05:30
Sam Rash 072b16c6df
Fix SQL Innterval.of() error message (#15454)
Better error message for poorly constructed intervals
2024-01-15 22:34:35 -06:00
Gian Merlino d359fb3d68
Cache value selectors in RowBasedColumnSelectorFactory. (#15615)
* Cache value selectors in RowBasedColumnSelectorFactory.

There was already caching for dimension selectors. This patch adds caching
for value (object and number) selectors. It's helpful when the same field is
read multiple times during processing of a single row (for example, by being
an input to both MIN and MAX aggregations).

* Fix typing.

* Fix logic.
2024-01-15 18:03:27 -08:00
Kashif Faraz 18d2a8957f
Refactor: Cleanup test impls of ServiceEmitter (#15683) 2024-01-15 17:37:00 +05:30
Abhishek Radhakrishnan 08c01f1dae
Handle and map errors in delete pending segments API (#15673)
Changes:
- Handle exception in deletePendingSegments API and map to correct HTTP status code
- Clean up exception message using `DruidException`
- Add unit tests
2024-01-15 10:09:01 +05:30
Ben Sykes e49a7bb3cd
Add SpectatorHistogram extension (#15340)
* Add SpectatorHistogram extension

* Clarify documentation
Cleanup comments

* Use ColumnValueSelector directly
so that we support being queried as a Number using longSum or doubleSum aggregators as well as a histogram.
When queried as a Number, we're returning the count of entries in the histogram.

* Apply suggestions from code review

Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>

* Fix references

* Fix spelling

* Update docs/development/extensions-contrib/spectator-histogram.md

Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>

---------

Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
2024-01-14 09:52:30 -08:00
Kashif Faraz f0c552b2f9
Fix basic auth integration test (#15679)
* Add some retries

* Add a delay to allow creds to propagate

* Checkstyle and stuff
2024-01-14 08:59:15 -08:00
Gian Merlino 500681d0cb
Add ImmutableLookupMap for static lookups. (#15675)
* Add ImmutableLookupMap for static lookups.

This patch adds a new ImmutableLookupMap, which comes with an
ImmutableLookupExtractor. It uses a fastutil open hashmap plus two
lists to store its data in such a way that forward and reverse
lookups can both be done quickly. I also observed footprint to be
somewhat smaller than Java HashMap + MapLookupExtractor for a 1 million
row lookup.

The main advantage, though, is that reverse lookups can be done much
more quickly than MapLookupExtractor (which iterates the entire map
for each call to unapplyAll). This speeds up the recently added
ReverseLookupRule (#15626) during SQL planning with very large lookups.

* Use in one more test.

* Fix benchmark.

* Object2ObjectOpenHashMap

* Fixes, and LookupExtractor interface update to have asMap.

* Remove commented-out code.

* Fix style.

* Fix import order.

* Add fastutil.

* Avoid storing Map entries.
2024-01-13 13:14:01 -08:00
Gian Merlino 866fe1cda6
Fix some naming related to AggregatePullUpLookupRule. (#15677)
It was called "split" rather than "pull up" in some places. This patch
standardizes on "pull up".
2024-01-12 15:41:58 -08:00
YongGang 0457c71d03
Fix k8sAndWorker mode in a zookeeper-less environment (#15445)
* Fix k8sAndWorker mode in a zookeeper-less environment

* add unit test

* code reformat

* minor refine

* change to inject Provider

* correct style

* bind HttpRemoteTaskRunnerFactory as provider

* change to bind on TaskRunnerFactory

* fix styling
2024-01-12 09:30:01 -08:00
Gian Merlino cccf13ea82
Reverse, pull up lookups in the SQL planner. (#15626)
* Reverse, pull up lookups in the SQL planner.

Adds two new rules:

1) ReverseLookupRule, which eliminates calls to LOOKUP by doing
   reverse lookups.

2) AggregatePullUpLookupRule, which pulls up calls to LOOKUP above
   GROUP BY, when the lookup is injective.

Adds configs `sqlReverseLookup` and `sqlPullUpLookup` to control whether
these rules fire. Both are enabled by default.

To minimize the chance of performance problems due to many keys mapping to
the same value, ReverseLookupRule refrains from reversing a lookup if there
are more keys than `inSubQueryThreshold`. The rationale for using this setting
is that reversal works by generating an IN, and the `inSubQueryThreshold`
describes the largest IN the user wants the planner to create.

* Add additional line.

* Style.

* Remove commented-out lines.

* Fix tests.

* Add test.

* Fix doc link.

* Fix docs.

* Add one more test.

* Fix tests.

* Logic, test updates.

* - Make FilterDecomposeConcatRule more flexible.

- Make CalciteRulesManager apply reduction rules til fixpoint.

* Additional tests, simplify code.
2024-01-12 00:06:31 -08:00
Zoltan Haindrich e597cc2949
Remove UnaryFunctionOperatorConversion and RoundOperatorConversion (#15566)
* get rid of roun op conv

* cleanup

* use DirectOperatorConversion instead unary

* import order
2024-01-12 10:06:23 +05:30
Gian Merlino 6c18434028
CONCAT flattening, filter decomposition. (#15634)
* CONCAT flattening, filter decomposition.

Flattening: CONCAT(CONCAT(x, y), z) is flattened to CONCAT(x, y, z). This
is especially useful for the || operator, which is a binary operator and
leads to non-flat CONCAT calls.

Filter decomposition: transforms CONCAT(x, '-', y) = 'a-b' into
x = 'a' AND y = 'b'.

* One more test.

* Fix two tests.

* Adjustments from review.

* Fix empty string problem, add tests.
2024-01-11 11:18:50 -08:00
Gian Merlino 2231cb30a4
Faster k-way merging using tournament trees, 8-byte key strides. (#15661)
* Faster k-way merging using tournament trees, 8-byte key strides.

Two speedups for FrameChannelMerger (which does k-way merging in MSQ):

1) Replace the priority queue with a tournament tree, which does fewer
   comparisons.

2) Compare keys using 8-byte strides, rather than 1 byte at a time.

* Adjust comments.

* Fix style.

* Adjust benchmark and test.

* Add eight-list test (power of two).
2024-01-11 08:36:22 -08:00
Clint Wylie 2118258b54
tidy up group by engines after removal of v1 (#15665) 2024-01-11 00:52:52 -08:00
Laksh Singla 87fbe42218
"Partition boost" the group by queries in MSQ for better splits (#15474)
"Partition boost" the group by queries in MSQ for better splits
2024-01-11 12:46:27 +05:30
Vadim Ogievetsky 5b769a7d32
Update load query detail archive dialog for file input support (#15632)
* Update execution-submit-dialog for file input support

Modified the execution-submit-dialog to support file inputs instead of text inputs for better usability. Users can now submit their queries by selecting a JSON file directly or dragging the file into the dialog. Made appropriate UI adjustments to accommodate this change in execution-submit-dialog styles file.

* Update web-console/src/views/workbench-view/execution-submit-dialog/execution-submit-dialog.tsx

Co-authored-by: Charles Smith <techdocsmith@gmail.com>

* Update web-console/src/views/workbench-view/execution-submit-dialog/execution-submit-dialog.tsx

Co-authored-by: Charles Smith <techdocsmith@gmail.com>

* Update web-console/src/views/workbench-view/execution-submit-dialog/execution-submit-dialog.tsx

Co-authored-by: Charles Smith <techdocsmith@gmail.com>

* Update drag-and-drop instructions in execution-submit-dialog

* Add snapshot tests for ExecutionSubmitDialog

* prettify

---------

Co-authored-by: Charles Smith <techdocsmith@gmail.com>
2024-01-10 20:48:46 -08:00
Kashif Faraz f445ba4d6b
Audit API DELETE datasource (markAllSegmentsAsUnused) (#15653)
Changes:
- Add audit for `DELETE /druid/coordinator/v1/datasources/{datasourceName}`
- Minor refactor
2024-01-11 09:43:32 +05:30
Gian Merlino ee77fa7fb3
Add tests for CASE decomposition. (#15639)
I was looking into adding a rule to do this, and found that it was already
happening as part of Calcite's RexSimplify. So this patch simply adds some
tests to ensure that it continues to happen.
2024-01-10 13:24:24 -08:00
Laksh Singla 0b91cc4db2
Fix incorrect tests in Sting first/last serde's null handling (#15657)
Fixes a couple of incorrect test cases, that got merged accidentally
2024-01-10 19:16:12 +05:30
Kashif Faraz d623756c66
Add cache for password hash in druid-basic-security (#15648)
Add class PasswordHashGenerator. Move hashing logic from BasicAuthUtils to this new class.
Add cache in the hash generator to contain the computed hash of passwords and boost validator performance
Cache has max size 1000 and expiry 1 hour
Key of the cache is an SHA-256 hash of the (password + random salt generated on service startup)
2024-01-10 17:46:09 +05:30
Zoltan Haindrich fefa763722
Resultcache fetch should deserialize aggregates when they are real results (#15654)
Fixes #15538
2024-01-10 06:42:33 -05:00
Yuanli Han 99d4b7dca7
Use to trigger search bar of official site (#15652) 2024-01-10 17:18:36 +08:00
Laksh Singla 4149f98934
Fixes a bug with long string pair serde where null and empty strings are treated equivalently (#15525)
This PR fixes a bug with the long string pair serde where null and empty strings are treated equivalently, and the return value is always null. When 'useDefaultValueForNull' was set to true by default, this wasn't a commonly seen issue, because nulls were equivalent to empty strings. However, since the default has changed to false, this can create incorrect results when the long string pairs are serded, where the empty strings are incorrectly converted to nulls.
2024-01-10 14:17:57 +05:30
Ankit Kothari 355c2f5da0
Add sql + ingestion compatibility for first/last on numeric values (#15607)
SQL compatibility for numeric last and first column types.
Ingestion UI now provides option for first and last aggregation as well.
2024-01-10 12:59:38 +05:30
PANKAJ KUMAR 047c7340ab
Adding retries to update the metadata store instead of failure (#15141)
Currently, If 2 tasks are consuming from the same partitions, try to publish the segment and update the metadata, the second task can fail because the end offset stored in the metadata store doesn't match with the start offset of the second task. We can fix this by retrying instead of failing.

AFAIK apart from the above issue, the metadata mismatch can happen in 2 scenarios:

- when we update the input topic name for the data source
- when we run 2 replicas of ingestion tasks(1 replica will publish and 1 will fail as the first replica has already updated the metadata).

Implemented the comparable function to compare the last committed end offset and new Sequence start offset. And return a specific error msg for this.

Add retry logic on indexers to retry for this specific error msg.

Updated the existing test case.
2024-01-10 12:30:54 +05:30
Clint Wylie 2938b8de53
fix issue with NestedPathArrayElement not correctly handling negative index for Object[] like it has for List (#15650) 2024-01-10 09:46:08 +05:30
Rishabh Singh 71f5307277
Eliminate Periodic Realtime Segment Metadata Queries: Task Now Publish Schema for Seamless Coordinator Updates (#15475)
The initial step in optimizing segment metadata was to centralize the construction of datasource schema in the Coordinator (#14985). Subsequently, our goal is to eliminate the requirement for regularly executing queries to obtain segment schema information. This task encompasses addressing both realtime and finalized segments.

This modification specifically addresses the issue with realtime segments. Tasks will now routinely communicate the schema for realtime segments during the segment announcement process. The Coordinator will identify the schema alongside the segment announcement and subsequently update the schema for realtime segments in the metadata cache.
2024-01-10 08:55:56 +05:30
Vadim Ogievetsky 85b8cf9f37
Web console: Fix concurrent tasks (#15649)
* Improve handling of concurrent tasks option

* Update snapshots
2024-01-09 16:09:42 -08:00
Pranav 747d973752
Skip waiting for first lookup version to get initialized (#15598) 2024-01-09 13:18:39 -08:00