Commit Graph

2610 Commits

Author SHA1 Message Date
Fangjin Yang 9340cae985 Merge pull request #2457 from bjozet/docs/fixes
Default value for maxRowsInMemory
2016-03-01 07:43:26 -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
Bingkun Guo 4edcb1b861 Refactor FireChief + UTs for RealtimeManagerTest
Add tests that verify whether RealtimeManager is querying the correct FireChief for a specific partition
make FireChief static and package private, add latches in the UT
2016-02-29 14:41:10 -06:00
Eric Tschetter 68631d89e9 Allow realtime nodes to have multiple shards of the same datasource 2016-02-29 12:30:25 -06:00
Fangjin Yang 29d29ba98d Merge pull request #2263 from jon-wei/flex_dims3
Allow IncrementalIndex to store Long/Float dimensions
2016-02-25 17:23:02 -08:00
Gian Merlino b331fb4a83 Fix parsing of druid.indexer.server.maxChatRequests. 2016-02-25 14:47:15 -08:00
jon-wei c17ce02467 Allow IncrementalIndex to store Long/Float dimensions 2016-02-24 13:51:57 -08:00
Himanshu Gupta a3b37e9225 In persistAndMerge, increase the scope of try-catch block so that any exception while persisting hydrants is caught and consequently that sink is abandoned or the task will forever wait for handoff to happen. 2016-02-23 22:22:33 -06:00
Nishant 6c9e1a28ad Merge pull request #2519 from gianm/unparseable-handling
Better handling of ParseExceptions.
2016-02-24 04:46:29 +05:30
Fangjin Yang 93540c0631 Merge pull request #2503 from gianm/jetty-qos
Add druid.indexer.server.maxChatRequests for QoS; deprecate separate ports.
2016-02-23 10:35:53 -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
Fangjin Yang 0c984f9e32 Merge pull request #2109 from himanshug/segments_in_delta_ingestion
idempotent batch delta ingestion
2016-02-22 14:00:45 -08:00
Fangjin Yang 3bdd757024 Merge pull request #1773 from b-slim/log_details
Adding downstream source when throwing QueryInterruptedException
2016-02-22 10:16:07 -08:00
Himanshu Gupta 21b0b8a07d new coordinator endpoint to get list of used segment given a dataSource and list of intervals 2016-02-21 23:17:58 -06:00
Slim Bouguerra 77925cc061 adding downstream source of QueryInterruptedException 2016-02-20 13:05:14 -06:00
Gian Merlino 23c993c9e7 Add druid.indexer.server.maxChatRequests for QoS; deprecate separate ports.
- Add druid.indexer.server.maxChatRequests, which sets up a QoSFilter on the main Jetty server.
- Deprecate druid.indexer.runner.separateIngestionEndpoint
- Deprecate druid.indexer.server.chathandler.*
2016-02-19 13:36:09 -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
Gian Merlino e0c049c0b0 Make startup properties logging optional.
Off by default, but enabled in the example config files. See also #2452.
2016-02-12 14:12:16 -08:00
Fangjin Yang 1430bc2c88 Merge pull request #2276 from harshjain2/feature-2021
Fix for issue 2021.
2016-02-10 17:04:45 -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
Harsh Jain a3eb863c8e Fix for issue 2021 2016-02-10 22:19:12 +05:30
Himanshu Gupta b40c342cd1 make Global stupid pool cache size configurable 2016-02-05 14:18:06 -06:00
Parag Jain 9002548eeb increase test time out and general clean up 2016-02-03 13:26:37 -06:00
Charles Allen 5111fd52f2 Add check for log4j-core in Log4jShutterDownerModule 2016-02-02 15:56:48 -08:00
Himanshu dc89cdd0f9 Merge pull request #2336 from himanshug/fix_2331
limit size of X-Druid-Response-Context header to 7K
2016-02-02 12:06:59 -06:00
navis.ryu c03918f89a AsyncQueryForwardingServletTest#testDeleteBroadcast sometimes fails by port conflict 2016-01-29 19:28:58 +09:00
Himanshu Gupta f6b4dbd697 bug fix and unit tests for DruidCoordinatorSegmentKiller 2016-01-28 14:10:17 -06:00
Himanshu Gupta ab3edfa8fc moving DruidCoordinatorSegmentKiller class out of DruidCoordinator 2016-01-28 14:03:56 -06:00
Nishant 3880f54b87 Merge pull request #2332 from himanshug/configurable_partial
make populateUncoveredIntervals a configuration in query context
2016-01-28 10:34:35 +05:30
Himanshu Gupta a7bde8f4da limit size of X-Druid-Response-Context header to 7K due to https://github.com/druid-io/druid/issues/2331 2016-01-27 15:18:08 -06:00
Xavier Léauté 5a3642bb93 Merge pull request #2247 from metamx/pedanticBuild
Enable strict building in travis
2016-01-27 10:27:03 -08:00
Charles Allen 508734c8b0 Long constant reformatting in tests `l` --> `L` 2016-01-27 08:59:19 -08:00
Nishant fd6bf3fe22 Use interval comparator instead of bucketMonthComparator
fix when two segments have same interval

review comments
2016-01-27 17:35:43 +05:30
Himanshu Gupta 3719b6e3c8 make populateUncoveredIntervals a configuration in query context 2016-01-26 15:13:45 -06:00
Harsh Jain 41730b96d4 Fix for issue 2021 2016-01-25 02:48:22 +05:30
Himanshu 7a6109f0ca Merge pull request #2321 from gianm/pending-index
Replace two-column index on pendingSegments table with one-column index.
2016-01-22 13:38:15 -06:00
Gian Merlino 0bd9bff075 Replace two-column index on pendingSegments table with one-column index.
Fixes #2319.
2016-01-22 10:50:21 -08:00
Himank Chaudhary 1a5d4e714c Adding custom mapper for json processing exception to return bad request instead of 500 2016-01-22 09:48:52 -08:00
Fangjin Yang 04d3054353 Merge pull request #2303 from CHOIJAEHONG1/localfirehouse-basedir-npe
Throw an IAE when baseDir is null in LocalFireHose
2016-01-21 07:58:52 -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 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 61aca6f9cc remove wrong checks
sink never have null hydrants and hydrants never have null adapters
2016-01-20 23:43:53 +05:30
Jaehong Choi 7132428bba throw IAE when baseDir is null in LocalFireHose 2016-01-21 01:27:32 +09:00
Nishant 59ea186af7 fix reference counting for segments 2016-01-20 17:24:21 +05:30
jon-wei 747343e621 Preserve dimension order across indexes during ingestion 2016-01-19 13:34:11 -08:00
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