Commit Graph

12231 Commits

Author SHA1 Message Date
Gian Merlino c6054b7cb7
Attach IO error to parse error when we can't contact Avro schema registry. (#13403)
* Attach IO error to parse error when we can't contact Avro schema registry.

The change in #12080 lost the original exception context. This patch
adds it back.

* Add hamcrest-core.

* Fix format string.
2022-11-21 22:20:26 -08:00
Adarsh Sanjeev 280a0f7158
Add sequential sketch merging to MSQ (#13205)
* Add sketch fetching framework

* Refactor code to support sequential merge

* Update worker sketch fetcher

* Refactor sketch fetcher

* Refactor sketch fetcher

* Add context parameter and threshold to trigger sequential merge

* Fix test

* Add integration test for non sequential merge

* Address review comments

* Address review comments

* Address review comments

* Resolve maxRetainedBytes

* Add new classes

* Renamed key statistics information class

* Rename fetchStatisticsSnapshotForTimeChunk function

* Address review comments

* Address review comments

* Update documentation and add comments

* Resolve build issues

* Resolve build issues

* Change worker APIs to async

* Address review comments

* Resolve build issues

* Add null time check

* Update integration tests

* Address review comments

* Add log messages and comments

* Resolve build issues

* Add unit tests

* Add unit tests

* Fix timing issue in tests
2022-11-22 09:56:32 +05:30
Vadim Ogievetsky fe34ecc5e3
add ability to make inputFormat part of the example datasets (#13402) 2022-11-21 12:50:44 -08:00
Jill Osborne 68018a808f
Firehose migration doc (#12981)
* Firehose migration doc

* Update migrate-from-firehose-ingestion.md

* Updated with review comments and suggestions

* Update migrate-from-firehose-ingestion.md

* Update migrate-from-firehose-ingestion.md

* Update migrate-from-firehose-ingestion.md
2022-11-21 11:17:12 -08:00
Kashif Faraz 133054bf27
Make batched segment sampling the default, minor cleanup of coordinator config (#13391)
The batch segment sampling performs significantly better than the older method
of sampling if there are a large number of used segments. It also avoids duplicates.

Changes:
- Make batch segment sampling the default
- Deprecate the property `useBatchedSegmentSampler`
- Remove unused coordinator config `druid.coordinator.loadqueuepeon.repeatDelay`
- Cleanup `KillUnusedSegments`
- Simplify `KillUnusedSegmentsTest`, add better tests, remove redundant tests
2022-11-21 20:31:46 +05:30
Gian Merlino bfffbabb56
Async task client for SeekableStreamSupervisors. (#13354)
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.
2022-11-21 19:20:26 +05:30
Gian Merlino f037776fd8
MSQ: Launch initial tasks faster. (#13393)
Notify the mainLoop thread to skip a sleep when the desired task
count changes.
2022-11-21 19:11:18 +05:30
Gian Merlino b8ca03d283
SeekableStreamSupervisor: Unique type name for GracefulShutdownNotice. (#13399)
Allows GracefulShutdownNotice to be differentiated from ShutdownNotice.
2022-11-21 19:10:14 +05:30
AmatyaAvadhanula de566eb0db
Fix shared lock acquisition criteria (#13390)
Currently, a shared lock is acquired only when all other locks are also shared locks.

This commit updates the behaviour and acquires a shared lock only if all locks
of equal or higher priority are either shared locks or are already revoked.
The lock type of locks with lower priority does not matter as they can be revoked.
2022-11-21 15:31:38 +05:30
Abhishek Agarwal 6c3f688a66
cron stage failure should always send a failure notification (#13397) 2022-11-21 14:38:15 +05:30
Gian Merlino c61313f4c4
Quieter streaming supervisors. (#13392)
Eliminates two common sources of noise with Kafka supervisors that have
large numbers of tasks and partitions:

1) Log the report at DEBUG rather than INFO level at each run cycle.
   It can get quite large, and can be retrieved via API when needed.

2) Use log4j2.xml to quiet down the org.apache.kafka.clients.consumer.internals
   package. Avoids a log message per-partition per-minute as part of seeking
   to the latest offset in the reporting thread. In the tasks, where this
   sort of logging might be more useful, we have another log message with
   the same information: "Seeking partition[%s] to[%s]".
2022-11-20 23:53:17 -08:00
Adam Peck fa3ab27a71
Bump apache curator from 5.3.0 to 5.4.0 (#13295)
Follow up to #12939. As noted in that PR there are a few fixes in 5.4.0 that should make running on Kubernetes more reliable. Notably:
- https://issues.apache.org/jira/browse/CURATOR-538
- https://issues.apache.org/jira/browse/CURATOR-649
2022-11-21 11:23:29 +05:30
Katya Macedo fd239305d9
Update metrics doc (#13316)
Changes:
- used inline code-style to format dimension names
- removed unnecessary punctuation
2022-11-21 09:43:52 +05:30
abhagraw 5172d76a67
Migrate current integration batch tests to equivalent MSQ tests (#13374)
* Migrate current integration batch tests to equivalent MSQ tests using new IT framework

* Fix build issues

* Trigger Build

* Adding more tests and addressing comments

* fixBuildIssues

* fix dependency issues

* Parameterized the test and addressed comments

* Addressing comments

* fixing checkstyle errors

* Adressing comments
2022-11-21 09:12:02 +05:30
Jill Osborne a860baf496
Updated docs on front coding (#13387) 2022-11-19 00:01:04 -08:00
Vadim Ogievetsky 08fa0383b9
add supported indexSpec options (#13388) 2022-11-19 00:00:32 -08:00
Vadim Ogievetsky c628947c31
Web console: streaming json input format specifics (#13381)
* streaming json input format specifics

* goodies
2022-11-18 14:15:16 -08:00
Vadim Ogievetsky a3d45f6086
treat user cancelation seriously (#13376) 2022-11-18 14:04:16 -08:00
Rohan Garg 5b625cea96
Improve performance for ReadableInputStreamFrameChannel (#13373)
* Improve performance for ReadableInputStreamFrameChannel

* Fix race condition leading to unnecessary sleep
2022-11-18 18:26:08 +05:30
Gian Merlino 092e769dd8
JvmMonitor: Report jvm/gc/cpu in nanos. (#13383)
Our documentation says we report this in nanos, and we actually did
prior to #12481. This patch restores the prior behavior.
2022-11-18 12:33:16 +05:30
Rohan Garg 6ccf31490e
Allow injection of node-role set to all non base modules (#13371) 2022-11-18 12:12:03 +05:30
Clint Wylie 7f4e386509
add missing vector object selector for multi-value string columns, refactor some stuff (#13379)
* add vector object selector for multi-value string columns, refactor some stuff

* use for nested columns too

* add test

* inspections
2022-11-17 21:08:54 -08:00
Clint Wylie 8c9ffcfe37
nested column support for ORC (#13375)
* nested column support for ORC

* more test
2022-11-17 21:08:34 -08:00
Tejaswini Bandlamudi bf10ff73a8
Fixes Kafka Supervisor Lag Report (#13380)
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.
2022-11-17 22:24:45 +05:30
imply-cheddar 6b9344cd39
Persist legacy LatestPairs for now (#13378)
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.
2022-11-17 21:37:02 +05:30
Tejaswini Bandlamudi 8e9e46b519
Add static-checks Github Action (#13347)
* Adds static-checks github action

* bug fixes

* bug fixes

* adds maven install step

* update permissions on scripts folder

* fix license check errors

* attempt for parallelization

* fix concurrency

* fix naming

* remove intellij inspections to add in different CI pipeline

* minimize naming, add new lines

* setting hadoop profile through matrix

* also runs on push triggers to master and release branches

* changes on review
2022-11-17 17:20:16 +05:30
Gian Merlino 78d0b0abce
Add string comparison methods to StringUtils, fix dictionary comparisons. (#13364)
* Add string comparison methods to StringUtils, fix dictionary comparisons.

There are various places in Druid code where we assume that String.compareTo
is consistent with Unicode code-point ordering. Sadly this is not the case.

To help deal with this, this patch introduces the following helpers:

1) compareUnicode: Compares two Strings in Unicode code-point order.
2) compareUtf8: Compares two UTF-8 byte arrays in Unicode code-point order.
   Equivalent to comparison as unsigned bytes.
3) compareUtf8UsingJavaStringOrdering: Compares two UTF-8 byte arrays, or
   ByteBuffers, in a manner consistent with String.compareTo.

There is no helper for comparing two Strings in a manner consistent
with String.compareTo, because for that we can use compareTo directly.

The patch also fixes an inconsistency between the String and UTF-8
dictionary GenericIndexed flavors of string-typed columns: they were
formerly using incompatible comparators.

* Adjust test.

* FrontCodedIndexed updates.

* Add test.

* Fix comments.
2022-11-16 07:15:00 -08:00
Kashif Faraz 71b133f3ff
Add `RoundRobinServerSelector` to speed up segment assignments (#13367)
Segment assignments can take very long due to the strategy cost computation
for a large number of segments. This commit allows segment assignments to be
done in a round-robin fashion within a tier. Only segment balancing takes cost-based
decisions to move segments around.

Changes
- Add dynamic config `useRoundRobinSegmentAssignment` with default value false
- Add `RoundRobinServerSelector`. This does not implement the `BalancerStrategy`
as it does not conform to that contract and may also be used in conjunction with a
strategy (round-robin for `RunRules` and a cost strategy for `BalanceSegments`)
- Drops are still cost-based even when round-robin assignment is enabled.
2022-11-16 20:05:17 +05:30
dependabot[bot] 61a7199f2b
Bump loader-utils from 1.4.0 to 1.4.2 in /web-console (#13372)
Bumps [loader-utils](https://github.com/webpack/loader-utils) from 1.4.0 to 1.4.2.
- [Release notes](https://github.com/webpack/loader-utils/releases)
- [Changelog](https://github.com/webpack/loader-utils/blob/v1.4.2/CHANGELOG.md)
- [Commits](https://github.com/webpack/loader-utils/compare/v1.4.0...v1.4.2)

---
updated-dependencies:
- dependency-name: loader-utils
  dependency-type: indirect
...

Signed-off-by: dependabot[bot] <support@github.com>

Signed-off-by: dependabot[bot] <support@github.com>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
2022-11-16 20:00:33 +08:00
Laksh Singla 9e938b5a6f
Add a limit to the number of columns in the CLUSTERED BY clause (#13352)
* Add clustered by limit

* change semantics, add docs

* add fault class to the module

* add test

* unambiguate test
2022-11-15 22:05:15 +05:30
Clint Wylie 309cae7b65
nested column support for Parquet and Avro (#13325)
* nested column support for Parquet and Avro

* style
2022-11-14 16:09:05 -08:00
Clint Wylie 1231ce3b75
dump-segment tool support for examining nested columns (#13356)
* add nested mode to dump segment tool to dump nested columns

* docs

* more test

* fix it
2022-11-14 16:08:47 -08:00
Adarsh Sanjeev a3edda3b63
Modify quantile sketches to add byte[] directly (#13351)
* Modify quantile sketchs to add byte[] directly

* Rename class and add test
2022-11-14 00:24:06 +05:30
Paul Rogers 81d005f267
Druid Catalog basics (#13165)
Druid catalog basics

Catalog object model for tables, columns
Druid metadata DB storage (as an extension)
REST API to update the catalog (as an extension)
Integration tests
Model only: no planner integration yet
2022-11-12 15:30:22 -08:00
Jill Osborne b0db2a87d8
Update Kafka ingestion tutorial (#13261)
* Update Kafka ingestion tutorial

* Update tutorial-kafka.md

Updated location of sample data file

* Added sample data file

* Update tutorial-kafka.md

* Add sample data file

* Update tutorial-kafka.md

Updated sample file location in curl commands

* Update and reuploading sample data files

* Updated spelling file

* Delete .spelling

* Added spelling file

* Update docs/tutorials/tutorial-kafka.md

Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>

* Update docs/tutorials/tutorial-kafka.md

Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>

* Updated after review

* Update tutorial-kafka.md

* Updated

* Update tutorial-kafka.md

* Update tutorial-kafka.md

* Update tutorial-kafka.md

* Updated sample data file and command

* Add files via upload

* Delete kttm-nested-data.json.tgz

* Delete kttm-nested-data.json.tgz

* Add files via upload

* Update tutorial-kafka.md

Co-authored-by: Victoria Lim <vtlim@users.noreply.github.com>
2022-11-11 14:47:54 -08:00
Laksh Singla 3e172d44ab
Bind DurableStorageCleaner only on the Overlord nodes (#13355) 2022-11-11 21:56:33 +05:30
Jill Osborne 47dd4ed2e7
Added experimental feature text for front coding feature (#13349) 2022-11-11 02:06:13 -08:00
Gian Merlino e78f648023
SeekableStreamSupervisor: Don't enqueue duplicate notices. (#13334)
* SeekableStreamSupervisor: Don't enqueue duplicate notices.

Similar goal to #12018, but more aggressive. Don't enqueue a notice at
all if it is equal to one currently in the queue.

* Adjustments from review.

* Update indexing-service/src/test/java/org/apache/druid/indexing/overlord/supervisor/NoticesQueueTest.java

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>

Co-authored-by: Kashif Faraz <kashif.faraz@gmail.com>
2022-11-11 01:54:01 -08:00
Didip Kerabat 56d5c9780d
Use standard library to correctly glob and stop at the correct folder structure when filtering cloud objects (#13027)
* 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>
2022-11-10 23:46:40 -08:00
Gian Merlino 77478f25fb
Add taskActionType dimension to task/action/run/time. (#13333)
* Add taskActionType dimension to task/action/run/time.

* Spelling.
2022-11-11 12:00:08 +05:30
Andreas Maechler 03175a2b8d
Add missing MSQ error code fields to docs (#13308)
* Fix typo

* Fix some spacing

* Add missing fields

* Cleanup table spacing

* Remove durable storage docs again

Thanks Brian for pointing out previous discussions.

* Update docs/multi-stage-query/reference.md

Co-authored-by: Charles Smith <techdocsmith@gmail.com>

* Mark codes as code

* And even more codes as code

* Another set of spaces

* Combine `ColumnTypeNotSupported`

Thanks Karan.

* More whitespaces and typos

* Add spelling and fix links

Co-authored-by: Charles Smith <techdocsmith@gmail.com>
2022-11-10 21:03:04 +05:30
AmatyaAvadhanula fb23e38aa7
Fix messageGap emission (#13346)
* Fix messageGap emission

* Do not emit messageGap after stopping reading events

* Refactoring

* Fix tests
2022-11-10 17:50:19 +05:30
Jill Osborne c2210c4e09
Update ingestion spec doc (#13329)
* Update ingestion spec doc

* Updated

* Updated

* Update docs/ingestion/ingestion-spec.md

Co-authored-by: Clint Wylie <cjwylie@gmail.com>

* Updated

* Updated

Co-authored-by: Clint Wylie <cjwylie@gmail.com>
2022-11-10 02:54:35 -08:00
Clint Wylie 44f29030dd
fix flaky RemoteTaskRunnerTest.testRunPendingTaskFailToAssignTask with ugly Thread.sleep (#13344) 2022-11-10 14:28:53 +05:30
Clint Wylie 27215d1ff1
fix complex_decode_base64 function, add SQL bindings (#13332)
* fix complex_decode_base64 function, add SQL bindings

* more permissive
2022-11-09 23:40:25 -08:00
Jill Osborne 965e41538e
Update nested columns doc (#13314)
* Updated nested columns doc

* Update nested-columns.md

* Update nested-columns.md
2022-11-10 09:53:28 +08:00
AmatyaAvadhanula 0512ae4922
Optimize metadata calls in SeekableStreamSupervisor (#13328)
* Optimize metadata calls

* Modify isTaskCurrent

* Fix tests

* Refactoring
2022-11-10 07:22:51 +05:30
Jason Koch 0040042863
HttpPostEmitter back off send() busy-loop (#12102)
* HttpPostEmitter back off send() busy-loop

The HttpPostEmitter gets in a loop until the flush timeout can be
triggered, OR until some new events arrive that reset the minimum
batch fill timeout delay. As a tactical fix, this introduces a
simple backoff delay to the send loop to prevent spamming logs.

* Update core/src/main/java/org/apache/druid/java/util/emitter/core/HttpPostEmitter.java

Co-authored-by: Frank Chen <frankchen@apache.org>

Co-authored-by: Frank Chen <frankchen@apache.org>
2022-11-09 14:32:40 -08:00
Clint Wylie 3e2bb4cf10
fix front-coded bucket size handling, better validation (#13335)
* fix front-coded bucket size handling, better validation

* Update FrontCodedIndexedTest.java
2022-11-09 13:33:01 -08:00
AmatyaAvadhanula a2013e6566
Enhance streaming ingestion metrics (#13331)
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
2022-11-09 23:44:15 +05:30