Commit Graph

11762 Commits

Author SHA1 Message Date
Jihoon Son e5ad862665
A new includeAllDimension flag for dimensionsSpec (#12276)
* includeAllDimensions in dimensionsSpec

* doc

* address comments

* unused import and doc spelling
2022-02-25 18:27:48 -08:00
Jason Koch eb1b53b7f8
perf: indexing: Introduce a bulk getValuesInto function to read values (#12105)
* 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
2022-02-25 12:19:04 -08:00
Karan Kumar b86f2d4c2e
Performance fixes in proto readers (#12267) 2022-02-24 23:21:48 +05:30
Xavier Léauté 009dd9e09a
upgrade core Apache Kafka dependencies to 3.1.0 (#12203)
Announcement: https://blogs.apache.org/kafka/entry/what-s-new-in-apache7
Release notes: https://dist.apache.org/repos/dist/release/kafka/3.1.0/RELEASE_NOTES.html

* upgrade core Apache Kafka dependencies to 3.1.0
* fix use of private Kafka APIs
* remove deprecated test rules
* remove mock calls that weren't verified in the first place
* remove the need for powermock in KafkaLookupExtractorFactoryTest
* align curator-test version with curator itself
* update easymock to 4.3.0
2022-02-23 18:42:51 -08:00
Jonathan Wei b1640a72ee
Re-enable segment metadata cache when using external schema (#12264) 2022-02-22 19:50:29 -06:00
Karan Kumar b94390ba33
Adding Shared Access resource support for azure (#12266)
Azure Blob storage has multiple modes of authentication. One of them is Shared access resource
. This is very useful in cases when we do not want to add the account key in the druid properties .
2022-02-22 18:27:43 +05:30
Maytas Monsereenusorn 6e2eded277
Allow coordinator run auto compaction duty period to be configured separately from other indexing duties (#12263)
* add impl

* add impl

* add unit tests

* add impl

* add impl

* add serde test

* add tests

* add docs

* fix test

* fix test

* fix docs

* fix docs

* fix spelling
2022-02-18 23:02:57 -08:00
AmatyaAvadhanula 1ec57cb935
Improve kinesis task assignment after resharding (#12235)
Problem:
- When a kinesis stream is resharded, the original shards are closed.
   Any intermediate shard created in the process is eventually closed as well.
- If a shard is closed before any record is put into it, it can be safely ignored for ingestion.
- It is expensive to determine if a closed shard is empty, since it requires a call to the Kinesis cluster.

Changes:
- Maintain a cache of closed empty and closed non-empty shards in `KinesisSupervisor`
- Add config `skipIngorableShards` to `KinesisSupervisorTuningConfig`
- The caches are used and updated only when `skipIgnorableShards = true`
2022-02-18 12:37:06 +05:30
tejaswini-imply 70c40c4281
Fix long overflow in SegmentCostCache.Bucket.toLocalInterval (#12257)
Problem:
When using a `CachingCostBalancerStrategy` with segments of granularity ALL,
no segment gets loaded.
- With granularity ALL, segments of eternity interval are created which have 
  `start = Long.MIN_VALUE / 2` and `end = Long.MAX_VALUE / 2`.
- For cost calculation in the balancer strategy, `toLocalInterval()` method is invoked where
  `Long.MIN_VALUE / 2` or `Long.MAX_VALUE / 2` cause an overflow thus resulting in no overlap.
- The strategy is unable to find any eligible server for loading a given segment.

Fix:
- Reverse order of operations to divide by `MILLIS_FACTOR` (~10^8) first,
   then do the subtraction to prevent Long overflow.
2022-02-17 15:13:51 +05:30
Abhishek Agarwal 575874705f
Fix the flakiness in getLockedIntervals test (#12172)
Fix the flakiness in getLockedIntervals test
2022-02-17 12:08:46 +05:30
Karan Kumar 5794331eb1
Adding new config for disabling group by on multiValue column (#12253)
As part of #12078 one of the followup's was to have a specific config which does not allow accidental unnesting of multi value columns if such columns become part of the grouping key.
Added a config groupByEnableMultiValueUnnesting which can be set in the query context.

The default value of groupByEnableMultiValueUnnesting is true, therefore it does not change the current engine behavior.
If groupByEnableMultiValueUnnesting is set to false, the query will fail if it encounters a multi-value column in the grouping key.
2022-02-16 20:53:26 +05:30
Laksh Singla 8fc0e5c95c
Explain plan for custom insert syntax (#12243)
* Initial commit, explain plan for custom insert syntax working

* Cleanup separate SqlInsert handling
2022-02-15 21:48:34 -08:00
somu-imply eae163a797
Moving in filter check to broker (#12195)
* Moving in filter check to broker

* Adding more unit tests, making error message meaningful

* Spelling and doc changes

* Updating default to -1 and making this feature hide by default. The number of IN filters can grow upto a max limit of 100

* Removing upper limit of 100, updated docs

* Making documentation more meaningful

* Moving check outside to PlannerConfig, updating test cases and adding back max limit

* Updated with some additional code comments

* Missed removing one line during the checkin

* Addressing doc changes and one forbidden API correction

* Final doc change

* Adding a speling exception, correcting a testcase

* Reading entire filter tree to address combinations of ANDs and ORs

* Specifying in docs that, this case works only for ORs

* Revert "Reading entire filter tree to address combinations of ANDs and ORs"

This reverts commit 81ca8f8496.

* Covering a class cast exception and updating docs

* Counting changed

Co-authored-by: Jihoon Son <jihoonson@apache.org>
2022-02-15 20:45:07 -08:00
Jason Koch 26bc4b7345
perf: cache row if it is a transformed row (#12113)
* perf: cache row if it is a transformed row

* perf: cache row if it is a transformed row (also cache DateTime object)
2022-02-15 10:08:41 -08:00
William Hyun 34bc361953
Update ORC to 1.7.2 (#12084) 2022-02-15 10:04:12 -08:00
AmatyaAvadhanula 393e9b68a8
Add config to limit task slots for parallel indexing tasks (#12221)
In extreme cases where many parallel indexing jobs are submitted together, it is possible
that the `ParallelIndexSupervisorTasks` take up all slots leaving no slot to schedule
their own sub-tasks thus stalling progress of all the indexing jobs.

Key changes:
- Add config `druid.indexer.runner.parallelIndexTaskSlotRatio` to limit the task slots
  for `ParallelIndexSupervisorTasks` per worker
- `ratio = 1` implies supervisor tasks can use all slots on a worker if needed (default behavior)
- `ratio = 0` implies supervisor tasks can not use any slot on a worker
   (actually, at least 1 slot is always available to ensure progress of parallel indexing jobs)
- `ImmutableWorkerInfo.canRunTask()`
- `WorkerHolder`, `ZkWorker`, `WorkerSelectUtils`
2022-02-15 23:15:09 +05:30
Daniel Koepke 47153cd7bd
Increase retries for Kinesis sharding integration tests. (#12255)
This fixes intermittent, spurious failures that we've observed in
the Kinesis sharding integration tests due to Kinesis taking
longer than the code expected to start a sharding operation. The
method that's changed is part of the integration test suite and
only used by the test cases that we've seen are flaky.

Prior to this change, the tests expected a sharding operation to
start in 9 seconds (30 retries * 300ms delay/retry). This change
bumps the number of retries to 100, giving Kinesis 30 seconds to
start the sharding.

This PR also makes a small, clarifying change to the condition
used to determine if sharding has started. Instead of checking if
the number of shards has increased (which was technically correct
even if the test is reducing the number of shards due to a Kinesis
implementation detail), we now just check if the shard count has
changed.
2022-02-14 23:33:13 -08:00
somu-imply 033989eb1d
Adding vectorized time_shift (#12254)
* Adding vectorized time_shift

* Vectorize time shift, addressing review comments

* Remove an unused import
2022-02-11 14:44:52 -08:00
Victoria Lim c61b19d443
Refactor SQL docs (#12239)
* refactor and link fixes

* add sql docs to left nav

* code format for needle

* updated web console script

* link fixes

* update earliest/latest functions

* edits for grammar and style

* more link fixes

* another link

* update with #12226

* update .spelling file
2022-02-11 14:43:30 -08:00
Laksh Singla 5bd646e10a
Surface a user friendly error when PARTITIONED BY is omitted (#12246)
#12163 makes PARTITIONED BY a required clause in INSERT queries. While this is required, if a user accidentally omits the clause, it emits a JavaCC/Calcite error, since it's syntactically incorrect. The error message is cryptic. Since it's a custom clause, this PR aims to make the clause optional on the syntactic side, but move the validation to DruidSqlInsert where we can surface a friendlier error.
2022-02-11 11:49:00 +05:30
Kashif Faraz 95b388d2d1
Assign partitionIds in the same order as bucketIds (#12236)
When `ParallelIndexSupervisorTask` converts `BucketNumberedShardSpecs`
to corresponding `BuildingShardSpecs`, the bucketId order gets lost.
Particularly, for range partitioning, this results in the partitionIds not being in the same order
as increasing partition boundaries.

Changes
- Refactor `ParallelIndexSupervisorTask.groupGenericPartitionLocationsPerPartition()`
2022-02-10 11:08:39 +05:30
Clint Wylie 3ee66bb492
allow optimizing sql expressions and virtual columns (#12241)
* rework sql planner expression and virtual column handling

* simplify a bit

* add back and deprecate old methods, more tests, fix multi-value string coercion bug and associated tests

* spotbugs

* fix bugs with multi-value string array expression handling

* javadocs and adjust test

* better

* fix tests
2022-02-09 14:55:50 -08:00
Jonathan Wei 33bc9226f0
Move task creation under stateChangeLock in SeekableStreamSupervisor (#12178) 2022-02-09 13:24:46 -06:00
John Gozde 0d23713862
Web console: update dev dependencies (#12240)
* Update dependencies

* Set "allowFunctions: true" for react/jsx-no-bind

* Prettify

* npm audit fix

* Bump playwright, set testEnvironment=node

* Bump node and npm

* Revert "Bump node and npm"

This reverts commit e93c8e00e7.

* Minor/patch bump node@14, npm@6
2022-02-08 16:37:36 -08:00
Jihoon Son ab3d994a17
Lazy instantiation for segmentKillers, segmentMovers, and segmentArchivers (#12207)
* working

* Lazily load segmentKillers, segmentMovers, and segmentArchivers

* more tests

* test-jar plugin

* more coverage

* lazy client

* clean up changes

* checkstyle

* i did not change the branch condition

* adjust failure rate to run tests faster

* javadocs

* checkstyle
2022-02-08 13:02:06 -08:00
Laksh Singla 4add2510ed
Add syntax support for PARTITIONED BY/CLUSTERED BY in INSERT queries (#12163)
This PR aims to add parser changes for supporting PARTITIONED BY and CLUSTERED BY as proposed in the issue #11929.
2022-02-08 16:23:15 +05:30
Clint Wylie ae71e05fc5
array_concat_agg and array_agg support for array inputs (#12226)
* array_concat_agg and array_agg support for array inputs
changes:
* added array_concat_agg to aggregate arrays into a single array
* added array_agg support for array inputs to make nested array
* added 'shouldAggregateNullInputs' and 'shouldCombineAggregateNullInputs' to fix a correctness issue with STRING_AGG and ARRAY_AGG when merging results, with dual purpose of being an optimization for aggregating

* fix test

* tie capabilities type to legacy mode flag about coercing arrays to strings

* oops

* better javadoc
2022-02-07 19:59:30 -08:00
Vadim Ogievetsky 090c429c8c
Web console: make it possible to namespace local storage, auto flatten spec generator should deal better with bad data (#12238)
* improve computeFlattenExprsForData

* allow local storage namespacing

* add test
2022-02-07 18:52:41 -08:00
Suneet Saldanha ced1389d4c
Enable auto kill segments by default (#12187)
* Enable auto-kill by default

* tests

* wip

* test

* fix IT

* fix it

* remove from docs

* make coverage bot happy
2022-02-07 06:57:54 -08:00
Maytas Monsereenusorn 2b8e7fc0b4
Add a flag to allow auto compaction task slot ratio to consider auto scaler slots (#12228)
* add impl

* fix checkstyle

* add unit tests

* checkstyle

* add IT

* fix IT

* add comments

* fix checkstyle
2022-02-06 20:46:05 -08:00
Suneet Saldanha 159f97dcb0
Update docs for druid.processing.numThreads in brokers (#12231)
* Update docs for druid.processing.numThreads

* error msg

* one more reference
2022-02-04 17:34:21 -08:00
Gian Merlino de82c611de
Harmonize implementations of "visit" for Exprs from ExprMacros. (#12230)
* Harmonize implementations of "visit" for Exprs from ExprMacros.

Many of them had bugs where they would not visit all of the original
arguments. I don't think this has user-visible consequences right now,
but it's possible it would in a future world where "visit" is used
for more stuff than it is today.

So, this patch all updates all implementations to a more consistent
style that emphasizes reapplying the macro to the shuttled args.

* Test fixes, test coverage, PR review comments.
2022-02-04 08:08:54 -08:00
tejaswini-imply 290130b1fa
Fix bug while adding `Range` header in HttpEntity (#12215)
Changes:
- Add `Range` header to the request before opening the connection
- Use header `Content-Range` instead of `Accept-Ranges` as `Content-Range` is guaranteed to be populated if the server is returning a partial response
2022-02-04 18:17:51 +05:30
Clint Wylie a3affe1471
make EncodedKeyComponent constructor public, remove nullable from DimensionIndexer.processRowValsToUnsortedEncodedKeyComponent (#12229) 2022-02-03 15:02:32 -08:00
Clint Wylie 8fd587b28c
remove duplicate Broker ServerInventoryView, improve HttpServerInventoryView logging (#12209)
* changes:
* remove SystemSchema duplicate ServerInventoryView in broker
* suppress duplicate segment added/removed warnings in HttpServerInventoryView when doing a full sync

* fixes
2022-02-03 12:57:34 -08:00
Maytas Monsereenusorn 3717693633
Fix java.lang.ClassCastException error when using useApproximateCountDistinct false for aggregation query (#12216)
* add imply

* add test

* add unit test

* add test
2022-02-03 12:01:13 -08:00
Vadim Ogievetsky fc76b014d1
Web console: fix supervisor stats table pagination (#12227)
* fixes #11627 supervisor stats table pagination

* use spread instead of assign
2022-02-03 00:09:21 -08:00
Kashif Faraz e648b01afb
Improve memory estimates in Aggregator and DimensionIndexer (#12073)
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`
2022-02-03 10:34:02 +05:30
Vadim Ogievetsky bc408bacc8
Web console: Adding a shard detail column to the segments view (#12212)
* shard spec details

* improve pattern match

* refactor spec cleanup

* better format detection

* update JSONbig

* add multiline option to autoform
2022-02-02 18:46:17 -08:00
AshishKapoor 801d9e7f1b
[Web Console] fix deprecated keyboard event method "keyCode" with "key" (#11947)
* 11946 fix keyboard event keyCode method

* fix with key and respective cases

* e.which method required since it's anyway deprecated too.

* updated as per feedback
2022-02-02 18:26:21 -08:00
zachjsh f47e1e0dcc
Reduce RemoteTaskRunnerTest flakiness (#12211)
* * add more logging to start / stop of RemoteTaskRunner

* * add more logging

* Increase timeout on RemoteTaskRunnerTest

* Apply suggestions from code review

Co-authored-by: Suneet Saldanha <suneet@apache.org>

Co-authored-by: Suneet Saldanha <suneet@apache.org>
2022-02-01 15:35:18 -08:00
Clint Wylie f9b406c8f2
add backwards compatibility mode for multi-value string array null value coercion (#12210) 2022-01-31 22:38:15 -08:00
Clint Wylie 978b8f7dde
do not explode if mysql transient exception class does not exist (#12213)
Follow up to #12205 to allow druid-mysql-extensions to work with mysql connector/j 8.x again, which does not contain MySQLTransientException, and while would have had the same problem as mariadb if a transient exception was checked, the new check eagerly loads the class when starting up, causing immediate failure.
2022-02-01 09:06:24 +05:30
Rohan Garg c4fa3ccfc4
Fix load-drop-load sequence for same segment and historical in http loadqueue peon (#11717)
Fixes an issue where a load-drop-load sequence for a segment and historical doesn't work correctly for http based load queue peon. The first cycle of load-drop works fine - the problem comes when there is an attempt to reload the segment. The historical caches load success for some recent segments and makes the reload as a no-op. But it doesn't consider that fact that the segment was also dropped in between the load requests.
This change invalidates the cache after a client tries to fetch a success result.
2022-01-31 13:16:58 +05:30
Vadim Ogievetsky fe8530dac4
Change link to Apache Druid Slack (#12206) 2022-01-27 21:10:40 -08:00
Jihoon Son eeed156dc0
Fix compile error in VirtualizedColumnSelectorFactoryTest (#12208) 2022-01-27 17:35:50 -08:00
Gian Merlino 99a5c2f3d3
Harmonize behavior when virtual columns reference each other. (#11955)
* VirtualizedColumnSelectorFactory: Allow virtual columns to reference each other.

This matches the behavior of QueryableIndex and IncrementalIndex based cursors.

* Fixes to getColumnCapabilities.
2022-01-27 14:31:48 -08:00
Clint Wylie 5d2291991e
use reflection to check for mysql transient exception type (#12205)
* use reflection to check for mysql transient exception type

* better

* oops
2022-01-27 13:13:16 -08:00
Victoria Lim 24716bfedc
Doc updates for metadata cleanup and storage (#12190)
* doc updates for metadata storage/cleanup

* Add comments for disabling cleanup

* Apply suggestions from code review

* updated for https://github.com/apache/druid/pull/12201

* Apply suggestions from code review

Co-authored-by: Maytas Monsereenusorn <maytasm@apache.org>

* move retention period line earlier; more concise text

* fix typo

Co-authored-by: Maytas Monsereenusorn <maytasm@apache.org>
2022-01-27 11:40:54 -08:00
Maytas Monsereenusorn fac6a48a8f
add impl (#12201) 2022-01-27 11:39:59 -08:00