Commit Graph

2103 Commits

Author SHA1 Message Date
Suneet Saldanha 6b0c2e8996
CompactionTask throws exception on conflicting segmentGranularity (#10996)
* CompactionTask throws exception on conflicting segmentGranularity

* add comment
2021-03-16 12:51:50 -07:00
Maytas Monsereenusorn ed91a2bb38
Fix Kinesis should not increment throwAway count on EOS record (#10976)
* fix Kinesis increament throwAway on EOS record

* fix checkstyle

* fix IT

* fix test

* fix IT

* fix IT

* fix IT

* fix IT
2021-03-11 22:04:58 -08:00
zhangyue19921010 bddacbb1c3
Dynamic auto scale Kafka-Stream ingest tasks (#10524)
* druid task auto scale based on kafka lag

* fix kafkaSupervisorIOConfig and KinesisSupervisorIOConfig

* druid task auto scale based on kafka lag

* fix kafkaSupervisorIOConfig and KinesisSupervisorIOConfig

* test dynamic auto scale done

* auto scale tasks tested on prd cluster

* auto scale tasks tested on prd cluster

* modify code style to solve 29055.10 29055.9 29055.17 29055.18 29055.19 29055.20

* rename test fiel function

* change codes and add docs based on capistrant reviewed

* midify test docs

* modify docs

* modify docs

* modify docs

* merge from master

* Extract the autoScale logic out of SeekableStreamSupervisor to minimize putting more stuff inside there &&  Make autoscaling algorithm configurable and scalable.

* fix ci failed

* revert msic.xml

* add uts to test autoscaler create && scale out/in and kafka ingest with scale enable

* add more uts

* fix inner class check

* add IT for kafka ingestion with autoscaler

* add new IT in groups=kafka-index named testKafkaIndexDataWithWithAutoscaler

* review change

* code review

* remove unused imports

* fix NLP

* fix docs and UTs

* revert misc.xml

* use jackson to build autoScaleConfig with default values

* add uts

* use jackson to init AutoScalerConfig in IOConfig instead of Map<>

* autoscalerConfig interface and provide a defaultAutoScalerConfig

* modify uts

* modify docs

* fix checkstyle

* revert misc.xml

* modify uts

* reviewed code change

* reviewed code change

* code reviewed

* code review

* log changed

* do StringUtils.encodeForFormat when create allocationExec

* code review && limit taskCountMax to partitionNumbers

* modify docs

* code review

Co-authored-by: yuezhang <yuezhang@freewheel.tv>
2021-03-06 14:36:52 +05:30
Maytas Monsereenusorn b7b0ee8362
Add query granularity to compaction task (#10900)
* add query granularity to compaction task

* fix checkstyle

* fix checkstyle

* fix test

* fix test

* add tests

* fix test

* fix test

* cleanup

* rename class

* fix test

* fix test

* add test

* fix test
2021-03-02 11:23:52 -08:00
Gian Merlino 07902f607b
Granularity: Introduce primitive-typed bucketStart, increment methods. (#10904)
* Granularity: Introduce primitive-typed bucketStart, increment methods.

Saves creation of unnecessary DateTime objects in timestamp_floor and
timestamp_ceil expressions.

* Fix style.

* Amp up the test coverage.
2021-02-25 07:59:20 -08:00
Agustin Gonzalez eabad0fb35
Keep query granularity of compacted segments after compaction (#10856)
* Keep query granularity of compacted segments after compaction

* Protect against null isRollup

* Fix bugspot check RC_REF_COMPARISON_BAD_PRACTICE_BOOLEAN & edit an existing comment

* Make sure that NONE is also included when comparing for the finer granularity

* Update integration test check for segment size due to query granularity propagation affecting size

* Minor code cleanup

* Added functional test to verify queryGranlarity after compaction

* Minor style fix

* Update unit tests
2021-02-18 01:35:10 -08:00
Maytas Monsereenusorn 6541178c21
Support segmentGranularity for auto-compaction (#10843)
* Support segmentGranularity for auto-compaction

* Support segmentGranularity for auto-compaction

* Support segmentGranularity for auto-compaction

* Support segmentGranularity for auto-compaction

* resolve conflict

* Support segmentGranularity for auto-compaction

* Support segmentGranularity for auto-compaction

* fix tests

* fix more tests

* fix checkstyle

* add unit tests

* fix checkstyle

* fix checkstyle

* fix checkstyle

* add unit tests

* add integration tests

* fix checkstyle

* fix checkstyle

* fix failing tests

* address comments

* address comments

* fix tests

* fix tests

* fix test

* fix test

* fix test

* fix test

* fix test

* fix test

* fix test

* fix test
2021-02-12 03:03:20 -08:00
Egor Riashin 5d8a69ecf9
fixed input source sampler buildReader exp (#10651)
* fixed input source sampler buildReader exp

* sampler exception unit test

* styling

Co-authored-by: egor-ryashin <egor.ryashin@rilldata.com>
2021-02-05 13:33:55 -08:00
Abhishek Agarwal 96d26e5338
Fix kinesis ingestion bugs (#10761)
* add offsetFetchPeriod to kinesis ingestion doc

* Remove jackson dependencies from extensions

* Use fixed delay for lag collection

* Metrics reset after finishing processing

* comments

* Broaden the list of exceptions to retry for

* Unit tests

* Add more tests

* Refactoring

* re-order metrics

* Doc suggestions

Co-authored-by: Charles Smith <38529548+techdocsmith@users.noreply.github.com>

* Add tests

Co-authored-by: Charles Smith <38529548+techdocsmith@users.noreply.github.com>
2021-02-05 02:49:58 -08:00
Jihoon Son 3f8f00a231
Fix CVE-2021-25646 (#10818) 2021-02-04 11:21:43 -08:00
Agustin Gonzalez 0e4750bac2
Granularity interval materialization (#10742)
* Prevent interval materialization for UniformGranularitySpec inside the overlord

* Change API of bucketIntervals in GranularitySpec to return an Iterable<Interval>

* Javadoc update, respect inputIntervals contract

* Eliminate dependency on wrappedspec (i.e. ArbitraryGranularity) in UniformGranularitySpec

* Added one boundary condition test to UniformGranularityTest and fixed Travis forbidden method errors in IntervalsByGranularity

* Fix Travis style & other checks

* Refactor TreeSet to facilitate re-use in UniformGranularitySpec

* Make sure intervals are unique when there is no segment granularity

* Style/bugspot fixes...

* More travis checks

* Add condensedIntervals method to GranularitySpec and pass it as needed to the lock method

* Style & PR feedback

* Fixed failing test

* Fixed bug in IntervalsByGranularity iterator that it would return repeated elements (see added unit tests that were broken before this change)

* Refactor so that we can get the condensed buckets without materializing the intervals

* Get rid of GranularitySpec::condensedInputIntervals ... not needed

* Travis failures fixes

* Travis checkstyle fix

* Edited/added javadoc comments and a method name (code review feedback)

* Fixed jacoco coverage by moving class and adding more coverage

* Avoid materializing the condensed intervals when locking

* Deal with overlapping intervals

* Remove code and use library code instead

* Refactor intervals by granularity using the FluentIterable, add sanity checks

* Change !hasNext() to inputIntervals().isEmpty()

* Remove redundant lambda

* Use materialized intervals here since this is outside the overlord (for performance)

* Name refactor to reflect the fact that bucket intervals are sorted.

* Style fixes

* Removed redundant method and have condensedIntervalIterator throw IAE when element is null for consistency with other methods in this class (as well that null interval when condensing does not make sense)

* Remove forbidden api

* Move helper class inside common base class to reduce public space pollution
2021-01-29 06:02:10 -08:00
Abhishek Agarwal 0080e333cc
Fix cardinality estimation (#10762)
* Fix cardinality estimation

* Add unit test

* code coverage

* fix typo
2021-01-28 15:06:10 -08:00
Maytas Monsereenusorn a46d561bd7
Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead (#10740)
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* fix checkstyle

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* fix test

* fix test

* add log

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* address comments

* fix checkstyle

* fix checkstyle

* add config to skip overhead memory calculation

* add test for the skipBytesInMemoryOverheadCheck config

* add docs

* fix checkstyle

* fix checkstyle

* fix spelling

* address comments

* fix travis

* address comments
2021-01-27 00:34:56 -08:00
zhangyue19921010 2590ad4f67
Historical unloads damaged segments automatically when lazy on start. (#10688)
* ready to test

* tested on dev cluster

* tested

* code review

* add UTs

* add UTs

* ut passed

* ut passed

* opti imports

* done

* done

* fix checkstyle

* modify uts

* modify logs

* changing the package of SegmentLazyLoadFailCallback.java to org.apache.druid.segment

* merge from master

* modify import orders

* merge from master

* merge from master

* modify logs

* modify docs

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

Co-authored-by: yuezhang <yuezhang@freewheel.tv>
2021-01-16 19:53:30 -08:00
Jihoon Son 95065bdf1a
Bump dev version to 0.22.0-SNAPSHOT (#10759) 2021-01-15 13:16:23 -08:00
Jihoon Son ca32652932
Fix potential deadlock in batch ingestion (#10736)
* Fix potential deadlock in batch ingestion

* fix checkstyle and comment

* this is better
2021-01-12 12:50:45 -08:00
Xavier Léauté 118b50195e
Introduce KafkaRecordEntity to support Kafka headers in InputFormats (#10730)
Today Kafka message support in streaming indexing tasks is limited to
message values, and does not provide a way to expose Kafka headers,
timestamps, or keys, which may be of interest to more specialized
Druid input formats. For instance, Kafka headers may be used to indicate
payload format/encoding or additional metadata, and timestamps are often
omitted from values in Kafka streams applications, since they are
included in the record.

This change proposes to introduce KafkaRecordEntity as InputEntity,
which would give input formats full access to the underlying Kafka record,
including headers, key, timestamps. It would also open access to low-level
information such as topic, partition, offset if needed.

KafkaEntity is a subclass of ByteEntity for backwards compatibility with
existing input formats, and to avoid introducing unnecessary complexity
for Kinesis indexing tasks.
2021-01-08 16:04:37 -08:00
Liran Funaro 08ab82f55c
IncrementalIndex Tests and Benchmarks Parametrization (#10593)
* Remove redundant IncrementalIndex.Builder

* Parametrize incremental index tests and benchmarks

- Reveal and fix a bug in OffheapIncrementalIndex

* Fix forbiddenapis error: Forbidden method invocation: java.lang.String#format(java.lang.String,java.lang.Object[]) [Uses default locale]

* Fix Intellij errors: declared exception is never thrown

* Add documentation and validate before closing objects on tearDown.

* Add documentation to OffheapIncrementalIndexTestSpec

* Doc corrections and minor changes.

* Add logging for generated rows.

* Refactor new tests/benchmarks.

* Improve IncrementalIndexCreator documentation

* Add required tests for DataGenerator

* Revert "rollupOpportunity" to be a string
2021-01-07 22:18:47 -08:00
Jonathan Wei 68bb038b31
Multiphase segment merge for IndexMergerV9 (#10689)
* Multiphase merge for IndexMergerV9

* JSON fix

* Cleanup temp files

* Docs

* Address logging and add IT

* Fix spelling and test unloader datasource name
2021-01-05 22:19:09 -08:00
Jihoon Son ea2d51d61f
Better error message for compaction task when it sees no segments for compaction (#10728) 2021-01-05 16:49:57 -08:00
Jonathan Wei 769c21cc87
Add sample method to IndexingServiceClient (#10729)
* Add sample method to IndexingServiceClient

* Add unit test

* Fix LGTM
2021-01-05 15:02:44 -08:00
Clint Wylie da0eabaa01
integration test for coordinator and overlord leadership client (#10680)
* integration test for coordinator and overlord leadership, added sys.servers is_leader column

* docs

* remove not needed

* fix comments

* fix compile heh

* oof

* revert unintended

* fix tests, split out docker-compose file selection from starting cluster, use docker-compose down to stop cluster

* fixes

* style

* dang

* heh

* scripts are hard

* fix spelling

* fix thing that must not matter since was already wrong ip, log when test fails

* needs more heap

* fix merge

* less aggro
2020-12-17 22:50:12 -08:00
Gian Merlino 96a387d972
Fixes and tests related to the Indexer process. (#10631)
* Fixes and tests related to the Indexer process.

Three bugs fixed:

1) Indexers would not announce themselves as segment servers if they
   did not have storage locations defined. This used to work, but was
   broken in #9971. Fixed this by adding an "isSegmentServer" method
   to ServerType and updating SegmentLoadDropHandler to always announce
   if this method returns true.

2) Certain batch task types were written in a way that assumed "isReady"
   would be called before "run", which is not guaranteed. In particular,
   they relied on it in order to initialize "taskLockHelper". Fixed this
   by updating AbstractBatchIndexTask to ensure "isReady" is called
   before "run" for these tasks.

3) UnifiedIndexerAppenderatorsManager did not properly handle complex
   datasources. Introduced DataSourceAnalysis in order to fix this.

Test changes:

1) Add a new "docker-compose.cli-indexer.yml" config that spins up an
   Indexer instead of a MiddleManager.

2) Introduce a "USE_INDEXER" environment variable that determines if
   docker-compose will start up an Indexer or a MiddleManager.

3) Duplicate all the jdk8 tests and run them in both MiddleManager and
   Indexer mode.

4) Various adjustments to encourage fail-fast errors in the Docker
   build scripts.

5) Various adjustments to speed up integration tests and reduce memory
   usage.

6) Add another Mac-specific approach to determining a machine's own IP.
   This was useful on my development machine.

7) Update segment-count check in ITCompactionTaskTest to eliminate a
   race condition (it was looking for 6 segments, which only exist
   together briefly, until the older 4 are marked unused).

Javadoc updates:

1) AbstractBatchIndexTask: Added javadocs to determineLockGranularityXXX
   that make it clear when taskLockHelper will be initialized as a side
   effect. (Related to the second bug above.)

2) Task: Clarified that "isReady" is not guaranteed to be called before
   "run". It was already implied, but now it's explicit.

3) ZkCoordinator: Clarified deprecation message.

4) DataSegmentServerAnnouncer: Clarified deprecation message.

* Fix stop_cluster script.

* Fix sanity check in script.

* Fix hashbang lines.

* Test and doc adjustments.

* Additional tests, and adjustments for tests.

* Split ITs back out.

* Revert change to druid_coordinator_period_indexingPeriod.

* Set Indexer capacity to match MM.

* Bump up Historical memory.

* Bump down coordinator, overlord memory.

* Bump up Broker memory.
2020-12-08 16:02:26 -08:00
Gian Merlino 9acab0b646
DruidInputSource: Sort segments by ID before grouping into splits. (#10646)
This is useful because it groups up segments for the same time chunk
into the same splits, which in turn is useful because it minimizes the
number of time chunks that each task will have to deal with.
2020-12-07 13:48:24 -08:00
egor-ryashin f46cc4faaf Revert "fixed input source sampler buildReader exp"
This reverts commit e688db8
2020-12-07 18:34:59 +03:00
egor-ryashin e688db8503 fixed input source sampler buildReader exp 2020-12-07 18:28:25 +03:00
Gian Merlino b7641f644c
Two fixes related to encoding of % symbols. (#10645)
* Two fixes related to encoding of % symbols.

1) TaskResourceFilter: Don't double-decode task ids. request.getPathSegments()
   returns already-decoded strings. Applying StringUtils.urlDecode on
   top of that causes erroneous behavior with '%' characters.

2) Update various ThreadFactoryBuilder name formats to escape '%'
   characters. This fixes situations where substrings starting with '%'
   are erroneously treated as format specifiers.

ITs are updated to include a '%' in extra.datasource.name.suffix.

* Avoid String.replace.

* Work around surefire bug.

* Fix xml encoding.

* Another try at the proper encoding.

* Give up on the emojis.

* Less ambitious testing.

* Fix an additional problem.

* Adjust encodeForFormat to return null if the input is null.
2020-12-06 22:35:11 -08:00
Gian Merlino 17f39ab91e
Fix misspellings in druid-forbidden-apis. (#10634)
These caused certain APIs to not actually be properly forbidden.

Also removed two MoreExecutors entries for methods that don't exist in
our version of Guava.
2020-12-05 15:26:57 -08:00
Jihoon Son 7462b0b953
Allow missing intervals for Parallel task with hash/range partitioning (#10592)
* Allow missing intervals for Parallel task

* fix row filter

* fix tests

* fix log
2020-11-25 14:50:22 -08:00
frank chen fe693a4f01
Improve doc and exception message for invalid user configurations (#10598)
* improve doc and exception message

* add spelling check rules and remove unused import

* add a test to improve test coverage
2020-11-23 15:03:13 -08:00
Lucas Capistrant 3447934a75
Ensure Krb auth before killing YARN apps in graceful shutdown (#9785) 2020-11-16 09:59:14 -08:00
frank chen e83d5cb59e
Fix ingestion failure of pretty-formatted JSON message (#10383)
* support multi-line text

* add test cases

* split json text into lines case by case

* improve exception handle

* fix CI

* use IntermediateRowParsingReader as base of JsonReader

* update doc

* ignore the non-immutable field in test case

* add more test cases

* mark `lineSplittable` as final

* fix testcases

* fix doc

* add a test case for SqlReader

* return all raw columns when exception occurs

* fix CI

* fix test cases

* resolve review comments

* handle ParseException returned by index.add

* apply Iterables.getOnlyElement

* fix CI

* fix test cases

* improve code in more graceful way

* fix test cases

* fix test cases

* add a test case to check multiple json string in one text block

* fix inspection check
2020-11-13 13:59:23 -08:00
Himanshu ee136303bb
optionally disable all of hardcoded zookeeper use (#9507)
* optionally disable all of hardcoded zookeeper use

* fix DruidCoordinatorTest compilation

* fix test in DruidCoordinatorTest

* fix strict compilation

Co-authored-by: Himanshu Gupta <fill email>
2020-10-26 22:35:59 -07:00
Liran Funaro f3a2903218
Configurable Index Type (#10335)
* Introduce a Configurable Index Type

* Change to @UnstableApi

* Add AppendableIndexSpecTest

* Update doc

* Add spelling exception

* Add tests coverage

* Revert some of the changes to reduce diff

* Minor fixes

* Update getMaxBytesInMemoryOrDefault() comment

* Fix typo, remove redundant interface

* Remove off-heap spec (postponed to a later PR)

* Add javadocs to AppendableIndexSpec

* Describe testCreateTask()

* Add tests for AppendableIndexSpec within TuningConfig

* Modify hashCode() to conform with equals()

* Add comment where building incremental-index

* Add "EqualsVerifier" tests

* Revert some of the API back to AppenderatorConfig

* Don't use multi-line comments

* Remove knob documentation (deferred)
2020-10-23 18:34:26 -07:00
Jihoon Son ad437dd655
Add shuffle metrics for parallel indexing (#10359)
* Add shuffle metrics for parallel indexing

* javadoc and concurrency test

* concurrency

* fix javadoc

* Feature flag

* doc

* fix doc and add a test

* checkstyle

* add tests

* fix build and address comments
2020-10-10 19:35:17 -07:00
Jihoon Son e9e7d82714
Fix compaction task slot computation in auto compaction (#10479)
* Fix compaction task slot computation in auto compaction

* add tests for task counting
2020-10-06 21:56:03 -07:00
Jonathan Wei 65c0d64676
Update version to 0.21.0-SNAPSHOT (#10450)
* [maven-release-plugin] prepare release druid-0.21.0

* [maven-release-plugin] prepare for next development iteration

* Update web-console versions
2020-10-03 16:08:34 -07:00
Abhishek Agarwal e282ab5695
Fix the task id creation in CompactionTask (#10445)
* Fix the task id creation in CompactionTask

* review comments

* Ignore test for range partitioning and segment lock
2020-10-01 15:02:04 -07:00
Mainak Ghosh 8168e14e92
Adding task slot count metrics to Druid Overlord (#10379)
* Adding more worker metrics to Druid Overlord

* Changing the nomenclature from worker to peon as that represents the metrics that we want to monitor better

* Few more instance of worker usage replaced with peon

* Modifying the peon idle count logic to only use eligible workers available capacity

* Changing the naming to task slot count instead of peon

* Adding some unit test coverage for the new test runner apis

* Addressing Review Comments

* Modifying the TaskSlotCountStatsProvider apis so that overlords which are not leader do not emit these metrics

* Fixing the spelling issue in the docs

* Setting the annotation Nullable on the TaskSlotCountStatsProvider methods
2020-09-28 23:50:38 -07:00
Jihoon Son 0cc9eb4903
Store hash partition function in dataSegment and allow segment pruning only when hash partition function is provided (#10288)
* Store hash partition function in dataSegment and allow segment pruning only when hash partition function is provided

* query context

* fix tests; add more test

* javadoc

* docs and more tests

* remove default and hadoop tests

* consistent name and fix javadoc

* spelling and field name

* default function for partitionsSpec

* other comments

* address comments

* fix tests and spelling

* test

* doc
2020-09-24 16:32:56 -07:00
Jonathan Wei cb30b1fe23
Automatically determine numShards for parallel ingestion hash partitioning (#10419)
* Automatically determine numShards for parallel ingestion hash partitioning

* Fix inspection, tests, coverage

* Docs and some PR comments

* Adjust locking

* Use HllSketch instead of HyperLogLogCollector

* Fix tests

* Address some PR comments

* Fix granularity bug

* Small doc fix
2020-09-24 13:47:53 -07:00
Maytas Monsereenusorn 72f1b55f56
Add last_compaction_state to sys.segments table (#10413)
* Add is_compacted to sys.segments table

* change is_compacted to last_compaction_state

* fix tests

* fix tests

* address comments
2020-09-23 15:29:36 -07:00
Jihoon Son 8f14ac814e
More structured way to handle parse exceptions (#10336)
* More structured way to handle parse exceptions

* checkstyle; add more tests

* forbidden api; test

* address comment; new test

* address review comments

* javadoc for parseException; remove redundant parseException in streaming ingestion

* fix tests

* unnecessary catch

* unused imports

* appenderator test

* unused import
2020-09-11 16:31:10 -07:00
Gian Merlino 8ab1979304
Remove implied profanity from error messages. (#10270)
i.e. WTF, WTH.
2020-08-28 11:38:50 -07:00
Jihoon Son f82fd22fa7
Move tools for indexing to TaskToolbox instead of injecting them in constructor (#10308)
* Move tools for indexing to TaskToolbox instead of injecting them in constructor

* oops, other changes

* fix test

* unnecessary new file

* fix test

* fix build
2020-08-26 17:08:12 -07:00
Gian Merlino 21703d81ac
Fix handling of 'join' on top of 'union' datasources. (#10318)
* Fix handling of 'join' on top of 'union' datasources.

The problem is that unions are typically rewritten into a series of
individual queries on the underlying tables, but this isn't done when
the union is wrapped in a join.

The main changes are in UnionQueryRunner:

1) Replace an instanceof UnionQueryRunner check with DataSourceAnalysis.
2) Replace a "query.withDataSource" call with a new function, "Queries.withBaseDataSource".

Together, these enable UnionQueryRunner to "see through" a join.

* Tests.

* Adjust heap sizes for integration tests.

* Different approach, more tests.

* Tweak.

* Styling.
2020-08-26 14:23:54 -07:00
Jihoon Son b9ff3483ac
Add support for all partitioing schemes for auto compaction (#10307)
* Add support for all partitioing schemes for auto compaction

* annotate last compaction state for multi phase parallel indexing

* fix build and tests

* test

* better home
2020-08-26 13:19:18 -07:00
Abhishek Agarwal d4ac62f284
Handle internal kinesis sequence numbers when reporting lag (#10315)
* Handle internal kinesis sequence numbers when reporting lag

* add unit test
2020-08-26 11:27:37 -07:00
Clint Wylie ab60661008
refactor internal type system (#9638)
* better type tracking: add typed postaggs, finalized types for agg factories

* more javadoc

* adjustments

* transition to getTypeName to be used exclusively for complex types

* remove unused fn

* adjust

* more better

* rename getTypeName to getComplexTypeName

* setup expression post agg for type inference existing

* more javadocs

* fixup

* oops

* more test

* more test

* more comments/javadoc

* nulls

* explicitly handle only numeric and complex aggregators for incremental index

* checkstyle

* more tests

* adjust

* more tests to showcase difference in behavior

* timeseries longsum array
2020-08-26 10:53:44 -07:00
Jihoon Son b5b3e6ecce
Add maxNumFiles to splitHintSpec (#10243)
* Add maxNumFiles to splitHintSpec

* missing link

* fix build failure; use maxNumFiles for integration tests

* spelling

* lower default

* Update docs/ingestion/native-batch.md

Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>

* address comments; change default maxSplitSize

* spelling

* typos and doc

* same change for segments splitHintSpec

* fix build

* fix build

Co-authored-by: Abhishek Agarwal <1477457+abhishekagarwal87@users.noreply.github.com>
2020-08-21 09:43:58 -07:00
Jihoon Son 9a81740281
Don't log the entire task spec (#10278)
* Don't log the entire task spec

* fix lgtm

* fix serde

* address comments and add tests

* fix tests

* remove unnecessary codes
2020-08-18 11:03:13 -07:00
Gian Merlino 6cca7242de
Add "offset" parameter to the Scan query. (#10233)
* Add "offset" parameter to the Scan query.

It works by doing the query as normal and then throwing away the first
"offset" number of rows on the broker.

* Fix constructor call.

* Fix up JSONs.

* Fix call to ScanQuery.

* Doc update.

* Fix javadocs.

* Spotbugs, LGTM suppressions.

* Javadocs.

* Fix suppression.

* Stabilize Scan query result order, add tests.

* Update LGTM comment.

* Fixup.

* Test different batch sizes too.

* Nicer tests.

* Fix comment.
2020-08-13 14:56:24 -07:00
frank chen 646fa84d04
Support unit on byte-related properties (#10203)
* support unit suffix on byte-related properties

* add doc

* change default value of byte-related properites in example files

* fix coding style

* fix doc

* fix CI

* suppress spelling errors

* improve code according to comments

* rename Bytes to HumanReadableBytes

* add getBytesInInt to get value safely

* improve doc

* fix problem reported by CI

* fix problem reported by CI

* resolve code review comments

* improve error message

* improve code & doc according to comments

* fix CI problem

* improve doc

* suppress spelling check errors
2020-07-31 09:58:48 +08:00
Jihoon Son 6fdce36e41
Add integration tests for query retry on missing segments (#10171)
* Add integration tests for query retry on missing segments

* add missing dependencies; fix travis conf

* address comments

* Integration tests extension

* remove unused dependency

* remove druid_main

* fix java agent port
2020-07-22 22:30:35 -07:00
Suneet Saldanha e6c9142129
Add validation for authenticator and authorizer name (#10106)
* Add validation for authorizer name

* fix deps

* add javadocs

* Do not use resource filters

* Fix BasicAuthenticatorResource as well

* Add integration tests

* fix test

* fix
2020-07-13 21:15:54 -07:00
Gian Merlino 11c0da8097
Add availability and consistency docs. (#10149)
* Add availability and consistency docs.

Describes transactional ingestion and atomic replacement. Also, this patch
deletes some bad advice from the javadocs for SegmentTransactionalInsertAction.

* Fix missing word.
2020-07-07 15:22:52 -07:00
Parag Jain 98ac7dfeff
mask secrets in MM task command log (#10128)
* mask secrets in MM task command log

* unit test for masked iterator

* checkstyle fix
2020-07-07 10:25:15 -07:00
Clint Wylie c86e7ce30b
bump version to 0.20.0-SNAPSHOT (#10124) 2020-07-06 15:08:32 -07:00
Suneet Saldanha 363d0d86be
QueryCountStatsMonitor can be injected in the Peon (#10092)
* QueryCountStatsMonitor can be injected in the Peon

This change fixes a dependency injection bug where there is a circular
dependency on getting the MonitorScheduler when a user configures the
QueryCountStatsMonitor to be used.

* fix tests

* Actually fix the tests this time
2020-06-29 21:03:07 -07:00
Chi Cao Minh 33a37d85d7
Fix native batch range partition segment sizing (#10089)
* Fix native batch range partition segment sizing

Fixes #10057.

Native batch range partitioning was only considering the partition
dimension value when grouping rows instead of using all of the row's
partition values. Thus, for schemas with multiple dimensions, the rollup
was overestimated, which would cause too many dimension values to be
packed into the same range partition. The resulting segments would then
be overly large (and not honor the target or max partition sizes).

Main changes:

- PartialDimensionDistributionTask: Consider all dimension values when
  grouping row

- RangePartitionMultiPhaseParallelIndexingTest: Regression test by
  having input with rows that should roll up and rows that should not
  roll up

* Use hadoop & native hash ingestion row group key
2020-06-29 17:49:52 -07:00
Jihoon Son c591ff8ea8
Add NonnullPair (#10013)
* Add NonnullPair

* new line

* test

* make it consistent
2020-06-26 09:52:06 -07:00
morrifeldman f6594fff60
Fix missing temp dir for native single_dim (#10046)
* Fix missing temp dir for native single_dim

Native single dim indexing throws a file not found exception from
InputEntityIteratingReader.java:81.  This MR creates the required
temporary directory when setting up the
PartialDimensionDistributionTask.  The change was tested on a Druid
cluster.  After installing the change native single_dim indexing
completes successfully.

* Fix indentation

* Use SinglePhaseSubTask as example for creating the temp dir

* Move temporary indexing dir creation in to TaskToolbox

* Remove unused dependency

Co-authored-by: Morri Feldman <morri@appsflyer.com>
2020-06-25 14:41:22 -07:00
Jihoon Son aaee72c781
Allow append to existing datasources when dynamic partitioning is used (#10033)
* Fill in the core partition set size properly for batch ingestion with
dynamic partitioning

* incomplete javadoc

* Address comments

* fix tests

* fix json serde, add tests

* checkstyle

* Set core partition set size for hash-partitioned segments properly in
batch ingestion

* test for both parallel and single-threaded task

* unused variables

* fix test

* unused imports

* add hash/range buckets

* some test adjustment and missing json serde

* centralized partition id allocation in parallel and simple tasks

* remove string partition chunk

* revive string partition chunk

* fill numCorePartitions for hadoop

* clean up hash stuffs

* resolved todos

* javadocs

* Fix tests

* add more tests

* doc

* unused imports

* Allow append to existing datasources when dynamic partitioing is used

* fix test

* checkstyle

* checkstyle

* fix test

* fix test

* fix other tests..

* checkstyle

* hansle unknown core partitions size in overlord segment allocation

* fail to append when numCorePartitions is unknown

* log

* fix comment; rename to be more intuitive

* double append test

* cleanup complete(); add tests

* fix build

* add tests

* address comments

* checkstyle
2020-06-25 13:37:31 -07:00
Jihoon Son d644a27f1a
Create packed core partitions for hash/range-partitioned segments in native batch ingestion (#10025)
* Fill in the core partition set size properly for batch ingestion with
dynamic partitioning

* incomplete javadoc

* Address comments

* fix tests

* fix json serde, add tests

* checkstyle

* Set core partition set size for hash-partitioned segments properly in
batch ingestion

* test for both parallel and single-threaded task

* unused variables

* fix test

* unused imports

* add hash/range buckets

* some test adjustment and missing json serde

* centralized partition id allocation in parallel and simple tasks

* remove string partition chunk

* revive string partition chunk

* fill numCorePartitions for hadoop

* clean up hash stuffs

* resolved todos

* javadocs

* Fix tests

* add more tests

* doc

* unused imports
2020-06-18 18:40:43 -07:00
Aleksey Plekhanov 2c384b61ff
IntelliJ inspection and checkstyle rule for "Collection.EMPTY_* field accesses replaceable with Collections.empty*()" (#9690)
* IntelliJ inspection and checkstyle rule for "Collection.EMPTY_* field accesses replaceable with Collections.empty*()"

* Reverted checkstyle rule

* Added tests to pass CI

* Codestyle
2020-06-18 09:47:07 -07:00
Jihoon Son 9a10f8352b
Set the core partition set size properly for batch ingestion with dynamic partitioning (#10012)
* Fill in the core partition set size properly for batch ingestion with
dynamic partitioning

* incomplete javadoc

* Address comments

* fix tests

* fix json serde, add tests

* checkstyle
2020-06-12 21:39:37 -07:00
Clint Wylie c5d6163c76
add a GeneratorInputSource to fill up a cluster with generated data for testing (#9946)
* move benchmark data generator into druid-processing, add a GeneratorInputSource to fill up a cluster with data

* newlines

* make test coverage not fail maybe

* remove useless test

* Update pom.xml

* Update GeneratorInputSourceTest.java

* less passive aggressive test names
2020-06-09 19:31:04 -07:00
Jonathan Wei 771870ae2d
Load broadcast datasources on broker and tasks (#9971)
* Load broadcast datasources on broker and tasks

* Add javadocs

* Support HTTP segment management

* Fix indexer maxSize

* inspection fix

* Make segment cache optional on non-historicals

* Fix build

* Fix inspections, some coverage, failed tests

* More tests

* Add CliIndexer to MainTest

* Fix inspection

* Rename UnprunedDataSegment to LoadableDataSegment

* Address PR comments

* Fix
2020-06-08 20:15:59 -07:00
Yuanli Han ee7bda5d8a
Fix compact partially overlapping segments (#9905)
* fix compact overlapping segments

* fix comment

* fix CI failure
2020-06-08 09:54:39 -07:00
Mainak Ghosh bcc066a27f
Empty partitionDimension has less rollup compared to when explicitly specified (#9861)
* Empty partitionDimension has less rollup compared to the case when it is explicitly specified

* Adding a unit test for the empty partitionDimension scenario. Fixing another test which was failing

* Fixing CI Build Inspection Issue

* Addressing all review comments

* Updating the javadocs for the hash method in HashBasedNumberedShardSpec
2020-06-05 12:42:42 -07:00
Jihoon Son 474f6fc99b
Fix shutdown reason for unknown tasks in taskQueue (#9954)
* Fix shutdown reason for unknown tasks in taskQueue

* unused imports
2020-06-03 15:40:28 -07:00
Xavier Léauté a934b2664c
remove ListenableFutures and revert to using the Guava implementation (#9944)
This change removes ListenableFutures.transformAsync in favor of the
existing Guava Futures.transform implementation. Our own implementation
had a bug which did not fail the future if the applied function threw an
exception, resulting in the future never completing.

An attempt was made to fix this bug, however when running againts Guava's own
tests, our version failed another half dozen tests, so it was decided to not
continue down that path and scrap our own implementation.

Explanation for how was this bug manifested itself:

An exception thrown in BaseAppenderatorDriver.publishInBackground when
invoked via transformAsync in StreamAppenderatorDriver.publish will
cause the resulting future to never complete.

This explains why when encountering https://github.com/apache/druid/issues/9845
the task will never complete, forever waiting for the publishFuture to
register the handoff. As a result, the corresponding "Error while
publishing segments ..." message only gets logged once the index task
times out and is forcefully shutdown when the future is force-cancelled
by the executor.
2020-06-03 10:46:03 -07:00
Clint Wylie c690d10a7d
support customized factory.json via IndexSpec for segment persist (#9957)
* support customized factory.json via IndexSpec for segment persist

* equals verifier
2020-06-01 16:36:32 -07:00
Maytas Monsereenusorn 5b4b5d77a8
Fails creation of TaskResource if availabilityGroup is null (#9892)
* Fails creation of TaskResource if availabilityGroup is null

* add check for requiredCapacity
2020-05-19 22:19:22 -07:00
Clint Wylie 2e9548d93d
refactor SeekableStreamSupervisor usage of RecordSupplier (#9819)
* refactor SeekableStreamSupervisor usage of RecordSupplier to reduce contention between background threads and main thread, refactor KinesisRecordSupplier, refactor Kinesis lag metric collection and emitting

* fix style and test

* cleanup, refactor, javadocs, test

* fixes

* keep collecting current offsets and lag if unhealthy in background reporting thread

* review stuffs

* add comment
2020-05-16 14:09:39 -07:00
Jihoon Son 46beaa0640
Fix potential resource leak in ParquetReader (#9852)
* Fix potential resource leak in ParquetReader

* add test

* never thrown exception

* catch potential exceptions
2020-05-16 09:57:12 -07:00
mcbrewster 28be107a1c
add flag to flattenSpec to keep null columns (#9814)
* add flag to flattenSpec to keep null columns

* remove changes to inputFormat interface

* add comment

* change comment message

* update web console e2e test

* move keepNullColmns to JSONParseSpec

* fix merge conflicts

* fix tests

* set keepNullColumns to false by default

* fix lgtm

* change Boolean to boolean, add keepNullColumns to hash, add tests for keepKeepNullColumns false + true with no nuulul columns

* Add equals verifier tests
2020-05-08 21:53:39 -07:00
Clint Wylie 267a6cc175
low hanging fruit - presize hash map for DruidSegmentReader (#9836) 2020-05-07 12:39:14 -07:00
Jihoon Son 6674d721bc
Avoid sorting values in InDimFilter if possible (#9800)
* Avoid sorting values in InDimFilter if possible

* tests

* more tests

* fix and and or filters

* fix build

* false and true vector matchers

* fix vector matchers

* checkstyle

* in filter null handling

* remove wrong test

* address comments

* remove unnecessary null check

* redundant separator

* address comments

* typo

* tests
2020-05-06 15:26:36 -07:00
Jihoon Son 964a1fc9df
Remove ParseSpec.toInputFormat() (#9815)
* Remove toInputFormat() from ParseSpec

* fix test
2020-05-05 11:17:57 -07:00
Maytas Monsereenusorn 8b78eebdbd
Test reading from empty kafka/kinesis partitions (#9729)
* add test for stream sequence number returns null

* fix checkstyle

* add index test for when stream returns null

* retrigger test
2020-04-27 10:23:56 -07:00
Jihoon Son 7fa72fbf15
Initialize SettableByteEntityReader only when inputFormat is not null (#9734)
* Lazy initialization of SettableByteEntityReader to avoid NPE

* toInputFormat for tsv

* address comments

* common code
2020-04-24 10:22:51 -07:00
Suneet Saldanha 642fe83897
Indexing Service validates externally received taskId (#9666)
Addresses issues flagged by https://lgtm.com/rules/5970070/
2020-04-10 10:36:26 -07:00
Suneet Saldanha 1ced3b33fb
IntelliJ inspections cleanup (#9339)
* IntelliJ inspections cleanup

* Standard Charset object can be used
* Redundant Collection.addAll() call
* String literal concatenation missing whitespace
* Statement with empty body
* Redundant Collection operation
* StringBuilder can be replaced with String
* Type parameter hides visible type

* fix warnings in test code

* more test fixes

* remove string concatenation inspection error

* fix extra curly brace

* cleanup AzureTestUtils

* fix charsets for RangerAdminClient

* review comments
2020-04-10 10:04:40 -07:00
Maytas Monsereenusorn b95a1b9878
Fix NPE in RemoteTaskRunner event handler causes JVM shutdown (#9610)
* Fix NPE in RemoteTaskRunner event handler causes JVM shutdown

* address comments

* fix compile

* fix checkstyle

* fix lgtm

* fix merge

* fix test

* fix tests

* change scope

* address comments

* address comments
2020-04-07 14:53:51 -07:00
Clint Wylie d267b1c414
check paths used for shuffle intermediary data manager get and delete (#9630)
* check paths used for shuffle intermediary data manager get and delete

* add test

* newline

* meh
2020-04-07 09:47:18 -07:00
Jihoon Son 82ce60b5c1
Reuse transformer in stream indexing (#9625)
* Reuse transformer in stream indexing

* remove unused method

* memoize complied pattern
2020-04-06 16:36:08 -07:00
Suneet Saldanha af3337dac8
DruidInputSource can add new dimensions during re-ingestion (#9590)
* WIP integration tests

* Add integration test for ingestion with transformSpec

* WIP almost working tests

* Add ignored tests

* checkstyle stuff

* remove newPage from index task ingestion spec

* more test cleanup

* still not quite working

* Actually disable the tests

* working tests

* fix codestyle

* dont use junit in integration tests

* actually fix the bug

* fix checkstyle

* bring index tests closer to reindex tests
2020-04-02 17:32:31 -07:00
Jihoon Son 0da8ffc3ff
Bump up development version to 0.19.0-SNAPSHOT (#9586) 2020-03-30 16:24:04 -07:00
Xavier Léauté b4ad3d0d88
fix nullhandling exceptions related to test ordering (#9570)
* fix nullhandling exceptions related to test ordering

Tests might get executed in different order depending on the maven
version and the test environment. This may lead to "NullHandling module
not initialized" errors for some tests where we do not initialize
null-handling explicitly.

* use InitializedNullHandlingTest
2020-03-27 09:46:31 -07:00
Suneet Saldanha 55c08e0746
DruidSegmentReader should work if timestamp is specified as a dimension (#9530)
* DruidSegmentReader should work if timestamp is specified as a dimension

* Add integration tests

Tests for compaction and re-indexing a datasource with the timestamp column

* Instructions to run integration tests against quickstart

* address pr
2020-03-25 13:47:34 -07:00
Clint Wylie bf85ea19b2
roaring bitmaps by default (#9548)
* it is finally time

* fix it

* more docs

* fix doc
2020-03-23 18:15:57 -07:00
Clint Wylie 142742f291
add kinesis lag metric (#9509)
* add kinesis lag metric

* fixes

* heh

* do it right this time

* more test

* split out supervisor report lags into lagMillis, remove latest offsets from kinesis supervisor report since always null, review stuffs
2020-03-16 21:39:53 -07:00
Jihoon Son 7401bb3f93
Improve OvershadowableManager performance (#9441)
* Use the iterator instead of higherKey(); use the iterator API instead of stream

* Fix tests; fix a concurrency bug in timeline

* fix test

* add tests for findNonOvershadowedObjectsInInterval

* fix test

* add missing tests; fix a bug in QueueEntry

* equals tests

* fix test
2020-03-10 13:22:19 -07:00
Clint Wylie 8b9fe6f584
query laning and load shedding (#9407)
* prototype

* merge QueryScheduler and QueryManager

* everything in its right place

* adjustments

* docs

* fixes

* doc fixes

* use resilience4j instead of semaphore

* more tests

* simplify

* checkstyle

* spelling

* oops heh

* remove unused

* simplify

* concurrency tests

* add SqlResource tests, refactor error response

* add json config tests

* use LongAdder instead of AtomicLong

* remove test only stuffs from scheduler

* javadocs, etc

* style

* partial review stuffs

* adjust

* review stuffs

* more javadoc

* error response documentation

* spelling

* preserve user specified lane for NoSchedulingStrategy

* more test, why not

* doc adjustment

* style

* missed review for make a thing a constant

* fixes and tests

* fix test

* Update docs/configuration/index.md

Co-Authored-By: sthetland <steve.hetland@imply.io>

* doc update

Co-authored-by: sthetland <steve.hetland@imply.io>
2020-03-10 02:57:16 -07:00
Jihoon Son f456d2fcf8
Resource leak in DruidSegmentReader (#9476)
* Close the Yielder in DruidSegmentReader

* forbidden api
2020-03-09 10:05:25 -07:00
Chi Cao Minh 4ed83f6af6
Fix superbatch merge last partition boundaries (#9448)
* Fix superbatch merge last partition boundaries

A bug in the computation for the last parallel merge partition could
cause an IndexOutOfBoundsException or precondition failure due to an
empty partition.

* Improve comments and tests
2020-03-04 10:35:21 -08:00
Lijia Liu 063811710e
#8690 use utc interval when create pedding segments (#9142)
Co-authored-by: Gian Merlino <gianmerlino@gmail.com>
2020-02-26 13:20:59 -08:00
Jihoon Son 3bc7ae782c
Create splits of multiple files for parallel indexing (#9360)
* Create splits of multiple files for parallel indexing

* fix wrong import and npe in test

* use the single file split in tests

* rename

* import order

* Remove specific local input source

* Update docs/ingestion/native-batch.md

Co-Authored-By: sthetland <steve.hetland@imply.io>

* Update docs/ingestion/native-batch.md

Co-Authored-By: sthetland <steve.hetland@imply.io>

* doc and error msg

* fix build

* fix a test and address comments

Co-authored-by: sthetland <steve.hetland@imply.io>
2020-02-24 17:34:39 -08:00
Jihoon Son 3bb9e7e53a
Inject things instead of subclassing everything for parallel task testing (#9353)
* Inject things instead of subclassing everything for parallel task
testing

* javadoc

* fix compilation

* fix wrong merge

* Address comments
2020-02-16 13:00:12 -08:00
Chi Cao Minh e8146d5914
More superbatch range partitioning tests (#9266)
More functional tests to cover handling of input data that has a
partition dimension that contains:

1) Null values: Should be in first partition

2) Multi values: Should cause superbatch task to abort
2020-02-10 15:17:53 -08:00
Suneet Saldanha 51d7864935
Codestyle - use java style array declaration (#9338)
* Codestyle - use java style array declaration

Replaced C-style array declarations with java style declarations and marked
the intelliJ inspection as an error

* cleanup test code
2020-02-10 14:25:26 -08:00
Clint Wylie 831ec172f1
Logging large segment list handling (#9312)
* better handling of large segment lists in logs

* more

* adjust

* exceptions

* fixes

* refactor

* debug

* heh

* dang
2020-02-07 21:42:45 -08:00
Jihoon Son e81230f9ab
Refactoring some codes around ingestion (#9274)
* Refactoring codes around ingestion:

- Parallel index task and simple task now use the same segment allocator implementation. This is reusable for the future implementation as well.
- Added PartitionAnalysis to store the analysis of the partitioning
- Move some util methods to SegmentLockHelper and rename it to TaskLockHelper

* fix build

* fix SingleDimensionShardSpecFactory

* optimize SingledimensionShardSpecFactory

* fix test

* shard spec builder

* import order

* shardSpecBuilder -> partialShardSpec

* build -> complete

* fix comment; add unit tests for partitionBoundaries

* add more tests and fix javadoc

* fix toString(); add serde tests for HashBasedNumberedPartialShardSpec and SegmentAllocateAction

* fix test

* add equality test for hash and range partial shard specs
2020-02-07 16:23:07 -08:00
Lucas Capistrant 53bb45fc9a
Forbid easily misused HashSet and HashMap constructors (#9165)
* Forbid easily misused HashSet and HashMap constructors

* Add two LinkedHashMap constructors to forbidden-apis and create utility method as replacement for them

* Fix visibility of constant in CollectionUtils.java

* Make an exception for an instance of LinkedHashMap#<init>(int) because proper sizing is used

* revert changes to sql module tests that should be in separate PR

* Finish reverting changes to sql module tests that were flagged in checkstyle during CI

* Add netty dependency resulting from SupressForbidden
2020-02-07 10:44:09 +03:00
Suneet Saldanha 303b02eba1
intelliJ inspections cleanup (#9260)
* intelliJ inspections cleanup

- remove redundant escapes
- performance warnings
- access static member via instance reference
- static method declared final
- inner class may be static

Most of these changes are aesthetic, however, they will allow inspections to
be enabled as part of CI checks going forward

The valuable changes in this delta are:
- using StringBuilder instead of string addition in a loop
    indexing-hadoop/.../Utils.java
    processing/.../ByteBufferMinMaxOffsetHeap.java
- Use class variables instead of static variables for parameterized test
    processing/src/.../ScanQueryLimitRowIteratorTest.java

* Add intelliJ inspection warnings as errors to druid profile

* one more static inner class
2020-01-29 11:50:52 -08:00
Roman Leventov b9186f8f9f Reconcile terminology and method naming to 'used/unused segments'; Rename MetadataSegmentManager to MetadataSegmentsManager (#7306)
* Reconcile terminology and method naming to 'used/unused segments'; Don't use terms 'enable/disable data source'; Rename MetadataSegmentManager to MetadataSegments; Make REST API methods which mark segments as used/unused to return server error instead of an empty response in case of error

* Fix brace

* Import order

* Rename withKillDataSourceWhitelist to withSpecificDataSourcesToKill

* Fix tests

* Fix tests by adding proper methods without interval parameters to IndexerMetadataStorageCoordinator instead of hacking with Intervals.ETERNITY

* More aligned names of DruidCoordinatorHelpers, rename several CoordinatorDynamicConfig parameters

* Rename ClientCompactTaskQuery to ClientCompactionTaskQuery for consistency with CompactionTask; ClientCompactQueryTuningConfig to ClientCompactionTaskQueryTuningConfig

* More variable and method renames

* Rename MetadataSegments to SegmentsMetadata

* Javadoc update

* Simplify SegmentsMetadata.getUnusedSegmentIntervals(), more javadocs

* Update Javadoc of VersionedIntervalTimeline.iterateAllObjects()

* Reorder imports

* Rename SegmentsMetadata.tryMark... methods to mark... and make them to return boolean and the numbers of segments changed and relay exceptions to callers

* Complete merge

* Add CollectionUtils.newTreeSet(); Refactor DruidCoordinatorRuntimeParams creation in tests

* Remove MetadataSegmentManager

* Rename millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments to leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments

* Fix tests, refactor DruidCluster creation in tests into DruidClusterBuilder

* Fix inspections

* Fix SQLMetadataSegmentManagerEmptyTest and rename it to SqlSegmentsMetadataEmptyTest

* Rename SegmentsAndMetadata to SegmentsAndCommitMetadata to reduce the similarity with SegmentsMetadata; Rename some methods

* Rename DruidCoordinatorHelper to CoordinatorDuty, refactor DruidCoordinator

* Unused import

* Optimize imports

* Rename IndexerSQLMetadataStorageCoordinator.getDataSourceMetadata() to retrieveDataSourceMetadata()

* Unused import

* Update terminology in datasource-view.tsx

* Fix label in datasource-view.spec.tsx.snap

* Fix lint errors in datasource-view.tsx

* Doc improvements

* Another attempt to please TSLint

* Another attempt to please TSLint

* Style fixes

* Fix IndexerSQLMetadataStorageCoordinator.createUsedSegmentsSqlQueryForIntervals() (wrong merge)

* Try to fix docs build issue

* Javadoc and spelling fixes

* Rename SegmentsMetadata to SegmentsMetadataManager, address other comments

* Address more comments
2020-01-27 11:24:29 -08:00
Gian Merlino 19b427e8f3
Add JoinableFactory interface and use it in the query stack. (#9247)
* Add JoinableFactory interface and use it in the query stack.

Also includes InlineJoinableFactory, which enables joining against
inline datasources. This is the first patch where a basic join query
actually works. It includes integration tests.

* Fix test issues.

* Adjustments from code review.
2020-01-24 13:10:01 -08:00
Gian Merlino f0f68570ec
Use DataSourceAnalysis throughout the query stack. (#9239)
Builds on #9235, using the datasource analysis functionality to replace various ad-hoc
approaches. The most interesting changes are in ClientQuerySegmentWalker (brokers),
ServerManager (historicals), and SinkQuerySegmentWalker (indexing tasks).

Other changes related to improving how we analyze queries:

1) Changes TimelineServerView to return an Optional timeline, which I thought made
   the analysis changes cleaner to implement.
2) Added QueryToolChest#canPerformSubquery, which is now used by query entry points to
   determine whether it is safe to pass a subquery dataSource to the query toolchest.
   Fixes an issue introduced in #5471 where subqueries under non-groupBy-typed queries
   were silently ignored, since neither the query entry point nor the toolchest did
   anything special with them.
3) Removes the QueryPlus.withQuerySegmentSpec method, which was mostly being used in
   error-prone ways (ignoring any potential subqueries, and not verifying that the
   underlying data source is actually a table). Replaces with a new function,
   Queries.withSpecificSegments, that includes sanity checks.
2020-01-23 14:07:14 -08:00
Gian Merlino d886463253
Add join-related DataSource types, and analysis functionality. (#9235)
* Add join-related DataSource types, and analysis functionality.

Builds on #9111 and implements the datasource analysis mentioned in #8728. Still can't
handle join datasources, but we're a step closer.

Join-related DataSource types:

1) Add "join", "lookup", and "inline" datasources.
2) Add "getChildren" and "withChildren" methods to DataSource, which will be used
   in the future for query rewriting (e.g. inlining of subqueries).

DataSource analysis functionality:

1) Add DataSourceAnalysis class, which breaks down datasources into three components:
   outer queries, a base datasource (left-most of the highest level left-leaning join
   tree), and other joined-in leaf datasources (the right-hand branches of the
   left-leaning join tree).
2) Add "isConcrete", "isGlobal", and "isCacheable" methods to DataSource in order to
   support analysis.

Other notes:

1) Renamed DataSource#getNames to DataSource#getTableNames, which I think is clearer.
   Also, made it a Set, so implementations don't need to worry about duplicates.
2) The addition of "isCacheable" should work around #8713, since UnionDataSource now
   returns false for cacheability.

* Remove javadoc comment.

* Updates reflecting code review.

* Add comments.

* Add more comments.
2020-01-22 14:54:47 -08:00
Jihoon Son d541cbe436 Support both IndexTuningConfig and ParallelIndexTuningConfig for compaction task (#9222)
* Support both IndexTuningConfig and ParallelIndexTuningConfig for compaction task

* tuningConfig module

* fix tests
2020-01-21 13:56:54 -08:00
Chi Cao Minh 0b0056b77f More tests for range partition parallel indexing (#9232)
Add more unit tests for range partition native batch parallel indexing.

Also, fix a bug where ParallelIndexPhaseRunner incorrectly thinks that
identical collected DimensionDistributionReports are not equal due to
not overriding equals() in DimensionDistributionReport.
2020-01-21 12:59:43 -08:00
Gian Merlino d21054f7c5
Remove the deprecated interval-chunking stuff. (#9216)
* Remove the deprecated interval-chunking stuff.

See https://github.com/apache/druid/pull/6591, https://github.com/apache/druid/pull/4004#issuecomment-284171911 for details.

* Remove unused import.

* Remove chunkInterval too.
2020-01-19 17:14:23 -08:00
Jihoon Son 153495068b Doc update for the new input source and the new input format (#9171)
* Doc update for new input source and input format.

- The input source and input format are promoted in all docs under docs/ingestion
- All input sources including core extension ones are located in docs/ingestion/native-batch.md
- All input formats and parsers including core extension ones are localted in docs/ingestion/data-formats.md
- New behavior of the parallel task with different partitionsSpecs are documented in docs/ingestion/native-batch.md

* parquet

* add warning for range partitioning with sequential mode

* hdfs + s3, gs

* add fs impl for gs

* address comments

* address comments

* gcs
2020-01-17 15:52:05 -08:00
Atul Mohan b642b1aa5b Fix deserialization of maxBytesInMemory (#9092)
* Fix deserialization of maxBytesInMemory

* Add maxBytes check
2020-01-12 20:08:07 -08:00
Clint Wylie f540216931 fix InputFormat serde issue with SeekableStream based supervisors (#9136) 2020-01-07 16:18:54 -06:00
Jonathan Wei aa539177ec De-incubation cleanup in code, docs, packaging (#9108)
* De-incubation cleanup in code, docs, packaging

* remove unused docs script
2020-01-03 12:33:19 -05:00
Jonathan Wei 4e8368a5d9 Set version to 0.18.0-SNAPSHOT (#9109) 2020-01-02 17:55:10 -05:00
Suneet Saldanha dec619ebf4 Optimize CachingLocalSegmentAllocator#getSequenceName (#8909)
* Optimize CachingLocalSegmentAllocator#getSequenceName

Replace StringUtils#format with string addition to generate the sequence
name for an interval and partition. This is faster because format uses a
Matcher under the covers to replace the string format with the variables.

* fix imports and add test

* Add comment about optimization

* Use renamed function for TaskToolbox

* Move tests after refactor

* Rename tests
2019-12-23 18:33:22 -08:00
Chi Cao Minh 6178f05da6 Fail superbatch range partition multi dim values (#9058)
* Fail superbatch range partition multi dim values

Change the behavior of parallel indexing range partitioning to fail
ingestion if any row had multiple values for the partition dimension.
After this change, the behavior matches that of hadoop indexing.
(Previously, rows with multiple dimension values would be skipped.)

* Improve err msg, rename method, rename test class
2019-12-18 10:14:03 -08:00
Himanshu 45101183bc
HRTR: make pending task execution handling to go through all tasks on not finding worker slots (#8697)
* HRTR: make pending task execution handling to go through all tasks on
not finding worker slots

* make HRTR methods package private that are meant to be used only in HttpRemoteTaskRunnerResource

* mark HttpRemoteTaskRunnerWorkItem.State global variables final

* hrtr: move immutableWorker NULL check outside of try-catch or finally block could have NPE

* add some explanatory comments

* add comment on explaining mechanics around hand off of pending tasks from submission to it getting picked up by a task execution thread

* fix spelling
2019-12-12 14:58:52 -08:00
Jonathan Wei 8af41d7cd0 Update version to 0.18.0-incubating-SNAPSHOT (#9009) 2019-12-11 14:04:03 -08:00
Chi Cao Minh 3de7ab8523 DataSketches jars in core (#9003)
Having DataSketches jars in core will allow potential improvements, for
example:
- Provide an alternative implementation of HLL:
  https://datasketches.github.io/docs/HLL/HllSketchVsDruidHyperLogLogCollector.html
- Range partitioning for native parallel batch indexing without having
  the user load extensions on the classpath

Dev mailing list discussion:
https://lists.apache.org/thread.html/301410d71ff799cf616bf17c4ebcf9999fc30829f5fa62909f403e6c%40%3Cdev.druid.apache.org%3E
2019-12-10 14:02:34 -08:00
Chi Cao Minh bab78fc80e Parallel indexing single dim partitions (#8925)
* Parallel indexing single dim partitions

Implements single dimension range partitioning for native parallel batch
indexing as described in #8769. This initial version requires the
druid-datasketches extension to be loaded.

The algorithm has 5 phases that are orchestrated by the supervisor in
`ParallelIndexSupervisorTask#runRangePartitionMultiPhaseParallel()`.
These phases and the main classes involved are described below:

1) In parallel, determine the distribution of dimension values for each
   input source split.

   `PartialDimensionDistributionTask` uses `StringSketch` to generate
   the approximate distribution of dimension values for each input
   source split. If the rows are ungrouped,
   `PartialDimensionDistributionTask.UngroupedRowDimensionValueFilter`
   uses a Bloom filter to skip rows that would be grouped. The final
   distribution is sent back to the supervisor via
   `DimensionDistributionReport`.

2) The range partitions are determined.

   In `ParallelIndexSupervisorTask#determineAllRangePartitions()`, the
   supervisor uses `StringSketchMerger` to merge the individual
   `StringSketch`es created in the preceding phase. The merged sketch is
   then used to create the range partitions.

3) In parallel, generate partial range-partitioned segments.

   `PartialRangeSegmentGenerateTask` uses the range partitions
   determined in the preceding phase and
   `RangePartitionCachingLocalSegmentAllocator` to generate
   `SingleDimensionShardSpec`s.  The partition information is sent back
   to the supervisor via `GeneratedGenericPartitionsReport`.

4) The partial range segments are grouped.

   In `ParallelIndexSupervisorTask#groupGenericPartitionLocationsPerPartition()`,
   the supervisor creates the `PartialGenericSegmentMergeIOConfig`s
   necessary for the next phase.

5) In parallel, merge partial range-partitioned segments.

   `PartialGenericSegmentMergeTask` uses `GenericPartitionLocation` to
   retrieve the partial range-partitioned segments generated earlier and
   then merges and publishes them.

* Fix dependencies & forbidden apis

* Fixes for integration test

* Address review comments

* Fix docs, strict compile, sketch check, rollup check

* Fix first shard spec, partition serde, single subtask

* Fix first partition check in test

* Misc rewording/refactoring to address code review

* Fix doc link

* Split batch index integration test

* Do not run parallel-batch-index twice

* Adjust last partition

* Split ITParallelIndexTest to reduce runtime

* Rename test class

* Allow null values in range partitions

* Indicate which phase failed

* Improve asserts in tests
2019-12-09 23:05:49 -08:00
Roman Leventov 1c62987783
Add SelfDiscoveryResource; rename org.apache.druid.discovery.No… (#6702)
* Add SelfDiscoveryResource

* Rename org.apache.druid.discovery.NodeType to NodeRole. Refactor CuratorDruidNodeDiscoveryProvider. Make SelfDiscoveryResource to listen to updates only about a single node (itself).

* Extended docs

* Fix brace

* Remove redundant throws in Lifecycle.Handler.stop()

* Import order

* Remove unresolvable link

* Address comments

* tmp

* tmp

* Rollback docker changes

* Remove extra .sh files

* Move filter

* Fix SecurityResourceFilterTest
2019-12-08 18:47:58 +03:00
Jonathan Wei c949a25210
Add DruidInputSource (replacement for IngestSegmentFirehose) (#8982)
* Add Druid input source and format

* Inherit dims/metrics from segment

* Add ingest segment firehose reindexing test

* Remove unnecessary module

* Fix unit tests, checkstyle

* Add doc entry

* Fix dimensionExclusions handling, add parallel index integration test

* Add spelling exclusion

* Address some PR comments

* Checkstyle

* wip

* Address rest of PR comments

* Address PR comments
2019-12-05 16:50:00 -08:00
Chi Cao Minh af74acaa85 Address security vulnerabilities CVSS >= 7 (#8980)
* Address security vulnerabilities CVSS >= 7

Update dependencies to address security vulnerabilities with CVSS scores
of 7 or higher. A new Travis CI job is added to prevent new
high/critical security vulnerabilities from being added.

Updated dependencies:
- api-util 1.0.0 -> 1.0.3
- jackson 2.9.10 -> 2.10.1
- kafka 2.1.0 -> 2.1.1
- libthrift 0.10.0 -> 0.13.0
- protobuf 3.2.0 -> 3.11.0

The following high/critical security vulnerabilities are currently
suppressed (so that the new Travis CI job can be added now) and are left
as future work to fix:
- hibernate-validator:5.2.5
- jackson-mapper-asl:1.9.13
- libthrift:0.6.1
- netty:3.10.6
- nimbus-jose-jwt:4.41.1

* Rename EDL1 license file

* Fix inspection errors
2019-12-05 14:34:35 -08:00
Fangyuan Deng 187cf0dd3f [Improvement] historical fast restart by lazy load columns metadata(20X faster) (#6988)
* historical fast restart by lazy load columns metadata

* delete repeated code

* add documentation for druid.segmentCache.lazyLoadOnStart

* fix unit test fail

* fix spellcheck

* update docs

* update docs mentioning a catch
2019-12-03 09:47:01 -08:00
Jonathan Wei 00ce18a0ea
Additional Kinesis resharding fixes (#8870)
* Additional Kinesis resharding fixes

* Address PR comments

* Remove unused method

* Adjust SegmentTransactionalInsertAction null handling

* Check for unchanged metadata on empty publish

* Add logs for empty publish

* Fix javadoc

* Clear offset when invalid endOffsets are seen

* Fix LGTM alert

* Fix build

* Add resharding note to Kinesis docs

* Checkstyle

* Spelling

* Address PR comments

* Checkstyle
2019-11-28 12:59:01 -08:00
Jihoon Son 86e8903523
Support orc format for native batch ingestion (#8950)
* Support orc format for native batch ingestion

* fix pom and remove wrong comment

* fix unnecessary condition check

* use flatMap back to handle exception properly

* move exceptionThrowingIterator to intermediateRowParsingReader

* runtime
2019-11-28 12:45:24 -08:00
jon-wei dfbc066163 Revert "[maven-release-plugin] prepare release druid-0.16.1-incubating-rc1"
This reverts commit a0f21d9b07.
2019-11-27 23:22:43 -08:00
jon-wei 0402ff85b8 Revert "[maven-release-plugin] prepare for next development iteration"
This reverts commit 8ffa71e7e6.
2019-11-27 23:22:32 -08:00
jon-wei 8ffa71e7e6 [maven-release-plugin] prepare for next development iteration 2019-11-27 23:18:48 -08:00
jon-wei a0f21d9b07 [maven-release-plugin] prepare release druid-0.16.1-incubating-rc1 2019-11-27 23:18:37 -08:00
Jihoon Son a2e6de4b16 Fix the potential race between SplittableInputSource.getNumSplits() and SplittableInputSource.createSplits() in TaskMonitor (#8924)
* Fix the potential race SplittableInputSource.getNumSplits() and SplittableInputSource.createSplits() in TaskMonitor

* Fix docs and javadoc

* Add unit tests for large or small estimated num splits

* add override
2019-11-23 01:38:08 -08:00
Gian Merlino e0eb85ace7 Add FileUtils.createTempDir() and enforce its usage. (#8932)
* Add FileUtils.createTempDir() and enforce its usage.

The purpose of this is to improve error messages. Previously, the error
message on a nonexistent or unwritable temp directory would be
"Failed to create directory within 10,000 attempts".

* Further updates.

* Another update.

* Remove commons-io from benchmark.

* Fix tests.
2019-11-22 19:48:49 -08:00
Rye 0514e5686e add TsvInputFormat (#8915)
* add TsvInputFormat

* refactor code

* fix grammar

* use enum replace string literal

* code refactor

* code refactor

* mark abstract for base class meant not to be instantiated

* remove constructor for test
2019-11-22 18:01:40 -08:00
Chi Cao Minh ff6217365b Refactor parallel indexing perfect rollup partitioning (#8852)
* Refactor parallel indexing perfect rollup partitioning

Refactoring to make it easier to later add range partitioning for
perfect rollup parallel indexing. This is accomplished by adding several
new base classes (e.g., PerfectRollupWorkerTask) and new classes for
encapsulating logic that needs to be changed for different partitioning
strategies (e.g., IndexTaskInputRowIteratorBuilder).

The code is functionally equivalent to before except for the following
small behavior changes:

1) PartialSegmentMergeTask: Previously, this task had a priority of
   DEFAULT_TASK_PRIORITY. It now has a priority of
   DEFAULT_BATCH_INDEX_TASK_PRIORITY (via the new PerfectRollupWorkerTask
   base class), since it is a batch index task.

2) ParallelIndexPhaseRunner: A decorator was added to
   subTaskSpecIterator to ensure the subtasks are generated with unique
   ids. Previously, only tests (i.e., MultiPhaseParallelIndexingTest)
   would have this decorator, but this behavior is desired for non-test
   code as well.

* Fix forbidden apis and pmd warnings

* Fix analyze dependencies warnings

* Fix IndexTask json and add IT diags

* Fix parallel index supervisor<->worker serde

* Fix TeamCity inspection errors/warnings

* Fix TeamCity inspection errors/warnings again

* Integrate changes with those from #8823

* Address review comments

* Address more review comments

* Fix forbidden apis

* Address more review comments
2019-11-20 17:24:12 -08:00
Jihoon Son ac6d703814 Support inputFormat and inputSource for sampler (#8901)
* Support inputFormat and inputSource for sampler

* Cleanup javadocs and names

* fix style

* fix timed shutoff input source reader

* fix timed shutoff input source reader again

* tidy up timed shutoff reader

* unused imports

* fix tc
2019-11-20 14:51:25 -08:00
Surekha d628bebbd7 Make supervisor API similar to submit task API (#8810)
* accept spec or dataSchema, tuningConfig, ioConfig while submitting task json

* fix test

* update docs

* lgtm warning

* Add original constructor back to IndexTask to minimize changes

* fix indentation in docs

* Allow spec to be specified in supervisor schema

* undo IndexTask spec changes

* update docs

* Add Nullable and deprecated annotations

* remove deprecated configs from SeekableStreamSupervisorSpec

* remove nullable annotation
2019-11-20 10:04:41 -08:00
Clint Wylie 3fcaa1a61b
fix sql compatible null handling config work with runtime.properties (#8876)
* fix sql compatible null handling config work with runtime.properties

* fix npe

* fix tests

* add friendly error

* comment, and friendlier still

* fix compile

* fix from merges
2019-11-20 03:55:29 -08:00
Atul Mohan f5fbd0bea0 Handle missing values for delimited text files when Nullhandling is enabled (#8779)
* Handle missing values

* Fix multi value tests

* Fix firehose tests

* Fix conflicts
2019-11-19 22:35:22 -08:00
Jihoon Son baefc65f80 Retrying with a backward compatible task type on unknown task type error in parallel indexing (#8905)
* Retrying with a backward compatible task type on unknown task type error in parallel indexing

* Register legacy class; add a serde test
2019-11-19 19:29:25 -08:00
Rye d0913475b7 sampler returns nulls in CSV (#8871)
* sampler returns nulls in CSV

* fixed kafka sampler test

* fix Kinesis test

* sql compatibility fix

* remove null to empty string conversion, use null

* fix sql compatibility
2019-11-19 13:59:44 -08:00
Gian Merlino c44452f0c1 Tidy up lifecycle, query, and ingestion logging. (#8889)
* Tidy up lifecycle, query, and ingestion logging.

The goal of this patch is to improve the clarity and usefulness of
Druid's logging for cluster operators. For more information, see
https://twitter.com/cowtowncoder/status/1195469299814555648.

Concretely, this patch does the following:

- Changes a lot of INFO logs to DEBUG, and DEBUG to TRACE, with the
  goal of reducing redundancy and improving clarity by avoiding
  showing rarely-useful log messages. This includes most "starting"
  and "stopping" messages, and most messages related to individual
  columns.
- Adds new log4j2 templates that show operators how to enabled DEBUG
  logging for certain important packages.
- Eliminate stack traces for query errors, unless log level is DEBUG
  or more. This is useful because query errors often indicate user
  error rather than system error, but dumping stack trace often gave
  operators the impression that there was a system failure.
- Adds task id to Appenderator, AppenderatorDriver thread names. In
  the default log4j2 configuration, this will put them in log lines
  as well. It's very useful if a user is using the Indexer, where
  multiple tasks run in the same JVM.
- More consistent terminology when it comes to "sequences" (sets of
  segments that are handed-off together by Kafka ingestion) and
  "offsets" (cursors in partitions). These terms had been confused in
  some log messages due to the fact that Kinesis calls offsets
  "sequence numbers".
- Replaces some ugly toString calls with either the JSONification or
  something more operator-accessible (like a URL or segment identifier,
  instead of JSON object representing the same).

* Adjustments.

* Adjust integration test.
2019-11-19 13:57:58 -08:00
Surekha cf6643eb9a add sequenceName and currentCheckPoint for backwards compatibility (#8864)
* add sequenceName and currentCheckPoint for backwards compatibility

* Add serde unit test in kafka

* fix checkstyle

* add hashcode

* update javadoc
2019-11-19 13:11:31 -08:00
Chi Cao Minh 8365bdf62a Address security vulnerabilities (#8878)
* Address security vulnerabilities

Security vulnerabilities addressed by upgrading 3rd party libs:

- Upgrade avro-ipc to 1.9.1
  - sonatype-2019-0115
- Upgrade caffeine to 2.8.0
  - sonatype-2019-0282
- Upgrade commons-beanutils to 1.9.4
  - CVE-2014-0114
- Upgrade commons-codec to 1.13
  - sonatype-2012-0050
- Upgrade commons-compress to 1.19
  - CVE-2019-12402
  - sonatype-2018-0293
- Upgrade hadoop-common to 2.8.5
  - CVE-2018-11767
- Upgrade hadoop-mapreduce-client-core to 2.8.5
  - CVE-2017-3166
- Upgrade hibernate-validator to 5.2.5
  - CVE-2017-7536
- Upgrade httpclient to 4.5.10
  - sonatype-2017-0359
- Upgrade icu4j to 55.1
  - CVE-2014-8147
- Upgrade jackson-databind to 2.6.7.3:
  - CVE-2017-7525
- Upgrade jetty-http to 9.4.12:
  - CVE-2017-7657
  - CVE-2017-7658
  - CVE-2017-7656
  - CVE-2018-12545
- Upgrade log4j-core to 2.8.2
  - CVE-2017-5645:
- Upgrade netty to 3.10.6
  - CVE-2015-2156
- Upgrade netty-common to 4.1.42
  - CVE-2019-9518
- Upgrade netty-codec-http to 4.1.42
  - CVE-2019-16869
- Upgrade nimbus-jose-jwt to 4.41.1
  - CVE-2017-12972
  - CVE-2017-12974
- Upgrade plexus-utils to 3.0.24
  - CVE-2017-1000487
  - sonatype-2015-0173
  - sonatype-2016-0398
- Upgrade postgresql to 42.2.8
  - CVE-2018-10936

Note that if users are using JDBC lookups with postgres, they may need
to update the JDBC jar used by the lookup extension.

* Fix license for postgresql
2019-11-19 09:14:33 -08:00
Atul Mohan 8515a03c6b Modify batch index task naming to accomodate simultaneous tasks (#8612)
* Change hadoop task naming

* Remove unused

* Add timestamp

* Fix build
2019-11-18 15:07:16 -08:00
Rye ea8e4066f6 Use earliest offset on kafka newly discovered partitions (#8748)
* Use earliest offset on kafka newly discovered partitions

* resolve conflicts

* remove redundant check cases

* simplified unit tests

* change test case

* rewrite comments

* add regression test

* add junit ignore annotation

* minor modifications

* indent

* override testableKafkaSupervisor and KafkaRecordSupplier to make the test runable

* modified test constructor of kafkaRecordSupplier

* simplify

* delegated constructor
2019-11-18 11:05:31 -08:00
Jihoon Son 1611792855
Add InputSource and InputFormat interfaces (#8823)
* Add InputSource and InputFormat interfaces

* revert orc dependency

* fix dimension exclusions and failing unit tests

* fix tests

* fix test

* fix test

* fix firehose and inputSource for parallel indexing task

* fix tc

* fix tc: remove unused method

* Formattable

* add needsFormat(); renamed to ObjectSource; pass metricsName for reader

* address comments

* fix closing resource

* fix checkstyle

* fix tests

* remove verify from csv

* Revert "remove verify from csv"

This reverts commit 1ea7758489.

* address comments

* fix import order and javadoc

* flatMap

* sampleLine

* Add IntermediateRowParsingReader

* Address comments

* move csv reader test

* remove test for verify

* adjust comments

* Fix InputEntityIteratingReader

* rename source -> entity

* address comments
2019-11-15 09:22:09 -08:00
Clint Wylie cc54b2a9df support for array expressions in TransformSpec with ExpressionTransform (#8744)
* transformSpec + array expressions

changes:
* added array expression support to transformSpec
* removed ParseSpec.verify since its only use afaict was preventing transform expr that did not replace their input from functioning
* hijacked index task test to test changes

* remove docs about being unsupported

* re-arrange test assert

* unused imports

* imports

* fix tests

* preserve types

* suppress warning, fixes, add test

* formatting

* cleanup

* better list to array type conversion and tests

* fix oops
2019-11-13 11:04:37 -08:00
chencb cc2bdb5f51 Fix hadoop task jdk11 compatible (#8799)
* Fix hadoop task jdk11 compatible

* Fix HadoopTaskTest
2019-11-13 02:32:46 -08:00
Roman Leventov 5c0fc0a13a Fix ambiguity about IndexerSQLMetadataStorageCoordinator.getUsedSegmentsForInterval() returning only non-overshadowed or all used segments (#8564)
* IndexerSQLMetadataStorageCoordinator.getTimelineForIntervalsWithHandle() don't fetch abutting intervals; simplify getUsedSegmentsForIntervals()

* Add VersionedIntervalTimeline.findNonOvershadowedObjectsInInterval() method; Propagate the decision about whether only visible segmetns or visible and overshadowed segments should be returned from IndexerMetadataStorageCoordinator's methods to the user logic; Rename SegmentListUsedAction to RetrieveUsedSegmentsAction, SegmetnListUnusedAction to RetrieveUnusedSegmentsAction, and UsedSegmentLister to UsedSegmentsRetriever

* Fix tests

* More fixes

* Add javadoc notes about returning Collection instead of Set. Add JacksonUtils.readValue() to reduce boilerplate code

* Fix KinesisIndexTaskTest, factor out common parts from KinesisIndexTaskTest and KafkaIndexTaskTest into SeekableStreamIndexTaskTestBase

* More test fixes

* More test fixes

* Add a comment to VersionedIntervalTimelineTestBase

* Fix tests

* Set DataSegment.size(0) in more tests

* Specify DataSegment.size(0) in more places in tests

* Fix more tests

* Fix DruidSchemaTest

* Set DataSegment's size in more tests and benchmarks

* Fix HdfsDataSegmentPusherTest

* Doc changes addressing comments

* Extended doc for visibility

* Typo

* Typo 2

* Address comment
2019-11-06 11:07:04 -08:00
Giuseppe Martino 9c171e2b1f Message rejection absolute date (#8656)
* Add option lateMessageRejectionStartDate

* Use option lateMessageRejectionStartDate

* Fix tests

* Add lateMessageRejectionStartDate to kafka indexing service

* Update tests kafka indexing service

* Fix tests for KafkaSupervisorTest

* Add lateMessageRejectionStartDate to KinesisSupervisorIOConfig

* Fix var name

* Update documentation

* Add check lateMessageRejectionStartDateTime and lateMessageRejectionPeriod, fails if both were specified.
2019-10-31 15:13:02 -07:00
Jihoon Son 2363b61983
Asynchronous file copy in the shuffle of parallel indexing task (#8783) 2019-10-30 18:00:05 -07:00
Xiaobao b9d10473a5 fix typo (#8745) 2019-10-25 19:21:56 +08:00
Jihoon Son 094936ca03 Remove commit() method Firehose (#8688)
* Remove commit() method Firehose

* fix javadoc
2019-10-23 16:52:02 -07:00
Jihoon Son 2518478b20 Remove deprecated parameter for Checkpoint request (#8707)
* Remove deprecated parameter for Checkpoint request

* fix wrong doc
2019-10-23 16:51:16 -07:00
Surekha 98f59ddd7e Add `sys.supervisors` table to system tables (#8547)
* Add supervisors table to SystemSchema

* Add docs

* fix checkstyle

* fix test

* fix CI

* Add comments

* Fix javadoc teamcity error

* comments

* fix links in docs

* fix links

* rename fullStatus query param to system and remove it from docs
2019-10-18 15:16:42 -07:00
Jihoon Son 30c15900be
Auto compaction based on parallel indexing (#8570)
* Auto compaction based on parallel indexing

* javadoc and doc

* typo

* update spell

* addressing comments

* address comments

* fix log

* fix build

* fix test

* increase default max input segment bytes per task

* fix test
2019-10-18 13:24:14 -07:00
Mingming Qiu 2c758ef5ff Support assign tasks to run on different categories of MiddleManagers (#7066)
* Support assign tasks to run on different tiers of MiddleManagers

* address comments

* address comments

* rename tier to category and docs

* doc

* fix doc

* fix spelling errors

* docs
2019-10-17 12:57:19 -07:00
Jonathan Wei 89ce6384f5
More Kinesis resharding adjustments (#8671)
* More Kinesis resharding adjustments

* Fix TC inspection

* Fix comment'

* Adjust comment, small refactor

* Make repartition transition time configurable

* Add spellcheck exclusion

* Spelling fix
2019-10-15 23:19:17 -07:00
Jihoon Son 4046c86d62
Stateful auto compaction (#8573)
* Stateful auto compaction

* javaodc

* add removed test back

* fix test

* adding indexSpec to compactionState

* fix build

* add lastCompactionState

* address comments

* extract CompactionState

* fix doc

* fix build and test

* Add a task context to store compaction state; add javadoc

* fix it test
2019-10-15 22:57:42 -07:00
Jonathan Wei 0c387c1d47 Fix Kinesis resharding issues (#8644)
* Fix Kinesis resharding issues

* PR comments

* Adjust metadata error message

* Remove unused method

* Use sha1 for shard id hashing

* Add metadata sanity check, add comment

* Only use shard ID hashing for group mapping

* Style fix

* Fix unused import

* update comment

* Fix teamcity inspection
2019-10-10 00:16:44 -07:00
Jihoon Son 96d8523ecb Use hash of Segment IDs instead of a list of explicit segments in auto compaction (#8571)
* IOConfig for compaction task

* add javadoc, doc, unit test

* fix webconsole test

* add spelling

* address comments

* fix build and test

* address comments
2019-10-09 11:12:00 -07:00
Chi Cao Minh b6b5517c20 Speed up ParallelIndexSupervisorTask tests (#8633)
Previously, some tests for ParallelIndexSupervisorTask were being run
twice unnecessarily.
2019-10-08 19:56:12 -07:00
Himanshu d91d1c8699 make TaskMonitor continue to monitor in the face of transient errors (#8625) 2019-10-04 09:42:20 -07:00
Fokko Driesprong 82bfe86d0c Make more package EverythingIsNonnullByDefault by default (#8198)
* Make more package EverythingIsNonnullByDefault by default

* Fixed additional voilations after pulling in master

* Change iterator to list.addAll

* Fix annotations
2019-09-30 18:53:18 -06:00
elloooooo 7f2b6577ef get active task by datasource when supervisor discover tasks (#8450)
* get active task by datasource when supervisor discover tasks

* fix ut

* fix ut

* fix ut

* remove unnecessary condition check

* fix ut

* remove stream in hot loop
2019-09-26 16:15:24 -07:00
Rye f2a444321b Added live reports for Kafka and Native batch task (#8557)
* Added live reports for Kafka and Native batch task

* Removed unused local variables

* Added the missing unit test

* Refine unit test logic, add implementation for HttpRemoteTaskRunner

* checksytle fixes

* Update doc descriptions for updated API

* remove unnecessary files

* Fix spellcheck complaints

* More details for api descriptions
2019-09-23 21:08:36 -07:00
Chi Cao Minh aeac0d4fd3 Adjust defaults for hashed partitioning (#8565)
* Adjust defaults for hashed partitioning

If neither the partition size nor the number of shards are specified,
default to partitions of 5,000,000 rows (similar to the behavior of
dynamic partitions). Previously, both could be null and cause incorrect
behavior.

Specifying both a partition size and a number of shards now results in
an error instead of ignoring the partition size in favor of using the
number of shards. This is a behavior change that makes it more apparent
to the user that only one of the two properties will be honored
(previously, a message was just logged when the specified partition size
was ignored).

* Fix test

* Handle -1 as null

* Add -1 as null tests for single dim partitioning

* Simplify logic to handle -1 as null

* Address review comments
2019-09-21 20:57:40 -07:00
Himanshu 62afbca7b9
update HRTR to account for task known to be running on a worker when it shows up (#8427) 2019-09-19 10:19:17 -07:00
Clint Wylie b00dd84fa2 clarify error messaging for parallel indexing task when when missing numShards or intervals (#8513) 2019-09-11 20:47:27 -07:00
Chi Cao Minh 5f61374cb3 Fix dependency analyze warnings (#8230)
* Fix dependency analyze warnings

Update the maven dependency plugin to the latest version and fix all
warnings for unused declared and used undeclared dependencies in the
compile scope. Added new travis job to add the check to CI. Also fixed
some source code files to use the correct packages for their imports and
updated druid-forbidden-apis to prevent regressions.

* Address review comments

* Adjust scope for org.glassfish.jaxb:jaxb-runtime

* Fix dependencies for hdfs-storage

* Consolidate netty4 versions
2019-09-09 14:37:21 -07:00
Clint Wylie c73a489335
bump master version to 0.17.0-incubating-SNAPSHOT (#8421) 2019-08-28 01:58:36 -07:00
Jihoon Son e5ef5ddafa Fix the shuffle with TLS enabled for parallel indexing; add an integration test; improve unit tests (#8350)
* Fix shuffle with tls enabled; add an integration test; improve unit tests

* remove debug log

* fix tests

* unused import

* add javadoc

* rename to getContent
2019-08-26 19:27:41 -07:00
Dylan Wylie b2821a8371 do not exclude client core jar (#8339)
make indexing service depend on hadoop client
2019-08-26 13:48:24 -07:00
Xavier Léauté 8e0c307e54
Do not assume system classloader is URLClassLoader in Java 9+ (#8392)
* Fallback to parsing classpath for hadoop task in Java 9+
In Java 9 and above we cannot assume that the system classloader is an
instance of URLClassLoader. This change adds a fallback method to parse
the system classpath in that case, and adds a unit test to validate it matches
what JDK8 would do.

Note: This has not been tested in an actual hadoop setup, so this is mostly
to help us pass unit tests.

* Remove granularity test of dubious value
One of our granularity tests relies on system classloader being a URLClassLoaders to
catch a bug related to class initialization and static initializers using a subclass (see
#2979)
This test was added to catch a potential regression, but it assumes we would add back
the same type of static initializers to this specific class, so it seems to be of dubious value
as a unit test and mostly serves to illustrate the bug.

relates to #5589
2019-08-24 20:47:54 -04:00
Jihoon Son 95fa609615 Fix wrong partitionsSpec type names in the document (#8297)
* Fix wrong type names for partitionsSpec

* add unit tests; add json properties for backward compatibility

* beautify conf names

* remove maxRowsPerSegment from hashed partitionsSpec

* fix doc build
2019-08-23 13:44:58 -07:00
SandishKumarHN 33f0753a70 Add Checkstyle for constant name static final (#8060)
* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* merging with upstream

* review-1

* unknow changes

* unknow changes

* review-2

* merging with master

* review-2 1 changes

* review changes-2 2

* bug fix
2019-08-23 13:13:54 +03:00
Surekha cf2a2dd917
Add group_id to the sys.tasks table (#8304)
* Add group_id to overlord tasks API and sys.tasks table

* adjust test

* modify docs

* Make groupId nullable

* fix integration test

* fix toString

* Remove groupId from TaskInfo

* Modify docs and tests

* modify TaskMonitorTest
2019-08-22 15:28:23 -07:00
Jihoon Son fba92ae469 Fix to always use end sequenceNumber for reset (#8305)
* Fix to always use end sequenceNumber for reset

* fix checkstyle

* fix style and add log
2019-08-22 16:51:25 -05:00
Jonathan Wei 96e2142ea3 Cleanup appenderators and segment walkers in UnifiedIndexerAppenderatorsManager (#8287)
* Cleanup Appenderators in UnifiedIndexerAppenderatorsManager

* PR comments

* More PR comments

* Fix test
2019-08-22 12:18:46 -07:00
Clint Wylie b95607d31c
remove YeOldePlumberSchool.java, unused (#8347) 2019-08-21 18:15:51 -07:00
Jihoon Son 22d6384d36
Fix unrealistic test variables in KafkaSupervisorTest and tidy up unused variable in checkpointing process (#7319)
* Fix unrealistic test arguments in KafkaSupervisorTest

* remove currentCheckpoint from checkpoint action

* rename variable
2019-08-21 10:58:22 -07:00
Benedict Jin 566dc8c719
Fix missing format argument (#8331) 2019-08-19 16:19:44 +08:00
Jihoon Son 31af4eb9ad
Rename maxNumSubTasks to maxNumConcurrentSubTasks for native parallel index task (#8324) 2019-08-16 15:57:13 -07:00
Jihoon Son 5dac6375f3
Add support for parallel native indexing with shuffle for perfect rollup (#8257)
* Add TaskResourceCleaner; fix a couple of concurrency bugs in batch tasks

* kill runner when it's ready

* add comment

* kill run thread

* fix test

* Take closeable out of Appenderator

* add javadoc

* fix test

* fix test

* update javadoc

* add javadoc about killed task

* address comment

* Add support for parallel native indexing with shuffle for perfect rollup.

* Add comment about volatiles

* fix test

* fix test

* handling missing exceptions

* more clear javadoc for stopGracefully

* unused import

* update javadoc

* Add missing statement in javadoc

* address comments; fix doc

* add javadoc for isGuaranteedRollup

* Rename confusing variable name and fix typos

* fix typos; move fetch() to a better home; fix the expiration time

* add support https
2019-08-15 17:43:35 -07:00
Jonathan Wei ef7b9606f2
Keep track of task location for completed tasks (#8286)
* Keep track of task location for completed tasks

* Add TaskLifecycleTest location checks
2019-08-15 16:57:02 -05:00
Jihoon Son 312cdc2452 Add TaskResourceCleaner; fix a couple of concurrency bugs in batch tasks (#8236)
* Add TaskResourceCleaner; fix a couple of concurrency bugs in batch tasks

* kill runner when it's ready

* add comment

* kill run thread

* fix test

* Take closeable out of Appenderator

* add javadoc

* fix test

* fix test

* update javadoc

* add javadoc about killed task

* address comment

* handling missing exceptions

* more clear javadoc for stopGracefully

* update javadoc

* Add missing statement in javadoc

* typo
2019-08-12 19:42:06 -05:00
Clint Wylie 1054d85171
add mechanism to control filter optimization in historical query processing (#8209)
* add support for mechanism to control filter optimization in historical query processing

* oops

* adjust

* woo

* javadoc

* review comments

* fix

* default

* oops

* oof

* this will fix it

* more nullable, refactor DimFilter.getRequiredColumns to use Set, formatting

* extract class DimFilterToStringBuilder with common code from custom DimFilter toString implementations

* adjust variable naming

* missing nullable

* more nullable

* fix javadocs

* nullable

* address review comments

* javadocs, precondition

* nullable

* rename method to be consistent

* review comments

* remove tuning from ColumnComparisonFilter/ColumnComparisonDimFilter
2019-08-09 16:36:18 -07:00
Jihoon Son ab5b3be6c6 Add shuffleSegmentPusher for data shuffle (#8115)
* Fix race between canHandle() and addSegment() in StorageLocation

* add comment

* Add shuffleSegmentPusher which is a dataSegmentPusher used for writing shuffle data in local storage.

* add comments

* unused import

* add comments

* fix test

* address comments

* remove <p> tag from javadoc

* address comments

* comparingLong

* Address comments

* fix test
2019-08-05 13:38:35 -07:00
Jihoon Son 1ee828ff49
Add a cluster-wide configuration to force timeChunk lock and add a doc for segment locking (#8173)
* Add a cluster-wide configuration to force timeChunk lock and add a doc for segment locking

* add more test

* javadoc for missingIntervalsInOverwriteMode

* Fix test

* Address comments

* avoid spotbugs
2019-08-02 20:30:05 -07:00
Jihoon Son 8a16a8e97f
Teach tasks what machine they are running on (#8190)
* Teach the middleManager port to tasks

* parent annotation

* Bind parent for indexer
2019-08-02 15:34:44 -07:00
Jonathan Wei 41893d4647 Simple memory allocation for CliIndexer tasks (#8201)
* Simple memory allocation for CliIndexer

* PR comments

* Checkstyle
2019-08-01 10:22:41 +08:00
Gian Merlino 77297f4e6f GroupBy array-based result rows. (#8196)
* GroupBy array-based result rows.

Fixes #8118; see that proposal for details.

Other than the GroupBy changes, the main other "interesting" classes are:

- ResultRow: The array-based result type.
- BaseQuery: T is no longer required to be Comparable.
- QueryToolChest: Adds "decorateObjectMapper" to enable query-aware serialization
  and deserialization of result rows (necessary due to their positional nature).
- QueryResource: Uses the new decoration functionality.
- DirectDruidClient: Also uses the new decoration functionality.
- QueryMaker (in Druid SQL): Modifications to read ResultRows.

These classes weren't changed, but got some new javadocs:

- BySegmentQueryRunner
- FinalizeResultsQueryRunner
- Query

* Adjustments for TC stuff.
2019-07-31 16:15:12 -07:00
Fokko Driesprong faf51107d5 Add SuppressWarnings SS_SHOULD_BE_STATIC (#8138)
* Spotbugs: SS_SHOULD_BE_STATIC (#8073)

* Add SuppressWarnings SS_SHOULD_BE_STATIC

Fixes #8073

* Fix the voilation

* Make them non-final

* Remove @Nonnull
2019-07-31 19:44:42 +03:00
Jihoon Son 385f492a55
Use PartitionsSpec for all task types (#8141)
* Use partitionsSpec for all task types

* fix doc

* fix typos and revert to use isPushRequired

* address comments

* move partitionsSpec to core

* remove hadoopPartitionsSpec
2019-07-30 17:24:39 -07:00
Fokko Driesprong e016995d1f Enable Spotbugs: WMI_WRONG_MAP_ITERATOR (#8005)
* WMI_WRONG_MAP_ITERATOR

* Fixed missing loop
2019-07-30 19:51:53 +03:00
Jihoon Son fb653ceef9 Add benchmark for VersionedIntervalTimeline (#8161)
* Add benchmark for VersionedIntervalTimeline

* rename
2019-07-30 08:10:00 -07:00
Jonathan Wei 640b7afc1c Add CliIndexer process type and initial task runner implementation (#8107)
* Add CliIndexer process type and initial task runner implementation

* Fix HttpRemoteTaskRunnerTest

* Remove batch sanity check on PeonAppenderatorsManager

* Fix paralle index tests

* PR comments

* Adjust Jersey resource logging

* Additional cleanup

* Fix SystemSchemaTest

* Add comment to LocalDataSegmentPusherTest absolute path test

* More PR comments

* Use Server annotated with RemoteChatHandler

* More PR comments

* Checkstyle

* PR comments

* Add task shutdown to stopGracefully

* Small cleanup

* Compile fix

* Address PR comments

* Adjust TaskReportFileWriter and fix nits

* Remove unnecessary closer

* More PR comments

* Minor adjustments

* PR comments

* ThreadingTaskRunner: cancel  task run future not shutdownFuture and remove thread from workitem
2019-07-29 17:06:33 -07:00
Chi Cao Minh ab71a2e1e4 Revert "Fix dependency analyze warnings (#8128)" (#8189)
This reverts commit 5dd0d8e873.
2019-07-29 11:42:16 -07:00
Jihoon Son adf7bafb9f Fix race between canHandle() and addSegment() in StorageLocation (#8114)
* Fix race between canHandle() and addSegment() in StorageLocation

* add comment

* add comments

* fix test

* address comments

* remove <p> tag from javadoc

* address comments

* comparingLong
2019-07-27 11:11:06 +03:00
Chi Cao Minh 5dd0d8e873 Fix dependency analyze warnings (#8128)
* Fix dependency analyze warnings

Update the maven dependency plugin to the latest version and fix all
warnings for unused declared and used undeclared dependencies in the
compile scope. Added new travis job to add the check to CI. Also fixed
some source code files to use the correct packages for their imports.

* Fix licenses and dependencies

* Fix licenses and dependencies again

* Fix integration test dependency

* Address review comments

* Fix unit test dependencies

* Fix integration test dependency

* Fix integration test dependency again

* Fix integration test dependency third time

* Fix integration test dependency fourth time

* Fix compile error

* Fix assert package
2019-07-26 10:49:03 -07:00
Jihoon Son db14946207
Add support minor compaction with segment locking (#7547)
* Segment locking

* Allow both timeChunk and segment lock in the same gruop

* fix it test

* Fix adding same chunk to atomicUpdateGroup

* resolving todos

* Fix segments to lock

* fix segments to lock

* fix kill task

* resolving todos

* resolving todos

* fix teamcity

* remove unused class

* fix single map

* resolving todos

* fix build

* fix SQLMetadataSegmentManager

* fix findInputSegments

* adding more tests

* fixing task lock checks

* add SegmentTransactionalOverwriteAction

* changing publisher

* fixing something

* fix for perfect rollup

* fix test

* adjust package-lock.json

* fix test

* fix style

* adding javadocs

* remove unused classes

* add more javadocs

* unused import

* fix test

* fix test

* Support forceTimeChunk context and force timeChunk lock for parallel index task if intervals are missing

* fix travis

* fix travis

* unused import

* spotbug

* revert getMaxVersion

* address comments

* fix tc

* add missing error handling

* fix backward compatibility

* unused import

* Fix perf of versionedIntervalTimeline

* fix timeline

* fix tc

* remove remaining todos

* add comment for parallel index

* fix javadoc and typos

* typo

* address comments
2019-07-24 17:35:46 -07:00
Eugene Sevastianov 799d20249f Response context refactoring (#8110)
* Response context refactoring

* Serialization/Deserialization of ResponseContext

* Added java doc comments

* Renamed vars related to ResponseContext

* Renamed empty() methods to createEmpty()

* Fixed ResponseContext usage

* Renamed multiple ResponseContext static fields

* Added PublicApi annotations

* Renamed QueryResponseContext class to ResourceIOReaderWriter

* Moved the protected method below public static constants

* Added createEmpty method to ResponseContext with DefaultResponseContext creation

* Fixed inspection error

* Added comments to the ResponseContext length limit and ResponseContext
http header name

* Added a comment of possible future refactoring

* Removed .gitignore file of indexing-service

* Removed a never-used method

* VisibleForTesting method reducing boilerplate

Co-Authored-By: Clint Wylie <cjwylie@gmail.com>

* Reduced boilerplate

* Renamed the method serialize to serializeWith

* Removed unused import

* Fixed incorrectly refactored test method

* Added comments for ResponseContext keys

* Fixed incorrectly refactored test method

* Fixed IntervalChunkingQueryRunnerTest mocks
2019-07-24 18:29:03 +03:00
Clint Wylie cb82d72547
fix forking task runner task shutdown to be more graceful (#8085)
* fix forking task runner shutdown to be more graceful

* javadoc
2019-07-23 11:47:23 -07:00
Himanshu 54a7b54d2d avoid 'must return non-void type' warning (#8105) 2019-07-18 15:02:27 -07:00
Jihoon Son c7eb7cd018
Add intermediary data server for shuffle (#8088)
* Add intermediary data server for shuffle

* javadoc

* adjust timeout

* resolved todo

* fix test

* style

* address comments

* rename to shuffleDataLocations

* Address comments

* bit adjustment StorageLocation

* fix test

* address comment & fix test

* handle interrupted exception
2019-07-18 14:46:47 -07:00
Surekha da16144495 Refactoring to use `CollectionUtils.mapValues` (#8059)
* doc updates and changes to use the CollectionUtils.mapValues utility method

* Add Structural Search patterns to intelliJ

* refactoring from PR comments

* put -> putIfAbsent

* do single key lookup
2019-07-17 23:02:22 -07:00
Roman Leventov ceb969903f
Refactor SQLMetadataSegmentManager; Change contract of REST met… (#7653)
* Refactor SQLMetadataSegmentManager; Change contract of REST methods in DataSourcesResource

* Style fixes

* Unused imports

* Fix tests

* Fix style

* Comments

* Comment fix

* Remove unresolvable Javadoc references; address comments

* Add comments to ImmutableDruidDataSource

* Merge with master

* Fix bad web-console merge

* Fixes in api-reference.md

* Rename in DruidCoordinatorRuntimeParams

* Fix compilation

* Residual changes
2019-07-17 17:18:48 +03:00
Clint Wylie 15fbf5983d add Class.getCanonicalName to forbidden-apis (#8086)
* add checkstyle to forbid unecessary use of Class.getCanonicalName

* use forbiddin-api instead of checkstyle

* add space
2019-07-16 15:21:50 -07:00
Himanshu 14aec7fcec
add config to optionally disable all compression in intermediate segment persists while ingestion (#7919)
* disable all compression in intermediate segment persists while ingestion

* more changes and build fix

* by default retain existing indexingSpec for intermediate persisted segments

* document indexSpecForIntermediatePersists index tuning config

* fix build issues

* update serde tests
2019-07-10 12:22:24 -07:00
Jihoon Son fcf56f2330 Add IS_INCREMENTAL_HANDOFF_SUPPORTED for KIS backward compatibility (#8050)
* Add IS_INCREMENTAL_HANDOFF_SUPPORTED for KIS backward compatibility

* do it for kafka only

* fix test
2019-07-10 08:29:37 -07:00
Gian Merlino 338b8b3fef
SupervisorManager: Add authorization checks to bulk endpoints. (#8044)
The endpoints added in #6272 were missing authorization checks. This patch removes the bulk
methods from SupervisorManager, and instead has SupervisorResource run the full list through
filterAuthorizedSupervisorIds before calling resume/suspend/terminate one by one.
2019-07-09 13:16:54 -07:00
Sashidhar Thallam 6701dc08fe Making StatusResponseHandler singleton and fixing all its instantiation invocations (#7969)
* Making StatusResponseHandler singleton and fixing all its instantiation invocations

* Using StatusResponseHandler.getInstance() where applicable
2019-07-08 13:33:00 +05:30
Chi Cao Minh 1166bbcb75 Remove static imports from tests (#8036)
Make static imports forbidden in tests and remove all occurrences to be
consistent with the non-test code.

Also, various changes to files affected by above:
- Reformat to adhere to druid style guide
- Fix various IntelliJ warnings
- Fix various SonarLint warnings (e.g., the expected/actual args to
  Assert.assertEquals() were flipped)
2019-07-06 09:33:12 -07:00
Clint Wylie 42a7b8849a remove FirehoseV2 and realtime node extensions (#8020)
* remove firehosev2 and realtime node extensions

* revert intellij stuff

* rat exclusion
2019-07-04 15:40:22 -07:00
Fokko Driesprong 4508d3ff55 Enable RCN_REDUNDANT_NULLCHECK_OF_NULL_VALUE (#7972) 2019-07-01 11:25:06 +05:30
Fokko Driesprong e8dedb3de4 Enable MS_MUTABLE_COLLECTION_PKGPROTECT (#7971) 2019-07-01 11:24:05 +05:30
Clint Wylie 494b8ebe56 multi-value string column support for expressions (#7588)
* array support for expression language for multi-value string columns

* fix tests?

* fixes

* more tests

* fixes

* cleanup

* more better, more test

* ignore inspection

* license

* license fix

* inspection

* remove dumb import

* more better

* some comments

* add expr rewrite for arrayfn args for more magic, tests

* test stuff

* more tests

* fix test

* fix test

* castfunc can deal with arrays

* needs more empty array

* more tests, make cast to long array more forgiving

* refactor

* simplify ExprMacro Expr implementations with base classes in core

* oops

* more test

* use Shuttle for Parser.flatten, javadoc, cleanup

* fixes and more tests

* unused import

* fixes

* javadocs, cleanup, refactors

* fix imports

* more javadoc

* more javadoc

* more

* more javadocs, nonnullbydefault, minor refactor

* markdown fix

* adjustments

* more doc

* move initial filter out

* docs

* map empty arg lambda, apply function argument validation

* check function args at parse time instead of eval time

* more immutable

* more more immutable

* clarify grammar

* fix docs

* empty array is string test, we need a way to make arrays better maybe in the future, or define empty arrays as other types..
2019-06-19 13:57:37 -07:00
Himanshu 417fcef385 WorkerTaskManager to create disk files atomically and ignore task file corruption (#7917)
* WorkerTaskManager to create disk files atomically and ignore task file
corruptions

* fixing weird checkstyle lambda indentation issues
2019-06-18 09:18:43 -07:00
SandishKumarHN 01881e3a98 Use only com.google.errorprone.annotations.concurrent.GuardedBy, not javax.annotations.concurrent.GuardedBy (#7889) 2019-06-17 15:58:51 +02:00
litao91 daed1bc141 Fix HttpRemoteTaskRunnerConfig not loading (#7902) 2019-06-16 00:31:42 -07:00
Sashidhar Thallam 3bee6adcf7 Use map.putIfAbsent() or map.computeIfAbsent() as appropriate instead of containsKey() + put() (#7764)
* https://github.com/apache/incubator-druid/issues/7316 Use Map.putIfAbsent() instead of containsKey() + put()

* fixing indentation

* Using map.computeIfAbsent() instead of map.putIfAbsent() where appropriate

* fixing checkstyle

* Changing the recommendation text

* Reverting auto changes made by IDE

* Implementing recommendation: A ConcurrentHashMap on which computeIfAbsent() is called should be assigned into variables of ConcurrentHashMap type, not ConcurrentMap

* Removing unused import
2019-06-14 17:59:36 +02:00
Clint Wylie 3fbb0a5e00 Supervisor list api with states and health (#7839)
* allow optionally listing all supervisors with their state and health

* docs

* add state to full

* clean

* casing

* format

* spelling
2019-06-07 16:26:33 -07:00
Surekha ea752ef562 Optimize overshadowed segments computation (#7595)
* Move the overshadowed segment computation to SQLMetadataSegmentManager's poll

* rename method in MetadataSegmentManager

* Fix tests

* PR comments

* PR comments

* PR comments

* fix indentation

* fix tests

*  fix test

*  add test for SegmentWithOvershadowedStatus serde format

* PR comments

* PR comments

* fix test

* remove snapshot updates outside poll

* PR comments

* PR comments

* PR comments

*  removed unused import
2019-06-07 19:15:54 +02:00
Jihoon Son 61ec521135
Remove keepSegmentGranularity option for compaction (#7747)
* Remove keepSegmentGranularity option from compaction

* fix it test

* clean up

* remove from web console

* fix test
2019-06-03 12:59:15 -07:00
Justin Borromeo 8032c4add8 Add errors and state to stream supervisor status API endpoint (#7428)
* Add state and error tracking for seekable stream supervisors

* Fixed nits in docs

* Made inner class static and updated spec test with jackson inject

* Review changes

* Remove redundant config param in supervisor

* Style

* Applied some of Jon's recommendations

* Add transience field

* write test

* implement code review changes except for reconsidering logic of markRunFinishedAndEvaluateHealth()

* remove transience reporting and fix SeekableStreamSupervisorStateManager impl

* move call to stateManager.markRunFinished() from RunNotice to runInternal() for tests

* remove stateHistory because it wasn't adding much value, some fixes, and add more tests

* fix tests

* code review changes and add HTTP health check status

* fix test failure

* refactor to split into a generic SupervisorStateManager and a specific SeekableStreamSupervisorStateManager

* fixup after merge

* code review changes - add additional docs

* cleanup KafkaIndexTaskTest

* add additional documentation for Kinesis indexing

* remove unused throws class
2019-05-31 17:16:01 -07:00
Jihoon Son 7abfbb066a Bump up snapshot version to 0.16.0 (#7802) 2019-05-30 17:17:33 -07:00
Roman Leventov 782863ed0f Fix some problems reported by PVS-Studio (#7738)
* Fix some problems reported by PVS-Studio

* Address comments
2019-05-29 11:20:45 -07:00
Jonathan Wei c97eb93e16 Fix dead code in IndexTask.collectIntervalsAndShardSpecs() (#7746) 2019-05-25 10:12:14 -07:00
Bartosz Ługowski cbdac49ab3 Web console - add enable/disable actions for middle manager workers (#7642)
* Overlord console - add enable/disable button for remote workers.

* Overlord console - add proxy for remote workers API.

* WorkerResourceTest - revert newline change.

* Remote worker proxy tests - remove empty line.

* Refactor remote worker proxy for readability and security

* Rename method in remote task runner tests for readability

* Remove enable/disable button for remote workers from old web console

* Add enable/disable actions for middle manager worker in new web console

* Fix variable type

* Add worker task runner query adapter

* Fix web console tests: segments-view, servers-view

* Fix overlord resource tests
2019-05-23 16:47:23 -07:00
Himanshu daf20b4b86 use memoized supplier for lazy singleton in SeekableStreamIndexTask.java (#7740) 2019-05-23 15:58:38 -07:00
Merlin Lee 26fad7e06a Add checkstyle for "Local variable names shouldn't start with capital" (#7681)
* Add checkstyle for "Local variable names shouldn't start with capital"

* Adjust some local variables to constants

* Replace StringUtils.LINE_SEPARATOR with System.lineSeparator()
2019-05-23 18:40:28 +02:00
Jihoon Son eff2be4f8f Remove LegacyKafkaIndexTaskRunner (#7735) 2019-05-23 09:25:35 -07:00
Gian Merlino 53b6467fc8 SeekableStreamIndexTaskRunner: Lazy init of runner. (#7729)
The main motivation is that this fixes #7724, by making it so the overlord
doesn't try to create a task runner and parser when all it really wants to
do is create a task object and serialize it.
2019-05-22 21:13:57 -07:00
David Lim d38457933f Data loader (sampler component) - Kafka/Kinesis samplers (#7566)
* implement Kafka/Kinesis sampler

* add KafkaSamplerSpecTest and KinesisSamplerSpecTest

* code review changes
2019-05-16 20:26:23 -07:00
Fokko Driesprong 2aa9613bed Bump Checkstyle to 8.20 (#7651)
* Bump Checkstyle to 8.20

Moderate severity vulnerability that affects:
com.puppycrawl.tools:checkstyle

Checkstyle prior to 8.18 loads external DTDs by default,
which can potentially lead to denial of service attacks
or the leaking of confidential information.

Affected versions: < 8.18

* Oops, missed one

* Oops, missed a few
2019-05-14 11:53:37 -07:00
Xavier Léauté b3c7463059
fix test pattern matching for jdk11 (#7602)
This change makes the error pattern more generic to account for
differences between JDK versions.
2019-05-06 21:02:44 -07:00
David Lim ec8562c885 Data loader (sampler component) (#7531)
* sampler initial check-in
fix checkstyle issues
add sampler fix to process CSV files from cache properly
change to composition and rename some classes
add tests and report num rows read and indexed
remove excludedByFilter flag and don't send filtered out data
fix tests to handle both settings for druid.generic.useDefaultValueForNull

* wrap sampler firehose in TimedShutoffFirehoseFactory to support timeouts

* code review changes - add additional comments, limit maxRows
2019-05-01 22:37:14 -07:00
Jihoon Son c60e7feab8 Fix encoded taskId check in chatHandlerResource (#7520)
* Fix encoded taskId check in chatHandlerResource

* fix tests
2019-04-20 18:08:34 -07:00
Gian Merlino 1fb5ec3989 Adds backwards-compatible serde for SeekableStreamStartSequenceNumbers. (#7512)
This allows them to be deserialized by older Druid versions as
KafkaPartitions objects.

Fixes #7470.
2019-04-19 13:19:45 -07:00
Jihoon Son 625c030744 Fix ParallelIndexTuningConfig constructor (#7479)
* Fix ParallelIndexTuningConfig constructor

* unused import
2019-04-15 17:41:53 -07:00
Lucas Capistrant 8acad27d99 Enhance the Http Firehose to work with URIs requiring basic authentication (#7145)
* Enhnace the HttpFirehose to work with both insecure URIs and URIs requiring basic authentication

* Improve security of enhanced HttpFirehoseFactory by not logging auth credentials

* Fix checkstyle failure in HttpFirehoseFactory.java

* Update docs and fix TeamCity build with required noinspection

* Indentation cleanup and logic modification for HttpFirehose object stream

* Remove default Empty string password provider in http firehose

* Add JavaDoc for MixIn describing its intended use

* Reverting documentation notation for json code to be inline with rest of doc

* Improve instantiation of ObjectMappers that require MixIn for redacting password from task logs

* Add comment to clarify fully qualified references of Objects in SQLMetadataStorageActionHandler
2019-04-15 14:29:01 -07:00
Mingming Qiu 7a38d28cf3 run pending tasks when assigned a task that is already pending (#6991)
* run pending tasks when assigned a task that is already pending

* add unit test

* fix pending tasks aren't going to run

* address comments
2019-04-14 23:37:15 +08:00
Eugene Sevastyanov c57e62bcd6 7153: Isolated a supplier's close by removing its invocation (#7459)
from a non-notice-processing thread
2019-04-12 19:13:08 -07:00
Justin Borromeo 2771ed50b0 Support Kafka supervisor adopting running tasks between versions (#7212)
* Recompute hash in isTaskCurrent() and added tests

* Fixed checkstyle stuff

* Fixed failing tests

* Make TestableKafkaSupervisorWithCustomIsTaskCurrent static

* Add doc

* baseSequenceName change

* Added comment

* WIP

* Fixed imports

* Undid lambda change for diff sake

* Cleanup

* Added comment

* Reinsert Kafka tests

* Readded kinesis test

* Readd bad partition assignment in kinesis supervisor test

* Nit

* Misnamed var
2019-04-10 18:16:38 -07:00
lxqfy 6b741f0800 Overwrite index task maxTotalRows with computed maxRowsPerSegments (#7338)
fixed identation

set maxTotalRows to higher value in case mutiple segments in appendrator
with comments
2019-04-03 11:47:59 -07:00
David Glasser 4e23c11345 Make IngestSegmentFirehoseFactory splittable for parallel ingestion (#7048)
* Make IngestSegmentFirehoseFactory splittable for parallel ingestion

* Code review feedback

- Get rid of WindowedSegment
- Don't document 'segments' parameter or support splitting firehoses that use it
- Require 'intervals' in WindowedSegmentId (since it won't be written by hand)

* Add missing @JsonProperty

* Integration test passes

* Add unit test

* Remove two FIXME comments from CompactionTask

I'd like to leave this PR in a potentially mergeable state, but I still would
appreciate reviewer eyes on the questions I'm removing here.

* Updates from code review
2019-04-02 14:59:17 -07:00
Charles Allen eeb3dbe79d Move GCP to a core extension (#6953)
* Move GCP to a core extension

* Don't provide druid-core >.<

* Keep AWS and GCP modules separate

* Move AWSModule to its own module

* Add aws ec2 extension and more modules in more places

* Fix bad imports

* Fix test jackson module

* Include AWS and GCP core in server

* Add simple empty method comment

* Update version to 15

* One more 0.13.0-->0.15.0 change

* Fix multi-binding problem

* Grep for s3-extensions and update docs

* Update extensions.md
2019-03-27 09:00:43 -07:00
Jihoon Son 5294277cb4
Fix exclusive start partitions for sequenceMetadata (#7339)
* Fix exclusvie start partitions for sequenceMetadata

* add empty check
2019-03-26 14:39:07 -07:00
Roman Leventov bca40dcdaf
Fix some IntelliJ inspections (#7273)
Prepare TeamCity for IntelliJ 2018.3.1 upgrade. Mostly removed redundant exceptions declarations in `throws` clauses.
2019-03-25 21:11:01 -03:00
Jihoon Son 0c5dcf5586 Fix exclusivity for start offset in kinesis indexing service & check exclusivity properly in IndexerSQLMetadataStorageCoordinator (#7291)
* Fix exclusivity for start offset in kinesis indexing service

* some adjustment

* Fix SeekableStreamDataSourceMetadata

* Add missing javadocs

* Add missing comments and unit test

* fix SeekableStreamStartSequenceNumbers.plus and add comments

* remove extra exclusivePartitions in KafkaIOConfig and fix downgrade issue

* Add javadocs

* fix compilation

* fix test

* remove unused variable
2019-03-21 13:12:22 -07:00
Surekha e170203876 Consolidate kafka consumer configs (#7249)
* Consolidate kafka consumer configs

* change the order of adding properties

* Add consumer properties to fix test

it seems kafka consumer does not reveive any message without these configs

* Use KafkaConsumerConfigs in integration test

* Update zookeeper and kafka versions in the setup.sh for the base druid image

*  use version 0.2 of base druid image

* Try to fix tests in KafkaRecordSupplierTest

* unused import

* Fix tests in KafkaSupervisorTest
2019-03-21 11:19:49 -07:00
Roman Leventov dfd27e00c0
Avoid many unnecessary materializations of collections of 'all segments in cluster' cardinality (#7185)
* Avoid many  unnecessary materializations of collections of 'all segments in cluster' cardinality

* Fix DruidCoordinatorTest; Renamed DruidCoordinator.getReplicationStatus() to computeUnderReplicationCountsPerDataSourcePerTier()

* More Javadocs, typos, refactor DruidCoordinatorRuntimeParams.createAvailableSegmentsSet()

* Style

* typo

* Disable StaticPseudoFunctionalStyleMethod inspection because of too much false positives

* Fixes
2019-03-19 18:22:56 -03:00
Jihoon Son 892d1d35d6
Deprecate NoneShardSpec and drop support for automatic segment merge (#6883)
* Deprecate noneShardSpec

* clean up noneShardSpec constructor

* revert unnecessary change

* Deprecate mergeTask

* add more doc

* remove convert from indexMerger

* Remove mergeTask

* remove HadoopDruidConverterConfig

* fix build

* fix build

* fix teamcity

* fix teamcity

* fix ServerModule

* fix compilation

* fix compilation
2019-03-15 23:29:25 -07:00
Gian Merlino a8c7132482 Logic adjustments to SeekableStreamIndexTaskRunner. (#7267)
* Logic adjustments to SeekableStreamIndexTaskRunner.

A mix of simplifications and bug fixes. They are intermingled because
some of the bugs were made difficult to fix, and also more likely to
happen in the first place, by how the code was structured. I tried to
keep restructuring to a minimum. The changes are:

- Remove "initialOffsetsSnapshot", which was used to determine when to
  skip start offsets. Replace it with "lastReadOffsets", which I hope
  is more intuitive. (There is a connection: start offsets must be
  skipped if and only if they have already been read, either by a
  previous task or by a previous sequence in the same task, post-restoring.)
- Remove "isStartingSequenceOffsetsExclusive", because it should always
  be the opposite of isEndOffsetExclusive. The reason is that starts are
  exclusive exactly when the prior ends are inclusive: they must match
  up in that way for adjacent reads to link up properly.
- Don't call "seekToStartingSequence" after the initial seek. There is
  no reason to, since we expect to read continuous message streams
  throughout the task. And calling it makes offset-tracking logic
  trickier, so better to avoid the need for trickiness. I believe the
  call being here was causing a bug in Kinesis ingestion where a
  message might get double-read.
- Remove the "continue" calls in the main read loop. They are bad
  because they prevent keeping currOffsets and lastReadOffsets up to
  date, and prevent us from detecting that we have finished reading.
- Rework "verifyInitialRecordAndSkipExclusivePartition" into
  "verifyRecordInRange". It no longer has side effects. It does a sanity
  check on the message offset and also makes sure that it is not past
  the endOffsets.
- Rework "assignPartitions" to replace inline comparisons with
  "isRecordAlreadyRead" and "isMoreToReadBeforeReadingRecord" calls. I
  believe this fixes an off-by-one error with Kinesis where the last
  record would not get read. It also makes the logic easier to read.
- When doing the final publish, only adjust end offsets of the final
  sequence, rather than potentially adjusting any unpublished sequence.
  Adjusting sequences other than the last one is a mistake since it
  will extend their endOffsets beyond what they actually read. (I'm not
  sure if this was an issue in practice, since I'm not sure if real
  world situations would have more than one unpublished sequence.)
- Rename "isEndSequenceOffsetsExclusive" to "isEndOffsetExclusive". It's
  shorter and more clear, I think.
- Add equals/hashCode/toString methods to OrderedSequenceNumber.

Kafka test changes:

- Added a Kafka "testRestoreAtEndOffset" test to verify that restores at
  the very end of the task lifecycle still work properly.

Kinesis test changes:

- Renamed "testRunOnNothing" to "testRunOnSingletonRange". I think that
  given Kinesis semantics, the right behavior when start offset equals
  end offset (and there aren't exclusive partitions set) is to read that
  single offset. This is because they are both meant to be treated as
  inclusive.
- Adjusted "testRestoreAfterPersistingSequences" to expect one more
  message read. I believe the old test was wrong; it expected the task
  not to read message number 5.
- Adjusted "testRunContextSequenceAheadOfStartingOffsets" to use a
  checkpoint starting from 1 rather than 2. I believe the old test was
  wrong here too; it was expecting the task to start reading from the
  checkpointed offset, but it actually should have started reading from
  one past the checkpointed offset.
- Adjusted "testIncrementalHandOffReadsThroughEndOffsets" to expect
  11 messages read instead of 12. It's starting at message 0 and reading
  up to 10, which should be 11 messages.

* Changes from code review.
2019-03-15 00:22:42 -07:00
Furkan KAMACI 7ada1c49f9 Prohibit Throwables.propagate() (#7121)
* Throw caught exception.

* Throw caught exceptions.

* Related checkstyle rule is added to prevent further bugs.

* RuntimeException() is used instead of Throwables.propagate().

* Missing import is added.

* Throwables are propogated if possible.

* Throwables are propogated if possible.

* Throwables are propogated if possible.

* Throwables are propogated if possible.

* * Checkstyle definition is improved.
* Throwables.propagate() usages are removed.

* Checkstyle pattern is changed for only scanning "Throwables.propagate(" instead of checking lookbehind.

* Throwable is kept before firing a Runtime Exception.

* Fix unused assignments.
2019-03-14 18:28:33 -03:00
Clint Wylie fb1489d313 fix SequenceMetadata deserialization (#7256)
* wip

* fix tests, stop reading if we are at end offset

* fix build

* remove restore at end offsets fix in favor of a separate PR

* use typereference from method for serialization too
2019-03-13 17:29:39 -04:00
Jihoon Son 873232954f Fix log level and throw NPE on null currOffset in SeekableStreamIndexTaskRunner (#7253) 2019-03-13 10:20:43 -04:00
Jihoon Son 32e86ea75e Fix record validation in SeekableStreamIndexTaskRunner (#7246)
* Fix record validation in SeekableStreamIndexTaskRunner

* add kinesis test
2019-03-12 21:12:21 -07:00
Gian Merlino dcfca03718
More accurate RealtimeMetricsMonitor messages. (#7230)
The old messages did not reflect the full range of reasons why messages
could be thrown away.
2019-03-11 19:50:32 -04:00
Jihoon Son e48a9c138e Reduce default max # of subTasks to 1 for native parallel task (#7181)
* Reduce # of max subTasks to 2

* fix typo and add more doc

* add more doc and link

* change default and add warning

* fix doc

* add test

* fix it test
2019-03-05 22:06:36 -08:00
Gian Merlino fa218f5160 Fix two SeekableStream serde issues. (#7176)
* Fix two SeekableStream serde issues.

1) Fix backwards-compatibility serde for SeekableStreamPartitions. It is needed
   for split 0.13 / 0.14 clusters to work properly during a rolling update.
2) Abstract classes don't need JsonCreator constructors; remove them.

* Comment fixes.
2019-03-01 22:27:08 -08:00
Jihoon Son 06c8229c08
Kill all running tasks when the supervisor task is killed (#7041)
* Kill all running tasks when the supervisor task is killed

* add some docs and simplify

* address comment
2019-03-01 11:28:03 -08:00
Jihoon Son cacdc83cad Improve error message for integer overflow in compaction task (#7131)
* improve error message for integer overflow in compaction task

* fix build
2019-02-28 11:07:37 +08:00
Himanshu Pandey 8b803cbc22 Added checkstyle for "Methods starting with Capital Letters" (#7118)
* Added checkstyle for "Methods starting with Capital Letters" and changed the method names violating this.

* Un-abbreviate the method names in the calcite tests

* Fixed checkstyle errors

* Changed asserts position in the code
2019-02-23 20:10:31 -08:00
David Glasser 1c2753ab90 ParallelIndexSubTask: support ingestSegment in delegating factories (#7089)
IndexTask had special-cased code to properly send a TaskToolbox to a
IngestSegmentFirehoseFactory that's nested inside a CombiningFirehoseFactory,
but ParallelIndexSubTask didn't.

This change refactors IngestSegmentFirehoseFactory so that it doesn't need a
TaskToolbox; it instead gets a CoordinatorClient and a SegmentLoaderFactory
directly injected into it.

This also refactors SegmentLoaderFactory so it doesn't depend on
an injectable SegmentLoaderConfig, since its only method always
replaces the preconfigured SegmentLoaderConfig anyway.
This makes it possible to use SegmentLoaderFactory without setting
druid.segmentCaches.locations to some dummy value.

Another goal of this PR is to make it possible for IngestSegmentFirehoseFactory
to list data segments outside of connect() --- specifically, to make it a
FiniteFirehoseFactory which can query the coordinator in order to calculate its
splits. See #7048.

This also adds missing datasource name URL-encoding to an API used by
CoordinatorBasedSegmentHandoffNotifier.
2019-02-23 17:02:56 -08:00
Jihoon Son 4e2b085201
Remove DataSegmentFinder, InsertSegmentToDb, and descriptor.json file in deep storage (#6911)
* Remove DataSegmentFinder, InsertSegmentToDb, and descriptor.json file

* delete descriptor.file when killing segments

* fix test

* Add doc for ha

* improve warning
2019-02-20 15:10:29 -08:00
David Glasser a81b1b8c9c index_parallel: support !appendToExisting with no explicit intervals (#7046)
* index_parallel: support !appendToExisting with no explicit intervals

This enables ParallelIndexSupervisorTask to dynamically request locks at runtime
if it is run without explicit intervals in the granularity spec and with
appendToExisting set to false.  Previously, it behaved as if appendToExisting
was set to true, which was undocumented and inconsistent with IndexTask and
Hadoop indexing.

Also, when ParallelIndexSupervisorTask allocates segments in the explicit
interval case, fail if its locks on the interval have been revoked.

Also make a few other additions/clarifications to native ingestion docs.

Fixes #6989.

* Review feedback.

PR description on GitHub updated to match.

* Make native batch ingestion partitions start at 0

* Fix to previous commit

* Unit test. Verified to fail without the other commits on this branch.

* Another round of review

* Slightly scarier warning
2019-02-20 10:54:26 -08:00
Clint Wylie cadb6c5280 Missing Overlord and MiddleManager api docs (#7042)
* document middle manager api

* re-arrange

* correction

* document more missing overlord api calls, minor re-arrange of some code i was referencing

* fix it

* this will fix it

* fixup

* link to other docs
2019-02-19 10:52:05 -08:00
Surekha 80a2ef7be4 Support kafka transactional topics (#5404) (#6496)
* Support kafka transactional topics

* update kafka to version 2.0.0
* Remove the skipOffsetGaps option since it's not used anymore
* Adjust kafka consumer to use transactional semantics
* Update tests

* Remove unused import from test

* Fix compilation

* Invoke transaction api to fix a unit test

* temporary modification of travis.yml for debugging

* another attempt to get travis tasklogs

* update kafka to 2.0.1 at all places

* Remove druid-kafka-eight dependency from integration-tests, remove the kafka firehose test and deprecate kafka-eight classes

* Add deprecated in docs for kafka-eight and kafka-simple extensions

* Remove skipOffsetGaps and code changes for transaction support

* Fix indentation

* remove skipOffsetGaps from kinesis

* Add transaction api to KafkaRecordSupplierTest

* Fix indent

* Fix test

* update kafka version to 2.1.0
2019-02-18 11:50:08 -08:00
Jihoon Son 1701fbcad3
Improve error message for revoked locks (#7035)
* Improve error message for revoked locks

* fix test

* fix test

* fix test

* fix toString
2019-02-13 11:22:48 -08:00
Mingming Qiu d0abf5c20a fix kafka index task doesn't resume when recieve duplicate request (#6990)
* fix kafka index task doesn't resume when recieve duplicate request

* add unit test
2019-02-12 13:24:28 -08:00
Ankit Kothari 16a4a50e91 [Issue #6967] NoClassDefFoundError when using druid-hdfs-storage (#7015)
* Fix:
  1. hadoop-common dependency for druid-hdfs and druid-kerberos extensions
 Refactoring:
  2. Hadoop config call in the inner static class to avoid class path conflicts for stopGracefully kill

* Fix:
  1. hadoop-common test dependency

* Fix:
  1. Avoid issue of kill command once the job is actually completed
2019-02-08 18:26:37 -08:00
Jonathan Wei fafbc4a80e
Set version to 0.15.0-incubating-SNAPSHOT (#7014) 2019-02-07 14:02:52 -08:00
Furkan KAMACI 58f9507ccf Improper equals override is fixed to prevent NullPointerException (#6938)
* Improper equals override is fixed to prevent NullPointerException

* Fixed curly brace indentation.

* Test method is added for equals method of TaskLockPosse class.
2019-02-06 14:08:50 -08:00
Jonathan Wei 8bc5eaa908
Set version to 0.14.0-incubating-SNAPSHOT (#7003) 2019-02-04 19:36:20 -08:00
David Glasser 7e48593b57 ParallelIndexSupervisorTask: don't warn about a default value (#6987)
Native batch indexing doesn't yet support the maxParseExceptions,
maxSavedParseExceptions, and logParseExceptions tuning config options, so
ParallelIndexSupervisorTask logs if these are set. But the default value for
maxParseExceptions is Integer.MAX_VALUE, which means that you'll get the
maxParseExceptions flavor of this warning even if you don't configure
maxParseExceptions.

This PR changes all three warnings to occur if you change the settings from the
default; this mostly affects the maxParseExceptions warning.
2019-02-04 12:00:26 -08:00
Roman Leventov 0e926e8652 Prohibit assigning concurrent maps into Map-typed variables and fields and fix a race condition in CoordinatorRuleManager (#6898)
* Prohibit assigning concurrent maps into Map-types variables and fields; Fix a race condition in CoordinatorRuleManager; improve logic in DirectDruidClient and ResourcePool

* Enforce that if compute(), computeIfAbsent(), computeIfPresent() or merge() is called on a ConcurrentHashMap, it's stored in a ConcurrentHashMap-typed variable, not ConcurrentMap; add comments explaining get()-before-computeIfAbsent() optimization; refactor Counters; fix a race condition in Intialization.java

* Remove unnecessary comment

* Checkstyle

* Fix getFromExtensions()

* Add a reference to the comment about guarded computeIfAbsent() optimization; IdentityHashMap optimization

* Fix UriCacheGeneratorTest

* Workaround issue with MaterializedViewQueryQueryToolChest

* Strengthen Appenderator's contract regarding concurrency
2019-02-04 09:18:12 -08:00
Vadim Ogievetsky 7f1b19bfb1 Adding a Unified web console. (#6923)
* Adding new web console.

* fixed css

* fix form height

* fix typo

* do import custom react-table css

* added repo field so npm does not complain

* ask travis for node 10

* move indexing-service/src/main/resources/indexer_static into web-console

* fix resource names and paths

* add licenses

* fix exclude file

* add licenses to misc files and tidy up

* remove rebase marker

* fix link

* updated env variable name

* tidy up licenses and surface errors

* cleanup

* remove unused code, fix missing await

* TeamCity does not like the name aux

* add more links to tasks view

* rm pages

* update gitignore

* update readme to be accurate

* make clean script

* removed old console dependancy

* update Jetty routes

* add a comment for welcome files for coordinator

* do not show inital notifaction for now

* renamed overlord console back to console.html

* fix coordinator console

* rename coordinator-console.html to index.html
2019-01-31 17:26:41 -08:00
Benedict Jin 72a571fbf7 For performance reasons, use `java.util.Base64` instead of Base64 in Apache Commons Codec and Guava (#6913)
* * Add few methods about base64 into StringUtils

* Use `java.util.Base64` instead of others

* Add org.apache.commons.codec.binary.Base64 & com.google.common.io.BaseEncoding into druid-forbidden-apis

* Rename encodeBase64String & decodeBase64String

* Update druid-forbidden-apis
2019-01-25 17:32:29 -08:00
Ankit Kothari 8492d94f59 Kill Hadoop MR task on kill of Hadoop ingestion task (#6828)
* KillTask from overlord UI now makes sure that it terminates the underlying MR job, thus saving unnecessary compute

Run in jobby is now split into 2
 1. submitAndGetHadoopJobId followed by 2. run
  submitAndGetHadoopJobId is responsible for submitting the job and returning the jobId as a string, run monitors this job for completion

JobHelper writes this jobId in the path provided by HadoopIndexTask which in turn is provided by the ForkingTaskRunner

HadoopIndexTask reads this path when kill task is clicked to get hte jobId and fire the kill command via the yarn api. This is taken care in the stopGracefully method which is called in SingleTaskBackgroundRunner. Have enabled `canRestore` method to return `true` for HadoopIndexTask in order for the stopGracefully method to be called

Hadoop*Job files have been changed to incorporate the changes to jobby

* Addressing PR comments

* Addressing PR comments - Fix taskDir

* Addressing PR comments - For changing the contract of Task.stopGracefully()
`SingleTaskBackgroundRunner` calls stopGracefully in stop() and then checks for canRestore condition to return the status of the task

* Addressing PR comments
 1. Formatting
 2. Removing `submitAndGetHadoopJobId` from `Jobby` and calling writeJobIdToFile in the job itself

* Addressing PR comments
 1. POM change. Moving hadoop dependency to indexing-hadoop

* Addressing PR comments
 1. stopGracefully now accepts TaskConfig as a param
     Handling isRestoreOnRestart in stopGracefully for `AppenderatorDriverRealtimeIndexTask, RealtimeIndexTask, SeekableStreamIndexTask`
     Changing tests to make TaskConfig param isRestoreOnRestart to true
2019-01-25 15:43:06 -08:00
Himanshu Pandey e1033bb412 Issue#6892- Replaced Math.random() with ThreadLocalRandom.current().nextDouble() (#6914)
*  Replacing Math.random() with ThreadLocalRandom.current().nextDouble()

* Added java.lang.Math#random() in forbidden-apis.txt

* Minor change in the message - druid-forbidden-apis.txt
2019-01-25 19:49:20 +08:00
Roman Leventov 8eae26fd4e Introduce SegmentId class (#6370)
* Introduce SegmentId class

* tmp

* Fix SelectQueryRunnerTest

* Fix indentation

* Fixes

* Remove Comparators.inverse() tests

* Refinements

* Fix tests

* Fix more tests

* Remove duplicate DataSegmentTest, fixes #6064

* SegmentDescriptor doc

* Fix SQLMetadataStorageUpdaterJobHandler

* Fix DataSegment deserialization for ignoring id

* Add comments

* More comments

* Address more comments

* Fix compilation

* Restore segment2 in SystemSchemaTest according to a comment

* Fix style

* fix testServerSegmentsTable

* Fix compilation

* Add comments about why SegmentId and SegmentIdWithShardSpec are separate classes

* Fix SystemSchemaTest

* Fix style

* Compare SegmentDescriptor with SegmentId in Javadoc and comments rather than with DataSegment

* Remove a link, see https://youtrack.jetbrains.com/issue/IDEA-205164

* Fix compilation
2019-01-21 11:11:10 -08:00
Clint Wylie 8ba33b2505 add 'init' lifecycle stage for finer control over startup and shutdown (#6864)
* add Lifecycle.Stage.INIT, put log shutter downer in init stage, tests, rad startup banner

* log cleanup

* log changes

* add task-master lifecycle to module lifecycle to gracefully stop task-master stuff

* fix it the right way

* remove announce spam

* unused import

* one more log

* updated comments

* wrap leadership lifecycle stop to prevent exceptions from wrecking rest of task master stop

* add precondition check
2019-01-21 09:01:36 -08:00
Jonathan Wei 8537a771b0 Some fixes and tests for spaces/non-ASCII chars in datasource names (#6761)
* Fixes and tests for spaces/non-ASCII datasource names

* Some unit test fixes

* Fix ITRealtimeIndexTaskTest

* Checkstyle

* TeamCity

* PR comments
2019-01-15 08:33:31 -08:00
Charles Allen 5d2947cd52 Use Guava Compatible immediate executor service (#6815)
* Use multi-guava version friendly direct executor implementation

* Don't use a singleton

* Fix strict compliation complaints

* Copy Guava's DirectExecutor

* Fix javadoc

* Imports are the devil
2019-01-11 10:42:19 -08:00
Jihoon Son 39780d914c Propagate errors to RemoteTaskActionClient (#6837)
* Propagate errors to RemoteTaskActionClient

* style
2019-01-10 20:53:18 -08:00
Jihoon Son c35a39d70b
Add support maxRowsPerSegment for auto compaction (#6780)
* Add support maxRowsPerSegment for auto compaction

* fix build

* fix build

* fix teamcity

* add test

* fix test

* address comment
2019-01-10 09:50:14 -08:00
Jihoon Son 934c83bca6 Fix TaskLockbox when there are multiple intervals of the same start but differerent end (#6822)
* Fix TaskLockbox when there are multiple intervals of the same start but differernt end

* fix build

* fix npe
2019-01-09 19:38:27 -08:00
Jihoon Son c4716d1639 Fix ParallelIndexTask when publishing empty segments (#6807)
* Fix ParallelIndexTask when publishing empty segments

* unused import
2019-01-08 17:15:16 -08:00
Jihoon Son 9ad6a733a5 Add support segmentGranularity for CompactionTask (#6758)
* Add support segmentGranularity

* add doc and fix combination of options

* improve doc
2019-01-03 17:50:45 -08:00
Joshua Sun 7c7997e8a1 Add Kinesis Indexing Service to core Druid (#6431)
* created seekablestream classes

* created seekablestreamsupervisor class

* first attempt to integrate kafa indexing service to use SeekableStream

* seekablestream bug fixes

* kafkarecordsupplier

* integrated kafka indexing service with seekablestream

* implemented resume/suspend and refactored some package names

* moved kinesis indexing service into core druid extensions

* merged some changes from kafka supervisor race condition

* integrated kinesis-indexing-service with seekablestream

* unite tests for kinesis-indexing-service

* various bug fixes for kinesis-indexing-service

* refactored kinesisindexingtask

* finished up more kinesis unit tests

* more bug fixes for kinesis-indexing-service

* finsihed refactoring kinesis unit tests

* removed KinesisParititons and KafkaPartitions to use SeekableStreamPartitions

* kinesis-indexing-service code cleanup and docs

* merge #6291

merge #6337

merge #6383

* added more docs and reordered methods

* fixd kinesis tests after merging master and added docs in seekablestream

* fix various things from pr comment

* improve recordsupplier and add unit tests

* migrated to aws-java-sdk-kinesis

* merge changes from master

* fix pom files and forbiddenapi checks

* checkpoint JavaType bug fix

* fix pom and stuff

* disable checkpointing in kinesis

* fix kinesis sequence number null in closed shard

* merge changes from master

* fixes for kinesis tasks

* capitalized <partitionType, sequenceType>

* removed abstract class loggers

* conform to guava api restrictions

* add docker for travis other modules test

* address comments

* improve RecordSupplier to supply records in batch

* fix strict compile issue

* add test scope for localstack dependency

* kinesis indexing task refactoring

* comments

* github comments

* minor fix

* removed unneeded readme

* fix deserialization bug

* fix various bugs

* KinesisRecordSupplier unable to catch up to earliest position in stream bug fix

* minor changes to kinesis

* implement deaggregate for kinesis

* Merge remote-tracking branch 'upstream/master' into seekablestream

* fix kinesis offset discrepancy with kafka

* kinesis record supplier disable getPosition

* pr comments

* mock for kinesis tests and remove docker dependency for unit tests

* PR comments

* avg lag in kafkasupervisor #6587

* refacotred SequenceMetadata in taskRunners

* small fix

* more small fix

* recordsupplier resource leak

* revert .travis.yml formatting

* fix style

* kinesis docs

* doc part2

* more docs

* comments

* comments*2

* revert string replace changes

* comments

* teamcity

* comments part 1

* comments part 2

* comments part 3

* merge #6754

* fix injection binding

* comments

* KinesisRegion refactor

* comments part idk lol

* can't think of a commit msg anymore

* remove possiblyResetDataSourceMetadata() for IncrementalPublishingTaskRunner

* commmmmmmmmmments

* extra error handling in KinesisRecordSupplier getRecords

* comments

* quickfix

* typo

* oof
2018-12-21 12:49:24 -07:00
Clint Wylie 9505074530 fix log typo (#6755)
* fix log typo, add DataSegmentUtils.getIdentifiersString util method

* fix indecisive oops
2018-12-18 15:10:25 -08:00
Clint Wylie 486c6f3cf9 emit logs that are only useful for debugging at debug level (#6741)
* make logs that are only useful for debugging be at debug level so log volume is much more chill

* info level messages for total merge buffer allocated/free

* more chill compaction logs
2018-12-17 14:20:28 +08:00
Gian Merlino 04e7c7fbdc FilteredRequestLogger: Fix start/stop, invalid delegate behavior. (#6637)
* FilteredRequestLogger: Fix start/stop, invalid delegate behavior.

Fixes two bugs:

1) FilteredRequestLogger did not start/stop the delegate.

2) FilteredRequestLogger would ignore an invalid delegate type, and
instead silently substitute the "noop" logger. This was due to a larger
problem with RequestLoggerProvider setup in general; the fix here is
to remove "defaultImpl" from the RequestLoggerProvider interface, and
instead have JsonConfigurator be responsible for creating the
default implementations. It is stricter about things than the old system
was, and is only willing to make a noop logger if it doesn't see any
request logger configs. Otherwise, it'll raise a provision error.

* Remove unneeded annotations.
2018-12-14 16:55:44 +08:00
dongyifeng 91e3cf7196 add charset UTF-8 to log api (#6709)
When I retrieve the task log in browser, the Chinese characters all end up as garbage.
![image](https://user-images.githubusercontent.com/1322134/49502749-bd614080-f8b0-11e8-839e-07f7117eebfd.png)
After adding charset UTF-8, it was correct.
![image](https://user-images.githubusercontent.com/1322134/49502804-dc5fd280-f8b0-11e8-916b-bda8f1e7f318.png)
2018-12-12 16:31:04 +01:00
Jihoon Son f727333b70 Fix shutdownAllTasks API for non-existing dataSource (#6706) 2018-12-08 09:54:01 -08:00
Mingming Qiu 607339003b Add TaskCountStatsMonitor to monitor task count stats (#6657)
* Add TaskCountStatsMonitor to monitor task count stats

* address comments

* add file header

* tweak test
2018-12-04 13:37:17 -08:00
Clint Wylie a1c9d0add2 autosize processing buffers based on direct memory sizing by default (#6588)
* autosize processing buffers based on direct memory sizing

* remove oops, more test

* max 1gb autosize buffers, test, start of docs

* fix oops

* revert accidental change

* print buffer size in exception

* change the things
2018-12-03 18:40:02 -07:00
Clint Wylie 43adb391c2 remove AbstractResourceFilter.isApplicable because it is not (#6691)
* remove AbstractResourceFilter.isApplicable because it is not, add tests for OverlordResource.doShutdown and OverlordResource.shutdownTasksForDatasource

* cleanup
2018-12-01 21:52:31 +08:00
Roman Leventov ec38df7575
Simplify DruidNodeDiscoveryProvider; add DruidNodeDiscovery.Listener.nodeViewInitialized() (#6606)
* Simplify DruidNodeDiscoveryProvider; add DruidNodeDiscovery.Listener.nodeViewInitialized() method; prohibit and eliminate some suboptimal Java 8 patterns

* Fix style

* Fix HttpEmitterTest.timeoutEmptyQueue()

* Add DruidNodeDiscovery.Listener.nodeViewInitialized() calls in tests

* Clarify code
2018-12-01 01:12:56 +01:00
Jihoon Son d6539abd0a Fix overlord api and console (#6686)
* Fix overlord APIs and console

* remove getRunningTasksByDataSource

* add missing path to isApplicable
2018-11-29 23:45:28 -08:00
Mingming Qiu c81cb94226 fix cannot resolve param at OverlordResource#getTasks (#6679) 2018-11-29 10:07:21 +08:00
David Glasser d150483fd3 indexing-service: fix HTML title on overlord console (#6671)
This follows a similar fix to the body of the page done in #5627.
2018-11-27 22:34:26 -08:00
Jihoon Son 422b76b33c Fix IndexTaskClient to retry on ChannelException (#6649)
* Fix IndexTaskClient to retry on ChannelException

* fix travis and add javadoc

* address comment
2018-11-27 15:54:38 -08:00
seoeun 22a5bf97a2 Fix issue that tasks tables in metadata storage are not cleared (#6592)
* tasks tables in metadata storage are not cleared

* address comments. remove tasklogs and revert obsolete changes

* address comments. change comment and update doc.

* address comments. update doc more detailed

* address comments. remove redundant log and update doc more detailed.

* address comments. update document
2018-11-22 11:50:31 +08:00
Marat 81c9a6177c Added support for filtering by unused parameter for HeapMemoryTaskStorage (#6510)
* 1. added support for unused DateTime start parameter in getRecentlyFinishedTaskInfoSince method:
 HeapMemoryTaskStorage.getRecentlyFinishedTaskInfoSince return the finished tasks by comparing TaskStuff.createdDate with the start time
2. added filtering by status complete to TaskStuff list stream in HeapMemoryTaskStorage.getNRecentlyFinishedTaskInfo method.
3. changed names of methods and parameters to present that public API method OverlordResource.getTasks return the list of completed tasks, which createdDate, not date of completion, belongs to the interval parameter.

* 1. added support for unused DateTime start parameter in getRecentlyFinishedTaskInfoSince method:
 HeapMemoryTaskStorage.getRecentlyFinishedTaskInfoSince return the finished tasks by comparing TaskStuff.createdDate with the start time
2. added filtering by status complete to TaskStuff list stream in HeapMemoryTaskStorage.getNRecentlyFinishedTaskInfo method.
3. changed names of methods and parameters to present that public API method OverlordResource.getTasks return the list of completed tasks, which createdDate, not date of completion, belongs to the interval parameter.

* Fixed OverlordResourceTest to Support changed methods names

* Changed methods and parameters names to make them more obvious to understand.

* Changed String.replace() for the StringUtils.replace()(#6607)

* Fixed checkstyle error
2018-11-20 13:42:44 -08:00
Roman Leventov 87b96fb1fd
Add checkstyle rules about imports and empty lines between members (#6543)
* Add checkstyle rules about imports and empty lines between members

* Add suppressions

* Update Eclipse import order

* Add empty line

* Fix StatsDEmitter
2018-11-20 12:42:15 +01:00
Jihoon Son d738ce4d2a Enforce logging when killing a task (#6621)
* Enforce logging when killing a task

* fix test

* address comment

* address comment
2018-11-16 10:01:56 +08:00
Roman Leventov 8f3fe9cd02 Prohibit String.replace() and String.replaceAll(), fix and prohibit some toString()-related redundancies (#6607)
* Prohibit String.replace() and String.replaceAll(), fix and prohibit some toString()-related redundancies

* Fix bug

* Replace checkstyle regexp with IntelliJ inspection
2018-11-15 13:21:34 -08:00
QiuMM f2b73f9df1 fix cannot resolve param at OverlordResource#getTasks (#6593) 2018-11-13 09:47:11 -08:00
Roman Leventov 54351a5c75 Fix various bugs; Enable more IntelliJ inspections and update error-prone (#6490)
* Fix various bugs; Enable more IntelliJ inspections and update error-prone

* Fix NPE

* Fix inspections

* Remove unused imports
2018-11-06 14:38:08 -08: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
Clint Wylie ee1fc93f97 fix exception in Supervisor.start causing overlord unable to become leader (#6516)
* fix exception thrown by Supervisor.start causing overlord unable to become leader

* fix style
2018-10-25 15:44:04 -07:00
Clint Wylie e1057ad47a Fix NPE in TaskLockbox that prevents overlord leadership (#6512)
* fix NPE that prevents overlord from assuming leadership if extension that provides indexing task type is not loaded

* heh
2018-10-25 13:06:11 -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
David Lim e1a53fd17a fix distribution to not include contrib extensions by default, don't pull the entire AWS SDK bundle (#6494) 2018-10-19 13:50:05 -07:00
Joshua Sun b662fe84c5 fix TaskRunnerUtils String formatting issue (#6492)
* fix TaskRunnerUtils String formatting issue

* additional fixes
2018-10-18 19:16:46 -06: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
David Lim 20ab213ba6 change project versions to 0.13.0-incubating-SNAPSHOT (#6453) 2018-10-11 19:28:01 -07:00
QiuMM f8f4526b16 Add suspend|resume|terminate all supervisors endpoints. (#6272)
* ability to showdown all supervisors

* add doc

* address comments

* fix code style

* address comments

* change ternary assignment to if statement

* better docs
2018-10-10 21:41:59 -07:00
Jihoon Son 9343cbc63a Fix CompactionTask to consider only latest segments (#6429)
* CompactionTask should consider only latest segments

* fix test
2018-10-08 21:53:16 -07:00
Jihoon Son 2b76d57347 Fail compactionTask if it fails to run one of indexTaskSpecs (#6428)
* Fail compactionTask if it fails to run one of indexTaskSpecs

* add log
2018-10-08 08:53:32 -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
Jonathan Wei c7ac8785a1 Prevent failed KafkaConsumer creation from blocking overlord startup (#6383)
* Prevent failed KafkaConsumer creation from blocking overlord startup

* PR comments

* Fix random task ID length

* Adjust test timer

* Use Integer.SIZE
2018-10-03 19:08:20 -07: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
Surekha 42e5385e56 make 0.13 tasks API backwards compatible with 0.12 (#6333) (#6334)
* Replace statusCode with status (#6333)

Also changed runnerStatusCode to runnerStatus to keep things consistent

* Add unit test

* Add status param to TaskStatusPlus

Revert to statusCode and runnerStatusCode

* Add additional status member to TaskStatusPlus

* Change TaskResponseObject to match overlord's response object

* Address PR comments

* address comments

* Add runtime exception after logging error

* Remove (deprecated)status member variable from TaskStatusPlus

* Minor change
2018-10-01 15:33:24 -07: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
dyf6372 63ba7f7bec overlord check task whether is present before get lock (#6308) 2018-09-28 16:57:40 -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
Jihoon Son 6fb503c073 Deprecate task audit logging (#6368)
* Deprecate task audit logging

* fix test

* fix it test
2018-09-26 16:28:02 -07:00
Clint Wylie 399a5659b2 fix incorrect precondition check in `SupervisorManager.suspendOrResumeSupervisor` (#6364)
This check is reverse from the intention
2018-09-21 17:40:14 -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
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
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
Himanshu c3aaf8122d fix TaskQueue-HRTR deadlock (#6212)
* fix TaskQueue-HRTR deadlock causing https://github.com/apache/incubator-druid/issues/6201

* address review comments
2018-08-25 14:15:57 -07:00