Commit Graph

1319 Commits

Author SHA1 Message Date
Charles Allen 15ccf451f9 Move QueryGranularity static fields to QueryGranularities (#2980)
* Move QueryGranularity static fields to QueryGranularityUtil
* Fixes #2979

* Add test showing #2979

* change name to QueryGranularities
2016-05-17 16:23:48 -07:00
Charles Allen eaaad01de7 [QTL] Datasource as lookupTier (#2955)
* 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
2016-05-17 15:44:42 -07:00
David Lim b489f63698 Supervisor for KafkaIndexTask (#2656)
* supervisor for kafka indexing tasks

* cr changes
2016-05-04 23:13:13 -07:00
Gian Merlino f8ddfb9a4b Split SegmentInsertAction and SegmentTransactionalInsertAction for backwards compat. (#2922)
Fixes #2912.
2016-05-04 13:54:34 -07:00
Himanshu 50065c8288 fix spurious failure of RTR concurrency test (#2915) 2016-05-04 10:30:20 -07:00
Charles Allen 3f71a4a302 Fix missing log arguments in PendingTaskBasedWorkerResourceManagementStrategy (#2898) 2016-04-28 18:15:41 -07:00
Parag Jain 0d745ee120 Basic authorization support in Druid (#2424)
- 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
2016-04-28 16:50:28 -07:00
Himanshu 9669e79df2 fix misleading error log due to race in RTR and concurrency test (#2878) 2016-04-28 10:28:00 -07:00
Nishant c29cb7d711 add pending task based resource management strategy (#2086) 2016-04-27 10:40:53 -07:00
Nishant bf5e5e7b75 fix #2886 (#2887)
Fixes https://github.com/druid-io/druid/issues/2886
2016-04-27 08:29:41 -07:00
David Lim 7641f2628f add control and status endpoints to KafkaIndexTask (#2730) 2016-04-21 15:34:59 -07:00
Nishant dbf63f738f Add ability to filter segments for specific dataSources on broker without creating tiers (#2848)
* Add back FilteredServerView removed in a32906c7fd to reduce memory usage using watched tiers.

* Add functionality to specify "druid.broker.segment.watchedDataSources"
2016-04-19 10:10:06 -07:00
Gian Merlino 08c784fbf6 KafkaIndexTask: Use a separate sequence per Kafka partition in order to make (#2844)
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.
2016-04-18 22:29:52 -07:00
jon-wei 0e481d6f93 Allow filters to use extraction functions 2016-04-05 13:24:56 -07:00
Fangjin Yang 1e02eeab13 Merge pull request #2683 from metamx/default_retry
Better defaults for Retry policy for task actions
2016-03-29 08:02:59 -07:00
Gian Merlino 195c9c5240 Overlord: Avoid a scary Jersey warning.
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
2016-03-28 19:08:56 -07:00
Fangjin Yang c2284929dc Merge pull request #2739 from gianm/fix-wtmtest-failure
Fix handling of InterruptedException in WorkerTaskMonitor's mainLoop.
2016-03-28 14:52:10 -07:00
Gian Merlino ee4bb96855 Fix handling of InterruptedException in WorkerTaskMonitor's mainLoop.
I believe this will fix #2664.
2016-03-25 12:17:33 -07:00
Himanshu Gupta 004b00bb96 config to explicitly specify classpath for hadoop container during hadoop ingestion 2016-03-25 10:51:28 -05:00
Himanshu 00d7021291 Merge pull request #2607 from jon-wei/dim_schema
Support use of DimensionSchema class in DimensionsSpec
2016-03-22 11:53:46 -05:00
Himanshu 3220b109ad Merge pull request #2570 from binlijin/single_dimension_partitioning
Single dimension hash-based partitioning
2016-03-22 11:51:06 -05:00
binlijin bce600f5d5 Single dimension hash-based partitioning 2016-03-22 13:15:33 +08:00
jon-wei a59c9ee1b1 Support use of DimensionSchema class in DimensionsSpec 2016-03-21 13:12:04 -07:00
Nishant ed8f39fcfe Better defaults for Retry policy for task actions
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.
2016-03-18 11:59:55 -07:00
Charles Allen c716af5b04 Merge pull request #2678 from metamx/fixImports
Fix some google related imports
2016-03-17 11:53:16 -07:00
Charles Allen a52c6d3bee Fix some google related imports 2016-03-17 11:03:29 -07:00
Gian Merlino 738dcd8cd9 Update version to 0.9.1-SNAPSHOT.
Fixes #2462
2016-03-17 10:34:20 -07:00
Nishant 9cceff2274 Use ImmutableWorkerInfo instead of ZKWorker
review comments

add test for equals and hashcode
2016-03-14 11:17:15 -07:00
Himanshu d51a0a0cf4 Merge pull request #2220 from gianm/appenderator-kafka
Appenderators, DataSource metadata, KafkaIndexTask
2016-03-14 13:14:36 -05:00
Nishant cf7f6da392 Merge pull request #2634 from gianm/stopGracefully-avoid-interrupt
ThreadPoolTaskRunner: Make graceful shutdown logs less scary.
2016-03-11 16:36:10 -08:00
Charles Allen a3f0048ea4 Merge pull request #2631 from gianm/plumbers-rpe
Better logging for ParseExceptions on index aggregation, and remove unnecessary exception handling.
2016-03-11 14:22:58 -08:00
Gian Merlino 79a95f7789 WorkerTaskMonitor: stop() waits for mainLoop to exit.
Fixes #2637.
2016-03-11 11:40:13 -08:00
Gian Merlino 05397a9b4f ThreadPoolTaskRunner: Make graceful shutdown logs less scary.
- 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.
2016-03-11 07:49:29 -08:00
Gian Merlino 187569e702 DataSource metadata.
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.
2016-03-10 17:41:50 -08:00
Gian Merlino 3d2214377d Appenderatoring.
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.
2016-03-10 17:41:50 -08:00
Gian Merlino 08284fea62 Publish test-jar for indexing-service. 2016-03-10 16:50:37 -08:00
Gian Merlino 92c828f904 Make SegmentHandoffNotifier Closeable. 2016-03-10 16:50:37 -08:00
Gian Merlino 8a11161b20 Plumbers: Move plumber.add out of try/catch for ParseException.
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.
2016-03-10 16:39:26 -08:00
Charles Allen d299540efc Make HadoopTask load hadoop dependency classes LAST for local isolated classrunner 2016-03-10 10:18:23 -08:00
Himanshu Gupta 0402636598 configurable handoffConditionTimeout in realtime tasks for segment handoff wait 2016-03-05 10:14:54 -06:00
Gian Merlino e9c23bf376 OverlordResource: Use getZkWorkers on RemoteTaskRunner.
Restores old behavior of this api, from before #2249 when getWorkers returned ZkWorkers.
2016-03-02 17:31:34 -08:00
Fangjin Yang 80d954578d Merge pull request #2572 from gianm/fix-rit-taskresource
Fix default TaskResource for RealtimeIndexTasks.
2016-03-02 10:20:27 -08:00
Gian Merlino acd95d3e28 TaskLocation: Add toString method.
Necessary because these objects are used in log messages.
2016-03-01 17:52:06 -08:00
Gian Merlino a355bfb7a9 Fix default TaskResource for RealtimeIndexTasks.
It was supposed to be the same as the task id, but it wasn't because
"makeTaskId" has a random component.
2016-03-01 16:54:22 -08:00
Björn Zettergren 2462c82c0e New defaults for maxRowsInMemory rowFlushBoundary
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.
2016-03-01 13:50:28 +01:00
Charles Allen c6803c4364 Allow specifying peon javaOpts as an array 2016-02-26 13:24:35 -08:00
Himanshu Gupta bc156effe7 RTR has multiple threads for assignment of pending tasks now. 2016-02-26 09:27:03 -06:00
Fangjin Yang 53a5f07c14 Merge pull request #2544 from metamx/fixMaxPort
Limit PortFinder to 0xFFFF
2016-02-25 17:12:53 -08:00
Fangjin Yang 143e85eaa5 Merge pull request #2419 from gianm/task-hostports
Plumb task peon host/ports back out to the overlord.
2016-02-25 17:11:53 -08:00
Charles Allen 3fa7a7ebfe Limit PortFinder to 0xFFFF 2016-02-25 08:16:40 -08:00
Charles Allen 187b788089 UnRegister port in ForkingTaskRunner 2016-02-25 08:04:25 -08:00
Gian Merlino cf0bc905fb Plumb task peon host/ports back out to the overlord.
- 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.
2016-02-24 15:13:10 -08:00
Nishant fb7eae34ed Merge pull request #2249 from metamx/workerExpanded
Use Worker instead of ZkWorker whenever possible
2016-02-24 13:23:22 +05:30
Charles Allen ac13a5942a Use Worker instead of ZkWorker whenver possible
* Moves last run task state information to Worker
* Makes WorkerTaskRunner a TaskRunner which has interfaces to help with getting information about a Worker
2016-02-23 15:02:03 -08:00
Gian Merlino 3534483433 Better handling of ParseExceptions.
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.
2016-02-23 10:11:43 -08:00
Bingkun Guo 499288ff4b Merge pull request #2509 from metamx/hadoopIsolatorTest
Add hadoop classloader isolation tests for HadoopTask
2016-02-19 14:23:22 -06:00
Fangjin Yang a3c29b91cc Merge pull request #2505 from gianm/rt-exceptions
Harmonize realtime indexing loop across the task and standalone nodes.
2016-02-19 11:23:14 -08:00
Charles Allen 9dff0e5dbd Add hadoop classloader isolation tests for HadoopTask 2016-02-19 11:15:53 -08:00
Fangjin Yang ddf913d626 Merge pull request #2508 from gianm/ftr-shutdown-logging
ForkingTaskRunner: Better logging during orderly shutdown.
2016-02-19 10:02:24 -08:00
Gian Merlino c0c6cf77fa ForkingTaskRunner: Better logging during orderly shutdown. 2016-02-19 09:17:16 -08:00
Gian Merlino 243ac5399b Harmonize realtime indexing loop across the task and standalone nodes.
- Both now catch ParseExceptions on plumber.add (see https://groups.google.com/d/topic/druid-user/wmiRDvx2RvM/discussion)
- Standalone now treats IndexSizeExceededException as fatal (previously only the task did)
2016-02-19 07:34:15 -08:00
Charles Allen 87752be740 Make HadoopTasks's classloader a single one 2016-02-18 20:58:09 -08:00
Andrés Gomez 07d714b1b5 Fixed equal distribution strategy when exist disable middleManager with same currCapacityUsed. 2016-02-17 08:40:42 +01:00
Himanshu 5779b32742 Merge pull request #2439 from metamx/fix2435
Make QuotableWhiteSpaceSplitter able to take JSON
2016-02-11 13:14:43 -06:00
Charles Allen 40ade32a1f Fix dependencies.
* 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
2016-02-11 07:30:14 -08:00
Charles Allen 3a6452c6d4 Make QuotableWhiteSpaceSplitter able to take json
* Fixes #2435
2016-02-10 16:42:14 -08:00
Xavier Léauté 91f23583f5 Merge pull request #2436 from gianm/mm-less-suppressey
Harmonize znode writing code in RTR and Worker.
2016-02-10 16:11:30 -08:00
Gian Merlino fa92b77f5a Harmonize znode writing code in RTR and Worker.
- 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.
2016-02-10 14:52:00 -08:00
Charles Allen 2bde8b1d68 Make hadoop classpath isolation more explicit
* Fixes #2428
2016-02-10 12:09:17 -08:00
Charles Allen a0728fa854 Allow ScalingStats to be null
* Fixes #2378
2016-02-02 18:01:01 -08:00
Parag Jain 7853a9cc41 clean up TaskLifecycleTest 2016-01-31 11:19:20 -06:00
Gian Merlino 5fd4b79373 RealtimeIndexTask: Fix NPE caused by calling stopGracefully before a firehose had been connected. 2016-01-29 11:20:23 -08:00
Gian Merlino c4fde52160 Fix 'graceful shutdown aborted' log message in ThreadPoolTaskRunner. 2016-01-29 11:07:17 -08:00
Nishant dcb7830330 Merge pull request #984 from drcrallen/thread-priority-rebase
Use thread priorities. (aka set `nice` values for background-like tasks)
2016-01-21 15:02:34 +05:30
Charles Allen 66e74b1a63 Minor field name change in RemoteTaskRunnerFactory to be more descriptive
* Addresses https://github.com/druid-io/druid/pull/2309#discussion_r50335081
2016-01-20 17:43:20 -08:00
Charles Allen 3152d08844 Fix overlord scheduled executor injection
* Fixes https://github.com/druid-io/druid/issues/2308
2016-01-20 14:16:14 -08:00
Charles Allen 2e1d6aaf3d Use thread priorities. (aka set `nice` values for background-like tasks)
* 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
2016-01-20 14:00:31 -08:00
Nishant ac6c90e657 Merge pull request #1953 from metamx/taskRunnerResourceManagement
Move resource managemnt to be the responsibility of the TaskRunner
2016-01-20 22:27:47 +05:30
Jonathan Wei df2906a91c Merge pull request #2290 from gianm/index-merger-v9-stuff
Respect buildV9Directly in PlumberSchools, so it works on standalone realtime.
2016-01-19 13:04:00 -08:00
Fangjin Yang 0c31f007fc Merge pull request #1728 from himanshug/aggregators_in_segment_metadata
Store AggregatorFactory[] in segment metadata
2016-01-19 12:55:49 -08:00
Himanshu fe841fd961 Merge pull request #2118 from guobingkun/fix_segment_loading
Fix loading segment for historical
2016-01-19 14:25:48 -06:00
Himanshu Gupta a99aef29a1 adding aggregators to segment metadata 2016-01-19 14:23:39 -06:00
Gian Merlino 1dcf22edb7 Respect buildV9Directly in PlumberSchools, so it works on standalone realtime nodes.
Also parameterize some tests to run with/without buildV9Directly:

- IndexGeneratorJobTest
- RealtimeIndexTaskTest
- RealtimePlumberSchoolTest
2016-01-19 12:15:06 -08:00
Bingkun Guo c4ad50f92c Fix loading segment for historical
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.
2016-01-19 10:29:49 -06:00
Himanshu Gupta 164b0aad7a removing Map<String,Object> segmentMetadata from methods in Index[Maker/Merger] and using Metadata class
instead of a Map to store segment metadata
2016-01-18 22:03:46 -06:00
Kurt Young 82ff98c2bf add config for build v9 directly and update docs 2016-01-16 11:26:34 +08:00
Charles Allen 976d4c965b Move resource managemnt to be the responsibility of the TaskRunner 2016-01-13 10:38:22 -08:00
Himanshu 82bdfbbbf1 Merge pull request #2155 from metamx/taskConfigTmpdir
Make TaskConfig pull from java.io.tmpdir
2016-01-05 13:58:39 -06:00
Nishant 45f402f22f increase timeout
tune timeouts
2016-01-05 19:06:04 +05:30
Charles Allen e18301d99c Make TaskConfig pull from java.io.tmpdir
* Also makes paths built off of java.nio.file.Paths instead of String.format
2016-01-04 10:17:08 -08:00
fjy b5c094d951 Fixes #2180 2016-01-01 16:56:41 -08:00
Nishant b68265399c Merge pull request #2168 from druid-io/remove-indexmaker
Remove IndexMaker
2015-12-30 12:24:29 +05:30
Nishant df893dbaf8 Merge pull request #2141 from gianm/fix-restoring-realtime
Fix some problems with restoring
2015-12-30 10:44:45 +05:30
Fangjin Yang 7ffa706655 Merge pull request #2152 from metamx/add-taskId
Add taskId to realtimeMetrics
2015-12-29 10:33:40 -08:00
fjy 38b0f1fbc2 fix transient failures in unit tests 2015-12-28 20:03:30 -08:00
fjy faf421726b remove IndexMaker 2015-12-28 14:19:02 -08:00
Fangjin Yang 8cb52bddd8 Merge pull request #2140 from navis/fix-sporadic-testfail4
Fix sporadic fail of RemoteTaskRunnerTest#testWorkerRemoved
2015-12-27 14:55:49 -08:00
Fangjin Yang 9aa62e4631 Merge pull request #2154 from navis/fix-testfail-WorkerTaskMonitorTest
Fix sporadic fail of WorkerTaskMonitorTest#testRunTask
2015-12-23 20:52:33 -08:00
navis.ryu a8f6c6110d Fix sporadic fail of WorkerTaskMonitorTest#testRunTask 2015-12-24 02:31:30 +09:00
navis.ryu 2c3c4a3f8f Another try to fix xxServerViewTests 2015-12-24 02:13:40 +09:00
Nishant 978a3fd8ae Add taskId to realtimeMetrics
Add task Id to Realtime Metrics
2015-12-23 18:05:25 +05:30
Gian Merlino 32edd1538d RealtimeIndexTask: Fix a couple of problems with restoring.
- 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.
2015-12-22 16:04:47 -08:00
Gian Merlino f4ce2b9bc5 TaskLockbox: Consider active tasks active even if they have no locks. 2015-12-22 16:04:16 -08:00
Gian Merlino bad270b6c4 druid.indexer.task.restoreTasksOnRestart configuration. 2015-12-22 10:59:15 -08:00
navis.ryu 8a179fc273 Fix sporadic fail of RemoteTaskRunnerTest#testWorkerRemoved 2015-12-22 14:33:37 +09:00
Himanshu Gupta 5e178499e8 trying to fix transient errors in testRealtimeIndexTask() by increasing overall timeout and unlimited wait for segment publish 2015-12-21 00:11:20 -06:00
Fangjin Yang 14229ba0f2 Merge pull request #1922 from metamx/jsonIgnoresFinalFields
Change DefaultObjectMapper to NOT overwrite final fields unless explicitly asked to
2015-12-18 15:38:32 -08:00
Bingkun Guo 1e5aa2f3ac fix getType() and Json serialization in ClientMergeQuery and add serde tests 2015-12-15 12:08:43 -06:00
Nishant a32906c7fd Remove FilteredServerView 2015-12-09 01:54:12 +05:30
Nishant 9491e8de3b Remove ServerView from RealtimeIndexTasks and use coordinator http endpoint for handoffs
- 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
2015-12-09 01:54:09 +05:30
Himanshu Gupta 62ba9ade37 unifying license header in all java files 2015-12-05 22:16:23 -06:00
Gian Merlino 20544d409b Merge pull request #1988 from himanshug/multi-interval-batch-delta
support multiple intervals in dataSource inputSpec
2015-12-04 09:07:52 -08:00
Gian Merlino 020a5e7081 Merge pull request #2024 from metamx/fairBigTaskQueueLock
Make the TaskQueue big lock fair
2015-12-03 19:32:53 -08:00
Himanshu Gupta 61aaa09012 support multiple intervals in dataSource input spec 2015-12-03 21:28:04 -06:00
Himanshu Gupta 86f0a36e83 support multiple intervals in SegmentListUsedAction 2015-12-03 21:28:04 -06:00
Himanshu Gupta 221fb95d07 add support for getting used segments for multiple interval in IndexerMetadataStorageCoordinator 2015-12-03 21:28:04 -06:00
Charles Allen dbaaa6af92 Make the TaskQueue big lock fair 2015-12-01 19:13:07 -08:00
Nishant 1eb8211346 Add datasource and taskId to metrics emitted by peons
This PR adds the datasource and taskId to the jvm and sys metrics
emitted by the peons.

fix spelling

review comment

review comment
2015-12-01 23:20:59 +05:30
Fangjin Yang 8e83d800d6 Merge pull request #1881 from gianm/restartable-tasks
Restorable indexing tasks
2015-11-23 21:14:37 -08:00
Gian Merlino 501dcb43fa Some changes that make it possible to restart tasks on the same hardware.
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.
2015-11-23 11:22:08 -08:00
Gian Merlino 666d785787 Switch TaskActions from Optionals to nullable.
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.
2015-11-20 09:14:07 -08:00
Fangjin Yang 21c84b5ff7 Merge pull request #1896 from gianm/allocate-segment
SegmentAllocateAction (fixes #1515)
2015-11-18 21:05:46 -08:00
Fangjin Yang e52c156066 Merge pull request #1880 from gianm/rtr-adjust
RTR: Ensure that there is only one cleanup task scheduled for a worker at once.
2015-11-18 15:12:55 -08:00
Charles Allen 8fcf2403e3 Merge pull request #1943 from metamx/realtime-caching
Enable caching on intermediate realtime persists
2015-11-17 15:06:43 -08:00
Charles Allen dbe201aeed Merge pull request #1929 from pjain1/jetty_threads
separate ingestion and query thread pool
2015-11-17 12:14:25 -08:00
Parag Jain 6c498b7d4a separate ingestion and query thread pool 2015-11-17 13:42:41 -06:00
Xavier Léauté d7eb2f717e enable query caching on intermediate realtime persists 2015-11-17 10:58:00 -08:00
Charles Allen 46527a9610 Merge pull request #1954 from metamx/fix-stupid-aws-limit
EC2 autoscaler: avoid hitting aws filter limits
2015-11-13 10:52:35 -08:00
Fangjin Yang 4f46d457f1 Merge pull request #1947 from noddi/feature/count-parameter-history-endpoints
Add count parameter to history endpoints
2015-11-12 10:23:44 -08:00
Xavier Léauté 749ac12f88 EC2 autoscaler: avoid hitting aws filter limits 2015-11-11 20:28:06 -08:00
Fangjin Yang 465cbcf9a7 Merge pull request #1956 from metamx/remove-unused-imports
Cleanup + remove unused imports
2015-11-11 17:36:47 -08:00
Gian Merlino e4e5f0375b SegmentAllocateAction (fixes #1515)
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.
2015-11-11 16:54:35 -08:00
Bartosz Ługowski 6e5d2c6745 Add count parameter to history endpoints. 2015-11-11 23:03:57 +01:00
Xavier Léauté fa6142e217 cleanup and remove unused imports 2015-11-11 12:25:21 -08:00
zhxiaog c197a4cf32 fix #1918, add unit tests for RemoteTaskActionClient 2015-11-12 03:15:17 +08:00
Charles Allen abae47850a Add backwards compatability for PR #1922 2015-11-11 10:27:00 -08:00
Charles Allen 1df4baf489 Move Jackson Guice adapters into io.druid
* Removes access to protected methods in com.fasterxml
* Eliminates druid-common's use of foreign package com.fasterxml
2015-11-09 10:50:45 -08:00
Gian Merlino fc55314d1c ForkingTaskRunner: Log without buffering.
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.
2015-11-07 15:16:53 -08:00
Charles Allen 929b981710 Change DefaultObjectMapper to NOT overwrite final fields unless explicitly asked to 2015-11-05 18:10:13 -08:00
Gian Merlino cb409ee928 RemoteTaskActionClient: Fix statusCode check. 2015-11-05 10:03:49 -08:00
fjy 8f231fd3e3 cleanup druid codebase 2015-11-04 13:59:53 -08:00
Himanshu Gupta 84f7d8d264 making static final variables in HadoopDruidIndexerConfig upper case 2015-11-02 23:24:26 -06:00
Himanshu Gupta 8b67417ac8 make methods in Index[Merger,Maker,IO] non-static so that they can have
appropriate ObjectMapper injected instead of creating one statically
2015-11-02 23:24:26 -06:00
Gian Merlino 16ae8866b8 Log and continue on failure to schedule cleanup for missing workers at startup. 2015-10-28 08:10:54 -07:00
Gian Merlino 513bc76252 RTR: Ensure that there is only one cleanup task scheduled for a worker at once.
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.
2015-10-27 21:16:58 -07:00
Fangjin Yang ea2267e08c Merge pull request #1868 from gianm/fix-announcements
Historical and MiddleManager server announcements should not remove parents.
2015-10-27 14:50:05 -07:00
Gian Merlino 7df7370935 Merge pull request #1862 from metamx/indexingServiceMMGone
Add timeout to shutdown request to middle manager for indexing service
2015-10-27 14:38:01 -07:00
Charles Allen 44a2b204df Add timeout to shutdown request to middle manager for indexing service 2015-10-27 13:56:03 -07:00
Gian Merlino 4b92752deb Historical and MiddleManager server announcements should not remove parents.
Removing parent paths causes watchers of the "announcements" path to get stuck
and stop seeing new updates.
2015-10-27 08:06:11 -07:00
Bingkun Guo 4914925d65 New extension loading mechanism
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.
2015-10-21 14:22:36 -05:00