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.
Follow up to #17214, adds implementations for substituteCombiningFactory so that more
datasketches aggs can match projections, along with some projections tests for datasketches.
The duty group is a low cardinality dimension and can be helpful in providing insight
into whether a particular duty group is not running fast enough on the coordinator.
* Logger: Log context of DruidExceptions.
There is often interesting and unique information available in the
"context" of a DruidException. This information is additive to both
the message and the cause, and was missed when we log. This patch adds
the DruidException context to log messages whenever stack traces are
enabled.
* Only log nonempty contexts.
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.
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.
Description
-----------
The `OverlordCompactionScheduler` may sometimes launch a duplicate compaction
task for an interval that has just been compacted.
This may happen as follows:
- Scheduler launches a compaction task for an uncompacted interval.
- While the compaction task is running, the `CompactionStatusTracker` does not consider
this interval as compactible and returns the `CompactionStatus` as `SKIPPED` for it.
- As soon as the compaction task finishes, the `CompactionStatusTracker` starts considering
the interval eligible for compaction again.
- This interval remains eligible for compaction until the newly published segments are polled
from the database.
- Once the new segments have been polled, the `CompactionStatus` of the interval changes
to `COMPLETE`.
Change
--------
- Keep track of the `snapshotTime` in `DataSourcesSnapshot`. This time represents the start of the poll.
- Use the `snapshotTime` to determine if a poll has happened after a compaction task completed.
- If not, then skip the interval to avoid launching duplicate tasks.
- For tests, use a future `snapshotTime` to ensure that compaction is always triggered.
Fix the logic for usage of segment descriptors from queries in SinkQuerySegmentWalker when there are upgraded segments as a result of concurrent replace.
Concurrent append and replace:
With the introduction of concurrent append and replace, for a given interval:
The same sink can correspond to a base segment V0_x0, and have multiple mappings to higher versions with distinct partition numbers such as V1_x1.... Vn_xn.
The initial segment allocation can happen on version V0, but there can be several allocations during the lifecycle of a task which can have different versions spanning from V0 to Vn.
Changes:
Maintain a new timeline of (An overshadowable holding a SegmentDescriptor)
Every segment allocation of version upgrade adds the latest segment descriptor to this timeline.
Iterate this timeline instead of the sinkTimeline to get the segment descriptors in getQueryRunnerForIntervals
Also maintain a mapping of the upgraded segment to its base segment.
When a sink is needed to process the query, find the base segment corresponding to a given descriptor, and then use the sinkTimeline to find its chunk.
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.
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.
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.
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.
The javadoc for SegmentDescriptor discusses differences between it and
SegmentId, but misses the most important difference: SegmentDescriptor
can have a narrower interval than the segment being referenced.
* from start to step 3 of Ingest data using Theta sketche
* updated upto "Query the Theta sketch column"
* fixed sentence
* another typo
* using sql ingestion instead of batch-sql
* waiting for explanations on DS_THETA
* Revert "using sql ingestion instead of batch-sql"
This reverts commit b95fcb9b32.
* Revert "using sql ingestion instead of batch-sql"
This reverts commit b95fcb9b32.
* just copy and pasting to where I was
* updated tutorial
* fixing images, and removing unused
* slightly updating explanatio
* Update docs/tutorials/tutorial-sketches-theta.md
* Apply suggestions from code review
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
* addressing comments in review
* made filter clause consitent with other instances
* Apply suggestions from code review
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
---------
Co-authored-by: Benedict Jin <asdf2014@apache.org>
Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
Fixes#16587
Streaming ingestion tasks operate by allocating segments before ingesting rows.
These allocations happen across replicas which may send different requests but
must get the same segment id for a given (datasource, interval, version, sequenceName)
across replicas.
This patch fixes the bug by ignoring the previousSegmentId when skipLineageCheck is true.
Refactors a few things.
- Adds SemanticUtils maps to columns.
- Add some addAll functions to reduce duplication, and for future reuse.
- Refactor VariantColumnAndIndexSupplier to only take a SmooshedFileMapper instead.
- Refactor LongColumnSerializerV2 to have separate functions for serializing a value and null.
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.
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
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.
This fixes a race where, if there is no output at all, setAllDoneIfPossible
could be called twice (once when the output partitions future resolves, and
once when the batcher finishes). If the calls happen in that order, it would
try to create nil output channels both times, resulting in a "Channel already set"
error.