* more consistent expression error messages
* review stuff
* add NamedFunction for Function, ApplyFunction, and ExprMacro to share common stuff
* fixes
* add expression transform name to transformer failure, better parse_json error messaging
* SQL: Morph QueryMakerFactory into SqlEngine.
Groundwork for introducing an indexing-service-task-based SQL engine
under the umbrella of #12262. Also includes some other changes related
to improving error behavior.
Main changes:
1) Elevate the QueryMakerFactory interface (an extension point that allows
customization of how queries are made) into SqlEngine. SQL engines
can influence planner behavior through EngineFeatures, and can fully
control the mechanics of query execution using QueryMakers.
2) Remove the server-wide QueryMakerFactory choice, in favor of the choice
being made by the SQL entrypoint. The indexing-service-task-based
SQL engine would be associated with its own entrypoint, like
/druid/v2/sql/task.
Other changes:
1) Adjust DruidPlanner to try either DRUID or BINDABLE convention based
on analysis of the planned rels; never try both. In particular, we
no longer try BINDABLE when DRUID fails. This simplifies the logic
and improves error messages.
2) Adjust error message "Cannot build plan for query" to omit the SQL
query text. Useful because the text can be quite long, which makes it
easy to miss the text about the problem.
3) Add a feature to block context parameters used internally by the SQL
planner from being supplied by end users.
4) Add a feature to enable adding row signature to the context for
Scan queries. This is useful in building the task-based engine.
5) Add saffron.properties file that turns off sets and graphviz dumps
in "cannot plan" errors. Significantly reduces log spam on the Broker.
* Fixes from CI.
* Changes from review.
* Can vectorize, now that join-to-filter is on by default.
* Checkstyle! And variable renames!
* Remove throws from test.
* Refactor SqlLifecycle into statement classes
Create direct & prepared statements
Remove redundant exceptions from tests
Tidy up Calcite query tests
Make PlannerConfig more testable
* Build fixes
* Added builder to SqlQueryPlus
* Moved Calcites system properties to saffron.properties
* Build fix
* Resolve merge conflict
* Fix IntelliJ inspection issue
* Revisions from reviews
Backed out a revision to Calcite tests that didn't work out as planned
* Build fix
* Fixed spelling errors
* Fixed failed test
Prepare now enforces security; before it did not.
* Rebase and fix IntelliJ inspections issue
* Clean up exception handling
* Fix handling of JDBC auth errors
* Build fix
* More tweaks to security messages
* Introduce defaultOnDiskStorage config for groupBy
* add debug log to groupby query config
* Apply config change suggestion from review
* Remove accidental new lines
* update default value of new default disk storage config
* update debug log to have more descriptive text
* Make maxOnDiskStorage and defaultOnDiskStorage HumanRedadableBytes
* improve test coverage
* Provide default implementation to new default method on advice of reviewer
Refactors the DruidSchema and DruidTable abstractions to prepare for the Druid Catalog.
As we add the catalog, we’ll want to combine physical segment metadata information with “hints” provided by the catalog. This is best done if we tidy up the existing code to more clearly separate responsibilities.
This PR is purely a refactoring move: no functionality changed. There is no difference to user functionality or external APIs. Functionality changes will come later as we add the catalog itself.
DruidSchema
In the present code, DruidSchema does three tasks:
Holds the segment metadata cache
Interfaces with an external schema manager
Acts as a schema to Calcite
This PR splits those responsibilities.
DruidSchema holds the Calcite schema for the druid namespace, combining information fro the segment metadata cache, from the external schema manager and (later) from the catalog.
SegmentMetadataCache holds the segment metadata cache formerly in DruidSchema.
DruidTable
The present DruidTable class is a bit of a kitchen sink: it holds all the various kinds of tables which Druid supports, and uses if-statements to handle behavior that differs between types. Yet, any given DruidTable will handle only one such table type. To more clearly model the actual table types, we split DruidTable into several classes:
DruidTable becomes an abstract base class to hold Druid-specific methods.
DatasourceTable represents a datasource.
ExternalTable represents an external table, such as from EXTERN or (later) from the catalog.
InlineTable represents the internal case in which we attach data directly to a table.
LookupTable represents Druid’s lookup table mechanism.
The new subclasses are more focused: they can be selective about the data they hold and the various predicates since they represent just one table type. This will be important as the catalog information will differ depending on table type and the new structure makes adding that logic cleaner.
DatasourceMetadata
Previously, the DruidSchema segment cache would work with DruidTable objects. With the catalog, we need a layer between the segment metadata and the table as presented to Calcite. To fix this, the new SegmentMetadataCache class uses a new DatasourceMetadata class as its cache entry to hold only the “physical” segment metadata information: it is up to the DruidTable to combine this with the catalog information in a later PR.
More Efficient Table Resolution
Calcite provides a convenient base class for schema objects: AbstractSchema. However, this class is a bit too convenient: all we have to do is provide a map of tables and Calcite does the rest. This means that, to resolve any single datasource, say, foo, we need to cache segment metadata, external schema information, and catalog information for all tables. Just so Calcite can do a map lookup.
There is nothing special about AbstractSchema. We can handle table lookups ourselves. The new AbstractTableSchema does this. In fact, all the rest of Calcite wants is to resolve individual tables by name, and, for commands we don’t use, to provide a list of table names.
DruidSchema now extends AbstractTableSchema. SegmentMetadataCache resolves individual tables (and provides table names.)
DruidSchemaManager
DruidSchemaManager provides a way to specify table schemas externally. In this sense, it is similar to the catalog, but only for datasources. It originally followed the AbstractSchema pattern: it implements provide a map of tables. This PR provides new optional methods for the table lookup and table names operations. The default implementations work the same way that AbstractSchema works: we get the entire map and pick out the information we need. Extensions that use this API should be revised to support the individual operations instead. Druid code no longer calls the original getTables() method.
The PR has one breaking change: since the DruidSchemaManager map is read-only to the rest of Druid, we should return a Map, not a ConcurrentMap.
add NumericRangeIndex interface and BoundFilter support
changes:
* NumericRangeIndex interface, like LexicographicalRangeIndex but for numbers
* BoundFilter now uses NumericRangeIndex if comparator is numeric and there is no extractionFn
* NestedFieldLiteralColumnIndexSupplier.java now supports supplying NumericRangeIndex for single typed numeric nested literal columns
* better faster stronger and (ever so slightly) more understandable
* more tests, fix bug
* fix style
* Preserve column order in DruidSchema, SegmentMetadataQuery.
Instead of putting columns in alphabetical order. This is helpful
because it makes query order better match ingestion order. It also
allows tools, like the reindexing flow in the web console, to more
easily do follow-on ingestions using a column order that matches the
pre-existing column order.
We prefer the order from the latest segments. The logic takes all
columns from the latest segments in the order they appear, then adds
on columns from older segments after those.
* Additional test adjustments.
* Adjust imports.
* Frame format for data transfer and short-term storage.
As we move towards query execution plans that involve more transfer
of data between servers, it's important to have a data format that
provides for doing this more efficiently than the options available to
us today.
This patch adds:
- Columnar frames, which support fast querying.
- Row-based frames, which support fast sorting via memory comparison
and fast whole-row copies via memory copying.
- Frame files, a container format that can be stored on disk or
transferred between servers.
The idea is we should use row-based frames when data is expected to
be sorted, and columnar frames when data is expected to be queried.
The code in this patch is not used in production yet. Therefore, the
patch involves minimal changes outside of the org.apache.druid.frame
package. The main ones are adjustments to SqlBenchmark to add benchmarks
for queries on frames, and the addition of a "forEach" method to Sequence.
* Fixes based on tests, static analysis.
* Additional fixes.
* Skip DS mapping tests on JDK 14+
* Better JDK checking in tests.
* Fix imports.
* Additional comment.
* Adjustments from code review.
* Update test case.
* Poison StupidPool and fix resource leaks
There are various resource leaks from test setup as well as some
corners in query processing. We poison the StupidPool to start failing
tests when the leaks come and fix any issues uncovered from that so
that we can start from a clean baseline.
Unfortunately, because of how poisoning works,
we can only fail future checkouts from the same pool,
which means that there is a natural race between a
leak happening -> GC occurs -> leak detected -> pool poisoned.
This race means that, depending on interleaving of tests,
if the very last time that an object is checked out
from the pool leaks, then it won't get caught.
At some point in the future, something will catch it,
however and from that point on it will be deterministic.
* Remove various things left over from iterations
* Clean up FilterAnalysis and add javadoc on StupidPool
* Revert changes to .idea/misc.xml that accidentally got pushed
* Style and test branches
* Stylistic woes
Often users are submitting queries, and ingestion specs that work only if the relevant extension is not loaded. However, the error is too technical for the users and doesn't suggest them to check for missing extensions. This PR modifies the error message so users can at least check their settings before assuming that the error is because of a bug.
* Adding zstandard compression library
* 1. Took @clintropolis's advice to have ZStandard decompressor use the byte array when the buffers are not direct.
2. Cleaned up checkstyle issues.
* Fixing zstandard version to latest stable version in pom's and updating license files
* Removing zstd from benchmarks and adding to processing (poms)
* fix the intellij inspection issue
* Removing the prefix v for the version in the license check for ztsd
* Fixing license checks
Co-authored-by: Rahul Gidwani <r_gidwani@apple.com>
* Direct UTF-8 access for "in" filters.
Directly related:
1) InDimFilter: Store stored Strings (in ValuesSet) plus sorted UTF-8
ByteBuffers (in valuesUtf8). Use valuesUtf8 whenever possible. If
necessary, the input set is copied into a ValuesSet. Much logic is
simplified, because we always know what type the values set will be.
I think that there won't even be an efficiency loss in most cases.
InDimFilter is most frequently created by deserialization, and this
patch updates the JsonCreator constructor to deserialize
directly into a ValuesSet.
2) Add Utf8ValueSetIndex, which InDimFilter uses to avoid UTF-8 decodes
during index lookups.
3) Add unsigned comparator to ByteBufferUtils and use it in
GenericIndexed.BYTE_BUFFER_STRATEGY. This is important because UTF-8
bytes can be compared as bytes if, and only if, the comparison
is unsigned.
4) Add specialization to GenericIndexed.singleThreaded().indexOf that
avoids needless ByteBuffer allocations.
5) Clarify that objects returned by ColumnIndexSupplier.as are not
thread-safe. DictionaryEncodedStringIndexSupplier now calls
singleThreaded() on all relevant GenericIndexed objects, saving
a ByteBuffer allocation per access.
Also:
1) Fix performance regression in LikeFilter: since #12315, it applied
the suffix matcher to all values in range even for type MATCH_ALL.
2) Add ObjectStrategy.canCompare() method. This fixes LikeFilterBenchmark,
which was broken due to calls to strategy.compare in
GenericIndexed.fromIterable.
* Add like-filter implementation tests.
* Add in-filter implementation tests.
* Add tests, fix issues.
* Fix style.
* Adjustments from review.
* Add ipaddress library as dependency.
* IPv4 functions to use the inet.ipaddr package.
* Remove unused imports.
* Add new function.
* Minor rename.
* Add more unit tests.
* IPv4 address expr utils unit tests and address options.
* Adjust the IPv4Util functions.
* Move the UTs a bit around.
* Javadoc comments.
* Add license info for IPAddress.
* Fix groupId, artifact and version in license.yaml.
* Remove redundant subnet in messages - fixes UT.
* Remove unused commons-net dependency for /processing project.
* Make class and methods public so it can be accessed.
* Add initial version of benchmark
* Add subnetutils package for benchmarks.
* Auto generate ip addresses.
* Add more v4 address representations in setup to avoid bias.
* Use ThreadLocalRandom to avoid forbidden API usage.
* Adjust IPv4AddressBenchmark to adhere to codestyle rules.
* Update ipaddress library to latest 5.3.4
* Add ipaddress package dependency to benchmarks project.
Following up on #12315, which pushed most of the logic of building ImmutableBitmap into BitmapIndex in order to hide the details of how column indexes are implemented from the Filter implementations, this PR totally refashions how Filter consume indexes. The end result, while a rather dramatic reshuffling of the existing code, should be extraordinarily flexible, eventually allowing us to model any type of index we can imagine, and providing the machinery to build the filters that use them, while also allowing for other column implementations to implement the built-in index types to provide adapters to make use indexing in the current set filters that Druid provides.
* DimensionRangeShardSpec speed boost.
Calling isEmpty() and equals() on RangeSets is expensive, because these
fall back on default implementations that call size(). And size() is
_also_ a default implementation that iterates the entire collection.
* Fix and test from code review.
* JvmMonitor: Handle more generation and collector scenarios.
ZGC on Java 11 only has a generation 1 (there is no 0). This causes
a NullPointerException when trying to extract the spacesCount for
generation 0. In addition, ZGC on Java 15 has a collector number 2
but no spaces in generation 2, which breaks the assumption that
collectors always have same-numbered spaces.
This patch adjusts things to be more robust, enabling the JvmMonitor
to work properly for ZGC on both Java 11 and 15.
* Test adjustments.
* Improve surefire arglines.
* Need a placeholder
* Vectorizing Latest aggregator Part 1
* Updating benchmark tests
* Changing appropriate logic for vectors for null handling
* Introducing an abstract class and moving the commonalities there
* Adding vectorization for StringLast aggregator (initial version)
* Updated bufferized version of numeric aggregators
* Adding some javadocs
* Making sure this PR vectorizes numeric latest agg only
* Adding another benchmarking test
* Fixing intellij inspections
* Adding tests for double
* Adding test cases for long and float
* Updating testcases
* Checkstyle oops..
* One tiny change in test case
* Fixing spotbug and rhs not being used
Currently, the CNF conversion of a filter is unbounded, which means that it can create as many filters as possible thereby also leading to OOMs in historical heap. We should throw an error or disable CNF conversion if the filter count starts getting out of hand. There are ways to do CNF conversion with linear increase in filters as well but that has been left out of the scope of this change since those algorithms add new variables in the predicate - which can be contentious.
* perf: indexing: Introduce a bulk getValuesInto function to read values in bulk
If large number of values are required from DimensionDictionary
during indexing, fetch them all in a single lock/unlock instead of
lock/unlock each individual item.
* refactor: rename key to keys in function args
* fix: check explicitly that argument length on arrays match
* refactor: getValuesInto renamed to getValues, now creates and returns a new T[] rather than filling
Fixes#12022
### Description
The current implementations of memory estimation in `OnHeapIncrementalIndex` and `StringDimensionIndexer` tend to over-estimate which leads to more persistence cycles than necessary.
This PR replaces the max estimation mechanism with getting the incremental memory used by the aggregator or indexer at each invocation of `aggregate` or `encode` respectively.
### Changes
- Add new flag `useMaxMemoryEstimates` in the task context. This overrides the same flag in DefaultTaskConfig i.e. `druid.indexer.task.default.context` map
- Add method `AggregatorFactory.factorizeWithSize()` that returns an `AggregatorAndSize` which contains
the aggregator instance and the estimated initial size of the aggregator
- Add method `Aggregator.aggregateWithSize()` which returns the incremental memory used by this aggregation step
- Update the method `DimensionIndexer.processRowValsToKeyComponent()` to return the encoded key component as well as its effective size in bytes
- Update `OnHeapIncrementalIndex` to use the new estimations only if `useMaxMemoryEstimates = false`
DruidSchema consists of a concurrent HashMap of DataSource -> Segement -> AvailableSegmentMetadata. AvailableSegmentMetadata contains RowSignature of the segment, and for each segment, a new object is getting created. RowSignature is an immutable class, and hence it can be interned, and this can lead to huge savings of memory being used in broker, since a lot of the segments of a table would potentially have same RowSignature.
* clean up the balancing code around the batched vs deprecated way of sampling segments to balance
* fix docs, clarify comments, add deprecated annotations to legacy code
* remove unused variable
* update dynamic config dialog in console to state percentOfSegmentsToConsiderPerMove deprecated
* fix dynamic config text for percentOfSegmentsToConsiderPerMove
* run prettier to cleanup coordinator-dynamic-config.tsx changes
* update jest snapshot
* update documentation per review feedback
changes:
* adds new config, druid.expressions.useStrictBooleans which make longs the official boolean type of all expressions
* vectorize logical operators and boolean functions, some only if useStrictBooleans is true
* Code cleanup from query profile project
* Fix spelling errors
* Fix Javadoc formatting
* Abstract out repeated test code
* Reuse constants in place of some string literals
* Fix up some parameterized types
* Reduce warnings reported by Eclipse
* Reverted change due to lack of tests
* SQL INSERT planner support.
The main changes are:
1) DruidPlanner is able to validate and authorize INSERT queries. They
require WRITE permission on the target datasource.
2) QueryMaker is now an interface, and there is a QueryMakerFactory that
creates instances of it. There is only one production implementation
of each (NativeQueryMaker and NativeQueryMakerFactory), which
together behave the same way as the former QueryMaker class. But this
opens the door to executing queries in ways other than the Druid
query stack, and is used by unit tests (CalciteInsertDmlTest) to
test the INSERT planning functionality.
3) Adds an EXTERN table macro that allows references external data using
InputSource and InputFormat from Druid's batch ingestion API. This is
not exposed in production yet, but is used by unit tests.
4) Adds a QueryFeature concept that enables the planner to change its
behavior slightly depending on the capabilities of the execution
system.
5) Adds an "AuthorizableOperator" concept that enables SqlOperators
to require additional permissions. This is used by the EXTERN table
macro.
Related odds and ends:
- Add equals, hashCode, toString methods to InlineInputSource. Aids in
the "from external" tests in CalciteInsertDmlTest.
- Add JSON-serializability to RowSignature.
- Move the SQL string inside PlannerContext so it is "baked into" the
planner when the planner is created. Cleans up the code a bit, since
in practice, the same query is passed in every time to the
same planner anyway.
* Fix up calls to CalciteTests.createMockQueryLifecycleFactory.
* Fix checkstyle issues.
* Adjustments for CI.
* Adjust DruidAvaticaHandlerTest for stricter test authorizations.
* add impl
* fix checkstyle
* add test
* add test
* add unit tests
* fix unit tests
* fix unit tests
* fix unit tests
* add IT
* add IT
* add comments
* fix spelling
* Remove OffheapIncrementalIndex and clarify aggregator thread-safety needs.
This patch does the following:
- Removes OffheapIncrementalIndex.
- Clarifies that Aggregators are required to be thread safe.
- Clarifies that BufferAggregators and VectorAggregators are not
required to be thread safe.
- Removes thread safety code from some DataSketches aggregators that
had it. (Not all of them did, and that's OK, because it wasn't necessary
anyway.)
- Makes enabling "useOffheap" with groupBy v1 an error.
Rationale for removing the offheap incremental index:
- It is only used in one rare scenario: groupBy v1 (which is non-default)
in "useOffheap" mode (also non-default). So you have to go pretty deep
into the wilderness to get this code to activate in production. It is
never used during ingestion.
- Its existence complicates developer efforts to reason about how
aggregators get used, because the way it uses buffer aggregators is so
different from how every other query engine uses them.
- It doesn't have meaningful testing.
By the way, I do believe that the given way the offheap incremental index
works, it actually didn't require buffer aggregators to be thread-safe.
It synchronizes on "aggregate" and doesn't call "get" until it has
stopped calling "aggregate". Nevertheless, this is a bother to think about,
and for the above reasons I think it makes sense to remove the code anyway.
* Remove things that are now unused.
* Revert removal of getFloat, getLong, getDouble from BufferAggregator.
* OAK-related warnings, suppressions.
* Unused item suppressions.
* latest datasketches-java and datasketches-memory
* updated versions of datasketches-java and datasketches-memory
Co-authored-by: AlexanderSaydakov <AlexanderSaydakov@users.noreply.github.com>
* better type system
* needle in a haystack
* ColumnCapabilities is a TypeSignature instead of having one, INFORMATION_SCHEMA support
* fixup merge
* more test
* fixup
* intern
* fix
* oops
* oops again
* ...
* more test coverage
* fix error message
* adjust interning, more javadocs
* oops
* more docs more better
* refactor sql authorization to get resource type from schema, refactor resource type from enum to string
* information schema auth filtering adjustments
* refactor
* minor stuff
* Update SqlResourceCollectorShuttle.java
* Configurable maxStreamLength for doubles sketches
* fix equals/hashcode and it test failure
* fix test
* fix it test
* benchmark
* doc
* grouping key
* fix comment
* dependency check
* Update docs/development/extensions-core/datasketches-quantiles.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
* Update docs/querying/sql.md
Co-authored-by: Charles Smith <techdocsmith@gmail.com>
Co-authored-by: Charles Smith <techdocsmith@gmail.com>