Commit Graph

686 Commits

Author SHA1 Message Date
Gian Merlino bf39b4d313
Window planning: use collation traits, improve subquery logic. (#13902)
* Window planning: use collation traits, improve subquery logic.

SQL changes:

1) Attach RelCollation (sorting) trait to any PartialDruidQuery
   that ends in AGGREGATE or AGGREGATE_PROJECT. This allows planning to
   take advantage of the fact that Druid sorts by dimensions when
   doing aggregations.

2) Windowing: inspect RelCollation trait from input, and insert naiveSort
   if, and only if, necessary.

3) Windowing: add support for Project after Window, when the Project
   is a simple mapping. Helps eliminate subqueries.

4) DruidRules: update logic for considering subqueries to reflect that
   subqueries are not required to be GroupBys, and that we have a bunch
   of new Stages now. With all of this evolution that has happened, the
   old logic didn't quite make sense.

Native changes:

1) Use merge sort (stable) rather than quicksort when sorting
   RowsAndColumns. Makes it easier to write test cases for plans that
   involve re-sorting the data.

* Changes from review.

* Mark the bad test as failing.

* Additional update.

* Fix failingTest.

* Fix tests.

* Mark a var final.
2023-03-09 15:48:13 -08:00
Clint Wylie 48ac5ce50b
use native nvl expression for SQL NVL and 2 argument COALESCE (#13897)
* use custom case operator conversion instead of direct operator conversion, to produce native nvl expression for SQL NVL and 2 argument COALESCE, and add optimization for certain case filters from coalesce and nvl statements
2023-03-09 05:46:17 -08:00
Gian Merlino 90d8f67e3d
Avoid creating new RelDataTypeFactory during SQL planning. (#13904)
* Avoid creating new RelDataTypeFactory during SQL planning.

Reduces unnecessary CPU cycles.

* Fix.
2023-03-08 21:55:49 -08:00
Gian Merlino 82f7a56475
Sort-merge join and hash shuffles for MSQ. (#13506)
* Sort-merge join and hash shuffles for MSQ.

The main changes are in the processing, multi-stage-query, and sql modules.

processing module:

1) Rename SortColumn to KeyColumn, replace boolean descending with KeyOrder.
   This makes it nicer to model hash keys, which use KeyOrder.NONE.

2) Add nullability checkers to the FieldReader interface, and an
   "isPartiallyNullKey" method to FrameComparisonWidget. The join
   processor uses this to detect null keys.

3) Add WritableFrameChannel.isClosed and OutputChannel.isReadableChannelReady
   so callers can tell which OutputChannels are ready for reading and which
   aren't.

4) Specialize FrameProcessors.makeCursor to return FrameCursor, a random-access
   implementation. The join processor uses this to rewind when it needs to
   replay a set of rows with a particular key.

5) Add MemoryAllocatorFactory, which is embedded inside FrameWriterFactory
   instead of a particular MemoryAllocator. This allows FrameWriterFactory
   to be shared in more scenarios.

multi-stage-query module:

1) ShuffleSpec: Add hash-based shuffles. New enum ShuffleKind helps callers
   figure out what kind of shuffle is happening. The change from SortColumn
   to KeyColumn allows ClusterBy to be used for both hash-based and sort-based
   shuffling.

2) WorkerImpl: Add ability to handle hash-based shuffles. Refactor the logic
   to be more readable by moving the work-order-running code to the inner
   class RunWorkOrder, and the shuffle-pipeline-building code to the inner
   class ShufflePipelineBuilder.

3) Add SortMergeJoinFrameProcessor and factory.

4) WorkerMemoryParameters: Adjust logic to reserve space for output frames
   for hash partitioning. (We need one frame per partition.)

sql module:

1) Add sqlJoinAlgorithm context parameter; can be "broadcast" or
   "sortMerge". With native, it must always be "broadcast", or it's a
   validation error. MSQ supports both. Default is "broadcast" in
   both engines.

2) Validate that MSQs do not use broadcast join with RIGHT or FULL join,
   as results are not correct for broadcast join with those types. Allow
   this in native for two reasons: legacy (the docs caution against it,
   but it's always been allowed), and the fact that it actually *does*
   generate correct results in native when the join is processed on the
   Broker. It is much less likely that MSQ will plan in such a way that
   generates correct results.

3) Remove subquery penalty in DruidJoinQueryRel when using sort-merge
   join, because subqueries are always required, so there's no reason
   to penalize them.

4) Move previously-disabled join reordering and manipulation rules to
   FANCY_JOIN_RULES, and enable them when using sort-merge join. Helps
   get to better plans where projections and filters are pushed down.

* Work around compiler problem.

* Updates from static analysis.

* Fix @param tag.

* Fix declared exception.

* Fix spelling.

* Minor adjustments.

* wip

* Merge fixups

* fixes

* Fix CalciteSelectQueryMSQTest

* Empty keys are sortable.

* Address comments from code review. Rename mux -> mix.

* Restore inspection config.

* Restore original doc.

* Reorder imports.

* Adjustments

* Fix.

* Fix imports.

* Adjustments from review.

* Update header.

* Adjust docs.
2023-03-08 14:19:39 -08:00
Adarsh Sanjeev ef82756176
Add validation for aggregations on __time (#13793)
* Add validation for aggregations on __time
2023-03-07 17:16:36 -08:00
Clint Wylie 3924f0eff4
use Calcites.getColumnTypeForRelDataType for SQL CAST operator conversion (#13890)
* use Calcites.getColumnTypeForRelDataType for SQL CAST operator conversion

* fix comment

* intervals are strings but also longs
2023-03-07 13:12:15 -08:00
Gian Merlino fcfb7b8ff6
Add warning comments to Granularity.getIterable. (#13888)
This function is notorious for causing memory exhaustion and excessive
CPU usage; so much so that it was valuable to work around it in the
SQL planner in #13206. Hopefully, a warning comment will encourage
developers to stay away and come up with solutions that do not involve
computing all possible buckets.
2023-03-06 22:57:10 -08:00
Clint Wylie 6cf754b0e0
move numeric null value coercion out of expression processing engine (#13809)
* move numeric null value coercion out of expression processing engine
* add ExprEval.valueOrDefault() to allow consumers to automatically coerce to default values
* rename Expr.buildVectorized as Expr.asVectorProcessor more consistent naming with Function and ApplyFunction; javadocs for some stuff
2023-02-28 18:10:07 -08:00
Paul Rogers 914eebb4b7
Wire up the catalog resolver (#13788)
Introduces the catalog resolver interface
Wires the resolver up to the planner factory
Refactors planner factory
2023-02-22 11:42:32 -08:00
zachjsh 665dee43bf
Revert "Operator conversion deny list (#13766)" (#13829)
This reverts commit 38e620aa4c.
2023-02-21 15:14:49 -08:00
Paul Rogers 85d36be085
Information schema now uses numeric column types (#13777)
Change to use SQL schemas to allow null numeric columns

* Updated docs
2023-02-17 14:39:31 -08:00
Clint Wylie 08b5951cc5
merge druid-core, extendedset, and druid-hll into druid-processing to simplify everything (#13698)
* 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
2023-02-17 14:27:41 -08:00
Paul Rogers 333196d207
Code cleanup & message improvements (#13778)
* Misc cleanup edits

Correct spacing
Add type parameters
Add toString() methods to formats so tests compare correctly
IT doc revisions
Error message edits
Display UT query results when tests fail

* Edit

* Build fix

* Build fixes
2023-02-15 15:22:54 +05:30
Clint Wylie fa4cab405f
fix bug with sql planner when virtual column capabilities are null (#13797) 2023-02-13 18:27:23 -08:00
Paul Rogers 842ee554de
Refinements to input-source specific table functions (#13780)
Refinements to table functions

Fixes various bugs
Improves the structure of the table function classes
Adds unit and integration tests
2023-02-13 16:21:27 -08:00
Clint Wylie f09f83697d
fix array_agg to work with complex types and bugs with expression aggregator complex array handling (#13781)
* fix array_agg to work with complex types and bugs with expression aggregator complex array handling
* more consistent handling of array expressions, numeric arrays more consistently honor druid.generic.useDefaultValueForNull, fix array_ordinal sql output type
2023-02-12 22:01:39 -08:00
zachjsh 38e620aa4c
Operator conversion deny list (#13766)
### 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
  }
}
```
2023-02-10 09:59:26 -08:00
AmatyaAvadhanula dcdae84888
Add server view initialization metrics (#13716)
* Add server view init metrics

* Test coverage

* Rename metrics
2023-02-07 20:02:00 +05:30
Clint Wylie 2d3bee8545
various nested column (and other) fixes (#13732)
changes:
* modified druid schema column type compution to special case COMPLEX<json> handling to choose COMPLEX<json> if any column in any segment is COMPLEX<json>
* NestedFieldVirtualColumn can now work correctly on any type of column, returning either a column selector if a root path, or nil selector if not
* fixed a random bug with NilVectorSelector when using a vector size larger than the default and druid.generic.useDefaultValueForNull=false would have the nulls vector set to all false instead of true
* fixed an overly aggressive check in ExprEval.ofType when handling complex types which would try to treat any string as base64 without gracefully falling back if it was not in fact base64 encoded, along with special handling for complex<json>
* added ExpressionVectorSelectors.castValueSelectorToObject and ExpressionVectorSelectors.castObjectSelectorToNumeric as convience methods to cast vector selectors using cast expressions without the trouble of constructing an expression. the polymorphic nature of the non-vectorized engine (and significantly larger overhead of non-vectorized expression processing) made adding similar methods for non-vectorized selectors less attractive and so have not been added at this time
* fix inconsistency between nested column indexer and serializer in handling values (coerce non primitive and non arrays of primitives using asString)
* ExprEval best effort mode now handles byte[] as string
* added test for ExprEval.bestEffortOf, and add missing conversion cases that tests uncovered
* more tests more better
2023-02-06 19:48:02 -08:00
imply-cheddar 706b8a0227
Adjust Operators to be Pausable (#13694)
* Adjust Operators to be Pausable

This enables "merge" style operations that
combine multiple streams.

This change includes a naive implementation
of one such merge operator just to provide
concrete evidence that the refactoring is
effective.
2023-01-23 20:52:06 -08:00
somu-imply 90d445536d
SQL version of unnest native druid function (#13576)
* adds the SQL component of the native unnest functionality in Druid to unnest SQL queries on a table dimension, virtual column or a constant array and convert them into native Druid queries
* unnest in SQL is implemented as a combination of Correlate (the comma join part) and Uncollect (the unnest part)
2023-01-23 12:53:31 -08:00
Laksh Singla a516eb1a41
Port Calcite's tests to run with MSQ (#13625)
* SQL test framework extensions

* Capture planner artifacts: logical plan, etc.
* Planner test builder validates the logical plan
* Validation for the SQL resut schema (we already have
  validation for the Druid row signature)
* Better Guice integration: properties, reuse Guice modules
* Avoid need for hand-coded expr, macro tables
* Retire some of the test-specific query component creation
* Fix query log hook race condition

Co-authored-by: Paul Rogers <progers@apache.org>
2023-01-19 08:51:11 -08:00
Paul Rogers 22630b0aab
Much improved table functions (#13627)
Much improved table functions

* Revises properties, definitions in the catalog
* Adds a "table function" abstraction to model such functions
* Specific functions for HTTP, inline, local and S3.
* Extended SQL types in the catalog
* Restructure external table definitions to use table functions
* EXTEND syntax for Druid's extern table function
* Support for array-valued table function parameters
* Support for array-valued SQL query parameters
* Much new documentation
2023-01-17 08:41:57 -08:00
imply-cheddar 7ff3722cb9
Swap LazySingleton for Singleton (#13673)
* Swap LazySingleton for Singleton
* Initialize WebserverTestUtils properly
2023-01-15 21:38:37 -08:00
Clint Wylie b5b740bbbb
allow using nested column indexer for schema discovery (#13653)
* single typed "root" only nested columns now mimic "regular" columns of those types
* incremental index can now use nested column indexer instead of string indexer for discovered columns
2023-01-12 18:31:12 -08:00
Adarsh Sanjeev 0a486c3bcf
Update forbidden apis with fixed executor (#13633)
* Update forbidden apis with fixed executor
2023-01-12 15:34:36 +05:30
imply-cheddar f1821a7c18
Add Sort Operator for Window Functions (#13619)
* Addition of NaiveSortMaker and Default implementation

Add the NaiveSortMaker which makes a sorter
object and a default implementation of the
interface.

This also allows us to plan multiple different window 
definitions on the same query.
2023-01-06 00:27:18 -08:00
imply-cheddar a8ecc48ffe
Validate response headers and fix exception logging (#13609)
* Validate response headers and fix exception logging

A class of QueryException were throwing away their
causes making it really hard to determine what's
going wrong when something goes wrong in the SQL
planner specifically.  Fix that and adjust tests
 to do more validation of response headers as well.

We allow 404s and 307s to be returned even without 
authorization validated, but others get converted to 403
2023-01-05 14:15:15 -08:00
Clint Wylie fd63e5a514
fix issue with jdbc and query metrics (#13608)
* fix issue with metrics emitting and jdbc results by getting yielder from query processing thread

* more better
2022-12-21 19:32:53 -08:00
imply-cheddar 0efd0879a8
Unify the handling of HTTP between SQL and Native (#13564)
* Unify the handling of HTTP between SQL and Native

The SqlResource and QueryResource have been
using independent logic for things like error
handling and response context stuff.  This
became abundantly clear and painful during a
change I was making for Window Functions, so
I unified them into using the same code for
walking the response and serializing it.

Things are still not perfectly unified (it would
be the absolute best if the SqlResource just
took SQL, planned it and then delegated the
query run entirely to the QueryResource), but
this refactor doesn't take that fully on.

The new code leverages async query processing
from our jetty container, the different
interaction model with the Resource means that
a lot of tests had to be adjusted to align with
the async query model.  The semantics of the
tests remain the same with one exception: the
SqlResource used to not log requests that failed
authorization checks, now it does.
2022-12-19 00:25:33 -08:00
Clint Wylie 9ae7a36ccd
improve nested column storage format for broader compatibility (#13568)
* bump nested column format version
changes:
* nested field files are now named by their position in field paths list, rather than directly by the path itself. this fixes issues with valid json properties with commas and newlines breaking the csv file meta.smoosh
* update StructuredDataProcessor to deal in NestedPathPart to be consistent with other abstract path handling rather than building JQ syntax strings directly
* add v3 format segment and test
2022-12-15 15:39:26 -08:00
imply-cheddar 089d8da561
Support Framing for Window Aggregations (#13514)
* Support Framing for Window Aggregations

This adds support for framing  over ROWS
for window aggregations.

Still not implemented as yet:
1. RANGE frames
2. Multiple different frames in the same query
3. Frames on last/first functions
2022-12-14 18:04:39 -08:00
Rohan Garg 35c983a351
Use template file for adding table functions grammar (#13553) 2022-12-14 21:52:09 +05:30
somu-imply 7682b0b6b1
Analysis refactor (#13501)
Refactor DataSource to have a getAnalysis method()

This removes various parts of the code where while loops and instanceof
checks were being used to walk through the structure of DataSource objects
in order to build a DataSourceAnalysis.  Instead we just ask the DataSource
for its analysis and allow the stack to rebuild whatever structure existed.
2022-12-12 17:35:44 -08:00
Paul Rogers 013a12e86f
Enhanced MSQ table functions (#13360)
* Enhanced MSQ table functions
* HTTP, LOCALFILES and INLINE table functions powered by
catalog metadata.
* Documentation
2022-12-08 13:56:02 -08:00
imply-cheddar 83261f9641
Starting on Window Functions (#13458)
* Processors for Window Processing

This is an initial take on how to use Processors
for Window Processing.  A Processor is an interface
that transforms RowsAndColumns objects.
RowsAndColumns objects are essentially combinations
of rows and columns.

The intention is that these Processors are the start
of a set of operators that more closely resemble what
DB engineers would be accustomed to seeing.

* Wire up windowed processors with a query type that
can run them end-to-end.  This code can be used to
actually run a query, so yay!

* Wire up windowed processors with a query type that
can run them end-to-end.  This code can be used to
actually run a query, so yay!

* Some SQL tests for window functions. Added wikipedia 
data to the indexes available to the
SQL queries and tests validating the windowing
functionality as it exists now.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2022-12-06 15:54:05 -08:00
Paul Rogers b76ff16d00
SQL test framework extensions (#13426)
SQL test framework extensions

* Capture planner artifacts: logical plan, etc.
* Planner test builder validates the logical plan
* Validation for the SQL resut schema (we already have
  validation for the Druid row signature)
* Better Guice integration: properties, reuse Guice modules
* Avoid need for hand-coded expr, macro tables
* Retire some of the test-specific query component creation
* Fix query log hook race condition
2022-12-02 09:11:59 -08:00
Kashif Faraz 656b6cdf62
Add MetricsVerifier to simplify verification of metric values in tests (#13442) 2022-11-28 19:32:37 +05:30
Kashif Faraz 7cf761cee4
Prepare master branch for next release, 26.0.0 (#13401)
* Prepare master branch for next release, 26.0.0

* Use docker image for druid 24.0.1

* Fix version in druid-it-cases pom.xml
2022-11-22 15:31:01 +05:30
Paul Rogers 81d005f267
Druid Catalog basics (#13165)
Druid catalog basics

Catalog object model for tables, columns
Druid metadata DB storage (as an extension)
REST API to update the catalog (as an extension)
Integration tests
Model only: no planner integration yet
2022-11-12 15:30:22 -08:00
Clint Wylie 27215d1ff1
fix complex_decode_base64 function, add SQL bindings (#13332)
* fix complex_decode_base64 function, add SQL bindings

* more permissive
2022-11-09 23:40:25 -08:00
Paul Rogers 7e600d2c63
Enhancements to the Calcite test framework (#13283)
* Enhancements to the Calcite test framework
* Standardize "Unauthorized" messages
* Additional test framework extension points
* Resolved joinable factory dependency issue
2022-11-08 14:28:49 -08:00
Gian Merlino 8f90589ce5
Always return sketches from DS_HLL, DS_THETA, DS_QUANTILES_SKETCH. (#13247)
* 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.
2022-11-03 09:43:00 -07:00
Paul Rogers 22c140251a
Removed unused planner context parameter (#13249)
Removed unused planner context parameter
2022-10-27 17:59:26 -07:00
somu-imply affc522b9f
Refactoring the data source before unnest (#13085)
* 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
2022-10-26 15:58:58 -07:00
Gian Merlino 2b0d873c7e
Fix two sources of SQL statement leaks. (#13259)
* Fix two sources of SQL statement leaks.

1) SqlTaskResource and DruidJdbcResultSet leaked statements 100% of the
   time, since they call stmt.plan(), which adds statements to
   SqlLifecycleManager, and they do not explicitly remove them.

2) SqlResource leaked statements if yielder.close() threw an exception.
   (And also would not emit metrics, since in that case it failed to
   call stmt.close as well.)

* Only closeQuietly is needed.
2022-10-25 09:31:56 -07:00
Paul Rogers 86e6e61e88
Modular Calcite Test Framework (#12965)
* 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.
2022-10-20 15:45:44 -07:00
Paul Rogers b34b4353f4
Async reads for JDBC (#13196)
Async reads for JDBC:
Prevents JDBC timeouts on long queries by returning empty batches
when a batch fetch takes too long. Uses an async model to run the
result fetch concurrently with JDBC requests.

Fixed race condition in Druid's Avatica server-side handler
Fixed issue with no-user connections
2022-10-18 11:40:57 -07:00
Gian Merlino 6aca61763e
SQL: Use timestamp_floor when granularity is not safe. (#13206)
* 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.
2022-10-17 08:22:45 -07:00
Paul Rogers f4dcc52dac
Redesign QueryContext class (#13071)
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
2022-10-15 11:02:11 +05:30