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.
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.
The current Supervisor interface is primarily focused on streaming use cases. However, as we introduce supervisors for non-streaming use cases, such as the recently added CompactionSupervisor (and the upcoming BatchSupervisor), certain operations like resetting offsets, checkpointing, task group handoff, etc., are not really applicable to non-streaming use cases.
So the methods are split between:
1. Supervisor: common methods that are applicable to both streaming and non-streaming use cases
2. SeekableStreamSupervisor: Supervisor + streaming-only operations. The existing streaming-only overrides exist along with the new abstract method public abstract LagStats computeLagStats(), for which custom implementations already exist in the concrete types
This PR is primarily a refactoring change with minimal functional adjustments (e.g., throwing an exception in a few places in SupervisorManager when the supervisor isn't the expected SeekableStreamSupervisor type).
Text-based input formats like csv and tsv currently parse inputs only as strings, following the RFC4180Parser spec).
To workaround this, the web-console and other tools need to further inspect the sample data returned to sample data returned by the Druid sampler API to parse them as numbers.
This patch introduces a new optional config, tryParseNumbers, for the csv and tsv input formats. If enabled, any numbers present in the input will be parsed in the following manner -- long data type for integer types and double for floating-point numbers, and if parsing fails for whatever reason, the input is treated as a string. By default, this configuration is set to false, so numeric strings will be treated as strings.
Revives #14024 and additionally supports,
Native queries
gRPC health check endpoint
This PR doesn't have the shaded module for packaging gRPC and Guava libraries since grpc-query module uses the same Guava version as that of Druid.
The response is gRPC-specific. It provides the result schema along with the results as a binary "blob". Results can be in CSV, JSON array lines or as an array of Protobuf objects. If using Protobuf, the corresponding class must be installed along with the gRPC query extension so it is available to the Broker at runtime.
* MSQ: Rework memory management.
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.
* Remove unused import.
* Fix errorprone annotation.
* Fixes for javadocs and inspections.
* Additional test coverage.
* Fix test.
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.
* 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
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.
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
* 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.
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`.
Handle the following Delta complex types:
a. StructType as JSON
b. ArrayType as Java list
c. MapType as Java map
Generate and add a new Delta table complex-types-table that contains the above complex types for testing.
Update the tests to include a parameterized test with complex-types-table, with the expectations defined in ComplexTypesDeltaTable.java.
* 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 PR adds indexer-level task metrics-
"indexer/task/failed/count"
"indexer/task/success/count"
the current "worker/task/completed/count" metric shows all the tasks completed irrespective of success or failure status so these metrics would help us get more visibility into the status of the completed tasks
* Coerce COMPLEX to number in numeric aggregators.
PR #15371 eliminated ObjectColumnSelector's built-in implementations of
numeric methods, which had been marked deprecated.
However, some complex types, like SpectatorHistogram, can be successfully coerced
to number. The documentation for spectator histograms encourages taking advantage of
this by aggregating complex columns with doubleSum and longSum. Currently, this
doesn't work properly for IncrementalIndex, where the behavior relied on those
deprecated ObjectColumnSelector methods.
This patch fixes the behavior by making two changes:
1) SimpleXYZAggregatorFactory (XYZ = type; base class for simple numeric aggregators;
all of these extend NullableNumericAggregatorFactory) use getObject for STRING
and COMPLEX. Previously, getObject was only used for STRING.
2) NullableNumericAggregatorFactory (base class for simple numeric aggregators)
has a new protected method "useGetObject". This allows the base class to
correctly check for null (using getObject or isNull).
The patch also adds a test for SpectatorHistogram + doubleSum + IncrementalIndex.
* Fix tests.
* Remove the special ColumnValueSelector.
* Add test.
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
In case of few aggregators for example BloomSqlAggregator, BaseVarianceSqlAggregator etc, the aggName is being updated from a0 to a0:agg, breaching the contract as we would expect the aggName as the name which is passed. This is causing a mismatch while creating a column accessor.
This commit aims to correct those violating sql aggregators.
index_realtime tasks were removed from the documentation in #13107. Even
at that time, they weren't really documented per se— just mentioned. They
existed solely to support Tranquility, which is an obsolete ingestion
method that predates migration of Druid to ASF and is no longer being
maintained. Tranquility docs were also de-linked from the sidebars and
the other doc pages in #11134. Only a stub remains, so people with
links to the page can see that it's no longer recommended.
index_realtime_appenderator tasks existed in the code base, but were
never documented, nor as far as I am aware were they used for any purpose.
This patch removes both task types completely, as well as removes all
supporting code that was otherwise unused. It also updates the stub
doc for Tranquility to be firmer that it is not compatible. (Previously,
the stub doc said it wasn't recommended, and pointed out that it is
built against an ancient 0.9.2 version of Druid.)
ITUnionQueryTest has been migrated to the new integration tests framework and updated to use Kafka ingestion.
Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
Upstream release: https://github.com/delta-io/delta/releases/tag/v3.2.0
- Upgrade kernel dependency to 3.2.0
- Notable breaking changes introduced in upstream that affects the Druid extension:
- Rename TableClient -> Engine
- Rename DefaultTableClient -> DefaultEngine
- Exceptions moved to a separate package
- Table.getPath() doesn't throw TableNotFoundException. Instead the exception is thrown
when getting snapshot info from the Table object
* enable quidem uri support for `druidtest:///?ComponentSupplier=Nested` and similar
* changes the way `SqlTestFrameworkConfig` is being applied; all options will have their own annotation (its kinda impossible to detect that an annotation has a set value or its the default)
* enables hierarchical processing of config annotation (was needed to enable class level supplier annotation)
* moves uri processing related string2config stuff into `SqlTestFrameworkConfig`
* Delta Lake support for filters.
* Updates
* cleanup comments
* Docs
* Remmove Enclosed runner
* Rename
* Cleanup test
* Serde test for the Delta input source and fix jackson annotation.
* Updates and docs.
* Update error messages to be clearer
* Fixes
* Handle NumberFormatException to provide a nicer error message.
* Apply suggestions from code review
Co-authored-by: 317brian <53799971+317brian@users.noreply.github.com>
* Doc fixes based on feedback
* Yes -> yes in docs; reword slightly.
* Update docs/ingestion/input-sources.md
Co-authored-by: Laksh Singla <lakshsingla@gmail.com>
* Update docs/ingestion/input-sources.md
Co-authored-by: Laksh Singla <lakshsingla@gmail.com>
* Documentation, javadoc and more updates.
* Not with an or expression end-to-end test.
* Break up =, >, >=, <, <= into its own types instead of sub-classing.
---------
Co-authored-by: 317brian <53799971+317brian@users.noreply.github.com>
Co-authored-by: Laksh Singla <lakshsingla@gmail.com>
Changes:
1) Check for handoff of upgraded realtime segments.
2) Drop sink only when all associated realtime segments have been abandoned.
3) Delete pending segments upon commit to prevent unnecessary upgrades and
partition space exhaustion when a concurrent replace happens. This also prevents
potential data duplication.
4) Register pending segment upgrade only on those tasks to which the segment is associated.