* Ensure ByteBuffers allocated in tests get freed.
Many tests had problems where a direct ByteBuffer would be allocated
and then not freed. This is bad because it causes flaky tests.
To fix this:
1) Add ByteBufferUtils.allocateDirect(size), which returns a ResourceHolder.
This makes it easy to free the direct buffer. Currently, it's only used
in tests, because production code seems OK.
2) Update all usages of ByteBuffer.allocateDirect (off-heap) in tests either
to ByteBuffer.allocate (on-heap, which are garbaged collected), or to
ByteBufferUtils.allocateDirect (wherever it seemed like there was a good
reason for the buffer to be off-heap). Make sure to close all direct
holders when done.
* Changes based on CI results.
* A different approach.
* Roll back BitmapOperationTest stuff.
* Try additional surefire memory.
* Revert "Roll back BitmapOperationTest stuff."
This reverts commit 49f846d9e3.
* Add TestBufferPool.
* Revert Xmx change in tests.
* Better behaved NestedQueryPushDownTest. Exit tests on OOME.
* Fix TestBufferPool.
* Remove T1C from ARM tests.
* Somewhat safer.
* Fix tests.
* Fix style stuff.
* Additional debugging.
* Reset null / expr configs better.
* ExpressionLambdaAggregatorFactory thread-safety.
* Alter forkNode to try to get better info when a JVM crashes.
* Fix buffer retention in ExpressionLambdaAggregatorFactory.
* Remove unused import.
* 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.
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.
Currently while loading a lookup for the first time, loading threads blocks
for `waitForFirstRunMs` incase the lookup failed to load. If the `waitForFirstRunMs`
is long (like 10 minutes), such blocking can slow down the loading of other lookups.
This commit allows the thread to progress as soon as the loading of the lookup fails.
amazon-kinesis-client was not covered undered the apache license and required separate insertion in the kinesis extension.
This can now be avoided since it is covered, and including it within druid helps prevent incompatibilities.
Allows enabling of deaggregation out of the box by packaging amazon-kinesis-client (1.14.4) with druid for kinesis ingestion.
listShards API was used to get all the shards for kinesis ingestion to improve its resiliency as part of #12161.
However, this may require additional permissions in the IAM policy where the stream is present. (Please refer to: https://docs.aws.amazon.com/kinesis/latest/APIReference/API_ListShards.html).
A dynamic configuration useListShards has been added to KinesisSupervisorTuningConfig to control the usage of this API and prevent issues upon upgrade. It can be safely turned on (and is recommended when using kinesis ingestion) by setting this configuration to true.
* 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
* kubernetes: restart watch on null response
Kubernetes watches allow a client to efficiently processes changes to
resources. However, they have some idiosyncrasies. In particular, they
can error out for various reasons leading to what would normally be seen
as an invalid result.
The Druid kubernetes node discovery subsystem does not handle a certain
case properly. The watch can return an item with a null object. These
leads to a null pointer exception. When this happens, the provider needs
to restart the watch, because rerunning the watch from the same resource
version leads to the same result: yet another null pointer exception.
This commit changes the provider to handle null objects by restarting
the watch.
* review: add more coverage
This adds a bit more coverage to the K8sDruidNodeDiscoveryProvider watch
loop, and removes an unnecessay return.
* kubernetes: reduce logging verbosity
The log messages about items being NULL don't really deserve to be at a
level other than DEBUG since they are not actionable, particularly since
we automatically recover now. Move them to the DEBUG level.
* 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.
These changes are to use the latest datasketches-java-3.1.0 and also to restore support for quantile and HLL4 sketches to be able to grow larger than a given buffer in a buffer aggregator and move to heap in rare cases. This was discussed in #11544.
Co-authored-by: AlexanderSaydakov <AlexanderSaydakov@users.noreply.github.com>
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
* remove use of mocks for ServiceMetricEvent
* simplify KafkaEmitterTests by moving to Mockito
* speed up KafkaEmitterTest by adjusting reporting frequency in tests
* remove unnecessary easymock and JUnitParams dependencies
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 .
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`
* 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
* 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
* 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.
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`
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.
Makes kinesis ingestion resilient to `LimitExceededException` caused by resharding.
Replace `describeStream` with `listShards` (recommended) to get shard related info.
`describeStream` has a limit (100) to the number of shards returned per call and a low default TPS limit of 10.
`listShards` returns the info for at most 1000 shards and has a higher TPS limit of 100 as well.
Key changed/added classes in this PR
* `KinesisRecordSupplier`
* `KinesisAdminClient`
This fixes a bug that causes TaskClient in overlord to continuously retry to pause tasks. This can happen when a task is not responding to the pause command. Ideally, in such a case when the task is unresponsive, the overlord would have given up after a few retries and would have killed the task. However, due to this bug, retries go on forever.
* Ingestion will fail for HLLSketchBuild instead of creating with incorrect values
* Addressing review comments for HLL< updated error message introduced test case
* Add jsonPath functions support
* Add jsonPath function test for Avro
* Add jsonPath function length() to Orc
* Add jsonPath function length() to Parquet
* Add more tests to ORC format
* update doc
* Fix exception during ingestion
* Add IT test case
* Revert "Fix exception during ingestion"
This reverts commit 5a5484b9ea.
* update IT test case
* Add 'keys()'
* Commit IT test case
* Fix UT
This PR fixes an issue in which if a lookup is configured incorreclty; does not serialize properly when being pulled by peon node, it causes the task to fail. The failure occurs because the peon and other leaf nodes (broker, historical), have retry logic that continues to retry the lookup loading for 3 minutes by default. The http listener thread on the peon task is not started until lookup loading completes, by default, the overlord waits 1 minute by default, to communicate with the peon task to get the task status, after which is orders the task to shut down, causing the ingestion task to fail.
To fix the issue, we catch the exception serialization error, and do not retry. Also fixed an issue in which a bad lookup config interferes with any other good lookup configs from being loaded.
* Enhancements to IndexTaskClient.
1) Ability to use handlers other than StringFullResponseHandler. This
functionality is not used in production code yet, but is useful
because it will allow tasks to communicate with each other in
non-string-based formats and in streaming fashion. In the future,
we'll be able to use this to make task-to-task communication
more efficient.
2) Truncate server errors at 1KB, so long errors do not pollute logs.
3) Change error log level for retryable errors from WARN to INFO. (The
final error is still WARN.)
4) Harmonize log and exception messages to have a more consistent format.
* Additional tests and improvements.
This PR fixes a problem where the com.sun.jndi.ldap.Connection tries to build BasicSecuritySSLSocketFactory when calling LDAPCredentialsValidator.validateCredentials since BasicSecuritySSLSocketFactory is in extension class loader and not visible to system classloader.
changes:
* adds new config, druid.expressions.useStrictBooleans which make longs the official boolean type of all expressions
* vectorize logical operators and boolean functions, some only if useStrictBooleans is true
* Code cleanup from query profile project
* Fix spelling errors
* Fix Javadoc formatting
* Abstract out repeated test code
* Reuse constants in place of some string literals
* Fix up some parameterized types
* Reduce warnings reported by Eclipse
* Reverted change due to lack of tests
Add a "guessAggregatorHeapFootprint" method to AggregatorFactory that
mitigates #6743 by enabling heap footprint estimates based on a specific
number of rows. The idea is that at ingestion time, the number of rows
that go into an aggregator will be 1 (if rollup is off) or will likely
be a small number (if rollup is on).
It's a heuristic, because of course nothing guarantees that the rollup
ratio is a small number. But it's a common case, and I expect this logic
to go wrong much less often than the current logic. Also, when it does
go wrong, users can fix it by lowering maxRowsInMemory or
maxBytesInMemory. The current situation is unintuitive: when the
estimation goes wrong, users get an OOME, but actually they need to
*raise* these limits to fix it.