Commit Graph

14429 Commits

Author SHA1 Message Date
Arun Ramani 13a52de07f
Skip generating task context reports for sub tasks (#17219) (#17223)
* Skip task context for sub tasks

* DRY a little + skip context for live report
2024-10-03 06:36:55 +05:30
Vadim Ogievetsky 7dd5e755bb
Backport #17147 and #17213 to 31 (#17222)
* Web console: add support for Dart engine (#17147)

* add console support for Dart engine

This reverts commit 6e46edf15dd55e5c51a1a4068e83deba4f22529b.

* feedback fixes

* surface new fields

* prioratize error over results

* better metadata refresh

* feedback fixes

* Web console: misc fixes to the Explore view (#17213)

* make record table able to hide column

* stickyness

* refactor query log

* fix measure drag

* start nested column dialog

* nested expand

* fix filtering on Measures

* use output name

* fix scrolling

* select all / none

* use ARRAY_CONCAT_AGG

* no need to limit if aggregating

* remove magic number

* better search

* update arg list

* add, don't replace
2024-10-02 10:36:31 -07:00
Kashif Faraz 491087fbe3
Modify DataSegmentProvider to also return DataSegment (#17021) (#17217)
Currently, TaskDataSegmentProvider fetches the DataSegment from the Coordinator while loading the segment, but just discards it later. This PR refactors this to also return the DataSegment so that it can be used by workers without a separate fetch.

Co-authored-by: Adarsh Sanjeev <adarshsanjeev@gmail.com>
2024-10-02 14:34:27 +05:30
Kashif Faraz cf44747c4d
MSQ: Fix calculation of suggested memory in WorkerMemoryParameters. (#17108) (#17215)
The "suggested server memory" figure needs to take into account
maxConcurrentStages. The fix here does not affect the main memory
calculations, but it does affect the accuracy of error messages.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-10-02 11:10:28 +05:30
Kashif Faraz 4a8008aebb
Use the whole frame when writing rows. (#17094) (#17195)
This patch makes the following adjustments to enable writing larger
single rows to frames:
1) RowBasedFrameWriter: Max out allocation size on the final doubling.
   i.e., if the final allocation "naturally" would be 1 MiB but the
   max frame size is 900 KiB, use 900 KiB rather than failing the 1 MiB
   allocation.
2) AppendableMemory: In reserveAdditional, release the last block if it
   is empty. This eliminates waste when a frame writer uses a
   successive-doubling approach to find the right allocation size.
3) ArenaMemoryAllocator: Reclaim memory from the last allocation when
   the last allocation is closed.

Prior to these changes, a single row could be much smaller than the
frame size and still fail to be added to the frame.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-10-02 07:44:33 +05:30
Vadim Ogievetsky 2f2807d379
fix input step typo (#17202) (#17207) 2024-10-01 08:03:47 -07:00
Kashif Faraz 23b9039a02
MSQ: Rework memory management. (#17057) (#17210)
This patch reworks memory management to better support multi-threaded
workers running in shared JVMs. There are two main changes.

First, processing buffers and threads are moved from a per-JVM model to
a per-worker model. This enables queries to hold processing buffers
without blocking other concurrently-running queries. Changes:

- Introduce ProcessingBuffersSet and ProcessingBuffers to hold the
  per-worker and per-work-order processing buffers (respectively). On Peons,
  this is the JVM-wide processing pool. On Indexers, this is a per-worker
  pool of on-heap buffers. (This change fixes a bug on Indexers where
  excessive processing buffers could be used if MSQ tasks ran concurrently
  with realtime tasks.)

- Add "bufferPool" argument to GroupingEngine#process so a per-worker pool
  can be passed in.

- Add "druid.msq.task.memory.maxThreads" property, which controls the
  maximum number of processing threads to use per task. This allows usage of
  multiple processing buffers per task if admins desire.

- IndexerWorkerContext acquires processingBuffers when creating the FrameContext
  for a work order, and releases them when closing the FrameContext.

- Add "usesProcessingBuffers()" to FrameProcessorFactory so workers know
  how many sets of processing buffers are needed to run a given query.

Second, adjustments to how WorkerMemoryParameters slices up bundles, to
favor more memory for sorting and segment generation. Changes:

- Instead of using same-sized bundles for processing and for sorting,
  workers now use minimally-sized processing bundles (just enough to read
  inputs plus a little overhead). The rest is devoted to broadcast data
  buffering, sorting, and segment-building.

- Segment-building is now limited to 1 concurrent segment per work order.
  This allows each segment-building action to use more memory. Note that
  segment-building is internally multi-threaded to a degree. (Build and
  persist can run concurrently.)

- Simplify frame size calculations by removing the distinction between
  "standard" and "large" frames. The new default frame size is the same
  as the old "standard" frames, 1 MB. The original goal of of the large
  frames was to reduce the number of temporary files during sorting, but
  I think we can achieve the same thing by simply merging a larger number
  of standard frames at once.

- Remove the small worker adjustment that was added in #14117 to account
  for an extra frame involved in writing to durable storage. Instead,
  account for the extra frame whenever we are actually using durable storage.

- Cap super-sorter parallelism using the number of output partitions, rather
  than using a hard coded cap at 4. Note that in practice, so far, this cap
  has not been relevant for tasks because they have only been using a single
  processing thread anyway.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-10-01 19:50:24 +05:30
Kashif Faraz 9b192bd5a2
add CursorHolder.isPreAggregated method to allow cursors on pre-aggregated data (#17058) (#17205)
changes:
* CursorHolder.isPreAggregated method indicates that a cursor has pre-aggregated data for all AggregatorFactory specified in a CursorBuildSpec. If true, engines should rewrite the query to use AggregatorFactory.getCombiningAggreggator, and column selector factories will provide selectors with the aggregator interediate type for the aggregator factory name
* Added groupby, timeseries, and topN support for CursorHolder.isPreAggregated
* Added synthetic test since no CursorHolder implementations support isPreAggregated at this point in time

Co-authored-by: Clint Wylie <cwylie@apache.org>
2024-10-01 15:33:16 +05:30
Kashif Faraz ef7c461722
Add support for selective loading of broadcast datasources in the task layer (#17027) (#17206)
Tasks control the loading of broadcast datasources via BroadcastDatasourceLoadingSpec getBroadcastDatasourceLoadingSpec(). By default, tasks download all broadcast datasources, unless there's an override as with kill and MSQ controller task.

The CLIPeon command line option --loadBroadcastSegments is deprecated in favor of --loadBroadcastDatasourceMode.

Broadcast datasources can be specified in SQL queries through JOIN and FROM clauses, or obtained from other sources such as lookups.To this effect, we have introduced a BroadcastDatasourceLoadingSpec. Finding the set of broadcast datasources during SQL planning will be done in a follow-up, which will apply only to MSQ tasks, so they load only required broadcast datasources. This PR primarily focuses on the skeletal changes around BroadcastDatasourceLoadingSpec and integrating it from the Task interface via CliPeon to SegmentBootstrapper.

Currently, only kill tasks and MSQ controller tasks skip loading broadcast datasources.

Co-authored-by: Abhishek Radhakrishnan <abhishek.rb19@gmail.com>
2024-10-01 11:09:52 +05:30
Vadim Ogievetsky 2062948721
[Backport] console work to 31 (#16991, #17132, #17133, #17135, #17180) (#17200)
* Web console query view improvements (#16991)
* Made maxNumTaskOptions configurable in the Query view
* Updated the copy for taskAssignment options
* Reordered options in engine menu for msq engine
* fixed snapshot
* maxNumTaskOptions -> maxTasksOptions
* added back select destination item
* fixed duplicate menu item
* snapshot
* Added the ability to hide certain engine menu options
* Added the ability to hide/show more menu items
* Make the tooltip better and improve structure (#17132)
* switch to using arrays by default (#17133)
* Web console: add stage graph (#17135)
* Web console: revamp the experimental explore view (#17180)
* explore revamp
* remove ToDo
* fix CodeQL
* add tooltips
* show issue on echart chars
* fix: browser back does not refresh chart
* fix maxRows 0
* be more resiliant to missing __time
---------
Co-authored-by: Sébastien <sebastien@imply.io>
2024-10-01 09:07:03 +05:30
Abhishek Radhakrishnan 8531a84d6c
Update Delta Kernel to 3.2.1 (#17179) (#17198)
Updated Delta Kernel from 3.2.0 to 3.2.1. This upstream version bump contains fixes to reading long columns, class loader and better retry mechanism when reading checkpoint files.
2024-09-30 11:50:04 -07:00
Abhishek Radhakrishnan c6f41dcd22
Web console: Fixed sampling for delta source in classic data loader and MSQ (#17160) (#17199) 2024-09-30 10:25:38 -07:00
Kashif Faraz 7638d29c40
PostJoinCursor should never advance without interruption (#17099) (#17196)
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
2024-09-30 20:37:18 +05:30
Kashif Faraz 25f2447fb2
Add test for exceptions in FutureUtils.transformAsync. (#17106) (#17197)
Adds an additional test case to FutureUtilsTest.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-09-30 20:36:56 +05:30
Kashif Faraz f4ad5d001b
FrameChannelMerger: Fix incorrect behavior of finished(). (#17088) (#17194)
Previously, the processor used "remainingChannels" to track the number of
non-null entries of currentFrame. Now, "remainingChannels" tracks the
number of channels that are unfinished.

The difference is subtle. In the previous code, when an input channel
was blocked upon exiting nextFrame(), the "currentFrames" entry would be
null, and therefore the "remainingChannels" variable would be decremented.
After the next await and call to populateCurrentFramesAndTournamentTree(),
"remainingChannels" would be incremented if the channel had become
unblocked after awaiting.

This means that finished(), which returned true if remainingChannels was
zero, would not be reliable if called between nextFrame() and the
next await + populateCurrentFramesAndTournamentTree().

This patch changes things such that finished() is always reliable. This
fixes a regression introduced in PR #16911, which added a call to
finished() that was, at that time, unsafe.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-09-30 19:47:10 +05:30
Kashif Faraz 8c7c422d8d
[Backport] Speed up FrameFileTest, SuperSorterTest. (#17068) (#17190)
These are two heavily parameterized tests that, together, account for
about 60% of runtime in the test suite.

FrameFileTest changes:
1) Cache frame files in a static, rather than building the frame file
   for each parameterization of the test.
2) Adjust TestArrayCursorFactory to cache the signature, rather than
   re-creating it on each call to getColumnCapabilities.

SuperSorterTest changes:
1) Dramatically reduce the number of tests that run with
   "maxRowsPerFrame" = 1. These are particularly slow due to writing so
   many small files. Some still run, since it's useful to test edge cases,
   but much fewer than before.
2) Reduce the "maxActiveProcessors" axis of the test from [1, 2, 4] to
   [1, 3]. The aim is to reduce the number of cases while still getting
   good coverage of the feature.
3) Reduce the "maxChannelsPerProcessor" axis of the test from [2, 3, 8]
   to [2, 7]. The aim is to reduce the number of cases while still getting
   good coverage of the feature.
4) Use in-memory input channels rather than file channels.
5) Defer formatting of assertion failure messages until they are needed.
6) Cache the cursor factory and its signature in a static.
7) Cache sorted test rows (used for verification) in a static.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-09-30 16:49:55 +05:30
Cece Mei c31da4f7a7
Incorporate `estimatedComputeCost` into all `BitmapColumnIndex` classes. (#17125) (#17172)
changes:
* filter index processing is now automatically ordered based on estimated 'cost', which is approximated based on how many expected bitmap operations are required to construct the bitmap used for the 'offset'
* cursorAutoArrangeFilters context flag now defaults to true, but can be set to false to disable cost based filter index sorting
2024-09-30 02:24:05 -07:00
Clint Wylie 2e5d30993e
fix a mistake in CursorGranularizer to check doneness after advance (#17175) (#17176)
Fixes a mistake introduced in #16533 which can result in CursorGranularizer incorrectly trying to get values from a selector after calling cursor.advance because of a missing check for cursor.isDone
2024-09-30 02:21:11 -07:00
Clint Wylie 4123f2ca90
add multi-value string object vector matcher and expression vector object selectors (#17162) (#17165) 2024-09-30 02:20:30 -07:00
Adarsh Sanjeev bd71a81183
Fix bug with while adding DistinctSketches (#17184) (#17189)
* Fix bug with checking the incorrect key

* Add tests
2024-09-30 14:28:57 +05:30
Kashif Faraz 97cc748be0
[Backport] Additional tests for ChannelStageOutputReader. (#17050) (#17187)
The existing tests are moved into a "WithMaximalBuffering" subclass,
and a new "WithMinimalBuffering" subclass is added to test cases
where only a single frame is buffered.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2024-09-30 13:21:55 +05:30
Kashif Faraz feca6000d7
[Backport] Fix maxRowsInMemory default for streaming (#17028) (#17186)
* fix maxRowsInMemory

* fix button css

Co-authored-by: Vadim Ogievetsky <vadim@ogievetsky.com>
2024-09-30 11:56:09 +05:30
Adarsh Sanjeev e364d84e12
[Backport] Make sketch encoding configurable (#17086) (#17153)
Makes sketch encoding in MSQ configurable by the user. This would allow a user to configure the sketch encoding method for a specific query.

The default is octet stream encoding.
2024-09-30 11:26:17 +05:30
Zoltan Haindrich a16b75a42c
Window Functions : Context Parameter to Enable Transfer of RACs over wire (#17150) (#17182)
(cherry picked from commit 661614129e)

Co-authored-by: Sree Charan Manamala <sree.manamala@imply.io>
2024-09-30 10:35:14 +05:30
Clint Wylie 51bea56948
add VirtualColumns.findEquivalent and VirtualColumn.EquivalenceKey (#17084) (#17166) 2024-09-27 17:54:06 -07:00
Victoria Lim bcc3da6f98
[Backport] DIV not implemented in Druid 30 and earlier (#17095)
Co-authored-by: Edgar Melendrez <evmelendrez@gmail.com>
2024-09-26 10:03:57 -07:00
Cece Mei a0c842e98b
Create a FilterBundle.Builder class and use it to construct FilterBundle. (#17055) (#17159) 2024-09-25 18:04:32 -07:00
Akshat Jain 986bc62b88
MSQ window functions: Fix boost column not being written to the frame in window stage (#17155) 2024-09-25 16:31:37 +05:30
Akshat Jain 627752922c
MSQ window functions: Reject MVDs during window processing (#17036) (#17127)
* MSQ window functions: Reject MVDs during window processing

* MSQ window functions: Reject MVDs during window processing

* Remove parameterization from MSQWindowTest
2024-09-25 15:00:02 +05:30
Kashif Faraz 8059b86c7f
Cleanup Coordinator logs, add duty status API (#16959) (#17154)
Description
-----------
Coordinator logs are fairly noisy and don't give much useful information (see example below).
Even when the Coordinator misbehaves, these logs are not very useful.

Main changes
------------
- Add API `GET /druid/coordinator/v1/duties` that returns a status list of all duty groups currently running on the Coordinator
- Emit metrics `segment/poll/time`, `segment/pollWithSchema/time`, `segment/buildSnapshot/time`
- Remove redundant logs that indicate normal operation of well-tested aspects of the Coordinator

Refactors
---------
- Move some logic from `DutiesRunnable` to `CoordinatorDutyGroup`
- Move stats collection from `CollectSegmentAndServerStats` to `PrepareBalancerAndLoadQueues`
- Minor cleanup of class `DruidCoordinator`
- Clean up class `DruidCoordinatorRuntimeParams`
  - Remove field `coordinatorStartTime`. Maintain start time in `MarkOvershadowedSegmentsAsUnused` instead.
  - Remove field `MetadataRuleManager`. Pass supplier to constructor of applicable duties instead.
  - Make `usedSegmentsNewestFirst` and `datasourcesSnapshot` as non-nullable as they are always required.
2024-09-25 14:59:53 +05:30
Clint Wylie 1096728fa4
use CastToObjectVectorProcessor for cast to string (#17148) (#17149) 2024-09-24 21:15:45 -07:00
Kashif Faraz d06327ab24
[Backport] Allow MSQ engine only for compaction supervisors (#17033) (#17143)
#16768 added the functionality to run compaction as a supervisor on the overlord.
This patch builds on top of that to restrict MSQ engine to compaction in the supervisor-mode only.
With these changes, users can no longer add MSQ engine as part of datasource compaction config,
or as the default cluster-level compaction engine, on the Coordinator. 

The patch also adds an Overlord runtime property `druid.supervisor.compaction.engine=<msq/native>`
to specify the default engine for compaction supervisors.

Since these updates require major changes to existing MSQ compaction integration tests,
this patch disables MSQ-specific compaction integration tests -- they will be taken up in a follow-up PR.

Key changed/added classes in this patch:
* CompactionSupervisor
* CompactionSupervisorSpec
* CoordinatorCompactionConfigsResource
* OverlordCompactionScheduler

Co-authored-by: Vishesh Garg <gargvishesh@gmail.com>
2024-09-25 09:29:00 +05:30
Clint Wylie cf00b4cd24
various fixes and improvements to vectorization fallback (#17098) (#17142)
changes:
* add `ApplyFunction` support to vectorization fallback, allowing many of the remaining expressions to be vectorized
* add `CastToObjectVectorProcessor` so that vector engine can correctly cast any type
* add support for array and complex vector constants
* reduce number of cases which can block vectorization in expression planner to be unknown inputs (such as unknown multi-valuedness)
* fix array constructor expression, apply map expression to make actual evaluated type match the output type inference
* fix bug in array_contains where something like array_contains([null], 'hello') would return true if the array was a numeric array since the non-null string value would cast to a null numeric
* fix isNull/isNotNull to correctly handle any type of input argument
2024-09-24 16:40:49 -07:00
Abhishek Radhakrishnan 0ae9988796
Support Iceberg ingestion from REST based catalogs (#17124) (#17145)
Adds support to the iceberg input source to read from Iceberg REST Catalogs.

Co-authored-by: Atul Mohan <atulmohan.mec@gmail.com>
2024-09-24 12:09:27 -07:00
Sree Charan Manamala b7cc0bb343
Window Functions : Remove enable windowing flag (#17087) (#17128)
(cherry picked from commit 67d361c9bf)
2024-09-24 10:28:11 +02:00
Abhishek Radhakrishnan 6c0ca77be4
Add Delta snapshot version to the web-console (#17023) (#17119)
Adds snapshot.version to the delta input source in the web-console:
2024-09-23 11:36:24 +05:30
Sree Charan Manamala 0c58f88ded
Add serde for ColumnBasedRowsAndColumns to fix window queries without group by (#16658) (#17111)
Register a Ser-De for RowsAndColumns so that the window operator query running on leaf operators would be transferred properly on the wire. Would fix the empty response given by window queries without group by on the native engine.

(cherry picked from commit bb1c3c1749)
2024-09-20 11:34:35 +02:00
Laksh Singla 2f13cd2500
Support maxSubqueryBytes for window functions (#16800) (#17085)
Window queries now acknowledge maxSubqueryBytes.
2024-09-19 19:31:17 +05:30
Rishabh Singh 60ed36c89b
Skip tombstone segment refresh in metadata cache (#17025) (#17112)
This PR #16890 introduced a change to skip adding tombstone segments to the cache.
It turns out that as a side effect tombstone segments appear unavailable in the console. This happens because availability of a segment in Broker is determined from the metadata cache.

The fix is to keep the segment in the metadata cache but skip them from refresh.

This doesn't affect any functionality as metadata query for tombstone returns empty causing continuous refresh of those segments.
2024-09-19 14:39:15 +05:30
Sree Charan Manamala 11727af2a6
Fix String Frame Readers to read String Arrays correctly (#16885) (#17103)
While writing to a frame, String arrays are written by setting the multivalue byte.
But while reading, it was hardcoded to false.

(cherry picked from commit c7c3307e61)
2024-09-19 09:02:12 +05:30
Akshat Jain 52929ed24a
Handle memory leaks from Mockito inline mocks (#17104) 2024-09-18 11:36:59 -07:00
Rishabh Singh a63ac2590a
Skip refresh for unused segments in metadata cache (#16990) (#17079)
* Skip refresh for unused segments in metadata cache

* Cover the condition where a used segment missing schema is marked for refresh

* Fix test
2024-09-17 17:18:53 -07:00
Clint Wylie c462e103b6
transition away from StorageAdapter (#16985) (#17024)
* transition away from StorageAdapter
changes:
* CursorHolderFactory has been renamed to CursorFactory and moved off of StorageAdapter, instead fetched directly from the segment via 'asCursorFactory'. The previous deprecated CursorFactory interface has been merged into StorageAdapter
* StorageAdapter is no longer used by any engines or tests and has been marked as deprecated with default implementations of all methods that throw exceptions indicating the new methods to call instead
* StorageAdapter methods not covered by CursorFactory (CursorHolderFactory prior to this change) have been moved into interfaces which are retrieved by Segment.as, the primary classes are the previously existing Metadata, as well as new interfaces PhysicalSegmentInspector and TopNOptimizationInspector
* added UnnestSegment and FilteredSegment that extend WrappedSegmentReference since their StorageAdapter implementations were previously provided by WrappedSegmentReference
* added PhysicalSegmentInspector which covers some of the previous StorageAdapter functionality which was primarily used for segment metadata queries and other metadata uses, and is implemented for QueryableIndexSegment and IncrementalIndexSegment
* added TopNOptimizationInspector to cover the oddly specific StorageAdapter.hasBuiltInFilters implementation, which is implemented for HashJoinSegment, UnnestSegment, and FilteredSegment
* Updated all engines and tests to no longer use StorageAdapter
2024-09-09 21:43:41 -07:00
abhishekagarwal87 2061c220b8 Prepare the release branch 2024-09-09 20:17:24 +05:30
Abhishek Radhakrishnan aa833a711c
Support for reading Delta Lake table snapshots (#17004)
Problem
Currently, the delta input source only supports reading from the latest snapshot of the given Delta Lake table. This is a known documented limitation.

Description
Add support for reading Delta snapshot. By default, the Druid-Delta connector reads the latest snapshot of the Delta table in order to preserve compatibility. Users can specify a snapshotVersion to ingest change data events from Delta tables into Druid.

In the future, we can also add support for time-based snapshot reads. The Delta API to read time-based snapshots is not clear currently.
2024-09-09 14:12:48 +05:30
Sree Charan Manamala 51fe3c08ab
Window Functions : Reject MVDs during window processing (#17002)
This commit aims to reject MVDs in window processing as we do not support them.
Earlier to this commit, query running a window aggregate partitioned by an MVD column would fail with ClassCastException
2024-09-09 12:07:54 +05:30
Rishabh Singh 67f5aa65e7
Set response type `application/json` in CustomExceptionMapper to return correct failure message (#17016)
* Add produces annotation to ParallelIndexSupervisorTask#report

* change to application/json

* Set response type in CustomExceptionMapper instead
2024-09-09 12:07:05 +05:30
Adarsh Sanjeev 616c46c958
Add framework for running MSQ tests with taskSpec instead of SQL (#16970)
* Add framework for running MSQ tests with taskSpec instead of SQL

* Allow configurable datasegment for tests

* Add test

* Revert "Add test"

This reverts commit 79fb241545.

* Revert "Allow configurable datasegment for tests"

This reverts commit caf04ede2b.
2024-09-09 11:38:28 +05:30
Vishesh Garg 37d4174245
Compute `range` partitionsSpec using effective `maxRowsPerSegment` (#16987)
In the compaction config, a range type partitionsSpec supports setting one of maxRowsPerSegment and targetRowsPerSegment. When compaction is run with the native engine, while maxRowsPerSegment = x results in segments of size x, targetRowsPerSegment = y results in segments of size 1.5 * y.

MSQ only supports rowsPerSegment = x as part of its tuning config, the resulting segment size being approx. x -- which is in line with maxRowsPerSegment behaviour in native compaction.

This PR makes the following changes:

use effective maxRowsPerSegment to pass as rowsPerSegment parameter for MSQ
persist rowsPerSegment as maxRowsPerSegment in lastCompactionState for MSQ
Use effective maxRowsPerSegment-based range spec in CompactionStatus check for both Native and MSQ.
2024-09-09 10:53:58 +05:30
Parth Agrawal b7a21a9f67
Revert "[CVE Fixes] Update version of Nimbus.jose.jwt (#16320)" (#16986)
This reverts commit f1d24c868f.

Updating nimbus to version 9+ is causing HTTP ERROR 500 java.lang.NoSuchMethodError: 'net.minidev.json.JSONObject com.nimbusds.jwt.JWTClaimsSet.toJSONObject()'
Refer to SAP/cloud-security-services-integration-library#429 (comment) for more details.

We would need to upgrade other libraries as well for updating nimbus.jose.jwt
2024-09-09 10:11:58 +05:30