Commit Graph

2088 Commits

Author SHA1 Message Date
zachjsh ba957a9b97
Add ability to limit the number of segments killed in kill task (#14662)
### Description

Previously, the `maxSegments` configured for auto kill could be ignored if an interval of data for a  given datasource had more than this number of unused segments, causing the kill task spawned with the task of deleting unused segments in that given interval of data to delete more than the `maxSegments` configured. Now each kill task spawned by the auto kill coordinator duty, will kill at most `limit` segments. This is done by adding a new config property to the `KillUnusedSegmentTask` which allows users to specify this limit.
2023-08-03 22:17:04 -04:00
Gian Merlino 5387f1bac0
Remove chatAsync parameter, so chat is always async. (#14692)
* Remove chatAsync parameter, so chat is always async.

chatAsync has been made default in Druid 26. I have seen good
battle-testing of it in production, and am comfortable removing the
older sync client.

This was the last remaining usage of IndexTaskClient, so this patch
deletes all that stuff too.

* Remove unthrown exception.

* Remove unthrown exception.

* No more TimeoutException.
2023-07-31 19:42:51 -07:00
Jason Koch 44d5c1a15f
split KillUnusedSegmentsTask to processing in smaller chunks (#14642)
split KillUnusedSegmentsTask to smaller batches

Processing in smaller chunks allows the task execution to yield the TaskLockbox lock,
which allows the overlord to continue being responsive to other tasks and users while
this particular kill task is executing.

* introduce KillUnusedSegmentsTask batchSize parameter to control size of batching

* provide an explanation for kill task batchSize parameter

* add logging details for kill batch progress
2023-07-31 12:56:27 -07:00
AmatyaAvadhanula c648b1cb36
Add task toolbox to DruidInputSource (#14507)
Add task toolbox to DruidInputSource
2023-07-31 13:12:01 +05:30
TSFenwick 9a9038c7ae
Speed up kill tasks by deleting segments in batch (#14131)
* allow for batched delete of segments instead of deleting segment data one by one

create new batchdelete method in datasegment killer that has default functionality
of iterating through all segments and calling delete on them. This will enable
a slow rollout of other deepstorage implementations to move to a batched delete
on their own time

* cleanup batchdelete segments

* batch delete with the omni data deleter

cleaned up code
just need to add tests and docs for this functionality

* update java doc to explain how it will try to use batch if function is overwritten

* rename killBatch to kill
add unit tests

* add omniDataSegmentKillerTest for deleting multiple segments at a time. fix checkstyle

* explain test peculiarity better

* clean up batch kill in s3.

* remove unused return value. cleanup comments and fix checkstyle

* default to batch delete. more specific java docs. list segments that couldn't be deleted
if there was a client error or server error

* simplify error handling

* add tests where an exception is thrown when killing multiple s3 segments

* add test for failing to delete two calls with the s3 client

* fix javadoc for kill(List<DataSegment> segments) clean up tests remove feature flag

* fix typo in javadocs

* fix test failure

* fix checkstyle and improve tests

* fix intellij inspections issues

* address comments, make delete multiple segments not assume same bucket

* fix test errors

* better grammar and punctuation. fix test. and better logging for exception

* remove unused code

* avoid extra arraylist instantiation

* fix broken test

* fix broken test

* fix tests to use assert.throws
2023-07-27 15:34:44 -07:00
Gian Merlino 986a271a7d
Merge core CoordinatorClient with MSQ CoordinatorServiceClient. (#14652)
* Merge core CoordinatorClient with MSQ CoordinatorServiceClient.

Continuing the work from #12696, this patch merges the MSQ
CoordinatorServiceClient into the core CoordinatorClient, yielding a single
interface that serves both needs and is based on the ServiceClient RPC
system rather than DruidLeaderClient.

Also removes the backwards-compatibility code for the handoff API in
CoordinatorBasedSegmentHandoffNotifier, because the new API was added
in 0.14.0. That's long enough ago that we don't need backwards
compatibility for rolling updates.

* Fixups.

* Trigger GHA.

* Remove unnecessary retrying in DruidInputSource. Add "about an hour"
retry policy and h

* EasyMock
2023-07-27 13:23:37 -07:00
Gian Merlino 2f9619a96f
Use OverlordClient for all Overlord RPCs. (#14581)
* Use OverlordClient for all Overlord RPCs.

Continuing the work from #12696, this patch removes HttpIndexingServiceClient
and the IndexingService flavor of DruidLeaderClient completely. All remaining
usages are migrated to OverlordClient.

Supporting changes include:

1) Add a variety of methods to OverlordClient.

2) Update MetadataTaskStorage to skip the complete-task lookup when
   the caller requests zero completed tasks. This helps performance of
   the "get active tasks" APIs, which don't want to see complete ones.

* Use less forbidden APIs.

* Fixes from CI.

* Add test coverage.

* Two more tests.

* Fix test.

* Updates from CR.

* Remove unthrown exceptions.

* Refactor to improve testability and test coverage.

* Add isNil tests.

* Remove unnecessary "deserialize" methods.
2023-07-24 21:14:27 -07:00
Gian Merlino bac5ef347c
Add ingest/input/bytes metric and Kafka consumer metrics. (#14582)
* Add ingest/input/bytes metric and Kafka consumer metrics.

New metrics:

1) ingest/input/bytes. Equivalent to processedBytes in the task reports.

2) kafka/consumer/bytesConsumed: Equivalent to the Kafka consumer
   metric "bytes-consumed-total". Only emitted for Kafka tasks.

3) kafka/consumer/recordsConsumed: Equivalent to the Kafka consumer
   metric "records-consumed-total". Only emitted for Kafka tasks.

* Fix anchor.

* Fix KafkaConsumerMonitor.

* Interface updates.

* Doc changes.

* Update indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java

Co-authored-by: Benedict Jin <asdf2014@apache.org>

---------

Co-authored-by: Benedict Jin <asdf2014@apache.org>
2023-07-20 10:56:22 +08:00
Clint Wylie 913416c669
add equality, null, and range filter (#14542)
changes:
* new filters that preserve match value typing to better handle filtering different column types
* sql planner uses new filters by default in sql compatible null handling mode
* remove isFilterable from column capabilities
* proper handling of array filtering, add array processor to column processors
* javadoc for sql test filter functions
* range filter support for arrays, tons more tests, fixes
* add dimension selector tests for mixed type roots
* support json equality
* rename semantic index maker thingys to mostly have plural names since they typically make many indexes, e.g. StringValueSetIndex -> StringValueSetIndexes
* add cooler equality index maker, ValueIndexes 
* fix missing string utf8 index supplier
* expression array comparator stuff
2023-07-18 12:15:22 -07:00
Maytas Monsereenusorn aef221f71b
Allow multiple consoleAppender to be used in peon logging (#14521)
* Allow multiple consoleAppender to be used in peon logging

* Fix Attempted to append to non-started appender error
2023-07-17 21:29:45 -07:00
AmatyaAvadhanula 0412f40d36
Prepare master branch for next release, 28.0.0 (#14595)
* Prepare master branch for next release, 28.0.0
2023-07-18 09:22:30 +05:30
Gian Merlino 450ecd6370
More efficient generation of ImmutableWorkerHolder from WorkerHolder. (#14546)
* More efficient generation of ImmutableWorkerHolder from WorkerHolder.

Taking the work done in #12096 a little further:

1) Applying a similar optimization to WorkerHolder (HttpRemoteTaskRunner).
   The original patch only helped with the ZkWorker (RemoteTaskRunner).

2) Improve the ZkWorker version somewhat by avoiding multiple iterations
   through the task announcements map.

* Pick better names and use better logic.

* Only runnable tasks.

* Fix test.

* Fix testBlacklistZKWorkers50Percent.
2023-07-13 07:57:16 -07:00
Gian Merlino 63ee69b4e8
Claim full support for Java 17. (#14384)
* Claim full support for Java 17.

No production code has changed, except the startup scripts.

Changes:

1) Allow Java 17 without DRUID_SKIP_JAVA_CHECK.

2) Include the full list of opens and exports on both Java 11 and 17.

3) Document that Java 17 is both supported and preferred.

4) Switch some tests from Java 11 to 17 to get better coverage on the
   preferred version.

* Doc update.

* Update errorprone.

* Update docker_build_containers.sh.

* Update errorprone in licenses.yaml.

* Add some more run-javas.

* Additional run-javas.

* Update errorprone.

* Suppress new errorprone error.

* Add exports and opens in ForkingTaskRunner for Java 11+.

Test, doc changes.

* Additional errorprone updates.

* Update for errorprone.

* Restore old fomatting in LdapCredentialsValidator.

* Copy bin/ too.

* Fix Java 15, 17 build line in docker_build_containers.sh.

* Update busybox image.

* One more java command.

* Fix interpolation.

* IT commandline refinements.

* Switch to busybox 1.34.1-glibc.

* POM adjustments, build and test one IT on 17.

* Additional debugging.

* Fix silly thing.

* Adjust command line.

* Add exports and opens one more place.

* Additional harmonization of strong encapsulation parameters.
2023-07-07 12:52:35 -07:00
Gian Merlino 021a01df45
RTR, HRTR: Fix incorrect maxLazyWorkers check in markLazyWorkers. (#14545)
Recently #14532 fixed a problem when maxLazyWorkers == 0 and lazyWorkers
starts out empty. Unfortunately, even after that patch, there remained
a more general version of this problem when maxLazyWorkers == lazyWorkers.size().
This patch fixes it.

I'm not sure if this would actually happen in production, because the
provisioning strategies do try to avoid calling markWorkersLazy until
previously-initiated terminations have finished. Nevertheless, it still
seems like a good thing to fix.
2023-07-07 10:08:12 -07:00
Kashif Faraz 40d0dc9e0e
Use separate executor to handle task updates in TaskQueue (#14533)
Description:
`TaskQueue.notifyStatus` is often a heavy call as it performs the following operations:
- Update task status in metadata DB
- Update task locks in metadata DB
- Request (synchronously) the task runner to shutdown the completed task
- Clean up in-memory data structures

This method can often be slow and can cause worker sync / task runners to slow down.

Main changes:
- Run task completion callbacks in a separate executor to handle task completion updates
- Add new config `druid.indexer.queue.taskCompleteHandlerNumThreads`
- Add metrics to monitor number of processed and queued items
- There are still other paths that can invoke `notifyStatus`, but those need not be moved to
the new executor as they are synchronous on purpose.

Other changes:
- Add new metrics `task/status/queue/count`, `task/status/handled/count`
- Add `TaskCountStatsProvider.getStats()` which deprecates the other `getXXXTaskCount` methods.
- Use `CoordinatorRunStats` to collect and report metrics. This class has been used as is
for now but will later be renamed and repurposed to use across all Druid services.
2023-07-07 20:43:12 +05:30
Gian Merlino 1fe61bc869
ChangeRequestHttpSyncer: Don't wait 1ms when checking isInitialized(). (#14547)
The wait doesn't seem to serve a purpose, other than causing delays
when checking isInitialized() for a large number of things that have
not yet been initialized.
2023-07-07 05:54:39 -07:00
Kashif Faraz d63eff3b1b
Reduce contention in HttpRemoteTaskRunner.getKnownTasks() (#14541) 2023-07-07 13:43:59 +05:30
Gian Merlino 037f09bef2
HttpRemoteTaskRunner: Fix markLazyWorkers for maxLazyWorkers == 0. (#14532) 2023-07-06 11:51:04 -07:00
Kashif Faraz 87bb1b9709
Fix bug during initialization of HttpServerInventoryView (#14517)
If a server is removed during `HttpServerInventoryView.serverInventoryInitialized`,
the initialization gets stuck as this server is never synced. The method eventually times
out (default 250s).

Fix: Mark a server as stopped if it is removed. `serverInventoryInitialized` only waits for
non-stopped servers to sync.

Other changes:
- Add new metrics for better debugging of slow broker/coordinator startup
  - `segment/serverview/sync/healthy`: whether the server view is syncing properly with a server
  - `segment/serverview/sync/unstableTime`: time for which sync with a server has been unstable  
- Clean up logging in `HttpServerInventoryView` and `ChangeRequestHttpSyncer`
- Minor refactor for readability
- Add utility class `Stopwatch`
- Add tests and stubs
2023-07-06 13:04:53 +05:30
AmatyaAvadhanula 609833c97b
Do not emit negative lag because of stale offsets (#14292)
The latest topic offsets are polled frequently and used to determine the lag based on the current offsets. However, when the offsets are stale (which can happen due to connection issues commonly), we may see a negative lag .

This PR prevents emission of metrics when the offsets are stale and at least one of the partitions has a negative lag.
2023-07-05 14:44:23 +05:30
Clint Wylie 277aaa5c57
remove druid.processing.columnCache.sizeBytes and CachingIndexed, combine string column implementations (#14500)
* combine string column implementations
changes:
* generic indexed, front-coded, and auto string columns now all share the same column and index supplier implementations
* remove CachingIndexed implementation, which I think is largely no longer needed by the switch of many things to directly using ByteBuffer, avoiding the cost of creating Strings
* remove ColumnConfig.columnCacheSizeBytes since CachingIndexed was the only user
2023-07-02 19:37:15 -07:00
Karan Kumar cb3a9d2b57
Adding Interactive API's for MSQ engine (#14416)
This PR aims to expose a new API called
"@path("/druid/v2/sql/statements/")" which takes the same payload as the current "/druid/v2/sql" endpoint and allows users to fetch results in an async manner.
2023-06-28 17:51:58 +05:30
Clint Wylie 31b9d5695d
Extend InitializedNullHandlingTest instead of NullHandlingTest (#14467)
NullHandlingTest is an actual test, it shouldn't be used as a base class
2023-06-22 15:01:50 +05:30
imply-cheddar cfd07a95b7
Errors take 3 (#14004)
Introduce DruidException, an exception whose goal in life is to be delivered to a user.

DruidException itself has javadoc on it to describe how it should be used.  This commit both introduces the Exception and adjusts some of the places that are generating exceptions to generate DruidException objects instead, as a way to show how the Exception should be used.

This work was a 3rd iteration on top of work that was started by Paul Rogers.  I don't know if his name will survive the squash-and-merge, so I'm calling it out here and thanking him for starting on this.
2023-06-19 01:11:13 -07:00
George Shiqi Wu 64af9bfe5b
Add groupId to metrics (#14402)
* Add group id as a dimension

* Revert changes

* Add to forking task runner

* Add missing metrics

* Fix indenting

* revert metrics

* Fix indentation
2023-06-16 09:28:16 -07:00
Gian Merlino 85656a467c
MSQ: Load broadcast tables on workers. (#14437)
They were not previously loaded because supportsQueries was false.
This patch sets supportsQueries to true, and clarifies in Task
javadocs that supportsQueries can be true for tasks that aren't
directly queryable over HTTP.
2023-06-16 12:02:20 +05:30
Clint Wylie 8454cc619a
auto columns fixes (#14422)
changes:
* auto columns no longer participate in generic 'null column' handling, this was a mistake to try to support and caused ingestion failures due to mismatched ColumnFormat, and will be replaced in the future with nested common format constant column functionality (not in this PR)
* fix bugs with auto columns which contain empty objects, empty arrays, or primitive types mixed with either of these empty constructs
* fix bug with bound filter when upper is null equivalent but is strict
2023-06-14 08:57:06 -07:00
Kashif Faraz 6e158704cb
Do not retry INSERT task into metadata if max_allowed_packet limit is violated (#14271)
Changes
- Add a `DruidException` which contains a user-facing error message, HTTP response code
- Make `EntryExistsException` extend `DruidException`
- If metadata store max_allowed_packet limit is violated while inserting a new task, throw
`DruidException` with response code 400 (bad request) to prevent retries
- Add `SQLMetadataConnector.isRootCausePacketTooBigException` with impl for MySQL
2023-06-10 12:15:44 +05:30
Harini Rajendran 4ff6026d30
Adding SegmentMetadataEvent and publishing them via KafkaEmitter (#14281)
In this PR, we are enhancing KafkaEmitter, to emit metadata about published segments (SegmentMetadataEvent) into a Kafka topic. This segment metadata information that gets published into Kafka, can be used by any other downstream services to query Druid intelligently based on the segments published. The segment metadata gets published into kafka topic in json string format similar to other events.
2023-06-02 21:28:26 +05:30
Andreas Maechler 45014bd5b4
Handle all types of exceptions when initializing input source in sampler API (#14355)
The sampler API returns a `400 bad request` response if it encounters a `SamplerException`.
Otherwise, it returns a generic `500 Internal server error` response, with the message
"The RuntimeException could not be mapped to a response, re-throwing to the HTTP container".

This commit updates `RecordSupplierInputSource` to handle all types of exceptions instead of just
`InterruptedException`and wrap them in a `SamplerException` so that the actual error is
propagated back to the user.
2023-06-02 19:43:53 +05:30
zachjsh 04a82da63d
Input source security fixes (#14266)
It was found that several supported tasks / input sources did not have implementations for the methods used by the input source security feature, causing these tasks and input sources to fail when used with this feature. This pr adds the needed missing implementations. Also securing the sampling endpoint with input source security, when enabled.
2023-06-01 16:37:19 -07:00
Rishabh Singh 2086ff88bc
Add logging for task stop operations (#14192)
Log more details when task cannot be stopped for various reasons
2023-05-30 18:50:52 +05:30
Alexander Saydakov 4131c0df13
use the latest datasketches-java-4.0.0 (#14334)
* use the latest datasketches-java-4.0.0

* updated versions of datasketches

* adjusted expectation

* fixed the expectations

---------

Co-authored-by: AlexanderSaydakov <AlexanderSaydakov@users.noreply.github.com>
2023-05-27 22:19:18 -07:00
Kashif Faraz 0cde3a8b52
Fix regression in batch segment allocation (#14337)
* Improve batch segment allocation logs

* Fix batch seg alloc regression

* Fix logs

* Fix logs

* Fix tests and logs
2023-05-25 22:34:54 -07:00
AmatyaAvadhanula e9913abbbf
Add new lock types: APPEND and REPLACE (#14258)
* Add new lock types: APPEND and REPLACE
2023-05-14 22:38:32 -07:00
imply-cheddar f9861808bc
Be able to load segments on Peons (#14239)
* Be able to load segments on Peons

This change introduces a new config on WorkerConfig
that indicates how many bytes of each storage
location to use for storage of a task.  Said config
is divided up amongst the locations and slots
and then used to set TaskConfig.tmpStorageBytesPerTask

The Peons use their local task dir and
tmpStorageBytesPerTask as their StorageLocations for
the SegmentManager such that they can accept broadcast
segments.
2023-05-12 16:51:00 -07:00
Kashif Faraz ba11b3d462
Refactor: Add OverlordDuty to replace OverlordHelper and align with CoordinatorDuty (#14235)
Changes:
- Replace `OverlordHelper` with `OverlordDuty` to align with `CoordinatorDuty`
  - Each duty has a `run()` method and defines a `Schedule` with an initial delay and period.
  - Update existing duties `TaskLogAutoCleaner` and `DurableStorageCleaner`
- Add utility class `Configs`
- Update log, error messages and javadocs
- Other minor style improvements
2023-05-12 22:39:56 +05:30
AmatyaAvadhanula 47e48ee657
Remove incorrect optimization (#14246) 2023-05-11 00:54:41 -07:00
Clint Wylie e833a4700d
suppress hadoop3 cve that seem not applicable to us (#14252) 2023-05-10 23:08:05 -07:00
Abhishek Radhakrishnan 46dabab36d
Fix NPE in test parse exception report. Add more tests with different thresholds. (#14209) 2023-05-05 10:05:41 -07:00
Abhishek Radhakrishnan 68f908e511
Fix uncaught `ParseException` when reading Avro from Kafka (#14183)
In StreamChunkParser#parseWithInputFormat, we call byteEntityReader.read() without handling a potential ParseException, which is thrown during this function call by the delegate AvroStreamReader#intermediateRowIterator.
A ParseException can be thrown if an Avro stream has corrupt data or data that doesn't conform to the schema specified or for other decoding reasons. This exception if uncaught, can cause ingestion to fail.
2023-05-04 12:35:36 +05:30
AmatyaAvadhanula ac7181bbda
Persist supervisor spec only after successful start (#14150)
* Persist spec after successful start

* Fix checkstyle.

* checkstyle after mvn install
2023-05-03 18:27:39 +05:30
Clint Wylie 90ea192d9c
fix bugs with auto encoded long vector deserializers (#14186)
This PR fixes an issue when using 'auto' encoded LONG typed columns and the 'vectorized' query engine. These columns use a delta based bit-packing mechanism, and errors in the vectorized reader would cause it to incorrectly read column values for some bit sizes (1 through 32 bits). This is a regression caused by #11004, which added the optimized readers to improve performance, so impacts Druid versions 0.22.0+.

While writing the test I finally got sad enough about IndexSpec not having a "builder", so I made one, and switched all the things to use it. Apologies for the noise in this bug fix PR, the only real changes are in VSizeLongSerde, and the tests that have been modified to cover the buggy behavior, VSizeLongSerdeTest and ExpressionVectorSelectorsTest. Everything else is just cleanup of IndexSpec usage.
2023-05-01 11:49:27 +05:30
Suneet Saldanha 84c11df980
Make LoggingEmitter more useful by using Markers (#14121)
* Make LoggingEmitter more useful

* Skip code coverage for facade classes

* fix spellcheck

* code review

* fix dependency

* logging.md

* fix checkstyle

* Add back jacoco version to main pom
2023-04-27 15:06:06 -07:00
Tejaswini Bandlamudi 774073b2e7
Update Hadoop3 as default build version (#14005)
Hadoop 2 often causes red security scans on Druid distribution because of the dependencies it brings. We want to move away from Hadoop 2 and provide Hadoop 3 distribution available. Switch druid to building with Hadoop 3 by default. Druid will still be compatible with Hadoop 2 and users can build hadoop-2 compatible distribution using hadoop2 profile.
2023-04-26 12:52:51 +05:30
Gian Merlino a7d4162195
Compaction: Block input specs not aligned with segmentGranularity. (#14127)
* Compaction: Block input specs not aligned with segmentGranularity.

When input intervals are not aligned with segmentGranularity, data may be
overshadowed if it lies in the space between the input intervals and the
output segmentGranularity.

In MSQ REPLACE, this is a validation error. IMO the same behavior makes
sense for compaction tasks. In case anyone was depending on the ability
to compact nonaligned intervals, a configuration parameter
allowNonAlignedInterval is provided. I don't expect it to be used much.

* Remove unused.

* ITCompactionTaskTest uses non-aligned intervals.
2023-04-25 17:06:16 -07:00
Nicholas Lippis 9d4cc501f7
return task status reported by peon (#14040)
* return task status reported by peon

* Write TaskStatus to file in AbstractTask.cleanUp

* Get TaskStatus from task log

* Fix merge conflicts in AbstractTaskTest

* Add unit tests for TaskLogPusher, TaskLogStreamer, NoopTaskLogs to satisfy code coverage

* Add license headerss

* Fix style

* Remove unknown exception declarations
2023-04-24 12:05:39 -07:00
TSFenwick accd5536df
Allow for Log4J to be configured for peons but still ensure console logging is enforced (#14094)
* Allow for Log4J to be configured for peons but still ensure console logging is enforced

This change will allow for log4j to be configured for peons but require console logging is still
configured for them to ensure peon logs are saved to deep storage.

Also fixed the test ConsoleLoggingEnforcementTest to use a valid appender for the non console
Config as the previous config was incorrect and would never return a logger.

* fix checkstyle

* add warning to logger when it overwrites all loggers to be console

* optimize calls for altering logging config for ConsoleLoggingEnforcementConfigurationFactory

add getName to the druid logger class

* update docs, and error message

* edit docs to be more clear

* fix checkstyle issues

* CI fixes - LoggerTest code coverage and fix spelling issue for logging docs
2023-04-24 10:41:56 -07:00
Clint Wylie 887f8db1b5
preserve explicitly specified dimension schema in "logical" schema of sampler response (#14144) 2023-04-23 21:28:05 +05:30
zachjsh 04da0102cb
KillTask should return empty inputSource resources (#14106)
### Description

This pr fixes a few bugs found with the inputSource security feature.

1. `KillUnusedSegmentsTask` previously had no definition for the `getInputSourceResources`, which caused an unsupportedOperationException to be thrown when this task type was submitted with the inputSource security feature enabled. This task type should not require any input source specific resources, so returning an empty set for this task type now.

2. Fixed a bug where when the input source type security feature is enabled, all of the input source type specific resources used where authenticated against:

`{"resource": {"name": "EXTERNAL", "type": "{INPUT_SOURCE_TYPE}"}, "action": "READ"}`

When they should be instead authenticated against:

`{"resource": {"name": "{INPUT_SOURCE_TYPE}", "type": "EXTERNAL"}, "action": "READ"}`

3. fixed bug where supervisor tasks were not authenticated against the specific input source types used, if input source security feature was enabled.
2023-04-18 15:27:16 -04:00