Commit Graph

111 Commits

Author SHA1 Message Date
Jihoon Son c48aa74a30 Fix NPE while handling CheckpointNotice in KafkaSupervisor (#5996)
* Fix NPE while handling CheckpointNotice

* fix code style

* Fix test

* fix test

* add a log for creating a new taskGroup

* fix backward compatibility in KafkaIOConfig
2018-07-13 17:14:57 -07:00
Gian Merlino 04ea3c9f8c
Update license headers. (#5976)
* Update license headers.

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

* More license adjustments.

* Fix mistakenly edited package line.
2018-07-11 09:55:18 -07:00
Gian Merlino 948e73da77 Extend various test timeouts. (#5978)
False failures on Travis due to spurious timeout (in turn due to noisy
neighbors) is a bigger problem than legitimate failures taking too long
to time out. So it makes sense to extend timeouts.
2018-07-10 13:02:14 -07:00
Jihoon Son b6c957b0d2 Allow reordered segment allocation in kafka indexing service (#5805)
* Allow reordered segment allocation in kafka indexing service

* address comments

* fix a bug
2018-07-02 15:09:12 -07:00
Charles Allen 8dc4aca25f
Add cgroup memory monitor (#5866)
* Add cgroup memory monitor

* Port of https://github.com/metamx/java-util/pull/67

* Fix copyright

* Don't use `String.format`
2018-06-18 10:03:44 -07:00
Dylan Wylie 8c6651022d Update jsonpath dependency (#5794)
* Update JSONPath Library

Re: #5792

- Add a unit test containing a JSONPath conditional
- Update the JSONPath library and no longer exclude the json-smart dependency.
- I believe the original reason for excluding this has been fixed: https://github.com/json-path/JsonPath/pull/315

* Add test

* Fix test
2018-06-15 13:50:48 -07:00
Joseph Glanville 1032387d78 Snappy decompression support (#5864)
Support decompression of files using Google's Snappy algorithm.

This only supports files compressed with the Snappy framing format
described here: https://github.com/google/snappy/blob/master/framing_format.txt
2018-06-14 10:55:42 +01:00
Jonathan Wei bc9da54e12
Fix Zip Slip vulnerability (#5850)
* Fix evil zip exploit

* PR comment, checkstyle

* PR comments

* Add link to vulnerability report

* Fix test
2018-06-12 10:03:08 -07:00
Jihoon Son 9dca5ec76b Simple cleanup for ThreadPoolTaskRunner and SetAndVerifyContextQueryRunner / Add ThreadPoolTaskRunnerTest (#5557)
* Simple fix for ThreadPoolTaskRunner

* fix build

* address comments

* update javadoc

* fix build

* fix test

* add dependency
2018-05-15 22:53:11 +05:30
kaijianding 4db9e39a71 fix NPE when buffersList contains null in SmooshedFileMapper (#5689)
* fix NPE when buffersList contains null

* address the comment
2018-04-27 18:15:04 -07:00
Roman Leventov a3a9ada843 Add GenericWhitespace checkstyle check (#5668) 2018-04-24 01:09:14 +05:30
Roman Leventov 124c89e435 Replace EmittedBatchCounter and UpdateCounter with ConcurrentAwaitableCounter (#5592)
* Replace EmittedBatchCounter and UpdateCounter with (both not safe for concurrent increments/updates) with ConcurrentAwaitableCounter (safe for concurrent increments)

* Fixes

* Fix EmitterTest

* Added Javadoc and make awaitCount() to throw exceptions on wrong count instead of masking errors
2018-04-13 00:07:11 -04:00
Atul Mohan 19f359957f Add getters for AlertEvent (#5522)
* Add getters for AlertEvent

* Move PublicApi and ExtensionPoint to java-util

* Fix publicapi annotation usage

* Add publicapi annotations to ServiceMetricEvent and RequestLogEvent
2018-04-12 23:38:20 +07:00
Gian Merlino ee37ff2502 CompressionUtils: Make gzipInputStream public once again. (#5590)
But add a reference to "decompress" and mention that it's preferred
when reading from streams that come from files.
2018-04-09 17:39:19 -07:00
Gian Merlino 5ab17668c0 CompressionUtils: Add support for decompressing xz, bz2, zip. (#5586)
Also switch various firehoses to the new method.

Fixes #5585.
2018-04-06 08:06:45 -07:00
Jonathan Wei 969342cd28
More error reporting and stats for ingestion tasks (#5418)
* Add more indexing task status and error reporting

* PR comments, add support in AppenderatorDriverRealtimeIndexTask

* Use TaskReport instead of metrics/context

* Fix tests

* Use TaskReport uploads

* Refactor fire department metrics retrieval

* Refactor input row serde in hadoop task

* Refactor hadoop task loader names

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

* PR comments
2018-04-05 21:38:57 -07:00
Kirill Kozlov 8878a7ff94 Replace guava Charsets with native java StandardCharsets (#5545) 2018-03-28 21:00:08 -07:00
Charles Allen 58f110f7f8 Future-proof some Guava usage (#5414)
* Future-proof some Guava usage

* Use a java-util EmptyIterator instead of Guava's
* Change some of the guava future handling to do manual async
transforms. Guava changes transform into transformAsync by deprecating
transform in ONLY Guava 19. Then its gone in 20

* Use `Collections.emptyIterator()`

* Pretty formatting

* Make listenable future transforms a thing in default druid

* Format fix

* Add forbidden guava apis

* Make the ListenableFutrues.transformAsync have comments

* Undo intellij bad pattern matching in comments

* Futrues --> Futures

* Add empty iterators forbidding

* Fix extra `A`

* Correct method signature

* Address review comments

* Finish Gian review comments

* Proper syntax from https://github.com/policeman-tools/forbidden-apis/wiki/SignaturesSyntax
2018-03-20 08:59:33 -07:00
Roman Leventov 693e3575f9
Remove unused code and exception declarations (#5461)
* Remove unused code and exception declarations

* Address comments

* Remove redundant Exception declarations

* Make FirehoseFactoryV2.connect() to throw IOException again
2018-03-16 22:11:12 +01:00
bolkedebruin 7d1163b0d9 Optimize chunkedCopy for sequential writes (#5477)
NativeIO.chunkedCopy fsyncs its writebuffer directly and
requires an O_DIRECT RandomAccessFile. By allowing the
kernel to start writing while filling the buffer the writes
will be more constant. In addition the O_DIRECT flag is not
required anymore and this will work faster in case fadvise
is not supported on some system.

This is based on Linus' post here:
http://lkml.iu.edu/hypermail/linux/kernel/1005.2/01845.html
2018-03-14 15:15:41 -07:00
Gian Merlino e096a8d6c5 Emitter: Clarify contract of "emit". (#5486)
* Emitter: Clarify contract of "emit".

* New wording
2018-03-14 22:07:08 +01:00
bolkedebruin 8f07a39af7 Skip OS cache on Linux when pulling segments (#5421)
Druid relies on the page cache of Linux in order to have memory segments.
However when loading segments from deep storage or rebalancing the page
cache can get poisoned by segments that should not be in memory yet.
This can significantly slow down Druid in case rebalancing happens
as data that might not be queried often is suddenly in the page cache.

This PR implements the same logic as is in Apache Cassandra and Apache
Bookkeeper.

Closes #4746
2018-03-08 07:54:21 -08:00
Gian Merlino 7416d1d02d Add "joda" option to timeFormat extractionFn. (#5448) 2018-03-02 19:59:26 -08:00
Niraja Mishra 0f009a41e1 Fixed PeriodGranularity for Asia pacific timezones (#5410) 2018-02-27 10:39:50 -08:00
Nishant Bangarwa 219e77aeac SQL compatible Null Handling Part - Expressions and Storage Changes (#5278)
* SQL compatible Null Handling Part - Expressions, Storage and Dimension Selector Changes

fix travis strict compilation

* fix teamcity error - remove unused method

* review comments

* review comments

* more comments

* review comments

* review comments

* Optimize isNull method

* Optimize isNull in ColumnarFloats/Longs/Doubles

* review comment - separate classes for null and non-null columns

fix intellij inspection

* remove unused import

* More Review comments

* improve comment

* More review comments

* fix checkstyle

* more review comments

* review comments.

fix javadoc links

remove Nullable from ConstantColumnValueSelector

* review comments.

* satisfy teamcity inspections
2018-02-21 13:27:26 +01:00
Roman Leventov 65225bd6e7 Make HttpPostEmitter more robust in the face of serious errors (like OutOfMemoryError) during onSealExclusive() (#5386) 2018-02-13 17:00:58 -08:00
Roman Leventov 49a6b0b8a6 Debug logging in HttpPostEmitter and Batch (#5365) 2018-02-13 11:19:04 -06:00
Roman Leventov e64ffb10c2 Standartize on using Integer.BYTES instead of Ints.BYTES from Guava, same for other primitives (#5366) 2018-02-07 13:24:30 -08:00
Gian Merlino 8c738c7076 Fix races in LookupSnapshotTaker, CoordinatorPollingBasicAuthenticatorCacheManager (#5344)
* Fix races in LookupSnapshotTaker, CoordinatorPollingBasicAuthenticatorCacheManager.

Both were susceptible to the following conditions:

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

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

Change-Id: Ieb5c18559879f3f3a0104912ce2f0a354ad0aac3

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

Change-Id: Iff97fa044758019ed0c231587d10e31a9cc18da0

* exclude class and remove other usage

Change-Id: Ib458c2caaa1865535767e1009fbf017a92c8f615

* remove it from test classes

Change-Id: I9b576324f6c7e17a74bd8b13879232c9a8cd40b4

* remove unused

Change-Id: If1c5b70c26c2b7c83c20434cb72b2060653f5052
2018-02-06 16:34:11 +05:30
Gian Merlino 7e02408510 Update versions to 0.13.0-SNAPSHOT. (#5323) 2018-02-02 12:06:38 -06:00
Jonathan Wei 2a892709e8 More memory limiting for HttpPostEmitter (#5300)
* More memory limiting for HttpPostEmitter

* Less aggressive large events test

* Fix tests

* Restrict batch queue size first, keep minimum of 2 queue items
2018-01-26 15:48:45 -08:00
Himanshu d6f47eaf76 reduce noise in the logs (#5302) 2018-01-26 13:10:02 -08:00
Jonathan Wei 80419752b5 Add metamx emitter, http clients, and metrics packages to druid java-util (#5289)
* Add metamx java-util emitter, http clients, and metrics packages to druid java-util

* Remove metamx java-util from pom.xml files

* Checkstyle fixes

* Import fix

* TeamCity inspection fixes

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

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

* Add ] to log msg, suppress inspection
2018-01-24 22:10:36 +01:00
Jihoon Son 241efafbb2
Automatic compaction by coordinators (#5102)
* Automatic compaction by coordinator

* add links

* skip compaction for very recent segments if they are small

* fix finding search interval

* fix finding search interval

* fix TimelineHolder iteration

* add test for newestSegmentFirstPolicy

* add CompactionSegmentIterator

* add numTargetCompactionSegments

* add missing config

* fix skipping huge shards

* fix handling large number of segments per shard

* fix test failure

* change recursive call to loop

* fix logging

* fix build

* fix test failure

* address comments

* change dataSources type

* check running pendingTasks at each run

* fix test

* address comments

* fix build

* fix test

* address comments

* address comments

* add doc for segment size optimization

* address comment
2018-01-13 13:52:37 +09:00
Clint Wylie 491f8cca81 fix timewarp query results when using timezones and crossing DST transitions (#5157)
* timewarp and timezones
changes:
* `TimewarpOperator` will now compensate for daylight savings time shifts between date translation ranges for queries using a `PeriodGranularity` with a timezone defined
* introduces a new abstract query type `TimeBucketedQuery` for all queries which have a `Granularity` (100% not attached to this name). `GroupByQuery`, `SearchQuery`, `SelectQuery`, `TimeseriesQuery`, and `TopNQuery` all extend `TimeBucke
tedQuery`, cutting down on some duplicate code and providing a mechanism for `TimewarpOperator` (and anything else) that needs to be aware of granularity

* move precondition check to TimeBucketedQuery, add Granularities.nullToAll, add getTimezone to TimeBucketQuery

* formatting

* more formatting

* unused import

* changes:
* add 'getGranularity' and 'getTimezone' to 'Query' interface
* merge 'TimeBucketedQuery' into 'BaseQuery'
* fixup tests from resulting serialization changes

* dedupe

* fix after merge

* suppress warning
2018-01-11 12:39:33 -08:00
Jihoon Son 5d0619f5ce Support retrying for PrefetchableTextFilesFirehoseFactory when prefetch is disabled (#5162)
* Add RetryingInputStream

* unnecessary exception

* fix PrefetchableTextFilesFirehoseFactoryTest

* Fix retrying on connection reset

* fix start offset

* fix checkstyle

* fix check connection reset

* address comments

* fix compile

* address comments

* address comments
2018-01-10 17:37:19 +01:00
Roman Leventov 579f9fbedf Add IndexedInts.debugToString() and AbstractIndex.toString(); Add Sequence.toList() and limit() (#5175)
* Add IndexedInts.debugToString() and AbstractIndex.toString()

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

* code review changes

* code review changes
2018-01-03 16:13:21 -08:00
Roman Leventov 5787d04fad Bump Druid version to 0.12.0 (#5138) 2017-12-15 07:37:01 -08:00
Roman Leventov a7a6a0487e Replace IOPeon with SegmentWriteOutMedium; Improve buffer compression (#4762)
* Replace IOPeon with OutputMedium; Improve compression

* Fix test

* Cleanup CompressionStrategy

* Javadocs

* Add OutputBytesTest

* Address comments

* Random access in OutputBytes and GenericIndexedWriter

* Fix bugs

* Fixes

* Test OutputBytes.readFully()

* Address comments

* Rename OutputMedium to SegmentWriteOutMedium and OutputBytes to WriteOutBytes

* Add comments to ByteBufferInputStream

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

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

remove parse method from implemetations

transform transformers

add string multiplier input row parser

fix withParseSpec

fix kafka batch indexing

fix isPersistRequired

comments

* add unit test

* make persist async

* review comments
2017-12-04 16:06:16 -06:00
Jihoon Son 5f3c863d5e Add compaction task (#4985)
* Add compaction task

* added doc

* use combining aggregators

* address comments

* add support for dimensionsSpec

* fix getUniqueDims and getUniqueMetics

* find unique dimensionsSpec

* fix compilation

* add unit test

* fix test

* fix test

* test for different dimension orderings and types, and doc for type and ordering

* add control for custom ordering and type

* update doc

* fix compile

* fix compile

* add segments param

* fix serde error

* fix build
2017-11-03 21:55:27 -06:00
Roman Leventov 5eb08c27cb Add Emitter monitoring (#4973)
* Add Emitter monitoring

* Fix typo

* Fixes

* testing new emitter

* Fix failed test (#71)

* testing new emitter

* fix on failed test

* Remove emitter's readTimeout from docs

* Update docs

* Add HttpEmittingMonitor

* Update java-util to 1.3.2
2017-11-03 21:27:57 -06:00
Himanshu ef4a8cb724 Optional segment load/drop management without zookeeper using http (#4966)
* introducing CuratorLoadQueuePeon

* HttpLoadQueuePeon based off of current code

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

This reverts commit d8b3bfa63c.

* SegmentLoadDropHandler copy/pasted from ZkCoordinator

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

This reverts commit e725ff4146.

* remove non-zk part from ZkCoordinator

* remove zk part from SegmentLoadDropHandler

* additional changes for segment load/drop management with http

* address review comments

* add some more logs

* Execs class is moved
2017-10-19 12:41:23 -07:00
Jihoon Son 52d7f74226 Add streaming aggregation as the last step of ConcurrentGrouper if data are spilled (#4704)
* Add steaming grouper

* Fix doc

* Use a single dictionary while combining

* Revert GroupByBenchmark

* Removed unused code

* More cleanup

* Remove unused config

* Fix some typos and bugs

* Refactor Groupers.mergeIterators()

* Add comments for combining tree

* Refactor buildCombineTree

* Refactor iterator

* Add ParallelCombiner

* Add ParallelCombinerTest

* Handle InterruptedException

* use AbstractPrioritizedCallable

* Address comments

* [maven-release-plugin] prepare release druid-0.11.0-sg

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

* Address comments

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

This reverts commit 5c6b31e488.

* Revert "[maven-release-plugin] prepare release druid-0.11.0-sg"

This reverts commit 0f5c3a8b82.

* Fix build failure

* Change list to array

* rename sortableIds

* Address comments

* change to foreach loop

* Fix comment

* Revert keyEquals()

* Remove loop

* Address comments

* Fix build fail

* Address comments

* Remove unused imports

* Fix method name

* Split intermediate and leaf combine degrees

* Add comments to StreamingMergeSortedGrouper

* Add more comments and fix overflow

* Address comments

* ConcurrentGrouperTest cleanup

* add thread number configuration for parallel combining

* improve doc

* address comments

* fix build
2017-10-17 23:24:08 -07:00
Roman Leventov dc7cb117a1 Refactor ColumnSelectorFactory; Rely on ColumnValueSelector's polymorphism (#4886)
* Refactor ColumnSelectorFactory; Rely on ColumnValueSelector's polymorphism

* Fix MapVirtualColumn.makeColumnValueSelector()

* Minor fixes

* Fix IndexGeneratorCombinerTest

* DimensionSelector to return zeros when treated as numeric ColumnValueSelector

* Fix IncrementalIndexTest

* Fix IncrementalIndex.makeColumnSelectorFactory()

* Optimize MapBasedRow.getMetric()

* Fix VarianceAggregatorTest

* Simplify IncrementalIndex.makeColumnSelectorFactory()

* Address comments

* More comments

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

* Fix to read cache when prefetch is disabled

* More tests

* Cleanup codes

* Add Fetcher

* Fix test failure

* Count file size

* Fix test

* rename generic parameter

* address comments

* address comments

* reuse buffer

* move Execs to java-util

* use execs

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

* fix tc fail
2017-10-13 07:56:19 -07:00
Gian Merlino 928b083a7a JSON: Fix incorrect translation of null to "null". (#4939) 2017-10-12 15:53:40 -07:00