* use SqlLifecyle to manage sql execution, add sqlId
* add sql request logger
* fix UT
* rename sqlId to sqlQueryId, sql/time to sqlQuery/time, etc
* add docs and more sql request logger impls
* add UT for http and jdbc
* fix forbidden use of com.google.common.base.Charsets
* fix UT in QuantileSqlAggregatorTest, supressed unused warning of getSqlQueryId
* do not use default method in QueryMetrics interface
* capitalize 'sql' everywhere in the non-property parts of the docs
* use RequestLogger interface to log sql query
* minor bugfixes and add switching request logger
* add filePattern configs for FileRequestLogger
* address review comments, adjust sql request log format
* fix inspection error
* try SuppressWarnings("RedundantThrows") to fix inspection error on ComposingRequestLoggerProvider
* Fix num_replicas count from sys.segments
* Adjust unit test for num_replica > 1
* Pass named arguments instead of passing boolean constants
* Address PR comments
* PR comments
* Use multi-guava version friendly direct executor implementation
* Don't use a singleton
* Fix strict compliation complaints
* Copy Guava's DirectExecutor
* Fix javadoc
* Imports are the devil
* Handoff should ignore segments that are dropped by drop rules
* fix travis-ci
* fix tests
* address comments
* remove line added by accident
* address comments
* add javadoc and logging the full stack trace of exception
* add error message
* Fix issue that tasks failed because of no sink for identifier
* make find sinks to persist run in one callable together with the actual persist work
* Revert "make find sinks to persist run in one callable together with the actual persist work"
This reverts commit a24a2d80ae.
* Broker: Await initialization before finishing startup.
In particular, hold off on announcing the service and starting the
HTTP server until the server view and SQL metadata cache are finished
initializing. This closes a window of time where a Broker could return
partial results shortly after startup.
As part of this, some simplification of server-lifecycle service
announcements. This helps ensure that the two different kinds of
announcements we do (legacy and new-style) stay in sync.
* Remove unused imports.
* Fix NPE in ServerRunnable.
* make logs that are only useful for debugging be at debug level so log volume is much more chill
* info level messages for total merge buffer allocated/free
* more chill compaction logs
* FilteredRequestLogger: Fix start/stop, invalid delegate behavior.
Fixes two bugs:
1) FilteredRequestLogger did not start/stop the delegate.
2) FilteredRequestLogger would ignore an invalid delegate type, and
instead silently substitute the "noop" logger. This was due to a larger
problem with RequestLoggerProvider setup in general; the fix here is
to remove "defaultImpl" from the RequestLoggerProvider interface, and
instead have JsonConfigurator be responsible for creating the
default implementations. It is stricter about things than the old system
was, and is only willing to make a noop logger if it doesn't see any
request logger configs. Otherwise, it'll raise a provision error.
* Remove unneeded annotations.
* autosize processing buffers based on direct memory sizing
* remove oops, more test
* max 1gb autosize buffers, test, start of docs
* fix oops
* revert accidental change
* print buffer size in exception
* change the things
* remove AbstractResourceFilter.isApplicable because it is not, add tests for OverlordResource.doShutdown and OverlordResource.shutdownTasksForDatasource
* cleanup
* Fix missing default config in some calls to coordinator dynamic configs.
The lack of a default config meant that if someone called an API
_without_ a default config before one _with_ a default config, then
the default value would get stuck at null instead of the intended
default value. I noticed this in a cluster where calling /druid/coordinator/v1/config
before a coordinator had fully started up would lead to NPEs during
DruidCoordinatorRuleRunner.
This patch makes the default configs consistent across all calls.
* Remove unnecessary null check.
* Add checkstyle rules about imports and empty lines between members
* Add suppressions
* Update Eclipse import order
* Add empty line
* Fix StatsDEmitter
* Use current coordinator leader instead of cached one (#6551)
Check the response status and throw exception if not OK
* Modify tests
* PR comment
* Add the correct check for status of BytesAccumulatingResponseHandler
* Move the status check into JsonParserIterator so sql query outputs meaningful message on failure
* Fix tests
* Period load/drop/broadcast rules should include the future by default
* address comments
* adjust coordinator console and tweak docs
* address comments
* fix travis-ci
This PR allows to control the fields in `RequestLogEvent`, emitted in `EmittingRequestLogger`. In our case, we want to get rid of the `intervals` fields of the query objects that are a part of `DefaultRequestLogEvent`. They are enormous (thousands of segments) and not useful.
Related to #5522, FYI @a2l007.
* Prohibit some guava collection APIs and use JDK APIs directly
* reset files that changed by accident
* sort codestyle/druid-forbidden-apis.txt alphabetically
* 1. Mysql default transaction isolation is REPEATABLE_READ, treat it as READ_COMMITTED will reduce insert id conflict.
2. Add an index to 'dataSource used end' is work well for the most of scenarios(get recently segments), and it will speed up sync add pending segments in DB.
3. 'select and insert' is not need within transaction.
* Use TaskLockbox.doInCriticalSection instead of synchronized syntax to speed up insert pending segments.
* fix typo for NullPointerException
* Use NodeType enum instead of Strings
* Make NodeType constants uppercase
* Fix CommonCacheNotifier and NodeType/ServerType comments
* Reconsidering comment
* Fix import
* Add a comment to CommonCacheNotifier.NODE_TYPES
* o- Query Response format to be based on http 'accept' header & Query Payload contenty type to be based on 'content-type' header
* o- Query Response format to be based on http 'accept' header & Query Payload contenty type to be based on 'content-type' header
o- if Accept header is absent, it defaults to Content-Type header
* Feature: Query Response format to be based on http 'accept' header & Query Payload content type to be based on 'content-type' PR #4033
Minor change to a comment - restoring to previous wording
* Feature: Query Response format to be based on http 'accept' header & Query Payload content type to be based on 'content-type' PR #4033
o- minor change to check for empty string
* Fix inconsistent segment size(#6448)
* Fix the segment size for published segments
* Changes to get numReplicas
* Make coordinator segments API truly streaming
* Changes to store partial segment data
* Simplify SegmentMetadataHolder
* Store partial the columns from available segments
* Address comments
* Added SystemSchema with following tables (#5989)
* SEGMENTS table provides details on served and published segments
* SERVERS table provides details on data servers
* SERVERSEGMETS table is the JOIN of SEGMENTS and SERVERS
* TASKS table provides details on tasks
* Add documentation for system schema
* Fix static-analysis warnings
* Address PR comments
*Add unit tests
* Fix a test
* Try to fix a test
* Fix a bug around replica count
* rename io.druid to org.apache.druid
* Major change is to make tasks and segment queries streaming
* Made tasks/segments stream to calcite instead of storing it in memory
* Add num_rows to segments table
* Refactor JsonParserIterator
* Replace with closeable iterator
* Fix docs, make num_rows column nullable, some unit test changes
* make num_rows column type long, allow it to be null
fix a compile error after merge, add TrafficCop param to InputStreamResponseHandler
* Filter null rows for segments table from Linq4j enumerable
* change num_replicas datatype to long in segments table
* Fix some tests and address comments
* Doc updates, other PR comments
* Update tests
* Address comments
* Add auth check
* Update docs
* Refactoring
* Fix teamcity warning, change the getQueryableServer in TimelineServerView
* Fix compilation after rebase
* Use the stream API from AuthorizationUtils
* Added LeaderClient interface and NoopDruidLeaderClient class
* Revert "Added LeaderClient interface and NoopDruidLeaderClient class"
This reverts commit 100fa46e39.
* Make the naming consistent to server_segments for the join table
* Add ForbiddenException on auth check failure
* Remove static block from SystemSchema
* Try to fix a test in CalciteQueryTest due to rename of server_segments
* Fix the json output format in the coordinator API
* Add auth check in the segments API
* Add null check to avoid NPE
* Use annonymous class object instead of mock for DruidLeaderClient in SqlBenchmark
* Fix test failures, type long/BIGINT can be nullable
* Revert long nullability to fix tests
* Fix style for tests
* PR comments
* Address PR comments
* Add the missing BytesAccumulatingResponseHandler class
* Use Sequences.withBaggage in DruidPlanner
* Fix docs, add comments
* Close the iterator if hasNext returns false
This PR accumulates many refactorings and small improvements that I did while preparing the next change set of https://github.com/druid-io/druid/projects/2. I finally decided to make them a separate PR to minimize the volume of the main PR.
Some of the changes:
- Renamed confusing "Generic Column" term to "Numeric Column" (what it actually implies) in many class names.
- Generified `ComplexMetricExtractor`
* Added backpressure metric
* Updated channelReadable to AtomicBoolean and fixed broken test
* Moved backpressure metric logic to NettyHttpClient
* Fix placement of calculating backPressureDuration
* Add support targetCompactionSizeBytes for compactionTask
* fix test
* fix a bug in keepSegmentGranularity
* fix wrong noinspection comment
* address comments
The indexes introduced in #6348 were on the wrong table. The tests
did not catch them due to retries on the create table steps (the
first try created the table but not the bogus indexes; the second
try noticed that the table already existed and did nothing). This
patch doesn't fix the issue with the tests, since the best way to
do that would be to do the table and index creation in a
transaction; but, this is not supported by all of our supported
database engines.
* Adding licenses and enable apache-rat-plugi.
Change-Id: I4685a2d9f1e147855dba69329b286f2d5bee3c18
* restore the copywrite of demo_table and add it to the list of allowed ones
Change-Id: I2a9efde6f4b984bc1ac90483e90d98e71f818a14
* revirew comments
Change-Id: I0256c930b7f9a5bb09b44b5e7a149e6ec48cb0ca
* more fixup
Change-Id: I1355e8a2549e76cd44487abec142be79bec59de2
* align
Change-Id: I70bc47ecb577bdf6b91639dd91b6f5642aa6b02f
* 'suspend' and 'resume' support for kafka indexing service
changes:
* introduces `SuspendableSupervisorSpec` interface to describe supervisors which support suspend/resume functionality controlled through the `SupervisorManager`, which will gracefully shutdown the supervisor and it's tasks, update it's `SupervisorSpec` with either a suspended or running state, and update with the toggled spec. Spec updates are provided by `SuspendableSupervisorSpec.createSuspendedSpec` and `SuspendableSupervisorSpec.createRunningSpec` respectively.
* `KafkaSupervisorSpec` extends `SuspendableSupervisorSpec` and now supports suspend/resume functionality. The difference in behavior between 'running' and 'suspended' state is whether the supervisor will attempt to ensure that indexing tasks are or are not running respectively. Behavior is identical otherwise.
* `SupervisorResource` now provides `/druid/indexer/v1/supervisor/{id}/suspend` and `/druid/indexer/v1/supervisor/{id}/resume` which are used to suspend/resume suspendable supervisors
* Deprecated `/druid/indexer/v1/supervisor/{id}/shutdown` and moved it's functionality to `/druid/indexer/v1/supervisor/{id}/terminate` since 'shutdown' is ambiguous verbage for something that effectively stops a supervisor forever
* Added ability to get all supervisor specs from `/druid/indexer/v1/supervisor` by supplying the 'full' query parameter `/druid/indexer/v1/supervisor?full` which will return a list of json objects of the form `{"id":<id>, "spec":<SupervisorSpec>}`
* Updated overlord console ui to enable suspend/resume, and changed 'shutdown' to 'terminate'
* move overlord console status to own column in supervisor table so does not look like garbage
* spacing
* padding
* other kind of spacing
* fix rebase fail
* fix more better
* all supervisors now suspendable, updated materialized view supervisor to support suspend, more tests
* fix log
* Broker backpressure.
Adds a new property "druid.broker.http.maxQueuedBytes" and a new context
parameter "maxQueuedBytes". Both represent a maximum number of bytes queued
per query before exerting backpressure on the channel to the data server.
Fixes#4933.
* Fix query context doc.
* resolves#5898 by adding maxTotalRows to incremental publishing kafka index task and appenderator based realtime indexing task, as available in IndexTask
* address review comments
* changes due to review
* merge fail
* Rename io.druid to org.apache.druid.
* Fix META-INF files and remove some benchmark results.
* MonitorsConfig update for metrics package migration.
* Reorder some dimensions in inner queries for some reason.
* Fix protobuf tests.
* Fix all inspection errors currently reported.
TeamCity builds on master are reporting inspection errors, possibly
because there was a while where it was not running due to the Apache
migration, and there was some drift.
* Fix one more location.
* Fix tests.
* Another fix.
* Fix three bugs with segment publishing.
1. In AppenderatorImpl: always use a unique path if requested, even if the segment
was already pushed. This is important because if we don't do this, it causes
the issue mentioned in #6124.
2. In IndexerSQLMetadataStorageCoordinator: Fix a bug that could cause it to return
a "not published" result instead of throwing an exception, when there was one
metadata update failure, followed by some random exception. This is done by
resetting the AtomicBoolean that tracks what case we're in, each time the
callback runs.
3. In BaseAppenderatorDriver: Only kill segments if we get an affirmative false
publish result. Skip killing if we just got some exception. The reason for this
is that we want to avoid killing segments if they are in an unknown state.
Two other changes to clarify the contracts a bit and hopefully prevent future bugs:
1. Return SegmentPublishResult from TransactionalSegmentPublisher, to make it
more similar to announceHistoricalSegments.
2. Make it explicit, at multiple levels of javadocs, that a "false" publish result
must indicate that the publish _definitely_ did not happen. Unknown states must be
exceptions. This helps BaseAppenderatorDriver do the right thing.
* Remove javadoc-only import.
* Updates.
* Fix test.
* Fix tests.
* Cache: Add maxEntrySize config.
The idea is this makes it more feasible to cache query types that
can potentially generate large result sets, like groupBy and select,
without fear of writing too much to the cache per query.
Includes a refactor of cache population code in CachingQueryRunner and
CachingClusteredClient, such that they now use the same CachePopulator
interface with two implementations: one for foreground and one for
background.
The main reason for splitting the foreground / background impls is
that the foreground impl can have a more effective implementation of
maxEntrySize. It can stop retaining subvalues for the cache early.
* Add CachePopulatorStats.
* Fix whitespace.
* Fix docs.
* Fix various tests.
* Add tests.
* Fix tests.
* Better tests
* Remove conflict markers.
* Fix licenses.
* Native parallel indexing without shuffle
* fix build
* fix ci
* fix ingestion without intervals
* fix retry
* fix retry
* add it test
* use chat handler
* fix build
* add docs
* fix ITUnionQueryTest
* fix failures
* disable metrics reporting
* working
* Fix split of static-s3 firehose
* Add endpoints to supervisor task and a unit test for endpoints
* increase timeout in test
* Added doc
* Address comments
* Fix overlapping locks
* address comments
* Fix static s3 firehose
* Fix test
* fix build
* fix test
* fix typo in docs
* add missing maxBytesInMemory to doc
* address comments
* fix race in test
* fix test
* Rename to ParallelIndexSupervisorTask
* fix teamcity
* address comments
* Fix license
* addressing comments
* addressing comments
* indexTaskClient-based segmentAllocator instead of CountingActionBasedSegmentAllocator
* Fix race in TaskMonitor and move HTTP endpoints to supervisorTask from runner
* Add more javadocs
* use StringUtils.nonStrictFormat for logging
* fix typo and remove unused class
* fix tests
* change package
* fix strict build
* tmp
* Fix overlord api according to the recent change in master
* Fix it test
* Optimize per-segment queries
* Always optimize, add unit test
* PR comments
* Only run IntervalDimFilter optimization on __time column
* PR comments
* Checkstyle fix
* Add test for non __time column
* Remove some unnecessary task storage internal APIs.
- Remove MetadataStorageActionHandler's getInactiveStatusesSince and getActiveEntriesWithStatus.
- Remove TaskStorage's getCreatedDateTimeAndDataSource.
- Remove TaskStorageQueryAdapter's getCreatedTime, and getCreatedDateAndDataSource.
- Migrated all callers to getActiveTaskInfo and getCompletedTaskInfo.
This has one side effect: since getActiveTaskInfo (new) warns and continues when it
sees unreadable tasks, but getActiveEntriesWithStatus threw an exception when it
encountered those, it means that after this patch bad tasks will be ignored when
syncing from metadata storage rather than causing an exception to be thrown.
IMO, this is an improvement, since the most likely reason for bad tasks is either:
- A new version introduced an additional validation, and a pre-existing task doesn't
pass it.
- You are rolling back from a newer version to an older version.
In both cases, I believe you would want to skip tasks that can't be deserialized,
rather than blocking overlord startup.
* Remove unused import.
* Fix formatting.
* Fix formatting.
* Various changes about druid-services module
* Patch improvements from reviewer
* Add ToArrayCallWithZeroLengthArrayArgument & ArraysAsListWithZeroOrOneArgument into inspection profile
* Fix ArraysAsListWithZeroOrOneArgument
* Fix conflict
* Fix ToArrayCallWithZeroLengthArrayArgument
* Fix AliEqualsAvoidNull
* Remove blank line
* Remove unused import clauses
* Fix code style in TopNQueryRunnerTest
* Fix conflict
* Don't use Collections.singletonList when converting the type of array type
* Add argLine into maven-surefire-plugin in druid-process module & increase the timeout value for testMoveSegment testcase
* Roll back the latest commit
* Add java.io.File#toURL() into druid-forbidden-apis
* Using Boolean.parseBoolean instead of Boolean.valueOf for CliCoordinator#isOverlord
* Add a new regexp element into stylecode xml file
* Fix style error for new regexp
* Set the level of ArraysAsListWithZeroOrOneArgument as WARNING
* Fix style error for new regexp
* Add option BY_LEVEL for ToArrayCallWithZeroLengthArrayArgument in inspection profile
* Roll back the level as ToArrayCallWithZeroLengthArrayArgument as ERROR
* Add toArray(new Object[0]) regexp into checkstyle config file & fix them
* Set the level of ArraysAsListWithZeroOrOneArgument as ERROR & Roll back the level of ToArrayCallWithZeroLengthArrayArgument as WARNING until Youtrack fix it
* Add a comment for string equals regexp in checkstyle config
* Fix code format
* Add RedundantTypeArguments as ERROR level inspection
* Fix cannot resolve symbol datasource
* Add support to filter on datasource for active tasks
* Added datasource filter to sql query for active tasks
* Fixed unit tests
* Address PR comments
* Fix NPE while handling CheckpointNotice
* fix code style
* Fix test
* fix test
* add a log for creating a new taskGroup
* fix backward compatibility in KafkaIOConfig
* this will fix it
* filter destinations to not consider servers already serving segment
* fix it
* cleanup
* fix opposite day in ImmutableDruidServer.equals
* simplify
False failures on Travis due to spurious timeout (in turn due to noisy
neighbors) is a bigger problem than legitimate failures taking too long
to time out. So it makes sense to extend timeouts.
* Fix SQL Server select query in createInactiveStatusesSinceQuery() method.
SQL server does not support LIMIT N in select queries. Instead it has TOP N to limiting number of query results.
And TOP N is already added in the select statement as per maxNumStatuses value.
* Add parentheses for TOP in SELECT statement as SQL Servers no longer support TOP without parentheses.
* Add the new tasks api in overlordResource
It takes 4 optional query params
* state(pending/running/waiting/compelte)
* dataSource
* interval (applies to completed tasks)
* maxCompletedTasks (applies to completed tasks)
If all params are null, the api returns all the tasks
* Add the state to each task returned by tasks endpoint
* divide active tasks into waiting, pending or running
* Add more unit tests
* Add UNKNOWN state to TaskState
* Fix the authorization calls
* WIP: PR comments
Added new class to capture task info for caching
Other refactoring
* Refactoring : move TaskStatus class to druid-api
so it can be accessed within server
And other related classes like TaskState and TaskStatusPlus are in api
* Remove unused class and apis accessing it
* Add a separate cache for recently completed tasks
This is to mainly capture the task type from payload
* Ignore a test
* Add a RuntimeTaskState to encompass all states a task can be in
* Revert "Add a RuntimeTaskState to encompass all states a task can be in"
This reverts commit 2a527a0731.
* Fix wrong api call
* Fix and unignore tests
* Remove waiting,pending state from TaskState
* Add RunnerTaskState
* Missed the annotation runnerStatusCode
* Fix the creationTime
* Fix the createdTime and queueInsertionTime for running/active tasks
* Clean up tests
* Add javadocs
* Potentially fix the teamcity build
* Address PR comments
*Get rid of TaskInfoBuilder
*Make TaskInfoMapper static nested class
*Other changes
* fix import in MaterializedViewSupervisor after merge
* Address PR comments on
* Replace global cache with local map
* combine multiple queries into one
* Removed unused code
* Fix unit tests
Fix a bug in securedTaskStatusPlus
* Remove getRecentlyFinishedTaskStatuses method
Change TaskInfoMapper signature to add generic type
* Address PR comments
* Passed datasource as argument to be used in sql query
* Other minor fixes
* Address PR comments
*Some minor changes, rename method, spacing changes
* Add early auth check if datasource is not null
* Fix test case
* Add max limit to getRecentlyFinishedTaskInfo in HeapMemoryTaskStorage
* Add TaskLocation to Anytask object
* Address PR comments
* Fix a bug in test case causing ClassCastException
* implement materialized view
* modify code according to jihoonson's comments
* modify code according to jihoonson's comments - 2
* add documentation about materialized view
* use new HadoopTuningConfig in pr 5583
* add minDataLag and fix optimizer bug
* correct value of DEFAULT_MIN_DATA_LAG_MS
* modify code according to jihoonson's comments - 3
* use the boolean expression instead of if-else
* Fix defaultQueryTimeout
- set default timeout in query context before query fail time is evaluated
Remove unused import
* Address failing checks
* Addressing code review comments
* Removed line that was no longer used
* Anonymous authenticator that authenticates all requests and then directs them to an authorizer.
* Adding documentation
* Removed some fields from class AnonymousAuthenticator
* Updating docs
* fix freeSpacePercent in segmentCache.locations
* the check should probably test the other way around
* documentation should put the option in the right place
* examples have a superfluous backslash
* add test to verify correct behavior
* switch to Path and test with jimfs
Path allows to use different filesystems.
Jimfs provides an actual (in memory) filesystem.
This also allows more complex test scenarios.
The behavior should be unchanged by this commit.
* Revert "switch to Path and test with jimfs"
This reverts commit 8b9a418d65.
* VersionedIntervalTimeline: Optimize construction with heavily populated holders.
Each time a segment is "add"ed to a timeline, "isComplete" is called on the holder
that it is added to. "isComplete" is an O(segments per chunk) operation, meaning
that adding N segments to a chunk is an O(N^2) operation. This blows up badly if
we have thousands of segments per chunk.
The patch defers the "isComplete" check until after all segments have been
inserted.
* Fix imports.
* The check for maxBytesInMemory should be >= 0 instead of > 0
* if the default value is 0, the actual check could be skipped
* fix the message for persistReasons
* Address PR comments
* if maxBytes set -1, make is Long.MAX_VAL, so we do not need to check if it's 0 or -1
* set the maxBytesTuningconfig in AppenderatorImpl constructor to avoid duplicate code
* fix the failing test cases
* Address PR comments
* Adding decoration method to proxy servlet
Change-Id: I872f9282fb60bfa20524271535980a36a87b9621
* moving the proxy request decoration to authenticators
Change-Id: I7f94b9ff5ecf08e8abf7169b58bc410f33148448
* added docs
Change-Id: I901543e52f0faf4666bfea6256a7c05593b1ae70
* use the authentication result to decorate request
Change-Id: I052650de9cd02b4faefdbcdaf2332dd3b2966af5
* adding authenticated by name
Change-Id: I074d2933460165feeddb19352eac9bd0f96f42ca
* ensure that authenticator is not null
Change-Id: Idb58e308f90db88224a06f3759114872165b24f5
* fix types and minor bug
Change-Id: I6801d49a05d5d8324406fc0280286954eb66db10
* fix typo
Change-Id: I390b12af74f44d760d0812a519125fbf0df4e97b
* use actual type names
Change-Id: I62c3ee763363781e52809ec912aafd50b8486b8e
* set authenitcatedBy to null for AutheticationResults created by
Escalator.
Change-Id: I4a675c372f59ebd8a8d19c61b85a1e4bf227a8ba
* add default caffeine cache size based on runtime Xmx or max 1GB
* update docs for caffeine cache
* fix formatting
* test caffeine size should never be less than 0
* set caffeine max default size to 1G not 1M
* fix caffeine cache tests
* This commit introduces a new tuning config called 'maxBytesInMemory' for ingestion tasks
Currently a config called 'maxRowsInMemory' is present which affects how much memory gets
used for indexing.If this value is not optimal for your JVM heap size, it could lead
to OutOfMemoryError sometimes. A lower value will lead to frequent persists which might
be bad for query performance and a higher value will limit number of persists but require
more jvm heap space and could lead to OOM.
'maxBytesInMemory' is an attempt to solve this problem. It limits the total number of bytes
kept in memory before persisting.
* The default value is 1/3(Runtime.maxMemory())
* To maintain the current behaviour set 'maxBytesInMemory' to -1
* If both 'maxRowsInMemory' and 'maxBytesInMemory' are present, both of them
will be respected i.e. the first one to go above threshold will trigger persist
* Fix check style and remove a comment
* Add overlord unsecured paths to coordinator when using combined service (#5579)
* Add overlord unsecured paths to coordinator when using combined service
* PR comment
* More error reporting and stats for ingestion tasks (#5418)
* Add more indexing task status and error reporting
* PR comments, add support in AppenderatorDriverRealtimeIndexTask
* Use TaskReport instead of metrics/context
* Fix tests
* Use TaskReport uploads
* Refactor fire department metrics retrieval
* Refactor input row serde in hadoop task
* Refactor hadoop task loader names
* Truncate error message in TaskStatus, add errorMsg to task report
* PR comments
* Allow getDomain to return disjointed intervals (#5570)
* Allow getDomain to return disjointed intervals
* Indentation issues
* Adding feature thetaSketchConstant to do some set operation in PostAgg (#5551)
* Adding feature thetaSketchConstant to do some set operation in PostAggregator
* Updated review comments for PR #5551 - Adding thetaSketchConstant
* Fixed CI build issue
* Updated review comments 2 for PR #5551 - Adding thetaSketchConstant
* Fix taskDuration docs for KafkaIndexingService (#5572)
* With incremental handoff the changed line is no longer true.
* Add doc for automatic pendingSegments (#5565)
* Add missing doc for automatic pendingSegments
* address comments
* Fix indexTask to respect forceExtendableShardSpecs (#5509)
* Fix indexTask to respect forceExtendableShardSpecs
* add comments
* Deprecate spark2 profile in pom.xml (#5581)
Deprecated due to https://github.com/druid-io/druid/pull/5382
* CompressionUtils: Add support for decompressing xz, bz2, zip. (#5586)
Also switch various firehoses to the new method.
Fixes#5585.
* This commit introduces a new tuning config called 'maxBytesInMemory' for ingestion tasks
Currently a config called 'maxRowsInMemory' is present which affects how much memory gets
used for indexing.If this value is not optimal for your JVM heap size, it could lead
to OutOfMemoryError sometimes. A lower value will lead to frequent persists which might
be bad for query performance and a higher value will limit number of persists but require
more jvm heap space and could lead to OOM.
'maxBytesInMemory' is an attempt to solve this problem. It limits the total number of bytes
kept in memory before persisting.
* The default value is 1/3(Runtime.maxMemory())
* To maintain the current behaviour set 'maxBytesInMemory' to -1
* If both 'maxRowsInMemory' and 'maxBytesInMemory' are present, both of them
will be respected i.e. the first one to go above threshold will trigger persist
* Address code review comments
* Fix the coding style according to druid conventions
* Add more javadocs
* Rename some variables/methods
* Other minor issues
* Address more code review comments
* Some refactoring to put defaults in IndexTaskUtils
* Added check for maxBytesInMemory in AppenderatorImpl
* Decrement bytes in abandonSegment
* Test unit test for multiple sinks in single appenderator
* Fix some merge conflicts after rebase
* Fix some style checks
* Merge conflicts
* Fix failing tests
Add back check for 0 maxBytesInMemory in OnHeapIncrementalIndex
* Address PR comments
* Put defaults for maxRows and maxBytes in TuningConfig
* Change/add javadocs
* Refactoring and renaming some variables/methods
* Fix TeamCity inspection warnings
* Added maxBytesInMemory config to HadoopTuningConfig
* Updated the docs and examples
* Added maxBytesInMemory config in docs
* Removed references to maxRowsInMemory under tuningConfig in examples
* Set maxBytesInMemory to 0 until used
Set the maxBytesInMemory to 0 if user does not set it as part of tuningConfing
and set to part of max jvm memory when ingestion task starts
* Update toString in KafkaSupervisorTuningConfig
* Use correct maxBytesInMemory value in AppenderatorImpl
* Update DEFAULT_MAX_BYTES_IN_MEMORY to 1/6 max jvm memory
Experimenting with various defaults, 1/3 jvm memory causes OOM
* Update docs to correct maxBytesInMemory default value
* Minor to rename and add comment
* Add more details in docs
* Address new PR comments
* Address PR comments
* Fix spelling typo
The "lock" object was used to synchronize start/stop as well as synchronize removals
from segmentsToDelete (when a segment is done dropping). This could cause a deadlock
if a segment-load throws an exception during loadLocalCache. loadLocalCache is run
by start() while it holds the lock, but then it spawns loading threads, and those
threads will try to acquire the "segmentsToDelete" lock if they want to drop a corrupt
segments.
I don't see any reason for these two locks to be the same lock, so I split them.
* Fix coordinator's dataSource api with full parameter
* address comment
* Add a constructor for json serde and fix result order
* Change to immutableSortedMap
* Revert immutableSortedMap to treeMap
* Add getters for AlertEvent
* Move PublicApi and ExtensionPoint to java-util
* Fix publicapi annotation usage
* Add publicapi annotations to ServiceMetricEvent and RequestLogEvent
* Add config to allow setting up custom unsecured paths for druid nodes.
* return all resources for Unsecured paths
* review comment - Add test
* fix tests
* fix test
* Add more indexing task status and error reporting
* PR comments, add support in AppenderatorDriverRealtimeIndexTask
* Use TaskReport instead of metrics/context
* Fix tests
* Use TaskReport uploads
* Refactor fire department metrics retrieval
* Refactor input row serde in hadoop task
* Refactor hadoop task loader names
* Truncate error message in TaskStatus, add errorMsg to task report
* PR comments
* drop selection through cost balancer
* use collections.emptyIterator
* add test to ensure does not drop from server with larger loading queue with cost balancer
* javadocs and comments to clear things up
* random drop for completeness
* fix issue where assign primary assigns segments to all historical servers in cluster
* fix test
* add test to ensure primary assignment will not assign to another server while loading is in progress
* Add graceful shutdown timeout
* Handle interruptedException
* Incorporate code review comments
* Address code review comments
* Poll for activeConnections to be zero
* Use statistics handler to get active requests
* Use native jetty shutdown gracefully
* Move log line back to where it was
* Add unannounce wait time
* Make the default retain prior behavior
* Update docs with new config defaults
* Make duration handling on jetty shutdown more consistent
* StatisticsHandler is a wrapper
* Move jetty lifecycle error logging to error
* Use the official aws-sdk instead of jet3t
* fix compile and serde tests
* address comments and fix test
* add http version string
* remove redundant dependencies, fix potential NPE, and fix test
* resolve TODOs
* fix build
* downgrade jackson version to 2.6.7
* fix test
* resolve the last TODO
* support proxy and endpoint configurations
* fix build
* remove debugging log
* downgrade hadoop version to 2.8.3
* fix tests
* remove unused log
* fix it test
* revert KerberosAuthenticator change
* change hadoop-aws scope to provided in hdfs-storage
* address comments
* address comments
* Future-proof some Guava usage
* Use a java-util EmptyIterator instead of Guava's
* Change some of the guava future handling to do manual async
transforms. Guava changes transform into transformAsync by deprecating
transform in ONLY Guava 19. Then its gone in 20
* Use `Collections.emptyIterator()`
* Pretty formatting
* Make listenable future transforms a thing in default druid
* Format fix
* Add forbidden guava apis
* Make the ListenableFutrues.transformAsync have comments
* Undo intellij bad pattern matching in comments
* Futrues --> Futures
* Add empty iterators forbidding
* Fix extra `A`
* Correct method signature
* Address review comments
* Finish Gian review comments
* Proper syntax from https://github.com/policeman-tools/forbidden-apis/wiki/SignaturesSyntax
* Fix round robining in router.
Say that ten times fast.
For query endpoints, AsyncQueryForwardingServlet called hostFinder.getDefaultServer()
to set a default server, followed by hostFinder.getServer(inputQuery) to override it
with query-specific routing. Since hostFinder is round-robin, this skips a server.
When there are only two servers, one server is _always_ skipped and the router sends
all queries to the same broker.
* Adjust spacing.
* adding a properties endpoint in status resource
* checkstyle fixes
* more checkstyle corrections
* correcting the resource filter for properties endpoint
* adding feature of hiding sensitive properties
* checkstyle changes
* review changes for adding default hidden properties and using jackson for arrays value
* making review changes
* Fix early publishing to early pushing in batch indexing & refactor appenderatorDriver
* fix compile
* rename and add more javadocs
* Fix conflicts
* address comments
* revert await executors
* fix test
* Properly set "identity" in query metrics.
This patch adds an "identity" field to QueryPlus and sets it in
QueryLifecycle when the query starts executing. This is important
because it allows it to be used for future QueryMetrics created
by that QueryPlus object.
We also add "identity" to the request-level QueryMetrics object
created in emitLogsAndMetrics.
* Remove unused method.