Commit Graph

3212 Commits

Author SHA1 Message Date
Jonathan Wei df51a7bcb7
Fix HTTP OPTIONS request auth handling (#5615)
* Fix HTTP OPTIONS request auth handling

* Flip configuration boolean
2018-04-12 14:02:20 -07:00
Gian Merlino d0400a0688 SegmentWithState: Add toString method. (#5635)
The class appears in log messages, and the default toString method
isn't very informative.
2018-04-12 14:01:09 -05:00
palanieppan-m dbea5cb9b7 Load rules should honor partial overlap (#5595)
Load rules should load segments that partially overlap with rule window,
instead of loading only segments that fully overlap.
2018-04-12 09:46:00 -07:00
Atul Mohan 19f359957f Add getters for AlertEvent (#5522)
* Add getters for AlertEvent

* Move PublicApi and ExtensionPoint to java-util

* Fix publicapi annotation usage

* Add publicapi annotations to ServiceMetricEvent and RequestLogEvent
2018-04-12 23:38:20 +07:00
Nishant Bangarwa e6efd75a3d Add config to allow setting up custom unsecured paths for druid nodes. (#5614)
* 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
2018-04-11 17:10:07 -07:00
Clint Wylie ea4f8544fb revert lambda conversion to fix occasional jvm error (#5591) 2018-04-06 14:18:55 -07:00
Gian Merlino 5ab17668c0 CompressionUtils: Add support for decompressing xz, bz2, zip. (#5586)
Also switch various firehoses to the new method.

Fixes #5585.
2018-04-06 08:06:45 -07:00
Niketh Sabbineni 270fd1ea15 Allow getDomain to return disjointed intervals (#5570)
* Allow getDomain to return disjointed intervals

* Indentation issues
2018-04-05 22:12:30 -07:00
Jonathan Wei 969342cd28
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
2018-04-05 21:38:57 -07:00
Niketh Sabbineni f0a94f5035 Remove unused config (#5564)
* Remove unused config

* Fix failing tests
2018-04-03 13:23:46 -07:00
Clint Wylie f31dba6c5b Coordinator drop segment selection through cost balancer (#5529)
* 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
2018-04-03 11:22:51 -07:00
Clint Wylie a81ae99021 add 'stopped' check and handling to HttpLoadQueuePeon load and drop segment methods (#5555)
* add stopped check and handling to HttpLoadQueuePeon load and drop segment methods

* fix unrelated timeout :(

* revert unintended change

* PR feedback: change logging

* fix dumb
2018-04-03 11:21:52 -07:00
Clint Wylie 6feac204e3 Coordinator primary segment assignment fix (#5532)
* 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
2018-04-02 09:40:20 -07:00
Jihoon Son 05547e29b2
Fix SQLMetadataSegmentManager to allow succesive start and stop (#5554)
* Fix SQLMetadataSegmentManager to allow succesive start and stop

* address comment

* add synchronization
2018-03-30 12:43:19 -07:00
Clint Wylie 30fc4d3ba0 Coordinator balancer move then drop fix (#5528)
* #5521 part 1

* formatting

* oops

* less magic tests
2018-03-29 10:30:12 -07:00
Kirill Kozlov 8878a7ff94 Replace guava Charsets with native java StandardCharsets (#5545) 2018-03-28 21:00:08 -07:00
Atul Mohan ec17a44e09 Add result level caching to Brokers (#5028)
* Add result level caching to Brokers

* Minor doc changes

* Simplify sequences

*  Move etag execution

* Modify cacheLimit criteria

* Fix incorrect etag computation

* Fix docs

* Add separate query runner for result level caching

* Update docs

* Add post aggregated results to result level cache

* Fix indents

* Check byte size for exceeding cache limit

* Fix indents

* Fix indents

* Add flag for result caching

* Remove logs

* Make cache object generation synchronous

* Avoid saving intermediate cache results to list

* Fix changes that handle etag based response

* Release bytestream after use

*  Address PR comments

*  Discard resultcache stream after use

* Fix docs

* Address comments

* Add comment about fluent workflow issue
2018-03-23 19:11:52 -07:00
Charles Allen ef21ce5a64
Add graceful shutdown timeout for Jetty (#5429)
* 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
2018-03-23 09:38:17 -07:00
Jihoon Son 1ad898bde2
Use the official aws-sdk instead of jet3t (#5382)
* 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
2018-03-21 15:36:54 -07:00
Charles Allen 58f110f7f8 Future-proof some Guava usage (#5414)
* 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
2018-03-20 08:59:33 -07:00
Jonathan Wei b22455b924
Fix supervisor tombstone auth handling (#5504) 2018-03-19 12:55:47 -07:00
Roman Leventov 693e3575f9
Remove unused code and exception declarations (#5461)
* Remove unused code and exception declarations

* Address comments

* Remove redundant Exception declarations

* Make FirehoseFactoryV2.connect() to throw IOException again
2018-03-16 22:11:12 +01:00
Jonathan Wei 30e6bdedf3 Authorize supervisor history instead of current active supervisors for supervisor history API (#5501) 2018-03-16 12:29:17 -07:00
Gian Merlino a08efe4683
Fix round robining in router. (#5500)
* 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.
2018-03-15 18:45:59 -07:00
Gian Merlino fdd55538e1 SQL: Remove unused escalator, authConfig from various classes. (#5483)
DruidPlanner.plan is responsible for checking authorization, so these objects
weren't needed in as many places as they were injected.
2018-03-14 13:28:51 -07:00
Jihoon Son 9b2a25bd84
Refactor supervisorReport to be type-safe (#5479)
* refactor supervisorReport

* use primitives
2018-03-13 09:28:44 -07:00
Himanshu e968811583 HttpServerInventoryView: fixed startup wait time and more informative logging (#5336) 2018-03-12 22:13:51 -07:00
Roman Leventov 6b158abe3f Enforce optimal IndexedInts iteration (#5456)
* Enforce optimal IndexedInts iteration

* Fix remaining suboptimal usages
2018-03-09 09:42:40 -08:00
Alexander Korablev 6a3a5350b8 Make memcached protocol and locator configurable. (#5438)
* Make memcached protocol and locator configurable.

* Style fix.

* Style fix.

* Style fix.
2018-02-28 17:16:43 -08:00
Niketh Sabbineni ac5034e241 Improve cache cost to handle heterogenous historicals (#5416) 2018-02-23 13:17:31 -08:00
Jonathan Wei e9977ce4ef
Automatically adjust com.metamx.metrics Monitor class references (#5412)
* Automatically adjust com.metamx.metrics monitor class references

* Log warning for old class names
2018-02-22 12:03:07 -08:00
vvc11 305ecc2a78 adding a properties endpoint in status resource (#5276)
* 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
2018-02-18 12:51:02 -08:00
David Lim 20a3164180 Support for router forwarding requests to active coordinator/overlord (#5369)
* allow router to forward requests to coordinator and overlord

* fix forbidden API

* more forbidden api fixes

* code review changes
2018-02-15 14:38:58 -08:00
Jihoon Son cd929000ca
Change early publishing to early pushing in indexTask & refactor AppenderatorDriver (#5297)
* 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
2018-02-14 12:48:33 -08:00
Jihoon Son 0105cdbc19 Fix Json Serde (#5370) 2018-02-08 13:13:52 -08:00
Roman Leventov e64ffb10c2 Standartize on using Integer.BYTES instead of Ints.BYTES from Guava, same for other primitives (#5366) 2018-02-07 13:24:30 -08:00
Gian Merlino 971d45ab3f Use a separate snapshot file per lookup tier. (#5358)
Prevents conflicts if two processes on the same machine use the
same lookup snapshot directory but are in different tiers.
2018-02-07 11:28:53 -08:00
Jihoon Son 2099b43e5f Add a new config object for compactConfig (#5264)
* add a new config object for compactConfig

* fix test

* address comments

* Update doc
2018-02-06 12:13:52 -08:00
Gian Merlino c21ff6e81c
Properly set "identity" in query metrics. (#5330)
* 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.
2018-02-06 10:53:00 -08:00
Kevin Conaway 93fdbcb364 Change RealtimeIndexTask to use AppenderatorDriver (#5261)
* Change RealtimeIndexTask to use AppenderatorDriver instead of RealtimePlumber.  Related to #4774

* Remove unused throwableDuringPublishing

* Fix usage of forbidden API

* Update realtime index IT to account for not skipping older data any more

* Separate out waiting on publish futures and handoff futures to avoid a race condition where the handoff timeout expires before the segment is published

* #5261 Add separate AppenderatorDriverRealtimeIndexTask and revert changes to RealtimeIndexTask

* #5261 Add separate AppenderatorDriverRealtimeIndexTask and revert changes to RealtimeIndexTask

* #5261 Readability improvements in AppenderatorDriverRealtimeIndexTask.  Combine publish and handoff futures in to single future

* #5261 Add separate tuningConfig for RealtimeAppenderatorIndexTask.  Revert changes to RealtimeTuningConfig

* #5261 Change JSON type to realtime_appenderator to keep the same naming pattern as RealtimeIndexTask
2018-02-06 10:21:31 -08:00
Gian Merlino 8c738c7076 Fix races in LookupSnapshotTaker, CoordinatorPollingBasicAuthenticatorCacheManager (#5344)
* Fix races in LookupSnapshotTaker, CoordinatorPollingBasicAuthenticatorCacheManager.

Both were susceptible to the following conditions:

1. Two JVMs on the same machine (perhaps two peons) could conflict by one reading while the
   other was writing, or by writing to the file at the same time.
2. One JVM could partially write a file, then crash, leaving a truncated file.

* Use StringUtils.format
2018-02-06 09:44:06 -08:00
Slim 37c09ce3f8 Use both Joad Ids and Java IDs as Timezone to string readers (#5349)
* Use both Joad Ids and Java IDs as Timezone to string readers

Change-Id: Ieb5c18559879f3f3a0104912ce2f0a354ad0aac3

* move the function to DateTimes and add org.joda.time.DateTimeZone#forID as part of forbidden api

Change-Id: Iff97fa044758019ed0c231587d10e31a9cc18da0

* exclude class and remove other usage

Change-Id: Ib458c2caaa1865535767e1009fbf017a92c8f615

* remove it from test classes

Change-Id: I9b576324f6c7e17a74bd8b13879232c9a8cd40b4

* remove unused

Change-Id: If1c5b70c26c2b7c83c20434cb72b2060653f5052
2018-02-06 16:34:11 +05:30
Gian Merlino 9a62b02cb7 Extensions: Option to load classes from extension jars first. (#5321)
The behavior is configurable through druid.extensions.useExtensionClassloaderFirst.
It is useful when extensions want to load a dependency different from one provided
by Druid, for example a different version of geoip or protobuf.
2018-02-06 16:14:03 +05:30
Jonathan Wei c9e7c0a817 Remove Escalator jetty http client escalation method (#5322) 2018-02-02 12:43:02 -06:00
Gian Merlino 7e02408510 Update versions to 0.13.0-SNAPSHOT. (#5323) 2018-02-02 12:06:38 -06:00
Gian Merlino 10b8540f80 CliCoordinator: LoadQueueTaskMaster should use an escalated http client. (#5329)
Also remove Guice annotations from LoadQueueTaskMaster, since it is
provided by CliCoordinator, so Guice does not need to know how to
build one directly.
2018-02-02 10:44:32 -06:00
Himanshu 4cd47de62f add LookupExtractorFactory.destroy() method (#5287)
* add LookupExtractorFactory.destroy() method

* fix LookupReferencesManagerTest
2018-02-01 22:56:09 -08:00
Gian Merlino ed47a1e1a9
Lookups: Inherit "injective" from registered lookups, improve docs. (#5316)
Code changes:
- In the lookup-based extractionFns, inherit injective property from
  the lookup itself if not specified.

Doc changes:
- Add a "Query execution" section to the lookups doc explaining how
  injective lookups and their optimizations work.
- Remove scary warnings against using registeredLookup extractionFns.
  They are necessary and important since they work with filters and
  function cascades -- two things that the dimension specs do not do.
  They deserve to be first class citizens.
- Move the "registeredLookup" fn above the "lookup" fn. It's probably
  more commonly used, so the docs read better this way.
2018-02-01 18:30:19 -08:00
Jihoon Son 3a69b0e513 Handle nullable taskTypes for rolling upgrade (#5309) 2018-01-30 13:32:54 -08:00
David Lim be66d4b822 clean up intermediate_pushes directory for LocalDataSegmentPusher (#5306) 2018-01-30 12:33:06 -06:00
Jonathan Wei f6749f1229 Allow separate truststore conf for HttpEmitter (#5298)
* Fix HttpEmitter TLS support, allow separate truststore conf

* PR comment, fix tests
2018-01-26 10:46:06 -06:00
Jonathan Wei 80419752b5 Add metamx emitter, http clients, and metrics packages to druid java-util (#5289)
* Add metamx java-util emitter, http clients, and metrics packages to druid java-util

* Remove metamx java-util from pom.xml files

* Checkstyle fixes

* Import fix

* TeamCity inspection fixes

* Use slf4j, move some version defs to master pom.xml

* Use parent jvm-attach-api and maven-surefire-plugin versions

* Add ] to log msg, suppress inspection
2018-01-24 22:10:36 +01:00
Nishant Bangarwa aca200fddb
Fix rewrite of queryPath for encoded joda intervals as query param on druid router (#5274)
* Fix rewrite of queryPath for encoded joda intervals as query param on druid router

* fix checkstyle

* fix comment
2018-01-24 02:20:07 +05:30
Roman Leventov 61e6878afd Check Javadoc reference integrity (#5279) 2018-01-22 13:51:28 -08:00
Roman Leventov a346bbc6f3 Enforce spacing around foreach colon with Checkstyle (#5271) 2018-01-22 11:48:51 -08:00
Roman Leventov f99c27e9e0 Fix bugs in ImmutableRTree; Merge bytebuffer-collections module into druid-processing (#5275)
* Fix bugs in ImmutableRTree; optimize ImmmutableRTreeObjectStrategy.writeTo(); Merge bytebuffer-collections module into druid-processing

* Remove unused declaration

* Fix another bug
2018-01-23 00:49:59 +05:30
Roman Leventov 87c744ac1d Add MethodParamPad, OneStatementPerLine and EmptyStatement Checkstyle checks (#5272) 2018-01-18 11:29:23 -08:00
Akash Dwivedi d6932c1621 java-util version update + Add UnusedConnectionTimeout config. (#5239)
* java-util version update + Add UnusedConnectionTimeout config.

* warn  if unusedConnectionTime >= readTimeout.

* Doc update + addressed comment.

* Use compareTo  to compare duration.

* remove unused variable.

* addressed comments and default for unusedConnectionTimeout.
2018-01-17 15:54:18 -06:00
Parag Jain b6b12db8b4 do not include the index in toString (#5268) 2018-01-17 20:03:53 +01:00
Jihoon Son 241efafbb2
Automatic compaction by coordinators (#5102)
* Automatic compaction by coordinator

* add links

* skip compaction for very recent segments if they are small

* fix finding search interval

* fix finding search interval

* fix TimelineHolder iteration

* add test for newestSegmentFirstPolicy

* add CompactionSegmentIterator

* add numTargetCompactionSegments

* add missing config

* fix skipping huge shards

* fix handling large number of segments per shard

* fix test failure

* change recursive call to loop

* fix logging

* fix build

* fix test failure

* address comments

* change dataSources type

* check running pendingTasks at each run

* fix test

* address comments

* fix build

* fix test

* address comments

* address comments

* add doc for segment size optimization

* address comment
2018-01-13 13:52:37 +09:00
Roman Leventov 8877ce38d6
Enforce modifier order with Checkstyle (#5246) 2018-01-11 09:50:42 +01:00
Jihoon Son 5d0619f5ce Support retrying for PrefetchableTextFilesFirehoseFactory when prefetch is disabled (#5162)
* Add RetryingInputStream

* unnecessary exception

* fix PrefetchableTextFilesFirehoseFactoryTest

* Fix retrying on connection reset

* fix start offset

* fix checkstyle

* fix check connection reset

* address comments

* fix compile

* address comments

* address comments
2018-01-10 17:37:19 +01:00
Parag Jain 83c6c48bed Fix state check bug in Kafka Index Task (#5204)
* fix state check for replacement task

* fix comments

* rebase with master
2018-01-08 18:01:36 -08:00
Himanshu a46d34daa2 HTTP based task/worker management. (#5104)
* just renaming of SegmentChangeRequestHistory etc

* additional change history refactoring changes

* WorkerTaskManager a replica of WorkerTaskMonitor

* HttpServerInventoryView refactoring to extract sync code and robustification

* Introducing HttpRemoteTaskRunner

* Additional Worker side updates
2018-01-04 19:19:35 -08:00
Roman Leventov 579f9fbedf Add IndexedInts.debugToString() and AbstractIndex.toString(); Add Sequence.toList() and limit() (#5175)
* Add IndexedInts.debugToString() and AbstractIndex.toString()

* Fix AppenderatorTest
2018-01-04 09:56:47 +09:00
David Lim a7967ade4d Support replaceExisting parameter for segments pushers (#5187)
* support replaceExisting parameter for segments pushers

* code review changes

* code review changes
2018-01-03 16:13:21 -08:00
Nishant Bangarwa 59af4d3b14 Fix broken KafkaEmitterConfig parsing (#5201)
* Fix broken KafkaEmitterConfig parsing

This was a regression introduced in
https://github.com/druid-io/druid/pull/4722

KafkaEmitterConfig property names have dot(.) in the name of properties
and JsonConfigurator behavior was changed to not support that.
Added a test and fixed parsing of properties that have dot(.) in
property names

* Fix test failure
2018-01-03 12:08:40 -08:00
Charles Allen 0f773aff80 Fix lookup logging on node start (#5206)
* Add better logging messages in lookups startup on query nodes

* Make sure list is mutable

* Move list to be with other `final` variables
2018-01-03 13:13:55 -06:00
Himanshu 0f5c7d1aec Add freeSpacePercent config in segment location to enforce free space while storing segments (#5137)
* Add freeSpacePercent config in segment location config to enforce free space while storing segments

* address review comments

* address review comments: more doc on freeSpacePercent and use Double for freeSpacePercent
2017-12-21 15:31:09 +03:00
Himanshu f57496ed8b FilteredHttpServerInventoryViewProvider to start with always false predicate for each segment discovered (#5123)
* FilteredHttpServerInventoryViewProvider to start with always false predicate for each segment discovered

* update HttpServerInventoryViewTest to ensure that predicates are honored

* add docs for HttpServerInventoryView.defaultFilter

* change to javadoc style comment
2017-12-20 18:56:00 -08:00
Nishant Bangarwa 494e0b79ed Allow configuring header size for druid requests (#5174)
* Allow configuring header size for druid requests

* fix configuration name in doc.

* add more info to docs.

* Add info to kerberos doc.
2017-12-20 18:51:40 -08:00
Jihoon Son 9199d61389 Automatic pendingSegments cleanup (#5149)
* PendingSegments cleanup

* fix build

* address comments

* address comments

* fix potential npe

* address comments

* fix build

* fix test

* fix test
2017-12-20 14:46:34 -08:00
Roman Leventov 5787d04fad Bump Druid version to 0.12.0 (#5138) 2017-12-15 07:37:01 -08:00
Jonathan Wei f48c9d7be1
Basic auth extension (#5099)
* Basic auth extension

* Add auth configuration integration test

* Fix missing authorizerName property

* PR comments

* Fix missing @JsonProperty annotation

* PR comments

* more PR comments
2017-12-14 10:36:04 -08:00
Roman Leventov 64848c7ebf DataSegment memory optimizations (#5094)
* Deduplicate DataSegments contents (loadSpec's keys, dimensions and metrics lists as a whole) more aggressively; use ArrayMap instead of default LinkedHashMap for DataSegment.loadSpec, because they have only 3 entries on average; prune DataSegment.loadSpec on brokers

* Fix DataSegmentTest

* Refinements

* Try to fix

* Fix the second DataSegmentTest

* Nullability

* Fix tests

* Fix tests, unify to use TestHelper.getJsonMapper()

* Revert TestUtil as ServerTestHelper, fix tests

* Add newline

* Fix indexing tests

* Fix s3 tests

* Try to fix tests, remove lazy caching of ObjectMapper in TestHelper, rename TestHelper.getJsonMapper() to makeJsonMapper()

* Fix HDFS tests

* Fix HdfsDataSegmentPusherTest

* Capitalize constant names
2017-12-12 11:41:40 -08:00
Gian Merlino 294bd3ffb3
DruidCoordinatorBalancer: Fix log message. (#5152) 2017-12-11 20:27:37 -08:00
Gian Merlino 4f5e2b4549 Fix some unemitted alerts. (#5141) 2017-12-06 18:37:01 -08:00
Roman Leventov a7a6a0487e Replace IOPeon with SegmentWriteOutMedium; Improve buffer compression (#4762)
* Replace IOPeon with OutputMedium; Improve compression

* Fix test

* Cleanup CompressionStrategy

* Javadocs

* Add OutputBytesTest

* Address comments

* Random access in OutputBytes and GenericIndexedWriter

* Fix bugs

* Fixes

* Test OutputBytes.readFully()

* Address comments

* Rename OutputMedium to SegmentWriteOutMedium and OutputBytes to WriteOutBytes

* Add comments to ByteBufferInputStream

* Remove unused declarations
2017-12-04 18:04:27 -08:00
Parag Jain 7c01f77b04 Parse Batch support (#5081)
* add parseBatch and deprecate parse method in InputRowParser

add addAll method, skip max rows in memory check for it

remove parse method from implemetations

transform transformers

add string multiplier input row parser

fix withParseSpec

fix kafka batch indexing

fix isPersistRequired

comments

* add unit test

* make persist async

* review comments
2017-12-04 16:06:16 -06:00
Gleb Smirnov aafd0373be Fix query/node/time metric calculation (#5131) 2017-12-01 09:39:04 -08:00
Jihoon Son 322b7f6a27 Fix AnnouncerTest.testSanity() (#5077)
* Fix announcer test

* rename

* split server module test

* add name

* polling until /test1 is created

* add comment
2017-11-20 20:27:41 -08:00
Roman Leventov dbb37b727d Add useL2 and populateL2 configs to HybridCache (#5088)
* Add useL2 and populateL2 configs to HybridCache

* typo
2017-11-20 16:57:05 -06:00
Parag Jain cb03efeb14 Kafka Index Task that supports Incremental handoffs (#4815)
* 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
2017-11-17 16:05:20 -06:00
Jonathan Wei af44d1142b Add unsecured /health endpoint, remove auth checks from isLeader (#5087)
* Add unsecured /health endpoint, remove auth checks from isLeader

* PR comments
2017-11-15 14:41:30 -06:00
Jonathan Wei 9ac150c23a
Split internal client escalation from Authenticator interface (#5073)
* Split internal client escalation from Authenticator interface

* PR comments
2017-11-13 19:29:08 -08:00
Akash Dwivedi c1538f29fc maxQueryTimeout property in runtime properties. (#4852)
* 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.
2017-11-13 19:23:11 -06:00
Jonathan Wei 819700cbc5 Automatically insert authenticator/authorizer names into config properties (#5071) 2017-11-13 13:12:31 -08:00
Himanshu 1c64f02574 make AsyncHttpCLient instance timer thread daemon (#5076) 2017-11-10 19:54:29 -06:00
Egor Riashin 3ad5f51f7e Integration tests port fix, logs output, lib share changes (#5061)
* integration tests port fix, logs output, lib share changes

* Fix test compilation

* docker image fix

* Annotate DiscoverySelector.pick() as Nullable, use formatting instead of string concatenation for logging

* integration-tests logging check

* corrected integration-tests log location

* removed an experimental travis test exception
2017-11-10 14:07:37 -08:00
Gian Merlino 9444da5038 SQL: Improved behavior when implicitly casting strings to date/time literals. (#5023)
* SQL: Improved behavior when implicitly casting strings to date/time literals.

- Handle all flavors of ISO8601 and SQL literals.
- Throw errors on other literals instead of silently transforming them to 0.

* Respect timeZone when format is null.
2017-11-10 17:43:22 +09:00
Himanshu bbb678efd7 fix lookups endpoint collisions (#5058)
* fix lookups endpoint collissions

* fix errors
2017-11-09 17:39:53 -08:00
Goh Wei Xiang 81f249874b Use daemon thread pool for AsyncHttpClient in emitters (#5057)
* use daemon thread pool for AsyncHttpClient in emitters

* changed to use existing helper methods

* refactored creation of AsyncHttpClient
2017-11-09 17:38:19 -08:00
Himanshu 2ecebb3173 Fix coordinator/overlord redirects when TLS is enabled (#5037)
* Fix coordinator/overlord redirects when TLS is enabled

* address review comment

* fix UTs

* workaround to not ignore URL instance to fix the teamcity build

* update tls doc
2017-11-09 13:10:28 -08:00
Jihoon Son c11c71ab3e Using ImmutableDruidDataSource as a key for map and set instead of DruidDataSource (#5054)
* use ImmutableDruidDataSource for map and set

* address comments

* unused import

* allow returning only ImmutableDruidDataSource in MetadataSegmentManager

* address comments

* remove TreeSet

* revert to use TreeSet
2017-11-09 16:07:58 -03:00
Roman Leventov 3541b7544b Prohibit and remove unused declarations in the processing module (#4930)
* Prohibit and remove unused declarations in the processing module

* Fix tests

* Fix integration tests

* Suppress unused

* Try to remove SuppressWarnings unused in VirtualColumn

* Remove reset 'false positives'

* Annotate CliCommandCreator as ExtensionPoint

* Unused import warning instead of error in IntelliJ

* Fixes

* Add comment

* Fix AzureBlob

* Fix CloudFilesBlob

* Address comments

* Add Project SDK section to INTELLIJ_SETUP.md

* Fix image
2017-11-09 09:27:27 -08:00
Jisoo Kim 1bf253f6e6 Fix Router race condition and use default broker service name for invalid priority (#5050)
* use default brokerServiceName when priority is not valid

* use AtomicInteger for NodesHolder.roundRobinIndex

* revert inspectionProfiles change

* adjust TieredBrokerHostSelectorTest

* combine if statements and ensure index does not become negative

* set next index with mod if overflows

* fix codestyle

* use nextIndex

* extract the while loop to a method
2017-11-09 09:31:18 -06:00
Roman Leventov a8dc056c09
Add retries for coordinator fetch and lookup start in LookupReferencesManager (#5029)
* Add retries for coordinator fetch and lookup start in LookupReferencesManager

* Fix LookupConfigTest

* Address comments

* Address more comments

* And address more comments

* Address comms

* Recognize 'not found' lookups in LookupReferencesManager.tryGetLookupListFromCoordinator(), by @egor-ryashin
2017-11-09 02:30:36 -03:00
Roman Leventov 5eb08c27cb Add Emitter monitoring (#4973)
* Add Emitter monitoring

* Fix typo

* Fixes

* testing new emitter

* Fix failed test (#71)

* testing new emitter

* fix on failed test

* Remove emitter's readTimeout from docs

* Update docs

* Add HttpEmittingMonitor

* Update java-util to 1.3.2
2017-11-03 21:27:57 -06:00
Jonathan Wei 13c0d88ffc Fix coordinator->overlord proxy auth failure (#5039)
* Fix coordinator->overlord proxy auth failure

* PR comment
2017-11-03 13:54:41 -07:00
Jonathan Wei 6840eabd87
Add Router connection balancers for Avatica queries (#4983)
* Add Router connection balancers for Avatica queries

* PR comments

* Adjust test bounds

* PR comments

* Add doc comments

* PR comments

* PR comment

* Checkstyle fix
2017-11-01 14:01:13 -07:00
Himanshu 654cdc07f5 Document HTTP based segment management and Deprecate classes to remove in future (#4997)
* document http segment management

* deprecated classes that shouldn't be used any further
2017-11-01 12:59:27 -04:00
Gian Merlino 6c725a7e06 Fix havingSpec on complex aggregators. (#5024)
* 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.
2017-11-01 12:58:08 -04:00
Jihoon Son e96daa2593 Fix SQLMetadataSegmentManager (#5001) 2017-10-31 08:02:41 -07:00
Gian Merlino 0ce406bdf1
Introduce "transformSpec" at ingest-time. (#4890)
* 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.
2017-10-30 17:38:52 -07:00
Jonathan Wei 3e0a6fc374 Filter unauthorized datasources in INFORMATION_SCHEMA queries (#4998)
* Filter unauthorized datasources in INFORMATION_SCHEMA queries

* PR comments
2017-10-26 12:36:47 -07:00
Roman Leventov 125a912067 Add ability to inject extra dimensions for service emitter (#4982)
* Add ability to inject extra dimensions for service emitter

* Docs
2017-10-26 23:57:01 +05:30
Andy Sloane ee66db900e Fix binary serialization in caching (#4993)
* Fix binary serialization in caching

The previous caching code just concatenated a list of objects into a
byte array -- this is actually not valid because jackson-databind uses
enumerated references to strings internally, and concatenating multiple
binary serialized objects can throw off the references.

This change uses a single JsonGenerator to serialize the object list
rather than concatenating byte arrays.

* remove unused imports
2017-10-23 12:10:24 -07:00
Roman Leventov 772ca783cd Fix race in CachingCostBalancerStrategyFactory (#4989)
* Fix race in CachingCostBalancerStrategyFactory

* Remote timeout
2017-10-20 16:53:51 -07:00
Himanshu ef4a8cb724 Optional segment load/drop management without zookeeper using http (#4966)
* introducing CuratorLoadQueuePeon

* HttpLoadQueuePeon based off of current code

* Revert "Remove SegmentLoaderConfig.numLoadingThreads config (#4829)"

This reverts commit d8b3bfa63c.

* SegmentLoadDropHandler copy/pasted from ZkCoordinator

* Revert "1-based counts in ZkCoordinator (#4917)"

This reverts commit e725ff4146.

* remove non-zk part from ZkCoordinator

* remove zk part from SegmentLoadDropHandler

* additional changes for segment load/drop management with http

* address review comments

* add some more logs

* Execs class is moved
2017-10-19 12:41:23 -07:00
Roman Leventov 26b87c9f8e Fix CachingCostBalancerStrategyFactory's constructor (#4974)
* Fix CachingCostBalancerStrategyFactory's constructor

* Fix CachingCostBalancerStrategyFactory not registered in Lifecycle
2017-10-18 16:21:54 -05:00
Gian Merlino 5fc6891404 Reduce code duplication between test ExprMacroTables. (#4979) 2017-10-18 15:57:49 -05:00
Gian Merlino 4881bb273b Only consider loaded replicants when computing replication status. (#4921)
* Only consider loaded replicants when computing replication status.

This affects the computation of segment/underReplicated/count and
segment/unavailable/count, as well as the loadstatus?simple and
loadstatus?full APIs.

I'm not sure why they currently consider segments in the load
queues, but it would make more sense to me if they only considered
segments that are actually loaded.

* Fix tests.

* Fix imports.
2017-10-18 11:11:42 -07:00
Roman Leventov dc7cb117a1 Refactor ColumnSelectorFactory; Rely on ColumnValueSelector's polymorphism (#4886)
* Refactor ColumnSelectorFactory; Rely on ColumnValueSelector's polymorphism

* Fix MapVirtualColumn.makeColumnValueSelector()

* Minor fixes

* Fix IndexGeneratorCombinerTest

* DimensionSelector to return zeros when treated as numeric ColumnValueSelector

* Fix IncrementalIndexTest

* Fix IncrementalIndex.makeColumnSelectorFactory()

* Optimize MapBasedRow.getMetric()

* Fix VarianceAggregatorTest

* Simplify IncrementalIndex.makeColumnSelectorFactory()

* Address comments

* More comments

* Test
2017-10-13 21:44:17 -05:00
Jihoon Son 8d9902831e Refactoring PrefetchableTextFilesFirehoseFactory (#4836)
* Refactoring prefetchable firehose

* Fix to read cache when prefetch is disabled

* More tests

* Cleanup codes

* Add Fetcher

* Fix test failure

* Count file size

* Fix test

* rename generic parameter

* address comments

* address comments

* reuse buffer

* move Execs to java-util

* use execs

* Fix build
2017-10-13 21:39:28 -05:00
Jihoon Son 675c6c00dd Add checkstyle and intellij rule to prohibit unnecessary qualifiers in interfaces (#4958)
* add checkstyle and intellij rule

* fix tc fail
2017-10-13 07:56:19 -07:00
Atul Mohan c07678b143 Synchronization of lookups during startup of druid processes (#4758)
* 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
2017-10-12 21:22:24 -05:00
Jihoon Son d95915f8d2 Implement get methods for PrefetchableFirehose (#4948) 2017-10-12 16:14:45 +09:00
Jihoon Son dfa9cdc982 Prioritized locking (#4550)
* Implementation of prioritized locking

* Fix build failure

* Fix tc fail

* Fix typos

* Fix IndexTaskTest

* Addressed comments

* Fix test

* Fix spacing

* Fix build error

* Fix build error

* Add lock status

* Cleanup suspicious method

* Add nullables

*  add doInCriticalSection to TaskLockBox and revert return type of task actions

* fix build

* refactor CriticalAction

* make replaceLock transactional

* fix formatting

* fix javadoc

* fix build
2017-10-11 23:16:31 -07:00
Roman Leventov 7a9940d624 Add /readiness to HistoricalResource (#4916)
* Add /loadStatusCode to HistoricalResource

* Address comments

* Fixes
2017-10-11 20:35:52 -07:00
Jihoon Son 56fb11ce0b Lazy initialization for JavaScript functions (#4871)
* Lazy initialization of JavaScript functions

* Fix test failure

* Fix thread-safety and postpone js conf check

* Fix test fail

* Fix test

* Fix KafkaIndexTaskTest

* Move config check
2017-10-10 21:52:42 -07:00
Roman Leventov e725ff4146 1-based counts in ZkCoordinator (#4917) 2017-10-10 13:00:51 -07:00
Kevin Conaway 1bc4b71a34 Reduce Chance of Duplicates in EventReceiverFireHose (#4903)
* Add ability to optionally specify a sequence identifier to reduce the possibility of duplicate events entering the event receiver firehose

* Add ability to optionally specify a sequence identifier to reduce the possibility of duplicate events entering the event receiver firehose

* Add a hard coded limit to the maximum number of possible producer IDs to prevent a malicious (or uninformed) client from overflowing the heap
2017-10-10 11:17:17 -07:00
Parag Jain 7cc18226cd add more tls configs to enable/disable specific cipher suites and protocols (#4902)
* add more tls configs to enable/disable specific cipher suites and protocols

* fix doc, allow empty list
2017-10-09 13:53:12 -07:00
Gian Merlino 797b54d283 DruidLeaderClient: Throw IOException on retryable errors. (#4913)
* DruidLeaderClient: Throw IOException on retryable errors.

Fixes #4911.

* Adjustments.
2017-10-06 15:12:09 -05:00
Himanshu 0e856ee806 add configs to enable fast request failure on broker and historical (#4540)
* add configs to enable fast request failure on broker

* address review comments

* fix styling error

* fix style error

* have enableRequestLimit config instead of having user specify max limit

* add comment

* fix style error

* add UT fo LimitRequestsFilter

* address review comments

* fix test

* make LimitRequestsFilterTest more robust

* fix JettyQosTest
2017-10-06 14:45:13 -05:00
praveev 4ff12e4394 Hadoop indexing: Fix NPE when intervals not provided (#4686)
* Fix #4647

* NPE protect bucketInterval as well

* Add test to verify timezone as well

* Also handle case when intervals are already present

* Fix checkstyle error

* Use factory method instead for Datetime

* Use Intervals factory method
2017-10-05 22:46:07 -07:00
Akash Dwivedi 2ee32399ff granularity method in QueryMetrics. (#4570)
* granularity method in QueryMetrics.

PR to emit granularity dimension for timeseries, search, groupBy,
select and topN queries.

* QueryMetricsFactory classes for search and select queries.

* Empty implementation  for  Granularity() method.

* Review comment changes.

* Remove unused import.

* empty query() method.

* checkstyle fix.

* Import fix.
2017-10-04 09:42:52 -07:00
Jonathan Wei 07aa405a6f Fix PreResponseAuthorizationCheckFilter HTTP error masking (#4900)
* Fix PreResponseAuthorizationCheckFilter HTTP error masking

* Add remote addr and host to missing auth check log message
2017-10-03 16:58:57 -05:00
Jonathan Wei 5e60ccade1 Add context map to AuthenticationResult (#4870) 2017-10-02 17:08:14 -05:00
Jonathan Wei 9deab26d8b Fix auth check in InventoryViewUtils (#4869) 2017-10-02 11:38:45 -07:00
Niketh Sabbineni 3e9391433d Coord resource throws NPE when segments are requested (#4759) 2017-10-02 10:13:27 -07:00
Gian Merlino 1f2074c247 Bump versions in master to 0.11.1-SNAPSHOT. (#4878)
* Bump versions in master to 0.11.1-SNAPSHOT.

* Missed a few.
2017-09-28 17:09:51 -05:00
Goh Wei Xiang 26fd2b3a8e Priority on loading for primary replica (#4757)
* Priority on loading for primary replica

* Simplicity fixes

* Fix on skipping drop for quick return.

* change to debug logging for no replicants.

* Fix on filter logic

* swapping if-else

* Fix on wrong "hasTier" logic

* Refactoring of LoadRule

* Rename createPredicate to createLoadQueueSizeLimitingPredicate

* Rename getHolderList to getFilteredHolders

* remove varargs

* extract out currentReplicantsInTier

* rename holders to holdersInTier

* don't do temporary removal of tier.

* rename primaryTier to tierToSkip

* change LinkedList to ArrayList

* Change MinMaxPriorityQueue in DruidCluster to TreeSet.

* Adding some comments.

* Modify log messages in light of predicates.

* Add in-method comments

* Don't create new Object2IntOpenHashMap for each run() call.

* Cache result from strategy call in the primary assignment to be reused during the same run.

* Spelling mistake

* Cleaning up javadoc.

* refactor out loading in progress check.

* Removed redundant comment.

* Removed forbidden API

* Correct non-forbidden API.

* Precision in variable type for NavigableSet.

* Obsolete comment.

* Clarity in method call and moving retrieval of ServerHolder into method call.

* Comment on mutability of CoordinatoorStats.

* Added auxiliary fixture for dropping.
2017-09-28 13:02:05 -07:00
Gian Merlino a19f22b5bb Add identity to query metrics, logs. (#4862)
* Add identity to query metrics, logs.

Also fix a bug where unauthorized requests would not emit any logs or metrics,
and instead would log a "Tried to emit logs and metrics twice" warning.

Also rename QueryResource's "getServer" to "cancelQuery", because that's what
it does.

* Do not emit identity by default.
2017-09-28 11:45:23 -07:00
Himanshu f69c9280c4 remove ServerConfig from DruidNode as all information needs to be present in DruidNode serialized form (#4858)
* remove ServerConfig from DruidNode as all information needs to be present in DruidNode serialized form

* sanitize output of /druid/coordinator/v1/cluster endpoint
2017-09-28 10:40:59 -05:00
Goh Wei Xiang 2c30d5ba55 Add org.joda.time.DateTime.parse() to forbidden APIs (#4857)
* Added org.joda.time.DateTime#(java.lang.String) to forbidden API.

* Added org.joda.time.DateTime#(java.lang.String, org.joda.time.format.DateTimeFormatter) to forbidden API.

* Add additional APIs that may create DateTime with default time zone

* Add helper function that accepts formatter to parse String.

* Add additional forbidden APIs

* Replace existing usage of forbidden APIs

* Use wrapper class to enforce Chronology on DateTimeFormatter.

* Creates constant UtcFormatter for constant ISODateTimeFormat.
2017-09-27 17:46:44 -05:00
Gian Merlino 999c6d800e Fix Router handling of SQL queries. (#4851) 2017-09-27 10:58:24 -07:00
Roman Leventov 9c126e2aa9 Forbid MapMaker (#4845)
* Forbid MapMaker

* Shorter syntax

* Forbid Maps.newConcurrentMap()
2017-09-27 06:49:47 -07:00
Charles Allen a6470c1d03 Move caffeine out of extension and make it the default cache implementation. (#4810)
* Move caffeine out of extension.

* Remove `JsonTypeName` from the class itself

* Fix bad docs

* Fix distribution pom

* Fix unused import

* Make caffeine default

* Address code comments

* Add more description around the jre version in the readme

* Add suggested comments
2017-09-22 10:46:55 -07:00
Jonathan Wei 09fcb75583 Add RequestLogEvent emitters config to graphite-emitter (#4678)
* Add RequestLogEvent emitters config to graphite-emitter

* eagerly compute emitter list

* use lambdas

* checkstyle
2017-09-22 06:14:32 -07:00
Roman Leventov e267f3901b Enforce Indentation with Checkstyle (#4799) 2017-09-21 13:06:48 -07:00
Roman Leventov d8b3bfa63c Remove SegmentLoaderConfig.numLoadingThreads config (#4829) 2017-09-20 21:27:43 -07:00
Charles Allen 47ebc48059 Use java 8 features in TierSelectorStrategy implementations (#4827)
* Use java 8 features in TierSelectorStrategy implementations
* Minor code cleanup

* More java8 coolness

* Code comments
2017-09-20 22:09:29 -05:00
Roman Leventov 88e9a80636 Rename ObjectValueSelector.get() to getObject(); Add getObject() and classOfObject() to ColumnValueSelector (#4801) 2017-09-19 14:47:20 -05:00
Jonathan Wei 3a4a483bb0 Single auth check for authorized resource filtering (#4818)
* Single auth check for authorized resource filtering

* PR comment

* PR comments
2017-09-19 21:46:08 +05:30
Jonathan Wei c2a0e753b6 Extension points for authentication/authorization (#4271)
* Extension points for authentication/authorization

* Address some PR comments

* Authorization result caching

* Add unit tests for SecuritySanityCheckFilter and PreResponseAuthorizationCheckFilter

* Use Set for auth caching, close outputstreams in filters

* Don't close output stream on success in sanity check filter

* Add ConfigResourceFilter to coordinator lookups

* Fix filtering authorization check for empty resource list

* HttpClient users must explicitly escalate the client

* Remove response modification from PreResponseAuthorizationCheckFilter

* Remove extraneous pom.xml

* Fix unit test

* Better lifecycle management

* Rename AuthorizationManager to Authorizer

* Fix authorization denials for empty supervisor list

* Address some PR comments

* Address more PR comments

* Small cleanup

* Add Jetty HttpClient wrapper to Authenticator

* Remove Authorizer start/stop

* Restore immutable context map in DruidConnection, UT fix

* Fix/update docs

* Add authorization checks to EventReceiverFirehose

* Fix router authorization check failure, restore PreResponseAuthorizationFilter changes

* Compile fixes

* Test fixes

* Update Authenticator/Authorizer doc comments

* Merge fixes

* PR comments

* Fix test

* Fix IT

* More PR comments

* PR comments

* SSL fix
2017-09-15 23:45:48 -07:00
Himanshu d37be5e6e9 don't hold thread while waiting after failure from server (#4795) 2017-09-14 17:19:25 -05:00
Akash Dwivedi a17e48fe69 search package name correction. (#4785)
* search package name correction.

* Refactor search.search pkg to search.

* remove unused import.
2017-09-14 13:50:23 -07:00
Roman Leventov 267f415dc3 Update emitter library and add support for ParametrizedUriEmitter (#4722)
* Move emitters from io.druid.server.initialization to the dedicated io.druid.server.emitter package; Update emitter library to 0.6.0; Add support for ParametrizedUriEmitter; Support hierarical properties in JsonConfigurator (was needed for ParametrizedUriEmitter)

* Log created RequestLoggers

* Fix forbidden API

* Test fix

* More Http and Parametrized Http Emitter docs

* Switch to debug level
2017-09-13 17:17:19 -05:00
Himanshu 7919469de6 fixes HttpServerInventoryView to call server/segment callbacks correctly and Unit Tests for the class (#4767)
* fixes HttpServerInventoryView to call server/segment callbacks correctly and Unit Tests for the class

* fix checkstyle and forbidden-api errors

* HttpServerInventoryView to finish start() only after server inventory is initialized

* fix compilation errors

* address review comments

* add exponential backoff instead of fixed 5 secs on successive failures

* update test to exercise server fail scenarios

* use AtomicInteger for requestNum and increment only once
2017-09-13 14:24:19 -05:00