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
This PR adds a way to store the topic name in a column. Such a column can be used to distinguish messages coming from different topics in multi-topic ingestion.
* 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!
In this PR, I have gotten rid of multiTopic parameter and instead added a topicPattern parameter. Kafka supervisor will pass topicPattern or topic as the stream name to the core ingestion engine. There is validation to ensure that only one of topic or topicPattern will be set. This new setting is easier to understand than overloading the topic field that earlier could be interpreted differently depending on the value of some other field.
This PR adds support to read from multiple Kafka topics in the same supervisor. A multi-topic ingestion can be useful in scenarios where a cluster admin has no control over input streams. Different teams in an org may create different input topics that they can write the data to. However, the cluster admin wants all this data to be queryable in one data source.
* Rolling supervior task publishing
* add an option for number of task groups to roll over
* better
* remove docs
* oops
* checkstyle
* wip test
* undo partial test change
* remove incomplete test
* Remove chatAsync parameter, so chat is always async.
chatAsync has been made default in Druid 26. I have seen good
battle-testing of it in production, and am comfortable removing the
older sync client.
This was the last remaining usage of IndexTaskClient, so this patch
deletes all that stuff too.
* Remove unthrown exception.
* Remove unthrown exception.
* No more TimeoutException.
* Add ingest/input/bytes metric and Kafka consumer metrics.
New metrics:
1) ingest/input/bytes. Equivalent to processedBytes in the task reports.
2) kafka/consumer/bytesConsumed: Equivalent to the Kafka consumer
metric "bytes-consumed-total". Only emitted for Kafka tasks.
3) kafka/consumer/recordsConsumed: Equivalent to the Kafka consumer
metric "records-consumed-total". Only emitted for Kafka tasks.
* Fix anchor.
* Fix KafkaConsumerMonitor.
* Interface updates.
* Doc changes.
* Update indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java
Co-authored-by: Benedict Jin <asdf2014@apache.org>
---------
Co-authored-by: Benedict Jin <asdf2014@apache.org>
* Change default handoffConditionTimeout to 15 minutes.
Most of the time, when handoff is taking this long, it's because something
is preventing Historicals from loading new data. In this case, we have
two choices:
1) Stop making progress on ingestion, wait for Historicals to load stuff,
and keep the waiting-for-handoff segments available on realtime tasks.
(handoffConditionTimeout = 0, the current default)
2) Continue making progress on ingestion, by exiting the realtime tasks
that were waiting for handoff. Once the Historicals get their act
together, the segments will be loaded, as they are still there on
deep storage. They will just not be continuously available.
(handoffConditionTimeout > 0)
I believe most users would prefer [2], because [1] risks ingestion falling
behind the stream, which causes many other problems. It can cause data loss
if the stream ages-out data before we have a chance to ingest it.
Due to the way tuningConfigs are serialized -- defaults are baked into the
serialized form that is written to the database -- this default change will
not change anyone's existing supervisors. It will take effect for newly
created supervisors.
* Fix tests.
* Update docs/development/extensions-core/kafka-supervisor-reference.md
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
* Update docs/development/extensions-core/kinesis-ingestion.md
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
---------
Co-authored-by: Katya Macedo <38017980+ektravel@users.noreply.github.com>
* Add ZooKeeper connection state alerts and metrics.
- New metric "zk/connected" is an indicator showing 1 when connected,
0 when disconnected.
- New metric "zk/disconnected/time" measures time spent disconnected.
- New alert when Curator connection state enters LOST or SUSPENDED.
* Use right GuardedBy.
* Test fixes, coverage.
* Adjustment.
* Fix tests.
* Fix ITs.
* Improved injection.
* Adjust metric name, add tests.
* fix kafka input format reader schema discovery and partial schema discovery to actually work right, by re-using dimension filtering logic of MapInputRowParser
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.
In StreamChunkParser#parseWithInputFormat, we call byteEntityReader.read() without handling a potential ParseException, which is thrown during this function call by the delegate AvroStreamReader#intermediateRowIterator.
A ParseException can be thrown if an Avro stream has corrupt data or data that doesn't conform to the schema specified or for other decoding reasons. This exception if uncaught, can cause ingestion to fail.
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.
### Description
This pr fixes a few bugs found with the inputSource security feature.
1. `KillUnusedSegmentsTask` previously had no definition for the `getInputSourceResources`, which caused an unsupportedOperationException to be thrown when this task type was submitted with the inputSource security feature enabled. This task type should not require any input source specific resources, so returning an empty set for this task type now.
2. Fixed a bug where when the input source type security feature is enabled, all of the input source type specific resources used where authenticated against:
`{"resource": {"name": "EXTERNAL", "type": "{INPUT_SOURCE_TYPE}"}, "action": "READ"}`
When they should be instead authenticated against:
`{"resource": {"name": "{INPUT_SOURCE_TYPE}", "type": "EXTERNAL"}, "action": "READ"}`
3. fixed bug where supervisor tasks were not authenticated against the specific input source types used, if input source security feature was enabled.
Fixes#13837.
### Description
This change allows for input source type security in the native task layer.
To enable this feature, the user must set the following property to true:
`druid.auth.enableInputSourceSecurity=true`
The default value for this property is false, which will continue the existing functionality of needing authorization to write to the respective datasource.
When this config is enabled, the users will be required to be authorized for the following resource action, in addition to write permission on the respective datasource.
`new ResourceAction(new Resource(ResourceType.EXTERNAL, {INPUT_SOURCE_TYPE}, Action.READ`
where `{INPUT_SOURCE_TYPE}` is the type of the input source being used;, http, inline, s3, etc..
Only tasks that provide a non-default implementation of the `getInputSourceResources` method can be submitted when config `druid.auth.enableInputSourceSecurity=true` is set. Otherwise, a 400 error will be thrown.
* Lower default maxRowsInMemory for realtime ingestion.
The thinking here is that for best ingestion throughput, we want
intermediate persists to be as big as possible without using up all
available memory. So, we rely mainly on maxBytesInMemory. The default
maxRowsInMemory (1 million) is really just a safety: in case we have
a large number of very small rows, we don't want to get overwhelmed
by per-row overheads.
However, maximum ingestion throughput isn't necessarily the primary
goal for realtime ingestion. Query performance is also important. And
because query performance is not as good on the in-memory dataset, it's
helpful to keep it from growing too large. 150k seems like a reasonable
balance here. It means that for a typical 5 million row segment, we
won't trigger more than 33 persists due to this limit, which is a
reasonable number of persists.
* Update tests.
* Update server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
* Fix test.
* Fix link.
---------
Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
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.
If the intermediate handoff period is less than the task duration and there is no new data in the input topic, task will continuously checkpoint the same offsets again and again. This PR fixes that bug by resetting the checkpoint time even when the task receives the same end offset request again.
* 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
* discover nested columns when using nested column indexer for schemaless
* move useNestedColumnIndexerForSchemaDiscovery from AppendableIndexSpec to DimensionsSpec
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`
* Zero-copy local deep storage.
This is useful for local deep storage, since it reduces disk usage and
makes Historicals able to load segments instantaneously.
Two changes:
1) Introduce "druid.storage.zip" parameter for local storage, which defaults
to false. This changes default behavior from writing an index.zip to writing
a regular directory. This is safe to do even during a rolling update, because
the older code actually already handled unzipped directories being present
on local deep storage.
2) In LocalDataSegmentPuller and LocalDataSegmentPusher, use hard links
instead of copies when possible. (Generally this is possible when the
source and destination directory are on the same filesystem.)
Main changes:
1) Convert SeekableStreamIndexTaskClient to an interface, move old code
to SeekableStreamIndexTaskClientSyncImpl, and add new implementation
SeekableStreamIndexTaskClientAsyncImpl that uses ServiceClient.
2) Add "chatAsync" parameter to seekable stream supervisors that causes
the supervisor to use an async task client.
3) In SeekableStreamSupervisor.discoverTasks, adjust logic to avoid making
blocking RPC calls in workerExec threads.
4) In SeekableStreamSupervisor generally, switch from Futures.successfulAsList
to FutureUtils.coalesce, so we can better capture the errors that occurred
with contacting individual tasks.
Other, related changes:
1) Add ServiceRetryPolicy.retryNotAvailable, which controls whether
ServiceClient retries unavailable services. Useful since we do not
want to retry calls unavailable tasks within the service client. (The
supervisor does its own higher-level retries.)
2) Add FutureUtils.transformAsync, a more lambda friendly version of
Futures.transform(f, AsyncFunction).
3) Add FutureUtils.coalesce. Similar to Futures.successfulAsList, but
returns Either instead of using null on error.
4) Add JacksonUtils.readValue overloads for JavaType and TypeReference.
Fixes inclusion of all stream partitions in all tasks.
The PR (Adds Idle feature to `SeekableStreamSupervisor` for inactive stream) - https://github.com/apache/druid/pull/13144 updates the resulting lag calculation map in `KafkaSupervisor` to include all the latest partitions from the stream to set the idle state accordingly rather than the previous way of lag calculation only for the partitions actively being read from the stream. This led to an explosion of metrics in lag reports in cases where 1000s of tasks per supervisor are present.
Changes:
- Add a new method to generate lags for only those partitions a single task is actively reading from while updating the Supervisor reports.
* 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>
* Remove basePersistDirectory from tuning configs.
Since the removal of CliRealtime, it serves no purpose, since it is
always overridden in production using withBasePersistDirectory given
some subdirectory of the task work directory.
Removing this from the tuning config has a benefit beyond removing
no-longer-needed logic: it also avoids the side effect of empty
"druid-realtime-persist" directories getting created in the systemwide
temp directory.
* Test adjustments to appropriately set basePersistDirectory.
* Remove unused import.
* Fix RATC constructor.