* MSQ: Add CPU and thread usage counters.
The main change adds "cpu" and "wall" counters. The "cpu" counter measures
CPU time (using JvmUtils.getCurrentThreadCpuTime) taken up by processors
in processing threads. The "wall" counter measures the amount of wall time
taken up by processors in those same processing threads. Both counters are
broken down by type of processor.
This patch also includes changes to support adding new counters. Due to an
oversight in the original design, older deserializers are not forwards-compatible;
they throw errors when encountering an unknown counter type. To manage this,
the following changes are made:
1) The defaultImpl NilQueryCounterSnapshot is added to QueryCounterSnapshot's
deserialization configuration. This means that any unrecognized counter types
will be read as "nil" by deserializers. Going forward, once all servers are
on the latest code, this is enough to enable easily adding new counters.
2) A new context parameter "includeAllCounters" is added, which defaults to "false".
When this parameter is set "false", only legacy counters are included. When set
to "true", all counters are included. This is currently undocumented. In a future
version, we should set the default to "true", and at that time, include a release
note that people updating from versions prior to Druid 31 should set this to
"false" until their upgrade is complete.
* Style, coverage.
* Fix.
Changes:
- Simplify exception handling in `CryptoService` by just catching a `Exception`
- Throw a `DruidException` as the exception is user facing
- Log the exception for easier debugging
- Add a test to verify thrown exception
Currently, if we have a query with window function having PARTITION BY xyz, and we have a million unique values for xyz each having 1 row, we'd end up creating a million individual RACs for processing, each having a single row. This is unnecessary, and we can batch the PARTITION BY keys together for processing, and process them only when we can't batch further rows to adhere to maxRowsMaterialized config.
The previous iteration of this PR was simplifying WindowOperatorQueryFrameProcessor to run all operators on all the rows instead of creating smaller RACs per partition by key. That approach was discarded in favor of the batching approach, and the details are summarized here: #16823 (comment).
changes:
* Adds new `CompressedComplexColumn`, `CompressedComplexColumnSerializer`, `CompressedComplexColumnSupplier` based on `CompressedVariableSizedBlobColumn` used by JSON columns
* Adds `IndexSpec.complexMetricCompression` which can be used to specify compression for the generic compressed complex column. Defaults to uncompressed because compressed columns are not backwards compatible.
* Adds new definition of `ComplexMetricSerde.getSerializer` which accepts an `IndexSpec` argument when creating a serializer. The old signature has been marked `@Deprecated` and has a default implementation that returns `null`, but it will be used by the default implementation of the new version if it is implemented to return a non-null value. The default implementation of the new method will use a `CompressedComplexColumnSerializer` if `IndexSpec.complexMetricCompression` is not null/none/uncompressed, or will use `LargeColumnSupportedComplexColumnSerializer` otherwise.
* Removed all duplicate generic implementations of `ComplexMetricSerde.getSerializer` and `ComplexMetricSerde.deserializeColumn` into default implementations `ComplexMetricSerde` instead of being copied all over the place. The default implementation of `deserializeColumn` will check if the first byte indicates that the new compression was used, otherwise will use the `GenericIndexed` based supplier.
* Complex columns with custom serializers/deserializers are unaffected and may continue doing whatever it is they do, either with specialized compression or whatever else, this new stuff is just to provide generic implementations built around `ObjectStrategy`.
* add ObjectStrategy.readRetainsBufferReference so CompressedComplexColumn only copies on read if required
* add copyValueOnRead flag down to CompressedBlockReader to avoid buffer duplicate if the value needs copied anyway
* MSQ: Fix validation of time position in collations.
It is possible for the collation to refer to a field that isn't mapped,
such as when the DML includes "CLUSTERED BY some_function(some_field)".
In this case, the collation refers to a projected column that is not
part of the field mappings. Prior to this patch, that would lead to an
out of bounds list access on fieldMappings.
This patch fixes the problem by identifying the position of __time in
the fieldMappings first, rather than retrieving each collation field
from fieldMappings.
Fixes a bug introduced in #16849.
* Fix test. Better warning message.
* Place __time in signatures according to sort order.
Updates a variety of places to put __time in row signatures according
to its position in the sort order, rather than always first, including:
- InputSourceSampler.
- ScanQueryEngine (in the default signature when "columns" is empty).
- Various StorageAdapters, which also have the effect of reordering
the column order in segmentMetadata queries, and therefore in SQL
schemas as well.
Follow-up to #16849.
* Fix compilation.
* Additional fixes.
* Fix.
* Fix style.
* Omit nonexistent columns from the row signature.
* Fix tests.
* Segments primarily sorted by non-time columns.
Currently, segments are always sorted by __time, followed by the sort
order provided by the user via dimensionsSpec or CLUSTERED BY. Sorting
by __time enables efficient execution of queries involving time-ordering
or granularity. Time-ordering is a simple matter of reading the rows in
stored order, and granular cursors can be generated in streaming fashion.
However, for various workloads, it's better for storage footprint and
query performance to sort by arbitrary orders that do not start with __time.
With this patch, users can sort segments by such orders.
For spec-based ingestion, users add "useExplicitSegmentSortOrder: true" to
dimensionsSpec. The "dimensions" list determines the sort order. To
define a sort order that includes "__time", users explicitly
include a dimension named "__time".
For SQL-based ingestion, users set the context parameter
"useExplicitSegmentSortOrder: true". The CLUSTERED BY clause is then
used as the explicit segment sort order.
In both cases, when the new "useExplicitSegmentSortOrder" parameter is
false (the default), __time is implicitly prepended to the sort order,
as it always was prior to this patch.
The new parameter is experimental for two main reasons. First, such
segments can cause errors when loaded by older servers, due to violating
their expectations that timestamps are always monotonically increasing.
Second, even on newer servers, not all queries can run on non-time-sorted
segments. Scan queries involving time-ordering and any query involving
granularity will not run. (To partially mitigate this, a currently-undocumented
SQL feature "sqlUseGranularity" is provided. When set to false the SQL planner
avoids using "granularity".)
Changes on the write path:
1) DimensionsSpec can now optionally contain a __time dimension, which
controls the placement of __time in the sort order. If not present,
__time is considered to be first in the sort order, as it has always
been.
2) IncrementalIndex and IndexMerger are updated to sort facts more
flexibly; not always by time first.
3) Metadata (stored in metadata.drd) gains a "sortOrder" field.
4) MSQ can generate range-based shard specs even when not all columns are
singly-valued strings. It merely stops accepting new clustering key
fields when it encounters the first one that isn't a singly-valued
string. This is useful because it enables range shard specs on
"someDim" to be created for clauses like "CLUSTERED BY someDim, __time".
Changes on the read path:
1) Add StorageAdapter#getSortOrder so query engines can tell how a
segment is sorted.
2) Update QueryableIndexStorageAdapter, IncrementalIndexStorageAdapter,
and VectorCursorGranularizer to throw errors when using granularities
on non-time-ordered segments.
3) Update ScanQueryEngine to throw an error when using the time-ordering
"order" parameter on non-time-ordered segments.
4) Update TimeBoundaryQueryRunnerFactory to perform a segment scan when
running on a non-time-ordered segment.
5) Add "sqlUseGranularity" context parameter that causes the SQL planner
to avoid using granularities other than ALL.
Other changes:
1) Rename DimensionsSpec "hasCustomDimensions" to "hasFixedDimensions"
and change the meaning subtly: it now returns true if the DimensionsSpec
represents an unchanging list of dimensions, or false if there is
some discovery happening. This is what call sites had expected anyway.
* Fixups from CI.
* Fixes.
* Fix missing arg.
* Additional changes.
* Fix logic.
* Fixes.
* Fix test.
* Adjust test.
* Remove throws.
* Fix styles.
* Fix javadocs.
* Cleanup.
* Smoother handling of null ordering.
* Fix tests.
* Missed a spot on the merge.
* Fixups.
* Avoid needless Filters.and.
* Add timeBoundaryInspector to test.
* Fix tests.
* Fix FrameStorageAdapterTest.
* Fix various tests.
* Use forceSegmentSortByTime instead of useExplicitSegmentSortOrder.
* Pom fix.
* Fix doc.
Previously, SeekableStreamIndexTaskRunner set ingestion state to
COMPLETED when it finished reading data from Kafka. This is incorrect.
After the changes in this patch, the transitions go:
1) The task stays in BUILD_SEGMENTS after it finishes reading from Kafka,
while it is building its final set of segments to publish.
2) The task transitions to SEGMENT_AVAILABILITY_WAIT after publishing,
while waiting for handoff.
3) The task transitions to COMPLETED immediately before exiting, when
truly done.
A follow-up PR for #16864. Just renames dimensionToSchemaMap to dimensionSchemas and always overrides ARRAY_INGEST_MODE context value to array for MSQ compaction.
changes:
* Added `CursorBuildSpec` which captures all of the 'interesting' stuff that goes into producing a cursor as a replacement for the method arguments of `CursorFactory.canVectorize`, `CursorFactory.makeCursor`, and `CursorFactory.makeVectorCursor`
* added new interface `CursorHolder` and new interface `CursorHolderFactory` as a replacement for `CursorFactory`, with method `makeCursorHolder`, which takes a `CursorBuildSpec` as an argument and replaces `CursorFactory.canVectorize`, `CursorFactory.makeCursor`, and `CursorFactory.makeVectorCursor`
* `CursorFactory.makeCursors` previously returned a `Sequence<Cursor>` corresponding to the query granularity buckets, with a separate `Cursor` per bucket. `CursorHolder.asCursor` instead returns a single `Cursor` (equivalent to 'ALL' granularity), and a new `CursorGranularizer` has been added for query engines to iterate over the cursor and divide into granularity buckets. This makes the non-vectorized engine behave the same way as the vectorized query engine (with its `VectorCursorGranularizer`), and simplifies a lot of stuff that has to read segments particularly if it does not care about bucketing the results into granularities.
* Deprecated `CursorFactory`, `CursorFactory.canVectorize`, `CursorFactory.makeCursors`, and `CursorFactory.makeVectorCursor`
* updated all `StorageAdapter` implementations to implement `makeCursorHolder`, transitioned direct `CursorFactory` implementations to instead implement `CursorMakerFactory`. `StorageAdapter` being a `CursorMakerFactory` is intended to be a transitional thing, ideally will not be released in favor of moving `CursorMakerFactory` to be fetched directly from `Segment`, however this PR was already large enough so this will be done in a follow-up.
* updated all query engines to use `makeCursorHolder`, granularity based engines to use `CursorGranularizer`.
Upgrade/Downgrade between any version till or before Druid 30 where the newer version runs a worker task, while the older version runs a controller task can fail. The patch removes that verification check till its safe to add it back.
Fixes#13936
In cases where a supervisor is idle and the overlord is restarted for some reason, the supervisor would
start spinning tasks again. In clusters where there are many low throughput streams, this would spike
the task count unnecessarily.
This commit compares the latest stream offset with the ones in metadata during the startup of supervisor
and sets it to idle state if they match.
This PR adds checks for verification of DataSourceCompactionConfig and CompactionTask with msq engine to ensure:
each aggregator in metricsSpec is idempotent
metricsSpec is non-null when rollup is set to true
Unit tests and existing compaction ITs have been updated accordingly.
This PR fixes query correctness issues for MSQ window functions when using more than 1 worker (that is, maxNumTasks > 2).
Currently, we were keeping the shuffle spec of the previous stage when we didn't have any partition columns for window stage. This PR changes it to override the shuffle spec of the previous stage to MixShuffleSpec (if we have a window function with empty over clause) so that the window stage gets a single partition to work on.
A test has been added for a query which returned incorrect results prior to this change when using more than 1 workers.
* enables to launch a fake broker based on test resources (druidtest uri)
* could record queries into new testfiles during usage
* instead of re-purpose Calcite's Hook migrates to use DruidHook which we can add further keys
* added a quidem-ut module which could be the place for tests which could iteract with modules/etc
This patch introduces an optional cluster configuration, druid.indexing.formats.stringMultiValueHandlingMode, allowing operators to override the default mode SORTED_SET for string dimensions. The possible values for the config are SORTED_SET, SORTED_ARRAY, or ARRAY (SORTED_SET is the default). Case insensitive values are allowed.
While this cluster property allows users to manage the multi-value handling mode for string dimension types, it's recommended to migrate to using real array types instead of MVDs.
This fixes a long-standing issue where compaction will honor the configured cluster wide property instead of rewriting it as the default SORTED_ARRAY always, even if the data was originally ingested with ARRAY or SORTED_SET.
* SQL syntax error should target USER persona
* * revert change to queryHandler and related tests, based on review comments
* * add test
* Introduce KinesisRecordEntity to support Kinesis headers in InputFormats
* * add kinesisInputFormat and Reader, and tests
* * bind KinesisInputFormat class to module
* * improve test coverage
* * remove references to kafka
* * resolve review comments
* * remove comment
* * fix grammer of comment
* * fix comment again
* * fix comment again
* * more review comments
* * add partitionKey
* * add check for same timestamp and partitionKey column name
* * fix intellij inspection
If the optional query parameter detail is supplied, then the response also includes the following:
* A stages object that summarizes information about the different stages being used for query execution, such as stage number, phase, start time, duration, input and output information, processing methods, and partitioning.
* A counters object that provides details on the rows, bytes, and files processed at various stages for each worker across different channels, along with sort progress.
* A warnings object that provides details about any warnings.
* MSQ worker: Support in-memory shuffles.
This patch is a follow-up to #16168, adding worker-side support for
in-memory shuffles. Changes include:
1) Worker-side code now respects the same context parameter "maxConcurrentStages"
that was added to the controller in #16168. The parameter remains undocumented
for now, to give us a chance to more fully develop and test this functionality.
1) WorkerImpl is broken up into WorkerImpl, RunWorkOrder, and RunWorkOrderListener
to improve readability.
2) WorkerImpl has a new StageOutputHolder + StageOutputReader concept, which
abstract over memory-based or file-based stage results.
3) RunWorkOrder is updated to create in-memory stage output channels when
instructed to.
4) ControllerResource is updated to add /doneReadingInput/, so the controller
can tell when workers that sort, but do not gather statistics, are done reading
their inputs.
5) WorkerMemoryParameters is updated to consider maxConcurrentStages.
Additionally, WorkerChatHandler is split into WorkerResource, so as to match
ControllerChatHandler and ControllerResource.
* Updates for static checks, test coverage.
* Fixes.
* Remove exception.
* Changes from review.
* Address static check.
* Changes from review.
* Improvements to docs and method names.
* Update comments, add test.
* Additional javadocs.
* Fix throws.
* Fix worker stopping in tests.
* Fix stuck test.
Follow-up to #16291, this commit enables a subset of existing native compaction ITs on the MSQ engine.
In the process, the following changes have been introduced in the MSQ compaction flow:
- Populate `metricsSpec` in `CompactionState` from `querySpec` in `MSQControllerTask` instead of `dataSchema`
- Add check for pre-rolled-up segments having `AggregatorFactory` with different input and output column names
- Fix passing missing cluster-by clause in scan queries
- Add annotation of `CompactionState` to tombstone segments
Changes:
- Add API `/druid/coordinator/v1/config/compaction/global` to update cluster level compaction config
- Add class `CompactionConfigUpdateRequest`
- Fix bug in `CoordinatorCompactionConfig` which caused compaction engine to not be persisted.
Use json field name `engine` instead of `compactionEngine` because JSON field names must align
with the getter name.
- Update MSQ validation error messages
- Complete overhaul of `CoordinatorCompactionConfigResourceTest` to remove unnecessary mocking
and add more meaningful tests.
- Add `TuningConfigBuilder` to easily build tuning configs for tests.
- Add `DatasourceCompactionConfigBuilder`
Changes the WindowFrame internals / representation a bit; introduces dedicated frametypes for rows and groups which corresponds to the implemented processing methods
* MSQ window functions: Revamp logic to create separate window stages when empty over() clause is present
* Fix tests
* Revert changes of creating separate stages for empty over clause
* Address review comments
changes:
* removes `druid.indexer.task.batchProcessingMode` in favor of always using `CLOSED_SEGMENT_SINKS` which uses `BatchAppenderator`. This was intended to become the default for native batch, but that was missed so `CLOSED_SEGMENTS` was the default (using `AppenderatorImpl`), however MSQ has been exclusively using `BatchAppenderator` with no problems so it seems safe to just roll it out as the only option for batch ingestion everywhere.
* with `batchProcessingMode` gone, there is no use for `AppenderatorImpl` so it has been removed
* implify `Appenderator` construction since there are only separate stream and batch versions now
* simplify tests since `batchProcessingMode` is gone
This PR aims to check if the complex column being queried aligns with the supported types in the aggregator and aggregator factories, and throws a user-friendly error message if they don't.
* Throw exception if DISTINCT used with window functions aggregate call
* Improve error message when unsupported aggregations are used with window functions
Fixes#16766
Change log level from INFO to DEBUG when processing an empty user map
during polling. An empty user map is a normal situation for some
authenticators (e.g. LDAP) and polling is frequent (1 minute by
default.)
changes:
* removed `Firehose` and `FirehoseFactory` and remaining implementations which were mostly no longer used after #16602
* Moved `IngestSegmentFirehose` which was still used internally by Hadoop ingestion to `DatasourceRecordReader.SegmentReader`
* Rename `SQLFirehoseFactoryDatabaseConnector` to `SQLInputSourceDatabaseConnector` and similar renames for sub-classes
* Moved anything remaining in a 'firehose' package somewhere else
* Clean up docs on firehose stuff
* #16717 defer provider instatiation
* add license header
* fix style, ignore new class in jacoco as it is still initialization code
---------
Co-authored-by: Alberto Lago Alvarado <albl@sitecore.net>
* When an ArrayList RAC creates a child RAC, the start and end offsets need to have the offset of parent's start offset
* Defaults the 2nd window bound to CURRENT ROW when only a single bound is specified
* Removes the windowingStrictValidation warning and throws a hard exception when Order By alongside RANGE clause is not provided with UNBOUNDED or CURRENT ROW as both bounds
Changes:
- No functional change
- Add class `TuningConfigBuilder` to build `IndexTuningConfig`, `CompactionTuningConfig`
- Remove old class `ParallelIndexTestingFactory.TuningConfigBuilder`
- Remove some unused fields and methods
Changes
- No functional change
- Remove unused method `IndexTuningConfig.withPartitionsSpec()`
- Remove unused method `ParallelIndexTuningConfig.withPartitionsSpec()`
- Remove redundant method `CompactTask.emitIngestionModeMetrics()`
- Remove Clock argument from `CompactionTask.createDataSchemasForInterval()` as it was only needed
for one test which was just verifying the value passed by the test itself. The code now uses a `Stopwatch`
instead and test simply verifies that the metric has been emitted.
- Other minor cleanup changes
Better fallback strategy when the broker is unable to materialize the subquery's results as frames for estimating the bytes:
a. We don't touch the subquery sequence till we know that we can materialize the result as frames