Suppress CVEs from dependencies with no available fix or false positives
hadoop-annotations: CVE-2022-25168, CVE-2021-33036
hadoop-client-runtime: CVE-2023-1370, CVE-2023-37475
okio: CVE-2023-3635
Upgrade grpc version to fix CVE-2023-33953
Currently, Druid is using Guava 16.0.1 version. This upgrade to 31.1-jre fixes the following issues.
CVE-2018-10237 (Unbounded memory allocation in Google Guava 11.0 through 24.x before 24.1.1 allows remote attackers to conduct denial of service attacks against servers that depend on this library and deserialize attacker-provided data because the AtomicDoubleArray class (when serialized with Java serialization) and the CompoundOrdering class (when serialized with GWT serialization) perform eager allocation without appropriate checks on what a client has sent and whether the data size is reasonable). We don't use Java or GWT serializations. Despite being false positive they're causing red security scans on Druid distribution.
Latest version of google-client-api is incompatible with the existing Guava version. This PR unblocks Update google client apis to latest version #14414
* Add supervisor /resetOffsets API.
- Add a new endpoint /druid/indexer/v1/supervisor/<supervisorId>/resetOffsets
which accepts DataSourceMetadata as a body parameter.
- Update logs, unit tests and docs.
* Add a new interface method for backwards compatibility.
* Rename
* Adjust tests and javadocs.
* Use CoreInjectorBuilder instead of deprecated makeInjectorWithModules
* UT fix
* Doc updates.
* remove extraneous debugging logs.
* Remove the boolean setting; only ResetHandle() and resetInternal()
* Relax constraints and add a new ResetOffsetsNotice; cleanup old logic.
* A separate ResetOffsetsNotice and some cleanup.
* Minor cleanup
* Add a check & test to verify that sequence numbers are only of type SeekableStreamEndSequenceNumbers
* Add unit tests for the no op implementations for test coverage
* CodeQL fix
* checkstyle from merge conflict
* Doc changes
* DOCUSAURUS code tabs fix. Thanks, Brian!
There are two type of DeterminePartitionsJob:
- When the input data is not assume grouped, there may be duplicate rows.
In this case, two MR jobs are launched. The first one do group job to remove duplicate rows.
And a second one to perform global sorting to find lower and upper bound for target segments.
- When the input data is assume grouped, we only need to launch the global sorting
MR job to find lower and upper bound for segments.
Sampling strategy:
- If the input data is assume grouped, sample by random at the mapper side of the global sort mr job.
- If the input data is not assume grouped, sample at the mapper of the group job. Use hash on time
and all dimensions and mod by sampling factor to sample, don't use random method because there
may be duplicate rows.
### Description
Added the following metrics, which are calculated from the `KillUnusedSegments` coordinatorDuty
`"killTask/availableSlot/count"`: calculates the number remaining task slots available for auto kill
`"killTask/maxSlot/count"`: calculates the maximum number of tasks available for auto kill
`"killTask/task/count"`: calculates the number of tasks submitted by auto kill.
#### Release note
NEW: metrics added for auto kill
`"killTask/availableSlot/count"`: calculates the number remaining task slots available for auto kill
`"killTask/maxSlot/count"`: calculates the maximum number of tasks available for auto kill
`"killTask/task/count"`: calculates the number of tasks submitted by auto kill.
* Updates `org.apache.jclouds:*` from 1.9.1 to 2.0.3
* Pin jclouds to 2.0.x since 2.1.x requires Guava 18+
* replace easymock with mockito
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
The current version of jackson-databind is flagged for vulnerabilities CVE-2020-28491 (Although cbor format is not used in druid), CVE-2020-36518 (Seems genuine as deeply nested json in can cause resource exhaustion). Updating the dependency to the latest version 2.12.7 to fix these vulnerabilities.
Changes:
* Add and invoke `StateListener` when state changes in `KubernetesPeonLifecycle`
* Report `task/pending/time` metric in `KubernetesTaskRunner` when state moves to RUNNING
* 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:
- Fix race condition in KubernetesTaskRunner introduced by #14435
- Perform addition and removal from map inside a synchronized block
- Update tests
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
This adds a new contrib extension: druid-iceberg-extensions which can be used to ingest data stored in Apache Iceberg format. It adds a new input source of type iceberg that connects to a catalog and retrieves the data files associated with an iceberg table and provides these data file paths to either an S3 or HDFS input source depending on the warehouse location.
Two important dependencies associated with Apache Iceberg tables are:
Catalog : This extension supports reading from either a Hive Metastore catalog or a Local file-based catalog. Support for AWS Glue is not available yet.
Warehouse : This extension supports reading data files from either HDFS or S3. Adapters for other cloud object locations should be easy to add by extending the AbstractInputSourceAdapter.
Apache Druid brings multiple direct and transitive dependencies that are affected by plethora of CVEs.
This PR attempts to update all the dependencies that did not require code refactoring.
This PR modifies pom files, license file and OWASP Dependency Check suppression file.
* Fix EarliestLatestBySqlAggregator signature; Include function name for all signatures.
* Single quote function signatures, space between args and remove \n.
* fixup UT assertion
In this PR, we are enhancing KafkaEmitter, to emit metadata about published segments (SegmentMetadataEvent) into a Kafka topic. This segment metadata information that gets published into Kafka, can be used by any other downstream services to query Druid intelligently based on the segments published. The segment metadata gets published into kafka topic in json string format similar to other events.
It was found that several supported tasks / input sources did not have implementations for the methods used by the input source security feature, causing these tasks and input sources to fail when used with this feature. This pr adds the needed missing implementations. Also securing the sampling endpoint with input source security, when enabled.
* Be able to load segments on Peons
This change introduces a new config on WorkerConfig
that indicates how many bytes of each storage
location to use for storage of a task. Said config
is divided up amongst the locations and slots
and then used to set TaskConfig.tmpStorageBytesPerTask
The Peons use their local task dir and
tmpStorageBytesPerTask as their StorageLocations for
the SegmentManager such that they can accept broadcast
segments.
* queue tasks if all slots in use
* Declare hamcrest-core dependency
* Use AtomicBoolean for shutdown requested
* Use AtomicReference for peon lifecycle state
* fix uninitialized read error
* fix indentations
* Make tasks protected
* fix KubernetesTaskRunnerConfig deserialization
* ensure k8s task runner max capacity is Integer.MAX_VALUE
* set job duration as task status duration
* Address pr comments
---------
Co-authored-by: George Shiqi Wu <george.wu@imply.io>
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.
Hadoop 2 often causes red security scans on Druid distribution because of the dependencies it brings. We want to move away from Hadoop 2 and provide Hadoop 3 distribution available. Switch druid to building with Hadoop 3 by default. Druid will still be compatible with Hadoop 2 and users can build hadoop-2 compatible distribution using hadoop2 profile.
* return task status reported by peon
* Write TaskStatus to file in AbstractTask.cleanUp
* Get TaskStatus from task log
* Fix merge conflicts in AbstractTaskTest
* Add unit tests for TaskLogPusher, TaskLogStreamer, NoopTaskLogs to satisfy code coverage
* Add license headerss
* Fix style
* Remove unknown exception declarations
* Make the tasks run with only a single directory
There was a change that tried to get indexing to run on multiple disks
It made a bunch of changes to how tasks run, effectively hiding the
"safe" directory for tasks to write files into from the task code itself
making it extremely difficult to do anything correctly inside of a task.
This change reverts those changes inside of the tasks and makes it so that
only the task runners are the ones that make decisions about which
mount points should be used for storing task-related files.
It adds the config druid.worker.baseTaskDirs which can be used by the
task runners to know which directories they should schedule tasks inside of.
The TaskConfig remains the authoritative source of configuration for where
and how an individual task should be operating.
* Always use file sizes when determining batch ingest splits.
Main changes:
1) Update CloudObjectInputSource and its subclasses (S3, GCS,
Azure, Aliyun OSS) to use SplitHintSpecs in all cases. Previously, they
were only used for prefixes, not uris or objects.
2) Update ExternalInputSpecSlicer (MSQ) to consider file size. Previously,
file size was ignored; all files were treated as equal weight when
determining splits.
A side effect of these changes is that we'll make additional network
calls to find the sizes of objects when users specify URIs or objects
as opposed to prefixes. IMO, this is worth it because it's the only way
to respect the user's split hint and task assignment settings.
Secondary changes:
1) S3, Aliyun OSS: Use getObjectMetadata instead of listObjects to get
metadata for a single object. This is a simpler call that is also
expected to be less expensive.
2) Azure: Fix a bug where getBlobLength did not populate blob
reference attributes, and therefore would not actually retrieve the
blob length.
3) MSQ: Align dynamic slicing logic between ExternalInputSpecSlicer and
TableInputSpecSlicer.
4) MSQ: Adjust WorkerInputs to ensure there is always at least one
worker, even if it has a nil slice.
* Add msqCompatible to testGroupByWithImpossibleTimeFilter.
* Fix tests.
* Add additional tests.
* Remove unused stuff.
* Remove more unused stuff.
* Adjust thresholds.
* Remove irrelevant test.
* Fix comments.
* Fix bug.
* Updates.
With the KubernetesTaskRunner, if a task is manually shutdown via the web console while running or the corresponding k8s job is manually deleted, the thread responsible for overseeing the task gets stuck in a loop because the fabric8 client sends one event to it that the job is null when the job is deleted, but this doesn't pass the condition.
This means that the thread is stuck waiting on a fabric8 event (the job being successful) that will never come up until maxTaskDuration (default 4 hours). If a user of the extension is trying to use a limited taskqueue maxSize, this can cause problems as the k8s executor pool is unable to pick up additional tasks (since threads are stuck waiting on the old tasks that have already been deleted).
* Hook up PodTemplateTaskAdapter
* Make task adapter TYPE parameters final
* Rename adapters types
* Include specified adapter name in exception message
* Documentation for sidecarSupport deprecation
* Fix order
* Set TASK_ID as environment variable in PodTemplateTaskAdapter (#13969)
* Update docs/development/extensions-contrib/k8s-jobs.md
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
* Hook up PodTemplateTaskAdapter
* Make task adapter TYPE parameters final
* Rename adapters types
* Include specified adapter name in exception message
* Documentation for sidecarSupport deprecation
* Fix order
* fix spelling errors
---------
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
* Pod template task adapter
* Use getBaseTaskDirPaths
* Remove unused task from getEnv
* Use Optional.ifPresent() instead of Optional.map()
* Pass absolute path
* Don't pass task to getEnv
* Assert the correct adapter is created
* Javadocs and Comments
* Add exception message to assertions
* Use TaskConfig to get task dir in KubernetesTaskRunner
* Use the first path specified in baseTaskDirPaths instead of deprecated baseTaskDirPath
* Use getBaseTaskDirPaths in generate command
The FiniteFirehoseFactory and InputRowParser classes were deprecated in 0.17.0 (#8823) in favor of InputSource & InputFormat. This PR removes the FiniteFirehoseFactory and all its implementations along with classes solely used by them like Fetcher (Used by PrefetchableTextFilesFirehoseFactory). Refactors classes including tests using FiniteFirehoseFactory to use InputSource instead.
Removing InputRowParser may not be as trivial as many classes that aren't deprecated depends on it (with no alternatives), like EventReceiverFirehoseFactory. Hence FirehoseFactory, EventReceiverFirehoseFactory, and Firehose are marked deprecated.
* 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
* Better sidecar support
* remove un-thrown exception from test
* Druid you are such a stickler about spelling :)
* Only require the primaryContainerName, no need to exclude containers
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
This commit adds a new class `InputStats` to track the total bytes processed by a task.
The field `processedBytes` is published in task reports along with other row stats.
Major changes:
- Add class `InputStats` to track processed bytes
- Add method `InputSourceReader.read(InputStats)` to read input rows while counting bytes.
> Since we need to count the bytes, we could not just have a wrapper around `InputSourceReader` or `InputEntityReader` (the way `CountableInputSourceReader` does) because the `InputSourceReader` only deals with `InputRow`s and the byte information is already lost.
- Classic batch: Use the new `InputSourceReader.read(inputStats)` in `AbstractBatchIndexTask`
- Streaming: Increment `processedBytes` in `StreamChunkParser`. This does not use the new `InputSourceReader.read(inputStats)` method.
- Extend `InputStats` with `RowIngestionMeters` so that bytes can be exposed in task reports
Other changes:
- Update tests to verify the value of `processedBytes`
- Rename `MutableRowIngestionMeters` to `SimpleRowIngestionMeters` and remove duplicate class
- Replace `CacheTestSegmentCacheManager` with `NoopSegmentCacheManager`
- Refactor `KafkaIndexTaskTest` and `KinesisIndexTaskTest`
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.
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
https://github.com/apache/druid/pull/13027 PR replaces `filter` parameter with
`objectGlob` in ingestion input source. However, this will cause existing ingestion
jobs to fail if they are using a filter already. This PR adds old filter functionality
alongside objectGlob to preserve backward compatibility.
We added compression to the latest/first pair storage, but
the code change was forcing new things to be persisted
with the new format, meaning that any segment created with
the new code cannot be read by the old code. Instead, we
need to default to creating the old format and then remove that default in a future version.
* Use standard library to correctly glob and stop at the correct folder structure when filtering cloud objects.
Removed:
import org.apache.commons.io.FilenameUtils;
Add:
import java.nio.file.FileSystems;
import java.nio.file.PathMatcher;
import java.nio.file.Paths;
* Forgot to update CloudObjectInputSource as well.
* Fix tests.
* Removed unused exceptions.
* Able to reduced user mistakes, by removing the protocol and the bucket on filter.
* add 1 more test.
* add comment on filterWithoutProtocolAndBucket
* Fix lint issue.
* Fix another lint issue.
* Replace all mention of filter -> objectGlob per convo here:
https://github.com/apache/druid/pull/13027#issuecomment-1266410707
* fix 1 bad constructor.
* Fix the documentation.
* Don’t do anything clever with the object path.
* Remove unused imports.
* Fix spelling error.
* Fix incorrect search and replace.
* Addressing Gian’s comment.
* add filename on .spelling
* Fix documentation.
* fix documentation again
Co-authored-by: Didip Kerabat <didip@apple.com>
Changes:
- Add a metric for partition-wise kafka/kinesis lag for streaming ingestion.
- Emit lag metrics for streaming ingestion when supervisor is not suspended and state is in {RUNNING, IDLE, UNHEALTHY_TASKS, UNHEALTHY_SUPERVISOR}
- Document metrics
* Support for middle manager less druid, tasks launch as k8s jobs
* Fixing forking task runner test
* Test cleanup, dependency cleanup, intellij inspections cleanup
* Changes per PR review
Add configuration option to disable http/https proxy for the k8s client
Update the docs to provide more detail about sidecar support
* Removing un-needed log lines
* Small changes per PR review
* Upon task completion we callback to the overlord to update the status / locaiton, for slower k8s clusters, this reduces locking time significantly
* Merge conflict fix
* Fixing tests and docs
* update tiny-cluster.yaml
changed `enableTaskLevelLogPush` to `encapsulatedTask`
* Apply suggestions from code review
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
* Minor changes per PR request
* Cleanup, adding test to AbstractTask
* Add comment in peon.sh
* Bumping code coverage
* More tests to make code coverage happy
* Doh a duplicate dependnecy
* Integration test setup is weird for k8s, will do this in a different PR
* Reverting back all integration test changes, will do in anotbher PR
* use StringUtils.base64 instead of Base64
* Jdk is nasty, if i compress in jdk 11 in jdk 17 the decompressed result is different
Co-authored-by: Rahul Gidwani <r_gidwani@apple.com>
Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
* 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
It was found that the namespace/cache/heapSizeInBytes metric that tracks the total heap size in bytes of all lookup caches loaded on a service instance was being under reported. We were not accounting for the memory overhead of the String object, which I've found in testing to be ~40 bytes. While this overhead may be java version dependent, it should not vary much, and accounting for this provides a better estimate. Also fixed some logging, and reading bytes from the JDBI result set a little more efficient by saving hash table lookups. Also added some of the lookup metrics to the default statsD emitter metric whitelist.
This adds min/max functions for CompressedBigDecimal. It exposes these
functions via sql (BIG_MAX, BIG_MIN--see the SqlAggFunction
implementations).
It also includes various bug fixes and cleanup to the original
CompressedBigDecimal code include the AggregatorFactories. Various null
handling was improved.
Additional test cases were added for both new and existing code
including a base test case for AggregationFactories. Other tests common
across sum,min,max may be refactored also to share the varoius cases in
the future.
This adds a sql function, "BIG_SUM", that uses
CompressedBigDecimal to do a sum. Other misc changes:
1. handle NumberFormatExceptions when parsing a string (default to set
to 0, configurable in agg factory to be strict and throw on error)
2. format pom file (whitespace) + add dependency
3. scaleUp -> scale and always require scale as a parameter
Optimizes the compareTo() function in
CompressedBigDecimal. It directly compares the int[] rather than
creating BigDecimal objects and using its compareTo.
It handles unequal sized CBDs, but does require
the scales to match.
1. remove unnecessary generic type from CompressedBigDecimal
2. support Number input types
3. support aggregator reading supported input types directly (uningested
data)
4. fix scaling bug in buffer aggregator
* prometheus-emitter supports sending metrics to pushgateway regularly and continuously
* spell check fix
* Optimization variable name and related documents
* Update docs/development/extensions-contrib/prometheus.md
OK, it looks more conspicuous
Co-authored-by: Frank Chen <frankchen@apache.org>
* Update doc
* Update docs/development/extensions-contrib/prometheus.md
Co-authored-by: Frank Chen <frankchen@apache.org>
* When PrometheusEmitter is closed, close the scheduler
* Ensure that registeredMetrics is thread safe.
* Local variable name optimization
* Remove unnecessary white space characters
Co-authored-by: Frank Chen <frankchen@apache.org>
Compressed Big Decimal is an extension which provides support for
Mutable big decimal value that can be used to accumulate values
without losing precision or reallocating memory. This type helps in
absolute precision arithmetic on large numbers in applications,
where greater level of accuracy is required, such as financial
applications, currency based transactions. This helps avoid rounding
issues where in potentially large amount of money can be lost.
Accumulation requires that the two numbers have the same scale,
but does not require that they are of the same size. If the value
being accumulated has a larger underlying array than this value
(the result), then the higher order bits are dropped, similar to what
happens when adding a long to an int and storing the result in an
int. A compressed big decimal that holds its data with an embedded
array.
Compressed big decimal is an absolute number based complex type
based on big decimal in Java. This supports all the functionalities
supported by Java Big Decimal. Java Big Decimal is not mutable in
order to avoid big garbage collection issues. Compressed big decimal
is needed to mutate the value in the accumulator.
* Fixing RACE in HTTP remote task Runner
* Changes in the interface
* Updating documentation
* Adding test cases to SwitchingTaskLogStreamer
* Adding more tests
Fixes KafkaEmitter not emitting queryType for a native query. The Event to JSON serialization was extracted to the external class: EventToJsonSerializer. This was done to simplify the testing logic for the serialization as well as extract the responsibility of serialization to the separate class.
The logic builds ObjectNode incrementally based on the event .toMap method. Parsing each entry individually ensures that the Jackson polymorphic annotations are respected. Not respecting these annotation caused the missing of the queryType from output event.
* 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
In a heterogeneous environment, sometimes you don't have control over the input folder. Upstream can put any folder they want. In this situation the S3InputSource.java is unusable.
Most people like me solved it by using Airflow to fetch the full list of parquet files and pass it over to Druid. But doing this explodes the JSON spec. We had a situation where 1 of the JSON spec is 16MB and that's simply too much for Overlord.
This patch allows users to pass {"filter": "*.parquet"} and let Druid performs the filtering of the input files.
I am using the glob notation to be consistent with the LocalFirehose syntax.
The web-console (indirectly) calls the Overlord’s GET tasks API to fetch the tasks' summary which in turn queries the metadata tasks table. This query tries to fetch several columns, including payload, of all the rows at once. This introduces a significant memory overhead and can cause unresponsiveness or overlord failure when the ingestion tab is opened multiple times (due to several parallel calls to this API)
Another thing to note is that the task table (the payload column in particular) can be very large. Extracting large payloads from such tables can be very slow, leading to slow UI. While we are fixing the memory pressure in the overlord, we can also fix the slowness in UI caused by fetching large payloads from the table. Fetching large payloads also puts pressure on the metadata store as reported in the community (Metadata store query performance degrades as the tasks in druid_tasks table grows · Issue #12318 · apache/druid )
The task summaries returned as a response for the API are several times smaller and can fit comfortably in memory. So, there is an opportunity here to fix the memory usage, slow ingestion, and under-pressure metadata store by removing the need to handle large payloads in every layer we can. Of course, the solution becomes complex as we try to fix more layers. With that in mind, this page captures two approaches. They vary in complexity and also in the degree to which they fix the aforementioned problems.
Often users are submitting queries, and ingestion specs that work only if the relevant extension is not loaded. However, the error is too technical for the users and doesn't suggest them to check for missing extensions. This PR modifies the error message so users can at least check their settings before assuming that the error is because of a bug.
* 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.
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.
The latest version of Error Prone now requires Java 11. Upgrading means we can
remove a lot of the maven profile complexity required to run checks with Java 8.
This also requires switching our strict build to use Java 11.
* update error-prone to 2.11
* remove need for specific maven profiles for Java 8 and Java 15
* fix additional Error Prone warnings with Java 11
* update strict build to use Java 11
* 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
* 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
* Refactor ResponseContext
Fixes a number of issues in preparation for request trailers
and the query profile.
* Converts keys from an enum to classes for smaller code
* Wraps stored values in functions for easier capture for other uses
* Reworks the "header squeezer" to handle types other than arrays.
* Uses metadata for visibility, and ability to compress,
to replace ad-hoc code.
* Cleans up JSON serialization for the response context.
* Other miscellaneous cleanup.
* Handle unknown keys in deserialization
Also, make "Visibility" into a boolean.
* Revised comment
* Renamd variable
Druid currently has 2 serverViews, regular serverView and filtered serverView. The regular serverView is used to monitor all segment announcements from all data nodes (historicals, tasks, indexers). The filtered serverView is used when you want to watch segment announcements from particular tiers. Since these server views keep track of different sets of druidServers and segments in memory, they should be maintained separately. However, they currently share the same name for their executorService, which can cause confusion and make debugging harder especially in the broker since it is using both serverViews, the filtered view for normal query processing and the regular view to serve the servers table (I'm unsure whether this is intended or whether this is a good behavior). This PR changes it to a more obvious name.
This PR also removes SingleServerInventoryView. This view was deprecated a long time ago and has not been documented at least since 0.13 (#6127). I also don't think this can be better in any case than BatchServerInventoryView. Finally, I merged AbstractCuratorServerInventoryView and BatchServerInventoryView as we no longer need AbstractCuratorServerInventoryView after SingleServerInventoryView is removed.
* Make nodeRole available during binding; add support for dynamic registration of DruidService
* fix checkstyle and test
* fix customRole test
* address comments
* add more javadoc
* Consolidate a bunch of ad-hoc segments metadata SQL; fix some bugs.
This patch gathers together a variety of SQL from SqlSegmentsMetadataManager
and IndexerSQLMetadataStorageCoordinator into a new class SqlSegmentsMetadataQuery.
It focuses on SQL related to retrieving segment payloads and marking
segments used and unused.
In addition to cleaning up the code a bit, this patch also fixes a bug
with years before 0 or after 9999. The prior SQL did not work properly
because dates outside this range cannot be compared as strings. The new
code does work for these far-past and far-future years.
So, if you're ever interested in using Druid to analyze things from
ancient Babylon, you better apply this patch first!
* Fix test compiling.
* Fixes and improvements.
* Fix forbidden API.
* Additional fixes.
* add back and deprecate aggregator factory methods so i can say i told you so when i delete these later
* rename to make less ambiguous, fix fill method
* adjust
* Add worker category as dimension in TaskSlotCountStatsMonitor
* Change description
* Add workerConfig as field
* Modify HttpRemoteTaskRunnerTest to test worker category in taskslot metrics
* Fixing tests
* Fixing alerts
* Adding unit test in SingleTaskBackgroundRunnerTest for task slot metrics APIs
* Resolving false positive spell check
* addressing comments
* throw UnsupportedOperationException for tasklotmetrics APIs in SingleTaskBackgroundRunner
Co-authored-by: Nikhil Navadiya <nnavadiya@twitter.com>
* add missing json type for ListFilteredVirtualColumn, and tests to try to avoid this happening again
* fixes
* ugly, but maybe this
* oops
* too many mappers
Add support for hadoop 3 profiles . Most of the details are captured in #11791 .
We use a combination of maven profiles and resource filtering to achieve this. Hadoop2 is supported by default and a new maven profile with the name hadoop3 is created. This will allow the user to choose the profile which is best suited for the use case.
* add ColumnInspector argument to PostAggregator.getType to allow post-aggs to compute their output type based on input types
* add test for test for coverage
* simplify
* Remove unused imports.
Co-authored-by: Gian Merlino <gian@imply.io>
* better type system
* needle in a haystack
* ColumnCapabilities is a TypeSignature instead of having one, INFORMATION_SCHEMA support
* fixup merge
* more test
* fixup
* intern
* fix
* oops
* oops again
* ...
* more test coverage
* fix error message
* adjust interning, more javadocs
* oops
* more docs more better