Changes:
- Move logic of `NewestSegmentFirstIterator.needsCompaction` to `CompactionStatus`
to improve testability and readability
- Capture the list of checks performed to determine if compaction is needed in a readable
manner in `CompactionStatus.CHECKS`
- Make `CompactionSegmentIterator` iterate over instances of `SegmentsToCompact`
instead of `List<DataSegment>`. This allows use of the `umbrellaInterval` later.
- Replace usages of `QueueEntry` with `SegmentsToCompact`
- Move `SegmentsToCompact` out of `NewestSegmentFirstIterator`
- Simplify `CompactionStatistics`
- Reduce level of less important logs to debug
- No change made to tests to ensure correctness
* Minimize PostAggregator computations
Since a change back in 2014, the topN query has been computing
all PostAggregators on all intermediate responses from leaf nodes
to brokers. This generates significant slow downs for queries
with relatively expensive PostAggregators. This change rewrites
the query that is pushed down to only have the minimal set of
PostAggregators such that it is impossible for downstream
processing to do too much work. The final PostAggregators are
applied at the very end.
Changes
- Rename `LoadQueuePeonTester` to `TestLoadQueuePeon`
- Simplify `TestLoadQueuePeon` by removing dependency on `CuratorLoadQueuePeon`
- Remove usages of mock peons in `LoadRuleTest` and use `TestLoadQueuePeon` instead
changes:
* new filters that preserve match value typing to better handle filtering different column types
* sql planner uses new filters by default in sql compatible null handling mode
* remove isFilterable from column capabilities
* proper handling of array filtering, add array processor to column processors
* javadoc for sql test filter functions
* range filter support for arrays, tons more tests, fixes
* add dimension selector tests for mixed type roots
* support json equality
* rename semantic index maker thingys to mostly have plural names since they typically make many indexes, e.g. StringValueSetIndex -> StringValueSetIndexes
* add cooler equality index maker, ValueIndexes
* fix missing string utf8 index supplier
* expression array comparator stuff
After #13197 , several coordinator configs are now redundant as they are not being
used anymore, neither with `smartSegmentLoading` nor otherwise.
Changes:
- Remove dynamic configs `emitBalancingStats`: balancer error stats are always
emitted, debug stats can be logged by using `debugDimensions`
- `useBatchedSegmentSampler`, `percentOfSegmentsToConsiderPerMove`:
batched segment sampling is always used
- Add test to verify deserialization with unknown properties
- Update `CoordinatorRunStats` to always track stats, this can be optimized later.
* combine string column implementations
changes:
* generic indexed, front-coded, and auto string columns now all share the same column and index supplier implementations
* remove CachingIndexed implementation, which I think is largely no longer needed by the switch of many things to directly using ByteBuffer, avoiding the cost of creating Strings
* remove ColumnConfig.columnCacheSizeBytes since CachingIndexed was the only user
* Add "stringEncoding" parameter to DataSketches HLL.
Builds on the concept from #11172 and adds a way to feed HLL sketches
with UTF-8 bytes.
This must be an option rather than always-on, because prior to this
patch, HLL sketches used UTF-16LE encoding when hashing strings. To
remain compatible with sketch images created prior to this patch -- which
matters during rolling updates and when reading sketches that have been
written to segments -- we must keep UTF-16LE as the default.
Not currently documented, because I'm not yet sure how best to expose
this functionality to users. I think the first place would be in the SQL
layer: we could have it automatically select UTF-8 or UTF-16LE when
building sketches at query time. We need to be careful about this, though,
because UTF-8 isn't always faster. Sometimes, like for the results of
expressions, UTF-16LE is faster. I expect we will sort this out in
future patches.
* Fix benchmark.
* Fix style issues, improve test coverage.
* Put round back, to make IT updates easier.
* Fix test.
* Fix issue with filtered aggregators and add test.
* Use DS native update(ByteBuffer) method. Improve test coverage.
* Add another suppression.
* Fix ITAutoCompactionTest.
* Update benchmarks.
* Updates.
* Fix conflict.
* Adjustments.
This commit does a complete revamp of the coordinator to address problem areas:
- Stability: Fix several bugs, add capabilities to prioritize and cancel load queue items
- Visibility: Add new metrics, improve logs, revamp `CoordinatorRunStats`
- Configuration: Add dynamic config `smartSegmentLoading` to automatically set
optimal values for all segment loading configs such as `maxSegmentsToMove`,
`replicationThrottleLimit` and `maxSegmentsInNodeLoadingQueue`.
Changed classes:
- Add `StrategicSegmentAssigner` to make assignment decisions for load, replicate and move
- Add `SegmentAction` to distinguish between load, replicate, drop and move operations
- Add `SegmentReplicationStatus` to capture current state of replication of all used segments
- Add `SegmentLoadingConfig` to contain recomputed dynamic config values
- Simplify classes `LoadRule`, `BroadcastRule`
- Simplify the `BalancerStrategy` and `CostBalancerStrategy`
- Add several new methods to `ServerHolder` to track loaded and queued segments
- Refactor `DruidCoordinator`
Impact:
- Enable `smartSegmentLoading` by default. With this enabled, none of the following
dynamic configs need to be set: `maxSegmentsToMove`, `replicationThrottleLimit`,
`maxSegmentsInNodeLoadingQueue`, `useRoundRobinSegmentAssignment`,
`emitBalancingStats` and `replicantLifetime`.
- Coordinator reports richer metrics and produces cleaner and more informative logs
- Coordinator uses an unlimited load queue for all serves, and makes better assignment decisions
Introduce DruidException, an exception whose goal in life is to be delivered to a user.
DruidException itself has javadoc on it to describe how it should be used. This commit both introduces the Exception and adjusts some of the places that are generating exceptions to generate DruidException objects instead, as a way to show how the Exception should be used.
This work was a 3rd iteration on top of work that was started by Paul Rogers. I don't know if his name will survive the squash-and-merge, so I'm calling it out here and thanking him for starting on this.
This PR fixes an issue when using 'auto' encoded LONG typed columns and the 'vectorized' query engine. These columns use a delta based bit-packing mechanism, and errors in the vectorized reader would cause it to incorrectly read column values for some bit sizes (1 through 32 bits). This is a regression caused by #11004, which added the optimized readers to improve performance, so impacts Druid versions 0.22.0+.
While writing the test I finally got sad enough about IndexSpec not having a "builder", so I made one, and switched all the things to use it. Apologies for the noise in this bug fix PR, the only real changes are in VSizeLongSerde, and the tests that have been modified to cover the buggy behavior, VSizeLongSerdeTest and ExpressionVectorSelectorsTest. Everything else is just cleanup of IndexSpec usage.
* smarter nested column index utilization
changes:
* adds skipValueRangeIndexScale and skipValuePredicateIndexScale to ColumnConfig (e.g. DruidProcessingConfig) available as system config via druid.processing.indexes.skipValueRangeIndexScale and druid.processing.indexes.skipValuePredicateIndexScale
* NestedColumnIndexSupplier uses skipValueRangeIndexScale and skipValuePredicateIndexScale to multiply by the total number of rows to be processed to determine the threshold at which we should no longer consider using bitmap indexes because it will be too many operations
* Default values for skipValueRangeIndexScale and skipValuePredicateIndexScale have been initially set to 0.08, but are separate to allow independent tuning
* these are not documented on purpose yet because they are kind of hard to explain, the mainly exist to help conduct larger scale experiments than the jmh benchmarks used to derive the initial set of values
* these changes provide a pretty sweet performance boost for filter processing on nested columns
This change introduces the concept of input source type security model, proposed in #13837.. With this change, this feature is only available at the SQL layer, but we will expand to native layer in a follow up PR.
To enable this feature, the user must set the following property to true:
druid.auth.enableInputSourceSecurity=true
The default value for this property is false, which will continue the existing functionality of having the usage all external sources being authorized against the hardcoded resource action
new ResourceAction(new Resource(ResourceType.EXTERNAL, ResourceType.EXTERNAL), Action.READ
When this config is enabled, the users will be required to be authorized for the following resource action
new ResourceAction(new Resource(ResourceType.EXTERNAL, {INPUT_SOURCE_TYPE}, Action.READ
where {INPUT_SOURCE_TYPE} is the type of the input source being used;, http, inline, s3, etc..
Documentation has not been added for the feature as it is not complete at the moment, as we still need to enable this for the native layer in a follow up pr.
* Adds new implementation of 'frontCoded' string encoding strategy, which writes out a v1 FrontCodedIndexed which stores buckets on a prefix of the previous value instead of the first value in the bucket
* Improve memory efficiency of WrappedRoaringBitmap.
Two changes:
1) Use an int[] for sizes 4 or below.
2) Remove the boolean compressRunOnSerialization. Doesn't save much
space, but it does save a little, and it isn't adding a ton of value
to have it be configurable. It was originally configurable in case
anything broke when enabling it, but it's been a while and nothing
has broken.
* Slight adjustment.
* Adjust for inspection.
* Updates.
* Update snaps.
* Update test.
* Adjust test.
* Fix snaps.
* merge druid-core, extendedset, and druid-hll into druid-processing to simplify everything
* fix poms and license stuff
* mockito is evil
* allow reset of JvmUtils RuntimeInfo if tests used static injection to override
### Description
This change adds a new config property `druid.sql.planner.operatorConversion.denyList`, which allows a user to specify
any operator conversions that they wish to disallow. A user may want to do this for a number of reasons, including security concerns. The default value of this property is the empty list `[]`, which does not disallow any operator conversions.
An example usage of this property is `druid.sql.planner.operatorConversion.denyList=["extern"]`, which disallows the usage of the `extern` operator conversion. If the property is configured this way, and a user of the Druid cluster tries to submit a query that uses the `extern` function, such as the example given [here](https://druid.apache.org/docs/latest/multi-stage-query/examples.html#insert-with-no-rollup), a response with http response code `400` is returned with en error body similar to the following:
```
{
"taskId": "4ec5b0b6-fa9b-4c3a-827d-2308294e9985",
"state": "FAILED",
"error": {
"error": "Plan validation failed",
"errorMessage": "org.apache.calcite.runtime.CalciteContextException: From line 28, column 5 to line 32, column 5: No match found for function signature EXTERN(<CHARACTER>, <CHARACTER>, <CHARACTER>)",
"errorClass": "org.apache.calcite.tools.ValidationException",
"host": null
}
}
```
* perf: provide a custom utf8 specific buffered line iterator (benchmark)
Benchmark Mode Cnt Score Error Units
JsonLineReaderBenchmark.baseline avgt 15 3459.871 ± 106.175 us/op
* perf: provide a custom utf8 specific buffered line iterator
Benchmark Mode Cnt Score Error Units
JsonLineReaderBenchmark.baseline avgt 15 3022.053 ± 51.286 us/op
* perf: provide a custom utf8 specific buffered line iterator (more tests)
* perf: provide a custom utf8 specific buffered line iterator (pr feedback)
Ensure field visibility is as limited as possible
Null check for buffer in constructor
* perf: provide a custom utf8 specific buffered line iterator (pr feedback)
Remove additional 'finished' variable.
* perf: provide a custom utf8 specific buffered line iterator (more tests and bugfix)
* Add string comparison methods to StringUtils, fix dictionary comparisons.
There are various places in Druid code where we assume that String.compareTo
is consistent with Unicode code-point ordering. Sadly this is not the case.
To help deal with this, this patch introduces the following helpers:
1) compareUnicode: Compares two Strings in Unicode code-point order.
2) compareUtf8: Compares two UTF-8 byte arrays in Unicode code-point order.
Equivalent to comparison as unsigned bytes.
3) compareUtf8UsingJavaStringOrdering: Compares two UTF-8 byte arrays, or
ByteBuffers, in a manner consistent with String.compareTo.
There is no helper for comparing two Strings in a manner consistent
with String.compareTo, because for that we can use compareTo directly.
The patch also fixes an inconsistency between the String and UTF-8
dictionary GenericIndexed flavors of string-typed columns: they were
formerly using incompatible comparators.
* Adjust test.
* FrontCodedIndexed updates.
* Add test.
* Fix comments.
* Always return sketches from DS_HLL, DS_THETA, DS_QUANTILES_SKETCH.
These aggregation functions are documented as creating sketches. However,
they are planned into native aggregators that include finalization logic
to convert the sketch to a number of some sort. This creates an
inconsistency: the functions sometimes return sketches, and sometimes
return numbers, depending on where they lie in the native query plan.
This patch changes these SQL aggregators to _never_ finalize, by using
the "shouldFinalize" feature of the native aggregators. It already
existed for theta sketches. This patch adds the feature for hll and
quantiles sketches.
As to impact, Druid finalizes aggregators in two cases:
- When they appear in the outer level of a query (not a subquery).
- When they are used as input to an expression or finalizing-field-access
post-aggregator (not any other kind of post-aggregator).
With this patch, the functions will no longer be finalized in these cases.
The second item is not likely to matter much. The SQL functions all declare
return type OTHER, which would be usable as an input to any other function
that makes sense and that would be planned into an expression.
So, the main effect of this patch is the first item. To provide backwards
compatibility with anyone that was depending on the old behavior, the
patch adds a "sqlFinalizeOuterSketches" query context parameter that
restores the old behavior.
Other changes:
1) Move various argument-checking logic from runtime to planning time in
DoublesSketchListArgBaseOperatorConversion, by adding an OperandTypeChecker.
2) Add various JsonIgnores to the sketches to simplify their JSON representations.
3) Allow chaining of ExpressionPostAggregators and other PostAggregators
in the SQL layer.
4) Avoid unnecessary FieldAccessPostAggregator wrapping in the SQL layer,
now that expressions can operate on complex inputs.
5) Adjust return type to thetaSketch (instead of OTHER) in
ThetaSketchSetBaseOperatorConversion.
* Fix benchmark class.
* Fix compilation error.
* Fix ThetaSketchSqlAggregatorTest.
* Hopefully fix ITAutoCompactionTest.
* Adjustment to ITAutoCompactionTest.
In clusters with a large number of segments, the duty `MarkAsUnusedOvershadowedSegments`
can take a long very long time to finish. This is because of the costly invocation of
`timeline.isOvershadowed` which is done for every used segment in every coordinator run.
Changes
- Use `DataSourceSnapshot.getOvershadowedSegments` to get all overshadowed segments
- Iterate over this set instead of all used segments to identify segments that can be marked as unused
- Mark segments as unused in the DB in batches rather than one at a time
- Refactor: Add class `SegmentTimeline` for ease of use and readability while using a
`VersionedIntervalTimeline` of segments.
* introduce a "tree" type to the flattenSpec
* feedback - rename exprs to nodes, use CollectionsUtils.isNullOrEmpty for guard
* feedback - expand docs to more clearly capture limitations of "tree" flattenSpec
* feedback - fix for typo on docs
* introduce a comment to explain defensive copy, tweak null handling
* fix: part of rebase
* mark ObjectFlatteners.FlattenerMaker as an ExtensionPoint and provide default for new tree type
* fix: objectflattener restore previous behavior to call getRootField for root type
* docs: ingestion/data-formats add note that ORC only supports path expressions
* chore: linter remove unused import
* fix: use correct newer form for empty DimensionsSpec in FlattenJSONBenchmark
* First set of changes for framework
* Second set of changes to move segment map function to data source
* Minot change to server manager
* Removing the createSegmentMapFunction from JoinableFactoryWrapper and moving to JoinDataSource
* Checkstyle fixes
* Patching Eric's fix for injection
* Checkstyle and fixing some CI issues
* Fixing code inspections and some failed tests and one injector for test in avatica
* Another set of changes for CI...almost there
* Equals and hashcode part update
* Fixing injector from Eric + refactoring for broadcastJoinHelper
* Updating second injector. Might revert later if better way found
* Fixing guice issue in JoinableFactory
* Addressing review comments part 1
* Temp changes refactoring
* Revert "Temp changes refactoring"
This reverts commit 9da42a9ef0.
* temp
* Temp discussions
* Refactoring temp
* Refatoring the query rewrite to refer to a datasource
* Refactoring getCacheKey by moving it inside data source
* Nullable annotation check in injector
* Addressing some comments, removing 2 analysis.isJoin() checks and correcting the benchmark files
* Minor changes for refactoring
* Addressing reviews part 1
* Refactoring part 2 with new test cases for broadcast join
* Set for nullables
* removing instance of checks
* Storing nullables in guice to avoid checking on reruns
* Fixing a test case and removing an irrelevant line
* Addressing the atomic reference review comments
* add FrontCodedIndexed for delta string encoding
* now for actual segments
* fix indexOf
* fixes and thread safety
* add bucket size 4, which seems generally better
* fixes
* fixes maybe
* update indexes to latest interfaces
* utf8 support
* adjust
* oops
* oops
* refactor, better, faster
* more test
* fixes
* revert
* adjustments
* fix prefixing
* more chill
* sql nested benchmark too
* refactor
* more comments and javadocs
* better get
* remove base class
* fix
* hot rod
* adjust comments
* faster still
* minor adjustments
* spatial index support
* spotbugs
* add isSorted to Indexed to strengthen indexOf contract if set, improve javadocs, add docs
* fix docs
* push into constructor
* use base buffer instead of copy
* oops
* Refactor Calcite test "framework" for planner tests
Refactors the current Calcite tests to make it a bit easier
to adjust the set of runtime objects used within a test.
* Move data creation out of CalciteTests into TestDataBuilder
* Move "framework" creation out of CalciteTests into
a QueryFramework
* Move injector-dependent functions from CalciteTests
into QueryFrameworkUtils
* Wrapper around the planner factory, etc. to allow
customization.
* Bulk of the "framework" created once per class rather
than once per test.
* Refactor tests to use a test builder
* Change all testQuery() methods to use the test builder.
Move test execution & verification into a test runner.
* SQL: Use timestamp_floor when granularity is not safe.
PR #12944 added a check at the execution layer to avoid materializing
excessive amounts of time-granular buckets. This patch modifies the SQL
planner to avoid generating queries that would throw such errors, by
switching certain plans to use the timestamp_floor function instead of
granularities. This applies both to the Timeseries query type, and the
GroupBy timestampResultFieldGranularity feature.
The patch also goes one step further: we switch to timestamp_floor
not just in the ETERNITY + non-ALL case, but also if the estimated
number of time-granular buckets exceeds 100,000.
Finally, the patch modifies the timestampResultFieldGranularity
field to consistently be a String rather than a Granularity. This
ensures that it can be round-trip serialized and deserialized, which is
useful when trying to execute the results of "EXPLAIN PLAN FOR" with
GroupBy queries that use the timestampResultFieldGranularity feature.
* Fix test, address PR comments.
* Fix ControllerImpl.
* Fix test.
* Fix unused import.
We introduce two new configuration keys that refine the query context security model controlled by druid.auth.authorizeQueryContextParams. When that value is set to true then two other configuration options become available:
druid.auth.unsecuredContextKeys: The set of query context keys that do not require a security check. Use this for the "white-list" of key to allow. All other keys go through the existing context key security checks.
druid.auth.securedContextKeys: The set of query context keys that do require a security check. Use this when you want to allow all but a specific set of keys: only these keys go through the existing context key security checks.
Both are set using JSON list format:
druid.auth.securedContextKeys=["secretKey1", "secretKey2"]
You generally set one or the other values. If both are set, unsecuredContextKeys acts as exceptions to securedContextKeys.
In addition, Druid defines two query context keys which always bypass checks because Druid uses them internally:
sqlQueryId
sqlStringifyArrays
* Expose HTTP Response headers from SqlResource
This change makes the SqlResource expose HTTP response
headers in the same way that the QueryResource exposes them.
Fundamentally, the change is to pipe the QueryResponse
object all the way through to the Resource so that it can
populate response headers. There is also some code
cleanup around DI, as there was a superfluous FactoryFactory
class muddying things up.
* 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