Commit Graph

7647 Commits

Author SHA1 Message Date
Jonathan Wei 182261f713 Allow configurable temp directory for query processing (#3893) 2017-02-02 10:22:28 -08:00
Charles Allen a73f1c9c70 Make s3 work better (#3898) 2017-02-02 10:04:30 -08:00
Jonathan Wei e6b95e80aa Remove deprecated Aggregator/AggregatorFactory methods (#3894) 2017-02-01 14:43:18 -08:00
Gian Merlino 151ff6d064 flattenSpec: Document that "expr" is ignored for type "root". (#3884) 2017-01-31 10:27:20 -08:00
Himanshu 17c6512ad8 integration-test update - use KafkaProducer in ITKafkaTest and add retries to task submission (#3888)
* use KafkaProducer instead of old Producer in ITKafkaTest

* add retries to OverlordResourceTestClient.submitTask(..)
2017-01-27 07:38:17 -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
Gian Merlino ac84a3e011 SQL: Add resolution parameter, fix filtering bug with APPROX_QUANTILE (#3868)
* SQL: Add resolution parameter to quantile agg, rename to APPROX_QUANTILE.

* Fix bug with re-use of filtered approximate histogram aggregators.

Also add APPROX_QUANTILE tests for filtering and running on complex columns.
Includes some slight refactoring to allow tests to make DruidTables that
include complex columns.

* Remove unused import
2017-01-25 18:39:26 -08:00
Roman Leventov 75d9e5e7a7 DimensionSelector-related bug fixes and optimizations (fixes #3799, part of #3798) (#3858)
*  * Add DimensionSelector.idLookup() and nameLookupPossibleInAdvance() to allow better inspection of features DimensionSelectors supports, and safer code working with DimensionSelectors in BaseTopNAlgorithm, BaseFilteredDimensionSpec, DimensionSelectorUtils;
 * Add PredicateFilteringDimensionSelector, to make BaseFilteredDimensionSpec to be able to decorate DimensionSelectors with unknown cardinality;
 * Add DimensionSelector.makeValueMatcher() (two kinds) for DimensionSelector-side specifics-aware optimization of ValueMatchers;
 * Optimize getRow() in BaseFilteredDimensionSpec's DimensionSelector, StringDimensionIndexer's DimensionSelector and SingleScanTimeDimSelector;
 * Use two static singletons, TrueValueMatcher and FalseValueMatcher, instead of BooleanValueMatcher;
 * Add NullStringObjectColumnSelector singleton and use it in MapVirtualColumn

* Rename DimensionSelectorUtils.makeNonDictionaryEncodedIndexedIntsBasedValueMatcher to makeNonDictionaryEncodedRowBasedValueMatcher

* Make ArrayBasedIndexedInts constructor private, replace it's usages with of() static factory method

* Cache baseIdLookup in ForwardingFilteredDimensionSelector

* Fix a bug in DimensionSelectorUtils.makeRowBasedValueMatcher(selector, predicate, matchNull)

* Employ precomputed BitSet optimization in DimensionSelector.makeValueMatcher(value, matchNull) when lookupId() is not available, but cardinality is known and lookupName() is available

* Doc fixes

* Addressed comments

* Fix

* Fix

* Adjust javadoc of DimensionSelector.nameLookupPossibleInAdvance() for SingleScanTimeDimSelector

* throw UnsupportedOperationException instead of IAE in BaseTopNAlgorithm
2017-01-25 15:28:27 -08:00
Gian Merlino 3136dfa421 LikeFilter: Read value lazily when doing a prefix-based match. (#3880)
This speeds up cases where we don't actually need to read the value,
such as "LIKE 'foo%'".
2017-01-25 13:22:07 -08:00
Parag Jain b3dae0efc3 catch all errors (#3844) 2017-01-24 18:01:30 -07:00
Parag Jain 515caa8a85 ITKafkaIndexingServiceTest fixes (#3872)
* remove wait between sends, wait for ingestion to complete before querying

send fixed number of events

more fixes

* handle interrupted exception

* remove while

* review comments
2017-01-22 14:37:35 -08:00
Himanshu efb1b40fe0 build sqlserver-metadata-storage contrib extension (#3871) 2017-01-20 14:39:15 -08:00
Niketh Sabbineni 2b8d3c102b Remove throttling on drop segments (#3736)
* Remove throttling on drop

* Throttle loadqueuepeon segment change requests to ZK

* Make initial delay configurable, add docs, shutdown gracefully

* Make loadqueuepeon repeat delay configurable
2017-01-20 10:02:19 -08:00
Gian Merlino bb7c496d88 SQL: Use topN for single-dim queries with LIMIT but no ORDER BY. (#3867) 2017-01-20 09:59:28 -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
Gian Merlino 9cc3015ddd Bypass Calcite's SemiJoinRule and use our own. (#3843)
This simplifies DruidSemiJoin, which no longer needs to add aggregation back
in. It also allows some more kinds of queries to plan properly, like the one
added in "testTopNFilterJoin".
2017-01-19 19:51:14 -08:00
Gian Merlino d51f5e058d SQL: Ditch CalciteConnection layer and add DruidMeta, extension aggregators. (#3852)
* SQL: Ditch CalciteConnection layer and add DruidMeta, extension aggregators.

Switched from CalciteConnection to Planner, bringing benefits:

- CalciteConnection's JDBC interface no longer sits between the SQL server
  (HTTP/Avatica) and Druid's query layer. Instead, the SQL servers can use
  Druid Sequence objects directly, reducing overhead in the query return path.

- Implemented our own Planner-based Avatica Meta, letting us control
  connection timeouts and connection / statement limits. The previous
  CalciteConnection-based implementation didn't have any limits or timeouts.

- The Planner interface lets us override the operator table, opening up
  SQL language extensions. This patch includes two: APPROX_COUNT_DISTINCT
  in core, and a QUANTILE aggregator in the druid-histogram extension.

Also:

- Added INFORMATION_SCHEMA metadata schema.

- Added tests for Unicode literals and escapes.

* Verify statement is actually open before closing it.

* More detailed INFORMATION_SCHEMA docs.
2017-01-19 16:32:20 -08:00
kaijianding 33ae9dd485 streaming version of select query (#3307)
* streaming version of select query

* use columns instead of dimensions and metrics;prepare for valueVector;remove granularity

* respect query limit within historical

* use constant

* fix thread name corrupted bug when using jetty qtp thread rather than processing thread while working with SpecificSegmentQueryRunner

* add some test for scan query

* add scan query document

* fix merge conflicts

* add compactedList resultFormat, this format is better for json ser/der

* respect query timeout

* respect query limit on broker

* use static consts and remove unused code
2017-01-19 16:09:53 -06:00
Slim ae5a349a54 Exclude the transitive dependency LGPL jar since it is not needed (#3865)
* Exclude the transitive dependency LGPL jar since it is not needed

* add reason why exclude

* exclude from the root dependency

* add banning tool  to enforce exclusions
2017-01-19 11:49:08 -08:00
David Lim ff52581bd3 IndexTask improvements (#3611)
* index task improvements

* code review changes

* add null check
2017-01-18 14:24:37 -08:00
Himanshu 7004f5d499 make ITKafkaTest less non-deterministic (#3856) 2017-01-17 16:52:51 -06:00
Akash Dwivedi e550d48772 Using fully qualified hdfs path. (#3705)
* Using fully qualified hdfs path.

* Review changes.

* Remove unused imports.

* Variable name change.
2017-01-17 14:40:22 -06:00
Slim 558dc365a4 renaming classes to be run by mvn and comment non operational tests (#3847) 2017-01-17 11:59:12 -08:00
Akash Dwivedi dd0c4e2ead Migrating extendedset from Metamarkets. (#3694)
* Migrating extendedset from Metamarkets.

* Notice change

* More details in NOTICE

* NOTICE formatting.

* suppress header checkstlye for extendedset.
2017-01-17 10:10:27 -08:00
Gian Merlino b0232b4e40 Replace our AggregateValuesRule with Calcite's. (#3845) 2017-01-12 15:51:50 -06:00
Fokko Driesprong 31bea380eb Updated Apache Zookeeper to the latest stable version (#3841) 2017-01-12 13:39:29 -08:00
Gian Merlino e86859b228 SQL support for nested groupBys. (#3806)
* SQL support for nested groupBys.

Allows, for example, doing exact count distinct by writing:

  SELECT COUNT(*) FROM (SELECT DISTINCT col FROM druid.foo)

Contrast with approximate count distinct, which is:

  SELECT COUNT(DISTINCT col) FROM druid.foo

* Add deeply-nested groupBy docs, tests, and maxQueryCount config.

* Extract magic constants into statics.

* Rework rules to put preconditions in the "matches" method.
2017-01-11 18:32:53 -08:00
Gian Merlino 76620615a1 Properly respect the enableAvatica and enableJsonOverHttp options. (#3834) 2017-01-11 14:43:34 -06:00
Jihoon Son d80bec83cc Enable auto license checking (#3836)
* Enable license checking

* Clean duplicated license headers
2017-01-10 18:13:47 -08:00
Jihoon Son c099977a5b Add an option to SearchQuery to choose a search query execution strategy (#3792)
* Add an option to SearchQuery to choose a search query execution strategy.

Supported strategies are
1) Index-only query execution
2) Cursor-based scan
3) Auto: choose an efficient strategy for a given query

* Add SearchStrategy and SearchQueryExecutor

* Address comments

* Rename strategies and set UseIndexesStrategy as the default strategy

* Add a cost-based planner for auto strategy

* Add document

* Fix code style

* apply code style

* apply comments
2017-01-10 18:04:20 -08:00
Gian Merlino ce0049d8ff DruidCoordinator: Leave the ServerInventoryView running when we lose leadership. (#3830) 2017-01-09 12:36:05 +05:30
Charles Allen 229559b46a Make TaskLockbox's ReentrantLock fair (#3828) 2017-01-07 12:34:47 -08:00
Gian Merlino 3c012305d1 SqlResource: Fix incorrect labeling of aliased columns. (#3829) 2017-01-07 12:33:53 -08:00
Clint Wylie 97ce006719 use correct joda-time (#3827) 2017-01-06 16:31:05 -08:00
Gian Merlino a4f81a6471 Update to Calcite 1.11.0. (#3825) 2017-01-06 14:45:17 -08:00
Niketh Sabbineni 2aa8385fc0 Include loadqueuepeon size to compute disknormalized cost (#3826) 2017-01-06 14:30:23 -08:00
Vinh Tran dddeae813a Update caching.md typo (#3824)
* Update caching.md

Typo of Command vs Comma

* Update index.md

Fixing `Command` typo
2017-01-06 12:14:07 -08:00
Niketh Sabbineni a5f82e8acf Randomly choose a server when multiple best servers are available (#3822)
* Randomly choose a server when multiple best servers are available

* Use one pass instead of two

* Fix code style issues
2017-01-05 23:34:05 -06:00
Gian Merlino 3c63cff57a Remove makeMathExpressionSelector from ColumnSelectorFactory. (#3815)
* Remove makeMathExpressionSelector from ColumnSelectorFactory.

* Add @Nullable annotations in places, fix Number.class check.

* Break up createBindings, add tests.

* Add null check.
2017-01-05 18:06:38 -08:00
Himanshu 7ced0e8759 log sizes of created smoosh files (#3817)
* log when merging of intermediate segments starts during batch ingestion

* log sizes of created smoosh files
2017-01-04 16:52:22 -08:00
Gian Merlino 220ca7ebb6 Ignore DimFilterHavingSpec testConcurrentUsage. (#3814) 2017-01-03 17:43:58 -07:00
Gian Merlino 1f35120c7e Downgrade to avatica-server 1.8.0, skip avatica-core. (#3813)
This matches the version bundled by Calcite 1.10.0.
2017-01-03 16:00:37 -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
Nishant 80e2394f94 Allow users to specify additional command line args for creating druid package (#3812)
* Allow users to specify additional command line args for creating tar balls

This PR allows users to specify additional command line options to the
pull deps command while creating druid distribution.
e.g. To also package graphite-emitter in druid tarball one can run -
mvn package -Ddruid.distribution.pulldeps.opts='-c
io.druid.extensions.contrib:graphite-emitter'

* Set default to --clean instead of blank value
2017-01-03 11:44:35 -08:00
Nishant 351d570684 Improve startup script - create PID and LOG dir if they do not exist (#3808) 2017-01-02 09:20:22 -08:00
Yuusaku Taniguchi 02519d5b64 Exhibitor Support (#3664)
* allow JsonConfigTesterBase to treat the fields of collections

* [Feature] Exhibitor Support (#3664)

This patch provides the integration of Druid & Netflix Exhibitor. Druid
currently use Apache Curator as ZooKeeper client. Curator can be
integrated with Exhibitor to achieve a live/updating list of the
ZooKeeper ensemble. This patch enables Druid to use this features.
2017-01-02 09:15:36 -08:00
Roman Leventov 49d71e9b38 Fix the build after #3697 (#3807) 2016-12-26 17:06:48 -06: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
Roman Leventov 76cb06a8d8 Lookup cache refactoring (the main part of #3667) (#3697)
* Lookup cache refactoring (the main part of druid-io/druid#3667)

* Use PowerMock's static methods in NamespaceLookupExtractorFactoryTest

* Fix KafkaLookupExtractorFactoryTest

* Use VisibleForTesting annotation instead of Javadoc comment

* Create a NamespaceExtractionCacheManager separately for each test in NamespaceExtractionCacheManagersTest

* Rename CacheScheduler.NoCache.ENTRY_DISPOSED to ENTRY_CLOSED

* Reduce visibility of NamespaceExtractionCacheManager.cacheCount() and monitor() implementations, and don't run NamespaceExtractionCacheManagerExecutorsTest with off-heap cache (it didn't before)

* In NamespaceLookupExtractorFactory, use safer idiom to check if CacheState is NoCache or VersionedCache

* More logging in CacheHandler constructor and close(), VersionedCache.close()

* PR comments addressed

* Make CacheScheduler.EntryImpl AutoCloseable, avoid 'dispose' verb in comments, logging and naming in CacheScheduler in favor of 'close'

* More Javadoc comments to CacheScheduler

* Fix NPE

* Remove logging in OnHeapNamespaceExtractionCacheManager.expungeCollectedCaches()

* Make NamespaceExtractionCacheManagersTest.testRacyCreation() to have similar load to what it be before the refactoring

* Unwrap NamespaceExtractionCacheManager.scheduledExecutorService from unneeded MoreExecutors.listeningDecorator() and specify that this is ScheduledThreadPoolExecutor, which ensures happens-before between periodic runs of the tasks

* More comments on MapDbCacheDisposer.disposed

* Replace concat with Long.toString()

* Comment on why NamespaceExtractionCacheManager.scheduledExecutorService() returns ScheduledThreadPoolExecutor

* Place logging statements in VersionedCache.close() and CacheHandler.close() after actual closing logic, because logging may fail

* Make JDBCExtractionNamespaceCacheFactory and StaticMapExtractionNamespaceCacheFactory to try to close newly created VersionedCache if population has failed, as it is done already in URIExtractionNamespaceCacheFactory

* Don't close the whole CacheScheduler.Entry, if the cache update task failed

* Replace AtomicLong updateCounter and firstRunLatch with Phaser-based UpdateCounter in CacheScheduler.EntryImpl
2016-12-23 18:04:27 -08:00
Jonathan Wei 0e5bd8b4d4 Add dimension type-based interface for query processing (#3570)
* Add dimension type-based interface for query processing

* PR comment changes

* Address PR comments

* Use getters for QueryDimensionInfo

* Split DimensionQueryHelper into base interface and query-specific interfaces

* Treat empty rows as nulls in v2 groupby

* Reduce boxing in SearchQueryRunner

* Add GroupBy empty row handling to MultiValuedDimensionTest

* Address PR comments

* PR comments and refactoring

* More PR comments

* PR comments
2016-12-21 20:11:37 -07:00