Commit Graph

1036 Commits

Author SHA1 Message Date
Roman Leventov d400f23791 Monomorphic processing of TopN queries with simple double aggregators over historical segments (part of #3798) (#4079)
* Monomorphic processing of topN queries with simple double aggregators and historical segments

* Add CalledFromHotLoop annocations to specialized methods in SimpleDoubleBufferAggregator

* Fix a bug in Historical1SimpleDoubleAggPooledTopNScannerPrototype

* Fix a bug in SpecializationService

* In SpecializationService, emit maxSpecializations warning only once

* Make GenericIndexed.theBuffer final

* Address comments

* Newline

* Reapply 439c906 (Make GenericIndexed.theBuffer final)

* Remove extra PooledTopNAlgorithm.capabilities field

* Improve CachingIndexed.inspectRuntimeShape()

* Fix CompressedVSizeIntsIndexedSupplier.inspectRuntimeShape()

* Don't override inspectRuntimeShape() in subclasses of CompressedVSizeIndexedInts

* Annotate methods in specializations of DimensionSelector and FloatColumnSelector with @CalledFromHotLoop

* Make ValueMatcher to implement HotLoopCallee

* Doc fix

* Fix inspectRuntimeShape() impl in ExpressionSelectors

* INFO logging of specialization events

* Remove modificator

* Fix OrFilter

* Fix AndFilter

* Refactor PooledTopNAlgorithm.scanAndAggregate()

* Small refactoring

* Add 'nothing to inspect' messages in empty HotLoopCallee.inspectRuntimeShape() implementations

* Don't care about runtime shape in tests

* Fix accessor bugs in Historical1SimpleDoubleAggPooledTopNScannerPrototype and HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype, cover them with tests

* Doc wording

* Address comments

* Remove MagicAccessorBridge and ensure Offset subclasses are public

* Attach error message to element
2017-05-16 16:19:55 -07:00
Roman Leventov b7a52286e8 Make @Override annotation obligatory (#4274)
* Make MissingOverride an error

* Make travis stript to fail fast

* Add missing Override annotations

* Comment
2017-05-16 13:30:30 -05:00
Benedict Jin e823085866 Improve `collection` related things that reusing a immutable object instead of creating a new object (#4135) 2017-05-17 01:38:51 +09:00
Roman Leventov 1ebfa22955 Update Error prone configuration; Fix bugs (#4252)
* Make Errorprone the default compiler

* Address comments

* Make Error Prone's ClassCanBeStatic rule a error

* Preconditions allow only %s pattern

* Fix DruidCoordinatorBalancerTester

* Try to give the compiler more memory

* Remove distribution module activation on jdk 1.8 because only jdk 1.8 is used now

* Don't show compiler warnings

* Try different travis script

* Fix travis.yml

* Make Error Prone optional again

* For error-prone compiler

* Increase compiler's maxmem

* Don't run Error Prone for benchmarks because of OOM

* Skip install step in Travis

* Remove MetricHolder.writeToChannel()

* In travis.yml, check compilation before tests, because it may fail faster
2017-05-12 15:55:17 +09:00
Himanshu 417714d228 additional lookup status discovery http endpoints at coordinator (#4228)
* additional lookup status discovery http endpoints at coordinator

* more changes

* jsonize the error msgs as well

* fix tests
2017-05-04 11:15:30 -07:00
Himanshu 5a5a2749cd improvements to coordinator lookups management (#3855)
* coordinator lookups mgmt improvements

* revert replaces removal, deprecate it instead

* convert and use older specs stored in db

* more tests and updates

* review comments

* add behavior for 0.10.0 to 0.9.2 downgrade

* incorporating more review comments

* remove explicit lock and use LifecycleLock in LookupReferencesManager. use LifecycleLock in LookupCoordinatorManager as well

* wip on LookupCoordinatorManager

* lifecycle lock

* refactor thread creation into utility method

* more review comments addressed

* support smooth roll back of lookup snapshots from 0.10.0 to 0.9.2

* correctly use LifecycleLock in LookupCoordinatorManager and remove synchronization from start/stop

* run lookup mgmt on leader coordinator only

* wip: changes to do multiple start() and stop() on LookupCoordinatorManager

* lifecycleLock fix usage in LookupReferencesManagerTest

* add LifecycleLock back

* fix license hdr

* some fixes

* make LookupReferencesManager.getAllLookupsState() consistent while still being lockless

* address review comments

* addressing leventov's comments

* address charle's comments

* add IOE.java

* for safety in LookupReferencesManager mainThread check for lifecycle started state on each loop in addition to interrupt

* move thread creation utility method to Execs

* fix names

* add tests for LookupCoordinatorManager.lookupManagementLoop()

* add further tests for figuring out toBeLoaded and toBeDropped on LookupCoordinatorManager

* address leventov comments

* remove LookupsStateWithMap and parameterize LookupsState

* address review comments

* address more review comments

* misc fixes
2017-04-28 08:41:38 -05:00
Gian Merlino 2ca7b00346 Update versions to 0.10.1-SNAPSHOT. (#4191) 2017-04-20 18:12:28 -07:00
Jihoon Son 5b69f2eff2 Make timeout behavior consistent to document (#4134)
* Make timeout behavior consistent to document

* Refactoring BlockingPool and add more methods to QueryContexts

* remove unused imports

* Addressed comments

* Address comments

* remove unused method

* Make default query timeout configurable

* Fix test failure

* Change timeout from period to millis
2017-04-19 09:47:53 +09:00
Roman Leventov 15f3a94474 Copy closer into Druid codebase (fixes #3652) (#4153) 2017-04-10 09:38:45 +09:00
Roman Leventov 73d9b31664 GenericIndexed minor bug fixes, optimizations and refactoring (#3951)
* Minor bug fixes in GenericIndexed; Refactor and optimize GenericIndexed; Remove some unnecessary ByteBuffer duplications in some deserialization paths; Add ZeroCopyByteArrayOutputStream

* Fixes

* Move GenericIndexedWriter.writeLongValueToOutputStream() and writeIntValueToOutputStream() to SerializerUtils

* Move constructors

* Add GenericIndexedBenchmark

* Comments

* Typo

* Note in Javadoc that IntermediateLongSupplierSerializer, LongColumnSerializer and LongMetricColumnSerializer are thread-unsafe

* Use primitive collections in IntermediateLongSupplierSerializer instead of BiMap

* Optimize TableLongEncodingWriter

* Add checks to SerializerUtils methods

* Don't restrict byte order in SerializerUtils.writeLongToOutputStream() and writeIntToOutputStream()

* Update GenericIndexedBenchmark

* SerializerUtils.writeIntToOutputStream() and writeLongToOutputStream() separate for big-endian and native-endian

* Add GenericIndexedBenchmark.indexOf()

* More checks in methods in SerializerUtils

* Use helperBuffer.arrayOffset()

* Optimizations in SerializerUtils
2017-03-27 14:17:31 -05:00
Charles Allen 805d85afda Allow compilation as Java8 source and target (#3328)
* Allow compilation as Java8 source and target for everything except API

* Remove conditions in tests which assume that we may run with Java 7

* Update easymock to 3.4

* Make Animal Sniffer to check Java 1.8 usage; remove redundant druid-caffeine-cache configuration

* Use try-with-resources in LargeColumnSupportedComplexColumnSerializerTest.testSanity()

* Remove java7 special for druid-api
2017-03-14 22:23:47 -06:00
Roman Leventov ea1f5b7954 LifecycleLock for better synchronization in lifecycled objects (#3964)
* Introduce LifecycleLock

* Add LifecycleLockTest

* Rename LifecycleLock.release() to exitStart()

* Rewrite LifecycleLock using AbstractQueuedSynchronizer for more safety, added tests

* Add LifecycleLock.exitStop() and reset()

* Add LifecycleLock.awaitStarted(timeout)

* Braces

* Fix
2017-03-02 12:22:57 -08:00
Jihoon Son 7200dce112 Atomic merge buffer acquisition for groupBys (#3939)
* Atomic merge buffer acquisition for groupBys

* documentation

* documentation

* address comments

* address comments

* fix test failure

* Addressed comments

- Add InsufficientResourcesException
- Renamed GroupByQueryBrokerResource to GroupByQueryResource

* addressed comments

* Add takeBatch() to BlockingPool
2017-02-22 14:49:37 -06:00
Jihoon Son a459db68b6 Fine grained buffer management for groupby (#3863)
* Fine-grained buffer management for group by queries

* Remove maxQueryCount from GroupByRules

* Fix code style

* Merge master

* Fix compilation failure

* Address comments

* Address comments

- Revert Sequence
- Add isInitialized() to Grouper
- Initialize the grouper in RowBasedGrouperHelper.Accumulator
- Simple refactoring RowBasedGrouperHelper.Accumulator
- Add tests for checking the number of used merge buffers
- Improve docs

* Revert unnecessary changes

* change to visible to testing

* fix misspelling
2017-02-14 12:55:54 -08:00
Jihoon Son 991e2852da Add PostAggregators to generator cache keys for top-n queries (#3899)
* Add PostAggregators to generator cache keys for top-n queries

* Add tests for strings

* Remove debug comments

* Add type keys and list sizes to cache key

* Make post aggregators used for sort are considered for cache key generation

* Use assertArrayEquals()

* Improve findPostAggregatorsForSort()

* Address comments

* fix test failure

* address comments
2017-02-13 12:23:44 -08:00
Gian Merlino 12317fd001 Bump version to 0.10.0-SNAPSHOT. (#3913) 2017-02-06 17:54:35 -08:00
Roman Leventov ca9f0e2b27 Don't override finalize() and reduce locking in LoadBalancingPool and ReferenceCountedResourceHandler (#3874)
* Specialize LoadBalancingPool as MemcacheClientPool, reduce locking and don't override Object.finalize()

* Remove locking and don't override Object.finalize() in ReferenceCountingResourceHolder

* Add leak counts in ReferenceCountingResourceHolder and MemcacheClientPool. Add tests for ReferenceCountingResourceHolder and MemcacheClientPool

* Fix a race condition in ReferenceCountingResourceHolder.increment()
2017-02-06 17:14:46 -08:00
Gian Merlino d3a3b7ba0c Add virtual column types, holder serde, and safety features. (#3823)
* Add virtual column types, holder serde, and safety features.

Virtual columns:
- add long, float, dimension selectors
- put cache IDs in VirtualColumnCacheHelper
- adjust serde so VirtualColumns can be the holder object for Jackson
- add fail-fast validation for cycle detection and duplicates
- add expression virtual column in core

Storage adapters:
- move virtual column hooks before checking base columns, to prevent surprises
  when a new base column is added that happens to have the same name as a
  virtual column.

* Fix ExtractionDimensionSpecs with virtual dimensions.

* Fix unused imports.

* CR comments

* Merge one more time, with feeling.
2017-01-26 18:15:51 -08:00
Roman Leventov af93a8d189 Sequences refactorings and removed unused code (part of #3798) (#3693)
* Removing unused code from io.druid.java.util.common.guava package; fix #3563 (more consistent and paranoiac resource handing in Sequences subsystem); Add Sequences.wrap() for DRY in MetricsEmittingQueryRunner, CPUTimeMetricQueryRunner and SpecificSegmentQueryRunner; Catch MissingSegmentsException in SpecificSegmentQueryRunner's yielder.next() method (follow up on #3617)

* Make Sequences.withEffect() execute the effect if the wrapped sequence throws exception from close()

* Fix strange code in MetricsEmittingQueryRunner

* Add comment on why YieldingSequenceBase is used in Sequences.withEffect()

* Use Closer in OrderedMergeSequence and MergeSequence to close multiple yielders
2017-01-19 20:07:43 -08:00
Jihoon Son d80bec83cc Enable auto license checking (#3836)
* Enable license checking

* Clean duplicated license headers
2017-01-10 18:13:47 -08:00
Gian Merlino d8702ebece Filters: Use ColumnSelectorFactory directly for building row-based matchers. (#3797)
* Filters: Use ColumnSelectorFactory directly for building row-based matchers.

* Adjustments based on code review.

- BoundDimFilter: fewer volatiles, rename matchesAnything to !matchesNothing.
- HavingSpecs: Clarify that they are not thread-safe, and make DimFilterHavingSpec
  not thread safe.
- Renamed rowType to rowSignature.
- Added specializations for time-based vs non-time-based DimensionSelector in RBCSF.
- Added convenience method DimensionHanderUtils.createColumnSelectorPlus.
- Added singleton ZeroIndexedInts.
- Added test cases for DimFilterHavingSpec.

* Make ValueMatcherColumnSelectorStrategy actually use the associated selector.

* Add RangeIndexedInts.

* DimFilterHavingSpec: Fix concurrent usage guard on jdk7.

* Add assertion to ZeroIndexedInts.

* Rename no-longer-volatile members.
2017-01-03 14:30:22 -08:00
Roman Leventov 33800122ad Don't return leaked Objects back to StupidPool, because this is dangerous. Reuse Cleaners in StupidPool. Make StupidPools named. Add StupidPool.leakedObjectCount(). Minor fixes (#3631) 2016-12-26 00:35:35 -06:00
Himanshu c5df30d813 fix JodaUtils.condenseIntervals(..) to correctly take end or current/next interval on overlap (#3793)
* remove unused duplicate JodaUtils.java

* fix JodaUtils.condenseIntervals(..) to correctly take end or current/next interval on overlap
2016-12-20 12:07:23 -08:00
Jonathan Wei 2bfcc8a592 First and Last Aggregator (#3566)
* add first and last aggregator

* add test and fix

* moving around

* separate aggregator valueType

* address PR comment

* add finalize inner query and adjust v1 inner indexing

* better test and fixes

* java-util import fixes

* PR comments

* Add first/last aggs to ITWikipediaQueryTest
2016-12-16 15:26:40 -08:00
Jihoon Son 5e39578eee Enable parallel test (#3774)
* Enable parallel test

* Remove unnecessary NotThreadSafe annocation

* Randomize the start port when finding available ports

* Fix test failure

* Change to handle all negatives
2016-12-14 21:05:56 -08:00
Navis Ryu 87c61fa749 Refactor boolean cast code, add tests (#3016) 2016-12-07 13:10:39 -08:00
Gian Merlino ff42058453 Expressions: Allow escapes in quoted identifiers. (#3735) 2016-12-06 00:17:55 +05:30
Roman Leventov c070b4a816 Fix concurrency defects, remove unnecessary volatiles (#3701) 2016-11-22 16:42:28 -08:00
Navis Ryu bb26636289 Constant flattening in math expression (#3090)
* Constant flatteing in math expression

* Addressed comments and fixed some bugs

* Addressed comments
2016-11-14 14:14:10 -08:00
Himanshu b76b3f8d85 reset-cluster command to clean up druid state stored on metadata and deep storage (#3670) 2016-11-09 11:07:01 -06:00
Gian Merlino 657e4512d2 Checkstyle checks for AvoidStaticImport, UnusedImports. (#3660)
Excludes tests from AvoidStaticImport, since those are used often there and
I didn't want to make this changeset too large. Production code use was minimal
and I switched those to non-static imports.
2016-11-05 11:34:36 -07:00
Navis Ryu e10def32f2 Support string type in math expression (#2836)
* Support string type in math expression

addressed comments

addressed comments

Addressed comments

* Updated math function document

* Addressed comments
2016-11-02 21:10:48 -06:00
Gian Merlino 45940d6e40 Math expressions support for missing columns. (#3630)
Also add SchemaEvolutionTest to help test this kind of thing.

Fixes #3627 and includes test for #3625.
2016-11-01 09:40:25 -07:00
Himanshu 32c5494e97 eagerly allocate the intermediate computation buffers (#3628) 2016-10-31 15:24:07 -07:00
Navis Ryu 898c1c21af More best-effort parse long (#3603)
* More best-effort parse long

* addressed comments
2016-10-25 10:31:51 -07:00
Himanshu 641469fc38 manage overshadowing efficiently at coordinator (#3584)
* manage overshadowing efficiently at coordinator

* take readlock in VersionedIntervalTimeline.isOvershadowed()
2016-10-24 22:49:08 +05:30
Akash Dwivedi 4b3bd8bd63 Migrating java-util from Metamarkets. (#3585)
* Migrating java-util from Metamarkets.

* checkstyle and updated license on java-util files.

* Removed unused imports from whole project.

* cherry pick metamx/java-util@826021f.

* Copyright changes on java-util pom, address review comments.
2016-10-21 14:57:07 -07:00
Navis Ryu 8b7ff4409a Math expressional parameters for aggregator (#2783)
* Supports expression-paramed aggregator (squashed and rebased on master) also includes math post aggregator (was #2820)

* Addressed comments

* addressed comments
2016-10-19 13:58:35 -05:00
David Lim c2ae734848 KafkaIndexTask: Allow run thread to stop gracefully instead of interrupting (#3534)
* allow run thread to gracefully complete instead of interrupting when stopGracefully() is called

* add comments
2016-10-17 10:52:19 -04:00
Gian Merlino ddc856214d When inserting segments, mark unused if already overshadowed. (#3499)
This is useful for the insert-segment-to-db tool, which would otherwise
potentially insert a lot of overshadowed segments as "used", causing
load and drop churn in the cluster.
2016-10-10 18:10:18 -07:00
Gian Merlino 40f2fe7893 Bump versions to 0.9.3-SNAPSHOT (#3524) 2016-09-29 13:53:32 -07:00
Slim 3175e17a3b Cached lookup module. first cut implementing JDBC cache (#2819) 2016-09-16 13:45:54 -07:00
jianran 18af480017 Rename fields in OrderedMergeIterator (#3149)
* code readable

* fix the pre middle manager peon no stop

* Revert "fix the pre middle manager peon no stop"

This reverts commit 6cef4980bf.
2016-08-11 09:42:12 -07:00
Nishant 8035c73409 Implement EnvironmentVariablePasswordProvider (#3329)
* Implement EnvironmentVariablePasswordProvider

* Review Comment : rename passwordKey to passwordVariable

* add docs

* improve doc layout

* review comment: rename property for variable
2016-08-10 05:33:51 +08:00
Gian Merlino 1aae5bd67d Nicer handling for cancelled groupBy v2 queries. (#3330)
1. Wrap temporaryStorage in a resource holder, to avoid spurious "Closed"
   errors from already-running processing tasks.
2. Exit early from the merging accumulator if the query is cancelled.
2016-08-05 14:48:06 -07:00
kaijianding 1fa681934c fix ConcurrentModificationException in CachingClusteredClient.run() (#3278)
* fix ConcurrentModificationException in CachingClusteredClient.run()

* obtain new copy of PartitionHolder to avoid potential multi-threads read/write issue
2016-07-28 19:52:50 -07:00
Gian Merlino 9b5523add3 Reference counting, better error handling for resources in groupBy v2. (#3268)
Refcounting prevents releasing the merge buffer, or closing the concurrent
grouper, before the processing threads have all finished. The better
error handling prevents an avalanche of per-runner exceptions when grouping
resources are exhausted, by grouping those all up into a single merged
exception.
2016-07-27 01:59:02 +05:30
Gian Merlino 4cc39b2ee7 Alternative groupBy strategy. (#2998)
This patch introduces a GroupByStrategy concept and two strategies: "v1"
is the current groupBy strategy and "v2" is a new one. It also introduces
a merge buffers concept in DruidProcessingModule, to try to better
manage memory used for merging.

Both of these are described in more detail in #2987.

There are two goals of this patch:

1. Make it possible for historical/realtime nodes to return larger groupBy
   result sets, faster, with better memory management.
2. Make it possible for brokers to merge streams when there are no order-by
   columns, avoiding materialization.

This patch does not do anything to help with memory management on the broker
when there are order-by columns or when there are nested queries. That could
potentially be done in a future patch.
2016-06-24 18:06:09 -07:00
Gian Merlino ebf890fe79 Update master version to 0.9.2-SNAPSHOT. (#3133) 2016-06-13 13:10:38 -07:00
sainath batthala d552a5c034 Documented getAuditTime, getPayload methods in AuditEntry.java (#3045)
* Documented getAuditTime, getPayload methods in AuditEntry.java

* author tag removed from documentation
2016-06-02 08:20:33 -07:00