* Kafka Index Task that supports Incremental handoffs
- Incrementally handoff segments when they hit maxRowsPerSegment limit
- Decouple segment partitioning from Kafka partitioning, all records from consumed partitions go to a single druid segment
- Support for restoring task on middle manager restarts by check pointing end offsets for segments
* take care of review comments
* make getCurrentOffsets call async, keep track of publishing sequence, review comments
* fix setEndoffset duplicate request handling, formatting
* fix unit test
* backward compatibility
* make AppenderatorDriverMetadata backwards compatible
* add unit test
* fix deadlock between persist and push executors in AppenderatorImpl
* fix formatting
* use persist dir instead of work dir
* review comments
* fix deadlock
* actually fix deadlock
* maxQueryTimeout property in runtime properties.
* extra line
* move withTimeoutAndMaxScatterGatherBytes method to QueryLifeCycle.
* Fix initialize method.
* remove unused import.
* doc update.
* some more details in doc about query failure..
* minor fix.
* decorating QueryRunner to set and verify context. Added by servers.
* remove whitespace.
* use ImmutableDruidDataSource for map and set
* address comments
* unused import
* allow returning only ImmutableDruidDataSource in MetadataSegmentManager
* address comments
* remove TreeSet
* revert to use TreeSet
* Add compaction task
* added doc
* use combining aggregators
* address comments
* add support for dimensionsSpec
* fix getUniqueDims and getUniqueMetics
* find unique dimensionsSpec
* fix compilation
* add unit test
* fix test
* fix test
* test for different dimension orderings and types, and doc for type and ordering
* add control for custom ordering and type
* update doc
* fix compile
* fix compile
* add segments param
* fix serde error
* fix build
* Fix havingSpec on complex aggregators.
- Uses the technique from #4883 on DimFilterHavingSpec too.
- Also uses Transformers from #4890, necessitating a move of that and other
related classes from druid-server to druid-processing. They probably make
more sense there anyway.
- Adds a SQL query test.
Fixes#4957.
* Remove unused import.
* Introduce "transformSpec" at ingest-time.
It accepts a "filter" (standard query filter object) and "transforms" (a
list of objects with "name" and "expression"). These can be used to do
filtering and single-row transforms without need for a separate data
processing job.
The "expression" fields use the same expression language as other
expression-based feature.
* Remove forbidden api.
* Fix compile error.
* Fix tests.
* Some more changes.
- Add nullable annotation to Firehose.nextRow.
- Add tests for index task, realtime task, kafka task, hadoop mapper,
and ingestSegment firehose.
* Fix bad merge.
* Adjust imports.
* Adjust whitespace.
* Make Transform into an interface.
* Add missing annotation.
* Switch logger.
* Switch logger.
* Adjust test.
* Adjustment to handling for DatasourceIngestionSpec.
* Fix test.
* CR comments.
* Remove unused method.
* Add javadocs.
* More javadocs, and always decorate.
* Fix bug in TransformingStringInputRowParser.
* Fix bad merge.
* Fix ISFF tests.
* Fix DORC test.
* Changes for lookup synchronization
* Refactor of Lookup classes
* Minor refactors and doc update
* Change coordinator instance to be retrieved by DruidLeaderClient
* Wait before thread shutdown
* Make disablelookups flag true by default
* Update docs
* Rename flag
* Move executorservice shutdown to finally block
* Update LookupConfig
* Refactoring and doc changes
* Remove lookup config constructor
* Revert Lookupconfig constructor changes
* Add tests to LookupConfig
* Make executorservice local
* Update LRM
* Move ListeningScheduledExecutorService to ExecutorCompletionService
* Move exception to outer block
* Remove check to see future is done
* Remove unnecessary assignment
* Add logging
"No worker selection strategy set." would get logged any time tryAssignTask runs
in the default configuration, which is often. It also doesn't provide much value.
* Make AutoScaler, ProvisioningStrategy and BaseWorkerBehaviorConfig extension points; More logging in PendingTaskBasedWorkerProvisioningStrategy
* Address comments and fix a bug
* Extract method
* debug logging
* Rename BaseWorkerBehaviorConfig to WorkerBehaviorConfig and WorkerBehaviorConfig to DefaultWorkerBehaviorConfig
* Fixes
* remove ServerConfig from DruidNode as all information needs to be present in DruidNode serialized form
* sanitize output of /druid/coordinator/v1/cluster endpoint
* Use internal-discovery and http for talking to overlord/coordinator leaders
* CuratorDruidNodeDiscovery.getAllNodes() best effort 30 sec wait for cache initialization
* DruidLeaderClientProvider to eagerly instantiate DruidNodeDiscovery when needed so that DruidNodeDiscovery impl cache gets initialized well in time
* Revert "DruidLeaderClientProvider to eagerly instantiate DruidNodeDiscovery when needed so that DruidNodeDiscovery impl cache gets initialized well in time"
This reverts commit f1a2432614ba56ddc2d55fe47e990d17fcfd6129.
* add lifecycle to DruidLeaderClient to early initialize DruidNodeDiscovery so that it has its cache update well in time
* Collapse worker select strategies, change default, add strong affinity.
- Change default worker select strategy to equalDistribution. It is
more generally useful than fillCapacity.
- Collapse the *WithAffinity strategies into the regular ones. The
*WithAffinity strategies are retained for backwards compatibility.
- Change WorkerSelectStrategy to return nullable instead of Optional.
- Fix a couple of errors in the docs.
* Fix test.
* Review adjustments.
* Remove unused imports.
* Switch to DateTimes.nowUtc.
* Simplify code.
* Fix tests (worker assignment started off on a different foot)
* DruidLeaderSelector interface for leader election and Curator based impl. DruidCoordinator/TaskMaster are updated to use the new interface.
* add fake DruidNode binding in integration-tests module
* add docs on DruidLeaderSelector interface
* remove start/stop and keep register/unregister Listener in DruidLeaderSelector interface
* updated comments on DruidLeaderSelector
* cache the listener executor in CuratorDruidLeaderSelector
* use same latch owner name that was used before
* remove stuff related to druid.zk.paths.indexer.leaderLatchPath config
* randomize the delay when giving up leadership and restarting leader latch
* Add "round" option to cardinality and hyperUnique aggregators.
Also turn it on by default in SQL, to make math on distinct counts
work more as expected.
* Fix some compile errors.
* Fix test.
* Formatting.
* internal-discovery: interfaces for announcement/discovery, curator impls
* more tests
* address some review comments
* more fixes
* address more review comments
* simplify ObjectMapper setup in CuratorDruidNodeAnnouncerAndDiscoveryTest
* fix KafkaIndexTaskTest
* make lookupTier overridable via RealtimeIndexTask and KafkaIndexTask context
* make teamcity build happy
* Stop RemoteTaskRunner's cleanupExec using TaskMaster's lifecycle, not global injected lifecycle
* Prohibit starting Lifecycle twice; Make Lifecycle to reject addMaybeStartHandler() attempts in the process of stopping rather than entering deadlock
* Fix Lifecycle.addMaybeStartHandler()
* Remove RemoteTaskRunnerFactoryTest
* Add docs
* Language
* Address comments
* Fix RemoteTaskRunnerTestUtils
This allows the tasks to run concurrently. Additionally, rework
the partition-determining code in a couple ways:
- Use a task-id based sequenceName so concurrently running append
tasks do not clobber each others' segments.
- Make the list of shardSpecs empty when rollup is non-guaranteed, and
let allocators handle the creation of incremental shardSpecs.
* Rename ResourceManagementStrategy to ProvisioningStrategy, similarly for related classes. Make ProvisioningService non-global, created per RemoteTaskRunner instead. Add OverlordBlinkLeadershipTest.
* Fix RemoteTaskRunnerFactoryTest.testExecNotSharedBetweenRunners()
* Small fix
* Make SimpleProvisioner and PendingProvisioner more similar in details
* Fix executor name
* Style fixes
* Use LifecycleLock in RemoteTaskRunner
* Remove some unnecessary use of boxed types.
* Fix some incorrect format strings.
* Enable IDEA's MalformedFormatString inspection.
* Add a Checkstyle check for finding uses of incorrect logging packages.
* Fix some incorrect usages of the metamx logger.
* Bypass incorrect logger Checkstyle check where using the correct logger is not simple.
* Fix some more places where the wrong number of arguments are provided to format strings.
* Suppress `MalformedFormatString` inspection on legacy logging test.
* Use @SuppressWarnings rather than a noinspection suppression comment.
* Fix some more incorrect format strings.
* Suppress some more incorrect format string warnings where the incorrect string is intentional.
* Log the aggregator when closing it fails.
* Remove some unneeded log lines.
* Early publishing segments in the middle of data ingestion
* Remove unnecessary logs
* Address comments
* Refactoring the patch according to #4292 and address comments
* Set the total shard number of NumberedShardSpec to 0
* refactoring
* Address comments
* Fix tests
* Address comments
* Fix sync problem of committer and retry push only
* Fix doc
* Fix build failure
* Address comments
* Fix compilation failure
* Fix transient test failure
* Avoid usages of Default system Locale and printing to System.out or System.err in production code
* Fix Charset in DruidKerberosUtil
* Remove redundant string format in GenericIndexed
* Rename StringUtils.safeFormat() to unimportantSafeFormat(); add StringUtils.format() which fails as well as String.format()
* Fix testSafeFormat()
* More fixes of redundant StringUtils.format() inside ISE
* Rename unimportantSafeFormat() to nonStrictFormat()
* Remove ability to create segments in v8 format
* Fix IndexGeneratorJobTest
* Fix parameterized test name in IndexMergerTest
* Remove extra legacy merging stuff
* Remove legacy serializer builders
* Remove ConciseBitmapIndexMergerTest and RoaringBitmapIndexMergerTest
* Adding a flag to indicate when ObjectCachingColumnSelectorFactory need not be threadsafe.
* - Use of computeIfAbsent over putIfAbsent
- Replace Maps.newXXXMap() with normal instantiation
- Documentations on when is thread-safe required.
- Use Builders for On/OffheapIncrementalIndex
* - Optimization on computeIfAbsent
- Constant EMPTY DimensionsSpec
- Improvement on IncrementalIndexSchema.Builder
- Remove setting of default values
- Use var args for metrics
- Correction on On/OffheapIncrementalIndex Builders
- Combine On/OffheapIncrementalIndex Builders
* - Removing unused imports.
* - Helper method for testing with IncrementalIndex.Builder
* - Correction on javadoc.
* Style fix
* Expressions: Add ExprMacros, which have the same syntax as functions, but
can convert themselves to any kind of Expr at parse-time.
ExprMacroTable is an extension point for adding new ExprMacros. Anything
that might need to parse expressions needs an ExprMacroTable, which can
be injected through Guice.
* Address code review comments.
* Enable most IntelliJ 'Probable bugs' inspections
* Fix in RemoteTestNG
* Fix IndexSpec's equals() and hashCode() to include longEncoding
* Fix inspection errors
* Extract global isntance of natural().nullsFirst(); address comments
* Fix
* Use noinspection comments instead of SuppressWarnings on method for IntelliJ-specific inspections
* Prohibit Ordering.natural().nullsFirst() using Checkstyle
* Make using implicit system charset an error
* Use StringUtils.toUtf8() and fromUtf8() instead of String.getBytes() and new String()
* Use English locale in StringUtils.safeFormat()
* Restore comment
* refactor lag reporting and report lag at status endpoint
* refactor offset reporting logic to fetch offsets periodically vs. at request time
* remove JavaCompatUtils
* code review changes
* code review changes
* Adding s3a schema and s3a implem to hdfs storage module.
* use 2.7.3
* use segment pusher to make loadspec
* move getStorageDir and makeLoad spec under DataSegmentPusher
* fix uts
* fix comment part1
* move to hadoop 2.8
* inject deep storage properties
* set version to 2.7.3
* fix build issue about static class
* fix comments
* fix default hadoop default coordinate
* fix create filesytem
* downgrade aws sdk
* bump the version
* Refactoring Appenderator
1) Added publishExecutor and handoffExecutor for background publishing and handing segments off
2) Change add() to not move segments out in it
* Address comments
1) Remove publishTimeout for KafkaIndexTask
2) Simplifying registerHandoff()
3) Add increamental handoff test
* Remove unused variable
* Add persist() to Appenderator and more tests for AppenderatorDriver
* Remove unused imports
* Fix strict build
* Address comments
This is useful for putting them behind load balancers or proxies, as it lets
the load balancer know which server is currently active through an http health
check.
Also makes the method naming a little more consistent between coordinator and
overlord code.
* Optional long-polling based segment announcement via HTTP instead of Zookeeper
* address review comments
* make endpoint /druid-internal/v1 instead of /druid/internal so that jetty qos filters can be configured easily when needed
* update segment callback initialization to be called only after first segment list fetch has been succeeded from all servers
* address review comments
* remove size check not required anymore as only segment servers announce themselves and not all peon processes
* annouce segment server on historical only after cached segments are loaded
* fix checkstyle errors
* Make Errorprone the default compiler
* Address comments
* Make Error Prone's ClassCanBeStatic rule a error
* Preconditions allow only %s pattern
* Fix DruidCoordinatorBalancerTester
* Try to give the compiler more memory
* Remove distribution module activation on jdk 1.8 because only jdk 1.8 is used now
* Don't show compiler warnings
* Try different travis script
* Fix travis.yml
* Make Error Prone optional again
* For error-prone compiler
* Increase compiler's maxmem
* Don't run Error Prone for benchmarks because of OOM
* Skip install step in Travis
* Remove MetricHolder.writeToChannel()
* In travis.yml, check compilation before tests, because it may fail faster
* optionally add extensions to explicitly specified hadoopContainerClassPath
* note extensions always pushed in hadoop container when druid.extensions.hadoopContainerDruidClasspath is not provided explicitly
* RealtimeIndexTask to support alertTimeout in context and raise alert if task process exists after the timeout
* move alertTimeout config to tuningConfig and document
It wasn't doing anything useful (the sequences were being concatted, and
cursor.getTime() wasn't being called) and it defaulted to Granularities.NONE.
Changing it to Granularities.ALL gave me a 700x+ performance boost on a
small dataset I was reindexing (2m27s to 365ms). Most of that was from avoiding
making a lot of unnecessary column selectors.
* Add SameIntervalMergeTask for easier usage of MergeTask
* fix a bug and add ut
* remove same_interval_merge_sub from Task.java and remove other no needed code
* NN optimization for hdfs data segments.
* HdfsDataSegmentKiller, HdfsDataSegment finder changes to use new storage
format.Docs update.
* Common utility function in DataSegmentPusherUtil.
* new static method `makeSegmentOutputPathUptoVersionForHdfs` in JobHelper
* reuse getHdfsStorageDirUptoVersion in
DataSegmentPusherUtil.getHdfsStorageDir()
* Addressed comments.
* Review comments.
* HdfsDataSegmentKiller requested changes.
* extra newline
* Add maprfs.
* No more singleton. Reduce iterations
* Granularities
* Fix the delay in the test
* Add license header
* Remove unused imports
* Lot more unused imports from all the rearranging
* CR feedback
* Move javadoc to constructor
* Refactor Segment Granularity
* Beginning of one granularity
* Copy the fix for custom periods in segment-grunalrity over here.
* Remove the custom serialization for now.
* Compilation cleanup
* Reformat code
* Fixing unit tests
* Unify to use a single iterable
* Backward compatibility for rolling upgrade
* Minor check style. Cosmetic changes.
* Rename length and millis to duration
* CR feedback
* Minor changes.
* auto reset option for Kafka Indexing service in case message at the offset being fetched is not present anymore at kafka brokers
* review comments
* review comments
* reverted last change
* review comments
* review comments
* fix typo
* Fix#3795 (Java 7 compatibility).
Also introduce Animal Sniffer checks during build, which would
have caught the original problems.
* Add Animal Sniffer on caffeine-cache for JDK8.
* Fix PathChildrenCache's executorService leak in Announcer, CuratorInventoryManager and RemoteTaskRunner
* Use a single ExecutorService for all workerStatusPathChildrenCaches in RemoteTaskRunner
* Blacklist workers if they fail for too many times
* Adding documentation
* Changing to timeout to period and updating docs
* 1. Add configurable maxPercentageBlacklistWorkers
2. Rename variable
* Change maxPercentageBlacklistWorkers to double
* Remove thread.sleep
* Update emitter dependency to 0.4.0 and emit "version" dimension for all druid metrics, not only query metrics
* Remove unused imports
* Use empty string instead of "testing-version" as a version placeholder
* ability to not rollup at index time, make pre aggregation an option
* rename getRowIndexForRollup to getPriorIndex
* fix doc misspelling
* test query using no-rollup indexes
* fix benchmark fail due to jmh bug
Fixes inconsistent metric handling between the two implementations. Formerly,
RealtimePlumber only emitted query/segmentAndCache/time and query/wait and
Appenderator only emitted query/partial/time and query/wait (all per sink).
Now they both do the same thing:
- query/segmentAndCache/time, query/segment/time are the time spent per sink.
- query/cpu/time is the CPU time spent per query.
- query/wait/time is the executor waiting time per sink.
These generally match historical metrics, except segmentAndCache & segment
mean the same thing here, because one Sink may be partially cached and
partially uncached and we aren't splitting that out.
* validate X-Druid-Task-Id header in request and add header to response
* modify KafkaIndexTaskClient to take a TaskLocationProvider as the TaskLocation may not remain constant
* Datasource as lookup tier
* Adds an option to let indexing service tasks pull their lookup tier from the datasource they are working for.
* Fix bad docs for lookups lookupTier
* Add Datasource name holder
* Move task and datasource to be pulled from Task file
* Make LookupModule pull from bound dataSource
* Fix test
* Fix code style on imports
* Fix formatting
* Make naming better
* Address code comments about naming
- Introduce `AuthorizationInfo` interface, specific implementations of which would be provided by extensions
- If the `druid.auth.enabled` is set to `true` then the `isAuthorized` method of `AuthorizationInfo` will be called to perform authorization checks
- `AuthorizationInfo` object will be created in the servlet filters of specific extension and will be passed as a request attribute with attribute name as `AuthConfig.DRUID_AUTH_TOKEN`
- As per the scope of this PR, all resources that needs to be secured are divided into 3 types - `DATASOURCE`, `CONFIG` and `STATE`. For any type of resource, possible actions are - `READ` or `WRITE`
- Specific ResourceFilters are used to perform auth checks for all endpoints that corresponds to a specific resource type. This prevents duplication of logic and need to inject HttpServletRequest inside each endpoint. For example
- `DatasourceResourceFilter` is used for endpoints where the datasource information is present after "datasources" segment in the request Path such as `/druid/coordinator/v1/datasources/`, `/druid/coordinator/v1/metadata/datasources/`, `/druid/v2/datasources/`
- `RulesResourceFilter` is used where the datasource information is present after "rules" segment in the request Path such as `/druid/coordinator/v1/rules/`
- `TaskResourceFilter` is used for endpoints is used where the datasource information is present after "task" segment in the request Path such as `druid/indexer/v1/task`
- `ConfigResourceFilter` is used for endpoints like `/druid/coordinator/v1/config`, `/druid/indexer/v1/worker`, `/druid/worker/v1` etc
- `StateResourceFilter` is used for endpoints like `/druid/broker/v1/loadstatus`, `/druid/coordinator/v1/leader`, `/druid/coordinator/v1/loadqueue`, `/druid/coordinator/v1/rules` etc
- For endpoints where a list of resources is returned like `/druid/coordinator/v1/datasources`, `/druid/indexer/v1/completeTasks` etc. the list is filtered to return only the resources to which the requested user has access. In these cases, `HttpServletRequest` instance needs to be injected in the endpoint method.
Note -
JAX-RS specification provides an interface called `SecurityContext`. However, we did not use this but provided our own interface `AuthorizationInfo` mainly because it provides more flexibility. For example, `SecurityContext` has a method called `isUserInRole(String role)` which would be used for auth checks and if used then the mapping of what roles can access what resource needs to be modeled inside Druid either using some convention or some other means which is not very flexible as Druid has dynamic resources like datasources. Fixes#2355 with PR #2424
* Add back FilteredServerView removed in a32906c7fd to reduce memory usage using watched tiers.
* Add functionality to specify "druid.broker.segment.watchedDataSources"
segment creation deterministic.
This means that each segment will contain data from just one Kafka
partition. So, users will probably not want to have a super high number
of Kafka partitions...
Fixes#2703.
Avoids the following message from being printed on Overlord startup:
WARNING: Parameter 1 of type io.druid.indexing.common.actions.TaskActionHolder<T> from
public <T> javax.ws.rs.core.Response io.druid.indexing.overlord.http.OverlordResource.doAction
(io.druid.indexing.common.actions.TaskActionHolder<T>) is not resolvable to a concrete type
This PR changes the retry of task actions to be a bit more aggressive
by reducing the maxWait. Current defaults were 1 min to 10 mins, which
lead to a very delayed recovery in case there are any transient network
issues between the overlord and the peons.
doc changes.
- It's okay to suppress InterruptedException during graceful shutdown, as
tasks may use it to accelerate their own shutdown.
- It's okay to ignore return statuses during graceful shutdown (which may
be FAILED!) because it actually doesn't matter what they are.
Geared towards supporting transactional inserts of new segments. This involves an
interface "DataSourceMetadata" that allows combining of partially specified metadata
(useful for partitioned ingestion).
DataSource metadata is stored in a new "dataSource" table.
Appenderators are a way of getting more control over the ingestion process
than a Plumber allows. The idea is that existing Plumbers could be implemented
using Appenderators, but you could also implement things that Plumbers can't do.
FiniteAppenderatorDrivers help simplify indexing a finite stream of data.
Also:
- Sink: Ability to consider itself "finished" vs "still writable".
- Sink: Ability to return the number of rows contained within the sink.
The incremental indexes handle that now so it's not necessary.
Also, add debug logging and more detailed exceptions to the incremental
indexes for the case where there are parse exceptions during aggregation.
To bring consistency to docs and source this commit changes the default
values for maxRowsInMemory and rowFlushBoundary to 75000 after
discussion in PR https://github.com/druid-io/druid/pull/2457.
The previous default was 500000 and it's lower now on the grounds that
it's better for a default to be somewhat less efficient, and work,
than to reach for the stars and possibly result in
"OutOfMemoryError: java heap space" errors.
- Add TaskLocation class
- Add registerListener to TaskRunner
- Add getLocation to TaskRunnerWorkItem
- Implement location tracking in existing TaskRunners
- Rework WorkerTaskMonitor to do management out of a single thread so it can
handle status and location updates more simply.
* Moves last run task state information to Worker
* Makes WorkerTaskRunner a TaskRunner which has interfaces to help with getting information about a Worker
Two changes:
- Allow IncrementalIndex to suppress ParseExceptions on "aggregate".
- Add "reportParseExceptions" option to realtime tuning configs. By default this is "false".
Behavior of the counters should now be:
- processed: Number of rows indexed, including rows where some fields could be parsed and some could not.
- thrownAway: Number of rows thrown away due to rejection policy.
- unparseable: Number of rows thrown away due to being completely unparseable (no fields salvageable at all).
If "reportParseExceptions" is true then "unparseable" will always be zero (because a parse error would
cause an exception to be thrown). In addition, "processed" will only include fully parseable rows
(because even partial parse failures will cause exceptions to be thrown).
Fixes#2510.
* Don't put druid****selfcontained.jar at the end of the hadoop isolated classpath
* Add `<scope>provided</scope>` to prevent repeated dependency inclusion in the extension directories
- Throw most exceptions rather than suppressing them, which should help
detect problems. Continue suppressing exceptions that make sense to
suppress.
- Handle payload length checks consistently, and improve error message.
- Remove unused WorkerCuratorCoordinator.announceTaskAnnouncement method.
- Max znode length should be int, not long.
- Add tests.
* Defaults the thread priority to java.util.Thread.NORM_PRIORITY in io.druid.indexing.common.task.AbstractTask
* Each exec service has its own Task Factory which is assigned a priority for spawned task. Therefore each priority class has a unique exec service
* Added priority to tasks as taskPriority in the task context. <0 means low, 0 means take default, >0 means high. It is up to any particular implementation to determine how to handle these numbers
* Add options to ForkingTaskRunner
* Add "-XX:+UseThreadPriorities" default option
* Add "-XX:ThreadPriorityPolicy=42" default option
* AbstractTask - Removed unneded @JsonIgnore on priority
* Added priority to RealtimePlumber executors. All sub-executors (non query runners) get Thread.MIN_PRIORITY
* Add persistThreadPriority and mergeThreadPriority to realtime tuning config
Historical will drop a segment that shouldn't be dropped in the following scenario:
Historical node tried to load segmentA, but failed with SegmentLoadingException,
then ZkCoordinator called removeSegment(segmentA, blah) to schedule a runnable that would drop segmentA by deleting its files. Now, before that runnable executed, another LOAD request was sent to this historical, this time historical actually succeeded on loading segmentA and announced it. But later on, the scheduled drop-of-segment runnable started executing and removed the segment files, while historical is still announcing segmentA.
- Shedding locks at startup is bad, we actually want to keep them. Stop doing that.
- stopGracefully now interrupts the run thread if had started running finishJob. This avoids
waiting for handoff unnecessarily.
- fixes#1970
- extracted out segment handoff callbacks in SegmentHandoffNotifier
which is responsible for tracking segment handoffs and doing callbacks
when handoff is complete.
- Coordinator now maintains a view of segments in the cluster, this
will affect the jam heap requirements for the overlord for large
clusters.
realtime index task and nodes now use HTTP end points exposed by the
coordinator to get serverView
review comment
fix realtime node guide injection
review comments
make test not rely on scheduled exec
fix compilation
fix import
review comment
introduce immutableSegmentLoadInfo
fix son reading
remove unnecessary logging
This is done by killing and respawning the jvms rather than reconnecting to existing
jvms, for a couple reasons. One is that it lets you restore tasks after server reboots
too, and another is that it lets you upgrade all the software on a box at once by just
restarting everything.
The main changes are,
1) Add "canRestore" and "stopGracefully" methods to Tasks that say if a task can
stop gracefully, and actually do a graceful stop. RealtimeIndexTask is the only
one that currently implements this.
2) Add "stop" method to TaskRunners that attempts to do an orderly shutdown.
ThreadPoolTaskRunner- call stopGracefully on restorable tasks, wait for exit
ForkingTaskRunner- close output stream to restorable tasks, wait for exit
RemoteTaskRunner- do nothing special, we actually don't want to shutdown
3) Add "restore" method to TaskRunners that attempts to bootstrap tasks from last run.
Only ForkingTaskRunner does anything here. It maintains a "restore.json" file with
a list of restorable tasks.
4) Have the CliPeon's ExecutorLifecycle lock the task base directory to avoid a restored
task and a zombie old task from stomping on each other.
Deserialization of Optionals does not work quite right- they come back as actual
nulls, rather than absent Optionals. So these probably only ever worked for the local
task action client.
This is a feature meant to allow realtime tasks to work without being told upfront
what shardSpec they should use (so we can potentially publish a variable number
of segments per interval).
The idea is that there is a "pendingSegments" table in the metadata store that
tracks allocated segments. Each one has a segment id (the same segment id we know
and love) and is also part of a sequence.
The sequences are an idea from @cheddar that offers a way of doing replication.
If there are N tasks reading exactly the same data with exactly the same logic
(think Kafka tasks reading a fixed range of offsets) then you can place them
in the same sequence, and they will generate the same sequence of segments.
In #933 the ForkingTaskRunner's logging was changed to buffered from
unbuffered. This means that the last few KB of the logs are generally
not visible while a task is running, which makes debugging running
tasks difficult.
This is accomplished by making sure that scheduleTasksCleanupForWorker is only called
from the PathChildrenCache event thread, having it cancel existing cleanup tasks when
it adds a new one, and having tasks check on finish that the thing they are removing
from the task list is actually themselves.
1) Remove maven client from downloading extensions at runtime.
2) Provide a way to load Druid extensions and hadoop dependencies through file system.
3) Refactor pull-deps so that it can download extensions into extension directories.
4) Add documents on how to use this new extension loading mechanism.
5) Change the way how Druid tarball is generated. Now all the extensions + hadoop-client 2.3.0
are packaged within the Druid tarball.
* Still places `druid.indexer.runner.javaOpts` on the command line, but the Peon no longer tries to have the property `druid.indexer.runner.javaOpts` set
* Fixes https://github.com/druid-io/druid/issues/1841
Fixes#1727.
revert to doing merging for results for union queries on broker.
revert unrelated changes
Add test for union query runner
Add test
remove unused imports
fix imports
fix renamed file
fix test
update docs.