Commit Graph

3965 Commits

Author SHA1 Message Date
Abhishek Agarwal 17936e2920
Add an option to enable HSTS in druid services (#13489)
* Add an option to enable HSTS

* Fix code and add docs

* Deduplicate headers

* unused import

* Fix spelling
2023-01-10 22:31:51 +05:30
imply-cheddar a8ecc48ffe
Validate response headers and fix exception logging (#13609)
* Validate response headers and fix exception logging

A class of QueryException were throwing away their
causes making it really hard to determine what's
going wrong when something goes wrong in the SQL
planner specifically.  Fix that and adjust tests
 to do more validation of response headers as well.

We allow 404s and 307s to be returned even without 
authorization validated, but others get converted to 403
2023-01-05 14:15:15 -08:00
Kashif Faraz 36e6765596
Fix flaky test (#13603) 2023-01-03 13:52:05 +05:30
imply-cheddar 7b92b85168
Unify DummyRequest with MockHttpServletRequest (#13602)
We had 2 different classes both creating fake
instances of an HttpServletRequest, this makes
it to that we only have one in a common location
2022-12-21 20:15:08 -08:00
imply-cheddar 0efd0879a8
Unify the handling of HTTP between SQL and Native (#13564)
* Unify the handling of HTTP between SQL and Native

The SqlResource and QueryResource have been
using independent logic for things like error
handling and response context stuff.  This
became abundantly clear and painful during a
change I was making for Window Functions, so
I unified them into using the same code for
walking the response and serializing it.

Things are still not perfectly unified (it would
be the absolute best if the SqlResource just
took SQL, planned it and then delegated the
query run entirely to the QueryResource), but
this refactor doesn't take that fully on.

The new code leverages async query processing
from our jetty container, the different
interaction model with the Resource means that
a lot of tests had to be adjusted to align with
the async query model.  The semantics of the
tests remain the same with one exception: the
SqlResource used to not log requests that failed
authorization checks, now it does.
2022-12-19 00:25:33 -08:00
Kashif Faraz 58a3acc2c4
Add InputStats to track bytes processed by a task (#13520)
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`
2022-12-13 18:54:42 +05:30
somu-imply 7682b0b6b1
Analysis refactor (#13501)
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.
2022-12-12 17:35:44 -08:00
Gian Merlino de5a4bafcb
Zero-copy local deep storage. (#13394)
* 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.)
2022-12-12 17:28:24 -08:00
Rishabh Singh 4ebdfe226d
Druid automated quickstart (#13365)
* Druid automated quickstart

* remove conf/druid/single-server/quickstart/_common/historical/jvm.config

* Minor changes in python script

* Add lower bound memory for some services

* Additional runtime properties for services

* Update supervise script to accept command arguments, corresponding changes in druid-quickstart.py

* File end newline

* Limit the ability to start multiple instances of a service, documentation changes

* simplify script arguments

* restore changes in medium profile

* run-druid refactor

* compute and pass middle manager runtime properties to run-druid
supervise script changes to process java opts array
use argparse, leave free memory, logging

* Remove extra quotes from mm task javaopts array

* Update logic to compute minimum memory

* simplify run-druid

* remove debug options from run-druid

* resolve the config_path provided

* comment out service specific runtime properties which are computed in the code

* simplify run-druid

* clean up docs, naming changes

* Throw ValueError exception on illegal state

* update docs

* rename args, compute_only -> compute, run_zk -> zk

* update help documentation

* update help documentation

* move task memory computation into separate method

* Add validation checks

* remove print

* Add validations

* remove start-druid bash script, rename start-druid-main

* Include tasks in lower bound memory calculation

* Fix test

* 256m instead of 256g

* caffeine cache uses 5% of heap

* ensure min task count is 2, task count is monotonic

* update configs and documentation for runtime props in conf/druid/single-server/quickstart

* Update docs

* Specify memory argument for each profile in single-server.md

* Update middleManager runtime.properties

* Move quickstart configs to conf/druid/base, add bash launch script, support python2

* Update supervise script

* rename base config directory to auto

* rename python script, changes to pass repeated args to supervise

* remove exmaples/conf/druid/base dir

* add docs

* restore changes in conf dir

* update start-druid-auto

* remove hashref for commands in supervise script

* start-druid-main java_opts array is comma separated

* update entry point script name in python script

* Update help docs

* documentation changes

* docs changes

* update docs

* add support for running indexer

* update supported services list

* update help

* Update python.md

* remove dir

* update .spelling

* Remove dependency on psutil and pathlib

* update docs

* Update get_physical_memory method

* Update help docs

* update docs

* update method to get physical memory on python

* udpate spelling

* update .spelling

* minor change

* Minor change

* memory comptuation for indexer

* update start-druid

* Update python.md

* Update single-server.md

* Update python.md

* run python3 --version to check if python is installed

* Update supervise script

* start-druid: echo message if python not found

* update anchor text

* minor change

* Update condition in supervise script

* JVM not jvm in docs
2022-12-09 11:04:02 -08:00
Paul Rogers 013a12e86f
Enhanced MSQ table functions (#13360)
* Enhanced MSQ table functions
* HTTP, LOCALFILES and INLINE table functions powered by
catalog metadata.
* Documentation
2022-12-08 13:56:02 -08:00
Clint Wylie 37d8833125
fix bug with broker parallel merge metrics emitting, add wall time, fast/slow partition time metrics (#13420) 2022-12-06 17:50:59 -08:00
imply-cheddar 83261f9641
Starting on Window Functions (#13458)
* Processors for Window Processing

This is an initial take on how to use Processors
for Window Processing.  A Processor is an interface
that transforms RowsAndColumns objects.
RowsAndColumns objects are essentially combinations
of rows and columns.

The intention is that these Processors are the start
of a set of operators that more closely resemble what
DB engineers would be accustomed to seeing.

* Wire up windowed processors with a query type that
can run them end-to-end.  This code can be used to
actually run a query, so yay!

* Wire up windowed processors with a query type that
can run them end-to-end.  This code can be used to
actually run a query, so yay!

* Some SQL tests for window functions. Added wikipedia 
data to the indexes available to the
SQL queries and tests validating the windowing
functionality as it exists now.

Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2022-12-06 15:54:05 -08:00
Clint Wylie cf472162a6
fix issue with jetty graceful shutdown of data servers when druid.serverview.type=http (#13499)
* fix issue with http server inventory view blocking data node http server shutdown with long polling

* adjust

* fix test inspections
2022-12-06 15:52:44 -08:00
AmatyaAvadhanula 658a9c2d35
Early stop on failed start (Alternative to #13087) (#13258)
* Make halt configurable. Don't halt in tests
2022-12-05 21:05:07 +05:30
TSFenwick 10bec54acc
Switching emitter. This will allow for a per feed emitter designation. (#13363)
* Switching emitter. This will allow for a per feed emitter designation.

This will work by looking at an event's feed and direct it to a specific emitter. If no specific feed is specified for a feed.
The emitter can direct the event to a default emitter.

* fix checkstyle issues and make docs for switching emitter use basic event feeds

* fix broken docs, add test, and guard against misconfigurations

* add module test
add switching emitter module test

* fix broken SwitchingEmitterModuleTest

* add apache license to top of test

* fix checkstyle issues

* address comments by adding javadocs, removing a todo, and making druid docs more clear
2022-12-05 16:04:34 +05:30
Kashif Faraz 45a8fa280c
Add SegmentAllocationQueue to batch SegmentAllocateActions (#13369)
In a cluster with a large number of streaming tasks (~1000), SegmentAllocateActions 
on the overlord can often take very long intervals of time to finish thus causing spikes 
in the `task/action/run/time`. This may result in lag building up while a task waits for a
segment to get allocated.

The root causes are:
- large number of metadata calls made to the segments and pending segments tables
- `giant` lock held in `TaskLockbox.tryLock()` to acquire task locks and allocate segments

Since the contention typically arises when several tasks of the same datasource try
to allocate segments for the same interval/granularity, the allocation run times can be
improved by batching the requests together.

Changes
- Add flags
   - `druid.indexer.tasklock.batchSegmentAllocation` (default `false`)
   - `druid.indexer.tasklock.batchAllocationMaxWaitTime` (in millis) (default `1000`)
- Add methods `canPerformAsync` and `performAsync` to `TaskAction`
- Submit each allocate action to a `SegmentAllocationQueue`, and add to correct batch
- Process batch after `batchAllocationMaxWaitTime`
- Acquire `giant` lock just once per batch in `TaskLockbox`
- Reduce metadata calls by batching statements together and updating query filters
- Except for batching, retain the whole behaviour (order of steps, retries, etc.)
- Respond to leadership changes and fail items in queue when not leader
- Emit batch and request level metrics
2022-12-05 14:00:07 +05:30
Paul Rogers b76ff16d00
SQL test framework extensions (#13426)
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
2022-12-02 09:11:59 -08:00
Gian Merlino 58c896ea0b
ServiceClient: More robust redirect handling. (#13413)
Detects self-redirects, redirect loops, long redirect chains, and redirects to unknown servers.
Treat all of these cases as an unavailable service, retrying if the retry policy allows it.

Previously, some of these cases would lead to a prompt, unretryable error. This caused
clients contacting an Overlord during a leader change to fail with error messages like:

org.apache.druid.rpc.RpcException: Service [overlord] redirected too many times

Additionally, a slight refactor of callbacks in ServiceClientImpl improves readability of
the flow through onSuccess.
2022-11-28 22:24:46 +05:30
Kashif Faraz 656b6cdf62
Add MetricsVerifier to simplify verification of metric values in tests (#13442) 2022-11-28 19:32:37 +05:30
Kashif Faraz 7cf761cee4
Prepare master branch for next release, 26.0.0 (#13401)
* Prepare master branch for next release, 26.0.0

* Use docker image for druid 24.0.1

* Fix version in druid-it-cases pom.xml
2022-11-22 15:31:01 +05:30
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
Rohan Garg 6ccf31490e
Allow injection of node-role set to all non base modules (#13371) 2022-11-18 12:12:03 +05:30
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
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
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
Paul Rogers 7e600d2c63
Enhancements to the Calcite test framework (#13283)
* Enhancements to the Calcite test framework
* Standardize "Unauthorized" messages
* Additional test framework extension points
* Resolved joinable factory dependency issue
2022-11-08 14:28:49 -08:00
Kashif Faraz 9f7fd57a69
Improve fetch of pending segments from metadata store (#13310)
* Deserialize only when needed

* Update query to fetch pending segments

* Revert unneeded changes

* Fix query
2022-11-08 05:46:19 -08:00
Kashif Faraz ff8e0c3397
Fix issues with caching cost strategy (#13321)
`cachingCost` strategy has some discrepancies when compared to cost strategy.
This commit addresses two of these by retaining the same behaviour as the `cost` strategy
when computing the cost of moving a segment to a server:
- subtract the self cost of a segment if it is being served by the target server
- subtract the cost of segments that are marked to be dropped

Other changes:
- Add tests to verify fixed strategy. These tests would fail without the fixes made to `CachingCostStrategy.computeCost()`
- Fix the definition of the segment related metrics in the docs.
- Fix some docs issues introduced in #13181
2022-11-08 16:11:39 +05:30
Tejaswini Bandlamudi 594545da55
Adds cluster level idleConfig setting for supervisor (#13311)
* adds cluster level idleConfig

* updates docs

* refactoring

* spelling nit

* nit

* nit

* refactoring
2022-11-08 14:54:14 +05:30
AmatyaAvadhanula 47c32a9d92
Skip ALL granularity compaction (#13304)
* Skip autocompaction for datasources with ETERNITY segments
2022-11-07 17:55:03 +05:30
AmatyaAvadhanula 650840ddaf
Add segment handoff time metric (#13238)
* Add segment handoff time metric

* Remove monitors on scheduler stop

* Add warning log for slow handoff

* Remove monitor when scheduler stops
2022-11-07 17:49:10 +05:30
Dr. Sizzles e5ad24ff9f
Support for middle manager less druid, tasks launch as k8s jobs (#13156)
* 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>
2022-11-02 19:44:47 -07:00
Kashif Faraz fd7864ae33
Improve run time of coordinator duty MarkAsUnusedOvershadowedSegments (#13287)
In clusters with a large number of segments, the duty `MarkAsUnusedOvershadowedSegments`
can take a long very long time to finish. This is because of the costly invocation of 
`timeline.isOvershadowed` which is done for every used segment in every coordinator run.

Changes
- Use `DataSourceSnapshot.getOvershadowedSegments` to get all overshadowed segments
- Iterate over this set instead of all used segments to identify segments that can be marked as unused
- Mark segments as unused in the DB in batches rather than one at a time
- Refactor: Add class `SegmentTimeline` for ease of use and readability while using a
`VersionedIntervalTimeline` of segments.
2022-11-01 20:19:52 +05:30
somu-imply affc522b9f
Refactoring the data source before unnest (#13085)
* First set of changes for framework

* Second set of changes to move segment map function to data source

* Minot change to server manager

* Removing the createSegmentMapFunction from JoinableFactoryWrapper and moving to JoinDataSource

* Checkstyle fixes

* Patching Eric's fix for injection

* Checkstyle and fixing some CI issues

* Fixing code inspections and some failed tests and one injector for test in avatica

* Another set of changes for CI...almost there

* Equals and hashcode part update

* Fixing injector from Eric + refactoring for broadcastJoinHelper

* Updating second injector. Might revert later if better way found

* Fixing guice issue in JoinableFactory

* Addressing review comments part 1

* Temp changes refactoring

* Revert "Temp changes refactoring"

This reverts commit 9da42a9ef0.

* temp

* Temp discussions

* Refactoring temp

* Refatoring the query rewrite to refer to a datasource

* Refactoring getCacheKey by moving it inside data source

* Nullable annotation check in injector

* Addressing some comments, removing 2 analysis.isJoin() checks and correcting the benchmark files

* Minor changes for refactoring

* Addressing reviews part 1

* Refactoring part 2 with new test cases for broadcast join

* Set for nullables

* removing instance of checks

* Storing nullables in guice to avoid checking on reruns

* Fixing a test case and removing an irrelevant line

* Addressing the atomic reference review comments
2022-10-26 15:58:58 -07:00
Gian Merlino d98c808d3f
Remove basePersistDirectory from tuning configs. (#13040)
* 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.
2022-10-21 17:25:36 -07:00
Paul Rogers 86e6e61e88
Modular Calcite Test Framework (#12965)
* 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.
2022-10-20 15:45:44 -07:00
Paul Rogers f4dcc52dac
Redesign QueryContext class (#13071)
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
2022-10-15 11:02:11 +05:30
Tejaswini Bandlamudi 3e13584e0e
Adds Idle feature to `SeekableStreamSupervisor` for inactive stream (#13144)
* Idle Seekable stream supervisor changes.

* nit

* nit

* nit

* Adds unit tests

* Supervisor decides it's idle state instead of AutoScaler

* docs update

* nit

* nit

* docs update

* Adds Kafka unit test

* Adds Kafka Integration test.

* Updates travis config.

* Updates kafka-indexing-service dependencies.

* updates previous offsets snapshot & doc

* Doesn't act if supervisor is suspended.

* Fixes highest current offsets fetch bug, adds new Kafka UT tests, doc changes.

* Reverts Kinesis Supervisor idle behaviour changes.

* nit

* nit

* Corrects SeekableStreamSupervisorSpec check on idle behaviour config, adds tests.

* Fixes getHighestCurrentOffsets to fetch offsets of publishing tasks too

* Adds Kafka Supervisor UT

* Improves test coverage in druid-server

* Corrects IT override config

* Doc updates and Syntactic changes

* nit

* supervisorSpec.ioConfig.idleConfig changes
2022-10-12 18:31:08 +05:30
Gian Merlino c19ae13323
Improve direct-memory check on startup. (#13207)
1) Better support for Java 9+ in RuntimeInfo. This means that in many cases,
   an actual validation can be done.

2) Clearer log message in cases where an actual validation cannot be done.
2022-10-12 05:10:25 +08:00
AmatyaAvadhanula 41e51b21c3
Make http options the default configurations (#13092)
Druid currently uses Zookeeper dependent options as the default.
This commit updates the following to use HTTP as the default instead.
- task runner. `druid.indexer.runner.type=remote -> httpRemote`
- load queue peon. `druid.coordinator.loadqueuepeon.type=curator -> http`
- server inventory view. `druid.serverview.type=curator -> http`
2022-10-05 05:35:17 +05:30
Abhishek Agarwal e3f9a0ed44
Lazy initialization of segment killers, movers and archivers (#13170)
* Lazy initialization of segment killers, movers and archivers

* Add test for lazy killer

* Add more tests

* Intellij fixes
2022-10-04 15:55:46 +05:30
Kashif Faraz ce5f55e5ce
Fix over-replication caused by balancing when inventory is not updated yet (#13114)
* Add coordinator test framework

* Remove outdated changes

* Add more tests

* Add option to auto-sync inventory

* Minor cleanup

* Fix inspections

* Add README for simulations, add SegmentLoadingNegativeTest

* Fix over-replication from balancing

* Fix README

* Cleanup unnecessary fields from DruidCoordinator

* Add a test

* Fix DruidCoordinatorTest

* Remove unused import

* Fix CuratorDruidCoordinatorTest

* Remove test log4j2.xml
2022-09-29 12:06:23 +05:30
Kashif Faraz 0039409817
Add test framework to simulate segment loading and balancing (#13074)
Fixes #12822 

The framework added here make it easy to write tests that verify the behaviour and interactions
of the following entities under various conditions:
- `DruidCoordinator`
- `HttpLoadQueuePeon`, `LoadQueueTaskMaster`
- coordinator duties: `BalanceSegments`, `RunRules`, `UnloadUnusedSegments`, etc.
- datasource retention rules: `LoadRule`, `DropRule`

Changes:
Add the following main classes:
- `CoordinatorSimulation` and related interfaces to dictate behaviour of simulation
- `CoordinatorSimulationBuilder` to build a simulation.
- `BlockingExecutorService` to keep submitted tasks in queue and execute them
  only when explicitly invoked.

Add tests:
- `CoordinatorSimulationBaseTest`, `SegmentLoadingTest`, `SegmentBalancingTest`
- `SegmentLoadingNegativeTest` to contain tests which assert the existing erroneous behaviour
of segment loading. Once the behaviour is fixed, these tests will be moved to the regular
`SegmentLoadingTest`.

Please refer to the README.md in `org.apache.druid.server.coordinator.simulate` for more details
2022-09-21 09:51:58 +05:30
Paul Rogers 8ce03eb094
Convert the Druid planner to use statement handlers (#12905)
* Converted Druid planner to use statement handlers

Converts the large collection of if-statements for statement
types into a set of classes: one per supported statement type.
Cleans up a few error messages.

* Revisions from review comments

* Build fix

* Build fix

* Resolve merge confict.

* More merges with QueryResponse PR

* More parameterized type cleanup

Forces a rebuild due to a flaky test
2022-09-19 11:58:45 +05:30
AmatyaAvadhanula 9b53b0184f
Allocate numCorePartitions using only used segments (#13070)
* Allocate numCorePartitions using only used segments

* Add corePartition checks in existing test

* Separate committedMaxId and overallMaxId

* Fix bug: replace overall with committed
2022-09-16 19:16:36 +05:30
AmatyaAvadhanula 1311e85f65
Faster fix for dangling tasks upon supervisor termination (#13072)
This commit fixes issues with delayed supervisor termination during certain transient states.
Tasks can be created during supervisor termination and left behind since the cleanup may
not consider these newly added tasks.

#12178 added a lock for the entire process of task creation to prevent such dangling tasks.
But it also introduced a deadlock scenario as follows:
- An invocation of `runInternal` is in progress.
- A `stop` request comes, acquires `stateChangeLock` and submit a `ShutdownNotice`
- `runInternal` keeps waiting to acquire the `stateChangeLock`
- `ShutdownNotice` remains stuck in the notice queue because `runInternal` is still running
- After some timeout, the supervisor goes through a forced termination

Fix:
 * `SeekableStreamSupervisor.runInternal` - do not try to acquire lock if supervisor is already stopping
 * `SupervisorStateManager.maybeSetState` - do not allow transitions from STOPPING state
2022-09-15 15:31:14 +05:30
Frank Chen aa9b0900d4
Move web-console dependency declaration from druid-server to druid-distribution (#12501)
* Move web-console dependency from druid-server to distribution

* Add a test to check if the web-console is correctly integrated

* exclude web-console from 'other integration tests'

* Revert "exclude web-console from 'other integration tests'"

This reverts commit 8d72225544.

* Revert "Add a test to check if the web-console is correctly integrated"

This reverts commit d6ac8f3087.
2022-09-15 10:39:30 +08:00
Clint Wylie f4ec50bf7a
fix JsonParserIteratorTest (#13083) 2022-09-13 20:49:57 -07:00
Frank Chen fd6c05eee8
Avoid ClassCastException when getting values from `QueryContext` (#13022)
* Use safe conversion methods

* Rename method

* Add getContextAsBoolean

* Update test case

* Remove generic from getContextValue

* Update catch-handler

* Add test

* Resolve comments

* Replace 'getContextXXX' to 'getQueryContext().getAsXXXX'
2022-09-13 18:00:09 +08:00