Commit Graph

7675 Commits

Author SHA1 Message Date
Jakub Kukul 28d85702ad Fix rolling of request log files. (#3916)
* Use common date format for request log files.

* Remove code duplication in creating logging FileWriter.
2017-02-14 09:33:43 -08:00
Gian Merlino af67e8904e PreComputedHyperUniquesSerde: Fix formatting. (#3932) 2017-02-14 09:32:29 -08:00
DaimonPl a2875a4d91 pre-computed HLL support for hyperUnique aggregator (#3909) 2017-02-13 15:26:20 -08:00
Akash Dwivedi 8854ce018e File.deleteOnExit() (#3923)
* Less use of File.deleteOnExit()
 * removed deleteOnExit from most of the tests/benchmarks/iopeon
 * Made IOpeon closable

* Formatting.

* Revert DeterminePartitionsJobTest, remove cleanup method from IOPeon
2017-02-13 15:12:14 -08:00
Himanshu 9dfcf0763a disable javascript execution by default (#3818) 2017-02-13 15:11:18 -08:00
Himanshu 8cf7ad1e3a druid.coordinator.asOverlord.enabled flag at coordinator to make it an overlord too (#3711) 2017-02-13 15:03:59 -08:00
Pierre 9ab9feced6 Close all aggregators when closing onHeapIncrementalIndex (#3926)
* Close all aggregators when closing onHeapIncrementalIndex

* Aggregators are now handled as Closeables, remove unnecessary mock in test

* Fix variable shadowing
2017-02-13 15:01:27 -08:00
Parag Jain 1f263fe50b alert when resetting offsets (#3931)
* alert when resetting offsets

* add more data to alerts
2017-02-13 13:49:24 -08:00
michaelschiff c1eee9bbf3 modified "end" column to `end` (#3903)
* modified "end" column to `end`.  "end" is interpretted as a string rather than dereferencing the column value

* SQLMetadataConnector.getQuoteString defines the string that should be used to quote string fields

* positional arguments for String.format

* for Connectors that use " need to include the \ escape as well
2017-02-13 12:36:27 -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
Parag Jain 8e31a465ad report hand off count finite appenderator driver (#3925) 2017-02-13 10:41:24 -08:00
baruchz b7a88706f3 Add maprfs scheme (#3920)
Add maprfs scheme to JobHelper to treated as HDFS deep storage
2017-02-12 18:37:58 -08:00
Roman Leventov 38000576ea Optimizations of union, intersection and iterators of concise bitsets (part of #3798) (#3883)
* Port of metamx/extendedset#10, metamx/extendedset#13, metamx/extendedset#14, metamx/extendedset#15, metamx/bytebuffer-collections@9b199e3349, metamx/bytebuffer-collections#38 to Druid, remove unused code from extendedset module

* Remove ConciseSet.modCount

* Replace comments with assertions in ImmutableConciseSet

* Fix comments

* Fix asssertions in ImmutableConciseSet

* Add tests

* Comment fix
2017-02-10 18:02:26 -08:00
Parag Jain 33c635aff2 use as() method of base segment in reference counting segment (#3921) 2017-02-09 20:24:47 -06:00
Jonathan Wei ca2b04f0fd Add long/float ColumnSelectorStrategy implementations (#3838)
* Add long/float ColumnSelectorStrategy implementations

* Address PR comments

* Add String strategy with internal dictionary to V2 groupby, remove dict from numeric wrapping selectors, more tests

* PR comments

* Use BaseSingleValueDimensionSelector for long/float wrapping

* remove unused import

* Address PR comments

* PR comments

* PR comments

* More PR comments

* Fix failing calcite histogram subquery tests

* ScanQuery test and comment about isInputRaw

* Add outputType to extractionDimensionSpec, tweak SQL tests

* Fix limit spec optimization for numerics

* Add cardinality sanity checks to TopN

* Fix import from merge

* Add tests for filtered dimension spec outputType

* Address PR comments

* Allow filtered dimspecs on numerics

* More comments
2017-02-08 20:39:29 -08:00
Himanshu e08cd0066b verify no duplicate aggregator names in DataSchema (#3917) 2017-02-08 16:12:07 -08:00
Gian Merlino 9191588656 Fix mvn javadoc:jar failure due to HadoopFsWrapper. (#3912) 2017-02-08 13:54:41 -06:00
Gian Merlino 97765fdfef Simplify LikeFilter implementation of getBitmapIndex, estimateSelectivity. (#3910)
* Simplify LikeFilter implementation of getBitmapIndex, estimateSelectivity.

LikeFilter:
- Reduce code duplication, and simplify methods, at the cost of incurring an extra box
  of ImmutableBitmap into a SingletonImmutableList. I think this is fine, since this
  should be cheap and the code path is not hot (just once per filter).

Filters:
- Make estimateSelectivity public since it seems intended that they be used by Filter
  implementations, and Filters from extensions may want to use them too. Removed
  @VisibleForTesting for the same reason.
- Rename one of the estimatePredicateSelectivity overloads to estimateSelectivity, since
  predicates aren't involved.

* Address PR comments.

* Remove unused import

* Change List to Collection
2017-02-08 13:46:01 -06: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
Himanshu 4c49a54517 package contrib extensions as well in separate mvn profile (#3870) 2017-02-06 13:13:01 -08:00
Jihoon Son ddd8c9ef97 Add filter selectivity estimation for auto search strategy (#3848)
* Add filter selectivity estimation for auto search strategy

* Addressed comments

* Lazy bitmap materialization for bitmap sampling and java docs

* Addressed comments.

- Fix wrong non-overlap ratio computation and added unit tests.
- Change Iterable<Integer> to IntIterable
- Remove unnecessary Iterable<Integer>

* Addressed comments

- Split a long ternary operation into if-else blocks
- Add IntListUtils.fromTo()

* Fix test failure and add a test for RangeIntList

* fix code style

* Diabled selectivity estimation for multi-valued dimensions

* Address comment
2017-02-06 11:15:03 -08:00
Parag Jain 8a13a85765 Introduce SegmentizerFactory (#3901)
* Introduce SegmentizerFactory
- that knows how to deserialize specific type of segment
- Default implementation is MMappedQueryableSegmentizerFactory which creates QueryableIndexSegment
- Unit test for the default behavior

* review comments
2017-02-06 10:05:12 -08:00
Erik Dubbelboer 2aa2fa57b5 Simple doc fix (#3907) 2017-02-06 15:52:17 +05:30
DaimonPl 93b71e265e Extract HLL related code to separate module (#3900) 2017-02-03 09:45:11 -08:00
Darío 8f4394ca49 Update segments.md (#3904) 2017-02-03 10:31:14 -06:00
Parag Jain 1aabb45a09 auto reset option for Kafka Indexing service (#3842)
* auto reset option for Kafka Indexing service in case message at the offset being fetched is not present anymore at kafka brokers

* review comments

* review comments

* reverted last change

* review comments

* review comments

* fix typo
2017-02-02 14:57:45 -06:00
Nishant Bangarwa a457cded28 Druid Extension to enable Authentication using Kerberos. (#3853)
* Add extension for supporting kerberos security

- This PR adds an extension for supporting druid authentication via
Kerberos.
- Working on the docs.

* Add docs

* review comments

* more review comments

* Block all paths by default

* more review comments - use proper Oid

* Allow extensions to override httpclient for integration tests

* Add kerberos lock to prevent multithreaded issues.

* review comment - remove enabled flag and fix router injection

* Add Cookie Handling and more detailed docs

* review comment - rename DruidKerberosConfig -> AuthKerberosConfig

* review comments

* fix travis failure on jdk7
2017-02-02 14:55:21 -06:00
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