* Mid-level service client and updated high-level clients.
Our servers talk to each other over HTTP. We have a low-level HTTP
client (HttpClient) that is super-asynchronous and super-customizable
through its handlers. It's also proven to be quite robust: we use it
for Broker -> Historical communication over the wide variety of query
types and workloads we support.
But the low-level client has no facilities for service location or
retries, which means we have a variety of high-level clients that
implement these in their own ways. Some high-level clients do a better
job than others. This patch adds a mid-level ServiceClient that makes
it easier for high-level clients to be built correctly and harmoniously,
and migrates some of the high-level logic to use ServiceClients.
Main changes:
1) Add ServiceClient org.apache.druid.rpc package. That package also
contains supporting stuff like ServiceLocator and RetryPolicy
interfaces, and a DiscoveryServiceLocator based on
DruidNodeDiscoveryProvider.
2) Add high-level OverlordClient in org.apache.druid.rpc.indexing.
3) Indexing task client creator in TaskServiceClients. It uses
SpecificTaskServiceLocator to find the tasks. This improves on
ClientInfoTaskProvider by caching task locations for up to 30 seconds
across calls, reducing load on the Overlord.
4) Rework ParallelIndexSupervisorTaskClient to use a ServiceClient
instead of extending IndexTaskClient.
5) Rework RemoteTaskActionClient to use a ServiceClient instead of
DruidLeaderClient.
6) Rework LocalIntermediaryDataManager, TaskMonitor, and
ParallelIndexSupervisorTask. As a result, MiddleManager, Peon, and
Overlord no longer need IndexingServiceClient (which internally used
DruidLeaderClient).
There are some concrete benefits over the prior logic, namely:
- DruidLeaderClient does retries in its "go" method, but only retries
exactly 5 times, does not sleep between retries, and does not retry
retryable HTTP codes like 502, 503, 504. (It only retries IOExceptions.)
ServiceClient handles retries in a more reasonable way.
- DruidLeaderClient's methods are all synchronous, whereas ServiceClient
methods are asynchronous. This is used in one place so far: the
SpecificTaskServiceLocator, so we don't need to block a thread trying
to locate a task. It can be used in other places in the future.
- HttpIndexingServiceClient does not properly handle all server errors.
In some cases, it tries to parse a server error as a successful
response (for example: in getTaskStatus).
- IndexTaskClient currently makes an Overlord call on every task-to-task
HTTP request, as a way to find where the target task is. ServiceClient,
through SpecificTaskServiceLocator, caches these target locations
for a period of time.
* Style adjustments.
* For the coverage.
* Adjustments.
* Better behaviors.
* Fixes.
* Fix flaky KafkaIndexTaskTest.
The testRunTransactionModeRollback case had many race conditions. Most notably,
it would commit a transaction and then immediately check to see that the results
were *not* indexed. This is racey because it relied on the indexing thread being
slower than the test thread.
Now, the case waits for the transaction to be processed by the indexing thread
before checking the results.
* Changes from review.
* 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
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).
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.
* 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.
### Description
Today we ingest a number of high cardinality metrics into Druid across dimensions. These metrics are rolled up on a per minute basis, and are very useful when looking at metrics on a partition or client basis. Events is another class of data that provides useful information about a particular incident/scenario inside a Kafka cluster. Events themselves are carried inside kafka payload, but nonetheless there are some very useful metadata that is carried in kafka headers that can serve as useful dimension for aggregation and in turn bringing better insights.
PR(https://github.com/apache/druid/pull/10730) introduced support of Kafka headers in InputFormats.
We still need an input format to parse out the headers and translate those into relevant columns in Druid. Until that’s implemented, none of the information available in the Kafka message headers would be exposed. So first there is a need to write an input format that can parse headers in any given format(provided we support the format) like we parse payloads today. Apart from headers there is also some useful information present in the key portion of the kafka record. We also need a way to expose the data present in the key as druid columns. We need a generic way to express at configuration time what attributes from headers, key and payload need to be ingested into druid. We need to keep the design generic enough so that users can specify different parsers for headers, key and payload.
This PR is designed to solve the above by providing wrapper around any existing input formats and merging the data into a single unified Druid row.
Lets look at a sample input format from the above discussion
"inputFormat":
{
"type": "kafka", // New input format type
"headerLabelPrefix": "kafka.header.", // Label prefix for header columns, this will avoid collusions while merging columns
"recordTimestampLabelPrefix": "kafka.", // Kafka record's timestamp is made available in case payload does not carry timestamp
"headerFormat": // Header parser specifying that values are of type string
{
"type": "string"
},
"valueFormat": // Value parser from json parsing
{
"type": "json",
"flattenSpec": {
"useFieldDiscovery": true,
"fields": [...]
}
},
"keyFormat": // Key parser also from json parsing
{
"type": "json"
}
}
Since we have independent sections for header, key and payload, it will enable parsing each section with its own parser, eg., headers coming in as string and payload as json.
KafkaInputFormat will be the uber class extending inputFormat interface and will be responsible for creating individual parsers for header, key and payload, blend the data resolving conflicts in columns and generating a single unified InputRow for Druid ingestion.
"headerFormat" will allow users to plug parser type for the header values and will add default header prefix as "kafka.header."(can be overridden) for attributes to avoid collision while merging attributes with payload.
Kafka payload parser will be responsible for parsing the Value portion of the Kafka record. This is where most of the data will come from and we should be able to plugin existing parser. One thing to note here is that if batching is performed, then the code is augmenting header and key values to every record in the batch.
Kafka key parser will handle parsing Key portion of the Kafka record and will ingest the Key with dimension name as "kafka.key".
## KafkaInputFormat Class:
This is the class that orchestrates sending the consumerRecord to each parser, retrieve rows, merge the columns into one final row for Druid consumption. KafkaInputformat should make sure to release the resources that gets allocated as a part of reader in CloseableIterator<InputRow> during normal and exception cases.
During conflicts in dimension/metrics names, the code will prefer dimension names from payload and ignore the dimension either from headers/key. This is done so that existing input formats can be easily migrated to this new format without worrying about losing information.
* Make persists concurrent with ingestion
* Remove semaphore but keep concurrent persists (with add) and add push in the backround as well
* Go back to documented default persists (zero)
* Move to debug
* Remove unnecessary Atomics
* Comments on synchronization (or not) for sinks & sinkMetadata
* Some cleanup for unit tests but they still need further work
* Shutdown & wait for persists and push on close
* Provide support for three existing batch appenderators using batchProcessingMode flag
* Fix reference to wrong appenderator
* Fix doc typos
* Add BatchAppenderators class test coverage
* Add log message to batchProcessingMode final value, fix typo in enum name
* Another typo and minor fix to log message
* LEGACY->OPEN_SEGMENTS, Edit docs
* Minor update legacy->open segments log message
* More code comments, mostly small adjustments to naming etc
* fix spelling
* Exclude BtachAppenderators from Jacoco since it is fully tested but Jacoco still refuses to ack coverage
* Coverage for Appenderators & BatchAppenderators, name change of a method that was still using "legacy" rather than "openSegments"
Co-authored-by: Clint Wylie <cjwylie@gmail.com>
* Add error msg to parallel task's TaskStatus
* Consolidate failure block
* Add failure test
* Make it fail
* Add fail while stopped
* Simplify hash task test using a runner that fails after so many runs (parameter)
* Remove unthrown exception
* Use runner names to identify phase
* Added range partition kill test & fixed a timing bug with the custom runner
* Forbidden api
* Style
* Unit test code cleanup
* Added message to invalid state exception and improved readability of the phase error messages for the parallel task failure unit tests
This PR splits current SegmentLoader into SegmentLoader and SegmentCacheManager.
SegmentLoader - this class is responsible for building the segment object but does not expose any methods for downloading, cache space management, etc. Default implementation delegates the download operations to SegmentCacheManager and only contains the logic for building segments once downloaded. . This class will be used in SegmentManager to construct Segment objects.
SegmentCacheManager - this class manages the segment cache on the local disk. It fetches the segment files to the local disk, can clean up the cache, and in the future, support reserve and release on cache space. [See https://github.com/Make SegmentLoader extensible and customizable #11398]. This class will be used in ingestion tasks such as compaction, re-indexing where segment files need to be downloaded locally.
This PR refactors the code for QueryRunnerFactory#mergeRunners to accept a new interface called QueryProcessingPool instead of ExecutorService for concurrent execution of query runners. This interface will let custom extensions inject their own implementation for deciding which query-runner to prioritize first. The default implementation is the same as today that takes the priority of query into account. QueryProcessingPool can also be used as a regular executor service. It has a dedicated method for accepting query execution work so implementations can differentiate between regular async tasks and query execution tasks. This dedicated method also passes the QueryRunner object as part of the task information. This hook will let custom extensions carry any state from QuerySegmentWalker to QueryProcessingPool#mergeRunners which is not possible currently.
With this change, Druid will only support ZooKeeper 3.5.x and later.
In order to support Java 15 we need to switch to ZK 3.5.x client libraries and drop support for ZK 3.4.x
(see #10780 for the detailed reasons)
* remove ZooKeeper 3.4.x compatibility
* exclude additional ZK 3.5.x netty dependencies to ensure we use our version
* keep ZooKeeper version used for integration tests in sync with client library version
* remove the need to specify ZK version at runtime for docker
* add support to run integration tests with JDK 15
* build and run unit tests with Java 15 in travis
* Avoid mapping hydrants in create segments phase for native ingestion
* Drop queriable indices after a given sink is fully merged
* Do not drop memory mappings for realtime ingestion
* Style fixes
* Renamed to match use case better
* Rollback memoization code and use the real time flag instead
* Null ptr fix in FireHydrant toString plus adjustments to memory pressure tracking calculations
* Style
* Log some count stats
* Make sure sinks size is obtained at the right time
* BatchAppenderator unit test
* Fix comment typos
* Renamed methods to make them more readable
* Move persisted metadata from FireHydrant class to AppenderatorImpl. Removed superfluous differences and fix comment typo. Removed custom comparator
* Missing dependency
* Make persisted hydrant metadata map concurrent and better reflect the fact that keys are Java references. Maintain persisted metadata when dropping/closing segments.
* Replaced concurrent variables with normal ones
* Added batchMemoryMappedIndex "fallback" flag with default "false". Set this to "true" make code fallback to previous code path.
* Style fix.
* Added note to new setting in doc, using Iterables.size (and removing a dependency), and fixing a typo in a comment.
* Forgot to commit this edited documentation message
* allow user to set group.id for Kafka ingestion task
* fix test coverage by removing deprecated code and add doc
* fix typo
* Update docs/development/extensions-core/kafka-ingestion.md
Co-authored-by: frank chen <frankchen@apache.org>
Co-authored-by: frank chen <frankchen@apache.org>
* upgrade to Apache Kafka 2.8.0 (release notes:
https://downloads.apache.org/kafka/2.8.0/RELEASE_NOTES.html)
* pass Kafka version as a Docker argument in integration tests
to keep in sync with maven version
* fix use of internal Kafka APIs in integration tests
* Add ability to wait for segment availability for batch jobs
* IT updates
* fix queries in legacy hadoop IT
* Fix broken indexing integration tests
* address an lgtm flag
* spell checker still flagging for hadoop doc. adding under that file header too
* fix compaction IT
* Updates to wait for availability method
* improve unit testing for patch
* fix bad indentation
* refactor waitForSegmentAvailability
* Fixes based off of review comments
* cleanup to get compile after merging with master
* fix failing test after previous logic update
* add back code that must have gotten deleted during conflict resolution
* update some logging code
* fixes to get compilation working after merge with master
* reset interrupt flag in catch block after code review pointed it out
* small changes following self-review
* fixup some issues brought on by merge with master
* small changes after review
* cleanup a little bit after merge with master
* Fix potential resource leak in AbstractBatchIndexTask
* syntax fix
* Add a Compcation TuningConfig type
* add docs stipulating the lack of support by Compaction tasks for the new config
* Fixup compilation errors after merge with master
* Remove erreneous newline
* DruidInputSource: Fix issues in column projection, timestamp handling.
DruidInputSource, DruidSegmentReader changes:
1) Remove "dimensions" and "metrics". They are not necessary, because we
can compute which columns we need to read based on what is going to
be used by the timestamp, transform, dimensions, and metrics.
2) Start using ColumnsFilter (see below) to decide which columns we need
to read.
3) Actually respect the "timestampSpec". Previously, it was ignored, and
the timestamp of the returned InputRows was set to the `__time` column
of the input datasource.
(1) and (2) together fix a bug in which the DruidInputSource would not
properly read columns that are used as inputs to a transformSpec.
(3) fixes a bug where the timestampSpec would be ignored if you attempted
to set the column to something other than `__time`.
(1) and (3) are breaking changes.
Web console changes:
1) Remove "Dimensions" and "Metrics" from the Druid input source.
2) Set timestampSpec to `{"column": "__time", "format": "millis"}` for
compatibility with the new behavior.
Other changes:
1) Add ColumnsFilter, a new class that allows input readers to determine
which columns they need to read. Currently, it's only used by the
DruidInputSource, but it could be used by other columnar input sources
in the future.
2) Add a ColumnsFilter to InputRowSchema.
3) Remove the metric names from InputRowSchema (they were unused).
4) Add InputRowSchemas.fromDataSchema method that computes the proper
ColumnsFilter for given timestamp, dimensions, transform, and metrics.
5) Add "getRequiredColumns" method to TransformSpec to support the above.
* Various fixups.
* Uncomment incorrectly commented lines.
* Move TransformSpecTest to the proper module.
* Add druid.indexer.task.ignoreTimestampSpecForDruidInputSource setting.
* Fix.
* Fix build.
* Checkstyle.
* Misc fixes.
* Fix test.
* Move config.
* Fix imports.
* Fixup.
* Fix ShuffleResourceTest.
* Add import.
* Smarter exclusions.
* Fixes based on tests.
Also, add TIME_COLUMN constant in the web console.
* Adjustments for tests.
* Reorder test data.
* Update docs.
* Update docs to say Druid 0.22.0 instead of 0.21.0.
* Fix test.
* Fix ITAutoCompactionTest.
* Changes from review & from merging.
* druid task auto scale based on kafka lag
* fix kafkaSupervisorIOConfig and KinesisSupervisorIOConfig
* druid task auto scale based on kafka lag
* fix kafkaSupervisorIOConfig and KinesisSupervisorIOConfig
* test dynamic auto scale done
* auto scale tasks tested on prd cluster
* auto scale tasks tested on prd cluster
* modify code style to solve 29055.10 29055.9 29055.17 29055.18 29055.19 29055.20
* rename test fiel function
* change codes and add docs based on capistrant reviewed
* midify test docs
* modify docs
* modify docs
* modify docs
* merge from master
* Extract the autoScale logic out of SeekableStreamSupervisor to minimize putting more stuff inside there && Make autoscaling algorithm configurable and scalable.
* fix ci failed
* revert msic.xml
* add uts to test autoscaler create && scale out/in and kafka ingest with scale enable
* add more uts
* fix inner class check
* add IT for kafka ingestion with autoscaler
* add new IT in groups=kafka-index named testKafkaIndexDataWithWithAutoscaler
* review change
* code review
* remove unused imports
* fix NLP
* fix docs and UTs
* revert misc.xml
* use jackson to build autoScaleConfig with default values
* add uts
* use jackson to init AutoScalerConfig in IOConfig instead of Map<>
* autoscalerConfig interface and provide a defaultAutoScalerConfig
* modify uts
* modify docs
* fix checkstyle
* revert misc.xml
* modify uts
* reviewed code change
* reviewed code change
* code reviewed
* code review
* log changed
* do StringUtils.encodeForFormat when create allocationExec
* code review && limit taskCountMax to partitionNumbers
* modify docs
* code review
Co-authored-by: yuezhang <yuezhang@freewheel.tv>
* add offsetFetchPeriod to kinesis ingestion doc
* Remove jackson dependencies from extensions
* Use fixed delay for lag collection
* Metrics reset after finishing processing
* comments
* Broaden the list of exceptions to retry for
* Unit tests
* Add more tests
* Refactoring
* re-order metrics
* Doc suggestions
Co-authored-by: Charles Smith <38529548+techdocsmith@users.noreply.github.com>
* Add tests
Co-authored-by: Charles Smith <38529548+techdocsmith@users.noreply.github.com>
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* fix checkstyle
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* fix test
* fix test
* add log
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* address comments
* fix checkstyle
* fix checkstyle
* add config to skip overhead memory calculation
* add test for the skipBytesInMemoryOverheadCheck config
* add docs
* fix checkstyle
* fix checkstyle
* fix spelling
* address comments
* fix travis
* address comments
Today Kafka message support in streaming indexing tasks is limited to
message values, and does not provide a way to expose Kafka headers,
timestamps, or keys, which may be of interest to more specialized
Druid input formats. For instance, Kafka headers may be used to indicate
payload format/encoding or additional metadata, and timestamps are often
omitted from values in Kafka streams applications, since they are
included in the record.
This change proposes to introduce KafkaRecordEntity as InputEntity,
which would give input formats full access to the underlying Kafka record,
including headers, key, timestamps. It would also open access to low-level
information such as topic, partition, offset if needed.
KafkaEntity is a subclass of ByteEntity for backwards compatibility with
existing input formats, and to avoid introducing unnecessary complexity
for Kinesis indexing tasks.
* support multi-line text
* add test cases
* split json text into lines case by case
* improve exception handle
* fix CI
* use IntermediateRowParsingReader as base of JsonReader
* update doc
* ignore the non-immutable field in test case
* add more test cases
* mark `lineSplittable` as final
* fix testcases
* fix doc
* add a test case for SqlReader
* return all raw columns when exception occurs
* fix CI
* fix test cases
* resolve review comments
* handle ParseException returned by index.add
* apply Iterables.getOnlyElement
* fix CI
* fix test cases
* improve code in more graceful way
* fix test cases
* fix test cases
* add a test case to check multiple json string in one text block
* fix inspection check
* Introduce a Configurable Index Type
* Change to @UnstableApi
* Add AppendableIndexSpecTest
* Update doc
* Add spelling exception
* Add tests coverage
* Revert some of the changes to reduce diff
* Minor fixes
* Update getMaxBytesInMemoryOrDefault() comment
* Fix typo, remove redundant interface
* Remove off-heap spec (postponed to a later PR)
* Add javadocs to AppendableIndexSpec
* Describe testCreateTask()
* Add tests for AppendableIndexSpec within TuningConfig
* Modify hashCode() to conform with equals()
* Add comment where building incremental-index
* Add "EqualsVerifier" tests
* Revert some of the API back to AppenderatorConfig
* Don't use multi-line comments
* Remove knob documentation (deferred)
* Move tools for indexing to TaskToolbox instead of injecting them in constructor
* oops, other changes
* fix test
* unnecessary new file
* fix test
* fix build
* Add validation for authorizer name
* fix deps
* add javadocs
* Do not use resource filters
* Fix BasicAuthenticatorResource as well
* Add integration tests
* fix test
* fix
* QueryCountStatsMonitor can be injected in the Peon
This change fixes a dependency injection bug where there is a circular
dependency on getting the MonitorScheduler when a user configures the
QueryCountStatsMonitor to be used.
* fix tests
* Actually fix the tests this time
* IntelliJ inspection and checkstyle rule for "Collection.EMPTY_* field accesses replaceable with Collections.empty*()"
* Reverted checkstyle rule
* Added tests to pass CI
* Codestyle