Commit Graph

1666 Commits

Author SHA1 Message Date
Akshat Jain fe0f4150c9
MSQ ingestion: Improve error message on encountering non-long timestamp column (#17411)
This PR improves the error message during MSQ ingestion if we encounter a non-long timestamp column.
2024-10-25 15:02:32 +05:30
Akshat Jain 1e96c85b38
WindowOperatorQueryFrameProcessor: Avoid writing multiple frames to output channel in runIncrementally() (#17373)
WindowOperatorQueryFrameProcessor: Avoid writing multiple frames to output channel in runIncrementally()
2024-10-23 10:34:37 +05:30
Gian Merlino 60daddedf8
SeekableStreamSupervisor: Use workerExec as the client connectExec. (#17394)
* SeekableStreamSupervisor: Use workerExec as the client connectExec.

This patch uses the already-existing per-supervisor workerExec as the
connectExec for task clients, rather than using the process-wide default
ServiceClientFactory pool.

This helps prevent callbacks from backlogging on the process-wide pool.
It's especially useful for retries, where callbacks may need to establish
new TCP connections or perform TLS handshakes.

* Fix compilation, tests.

* Fix style.
2024-10-22 20:21:21 -07:00
Vishesh Garg 5da9949992
Fail MSQ compaction if multi-valued partition dimensions are found (#17344)
MSQ currently supports only single-valued string dimensions as partition keys.
This patch adds a check to ensure that partition keys are single-valued in case
this info is available by virtue of segment download for schema inference.

During compaction, if MSQ finds multi-valued dimensions (MVDs) declared as part
of `range` partitionsSpec, it switches partitioning type to dynamic, ending up in
repeated compactions of the same interval. To avoid this scenario, the segment
download logic is also updated to always download segments if info on multi-valued
dimensions is required.
2024-10-19 13:33:33 +05:30
Abhishek Radhakrishnan 9a16d4e219
Move SqlTaskStatus and SqlTaskStausTest from msq module to sql module. (#17380)
- This is a non-functional change that moves SqlTaskStatus and its unit test SqlTaskStatusTest from the msq module to the sql module to help class reuse in other places.
- This refactor is extracted from this PR to facilitate easier review.
- Fix a minor spacing issue in the TaskStartTimeoutFault error message.
2024-10-18 14:39:01 -07:00
Laksh Singla 5b09329479
Fixes an issue with AppendableMemory that can cause MSQ jobs to fail (#17369) 2024-10-18 09:05:53 +05:30
Adithya Chakilam e834e49290
supervisor/autoscaler: Fix clearing of collected lags on skipped scale actions (#17356)
* superviosr/autoscaler: Fix clearing of collected lags on skipped scale actions

* comments

* supervisor/autoscaler: Skip scaling when partitions are less than minTaskCount (#17335)

* Fix pip installation after ubuntu upgrade (#17358)

* fix tests

---------

Co-authored-by: Pranav <pranavbhole@gmail.com>
2024-10-17 11:05:16 -07:00
Akshat Jain 8c52be81d3
Fix postgres metadata storage warning logs because of tablename causing issues (#17351) 2024-10-17 15:48:22 +05:30
Akshat Jain 450fb0147b
Add GlueingPartitioningOperator + Corresponding changes in window function layer to consume it for MSQ (#17038)
*    GlueingPartitioningOperator: It continuously receives data, and outputs batches of partitioned RACs. It maintains a last-partitioning-boundary of the last-pushed-RAC, and attempts to glue it with the next RAC it receives, ensuring that partitions are handled correctly, even across multiple RACs. You can check GlueingPartitioningOperatorTest for some good examples of the "glueing" work.
*    PartitionSortOperator: It sorts rows inside partitioned RACs, on the sort columns. The input RACs it receives are expected to be "complete / separate" partitions of data.
2024-10-17 10:54:52 +05:30
TessaIO a9f582711e
Fix loading lookup extension (#17212)
We introduce the option to iterate over fetched data from the dataFetcher for loadingLookups in the lookups-cached-single extension. Also, added the handling of a use case where the data exists in Druid but not in the actual data fetcher, which is in our use-case JDBC Data fetcher, where the value returned is null.

Signed-off-by: TessaIO <ahmedgrati1999@gmail.com>
2024-10-16 07:28:32 -07:00
Gian Merlino b287b219a8
MSQ: Include stageId, workerNumber in processing thread names. (#17324)
* MSQ: Include stageId, workerNumber in processing thread names.

Helps identify which query was running in a thread dump.

* s/dart/msq/
2024-10-11 08:37:15 -07:00
Gian Merlino a0c29f8bbb
MSQ WorkerResource: Fix timeout handler for httpGetChannelData. (#17328)
The timeout handler should fire if the response has not been handled yet
(i.e. if responseResolved was previously false). However, it erroneously
fires only if the response *was* handled. This causes HTTP 500 errors if
the timeout actually does fire. The timeout is 30 seconds, which can be
hit during pipelined queries, if an earlier stage of the query hasn't
produced its first frame within 30 seconds.

This fixes a regression introduced in #17140.
2024-10-11 16:29:04 +05:30
Karan Kumar 034bb9dbea
Removing enable windowing from MSQ tests. (#17276) 2024-10-11 05:33:27 +02:00
Clint Wylie a6236c3d15
add substituteCombiningFactory implementations for datasketches aggs (#17314)
Follow up to #17214, adds implementations for substituteCombiningFactory so that more
datasketches aggs can match projections, along with some projections tests for datasketches.
2024-10-10 16:14:06 +05:30
Gian Merlino 074944e02c
Dart: Only use historicals as workers. (#17319)
Only historicals load the Dart worker modules. Other types of servers in
the server view (such as realtime tasks) should not be included.
2024-10-10 13:47:58 +05:30
Gian Merlino 4092f3fe47
MSQ: Call "onQueryComplete" after the query is closed. (#17313)
This fixes a concurrency issue where, for failed queries, "onQueryComplete"
could be called concurrently with "onResultsStart" or "onResultRow". Fully
closing the controller ensures that the result reader is no longer active,
which eliminates the race.
2024-10-10 10:44:44 +05:30
Gian Merlino b27712933e
MSQ: Use leaf worker count for stages that have any leaf inputs. (#17312)
Previously, the leaf worker count was used for stages that have *no*
stage inputs. It should actually be used for stages that have *any*
non-broadcast, non-stage inputs.

This fixes a bug with broadcast joins. In a broadcast join, the stage has
both a table and a broadcast stage as input. Previously, it would be planned
using the non-leaf worker count. It should actually be planned using the
leaf worker count.
2024-10-10 10:44:31 +05:30
Gian Merlino baa16f30f6
DartWorkerContext: Return the correct workerId(). (#17280)
Prior to this patch, the workerId() method did not actually return
the worker ID. It returned some other string that had similar information,
but was different.

This caused the /druid/dart-worker/workers API, to return an internal
server error. The API is useful for debugging, although it is not used
during actual queries.
2024-10-08 09:52:55 -07:00
Gian Merlino 152330c5a8
WorkerManager: Correct javadoc for "stop". (#17279)
The javadoc had a factual error: Dart's implementation does not in
fact always return immediately.
2024-10-08 15:49:43 +05:30
Gian Merlino 0a279e634a
DartSqlResource: Return HTTP 202 on cancellation even if no such query. (#17278)
Return HTTP 202 (Accepted) on cancellation, even if the requested query
ID was not found.

The main reason for this is that when the Router broadcasts DELETE requests
to all Brokers, it returns the response from one of them randomly. If we
return 404 when a query ID isn't found, then the Router randomly returns 404s
even when the query really was found and canceled.

This is also arguably still correct behavior. The cancellation request
*was* accepted, it just won't do anything because the query was not in
fact running.
2024-10-08 15:49:34 +05:30
Gian Merlino 01baf99148
DartWorkerModule: Replace en dash with regular dash. (#17281)
Due to a typo, the thread name of the worker executor used an en dash (–)
rather than a regular hyphen (-). This was unintentional, and makes it
difficult to search for in thread dumps.
2024-10-08 15:48:10 +05:30
Gian Merlino 2309aa7bdf
DartSqlResource: Add controllerHost to GetQueriesResponse. (#17283)
This helps find the specific Broker that is executing a query.
2024-10-08 15:47:32 +05:30
Gian Merlino 9921ac1b19
DartSqlResource: Sort queries by start time. (#17282)
* DartSqlResource: Sort queries by start time.

This keeps the list of queries returned by the API in a consistent order.

* Fix test.
2024-10-08 15:47:21 +05:30
Gian Merlino 06bbdb38ce
MSQ: Allow for worker gaps. (#17277)
In a Dart query, all Historicals are given worker IDs, but not all of them
are going to actually be started or receive work orders. This can create gaps
in the set of workers. For example, workers 1 and 3 could have work assigned
while workers 0 and 2 do not.

This patch updates ControllerStageTracker and WorkerInputs to handle such
gaps, by using the set of actual worker numbers, rather than 0..workerCount,
in various places.
2024-10-08 15:07:57 +05:30
Vishesh Garg 7e35e50052
Fix issues with MSQ Compaction (#17250)
The patch makes the following changes:
1. Fixes a bug causing compaction to fail on array, complex, and other non-primitive-type columns
2. Updates compaction status check to be conscious of partition dimensions when comparing dimension ordering.
3. Ensures only string columns are specified as partition dimensions
4. Ensures `rollup` is true if and only if metricsSpec is non-empty
5. Ensures disjoint intervals aren't submitted for compaction
6. Adds `compactionReason` to compaction task context.
2024-10-06 21:48:26 +05:30
Clint Wylie 0bd13bcd51
Projections prototype (#17214) 2024-10-05 04:38:57 -07:00
Clint Wylie 04fe56835d
add druid.expressions.allowVectorizeFallback and default to false (#17248)
changes:

adds ExpressionProcessing.allowVectorizeFallback() and ExpressionProcessingConfig.allowVectorizeFallback(), defaulting to false until few remaining bugs can be fixed (mostly complex types and some odd interactions with mixed types)
add cannotVectorizeUnlessFallback functions to make it easy to toggle the default of this config, and easy to know what to delete when we remove it in the future
2024-10-05 12:42:42 +05:30
Gian Merlino d1709a329f
Dart: Skip final getCounters, postFinish to idle historicals. (#17255)
In a Dart query, all Historicals are given worker IDs, but not all of them
are going to actually be started or receive work orders.

Attempting to send a getCounters or postFinish command to a worker that
never received a work order is not only wasteful, but it causes errors due
to the workers not knowing about that query ID.
2024-10-04 23:05:21 -07:00
Shivam Garg 93b5a8326b
Upgrade commons-io to 2.17.0 (#17227) 2024-10-04 09:56:56 +02:00
Gian Merlino fc00664760
KafkaInputFormat: Fix handling of CSV/TSV keyFormat. (#17226)
* KafkaInputFormat: Fix handling of CSV/TSV keyFormat.

Follow-up to #16630, which fixed a similar issue for the valueFormat.

* Simplify.
2024-10-03 13:05:09 -07:00
Gian Merlino db7cc4634c
Dart: Smoother handling of stage early-exit. (#17228)
Stages can be instructed to exit before they finish, especially when a
downstream stage includes a "LIMIT". This patch has improvements related
to early-exiting stages.

Bug fix:

- WorkerStageKernel: Don't allow fail() to set an exception if the stage is
  already in a terminal state (FINISHED or FAILED). If fail() is called while
  in a terminal state, log the exception, then throw it away. If it's a
  cancellation exception, don't even log it. This fixes a bug where a stage
  that exited early could transition to FINISHED and then to FAILED, causing
  the overall query to fail.

Performance:

- DartWorkerManager previously sent stopWorker commands to workers
  even when "interrupt" was false. Now it only sends those commands when
  "interrupt" is true. The method javadoc already claimed this is what the
  method did, but the implementation did not match the javadoc. This reduces
  the number of RPCs by 1 per worker per query.

Quieter logging:

- In ReadableByteChunksFrameChannel, skip logging exception from setError if
  the channel has been closed. Channels are closed when readers are done with
  them, so at that point, we wouldn't be interested in the errors.

- In RunWorkOrder, skip calling notifyListener on failure of the main work,
  in the case when stop() has already been called. The stop() method will
  set its own error using CanceledFault. This enables callers to detect
  when a stage was canceled vs. failed for some other reason.

- In WorkerStageKernel, skip logging cancellation errors in fail(). This is
  made possible by the previous change in RunWorkOrder.
2024-10-03 20:09:02 +05:30
Gian Merlino 316f8c81d2
RunWorkOrder: Account for two simultaneous statistics collectors. (#17216)
* RunWorkOrder: Account for two simultaneous statistics collectors.

As a follow up to #17057, divide the amount of partitionStatsMemory
by two, to account for the fact that there are at some times going to
be two copies of the full collector. First there will be one for processors
and one for the accumulated collector. Then, after the processor ones are
GCed, a snapshot of the accumulated collector will be created.

Also includes an optimization to "addAll" for the two KeyCollectors,
for the case where we're adding into an empty collector. This is always
going to happen once per stage due to the "withAccumulation" call.

* Fix missing variable.

* Don't divide by numProcessingThreads twice.

* Fix test.
2024-10-03 16:25:01 +05:30
Akshat Jain edc235cfe1
WindowOperatorQueryFrameProcessor: Avoid unnecessary re-runs of runIncrementally() (#17211) 2024-10-03 15:33:50 +05:30
Akshat Jain 135ca8f6a7
WindowOperatorQueryFrameProcessor: Fix frame writer capacity issues + adhere to FrameProcessor's contract (#17209)
This PR fixes the above issue by maintaining the state of last rowId flushed to output channel, and triggering another iteration of runIncrementally() method if frame writer has rows pending flush to the output channel.

The above is done keeping in mind FrameProcessor's contract which enforces that we should write only a single frame to each output channel in any given iteration of runIncrementally().
2024-10-03 10:39:22 +05:30
Gian Merlino fbc1221837
DartTableInputSpecSlicer: Fix for TLS workers. (#17224)
We should use getHost(), which returns TLS if configured or plaintext
otherwise. getHostAndPort() returns plaintext only.
2024-10-03 11:01:11 +08:00
Gian Merlino 878adff9aa
MSQ profile for Brokers and Historicals. (#17140)
This patch adds a profile of MSQ named "Dart" that runs on Brokers and
Historicals, and which is compatible with the standard SQL query API.
For more high-level description, and notes on future work, refer to #17139.

This patch contains the following changes, grouped into packages.

Controller (org.apache.druid.msq.dart.controller):

The controller runs on Brokers. Main classes are,

- DartSqlResource, which serves /druid/v2/sql/dart/.
- DartSqlEngine and DartQueryMaker, the entry points from SQL that actually
  run the MSQ controller code.
- DartControllerContext, which configures the MSQ controller.
- DartMessageRelays, which sets up relays (see "message relays" below) to read
  messages from workers' DartControllerClients.
- DartTableInputSpecSlicer, which assigns work based on a TimelineServerView.

Worker (org.apache.druid.msq.dart.worker)

The worker runs on Historicals. Main classes are,

- DartWorkerResource, which supplies the regular MSQ WorkerResource, plus
  Dart-specific APIs.
- DartWorkerRunner, which runs MSQ worker code.
- DartWorkerContext, which configures the MSQ worker.
- DartProcessingBuffersProvider, which provides processing buffers from
  sliced-up merge buffers.
- DartDataSegmentProvider, which provides segments from the Historical's
  local cache.

Message relays (org.apache.druid.messages):

To avoid the need for Historicals to contact Brokers during a query, which
would create opportunities for queries to get stuck, all connections are
opened from Broker to Historical. This is made possible by a message relay
system, where the relay server (worker) has an outbox of messages.

The relay client (controller) connects to the outbox and retrieves messages.
Code for this system lives in the "server" package to keep it separate from
the MSQ extension and make it easier to maintain. The worker-to-controller
ControllerClient is implemented using message relays.

Other changes:

- Controller: Added the method "hasWorker". Used by the ControllerMessageListener
  to notify the appropriate controllers when a worker fails.
- WorkerResource: No longer tries to respond more than once in the
  "httpGetChannelData" API. This comes up when a response due to resolved future
  is ready at about the same time as a timeout occurs.
- MSQTaskQueryMaker: Refactor to separate out some useful functions for reuse
  in DartQueryMaker.
- SqlEngine: Add "queryContext" to "resultTypeForSelect" and "resultTypeForInsert".
  This allows the DartSqlEngine to modify result format based on whether a "fullReport"
  context parameter is set.
- LimitedOutputStream: New utility class. Used when in "fullReport" mode.
- TimelineServerView: Add getDruidServerMetadata as a performance optimization.
- CliHistorical: Add SegmentWrangler, so it can query inline data, lookups, etc.
- ServiceLocation: Add "fromUri" method, relocating some code from ServiceClientImpl.
- FixedServiceLocator: New locator for a fixed set of service locations. Useful for
  URI locations.
2024-10-01 14:38:55 -07:00
Kashif Faraz 28fead58b4
MSQ: Use task context flag useConcurrentLocks to determine task lock type (#17193) 2024-09-30 21:15:25 +05:30
Adarsh Sanjeev 95fb813b6d
Fix bug with while adding DistinctSketches (#17184)
* Fix bug with checking the incorrect key

* Add tests
2024-09-30 11:35:32 +05:30
Shivam Garg ab361747a8
Migrated commons-lang usages to commons-lang3 (#17156) 2024-09-28 10:28:11 +02:00
Gian Merlino dc223f22db
SQL: Use regular filters for time filtering in subqueries. (#17173)
* SQL: Use regular filters for time filtering in subqueries.

Using the "intervals" feature on subqueries, or any non-table, should be
avoided because it isn't a meaningful optimization in those cases, and
it's simpler for runtime implementations if they can assume all filters
are located in the regular filter object.

Two changes:

1) Fix the logic in DruidQuery.canUseIntervalFiltering. It was intended
   to return false for QueryDataSource, but actually returned true.

2) Add a validation to ScanQueryFrameProcessor to ensure that when running
   on an input channel (which would include any subquery), the query has
   "intervals" set to ONLY_ETERNITY.

Prior to this patch, the new test case in testTimeFilterOnSubquery would
throw a "Can only handle a single interval" error in the native engine,
and "QueryNotSupported" in the MSQ engine.

* Mark new case as having extra columns in decoupled mode.

* Adjust test.
2024-09-27 10:32:30 +05:30
Clint Wylie 6ee9e42f03
fix issue with ScanQueryFrameProcessor cursor build not adjusting intervals (#17168)
* fix issue with ScanQueryFrameProcessor cursor build not adjusting intervals

* all hail the robot overlords

* style bot
2024-09-26 21:09:14 +05:30
Clint Wylie d77637344d
log.warn anytime a column is relying on ArrayIngestMode.MVD (#17164)
* log.warn anytime a column is relying on ArrayIngestMode.MVD
2024-09-26 13:44:37 +05:30
Gian Merlino 2e531bb724
ScanQueryFrameProcessor: Close CursorHolders as we go along. (#17152)
* ScanQueryFrameProcessor: Close CursorHolders as we go along.

The change in #16533 added CursorHolders to the processor-level Closer.
This is problematic when running on an input channel: it means we started
keeping around all CursorHolders for all frames we process and closing them
when the channel is complete, rather than closing them as we go along.

This patch restores the prior behavior, where resources are closed as we go.

* Fix other call sites.

* Fix reference.

* Improvements.
2024-09-25 15:51:52 -07:00
Akshat Jain 446ffc325c
MSQ window functions: Fix boost column not being written to the frame in window stage (#17141)
In WindowOperatorQueryFrameProcessor, currently we were not adding the partition boost column to the ColumnSelectorFactory used to create the FrameWriter. Because of this, the boost column was not being written to the frame.
2024-09-25 09:50:42 +05:30
Adarsh Sanjeev 464b153850
Make sketch encoding configurable (#17086)
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-25 09:15:32 +05:30
George Shiqi Wu d1bfabbf4d
inter-Extension dependency support (#16973)
* update docs for kafka lookup extension to specify correct extension ordering

* fix first line

* test with extension dependencies

* save work on dependency management

* working dependency graph

* working pull

* fix style

* fix style

* remove name

* load extension dependencies recursively

* generate depenencies on classloader creation

* add check for circular dependencies

* fix style

* revert style changes

* remove mutable class loader

* clean up class heirarchy

* extensions loader test working

* add unit tests

* pr comments

* fix unit tests
2024-09-24 14:17:33 -04:00
zachjsh ba8245f114
Properly url encode schema and table names when syncing catalog (#17131)
* SQL syntax error should target USER persona

* * revert change to queryHandler and related tests, based on review comments

* * add test

* Properly encode schema and table names when syncing catalog

Previously table names and schema names were not being properly url
encododed when requested. Now they are.
2024-09-23 23:23:26 -04:00
Abhishek Radhakrishnan 37a2a12d79
rerwrite node so dynamic parameter applies to ingest node as well. (#17126) 2024-09-23 12:49:46 -07:00
Sree Charan Manamala 67d361c9bf
Window Functions : Remove enable windowing flag (#17087) 2024-09-23 08:24:26 +02:00
Akshat Jain 40414cfe78
MSQ window functions: Reject MVDs during window processing (#17036)
* MSQ window functions: Reject MVDs during window processing

* MSQ window functions: Reject MVDs during window processing

* Remove parameterization from MSQWindowTest
2024-09-23 11:39:35 +05:30