Allow a Druid cluster to kill segments whose interval_end is a date in the future. This can be done by setting druid.coordinator.kill.durationToRetain to a negative period. For example PT-24H would allow segments to be killed if their interval_end date was 24 hours or less into the future at the time that the kill task is generated by the system.
A cluster operator can also disregard the druid.coordinator.kill.durationToRetain entirely by setting a new configuration, druid.coordinator.kill.ignoreDurationToRetain=true. This ignores interval_end date when looking for segments to kill, and instead is capable of killing any segment marked unused. This new configuration is off by default, and a cluster operator should fully understand and accept the risks if they enable it.
* GroupBy: Reduce allocations by reusing entry and key holders.
Two main changes:
1) Reuse Entry objects returned by various implementations of
Grouper.iterator.
2) Reuse key objects contained within those Entry objects.
This is allowed by the contract, which states that entries must be
processed and immediately discarded. However, not all call sites
respected this, so this patch also updates those call sites.
One particularly sneaky way that the old code retained entries too long
is due to Guava's MergingIterator and CombiningIterator. Internally,
these both advance to the next value prior to returning the current
value. So, this patch addresses that in two ways:
1) For merging, we have our own implementation MergeIterator already,
although it had the same problem. So, this patch updates our
implementation to return the current item prior to advancing to the
next item. It also adds a forbidden-api entry to ensure that this
safer implementation is used instead of Guava's.
2) For combining, we address the problem in a different way: by copying
the key when creating the new, combined entry.
* Attempt to fix test.
* Remove unused import.
* Reduce allocations due to Jackson serialization.
This patch attacks two sources of allocations during Jackson
serialization:
1) ObjectMapper.writeValue and JsonGenerator.writeObject create a new
DefaultSerializerProvider instance for each call. It has lots of
fields and creates pressure on the garbage collector. So, this patch
adds helper functions in JacksonUtils that enable reuse of
SerializerProvider objects and updates various call sites to make
use of this.
2) GroupByQueryToolChest copies the ObjectMapper for every query to
install a special module that supports backwards compatibility with
map-based rows. This isn't needed if resultAsArray is set and
all servers are running Druid 0.16.0 or later. This release was a
while ago. So, this patch disables backwards compatibility by default,
which eliminates the need to copy the heavyweight ObjectMapper. The
patch also introduces a configuration option that allows admins to
explicitly enable backwards compatibility.
* Add test.
* Update additional call sites and add to forbidden APIs.
* Support array based results in timeBoundary query
* Fix bug with query interval in timeBoundary
* Convert min(__time) and max(__time) SQL queries to timeBoundary
* Add tests for timeBoundary backed SQL queries
* Fix query plans for existing tests
* fixup! Convert min(__time) and max(__time) SQL queries to timeBoundary
* fixup! Add tests for timeBoundary backed SQL queries
* fixup! Fix bug with query interval in timeBoundary
The query context is a way that the user gives a hint to the Druid query engine, so that they enforce a certain behavior or at least let the query engine prefer a certain plan during query planning. Today, there are 3 types of query context params as below.
Default context params. They are set via druid.query.default.context in runtime properties. Any user context params can be default params.
User context params. They are set in the user query request. See https://druid.apache.org/docs/latest/querying/query-context.html for parameters.
System context params. They are set by the Druid query engine during query processing. These params override other context params.
Today, any context params are allowed to users. This can cause
1) a bad UX if the context param is not matured yet or
2) even query failure or system fault in the worst case if a sensitive param is abused, ex) maxSubqueryRows.
This PR adds an ability to limit context params per user role. That means, a query will fail if you have a context param set in the query that is not allowed to you. To do that, this PR adds a new built-in resource type, QUERY_CONTEXT. The resource to authorize has a name of the context param (such as maxSubqueryRows) and the type of QUERY_CONTEXT. To allow a certain context param for a user, the user should be granted WRITE permission on the context param resource. Here is an example of the permission.
{
"resourceAction" : {
"resource" : {
"name" : "maxSubqueryRows",
"type" : "QUERY_CONTEXT"
},
"action" : "WRITE"
},
"resourceNamePattern" : "maxSubqueryRows"
}
Each role can have multiple permissions for context params. Each permission should be set for different context params.
When a query is issued with a query context X, the query will fail if the user who issued the query does not have WRITE permission on the query context X. In this case,
HTTP endpoints will return 403 response code.
JDBC will throw ForbiddenException.
Note: there is a context param called brokerService that is used only by the router. This param is used to pin your query to run it in a specific broker. Because the authorization is done not in the router, but in the broker, if you have brokerService set in your query without a proper permission, your query will fail in the broker after routing is done. Technically, this is not right because the authorization is checked after the context param takes effect. However, this should not cause any user-facing issue and thus should be OK. The query will still fail if the user doesn’t have permission for brokerService.
The context param authorization can be enabled using druid.auth.authorizeQueryContextParams. This is disabled by default to avoid any hassle when someone upgrades his cluster blindly without reading release notes.
* Make tombstones ingestible by having them return an empty result set.
* Spotbug
* Coverage
* Coverage
* Remove unnecessary exception (checkstyle)
* Fix integration test and add one more to test dropExisting set to false over tombstones
* Force dropExisting to true in auto-compaction when the interval contains only tombstones
* Checkstyle, fix unit test
* Changed flag by mistake, fixing it
* Remove method from interface since this method is specific to only DruidSegmentInputentity
* Fix typo
* Adapt to latest code
* Update comments when only tombstones to compact
* Move empty iterator to a new DruidTombstoneSegmentReader
* Code review feedback
* Checkstyle
* Review feedback
* Coverage
* Optionally load segment index files into page cache on bootstrap and new segment download
* Fix unit test failure
* Fix test case
* fix spelling
* fix spelling
* fix test and test coverage issues
Co-authored-by: Jian Wang <wjhypo@gmail.com>
* add impl
* add impl
* fix checkstyle
* add impl
* add unit test
* fix stuff
* fix stuff
* fix stuff
* add unit test
* add more unit tests
* add more unit tests
* add IT
* add IT
* add IT
* add IT
* add ITs
* address comments
* fix test
* fix test
* fix test
* address comments
* address comments
* address comments
* fix conflict
* fix checkstyle
* address comments
* fix test
* fix checkstyle
* fix test
* fix test
* fix IT
The current default value of inputSegmentSizeBytes is 400MB, which is pretty
low for most compaction use cases. Thus most users are forced to override the
default.
The default value is now increased to Long.MAX_VALUE.
* Store null columns in the segments
* fix test
* remove NullNumericColumn and unused dependency
* fix compile failure
* use guava instead of apache commons
* split new tests
* unused imports
* address comments
* finds complete and active tasks from the same snapshot
* overlord resource
* unit test
* integration test
* javadoc and cleanup
* more cleanup
* fix test and add more
Add config for eager / lazy connection initialization in ResourcePool
Description
Currently, when multiple tasks are launched, each of them eagerly initializes a full pool's worth of connections to the coordinator.
While this is acceptable when the parameter for number of eagerConnections (== maxSize) is small, this can be problematic in environments where it's a large value (say 1000) and multiple tasks are launched simultaneously, which can cause a large number of connections to be created to the coordinator, thereby overwhelming it.
Patch
Nodes like the broker may require eager initialization of resources and do not create connections with the Coordinator.
It is unnecessary to do this with other types of nodes.
A config parameter eagerInitialization is added, which when set to true, initializes the max permissible connections when ResourcePool is initialized.
If set to false, lazy initialization of connection resources takes place.
NOTE: All nodes except the broker have this new parameter set to false in the quickstart as part of this PR
Algorithm
The current implementation relies on the creation of maxSize resources eagerly.
The new implementation's behaviour is as follows:
If a resource has been previously created and is available, lend it.
Else if the number of created resources is less than the allowed parameter, create and lend it.
Else, wait for one of the lent resources to be returned.
* Tombstone support for replace functionality
* A used segment interval is the interval of a current used segment that overlaps any of the input intervals for the spec
* Update compaction test to match replace behavior
* Adapt ITAutoCompactionTest to work with tombstones rather than dropping segments. Add support for tombstones in the broker.
* Style plus simple queriableindex test
* Add segment cache loader tombstone test
* Add more tests
* Add a method to the LogicalSegment to test whether it has any data
* Test filter with some empty logical segments
* Refactor more compaction/dropexisting tests
* Code coverage
* Support for all empty segments
* Skip tombstones when looking-up broker's timeline. Discard changes made to tool chest to avoid empty segments since they will no longer have empty segments after lookup because we are skipping over them.
* Fix null ptr when segment does not have a queriable index
* Add support for empty replace interval (all input data has been filtered out)
* Fixed coverage & style
* Find tombstone versions from lock versions
* Test failures & style
* Interner was making this fail since the two segments were consider equal due to their id's being equal
* Cleanup tombstone version code
* Force timeChunkLock whenever replace (i.e. dropExisting=true) is being used
* Reject replace spec when input intervals are empty
* Documentation
* Style and unit test
* Restore test code deleted by mistake
* Allocate forces TIME_CHUNK locking and uses lock versions. TombstoneShardSpec added.
* Unused imports. Dead code. Test coverage.
* Coverage.
* Prevent killer from throwing an exception for tombstones. This is the killer used in the peon for killing segments.
* Fix OmniKiller + more test coverage.
* Tombstones are now marked using a shard spec
* Drop a segment factory.json in the segment cache for tombstones
* Style
* Style + coverage
* style
* Add TombstoneLoadSpec.class to mapper in test
* Update core/src/main/java/org/apache/druid/segment/loading/TombstoneLoadSpec.java
Typo
Co-authored-by: Jonathan Wei <jon-wei@users.noreply.github.com>
* Update docs/configuration/index.md
Missing
Co-authored-by: Jonathan Wei <jon-wei@users.noreply.github.com>
* Typo
* Integrated replace with an existing test since the replace part was redundant and more importantly, the test file was very close or exceeding the 10 min default "no output" CI Travis threshold.
* Range does not work with multi-dim
Co-authored-by: Jonathan Wei <jon-wei@users.noreply.github.com>
* Always reopen stream in FileUtils.copyLarge, RetryingInputStream.
When an InputStream throws an exception from one of its read methods,
we should assume it's bad and reopen it.
The main changes here are:
- In FileUtils.copyLarge, replace InputStream with InputStreamSupplier.
- In RetryingInputStream, collapse retryCondition and resetCondition
into a single condition. Also, make it required, since every usage
is passing in a specific condition anyway.
* Test fixes.
* Fix read impl.
* add a new query laning metrics to visualize lane assignment
* fixes :spotbugs check
* Update docs/operations/metrics.md
Co-authored-by: Benedict Jin <asdf2014@apache.org>
* Update server/src/main/java/org/apache/druid/server/QueryScheduler.java
Co-authored-by: Benedict Jin <asdf2014@apache.org>
* Update server/src/main/java/org/apache/druid/server/QueryScheduler.java
Co-authored-by: Benedict Jin <asdf2014@apache.org>
Co-authored-by: Benedict Jin <asdf2014@apache.org>
This PR aims to make the ParseExceptions in Druid more informative, by adding additional information (metadata) to the ParseException, which can contain additional information about the exception. For example - the path of the file generating the issue, the line number (where it can be easily fetched - like CsvReader)
Following changes are addressed in this PR:
A new class CloseableIteratorWithMetadata has been created which is like CloseableIterator but also has a metadata method that returns a context Map<String, Object> about the current element returned by next().
IntermediateRowParsingReader#read() now attaches the InputEntity and the "record number" which created the exception (while parsing them), and IntermediateRowParsingReader#sample attaches the InputEntity (but not the "record number").
TextReader (and its subclasses), which is a specific implementation of the IntermediateRowParsingReader also include the line number which caused the generation of the error.
This will also help in triaging the issues when InputSourceReader generates ParseException because it can point to the specific InputEntity which caused the exception (while trying to read it).
Mockito now supports all our needs and plays much better with recent Java versions.
Migrating to Mockito also simplifies running the kind of tests that required PowerMock in the past.
* replace all uses of powermock with mockito-inline
* upgrade mockito to 4.3.1 and fix use of deprecated methods
* import mockito bom to align all our mockito dependencies
* add powermock to forbidden-apis to avoid accidentally reintroducing it in the future
* upgrade Airline to Airline 2
https://github.com/airlift/airline is no longer maintained, updating to
https://github.com/rvesse/airline (Airline 2) to use an actively
maintained version, while minimizing breaking changes.
Note, this is a backwards incompatible change, and extensions relying on
the CliCommandCreator extension point will also need to be updated.
* fix dependency checks where jakarta.inject is now resolved first instead
of javax.inject, due to Airline 2 using jakarta
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.
* 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
* changes:
* remove SystemSchema duplicate ServerInventoryView in broker
* suppress duplicate segment added/removed warnings in HttpServerInventoryView when doing a full sync
* fixes
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`
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.
* init multiValue column group by
* Changing sorting to Lexicographic as default
* Adding initial tests
* 1.Fixing test cases adding
2.Optimized inmem structs
* Linking SQL layer to native layer
* Adding multiDimension support to group by column strategy
* 1. Removing array coercion in Calcite layer
2. Removing ResultRowDeserializer
* 1. Supporting all primitive array types
2. Removing dimension spec as part of columnSelector
* 1. Supporting all primitive array types
2. Removing dimension spec as part of columnSelector
* 1. Checkstyle things
2. Removing flag
* Minor naming things
* CheckStyle Things
* Fixing test case
* Fixing hashing
* 1. Adding the MV function
2. Added few test cases
* 1. Adding MV function test cases
* Adding Selector strategy function test cases
* Fixing ClientQuerySegmentWalkerTest
* Adding GroupByQueryRunnerTest test cases
* Fixing test cases
* Adding few more test cases
* Fixing Exception asset statement and intellij inspection
* Adding null compatibility tests
* Review comments
* Fixing few failing tests
* Fixing few failing tests
* Do no convert to topN Q incase of group by on array
* Fixing checkstyle
* Fixing differences between jdk's class cast exception message
* 1. Fixing ordering if the grouping key is an array
* Fixing DefaultLimitSpec
* Fixing CalciteArraysQueryTest
* Dummy commit for LGTM
* changes:
* only coerce multi-value string null values when `ExpressionPlan.Trait.NEEDS_APPLIED` is set
* correct return type inference for ARRAY_APPEND,ARRAY_PREPEND,ARRAY_SLICE,ARRAY_CONCAT
* fix bug with ExprEval.ofType when actual type of object from binding doesn't match its claimed type
* Review comments
* Fixing test cases
* Fixing spot bugs
* Fixing strict compile
Co-authored-by: Clint Wylie <cwylie@apache.org>
* Add http response status code to org.eclipse.jetty.server.RequestLog
* http response code is expressed as an int. Set log msg interpolation based on digit
* trying to add an unit test to verify if the logger.debug method is called
* trying to add an unit test to verify if the logger.debug method is called
* fix compilation issues
* remove test
* Thread pool for broker
* Updating two tests to improve coverage for new method added
* Updating druidProcessingConfigTest to cover coverage
* Adding missed spelling errors caused in doc
* Adding test to cover lines of new function added