Commit Graph

14403 Commits

Author SHA1 Message Date
Rishabh Singh a8c06e93aa
Skip tombstone segment refresh in metadata cache (#17025)
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-13 11:47:11 +05:30
Akshat Jain fff3e81dcc
Add window function drill tests for array_concat_agg for empty over scenarios (#17026)
* Add window function drill tests for array_concat_agg for empty over scenarios

* Cleanup sqlNativeIncompatible() as it's not needed now

* Address review comment
2024-09-13 11:35:45 +05:30
Abhishek Radhakrishnan 668169d9a9
Provide `chmod` command for `-XX:OnOutOfMemoryError` from shell script (#17054)
A command line arg -XX:OnOutOfMemoryError='chmod 644 ${project.parent.basedir}/target/*.hprof' was added to collect heap dumps: #17029

This arg is causing problems when running tests from Intellij. Intellij doesn't seem to likechmod 644, but this command works as expected in mvn. So as a workaround, add the chmod 644 ${BASE_DIR/target/*.hprof' command in a shell script that can then be executed when OnOutOfMemoryError happens to make Intellij happy.
2024-09-13 00:17:28 -04:00
Abhishek Radhakrishnan 5ef94c9dee
Add support for selective loading of broadcast datasources in the task layer (#17027)
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.
2024-09-12 13:30:28 -04:00
Adithya Chakilam 6ef8d5d8e1
OshiSysMonitor: Add ability to skip emitting metrics (#16972)
* OshiSysMonitor: Add ability to skip emitting metrics

* comments

* static checks

* remove oshi
2024-09-12 11:32:31 -04:00
Abhishek Radhakrishnan c077daaade
GHA steps to collect and upload heap dumps to debug UT OOM errors (#17029)
* Add GHA steps to tar and upload any heap dumps on failure to debug UT OOM issues.

* Add jvm options to heap dump OnOutOfMemoryError

Co-authored-by: Elliott Freis <108356317+imply-elliott@users.noreply.github.com>

---------

Co-authored-by: Elliott Freis <108356317+imply-elliott@users.noreply.github.com>
2024-09-12 09:06:35 -04:00
Laksh Singla d3392a23ce
Cancel the group by processing tasks if the merging runner gets scheduled post the query timeout (#17037)
If the GroupByMergingQueryRunner gets scheduled after the query timeout, it fails to clean up the processing tasks that have been scheduled. This can lead to unnecessary processing being done for the tasks whos results won't get consumed.
2024-09-12 15:10:27 +05:30
Pranav a95397e712
Allow request headers in HttpInputSource in native and MSQ Ingestion (#16974)
Support for adding the request headers in http input source. we can now pass the additional headers as json in both native and MSQ.
2024-09-12 11:18:44 +05:30
Rishabh Singh a18f582ef0
Skip refresh for unused segments in metadata cache (#16990)
* 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-12 10:39:59 +05:30
George Shiqi Wu 428f58cf15
Support maxColumnsToMerge in supervisor tuningConfig (#17030)
* support maxColumnsToMerge in supervisor specs

* remove log line

* fix style

* add docs

* fix unit tests
2024-09-11 18:00:13 -04:00
Vadim Ogievetsky 9e1544e9c4
Fix maxRowsInMemory default for streaming (#17028)
* fix maxRowsInMemory

* fix button css
2024-09-11 08:43:00 -07:00
Sébastien 5de84253d8
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

* -> fn

* -> fn
2024-09-10 11:34:49 -07:00
aho135 2427972c10
Implement segment range threshold for automatic query prioritization (#17009)
Implements threshold based automatic query prioritization using the time period of the actual segments scanned. This differs from the current implementation of durationThreshold which uses the duration in the user supplied query. There are some usability constraints with using durationThreshold from the user supplied query, especially when using SQL. For example, if a client does not explicitly specify both start and end timestamps then the duration is extremely large and will always exceed the configured durationThreshold. This is one example interval from a query that specifies no end timestamp:
"interval":["2024-08-30T08:05:41.944Z/146140482-04-24T15:36:27.903Z"]. This interval is generated from a query like SELECT * FROM table WHERE __time > CURRENT_TIMESTAMP - INTERVAL '15' HOUR. Using the time period of the actual segments scanned allows proper prioritization without explicitly having to specify start and end timestamps. This PR adds onto #9493
2024-09-10 15:01:52 +05:30
Sree Charan Manamala c7c3307e61
Fix String Frame Readers to read String Arrays correctly (#16885)
While writing to a frame, String arrays are written by setting the multivalue byte.
But while reading, it was hardcoded to false.
2024-09-10 14:20:54 +05:30
Laksh Singla 72fbaf2e56
Non querying tasks shouldn't use processing buffers / merge buffers (#16887)
Tasks that do not support querying or query processing i.e. supportsQueries = false do not require processing threads, processing buffers, and merge buffers.
2024-09-10 11:36:36 +05:30
Abhishek Agarwal 78775ad398
Prepare master for 32.0.0 release (#17022) 2024-09-10 11:01:20 +05:30
Clint Wylie f57cd6f7af
transition away from StorageAdapter (#16985)
* 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 14:55:29 -07:00
Abhishek Radhakrishnan f4261c0e4d
Add Delta snapshot version to the web-console (#17023)
* Web-console change to add Delta snapshot version.

Web-console change for https://github.com/apache/druid/pull/17004.

* Update web-console/src/druid-models/input-source/input-source.tsx

* Update web-console/src/druid-models/ingestion-spec/ingestion-spec.tsx
2024-09-09 11:49:53 -07:00
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
Clint Wylie b0f36c1b89
fix bug with CastOperatorConversion with types which cannot be mapped to native druid types (#17011) 2024-09-06 17:07:32 -07:00
Edgar Melendrez 48a758ee08
[docs] reverting changes for sql-functions.md (#17019) 2024-09-06 16:07:32 -07:00
Katya Macedo 94b0705109
Docs - Update the architecture diagram (#17007) 2024-09-06 12:21:27 -07:00
Edgar Melendrez 2d9e92ce78
[docs] Batch11 date and time functions (#16926)
* first draft of functions

* minor improvments

* Update docs/querying/sql-functions.md

* Update docs/querying/sql-scalar.md

* Apply suggestions from code review

Accepted as is

Co-authored-by: Katya Macedo  <38017980+ektravel@users.noreply.github.com>

* applying next round of suggestions

* fixing missing column name

* addressing floor and ceil functions

* Apply suggestions from code review

Co-authored-by: Katya Macedo  <38017980+ektravel@users.noreply.github.com>

* Apply suggestions from code review

Co-authored-by: Katya Macedo  <38017980+ektravel@users.noreply.github.com>

* re-wording TIMESTAMPADD

---------

Co-authored-by: Benedict Jin <asdf2014@apache.org>
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
2024-09-06 12:20:47 -07:00
Edgar Melendrez ed811262e3
[docs] Batch13 IP functions (#16947)
* new datasource

* reviewing before pr

* Update docs/querying/sql-functions.md

* Apply suggestions from code review

Co-authored-by: Katya Macedo  <38017980+ektravel@users.noreply.github.com>

* Apply suggestions from code review

Co-authored-by: Katya Macedo  <38017980+ektravel@users.noreply.github.com>

* Apply suggestions from code review

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

* Applying suggestions to IPV4_PARSE

---------

Co-authored-by: Benedict Jin <asdf2014@apache.org>
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
2024-09-06 12:19:36 -07:00
Adarsh Sanjeev 73ff9f9047
Convert MSQTerminalStageSpecFactory into an interface (#16996)
* Convert MSQTerminalStageSpecFactory into an interface

* Rename class and remove useless constructor
2024-09-06 09:56:35 +05:30
Virushade 476b205efa
Docs: Fix language in Schema Design docs (#17010) 2024-09-06 08:48:00 +05:30
Gian Merlino 175636b28f
Frame writers: Coerce numeric and array types in certain cases. (#16994)
This patch adds "TypeCastSelectors", which is used when writing frames to
perform two coercions:

- When a numeric type is desired and the underlying type is non-numeric or
  unknown, the underlying selector is wrapped, "getObject" is called and the
  result is coerced using "ExprEval.ofType". This differs from the prior
  behavior where the primitive methods like "getLong", "getDouble", etc, would
  be called directly. This fixes an issue where a column would be read as
  all-zeroes when its SQL type is numeric and its physical type is string, which
  can happen when evolving a column's type from string to number.

-  When an array type is desired, the underlying selector is wrapped,
   "getObject" is called, and the result is coerced to Object[]. This coercion
   replaces some earlier logic from #15917.
2024-09-05 17:20:00 -07:00
Edgar Melendrez c49dc83b22
[docs] batch 12: reduction functions (#16930)
* [docs] batch 12: reduction functions

* Update docs/querying/sql-functions.md

* Update docs/querying/sql-functions.md

* Update docs/querying/sql-functions.md

* applying suggestions

* Apply suggestions from code review

Co-authored-by: Katya Macedo  <38017980+ektravel@users.noreply.github.com>

---------

Co-authored-by: Benedict Jin <asdf2014@apache.org>
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
2024-09-05 17:02:45 -07:00
Vadim Ogievetsky dc5c55a836
Web console: better tooltip when no size is available (#17008)
* better tooltip when no size is available

* better labels for columns

* fix label in segments view
2024-09-05 13:51:03 -07:00
Kashif Faraz ba6f804f48
Fix compaction status API response (#17006)
Description:
#16768 introduces new compaction APIs on the Overlord `/compact/status` and `/compact/progress`.
But the corresponding `OverlordClient` methods do not return an object compatible with the actual
endpoints defined in `OverlordCompactionResource`.

This patch ensures that the objects are compatible.

Changes:
- Add `CompactionStatusResponse` and `CompactionProgressResponse`
- Use these as the return type in `OverlordClient` methods and as the response entity in `OverlordCompactionResource`
- Add `SupervisorCleanupModule` bound on the Coordinator to perform cleanup of supervisors.
Without this module, Coordinator cannot deserialize compaction supervisors.
2024-09-05 23:22:01 +05:30
Jill Osborne b4d83a86c2
Middle Manager wording update in docs (#17005) 2024-09-05 10:25:30 -07:00
Rishabh Singh 40f38f0191
Remove migrated deep storage standard ITs (#16933) 2024-09-05 16:07:33 +05:30
Rishabh Singh 18a9a7570a
Log a small subset of segments to refresh for debugging Coordinator refresh logic (#16998)
* Log a small number of segments to refresh per datasource in the Coordinator

* review comments

* Update log message
2024-09-05 11:00:25 +05:30
Rishabh Singh 39161b0b23
Use vault.centos.org to build Hadoop docker image (#16999)
The Dockerfile for building hadoop image is broken due to Centos 7 EOL.
Fixed it as per https://serverfault.com/a/1161847.
2024-09-05 10:36:55 +05:30
Rishabh Singh 4e02e5b856
Remove alert for pre-existing new columns while merging realtime schema (#16989)
Currently, an alert is thrown while merging datasource schema with realtime
segment schema when the datasource schema already has update columns from the delta schema.

This isn't an error condition since the datasource schema can have those columns from a different segment.

One scenario in which this can occur is when multiple replicas for a task is run.
2024-09-05 07:58:24 +05:30
Hugh Evans 9162339fa8
Replace dsql instructions in example (#16977) 2024-09-04 12:45:58 -07:00
AmatyaAvadhanula bfbd21bce0
Revert "Add integration tests for concurrent append and replace (#16755)" (#17000)
This reverts commit 70bad948e3.
2024-09-04 23:36:49 +05:30
Katya Macedo 03c37b3143
Fix spelling (#17001) 2024-09-04 13:33:17 -04:00
Laksh Singla b698440bfe
suppress cve (#16997) 2024-09-04 19:37:23 +05:30
Vishesh Garg e28424ea25
Enable rollup on multi-value dimensions for compaction with MSQ engine (#16937)
Currently compaction with MSQ engine doesn't work for rollup on multi-value dimensions (MVDs), the reason being the default behaviour of grouping on MVD dimensions to unnest the dimension values; for instance grouping on `[s1,s2]` with aggregate `a` will result in two rows: `<s1,a>` and `<s2,a>`. 

This change enables rollup on MVDs (without unnest) by converting MVDs to Arrays before rollup using virtual columns, and then converting them back to MVDs using post aggregators. If segment schema is available to the compaction task (when it ends up downloading segments to get existing dimensions/metrics/granularity), it selectively does the MVD-Array conversion only for known multi-valued columns; else it conservatively performs this conversion for all `string` columns.
2024-09-04 16:28:04 +05:30
Gian Merlino 76b8c20f4d
Create fewer temporary maps when querying sys.segments. (#16981)
Eliminates two map creations (availableSegmentMetadata, partialSegmentDataMap).
The segmentsAlreadySeen set remains.
2024-09-03 20:04:44 -07:00
Clint Wylie 57bf053dc9
remove compiler warnings about unqualified calls to yield() (#16995) 2024-09-03 20:04:30 -07:00
Gian Merlino 57c4b552d9
Fix logical merge conflict in SuperSorterTest. (#16993)
Logical merge conflict between #16911 and #16914.
2024-09-03 16:14:59 -04:00
Hardik Bajaj 2ef936be40
Update Documentation on meregeBuffer/pendingRequests for Real-time nodes (#16992)
#15025 adds mergeBuffer/pendingRequests metric in QueryCountStatsMonitor. Since real-time nodes also use the same merge buffers for queries and have QueryCountStatsMonitor , the documentation is being updated to include this metric.
2024-09-04 00:25:09 +05:30
Gian Merlino 786c959e9e
MSQ: Add limitHint to global-sort shuffles. (#16911)
* MSQ: Add limitHint to global-sort shuffles.

This allows pushing down limits into the SuperSorter.

* Test fixes.

* Add limitSpec to ScanQueryKit. Fix SuperSorter tracking.
2024-09-03 09:05:29 -07:00