Commit Graph

3432 Commits

Author SHA1 Message Date
Roman Leventov 2cdce2e2a6
Add RequestLogEventBuilderFactory (#6477)
This PR allows to control the fields in `RequestLogEvent`, emitted in `EmittingRequestLogger`. In our case, we want to get rid of the `intervals` fields of the query objects that are a part of `DefaultRequestLogEvent`. They are enormous (thousands of segments) and not useful.

Related to #5522, FYI @a2l007.
2018-10-31 22:24:37 +01:00
QiuMM 676f5e6d7f Prohibit some guava collection APIs and use JDK collection APIs directly (#6511)
* Prohibit some guava collection APIs and use JDK APIs directly

* reset files that changed by accident

* sort codestyle/druid-forbidden-apis.txt alphabetically
2018-10-29 13:02:43 +01:00
Jonathan Wei b2d9b6f23d Allow custom TLS cert checks (#6432)
* Allow custom TLS cert checks

* PR comment

* Checkstyle, PR comment
2018-10-24 16:31:52 -07:00
QiuMM 601183b4c7 Add period drop before rule (#6415)
* Add period drop before rule

* add license header

* support period drop before rule in coordinator console

* address comments
2018-10-24 12:44:30 -07:00
Roman Leventov 84ac18dc1b
Catch some incorrect method parameter or call argument formatting patterns with checkstyle (#6461)
* Catch some incorrect method parameter or call argument formatting patterns with checkstyle

* Fix DiscoveryModule

* Inline parameters_and_arguments.txt

* Fix a bug in PolyBind

* Fix formatting
2018-10-23 07:17:38 -03:00
Faxian Zhao c5bf4e7503 update insert pending segments logic to synchronous (#6336)
* 1. Mysql default transaction isolation is REPEATABLE_READ, treat it as READ_COMMITTED will reduce insert id conflict.
2. Add an index to 'dataSource used end' is work well for the most of scenarios(get recently segments), and it will speed up sync add pending segments in DB.
3. 'select and insert' is not need within transaction.

* Use TaskLockbox.doInCriticalSection instead of synchronized syntax to speed up insert pending segments.

* fix typo for NullPointerException
2018-10-22 19:48:20 -07:00
Samarth Jain 359576a80b Implement force push down for nested group by query (#5471)
* Force nested query push down

* Code review changes
2018-10-22 13:43:47 -07:00
QiuMM f5f4171a45 QueryCountStatsMonitor: emit query/count (#6473)
Let `QueryCountStatsMonitor` emit `query/count`, then I can monitor QPS of my services, or I have to count it by myself.
2018-10-19 10:15:02 -03:00
patelh c780aacc03 Add ability to specify dbcp properties file (#6419)
* Add ability to specify dbcp properties file

* Address PR comments, use mock config, remove setter

* Add documentation

* APRC, updated docs with example file contents

* APRC, add @Nullable, @VisibileForTesting, update doc

* APRC, remove error log, use props directly as jackson binding

* Remove unused files
2018-10-16 12:27:19 -07:00
QiuMM 85a89e2703 make druid node bind address configurable (#6464)
* make druid node bind address configurable

* fix tests

* fix travis-ci
2018-10-15 14:19:40 -07:00
Roman Leventov aa121da25f Use NodeType enum instead of Strings (#6377)
* Use NodeType enum instead of Strings

* Make NodeType constants uppercase

* Fix CommonCacheNotifier and NodeType/ServerType comments

* Reconsidering comment

* Fix import

* Add a comment to CommonCacheNotifier.NODE_TYPES
2018-10-14 20:49:38 -07:00
Clint Wylie 84598fba3b combine druid-api, druid-common, java-util into druid-core (#6443)
* combine druid-api, druid-common, java-util

* spacing
2018-10-14 20:37:37 -07:00
vishnu rao 6567fff9e7 Query Response format to be based on http 'accept' header & Query Payload content type to be based on 'content-type' header (#4033)
* o- Query Response format to be based on http 'accept' header & Query Payload contenty type to be based on 'content-type' header

* o- Query Response format to be based on http 'accept' header & Query Payload contenty type to be based on 'content-type' header
o- if Accept header is absent, it defaults to Content-Type header

* Feature: Query Response format to be based on http 'accept' header & Query Payload content type to be based on 'content-type'  PR #4033
Minor change to a comment - restoring to previous wording

* Feature: Query Response format to be based on http 'accept' header & Query Payload content type to be based on 'content-type'  PR #4033
o- minor change to check for empty string
2018-10-12 14:29:14 -07:00
Surekha 3be4a97150 Fix inconsistent segment size(#6448) (#6451)
* Fix inconsistent segment size(#6448)

* Fix the segment size for published segments
* Changes to get numReplicas
* Make coordinator segments API truly streaming

* Changes to store partial segment data

* Simplify SegmentMetadataHolder
* Store partial the columns from available segments

* Address comments
2018-10-12 12:55:20 -07:00
Clint Wylie 39d61b9ae5 update druid-console to 0.0.4 (#6450) 2018-10-11 22:37:08 -06:00
David Lim 20ab213ba6 change project versions to 0.13.0-incubating-SNAPSHOT (#6453) 2018-10-11 19:28:01 -07:00
Clint Wylie f7775d1db3 fixes for LookupReferencesManagerTest (#6444)
* some fixes for LookupReferencesManagerTest

* docs

* formatting

* more formatting fixes
2018-10-10 18:02:11 -07:00
Surekha 3a0a667fe0 Introduce SystemSchema tables (#5989) (#6094)
* Added SystemSchema with following tables (#5989)

* SEGMENTS table provides details on served and published segments
* SERVERS table provides details on data servers
* SERVERSEGMETS table is the JOIN of SEGMENTS and SERVERS
* TASKS table provides details on tasks

* Add documentation for system schema

* Fix static-analysis warnings

* Address PR comments

*Add unit tests

* Fix a test

* Try to fix a test

* Fix a bug around replica count

* rename io.druid to org.apache.druid

* Major change is to make tasks and segment queries streaming

* Made tasks/segments stream to calcite instead of storing it in memory
* Add num_rows to segments table
* Refactor JsonParserIterator
* Replace with closeable iterator

* Fix docs, make num_rows column nullable, some unit test changes

* make num_rows column type long, allow it to be null

fix a compile error after merge, add TrafficCop param to InputStreamResponseHandler

* Filter null rows for segments table from Linq4j enumerable

* change num_replicas datatype to long in segments table

* Fix some tests and address comments

* Doc updates, other PR comments

* Update tests

* Address comments

* Add auth check
* Update docs
* Refactoring

* Fix teamcity warning, change the getQueryableServer in TimelineServerView

* Fix compilation after rebase

* Use the stream API from AuthorizationUtils

* Added LeaderClient interface and NoopDruidLeaderClient class

* Revert "Added LeaderClient interface and NoopDruidLeaderClient class"

This reverts commit 100fa46e39.

* Make the naming consistent to server_segments for the join table

* Add ForbiddenException on auth check failure
* Remove static block from SystemSchema

* Try to fix a test in CalciteQueryTest due to rename of server_segments

* Fix the json output format in the coordinator API

* Add auth check in the segments API
* Add null check to avoid NPE

* Use annonymous class object instead of mock for DruidLeaderClient in SqlBenchmark

* Fix test failures, type long/BIGINT can be nullable

* Revert long nullability to fix tests

* Fix style for tests

* PR comments

* Address PR comments

* Add the missing BytesAccumulatingResponseHandler class

* Use Sequences.withBaggage in DruidPlanner

* Fix docs, add comments

* Close the iterator if hasNext returns false
2018-10-10 17:17:29 -07:00
Jihoon Son 88d23b77b7 Add support keepSegmentGranularity for automatic compaction (#6407)
* Add support keepSegmentGranularity for automatic compaction

* skip unknown dataSource

* ignore single semgnet to compact

* add doc

* address comments

* address comment
2018-10-07 16:48:58 -07:00
Jihoon Son 45aa51a00c Add support hash partitioning by a subset of dimensions to indexTask (#6326)
* Add support hash partitioning by a subset of dimensions to indexTask

* add doc

* fix style

* fix test

* fix doc

* fix build
2018-10-06 16:45:07 -07:00
QiuMM 0b8085aff7 Prohibit jackson ObjectMapper#reader methods which are deprecated (#6386)
* Prohibit jackson ObjectMapper#reader methods which are deprecated

* address comments
2018-10-03 17:55:20 -03:00
Roman Leventov 3ae563263a
Renamed 'Generic Column' -> 'Numeric Column'; Fixed a few resource leaks in processing; misc refinements (#5957)
This PR accumulates many refactorings and small improvements that I did while preparing the next change set of https://github.com/druid-io/druid/projects/2. I finally decided to make them a separate PR to minimize the volume of the main PR.

Some of the changes:
 - Renamed confusing "Generic Column" term to "Numeric Column" (what it actually implies) in many class names.
 - Generified `ComplexMetricExtractor`
2018-10-02 14:50:22 -03:00
Jihoon Son cb14a43038 Remove ConvertSegmentTask, HadoopConverterTask, and ConvertSegmentBackwardsCompatibleTask (#6393)
* Remove ConvertSegmentTask, HadoopConverterTask, and ConvertSegmentBackwardsCompatibleTask

* update doc and remove auto conversion

* remove remaining doc

* fix teamcity
2018-10-01 12:03:35 -07:00
Gian Merlino 9fa4afdb8e URL encode datasources, task ids, authenticator names. (#5938)
* URL encode datasources, task ids, authenticator names.

* Fix URL encoding for router forwarding servlets.

* Fix log-with-offset API.

* Fix test.

* Test adjustments.

* Task client fixes.

* Remove unused import.
2018-09-30 12:29:51 -07:00
Shiv Toolsidass 5a894f830b Added backpressure metric (#6335)
* Added backpressure metric

* Updated channelReadable to AtomicBoolean and fixed broken test

* Moved backpressure metric logic to NettyHttpClient

* Fix placement of calculating backPressureDuration
2018-09-29 14:24:04 -07:00
Jihoon Son 122caec7b1
Add support targetCompactionSizeBytes for compactionTask (#6203)
* Add support targetCompactionSizeBytes for compactionTask

* fix test

* fix a bug in keepSegmentGranularity

* fix wrong noinspection comment

* address comments
2018-09-28 11:16:35 -07:00
Jihoon Son aef022de98 Fix race in taskMaster (#6388) 2018-09-26 21:48:02 -07:00
Clint Wylie fc1d5795c1 remove wikipedia irc firehose and dependencies from core server module to examples (#6391) 2018-09-26 21:46:37 -07:00
Roman Leventov 8978d3751b Don't convert DruidServer to ImmutableDruidServers multiple times in CoordinatorHistoricalManagerRunnable (#6385) 2018-09-26 09:14:14 -07:00
Gian Merlino a92a20e037 Fix indexes introduced in #6348. (#6356)
The indexes introduced in #6348 were on the wrong table. The tests
did not catch them due to retries on the create table steps (the
first try created the table but not the bogus indexes; the second
try noticed that the table already existed and did nothing). This
patch doesn't fix the issue with the tests, since the best way to
do that would be to do the table and index creation in a
transaction; but, this is not supported by all of our supported
database engines.
2018-09-25 20:49:13 -07:00
Jihoon Son d08c2c5eba
Make JvmThreadsMonitor injectable (#6369) 2018-09-24 20:41:17 -07:00
Jihoon Son 99428e20d2 Deprecate dimensions / metrics APIs on brokers (#6361)
* Deprecate dimensions / metrics APIs on brokers

* add segmentMetadataQuery link

* add more doc
2018-09-24 17:56:38 -07:00
Roman Leventov 9a3195e98c Improve interning in SQLMetadataSegmentManager (#6357)
* Improve interning in SQLMetadataSegmentManager

* typo
2018-09-22 13:23:30 -07:00
Jonathan Wei 364bf9d1f9 Fix non org.apache.druid files and add package name checkstyle rule (#6367)
* Fix non org.apache.druid files and add package name checkstyle rule

* PR comment
2018-09-21 17:58:19 -07:00
Gian Merlino e1c649e906 Add metadata indexes to help with segment allocation. (#6348)
Segment allocation queries can take a long time (10s of seconds) when
you have a lot of segments. Adding these indexes helps greatly.
2018-09-19 15:54:13 -07:00
Jonathan Wei 8972244c68 Mutual TLS support (#6076)
* Mutual TLS support

* Kafka test fixes

* TeamCity fix

* Split integration tests

* Use localhost DOCKER_IP

* Increase server thread count

* Increase SSL handshake timeouts

* Add broken pipe retries, use injected client config params

* PR comments, Rat license check exclusion
2018-09-19 09:56:15 -07:00
Slim Bouguerra 028354eea8 Adding licenses and enable apache-rat-plugin. (#6215)
* Adding licenses and enable apache-rat-plugi.

Change-Id: I4685a2d9f1e147855dba69329b286f2d5bee3c18

* restore the copywrite of demo_table and add it to the list of allowed ones

Change-Id: I2a9efde6f4b984bc1ac90483e90d98e71f818a14

* revirew comments

Change-Id: I0256c930b7f9a5bb09b44b5e7a149e6ec48cb0ca

* more fixup

Change-Id: I1355e8a2549e76cd44487abec142be79bec59de2

* align

Change-Id: I70bc47ecb577bdf6b91639dd91b6f5642aa6b02f
2018-09-18 08:39:26 -07:00
Hongze Zhang 2fac6743d4 Add maxIdleTime option to EventReceiverFirehose (#5997) 2018-09-17 13:50:56 -07:00
Roman Leventov 0c4bd2b57b Prohibit some Random usage patterns (#6226)
* Prohibit Random usage patterns

* Fix FlattenJSONBenchmarkUtil
2018-09-14 13:35:51 -07:00
QiuMM 87ccee05f7 Add ability to specify list of task ports and port range (#6263)
* support specify list of task ports

* fix typos

* address comments

* remove druid.indexer.runner.separateIngestionEndpoint config

* tweak doc

* fix doc

* code cleanup

* keep some useful comments
2018-09-13 19:36:04 -07:00
Roman Leventov d50b69e6d4 Prohibit LinkedList (#6112)
* Prohibit LinkedList

* Fix tests

* Fix

* Remove unused import
2018-09-13 18:07:06 -07:00
Clint Wylie 91a37c692d 'suspend' and 'resume' support for supervisors (kafka indexing service, materialized views) (#6234)
* 'suspend' and 'resume' support for kafka indexing service
changes:
* introduces `SuspendableSupervisorSpec` interface to describe supervisors which support suspend/resume functionality controlled through the `SupervisorManager`, which will gracefully shutdown the supervisor and it's tasks, update it's `SupervisorSpec` with either a suspended or running state, and update with the toggled spec. Spec updates are provided by `SuspendableSupervisorSpec.createSuspendedSpec` and `SuspendableSupervisorSpec.createRunningSpec` respectively.
* `KafkaSupervisorSpec` extends `SuspendableSupervisorSpec` and now supports suspend/resume functionality. The difference in behavior between 'running' and 'suspended' state is whether the supervisor will attempt to ensure that indexing tasks are or are not running respectively. Behavior is identical otherwise.
* `SupervisorResource` now provides `/druid/indexer/v1/supervisor/{id}/suspend` and `/druid/indexer/v1/supervisor/{id}/resume` which are used to suspend/resume suspendable supervisors
* Deprecated `/druid/indexer/v1/supervisor/{id}/shutdown` and moved it's functionality to `/druid/indexer/v1/supervisor/{id}/terminate` since 'shutdown' is ambiguous verbage for something that effectively stops a supervisor forever
* Added ability to get all supervisor specs from `/druid/indexer/v1/supervisor` by supplying the 'full' query parameter `/druid/indexer/v1/supervisor?full` which will return a list of json objects of the form `{"id":<id>, "spec":<SupervisorSpec>}`
* Updated overlord console ui to enable suspend/resume, and changed 'shutdown' to 'terminate'

* move overlord console status to own column in supervisor table so does not look like garbage

* spacing

* padding

* other kind of spacing

* fix rebase fail

* fix more better

* all supervisors now suspendable, updated materialized view supervisor to support suspend, more tests

* fix log
2018-09-13 14:42:18 -07:00
Gian Merlino d6cbdf86c2
Broker backpressure. (#6313)
* Broker backpressure.

Adds a new property "druid.broker.http.maxQueuedBytes" and a new context
parameter "maxQueuedBytes". Both represent a maximum number of bytes queued
per query before exerting backpressure on the channel to the data server.

Fixes #4933.

* Fix query context doc.
2018-09-10 09:33:29 -07:00
Clint Wylie e6e068ce60 Add support for 'maxTotalRows' to incremental publishing kafka indexing task and appenderator based realtime task (#6129)
* resolves #5898 by adding maxTotalRows to incremental publishing kafka index task and appenderator based realtime indexing task, as available in IndexTask

* address review comments

* changes due to review

* merge fail
2018-09-07 13:17:49 -07:00
Gian Merlino 431d3d8497
Rename io.druid to org.apache.druid. (#6266)
* Rename io.druid to org.apache.druid.

* Fix META-INF files and remove some benchmark results.

* MonitorsConfig update for metrics package migration.

* Reorder some dimensions in inner queries for some reason.

* Fix protobuf tests.
2018-08-30 09:56:26 -07:00
Gian Merlino cb40b6d369 Fix all inspection errors currently reported. (#6236)
* Fix all inspection errors currently reported.

TeamCity builds on master are reporting inspection errors, possibly
because there was a while where it was not running due to the Apache
migration, and there was some drift.

* Fix one more location.

* Fix tests.

* Another fix.
2018-08-26 18:36:01 -06:00
Benedict Jin 3647d4c94a Make time-related variables more readable (#6158)
* Make time-related variables more readable

* Patch some improvements from the code reviewer

* Remove unnecessary boxing of Long type variables
2018-08-21 15:29:40 -07:00
Jihoon Son 2bfe1b6a5a Fix NPE for taskGroupId when rolling update (#6168)
* Fix NPE for taskGroupId

* missing changes

* fix wrong annotation

* fix potential race

* keep baseSequenceName

* make deprecated old param
2018-08-17 10:15:45 -07:00
Samarth Jain 1c8032f9f3 Composite request logger doesn't invoke @LifeCycleStart and @LifeCycleStop methods on its dependencies (#6173) 2018-08-17 12:34:25 -04:00
Gian Merlino 5ce3185b9c Fix three bugs with segment publishing. (#6155)
* Fix three bugs with segment publishing.

1. In AppenderatorImpl: always use a unique path if requested, even if the segment
   was already pushed. This is important because if we don't do this, it causes
   the issue mentioned in #6124.
2. In IndexerSQLMetadataStorageCoordinator: Fix a bug that could cause it to return
   a "not published" result instead of throwing an exception, when there was one
   metadata update failure, followed by some random exception. This is done by
   resetting the AtomicBoolean that tracks what case we're in, each time the
   callback runs.
3. In BaseAppenderatorDriver: Only kill segments if we get an affirmative false
   publish result. Skip killing if we just got some exception. The reason for this
   is that we want to avoid killing segments if they are in an unknown state.

Two other changes to clarify the contracts a bit and hopefully prevent future bugs:

1. Return SegmentPublishResult from TransactionalSegmentPublisher, to make it
more similar to announceHistoricalSegments.
2. Make it explicit, at multiple levels of javadocs, that a "false" publish result
must indicate that the publish _definitely_ did not happen. Unknown states must be
exceptions. This helps BaseAppenderatorDriver do the right thing.

* Remove javadoc-only import.

* Updates.

* Fix test.

* Fix tests.
2018-08-15 13:55:53 -07:00
Jihoon Son ecee3e0a24 Further optimize memory for Travis jobs (#6150)
* Further optimize memory for Travis jobs

* fix build

* sudo false
2018-08-10 22:03:36 -07:00
Christoph Hösler 1a37dfdcd1 Fetch unhandled curator exceptions (#6131)
* fix: stop druid on unhandled curator exceptions

* catch exceptions when stopping lifecycle
2018-08-09 21:47:42 -07:00
Jihoon Son d6a02de5b5 Add support 'keepSegmentGranularity' for compactionTask (#6095)
* Add keepSegmentGranularity for compactionTask

* fix build

* createIoConfig method

* fix build

* fix build

* address comments

* fix build
2018-08-09 13:51:20 -07:00
Gian Merlino 3525d4059e
Cache: Add maxEntrySize config, make groupBy cacheable by default. (#5108)
* Cache: Add maxEntrySize config.

The idea is this makes it more feasible to cache query types that
can potentially generate large result sets, like groupBy and select,
without fear of writing too much to the cache per query.

Includes a refactor of cache population code in CachingQueryRunner and
CachingClusteredClient, such that they now use the same CachePopulator
interface with two implementations: one for foreground and one for
background.

The main reason for splitting the foreground / background impls is
that the foreground impl can have a more effective implementation of
maxEntrySize. It can stop retaining subvalues for the cache early.

* Add CachePopulatorStats.

* Fix whitespace.

* Fix docs.

* Fix various tests.

* Add tests.

* Fix tests.

* Better tests

* Remove conflict markers.

* Fix licenses.
2018-08-07 10:23:15 -07:00
Jihoon Son 56ab4363ea
Native parallel batch indexing without shuffle (#5492)
* Native parallel indexing without shuffle

* fix build

* fix ci

* fix ingestion without intervals

* fix retry

* fix retry

* add it test

* use chat handler

* fix build

* add docs

* fix ITUnionQueryTest

* fix failures

* disable metrics reporting

* working

* Fix split of static-s3 firehose

* Add endpoints to supervisor task and a unit test for endpoints

* increase timeout in test

* Added doc

* Address comments

* Fix overlapping locks

* address comments

* Fix static s3 firehose

* Fix test

* fix build

* fix test

* fix typo in docs

* add missing maxBytesInMemory to doc

* address comments

* fix race in test

* fix test

* Rename to ParallelIndexSupervisorTask

* fix teamcity

* address comments

* Fix license

* addressing comments

* addressing comments

* indexTaskClient-based segmentAllocator instead of CountingActionBasedSegmentAllocator

* Fix race in TaskMonitor and move HTTP endpoints to supervisorTask from runner

* Add more javadocs

* use StringUtils.nonStrictFormat for logging

* fix typo and remove unused class

* fix tests

* change package

* fix strict build

* tmp

* Fix overlord api according to the recent change in master

* Fix it test
2018-08-06 23:59:42 -07:00
Nishant Bangarwa 75c8a87ce1 Part 2 of changes for SQL Compatible Null Handling (#5958)
* Part 2 of changes for SQL Compatible Null Handling

* Review comments - break lines longer than 120 characters

* review comments

* review comments

* fix license

* fix test failure

* fix CalciteQueryTest failure

* Null Handling - Review comments

* review comments

* review comments

* fix checkstyle

* fix checkstyle

* remove unrelated change

* fix test failure

* fix failing test

* fix travis failures

* Make StringLast and StringFirst aggregators nullable and fix travis failures
2018-08-02 08:20:25 -07:00
Jonathan Wei b9c445c780
Optimize filtered aggs with interval filters in per-segment queries (#5857)
* Optimize per-segment queries

* Always optimize, add unit test

* PR comments

* Only run IntervalDimFilter optimization on __time column

* PR comments

* Checkstyle fix

* Add test for non __time column
2018-08-01 14:39:38 -07:00
Clint Wylie 297810e7a4 log correct moved count on balance instead of snapshot of currently moving (#6032) 2018-08-01 03:36:10 -07:00
Roman Leventov 0754d78a2e Prohibit Lists.newArrayList() with a single argument (#6068)
* Prohibit Lists.newArrayList() with a single argument

* Test fixes

* Add Javadoc to Node constructor
2018-07-31 20:09:10 -07:00
Gian Merlino 3aa7017975 Remove some unnecessary task storage internal APIs. (#6058)
* Remove some unnecessary task storage internal APIs.

- Remove MetadataStorageActionHandler's getInactiveStatusesSince and getActiveEntriesWithStatus.
- Remove TaskStorage's getCreatedDateTimeAndDataSource.
- Remove TaskStorageQueryAdapter's getCreatedTime, and getCreatedDateAndDataSource.
- Migrated all callers to getActiveTaskInfo and getCompletedTaskInfo.

This has one side effect: since getActiveTaskInfo (new) warns and continues when it
sees unreadable tasks, but getActiveEntriesWithStatus threw an exception when it
encountered those, it means that after this patch bad tasks will be ignored when
syncing from metadata storage rather than causing an exception to be thrown.

IMO, this is an improvement, since the most likely reason for bad tasks is either:

- A new version introduced an additional validation, and a pre-existing task doesn't
  pass it.
- You are rolling back from a newer version to an older version.

In both cases, I believe you would want to skip tasks that can't be deserialized,
rather than blocking overlord startup.

* Remove unused import.

* Fix formatting.

* Fix formatting.
2018-07-30 18:35:06 -07:00
Benedict Jin 331a0afb98 Remove redundant type parameters and enforce some other style and inspection rules (#5980)
* Various changes about druid-services module

* Patch improvements from reviewer

* Add ToArrayCallWithZeroLengthArrayArgument & ArraysAsListWithZeroOrOneArgument into inspection profile

* Fix ArraysAsListWithZeroOrOneArgument

* Fix conflict

* Fix ToArrayCallWithZeroLengthArrayArgument

* Fix AliEqualsAvoidNull

* Remove blank line

* Remove unused import clauses

* Fix code style in TopNQueryRunnerTest

* Fix conflict

* Don't use Collections.singletonList when converting the type of array type

* Add argLine into maven-surefire-plugin in druid-process module & increase the timeout value for testMoveSegment testcase

* Roll back the latest commit

* Add java.io.File#toURL() into druid-forbidden-apis

* Using Boolean.parseBoolean instead of Boolean.valueOf for CliCoordinator#isOverlord

* Add a new regexp element into stylecode xml file

* Fix style error for new regexp

* Set the level of ArraysAsListWithZeroOrOneArgument as WARNING

* Fix style error for new regexp

* Add option BY_LEVEL for ToArrayCallWithZeroLengthArrayArgument in inspection profile

* Roll back the level as ToArrayCallWithZeroLengthArrayArgument as ERROR

* Add toArray(new Object[0]) regexp into checkstyle config file & fix them

* Set the level of ArraysAsListWithZeroOrOneArgument as ERROR & Roll back the level of ToArrayCallWithZeroLengthArrayArgument as WARNING until Youtrack fix it

* Add a comment for string equals regexp in checkstyle config

* Fix code format

* Add RedundantTypeArguments as ERROR level inspection

* Fix cannot resolve symbol datasource
2018-07-27 16:56:49 -05:00
Jihoon Son 1524af703d
Fix IllegalArgumentException in TaskLockBox.syncFromStorage() (#6050) 2018-07-27 10:43:32 -07:00
kaijianding 7919e4d5df move rangeSet compare into shardspec (#5688) 2018-07-26 14:17:57 -07:00
Jihoon Son 5ee7b0cada Synchronize scheduled poll() calls in SQLMetadataSegmentManager (#6041)
Similar issue to https://github.com/apache/incubator-druid/issues/6028.
2018-07-24 22:57:30 -05:00
Roman Leventov 7d5eb0c21a Synchronize scheduled poll() calls in SQLMetadataRuleManager to prevent flakiness in SqlMetadataRuleManagerTest (#6033) 2018-07-24 12:00:48 -07:00
Surekha 414487a78e Add support to filter on datasource for active tasks (#5998)
* Add support to filter on datasource for active tasks

* Added datasource filter to sql query for active tasks
* Fixed unit tests

* Address PR comments
2018-07-19 16:33:46 -07:00
Jihoon Son 4a2df2b23a Log the full stack trace when an HTTP request fails (#6022) 2018-07-19 12:05:46 -07:00
Jihoon Son c48aa74a30 Fix NPE while handling CheckpointNotice in KafkaSupervisor (#5996)
* Fix NPE while handling CheckpointNotice

* fix code style

* Fix test

* fix test

* add a log for creating a new taskGroup

* fix backward compatibility in KafkaIOConfig
2018-07-13 17:14:57 -07:00
Clint Wylie 31c2179fe1 Coordinator fix balancer stuck (#5987)
* this will fix it

* filter destinations to not consider servers already serving segment

* fix it

* cleanup

* fix opposite day in ImmutableDruidServer.equals

* simplify
2018-07-11 20:19:11 -07:00
Clint Wylie ac194cc082 Coordinator fix exception caused by additional logging (#5988)
* fix explosion in curator load queue peon caused by additional logging, as well as annoying chatty log

* remove log message
2018-07-11 16:13:32 -07:00
Gian Merlino 04ea3c9f8c
Update license headers. (#5976)
* Update license headers.

For compliance with http://www.apache.org/legal/src-headers.html.

* More license adjustments.

* Fix mistakenly edited package line.
2018-07-11 09:55:18 -07:00
Gian Merlino 948e73da77 Extend various test timeouts. (#5978)
False failures on Travis due to spurious timeout (in turn due to noisy
neighbors) is a bigger problem than legitimate failures taking too long
to time out. So it makes sense to extend timeouts.
2018-07-10 13:02:14 -07:00
Gian Merlino 24c20b4734 Forbid slashes in datasource names. (#5937)
They are bad because datasources are used as paths on filesystems,
and slashes invariably make things get stored improperly.
2018-07-05 09:49:16 -07:00
Clint Wylie aa4987b871 change default compaction task target size from 800MB to 400MB to fall within range of what docs recommend for segment sizing (#5930) 2018-07-05 00:12:31 -07:00
Jihoon Son 4cd14e8158 Proper handling of the exceptions from auto persisting in AppenderatorImpl.add() (#5932) 2018-07-04 23:42:41 -07:00
Clint Wylie 39371b0ff8 More coordinator logging to help give context to load queue peon log messages (#5929)
* more coordinator logging to help give context to load queue peon log messages

* fix style

* more chill load queue peon log messages
2018-07-04 23:40:25 -07:00
Clint Wylie 0a472d3fa0 coordinator slight optimze load rule to skip drop if numToDrop is 0 (#5928) 2018-07-03 17:56:11 -07:00
Clint Wylie d5a3871864 Coordinator fix balance to try to move max segments instead of up to max segments (#5927)
* fix move to try to move max segments instead of "up to" max segments

* fix

* fix oops
2018-07-03 17:06:38 -07:00
Jihoon Son 1ccabab98e Fix the broken Appenderator contract in KafkaIndexTask (#5905)
* Fix broken Appenderator contract in KafkaIndexTask

* fix build

* add publishFuture

* reuse sequenceToUse if possible
2018-07-03 13:31:29 -07:00
mhshimul 867f6a9e2b Fix SQL Server select query in createInactiveStatusesSinceQuery() method. (#5901)
* Fix SQL Server select query in createInactiveStatusesSinceQuery() method.

SQL server does not support LIMIT N in select queries. Instead it has TOP N to limiting number of query results.
And TOP N is already added in the select statement as per maxNumStatuses value.

* Add parentheses for TOP in SELECT statement as SQL Servers no longer support TOP without parentheses.
2018-07-03 23:16:47 +05:30
Jihoon Son b6c957b0d2 Allow reordered segment allocation in kafka indexing service (#5805)
* Allow reordered segment allocation in kafka indexing service

* address comments

* fix a bug
2018-07-02 15:09:12 -07:00
Surekha 933b25416c Handle task deserialization failure in the tasks api (#5911)
If task payload fails to deserialize json to Java, make the task null and handle null task in OverlordResource
2018-06-29 11:57:48 -07:00
Gian Merlino a28314349c
Fix spelling of "propagate" in various places. (#5896)
One of these is a configuration parameter (introduced in #5429),
but it's never been in a release, so I think it's ok to rename it.
2018-06-25 09:18:08 -07:00
George Paraskevas 4b111929ec Fix typo lage->large , improve warning message (#5890) 2018-06-22 17:33:02 -07:00
Clint Wylie 1a7adabf57 Coordinator segment balancer max load queue fix (#5888)
* Coordinator segment balancer will now respect "maxSegmentsInNodeLoadingQueue" config

* allow moves from full load queues

* better variable names
2018-06-20 23:04:41 -07:00
Niketh Sabbineni 0982472c90 Use historical node instead of realtime for querying (#4764)
* Use historical node instead of realtime for querying

* Incorporated code review comments

* Incorporate code review comments

* Remove artifact comment

* Consider non-historical nodes as realtime
2018-06-20 22:53:56 -07:00
Surekha 8619adb5b9 Improve task retrieval APIs on Overlord (#5801)
* Add the new tasks api in overlordResource

It takes 4 optional query params
* state(pending/running/waiting/compelte)
* dataSource
* interval (applies to completed tasks)
* maxCompletedTasks (applies to completed tasks)

If all params are null, the api returns all the tasks

* Add the state to each task returned by tasks endpoint

* divide active tasks into waiting, pending or running
* Add more unit tests

* Add UNKNOWN state to TaskState

* Fix the authorization calls

* WIP: PR comments

Added new class to capture task info for caching
Other refactoring

* Refactoring : move TaskStatus class to druid-api

so it can be accessed within server
And other related classes like TaskState and TaskStatusPlus are in api

* Remove unused class and apis accessing it

* Add a separate cache for recently completed tasks

This is to mainly capture the task type from payload

* Ignore a test

* Add a RuntimeTaskState to encompass all states a task can be in

* Revert "Add a RuntimeTaskState to encompass all states a task can be in"

This reverts commit 2a527a0731.

* Fix wrong api call

* Fix and unignore tests

* Remove waiting,pending state from TaskState

* Add RunnerTaskState

* Missed the annotation runnerStatusCode

* Fix the creationTime

* Fix the createdTime and queueInsertionTime for running/active tasks
* Clean up tests

* Add javadocs

* Potentially fix the teamcity build

* Address PR comments

*Get rid of TaskInfoBuilder
*Make TaskInfoMapper static nested class
*Other changes

* fix import in MaterializedViewSupervisor after merge

* Address PR comments on

* Replace global cache with local map
* combine multiple queries into one
* Removed unused code

* Fix unit tests

Fix a bug in securedTaskStatusPlus

* Remove getRecentlyFinishedTaskStatuses method

Change TaskInfoMapper signature to add generic type

* Address PR comments

* Passed datasource as argument to be used in sql query
* Other minor fixes

* Address PR comments

*Some minor changes, rename method, spacing changes

* Add early auth check if datasource is not null

* Fix test case

* Add max limit to getRecentlyFinishedTaskInfo in HeapMemoryTaskStorage
* Add TaskLocation to Anytask object

* Address PR comments

* Fix a bug in test case causing ClassCastException
2018-06-19 11:34:59 -07:00
varaga b4b1b2a020 Provisioning support for ZooKeeper Authorization (#5701)
Review comments implemented
2018-06-15 14:02:01 -07:00
Jonathan Wei dc67b77ec2 Immediately send 401 on basic HTTP authentication failure (#5856)
* Immediately send 401 on basic HTTP authentication failure

* Add unit tests
2018-06-14 10:23:10 -07:00
Jonathan Wei 24efbb054c
Fix inefficient available segment cache population in SQLMetadataSegmentManager (#5878) 2018-06-12 18:53:30 -07:00
zhangxinyu e43e5ebbcd Materialized view implementation (#5556)
* implement materialized view

* modify code according to jihoonson's comments

* modify code according to jihoonson's comments - 2

* add documentation about materialized view

* use new HadoopTuningConfig in pr 5583

* add minDataLag and fix optimizer bug

* correct value of DEFAULT_MIN_DATA_LAG_MS

* modify code according to jihoonson's comments - 3

* use the boolean expression instead of if-else
2018-06-09 12:24:54 -07:00
awelsh93 6f0aedd6ab Fix defaultQueryTimeout (#5807)
* Fix defaultQueryTimeout

- set default timeout in query context before query fail time is evaluated

Remove unused import

* Address failing checks

* Addressing code review comments

* Removed line that was no longer used
2018-06-08 15:34:10 -07:00
Hongze Zhang cfa94b747b Update to jetty 9.4; Enable request decompression (#5624)
* Update to jetty 9.4; Enable request decompression; Add http compression config options

* Fix BadMessageException from jetty server at HttpGenerator.generateHeaders(...)
2018-06-08 14:53:08 -07:00
awelsh93 adbe22c05b Security - add anonymous authenticator (#5842)
* Anonymous authenticator that authenticates all requests and then directs them to an authorizer.

* Adding documentation

* Removed some fields from class AnonymousAuthenticator

* Updating docs
2018-06-07 10:17:54 -07:00
Jonathan Wei 684b5d18c1
Moving averages for ingestion row stats (#5748)
* Moving averages for ingestion row stats

* PR comments

* Make RowIngestionMeters extensible

* test and checkstyle fixes

* More PR comments

* Fix metrics

* Add some comments

* PR comments

* Comments
2018-06-05 09:08:57 -07:00
Michael Schnupp 33b4eb624d fix freeSpacePercent in segmentCache.locations (#5765)
* fix freeSpacePercent in segmentCache.locations

* the check should probably test the other way around
* documentation should put the option in the right place
* examples have a superfluous backslash

* add test to verify correct behavior

* switch to Path and test with jimfs

Path allows to use different filesystems.
Jimfs provides an actual (in memory) filesystem.
This also allows more complex test scenarios.

The behavior should be unchanged by this commit.

* Revert "switch to Path and test with jimfs"

This reverts commit 8b9a418d65.
2018-05-24 11:15:30 +09:00
Atul Mohan 1b9611a60e Local indexing from RDBMS (#5441)
* Local indexing from RDBMS

*  Fix content

* Remove pom changes

* Remove extraneous space

* Add tests and update documentation

* Fix comments

* Fix docs

*  Fix build related issue

*  Handle invalid strings

* Make target database independent of metadata storage

* Add firehose connector

* Fix accessibility

* Add docs

* Remove unused def

* Remove lazy instantiation of jsoniterator

* Move unused changes

* Move unused changes

* Fix build

* Make Sqlfirehose method private
2018-05-22 12:33:01 +09:00
Dylan Wylie c537ea56f6 Validate dataschema datasource (#5785)
* Validate dataschema has a datasource

* Fix tests

* Use Guava Strings.isNullOrEmpty

* Inverse nullempty check, whoops
2018-05-18 16:29:06 -07:00
Gian Merlino f2cc6ce4d5
VersionedIntervalTimeline: Optimize construction with heavily populated holders. (#5777)
* VersionedIntervalTimeline: Optimize construction with heavily populated holders.

Each time a segment is "add"ed to a timeline, "isComplete" is called on the holder
that it is added to. "isComplete" is an O(segments per chunk) operation, meaning
that adding N segments to a chunk is an O(N^2) operation. This blows up badly if
we have thousands of segments per chunk.

The patch defers the "isComplete" check until after all segments have been
inserted.

* Fix imports.
2018-05-16 09:16:59 -07:00
Jihoon Son 9dca5ec76b Simple cleanup for ThreadPoolTaskRunner and SetAndVerifyContextQueryRunner / Add ThreadPoolTaskRunnerTest (#5557)
* Simple fix for ThreadPoolTaskRunner

* fix build

* address comments

* update javadoc

* fix build

* fix test

* add dependency
2018-05-15 22:53:11 +05:30
Surekha 2f8904e25f Check against the real default of maxBytes(1/6 max mem) in AppenderatorImpl's add (#5758)
* The check for maxBytesInMemory should be >= 0 instead of > 0

* if the default value is 0, the actual check could be skipped
* fix the message for persistReasons

* Address PR comments

* if maxBytes set -1, make is Long.MAX_VAL, so we do not need to check if it's 0 or -1
* set the maxBytesTuningconfig in AppenderatorImpl constructor to avoid duplicate code

* fix the failing test cases

* Address PR comments
2018-05-09 13:41:51 -07:00
Jihoon Son c7a59394e0 Consider waiting and pending compaction tasks as well as running tasks in DruidCoordinatorSegmentCompactor (#5704)
* Consider waiting and pending compaction tasks as well as running tasks in DruidCoordinatorSegmentCompactor

* fix build

* fix logging
2018-05-08 19:03:54 -07:00
Kirill Kozlov 67d0b0ee42 Add taskType dimension to task metrics (#5664) 2018-05-07 09:42:26 -07:00
Fokko Driesprong a95ec92296 Move to the org.lz4 dependency (#5746)
The net.jpountz.lz4 moved to org.lz4
2018-05-07 08:16:45 -07:00
Slim Bouguerra 8aa8d9fa5b
Kerberos Spnego Authentication Router Issue (#5706)
* Adding decoration method to proxy servlet

Change-Id: I872f9282fb60bfa20524271535980a36a87b9621

* moving the proxy request decoration to authenticators

Change-Id: I7f94b9ff5ecf08e8abf7169b58bc410f33148448

* added docs

Change-Id: I901543e52f0faf4666bfea6256a7c05593b1ae70

* use the authentication result to decorate request

Change-Id: I052650de9cd02b4faefdbcdaf2332dd3b2966af5

* adding authenticated by name

Change-Id: I074d2933460165feeddb19352eac9bd0f96f42ca

* ensure that authenticator is not null

Change-Id: Idb58e308f90db88224a06f3759114872165b24f5

* fix types and minor bug

Change-Id: I6801d49a05d5d8324406fc0280286954eb66db10

* fix typo

Change-Id: I390b12af74f44d760d0812a519125fbf0df4e97b

* use actual type names

Change-Id: I62c3ee763363781e52809ec912aafd50b8486b8e

* set authenitcatedBy to null for AutheticationResults created by
Escalator.

Change-Id: I4a675c372f59ebd8a8d19c61b85a1e4bf227a8ba
2018-05-05 20:33:51 -07:00
kaijianding c12c16385e support throw duplcate row during realtime ingestion in RealtimePlumber (#5693) 2018-05-04 10:12:25 -07:00
Stuart McLean c2b5e5ec95 Default caffeine cache size (#5738)
* add default caffeine cache size based on runtime Xmx or max 1GB

* update docs for caffeine cache

* fix formatting

* test caffeine size should never be less than 0

* set caffeine max default size to 1G not 1M

* fix caffeine cache tests
2018-05-04 09:29:11 -07:00
Surekha 13c616ba24 'maxBytesInMemory' tuningConfig introduced for ingestion tasks (#5583)
* This commit introduces a new tuning config called 'maxBytesInMemory' for ingestion tasks

Currently a config called 'maxRowsInMemory' is present which affects how much memory gets
used for indexing.If this value is not optimal for your JVM heap size, it could lead
to OutOfMemoryError sometimes. A lower value will lead to frequent persists which might
be bad for query performance and a higher value will limit number of persists but require
more jvm heap space and could lead to OOM.
'maxBytesInMemory' is an attempt to solve this problem. It limits the total number of bytes
kept in memory before persisting.

 * The default value is 1/3(Runtime.maxMemory())
 * To maintain the current behaviour set 'maxBytesInMemory' to -1
 * If both 'maxRowsInMemory' and 'maxBytesInMemory' are present, both of them
   will be respected i.e. the first one to go above threshold will trigger persist

* Fix check style and remove a comment

* Add overlord unsecured paths to coordinator when using combined service (#5579)

* Add overlord unsecured paths to coordinator when using combined service

* PR comment

* More error reporting and stats for ingestion tasks (#5418)

* Add more indexing task status and error reporting

* PR comments, add support in AppenderatorDriverRealtimeIndexTask

* Use TaskReport instead of metrics/context

* Fix tests

* Use TaskReport uploads

* Refactor fire department metrics retrieval

* Refactor input row serde in hadoop task

* Refactor hadoop task loader names

* Truncate error message in TaskStatus, add errorMsg to task report

* PR comments

* Allow getDomain to return disjointed intervals (#5570)

* Allow getDomain to return disjointed intervals

* Indentation issues

* Adding feature thetaSketchConstant to do some set operation in PostAgg (#5551)

* Adding feature thetaSketchConstant to do some set operation in PostAggregator

* Updated review comments for PR #5551 - Adding thetaSketchConstant

* Fixed CI build issue

* Updated review comments 2 for PR #5551 - Adding thetaSketchConstant

* Fix taskDuration docs for KafkaIndexingService (#5572)

* With incremental handoff the changed line is no longer true.

* Add doc for automatic pendingSegments (#5565)

* Add missing doc for automatic pendingSegments

* address comments

* Fix indexTask to respect forceExtendableShardSpecs (#5509)

* Fix indexTask to respect forceExtendableShardSpecs

* add comments

* Deprecate spark2 profile in pom.xml (#5581)

Deprecated due to https://github.com/druid-io/druid/pull/5382

* CompressionUtils: Add support for decompressing xz, bz2, zip. (#5586)

Also switch various firehoses to the new method.

Fixes #5585.

* This commit introduces a new tuning config called 'maxBytesInMemory' for ingestion tasks

Currently a config called 'maxRowsInMemory' is present which affects how much memory gets
used for indexing.If this value is not optimal for your JVM heap size, it could lead
to OutOfMemoryError sometimes. A lower value will lead to frequent persists which might
be bad for query performance and a higher value will limit number of persists but require
more jvm heap space and could lead to OOM.
'maxBytesInMemory' is an attempt to solve this problem. It limits the total number of bytes
kept in memory before persisting.

 * The default value is 1/3(Runtime.maxMemory())
 * To maintain the current behaviour set 'maxBytesInMemory' to -1
 * If both 'maxRowsInMemory' and 'maxBytesInMemory' are present, both of them
   will be respected i.e. the first one to go above threshold will trigger persist

* Address code review comments

* Fix the coding style according to druid conventions
* Add more javadocs
* Rename some variables/methods
* Other minor issues

* Address more code review comments

* Some refactoring to put defaults in IndexTaskUtils
* Added check for maxBytesInMemory in AppenderatorImpl
* Decrement bytes in abandonSegment
* Test unit test for multiple sinks in single appenderator
* Fix some merge conflicts after rebase

* Fix some style checks

* Merge conflicts

* Fix failing tests

Add back check for 0 maxBytesInMemory in OnHeapIncrementalIndex

* Address PR comments

* Put defaults for maxRows and maxBytes in TuningConfig
* Change/add javadocs
* Refactoring and renaming some variables/methods

* Fix TeamCity inspection warnings

* Added maxBytesInMemory config to HadoopTuningConfig

* Updated the docs and examples

* Added maxBytesInMemory config in docs
* Removed references to maxRowsInMemory under tuningConfig in examples

* Set maxBytesInMemory to 0 until used

Set the maxBytesInMemory to 0 if user does not set it as part of tuningConfing
and set to part of max jvm memory when ingestion task starts

* Update toString in KafkaSupervisorTuningConfig

* Use correct maxBytesInMemory value in AppenderatorImpl

* Update DEFAULT_MAX_BYTES_IN_MEMORY to 1/6 max jvm memory

Experimenting with various defaults, 1/3 jvm memory causes OOM

* Update docs to correct maxBytesInMemory default value

* Minor to rename and add comment

* Add more details in docs

* Address new PR comments

* Address PR comments

* Fix spelling typo
2018-05-03 16:25:58 -07:00
Gian Merlino df01998213 SegmentLoadDropHandler: Fix deadlock when segments have errors loading on startup. (#5735)
The "lock" object was used to synchronize start/stop as well as synchronize removals
from segmentsToDelete (when a segment is done dropping). This could cause a deadlock
if a segment-load throws an exception during loadLocalCache. loadLocalCache is run
by start() while it holds the lock, but then it spawns loading threads, and those
threads will try to acquire the "segmentsToDelete" lock if they want to drop a corrupt
segments.

I don't see any reason for these two locks to be the same lock, so I split them.
2018-05-03 09:59:01 -07:00
Jihoon Son 2c8296f94d Fix Appenderator.push() to commit the metadata of all segments (#5730)
* Remove persist from Appenderator

* fix javadoc
2018-05-02 13:17:54 -07:00
Jihoon Son d4311b4a5a Support enablePathStyleAccess, disableChunkedEncoding, and forceGlobalBucketAccessEnabled for aws client (#5702)
* Support enablePathStyleAccess and disableChunkedEncoding for aws client

* add an option for forceGlobalBucketAccessEnabled

* add missing doc
2018-05-02 10:45:38 -07:00
David Lim 8ec2d2fe18 Use unique segment paths for Kafka indexing (#5692)
* support unique segment file paths

* forbiddenapis

* code review changes

* code review changes

* code review changes

* checkstyle fix
2018-04-29 21:59:48 -07:00
Roman Leventov 9be000758d Refactor index merging, replace Rowboats with RowIterators and RowPointers (#5335)
* Refactor index merging, replace Rowboats with RowIterators and RowPointers

* Add javadocs

* Fix a bug in QueryableIndexIndexableAdapter

* Fixes

* Remove unused declarations

* Remove unused GenericColumn.isNull() method

* Fix test

* Address comments

* Rearrange some code in MergingRowIterator for more clarity

* Self-review

* Fix style

* Improve docs

* Fix docs

* Rename IndexMergerV9.writeDimValueAndSetupDimConversion to setUpDimConversion()

* Update Javadocs

* Minor fixes

* Doc fixes, more code comments, cleanup of RowCombiningTimeAndDimsIterator

* Fix doc link
2018-04-27 17:34:32 -07:00
David Lim 55b003e5e8 Fix loadstatus?full double counting expected segments (#5667)
* fix loadstatus?full double counting expected segments

* remove possible flakiness from Thread.sleep() in test
2018-04-24 01:11:16 +05:30
Roman Leventov a3a9ada843 Add GenericWhitespace checkstyle check (#5668) 2018-04-24 01:09:14 +05:30
Jihoon Son ca3f833426 Fix coordinator's dataSource api with full parameter (#5662)
* Fix coordinator's dataSource api with full parameter

* address comment

* Add a constructor for json serde and fix result order

* Change to immutableSortedMap

* Revert immutableSortedMap to treeMap
2018-04-19 17:41:53 -07:00
Kirill Kozlov a7ba2bf275 Detailed error message when unable to create temp dir (#5648) 2018-04-17 15:12:46 -07:00
Jonathan Wei d0b66a6af5 Fix HTTP OPTIONS request auth handling (#5638)
* Fix HTTP OPTIONS request auth handling

* PR comment

* More PR comments

* Fix

* PR comment
2018-04-16 18:09:56 -07:00
Jonathan Wei 882b172318
Revert "Fix HTTP OPTIONS request auth handling (#5615)" (#5637)
This reverts commit df51a7bcb7.
2018-04-12 16:43:54 -07:00
Jonathan Wei e91add6843
Fix coordinator loadStatus performance (#5632)
* Optimize coordinator loadStatus

* Add comment

* Fix teamcity

* Checkstyle

* More checkstyle

* Checkstyle
2018-04-12 15:07:52 -07:00
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