Commit Graph

1976 Commits

Author SHA1 Message Date
Gian Merlino 2ef5c17441
Link up row-based datasources to serving layer. (#9503)
* Link up row-based datasources to serving layer.

- Add SegmentWrangler interface that allows linking of DataSources to Segments.
- Add LocalQuerySegmentWalker that uses SegmentWranglers to compute queries on
  data that is available locally.
- Modify ClientQuerySegmentWalker to use LocalQuerySegmentWalker when the base
  datasource is concrete and not a table.
- Add SegmentWranglerModule to the Broker so it has them available and can
  properly instantiate . LocalQuerySegmentWalkers.
- Set InlineDataSource and LookupDataSource to concrete, since they can be
  directly queried now.

* Fix tests.
2020-03-11 11:32:27 -07:00
Gian Merlino 4f085896c6
Ability to directly query row-based datasources. (#9502)
* Ability to directly query row-based datasources.

Includes:

- Foundational classes RowBasedSegment, RowBasedStorageAdapter,
  RowBasedCursor provide a queryable interface on top of a
  RowBasedColumnSelectorFactory.
- Add LookupSegment: A RowBasedSegment that is built on lookup data.
- Improve capability reporting in RowBasedColumnSelectorFactory.

* Fix import.

* Remove unthrown IOException.
2020-03-10 20:39:01 -07:00
Samarth Jain c74749f0f4
Don't exclude null dimension values from the map based query response (#9438) 2020-03-10 15:06:03 -07:00
Gian Merlino c6c2282b59
Harmonization and bug-fixing for selector and filter behavior on unknown types. (#9484)
* Harmonization and bug-fixing for selector and filter behavior on unknown types.

- Migrate ValueMatcherColumnSelectorStrategy to newer ColumnProcessorFactory
  system, and set defaultType COMPLEX so unknown types can be dynamically matched.
- Remove ValueGetters in favor of ColumnComparisonFilter doing its own thing.
- Switch various methods to use convertObjectToX when casting to numbers, rather
  than ad-hoc and inconsistent logic.
- Fix bug in RowBasedExpressionColumnValueSelector: isBindingArray should return
  true even for 0- or 1- element arrays.
- Adjust various javadocs.

* Add throwParseExceptions option to Rows.objectToNumber, switch back to that.

* Update tests.

* Adjust moment sketch tests.
2020-03-10 07:15:57 -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 75e2051195
Convert array_contains() and array_overlaps() into native filters if possible (#9487)
* Convert array_contains() and array_overlaps() into native filters if
possible

* make spotbugs happy and fix null results when null compatible
2020-03-09 22:50:38 -07:00
Jonathan Wei 0136dba95d
Add option to control join filter rewrites (#9472)
* Add option to control join filter rewrites

* Fix inspections
2020-03-09 17:36:07 -07:00
Clint Wylie a677664811
allow optimization of single multi-value column input expr with repeated identifier (#9425)
* allow optimization of single multi-value column input expr with repeated identifier

* add test
2020-03-06 12:53:32 -08:00
Julian Jaffe eda03630d0
Add OnHeapMemorySegmentWriteOutMediumFactory (#9454)
* Add OnHeapMemorySegmentWriteOutMediumFactory

Add a factory for OnHeapMemorySegmentWriteOutMedium to support direct writing via Spark.

* Register OnHeapMemorySegmentWriteOutMediumFactory.

Register OnHeapMemorySegmentWriteOutMediumFactory with SegmentWriteOutMediumFactory.

* Remove unnecessary throws

The base `makeSegmentWriteOutMedium` throws an IOException, but the particular implementation of OnHeapMemorySegmentWriteOutMediumFactory does not throw a checked exception.

* Update SegmentWriteOutMedium docs to include onHeapMemory

Update the SegmentWriteOutMedium section of the indexing docs to include a description of the new OnHeapSegmentMediumWriteOut option.
2020-03-05 22:34:08 -08:00
Jihoon Son 3016057178
Make Transform an ExtensionPoint (#9319)
* Make Transform an ExtensionPoint

* Add transform to the list of documented extensions

* Add example transform implementation
2020-03-04 12:13:14 -08:00
Gian Merlino 1fd865b7c1
BufferArrayGrouper: Fix potential overflow in requiredBufferCapacity. (#9435)
* BufferArrayGrouper: Fix potential overflow in requiredBufferCapacity.

If cardinality was high, the computation could overflow an int. There
were tests for this, but the tests were wrong.

* Nicer.
2020-02-28 14:27:52 -08:00
Gian Merlino 81d8be6e39
CacheStrategy: Improve Javadocs. (#9280)
* CacheStrategy: Improve Javadocs.

* Update processing/src/main/java/org/apache/druid/query/CacheStrategy.java

Co-Authored-By: Suneet Saldanha <44787917+suneet-s@users.noreply.github.com>

Co-authored-by: Suneet Saldanha <44787917+suneet-s@users.noreply.github.com>
2020-02-28 11:30:58 -08:00
Gian Merlino ef3d24e886
Add javadocs for enableFilterPushDown. (#9423) 2020-02-26 22:07:33 -08:00
Gian Merlino c9faf3e148
Add SQL GROUPING SETS support. (#9122)
* Add SQL GROUPING SETS support.

Built on top of the subtotalsSpec feature in the groupBy query. This also involves
two changes to subtotalsSpec:

- Alter behavior so limitSpec is applied after subtotalsSpec, rather than applied to
  each grouping set. This is more in line with SQL standard behavior. I think it is okay
  to make this change, since the old behavior was not documented, so users should
  hopefully not be depending on it.
- Fix a bug where virtual columns were included in the subtotal queries, but they
  should not have been.

Also fixes two bugs in query equality checking:

- BaseQuery: Use getDuration() instead of "duration" in equals and hashCode, since the
  latter is lazily initialized and might be null in one query but not the other.
- GroupByQuery: Include subtotalsSpec in equals and hashCode.

* Fix bugs.

* Fix tests.

* PR updates.

* Grouping class hygiene.
2020-02-26 08:52:39 -08:00
Jonathan Wei 5ce9c81b68
Add join prefix duplicate/shadowing check (#9384)
* Add join prefix duplicate/shadowing check

* Fix format string

* PR comments

* PR comment

* Optimize loop PR comment
2020-02-25 18:17:23 -08:00
Clint Wylie 6d8dd5ec10
string -> expression -> string -> expression (#9367)
* add Expr.stringify which produces parseable expression strings, parser support for null values in arrays, and parser support for empty numeric arrays

* oops, macros are expressions too

* style

* spotbugs

* qualified type arrays

* review stuffs

* simplify grammar

* more permissive array parsing

* reuse expr joiner

* fix it
2020-02-21 15:43:02 -08:00
Jonathan Wei cab08f941d
Fix join filter push down post-join virtual column handling (#9373)
* Fix join filter push down post-join virtual column handling

* Remove unused adapter param, update javadocs

* Fix TC

* Update processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java

Co-Authored-By: Suneet Saldanha <44787917+suneet-s@users.noreply.github.com>

* Address PR comments

Co-authored-by: Suneet Saldanha <44787917+suneet-s@users.noreply.github.com>
2020-02-19 15:51:05 -08:00
Chi Cao Minh e7eb45e648
Run IntelliJ inspections on Travis (#9179)
* Run IntelliJ inspections on Travis

Running IntelliJ inspections currently takes about 90 minutes, but they
can be run in about 30 minutes on Travis.

* Restore assert statements
2020-02-19 11:34:19 +03:00
Jonathan Wei 73a0181e34
Fix handling for columns that appear multiple times in join conditions (#9362)
* Fix handling for columns that appear multiple times in join conditions

* Remove unneeded comment

* Fix test
2020-02-17 10:54:04 -08:00
Suneet Saldanha b1f38131af
Fix timestamp extract fn to match postgreSQL (#9337)
* Fix timestamp extract fn to match postgres

Update the timestamp extract function so that it matches the PostgreSQL docs.
Examples from the PostgreSQL docs were added as tests for DECADE, CENTURY
and MILLENIUM extraction.

There were bugs in CENTURY and MILLENIUM that were spotted because of intelliJ
inspections - 'Integer division in floating point context'

* Update CalciteQueryTest

* remove useless round

* mark integer division as an error
2020-02-12 15:39:19 -08:00
Maytas Monsereenusorn c30579e47b
ANY Aggregator should not skip null values implementation (#9317)
* ANY Aggregator should not skip null values implementation

* add tests

* add more tests

* Update documentation

* add more tests

* address review comments

* optimize StringAnyBufferAggregator

* fix failing tests

* address pr comments
2020-02-12 14:01:41 -08:00
Jonathan Wei b2c00b3a79
Add query context option to disable join filter push down (#9335) 2020-02-11 15:31:34 -08:00
Suneet Saldanha ea006dc72a
Optimize TimeExtractionTopNAlgorithm (#9336)
When the time extraction Top N algorithm is looking for aggregators, it makes
2 calls to hashCode on the key. Use Map#computeIfAbsent instead so that the
hashCode is calculated only once
2020-02-10 14:26:10 -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
Jonathan Wei ad8afc565c
Join filter pushdown initial implementation (#9301)
* Join filter pushdown initial implementation

* Fix test and spotbugs check

* Address PR comments

* More PR comments

* Address some PR comments

* Address more PR comments

* Fix TC failures and address PR comments
2020-02-07 16:23:37 -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
Gian Merlino 0aa7a2a3ee
Add HashVectorGrouper based on MemoryOpenHashTable. (#9314)
* Add HashVectorGrouper based on MemoryOpenHashTable.

Additional supporting changes:

1) Modifies VectorGrouper interface to use Memory instead of ByteBuffers.
2) Modifies BufferArrayGrouper to match the new VectorGrouper interface.
3) Removes "implements VectorGrouper" from BufferHashGrouper.

* Fix comment.

* Fix another comment.

* Remove unused stuff.

* Include hoisted bounds checks.

* Checks against too-large keySpaces.
2020-02-06 15:29:14 -08:00
Gian Merlino 3ef5c2f2e8
Add MemoryOpenHashTable, a table similar to ByteBufferHashTable. (#9308)
* Add MemoryOpenHashTable, a table similar to ByteBufferHashTable.

With some key differences to improve speed and design simplicity:

1) Uses Memory rather than ByteBuffer for its backing storage.
2) Uses faster hashing and comparison routines (see HashTableUtils).
3) Capacity is always a power of two, allowing simpler design and more
   efficient implementation of findBucket.
4) Does not implement growability; instead, leaves that to its callers.
   The idea is this removes the need for subclasses, while still giving
   callers flexibility in how to handle table-full scenarios.

* Fix LGTM warnings.

* Adjust dependencies.

* Remove easymock from druid-benchmarks.

* Adjustments from review.

* Fix datasketches unit tests.

* Fix checkstyle.
2020-02-04 19:57:59 -08:00
Chi Cao Minh 0d2b16c1d0
Speed up joins on indexed tables with string keys (#9278)
* Speed up joins on indexed tables with string keys

When joining on index tables with string keys, caching the computation
of row id to row numbers improves performance on the
JoinAndLookupBenchmark.joinIndexTableStringKey* benchmarks by about 10%
if the column cache is enabled an by about 100% if the column cache is
disabled.

* Faster cache impl and handle unknown cardinality

* Remove unused dependency

* Hoist cardinality check outside of hot loop

* Fix dummy DimensionSelector for tests
2020-02-04 17:34:55 -08:00
Suneet Saldanha 33a97dfaae
Guicify druid sql module (#9279)
* Guicify druid sql module

Break up the SQLModule in to smaller modules and provide a binding that
modules can use to register schemas with druid sql.

* fix some tests

* address code review

* tests compile

* Working tests

* Add all the tests

* fix up licenses and dependencies

* add calcite dependency to druid-benchmarks

* tests pass

* rename the schemas
2020-02-04 11:33:48 -08:00
Gian Merlino b411443d22
SQL join support for lookups. (#9294)
* SQL join support for lookups.

1) Add LookupSchema to SQL, so lookups show up in the catalog.
2) Add join-related rels and rules to SQL, allowing joins to be planned into
   native Druid queries.

* Add two missing LookupSchema calls in tests.

* Fix tests.

* Fix typo.
2020-01-31 23:51:16 -08:00
Gian Merlino 85d0d57fc9
Fix timestamp_format expr outside UTC timeZone. (#9282) 2020-01-31 16:20:35 -08:00
Gian Merlino 204ba9966f
Add LookupJoinableFactory. (#9281)
* Add LookupJoinableFactory.

Enables joins where the right-hand side is a lookup. Includes an
integration test.

Also, includes changes to LookupExtractorFactoryContainerProvider:

1) Add "getAllLookupNames", which will be needed to eventually connect
   lookups to Druid's SQL catalog.
2) Convert "get" from nullable to Optional return.
3) Swap out most usages of LookupReferencesManager in favor of the
   simpler LookupExtractorFactoryContainerProvider interface.

* Fixes for tests.

* Fix another test.

* Java 11 message fix.

* Fixups.

* Fixup benchmark class.
2020-01-30 14:46:21 -08:00
Suneet Saldanha 6b44d4aa80
Add getRightEquiConditionKeys to JoinConditionAnalysis (#9287)
* Add getRightColumns to JoinConditionAnalysis

This change other implementations of JoinableFactory to ask the analysis
for the right key columns instead of having to calculate it themselves.

* Address some review comments

* more code review stuff
2020-01-29 22:31:29 -08:00
Chi Cao Minh a1494c30e0
Join microbenchmark (#9267)
Add microbenchmark for joins. Enabling the column cache improves
performance by ~70% for the benchmarks for joins with string keys.
Adjusting LookupJoinMatcher.matchCondition() to have fewer branches,
improves performance by ~10% for the benchmarks for joins with lookups.
2020-01-29 14:08:19 -08: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
Suneet Saldanha 6ee0afa8e5
Rename MapDataSourceJoinableFactoryWarehouse (#9275) 2020-01-28 19:00:07 -08:00
Suneet Saldanha 0ccfe5ca89 Expose JoinableFactory through Guice Bindings (#9271)
* Make JoinableFactory an extension point

This change makes it so that extensions can register a JoinableFactory that
should be used for a DataSource.

Extensions can provide the factories via DruidBinders#joinableFactoryBinder
Known DataSources - like InlineDataSource are provided in the
JoinableFactoryModule. This module installs a FactoryWarehouse that is
used to decide which factory should be used to generate the Joinable for
the provided DataSource.

The ExtensionPoint is marked as Beta since it is not yet clear if this
needs to remain available to other extensions or if the best way to
register a factory is by using the datasource class.

* Add module test

* remove useless bindings in test

* remove ExtensionPoint annotation

* Make LifecycleLock not final to help with testing
2020-01-28 13:59:06 -08:00
Clint Wylie 14253c63d6
removed AsyncQueryRunner since was only used by removed interval chunking stuff (#9252) 2020-01-27 18:53:17 -08:00
Clint Wylie 36c5efe2ab fix some issues with filters on numeric columns with nulls (#9251)
* fix issue with long column predicate filters and nulls

* dang

* uncomment a thing

* styles

* oops

* allcaps

* review stuff
2020-01-27 18:01:01 -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
Suneet Saldanha a2939bbd1a Optimize JoinCondition matching (#9200)
* Optimize JoinCondition matching

The LookupJoinMatcher needs to check if a condition is always true or false
multiple times. This can be pre-computed to speed up the match checking

This change reduces the time it takes to perform a for joining on a long key
from ~ 36 ms/op to 23 ms/ op

* Rename variables

* fix typo
2020-01-21 09:11:50 -08:00
Clint Wylie 8011211a0c first/last aggregators and nulls (#9161)
* null handling for numeric first/last aggregators, refactor to not extend nullable numeric agg since they are complex typed aggs

* initially null or not based on config

* review stuff, make string first/last consistent with null handling of numeric columns, more tests

* docs

* handle nil selectors, revert to primitive first/last types so groupby v1 works...
2020-01-20 11:51:54 -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
Clint Wylie f0dddaa51a fix topn aggregation on numeric columns with null values (#9183)
* fix topn issue with aggregating on numeric columns with null values

* adjustments

* rename

* add more tests

* fix comments

* more javadocs

* computeIfAbsent
2020-01-17 18:12:24 -08:00
Maytas Monsereenusorn 68ed2a2c8f Fix LATEST / EARLIEST Buffer Aggregator does not work on String column (#9197)
* fix buff limit bug

* add tests

* add test

* add tests

* fix checkstyle
2020-01-16 21:02:37 -08:00
Gian Merlino 448da78765 Speed up String first/last aggregators when folding isn't needed. (#9181)
* Speed up String first/last aggregators when folding isn't needed.

Examines the value column, and disables fold checking via a needsFoldCheck
flag if that column can't possibly contain SerializableLongStringPairs. This
is helpful because it avoids calling getObject on the value selector when
unnecessary; say, because the time selector didn't yield an earlier or later
value.

* PR comments.

* Move fastLooseChop to StringUtils.
2020-01-16 21:02:02 -08:00
Gian Merlino bd49ec03bc
Move result-to-array logic from SQL layer into QueryToolChests. (#9130)
* Move result-to-array logic from SQL layer into QueryToolChests.

* Checkstyle adjustment.

* Fix typo.
2020-01-16 15:42:10 -08:00
Gian Merlino bfcb30e48f
Add javadocs and small improvements to join code. (#9196)
A follow-up to #9111.
2020-01-16 15:25:38 -08:00
Maytas Monsereenusorn 42359c93dd Implement ANY aggregator (#9187)
* Implement ANY aggregator

* Add copyright headers

* Add unit tests

* fix BufferAggregator

* Fix bug in BufferAggregator

* hook up the SQL command

* add check for buffer aggregator

* Address comment

* address comments

* add docs

* Address comments

* add more tests for numeric columns that have null values when run in sql compatible null mode

* fix checkstyle errors

* fix failing tests

* fix failing tests
2020-01-16 14:40:32 -08:00
Gian Merlino a87db7f353
Add HashJoinSegment, a virtual segment for joins. (#9111)
* Add HashJoinSegment, a virtual segment for joins.

An initial step towards #8728. This patch adds enough functionality to implement a joining
cursor on top of a normal datasource. It does not include enough to actually do a query. For
that, future patches will need to wire this low-level functionality into the query language.

* Fixups.

* Fix missing format argument.

* Various tests and minor improvements.

* Changes.

* Remove or add tests for unused stuff.

* Fix up package locations.
2020-01-16 13:14:20 -08:00
Lucas Capistrant 4716e0b585 Fix concurrency of ComplexMetrics.java (#9134) 2020-01-15 17:19:45 +03:00
Clint Wylie 85219ece13 fix null handling for arithmetic post aggregator comparator (#9159)
* fix null handling for arithmetic postagg comparator, add test for comparator for min/max/quantile postaggs in histogram ext

* fix
2020-01-10 13:49:19 -08:00
Jonathan Wei 8c53818fa9
Add numeric nulls to sample data, fix some numeric null handling issues (#9154)
* Fix LongSumAggregator comparator null handling

* Remove unneeded GroupBy test change

* Checkstyle

* Update other processing tests for new sample data

* Remove unused code

* Fix SearchQueryRunner column selectors

* Fix DimensionIndexer null handling and ScanQueryRunnerTest

* Fix TeamCity errors
2020-01-10 13:49:06 -08:00
Clint Wylie c248e00984 fix moment sketch null handling (#9075) 2020-01-07 14:15:59 -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
Clint Wylie 8ccce9857a fix vectorized query engine numeric filter matchers against null values (#9063)
* fix druid-sql issue with filtering numeric columns by null values

* fix vector numeric column matchers to check null vector for null matches
2019-12-20 13:15:48 -08:00
Gian Merlino d452cbbb82 GenericIndexedWriter: Fix issue when writing large values to large columns. (#9029) 2019-12-13 15:33:14 -08:00
Clint Wylie 4327892b84 modify multi-value expression transformation behavior to not treat re-use of the same input as a candidate for cartesian mapping (#8957) 2019-12-09 20:38:15 -08:00
Clint Wylie 06cd30460e
add query metrics for broker parallel merges, off by default (#8981)
* add a bunch of metrics for broker parallel merges, off by default, and tests

* fix tests

* review stuffs

* propogateIfPossible
2019-12-06 13:42:53 -08:00
Q 391646123e Fix double-checked locking in predicate suppliers in BoundDimFi… (#8974)
* Fix double-checked locking in predicate suppliers in BoundDimFilter

* Fix double-checked locking in predicate suppliers in BoundDimFilter

* 1. Use Suppliers.memoize() to initialize and publish singleton.

2. Fix coding style.

* Fix coding style

* Fix double-checked locking bug for predicate suppliers in InDimFilter
2019-12-04 20:01:52 +03: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
Chi Cao Minh fba876b607 Update jackson to 2.9.10 (#8940)
Addresses security vulnerabilities:

- sonatype-2016-0397:
  https://github.com/FasterXML/jackson-core/issues/315

- sonatype-2017-0355:
  https://github.com/FasterXML/jackson-core/pull/322
2019-11-26 21:41:14 -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
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
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
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
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
Gian Merlino ce4ee42459 Fix LIKE filter wildcards to match newlines. (#8863) 2019-11-13 23:00:54 -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
Clint Wylie 9ed9a80b9d optimize numeric column null value checking for low filter selectivity (more rows) (#8822)
* use peekable iterator for numeric column selector null checking instead of bitmap.get for those sweet sweet nanoseconds

* remove unused method

* slight optimization i think

* remove clone from wrappers since we do not use and is confusing

* fixes and tests

* int instead of Integer

* fix it

* fixes, more tests

* fix
2019-11-13 10:53:46 -08:00
Gian Merlino 0e8c3f74d0 SQL: EARLIEST, LATEST aggregators. (#8815)
* SQL: EARLIEST, LATEST aggregators.

I chose these names instead of FIRST, LAST because those are already
reserved functions in Calcite that mean something different. I think
these are also better names anyway.

* Finalify.

* SQL updates.

* Adjust aggregator calls.

* Validations, test updates.

* Review docs.
2019-11-08 16:29:25 -08:00
Gian Merlino c204d68376 Fixes, adjustments to numeric null handling and string first/last aggregators. (#8834)
There is a class of bugs due to the fact that BaseObjectColumnValueSelector
has both "getObject" and "isNull" methods, but in most selector implementations
and most call sites, it is clear that the intent of "isNull" is only to apply
to the primitive getters, not the object getter. This makes sense, because the
purpose of isNull is to enable detection of nulls in otherwise-primitive columns.
Imagine a string column with a numeric selector built on top of it. You would
want it to return isNull = true, so numeric aggregators don't treat it as
all zeroes.

Sometimes this design leads people to accidentally guard non-primitive get
methods with "selector.isNull" checks, which is improper.

This patch has three goals:

1) Fix null-handling bugs that already exist in this class.
2) Make interface and doc changes that reduce the probability of future bugs.
3) Fix other, unrelated bugs I noticed in the stringFirst and stringLast
   aggregators while fixing null-handling bugs. I thought about splitting this
   into its own patch, but it ended up being tough to split from the
   null-handling fixes.

For (1) the fixes are,

- Fix StringFirst and StringLastAggregatorFactory to stop guarding getObject
  calls on isNull, by no longer extending NullableAggregatorFactory. Now uses
  -1 as a sigil value for null, to differentiate nulls and empty strings.
- Fix ExpressionFilter to stop guarding getObject calls on isNull. Also, use
  eval.asBoolean() to avoid calling getLong on the selector after already
  calling getObject.
- Fix ObjectBloomFilterAggregator to stop guarding DimensionSelector calls
  on isNull. Also, refactored slightly to avoid the overhead of calling
  getObject followed by another getter (see BloomFilterAggregatorFactory for
  part of this).

For (2) the main changes are,

- Remove the "isNull" method from BaseObjectColumnValueSelector.
- Clarify "isNull" doc on BaseNullableColumnValueSelector.
- Rename NullableAggregatorFactory -> NullbleNumericAggregatorFactory to emphasize
  that it only works on aggregators that take numbers as input.
- Similar naming changes to the Aggregator, BufferAggregator, and AggregateCombiner.
- Similar naming changes to helper methods for groupBy, ValueMatchers, etc.

For (3) the other fixes for StringFirst and StringLastAggregatorFactory are,

- Fixed buffer overrun in the buffer aggregators when some characters in the string
  code into more than one byte (the old code used "substring" to apply a byte limit,
  which is bad). I did this by introducing a new StringUtils.toUtf8WithLimit method.
- Fixed weird IncrementalIndex logic that led to reading nulls for the timestamp.
- Adjusted weird StringFirst/Last logic that worked around the weird IncrementalIndex
  behavior.
- Refactored to share code between the four aggregators.
- Improved test coverage.
- Made the base stringFirst, stringLast aggregators adaptive, and streamlined the
  xFold versions into aliases. The adaptiveness is similar to how other aggregators
  like hyperUnique work.
2019-11-07 17:46:59 -08:00
Clint Wylie 7aafcf8bca parallel broker merges on fork join pool (#8578)
* sketch of broker parallel merges done in small batches on fork join pool

* fix non-terminating sequences, auto compute parallelism

* adjust benches

* adjust benchmarks

* now hella more faster, fixed dumb

* fix

* remove comments

* log.info for debug

* javadoc

* safer block for sequence to yielder conversion

* refactor LifecycleForkJoinPool into LifecycleForkJoinPoolProvider which wraps a ForkJoinPool

* smooth yield rate adjustment, more logs to help tune

* cleanup, less logs

* error handling, bug fixes, on by default, more parallel, more tests

* remove unused var

* comments

* timeboundary mergeFn

* simplify, more javadoc

* formatting

* pushdown config

* use nanos consistently, move logs back to debug level, bit more javadoc

* static terminal result batch

* javadoc for nullability of createMergeFn

* cleanup

* oops

* fix race, add docs

* spelling, remove todo, add unhandled exception log

* cleanup, revert unintended change

* another unintended change

* review stuff

* add ParallelMergeCombiningSequenceBenchmark, fixes

* hyper-threading is the enemy

* fix initial start delay, lol

* parallelism computer now balances partition sizes to partition counts using sqrt of sequence count instead of sequence count by 2

* fix those important style issues with the benchmarks code

* lazy sequence creation for benchmarks

* more benchmark comments

* stable sequence generation time

* update defaults to use 100ms target time, 4096 batch size, 16384 initial yield, also update user docs

* add jmh thread based benchmarks, cleanup some stuff

* oops

* style

* add spread to jmh thread benchmark start range, more comments to benchmarks parameters and purpose

* retool benchmark to allow modeling more typical heterogenous heavy workloads

* spelling

* fix

* refactor benchmarks

* formatting

* docs

* add maxThreadStartDelay parameter to threaded benchmark

* why does catch need to be on its own line but else doesnt
2019-11-07 11:58:46 -08:00
Clint Wylie 3ff5e02237 remove select query (#8739)
* remove select query

* thanks teamcity

* oops

* oops

* add back a SelectQuery class that throws RuntimeExceptions linking to docs

* adjust text

* update docs per review

* deprecated
2019-10-30 19:29:56 -07:00
Jihoon Son f5b9bf5525 Cluster-wide configuration for query vectorization (#8657)
* Cluster-wide configuration for query vectorization

* add doc

* fix build

* fix doc

* rename to QueryConfig and add javadoc

* fix checkstyle

* fix variable names
2019-10-23 21:44:28 +08:00
Jonathan Wei d88075237a
Add initial SQL support for non-expression sketch postaggs (#8487)
* Add initial SQL support for non-expression sketch postaggs

* Checkstyle, spotbugs

* checkstyle

* imports

* Update SQL docs

* Checkstyle

* Fix theta sketch operator docs

* PR comments

* Checkstyle fixes

* Add missing entries for HLL sketch module

* PR comments, add round param to HLL estimate operator, fix optional HLL param
2019-10-18 14:59:44 -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
Himanshu 46ddaf3aa1 fix sorting for resultRow object when numeric dimension not in limitSpec (#8645) 2019-10-08 16:37:15 -07:00
Himanshu c078ed40fd
groupBy query: optional limit push down to segment scan (#8426)
* groupBy query: optional limit push down to segment scan

* make segment level limit push down configurable

* fix teamcity errors

* fix segment limit pushdown flag handling on query level config override

* use equals for comparator check

* fix sql and null handling

* fix unused imports

* handle null offset in NullableValueGroupByColumnSelectorStrategy for buffer comparator similar to RowBasedGrouperHelper.NullableRowBasedKeySerdeHelper
2019-10-08 15:35:07 -07:00
Clint Wylie 7781820dea JsonParserIterator.init future timeout (#8550)
* add timeout support for JsonParserIterator init future

* add queryId

* should be less than 1

* fix

* fix npe

* fix lgtm

* adjust exception, nullable

* fix test

* refactor

* revert queryId change

* add log.warn to tie exception to json parser iterator
2019-09-27 09:13:37 +09:00
Himanshu 9f1f5e115c
doubleMean aggregator to be used at query time (#8459)
* doubleMean aggregator for computing mean

* make docs

* build fixes

* address review comment: handle null args
2019-09-26 08:04:33 -07:00
Gian Merlino d96ca9bd61 Fix serde of FilterTuning maxCardinalityToUseBitmapIndex. (#8551) 2019-09-17 12:46:46 -07:00
Chi Cao Minh baec3a06e9 Fix IntelliJ inspection error (#8553)
Change by #8535 causes TeamCity inspection error in CI (although it does
not show the error in the local IDE).
2019-09-17 12:45:25 -07:00
Benedict Jin c6f4f09557 Fix missing space in string literal and spurious Javadoc @param tags from LGTM (#8491)
* Fix missing space in string literal

* Fix spurious Javadoc @param tags
2019-09-16 14:37:47 +05:30
Clint Wylie df14e5d696
fix caching bug with multi-column group-by (#8535)
* fix caching bug with multi-column group-by

* review
2019-09-13 17:41:23 -07:00
Benedict Jin 9fa3407596 Suppress index-out-of-bounds warning from LGTM about loop unrolling (#8380)
* Suppress index-out-of-bounds warning from LGTM about loop unrolling

* Remove space

* Patch comments
2019-09-06 14:46:33 -07:00
Himanshu 1fe4ecf17a
StringDictionaryEncodedColumn dimSelector to return CARDINALITY_UNKNOWN with extractionFn (#8433)
* update DimensionDictionarySelector.getValueCardinality() javadoc

* unknown cardinality in StringDictionaryEncodedColumn dim selector

* revert StringDictionaryEncodedColumn change as that fails GroupBy-v1 execution for many working queries

* fix/add more comments
2019-09-06 14:19:25 -07:00
Jonathan Wei f36fd73f60 Speed up StringDimensionIndexer.estimateEncodedKeyComponentSize (#8466)
* Speed up StringDimensionIndexer.estimateEncodedKeyComponentSize

* Remove print

* Move benchmark, add header
2019-09-04 20:26:04 -07:00
Benedict Jin de18840412
Fix inconsistent equals and hashCode (#8381)
* Fix inconsistent equals and hashCode

* Patch comments

* Remove equals and hashCode from InsensitiveContainsSearchQuerySpec
2019-09-04 13:48:08 +08:00
Himanshu ee4ebb496a
make single/multi value string column handling official in aggregation (#8428) 2019-09-03 13:47:09 -07:00
Himanshu 5c3db41c2b
string column handling for long/float min/max/sum aggregators (#8319)
* string column handling for long min/max/sum aggregators

* add apache license to new files

* use 'L' as suffix for long literal instead of 'l'

* return null in ParallelCombiner.SettableColumnSelectorFactory.getColumnCapabilities(String) as is required by contract of ColumnSelectorFactory interface

* fix more tests
2019-08-27 16:10:59 -07:00
Himanshu d5d170f866
skip unnecessary aggregate(..) calls with LimitedBufferHashGrouper (#8412)
* skip unnecessary aggregate(..) calls with LimitedBufferHashGrouper

* remove unused bucketWasUsed arg from canSkipAggregate(..)
2019-08-27 15:01:07 -07:00
Himanshu 4d87a19547
Logging emitter to publish query and other metric events as valid json objects (#8359)
* LoggingEmitter: print event as json

* use DefaultRequestLogEventBuilderFactory in emitting request logger by default

* print context in query metric as json

* removed unused jsonMapper from DefaultQueryMetrics

* add comment

* remove change to DefaultRequestLogEventBuilderFactory.java
2019-08-27 15:00:23 -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
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
Xavier Léauté 20f7db5d22
Fix ConcurrentModificationException in JDK11 (#8391)
When building column/dimension selectors, calling computeIfAbsent can
cause the applied function to modify the same cache through virtual
column references. The JDK11 map implementation detects this change and
will throw an exception.

This fix – while not as elegant – breaks the single call into two
steps to avoid this problem.
2019-08-24 18:24:50 -04:00
Jonathan Wei 368ace4e87
Fix ClassCastException for TopN with long-type dimension (#8349)
* Fix ClassCastException for TopN with long-type dimension

* Add DimValHolderTest
2019-08-23 14:55:31 -05: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
Clint Wylie c87b68d2a4 use Number instead of long for response context (#8342)
* use Number instead of long for response context to be forgiving of json serde to int or long

* test that encounters issue without fix

* now with more test

* is ints
2019-08-20 19:05:49 -07:00
Jonathan Wei e2a25fb51e
Add logging for LZ4Factory instance type (#8341) 2019-08-20 15:24:53 -05:00
Fokko Driesprong 818bf4990c Enable Spotbugs NP_NONNULL_RETURN_VIOLATION (#8234) 2019-08-20 17:23:46 +03:00
Benedict Jin 781873ba53 Fix resource leak (#8337)
* Fix resource leak

* Patch comments
2019-08-20 12:55:41 +03:00
Himanshu 176da53996
make double sum/min/max agg work on string columns (#8243)
* make double sum/min/max agg work on string columns

* style and compilation fixes

* fix tests

* address review comments

* add comment on SimpleDoubleAggregatorFactory

* make checkstyle happy
2019-08-13 15:55:14 -07: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 8fa114c349 Fix bugs in overshadowableManager and add unit tests (#8222)
* Fix bugs in overshadowableManager and add unit tests

* Fix SegmentManager

* add segment manager test

* Address comments

* Address comments
2019-08-07 15:51:21 -05:00
Fokko Driesprong 7702005f8f Use Closer instead of List<Closeable> (#8235)
* Use Closer instead of List<Closeable>

* Process comments

* Catch an Exception instead

* Removed unused import
2019-08-07 14:29:03 +08:00
Himanshu 4507a4f8f1 fix merging of groupBy subtotal spec results (#8109)
* fix merging of groupBy subtotal spec results

* add post agg to subtotals spec ut

* add comment

* remove unnecessary agg transformation

* fix build

* fix test

* ignore unknown columns in ordering spec

* change variable names based on comment for easy read

* formatting

* don't ignore unknown columns in DefaultLimitSpec to not change existing behavior

* handle limit spec columns correctly

* uncomment inadvertantly commented lines

* GroupByStrategyV2 changes

* test changes wip

* more fixes to handle merge buffer closing and limit spec

* uncomment line commented accidentally
2019-08-06 07:06:28 -07:00
Samarth Jain 93cf9d4ad4 SQL support for t-digest based sketch aggregators (#8100)
* SQL support for t-digest based sketch aggregators

* Fix teamcity errors

* Add missing dependencies

* Remove unused dependency

* Address code review comments

* Add checks for compression param
2019-08-05 12:01:42 -07:00
Eugene Sevastianov 3f3162b85e Enum of ResponseContext keys (#8157)
* Refactored ResponseContext and aggregated its keys into Enum

* Added unit tests for ResponseContext and refactored the serialization

* Removed unused methods

* Fixed code style

* Fixed code style

* Fixed code style

* Made SerializationResult static

* Updated according to the PR discussion:

Renamed an argument

Updated comparator

Replaced Pair usage with Map.Entry

Added a comment about quadratic complexity

Removed boolean field with an expression

Renamed SerializationResult field

Renamed the method merge to add and renamed several context keys

Renamed field and method related to scanRowsLimit

Updated a comment

Simplified a block of code

Renamed a variable

* Added JsonProperty annotation to renamed ScanQuery field

* Extension-friendly context key implementation

* Refactored ResponseContext: updated delegate type, comments and exceptions

Reducing serialized context length by removing some of its'
collection elements

* Fixed tests

* Simplified response context truncation during serialization

* Extracted a method of removing elements from a response context and
added some comments

* Fixed typos and updated comments
2019-08-03 12:05:21 +03:00
Clint Wylie e7c6deac76 optimize single input column multi-value expressions (#8047)
* optimize single input column multi-value expressions

* javadocs

* merge fixup

* vectorization fixup

* more fixes

* more docs

* more links

* empty

* javadocs are hard

* suppress javadoc refs issue

* fix it
2019-08-02 13:21:25 -07:00
Fokko Driesprong 91743eeebe Spotbugs: NP_NONNULL_PARAM_VIOLATION (#8129) 2019-08-02 19:20:22 +03:00
Chi Cao Minh 7783b31846 Add IPv4 druid expressions (#8197)
* Add IPv4 druid expressions

New druid expressions for filtering IPv4 addresses:
- ipv4address_match: Check if IP address belongs to a subnet
- ipv4address_parse: Convert string IP address to long
- ipv4address_stringify: Convert long IP address to string

These expressions operate on IP addresses represented as either strings
or longs, so that they can be applied to dimensions with mixed
representation of IP addresses. The filtering is more efficient when
operating on IP addresses as longs. In other words, the intended use
case is:

1) Use ipv4address_parse to convert to long at ingestion time
2) Use ipv4address_match to filter (on longs) at query time
3) Use ipv4adress_stringify to convert to (readable) string at query
time

* Fix licenses and null handling

* Simplify IPv4 expressions

* Fix tests

* Fix check for valid ipv4 address string
2019-08-01 11:45:04 -07: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
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
Fokko Driesprong 714a2726a7 Spotbugs: RC_REF_COMPARISON_BAD_PRACTICE_BOOLEAN (#8076)
* Spotbugs: RC_REF_COMPARISON_BAD_PRACTICE_BOOLEAN

* Use Objects instead
2019-07-24 16:17:02 -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 83514958db remove unnecessary lock in ForegroundCachePopulator leading to a lot of contention (#8116)
* remove unecessary lock in ForegroundCachePopulator leading to a lot of contention

* mutableboolean, javadocs,document some cache configs that were missing

* more doc stuff

* adjustments

* remove background documentation
2019-07-23 10:57:59 -07:00
Fokko Driesprong e1a745717e Spotbugs: NP_STORE_INTO_NONNULL_FIELD (#8021) 2019-07-21 21:23:47 +08:00
Sashidhar Thallam ea4bad7836 Druid SQL EXTRACT time function - adding support for additional Time Units (#8068)
* 1. Added TimestampExtractExprMacro.Unit for MILLISECOND 2. expr eval for MILLISECOND 3. Added a test case to test extracting millisecond from expression. #7935

* 1. Adding DATASOURCE4 in tests. 2. Adding test TimeExtractWithMilliseconds

* Fixing testInformationSchemaTables test

* Fixing failing tests in DruidAvaticaHandlerTest

* Adding cannotVectorize() call before the test

* Extract time function - Adding support for MICROSECOND, ISODOW, ISOYEAR and CENTURY time units, documentation changes.

* Adding MILLISECOND in test case

* Adding support DECADE and MILLENNIUM, updating test case and documentation

* Fixing expression eval for DECADE and MILLENIUM
2019-07-19 20:38:32 -07:00
Clint Wylie 03e55d30eb
add CachingClusteredClient benchmark, refactor some stuff (#8089)
* add CachingClusteredClient benchmark, refactor some stuff

* revert WeightedServerSelectorStrategy to ConnectionCountServerSelectorStrategy and remove getWeight since felt artificial, default mergeResults in toolchest implementation for topn, search, select

* adjust javadoc

* adjustments

* oops

* use it

* use BinaryOperator, remove CombiningFunction, use Comparator instead of Ordering, other review adjustments

* rename createComparator to createResultComparator, fix typo, firstNonNull nullable parameters
2019-07-18 13:16:28 -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
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
Gian Merlino ffa25b7832
Query vectorization. (#6794)
* Benchmarks: New SqlBenchmark, add caching & vectorization to some others.

- Introduce a new SqlBenchmark geared towards benchmarking a wide
  variety of SQL queries. Rename the old SqlBenchmark to
  SqlVsNativeBenchmark.
- Add (optional) caching to SegmentGenerator to enable easier
  benchmarking of larger segments.
- Add vectorization to FilteredAggregatorBenchmark and GroupByBenchmark.

* Query vectorization.

This patch includes vectorized timeseries and groupBy engines, as well
as some analogs of your favorite Druid classes:

- VectorCursor is like Cursor. (It comes from StorageAdapter.makeVectorCursor.)
- VectorColumnSelectorFactory is like ColumnSelectorFactory, and it has
  methods to create analogs of the column selectors you know and love.
- VectorOffset and ReadableVectorOffset are like Offset and ReadableOffset.
- VectorAggregator is like BufferAggregator.
- VectorValueMatcher is like ValueMatcher.

There are some noticeable differences between vectorized and regular
execution:

- Unlike regular cursors, vector cursors do not understand time
  granularity. They expect query engines to handle this on their own,
  which a new VectorCursorGranularizer class helps with. This is to
  avoid too much batch-splitting and to respect the fact that vector
  selectors are somewhat more heavyweight than regular selectors.
- Unlike FilteredOffset, FilteredVectorOffset does not leverage indexes
  for filters that might partially support them (like an OR of one
  filter that supports indexing and another that doesn't). I'm not sure
  that this behavior is desirable anyway (it is potentially too eager)
  but, at any rate, it'd be better to harmonize it between the two
  classes. Potentially they should both do some different thing that
  is smarter than what either of them is doing right now.
- When vector cursors are created by QueryableIndexCursorSequenceBuilder,
  they use a morphing binary-then-linear search to find their start and
  end rows, rather than linear search.

Limitations in this patch are:

- Only timeseries and groupBy have vectorized engines.
- GroupBy doesn't handle multi-value dimensions yet.
- Vector cursors cannot handle virtual columns or descending order.
- Only some filters have vectorized matchers: "selector", "bound", "in",
  "like", "regex", "search", "and", "or", and "not".
- Only some aggregators have vectorized implementations: "count",
  "doubleSum", "floatSum", "longSum", "hyperUnique", and "filtered".
- Dimension specs other than "default" don't work yet (no extraction
  functions or filtered dimension specs).

Currently, the testing strategy includes adding vectorization-enabled
tests to TimeseriesQueryRunnerTest, GroupByQueryRunnerTest,
GroupByTimeseriesQueryRunnerTest, CalciteQueryTest, and all of the
filtering tests that extend BaseFilterTest. In all of those classes,
there are some test cases that don't support vectorization. They are
marked by special function calls like "cannotVectorize" or "skipVectorize"
that tell the test harness to either expect an exception or to skip the
test case.

Testing should be expanded in the future -- a project in and of itself.

Related to #3011.

* WIP

* Adjustments for unused things.

* Adjust javadocs.

* DimensionDictionarySelector adjustments.

* Add "clone" to BatchIteratorAdapter.

* ValueMatcher javadocs.

* Fix benchmark.

* Fixups post-merge.

* Expect exception on testGroupByWithStringVirtualColumn for IncrementalIndex.

* BloomDimFilterSqlTest: Tag two non-vectorizable tests.

* Minor adjustments.

* Update surefire, bump up Xmx in Travis.

* Some more adjustments.

* Javadoc adjustments

* AggregatorAdapters adjustments.

* Additional comments.

* Remove switching search.

* Only missiles.
2019-07-12 12:54:07 -07:00
Clint Wylie abf9843e2a fail complex type 'serde' registration when registered type does not match expected type (#7985)
* make ComplexMetrics.registerSerde type check on register, resolves #7959

* add test

* simplify

* unused imports :/

* simplify

* burned by imports yet again

* remove unused constructor

* switch to getName

* heh oops
2019-07-11 23:03:15 -07:00
Fokko Driesprong 0aabeb4b1a Enable Spotbugs: MS_OOI_PKGPROTECT (#8022) 2019-07-08 13:17:56 +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
Gian Merlino 9b499df14e Fix ExpressionVirtualColumn capabilities; fix groupBy's improper uses of StorageAdapter#getColumnCapabilities. (#8013)
* GroupBy: Fix improper uses of StorageAdapter#getColumnCapabilities.

1) A usage in "isArrayAggregateApplicable" that would potentially incorrectly use
   array-based aggregation on a virtual column that shadows a real column.
2) A usage in "process" that would potentially use the more expensive multi-value
   aggregation path on a singly-valued virtual column. (No correctness issue, but
   a performance issue.)

* Add addl javadoc.

* ExpressionVirtualColumn: Set multi-value flag.
2019-07-05 13:17:05 -07:00
Clint Wylie 0344a020bb optimize single string dimension expression selector (#8014)
* optimize single string dimension expression selector

* more javadoc

* oops

* fix

* fix it

* import
2019-07-04 16:26:10 -07:00
Clint Wylie e6ba258197 multi-value string expression transformation fix (#8019)
* multi-value string expression transformation fix

* fixes

* more docs and test

* revert unintended doc change

* formatting

* change tostring to print binding identifier

* review fixup

* oops
2019-07-03 23:03:47 -07:00
Clint Wylie c556d44a19
more sql support for expression array functions (#7974)
* more sql support for expression array functions

* prepend/slice

* doc fixes

* fix imports

* fix tests

* add null numeric expr for proper conversions between ExprEval and Expr and back to ExprEval

* re-arrange

* imports :(

* add append/prepend test
2019-07-02 21:39:26 -07:00
Alexander Saydakov f38a62e949 theta sketch to string post agg (#7937) 2019-06-27 15:09:57 -07:00
Clint Wylie 151edeec3c
expression virtual column selector fix for expressions which produce array types (#7958)
* fix bug in multi-value string expression column selector

* more test

* imports!!

* fixes
2019-06-26 16:57:13 -07:00
Xue Yu b9c6a26c0e Use ComplexMetrics.registerSerde() across the codebase (#7925)
* refactor complexmetric registerserde

* fix error

* feedback address
2019-06-25 11:39:04 +03:00
Fokko Driesprong 82b248cc17 Spotbugs: Enable MS_SHOULD_BE_FINAL (#7946) 2019-06-23 15:42:18 -07:00
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
SandishKumarHN 01881e3a98 Use only com.google.errorprone.annotations.concurrent.GuardedBy, not javax.annotations.concurrent.GuardedBy (#7889) 2019-06-17 15:58:51 +02:00
Clint Wylie 12a1ecfc2b allow sql lookup function to take advantage of injective lookups (#7655) 2019-06-06 14:36:10 -07:00
Himanshu 0493780799
discard filter when processing subtotalsSpec (#7827) 2019-06-04 10:59:22 -07:00
Xue Yu d482da6e9b fix timestamp ceil lower bound bug (#7823) 2019-06-04 01:16:31 -07:00
litao91 55af692b56 Fix repeated expr parsing in ExpressionPostAggregation (#7791)
* Fix repeatedly expr parsing in ExpressionPostAggregation

Change-Id: Ib739fb1cbc460afeb59a255f635305441dc6997b

* Style fix and avoid code copying

Change-Id: I2d6ba3d1ae37f1fb84b6f7eaab5dab817e1980ec

* Lazilly parse expressions in ExpressionVirtualColumn and ExpressionDimFilter

Change-Id: I5ae2bb3ef9a18fbbfb5e0780c86f6bc0039edc83
2019-05-31 20:56:31 -07:00
Clint Wylie aaefdb3386 fix group-by v2 BufferArrayGrouper for empty multi-value dimension row (#7794)
* fix groupby v2 BufferArrayGrouper

* better name test

* fix sql compatible null handling array grouper bug

* another test
2019-05-30 12:59:59 -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
BIGrey 42cf078843 Fix memory problem (OOM/FGC) when expression is used in metricsSpec (#7716)
* AggregatorUtil should cache parsed expression to avoid memory problem (OOM/FGC) when Expression is used in metricsSpec

* remove debug log check in Parser.parse

* remove cache and use suppliers.memorize
2019-05-27 09:46:17 -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
Clint Wylie ffc2397bcd fix AggregatorFactory.finalizeComputation implementations to be ok with null inputs (#7731)
* AggregatorFactory finalizeComputation is nullable with nullable input, make implementations honor this

* fixes
2019-05-22 21:13:09 -07:00
Himanshu fb0c846941
Virtual column updates for exploiting base column internal structure (#7618)
* VirtualColumn updates for exploiting base column internal structure

* unit tests for virtual column interface updates

* groupBy needs to use VirtualizedColumnSelectorFactory if outer query in
nested groupBy has virtual columns.

* fix strict compile checks

* fix teamcity build errors

* add comment explaining useVirtualizedColumnSelectorFactory flag in RowBasedGrouperHelper.createGrouperAccumulatorPair(..)
2019-05-20 17:04:35 -07:00
Clint Wylie c4a4223c9c fix issue where result level cache was recomputing post aggs that were already cached, causing issues with finalizing aggregators (#7708) 2019-05-20 16:51:50 -07:00
Himanshu 8687f424f9
make ComplexColumn an interface and ExtensionPoint (#7633)
* make ComplexColumn an interface and ExtensionPoint

* incorporate review comments

* make ColumnValueSelector @ExtensionPoint

* more java docs

* add close() method to ComplexColumn interface
2019-05-15 20:59:55 -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
Alexander Saydakov ca1a6649f6 Datasketches quantiles more post-aggs (#7550)
* rank and CDF post-aggs

* added post-aggs to the module

* added new post-aggs

* moved post-agg IDs

* moved post-agg IDs
2019-05-10 11:46:54 -07:00
Xavier Léauté 1d49364d08 Set direct memory if unable to detect JVM config (#7606)
* Set direct memory if unable to detect JVM config

Java 9 and above prevents us from detecting the maximum available direct
memory.

This change adds a fallback method to use at most 25% of maximum heap
size, which should be a reasonable default.

Unless -XX:MaxDirectMemorySize is set, recent JVMs will default maximum
direct memory to match the maximum heap size, so this should work out of
the box in most cases. For completeness we print instructions in the log
to explain how to adjust settings if necessary.

* skip test rather than succeeding

* reword log message

Co-Authored-By: Himanshu <g.himanshu@gmail.com>
2019-05-09 22:30:42 -07:00
Jihoon Son 18e0d6acb4 Fix resultLevelCache for timeseries with grandTotal (#7624)
* Fix resultLevelCache for timeseries with grandTotal

* Address comment

* fix test
2019-05-09 18:11:04 -07:00
Samarth Jain b542bb9f34 TDigest backed sketch aggregators (#7331)
* First set of changes for tDigest histogram

* Add license

* Address code review comments

* Add a doc page for new T-Digest sketch aggregators. Minor code cleanup and comments.

* Remove synchronization from BufferAggregators. Address code review comments

* Fix typo
2019-05-09 17:22:55 -07:00
Jonathan Wei 1b577c9b1d
Fix exception when using complex aggs with result level caching (#7614)
* Fix exception when using complex aggs with result level caching

* Add test comments

* checkstyle

* Add helper function for getting aggs from cache

* Move method to CacheStrategy

* Revert QueryToolChest changes

* Update test comments
2019-05-09 13:49:11 -07:00
Xavier Léauté f7bfe8f269
Update mocking libraries for Java 11 support (#7596)
* update easymock / powermock for to 4.0.2 / 2.0.2 for JDK11 support
* update tests to use new easymock interfaces
* fix tests failing due to easymock fixes
* remove dependency on jmockit
* fix race condition in ResourcePoolTest
2019-05-06 12:28:56 -07:00
Gian Merlino f776b94089 AggregatorFactory: Clarify methods that return other AggregatorFactories. (#7293) 2019-04-29 19:27:30 +02:00
Xavier Léauté 30fed78daf Java 9 compatible specialized class compilation (#7477)
* Java 9 compatible specialized class compilation

We currently use Unsafe.defineClass to compile specialized classes,
which has been removed in Java 9 and above. This change switches to
MethodHandles.Lookup.defineClass at runtime, which provides similar
functionality in newer JDK versions.

* add comments

* fix incorrect comment

* add unsafe utility class

* make comments java-doc style

* fix checkstyle errors

* rename unsafe -> unsafeutil

* move defineClass method to utility class

* rename unsafeutil -> unsafeutils to match other utility class names
* remove extra lookup method

* add utiliy class docs

* more comments

* minor comments and formatting
2019-04-29 18:44:28 +02:00
Justin Borromeo 07dd742e35 Fix time-ordered scan queries on realtime segments (#7546)
* Initial commit

* Added test for int to long conversion

* Add appenderator test for realtime scan query

* get rid of todo

* Fix forbidden apis

* Jon's recommendations

* Formatting
2019-04-26 16:12:10 -07:00
Roman Leventov 6fd6e5de89 Make JavaScript and XML errors non-TeamCity errors; Update JavaScript language level to ES6 in IntelliJ settings (#7541)
* Make JavaScript and XML errors non-TeamCity errors; Update JavaScript language level to ES6 in IntelliJ settings

* Add license comment to assembly-2.0.0.xsd

* Add .idea/README.md with comments
2019-04-25 11:21:58 -07:00
Qi Chen b59b9ef8c7 Fix too many dentry cache slab objs#7508. (#7509) 2019-04-19 20:39:50 -07:00
Surekha c2a42e05bb Fix result-level cache for queries (#7325)
* Add SegmentDescriptor interval in the hash while calculating Etag

* Add computeResultLevelCacheKey to CacheStrategy

Make HavingSpec cacheable and implement getCacheKey for subclasses
Add unit tests for computeResultLevelCacheKey

* Add more tests

* Use CacheKeyBuilder for HavingSpec's getCacheKey

* Initialize aggregators map to avoid NPE

* adjust cachekey builder for HavingSpec to ignore aggregators

* unused import

* PR comments
2019-04-18 13:31:29 -07:00
Justin Borromeo 85f10ed0d0 Support querying realtime segments using time-ordered scan queries and fix broken scan queries without time column (#7454)
* Update scan query runner factory to accept SpecificSegmentSpec

*  nit

* Sorry travis

* Improve logging and fix doc

* Bug fix

* Friendlier error msgs and tests to cover bug

* Address Gian's comments

* Fix doc

* Added tests for empty and null column list

* Style

* Fix checking wrong order (looking at query param when it should be
looking at the null-handled order)

* Add test case for null order

* Fix ScanQueryRunnerTest

* Forbidden APIs fixed
2019-04-12 19:08:34 -07:00
Jonathan Wei 7d9cb6944b Adjust BufferAggregator.get() impls to return copies (#7464)
* Adjust BufferAggregator.get() impls to return copies

* Update BufferAggregator docs, more agg fixes

* Update BufferAggregator get() doc
2019-04-12 19:04:07 -07:00
Justin Borromeo 799c66d9ac Allow max rows and max segments for time-ordered scans to be overridden using the scan query JSON spec (#7413)
* Initial changes

* Fixed NPEs

* Fixed failing spec test

* Fixed failing Calcite test

* Move configs to context

* Validated and added docs

* fixed weird indentation

* Update default context vals in doc

* Fixed allowable values
2019-04-07 20:12:52 -07:00
Clint Wylie 76b4a5c62e refactor lookups to be more chill to router (#7222)
* refactor lookups to be more chill to router

* remove accidental change

* fix and combine LookupIntrospectionResourceTest

* fix inspection

* rename RouterLookupModule to LookupSerdeModule and RouterLookupExtractorFactoryContainerProvider to NoopLookupExtractorFactoryContainerProvider

* make comment generic

* use ConfigResourceFilter instead of StateResourceFilter

* fix indentation

* unused import

* another unused import

* refactor some stuff into processing module, split up LookupModule.java classes into their own files
2019-04-05 14:49:41 -07:00
Richard Startin d29a32062f upgrade to RoaringBitmap 0.8.0 and serialise directly to ByteBuffer (#7408) 2019-04-04 13:22:50 -04:00
Clint Wylie a99f0ff450 prefix no-op aggs with "Noop" (#6960) 2019-04-02 15:05:07 -07:00
Justin Borromeo ad7862c58a Time Ordering On Scans (#7133)
* Moved Scan Builder to Druids class and started on Scan Benchmark setup

* Need to form queries

* It runs.

* Stuff for time-ordered scan query

* Move ScanResultValue timestamp comparator to a separate class for testing

* Licensing stuff

* Change benchmark

* Remove todos

* Added TimestampComparator tests

* Change number of benchmark iterations

* Added time ordering to the scan benchmark

* Changed benchmark params

* More param changes

* Benchmark param change

* Made Jon's changes and removed TODOs

* Broke some long lines into two lines

* nit

* Decrease segment size for less memory usage

* Wrote tests for heapsort scan result values and fixed bug where iterator
wasn't returning elements in correct order

* Wrote more tests for scan result value sort

* Committing a param change to kick teamcity

* Fixed codestyle and forbidden API errors

* .

* Improved conciseness

* nit

* Created an error message for when someone tries to time order a result
set > threshold limit

* Set to spaces over tabs

* Fixing tests WIP

* Fixed failing calcite tests

* Kicking travis with change to benchmark param

* added all query types to scan benchmark

* Fixed benchmark queries

* Renamed sort function

* Added javadoc on ScanResultValueTimestampComparator

* Unused import

* Added more javadoc

* improved doc

* Removed unused import to satisfy PMD check

* Small changes

* Changes based on Gian's comments

* Fixed failing test due to null resultFormat

* Added config and get # of segments

* Set up time ordering strategy decision tree

* Refactor and pQueue works

* Cleanup

* Ordering is correct on n-way merge -> still need to batch events into
ScanResultValues

* WIP

* Sequence stuff is so dirty :(

* Fixed bug introduced by replacing deque with list

* Wrote docs

* Multi-historical setup works

* WIP

* Change so batching only occurs on broker for time-ordered scans

Restricted batching to broker for time-ordered queries and adjusted
tests

Formatting

Cleanup

* Fixed mistakes in merge

* Fixed failing tests

* Reset config

* Wrote tests and added Javadoc

* Nit-change on javadoc

* Checkstyle fix

* Improved test and appeased TeamCity

* Sorry, checkstyle

* Applied Jon's recommended changes

* Checkstyle fix

* Optimization

* Fixed tests

* Updated error message

* Added error message for UOE

* Renaming

* Finish rename

* Smarter limiting for pQueue method

* Optimized n-way merge strategy

* Rename segment limit -> segment partitions limit

* Added a bit of docs

* More comments

* Fix checkstyle and test

* Nit comment

* Fixed failing tests -> allow usage of all types of segment spec

* Fixed failing tests -> allow usage of all types of segment spec

* Revert "Fixed failing tests -> allow usage of all types of segment spec"

This reverts commit ec470288c7.

* Revert "Merge branch '6088-Time-Ordering-On-Scans-N-Way-Merge' of github.com:justinborromeo/incubator-druid into 6088-Time-Ordering-On-Scans-N-Way-Merge"

This reverts commit 57033f36df, reversing
changes made to 8f01d8dd16.

* Check type of segment spec before using for time ordering

* Fix bug in numRowsScanned

* Fix bug messing up count of rows

* Fix docs and flipped boolean in ScanQueryLimitRowIterator

* Refactor n-way merge

* Added test for n-way merge

* Refixed regression

* Checkstyle and doc update

* Modified sequence limit to accept longs and added test for long limits

* doc fix

* Implemented Clint's recommendations
2019-03-28 14:37:09 -07:00
Justin Borromeo c7fea6ac8f Added better QueryInterruptedException error message for UnsupportedOperationException (#7248)
* Added error message for UOE

* Updated docs

* Doc change

* Doc change
2019-03-26 15:20:24 -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 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
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
Furkan KAMACI 48bc523bdf Locale problem is fixed which fails tests. (#7120)
* Locale problem is fixed which fails tests.

* Forbidden apis definition is improved to prevent using com.ibm.icu.text.SimpleDateFormat and com.ibm.icu.text.DateFormatSymbols without using any Locale defined.

* Error message is improved.
2019-03-13 18:47:14 -03:00
Gian Merlino 98a1b5537f Fix time-extraction topN with non-STRING outputType. (#7257)
Similar to other bugs fixed in #6220, but this one was missed. This bug would
cause "extraction" dimensionSpecs on the "__time" column with non-STRING
outputTypes to potentially be output as STRING sometimes instead of LONG,
causing incompletely merged results.
2019-03-13 13:53:07 -07:00
Gian Merlino 4290e5ae7a Cache selectors in QueryableIndexColumnSelectorFactory. (#7216)
For selectors with internal caches (like SingleScanTimeDimensionSelector,
SingleLongInputCachingExpressionColumnValueSelector, etc) we can get a perf
boost and memory usage decrease by sharing selectors.
2019-03-11 11:33:01 -07:00
Jihoon Son 9bebf113ba
Fix race in historical when loading segments in parallel (#7203)
* Fix race in historical when loading segments in parallel

* revert unnecessary change

* remove synchronized

* add reference counting locking

* fix build

* fix comment
2019-03-08 17:54:05 -08:00
Clint Wylie 9fa649b3bd segment metadata fallback analysis if no bitmaps (#7116)
* segment metadata fallback analysis if no bitmaps

* remove accidental line

* remove nonsense size estimation

* less ternary

* fix it

* do the thing
2019-02-26 11:27:41 -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
Justin Borromeo c7eeeabf45 2528 Replace Incremental Index Global Flags with Getters (#7043)
* Eliminated reportParseExceptions and deserializeComplexMetrics

* Removed more global flags

* Cleanup

* Addressed Surekha's recommendations
2019-02-15 13:36:46 -08:00
Jonathan Wei 1f29940811
Fix momentsketch build issues (#7074)
* Fix momentsketch build issues

* Remove unused section in pom

* Fix test

* Remove unused method

* Checkstyle
2019-02-13 21:32:43 -08:00
Edward Gan 90c1a54b86 Moments Sketch custom aggregator (#6581)
* Moments Sketch Integration with Druid

* updates, add documentation, fix warnings

* nits

* disallowed base64

* update to druid 0.14
2019-02-13 14:03:47 -08:00
Jihoon Son c9f21bc782 Fix filterSegments for TimeBoundary and DataSourceMetadata queries (#7023)
* Fix filterSegments for TimeBoundary and DataSourceMetadata queries

* add javadoc

* fix build
2019-02-08 10:03:02 -08:00
Justin Borromeo 6723243ed2 Create Scan Benchmark (#6986)
* Moved Scan Builder to Druids class and started on Scan Benchmark setup

* Need to form queries

* It runs.

* Remove todos

* Change number of benchmark iterations

* Changed benchmark params

* More param changes

* Made Jon's changes and removed TODOs

* Broke some long lines into two lines

* Decrease segment size for less memory usage

* Committing a param change to kick teamcity
2019-02-06 14:45:01 -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
Roman Leventov f7df5fedcc Add several missing inspectRuntimeShape() calls (#6893)
* Add several missing inspectRuntimeShape() calls

* Add lgK to runtime shapes
2019-01-31 20:04:26 -08:00
Furkan KAMACI 30ec608038 Fix mixed up segment ids at SelectBinaryFnTest.java (#6946) 2019-01-30 20:04:16 -08:00
Clint Wylie de810286cd fix bug with expression virtual column selectors backed by a single long column (#6957)
* fix issue with SingleLongInputCachingExpressionColumnValueSelector when sql compatible null handling enabled

* add test with doubles to show same behavior for floats/doubles that lack the optimization of longs

* simplify

* fix import
2019-01-30 10:13:07 -05:00
Clint Wylie a6d81c0d16 Adds bloom filter aggregator to 'druid-bloom-filters' extension (#6397)
* blooming aggs

* partially address review

* fix docs

* minor test refactor after rebase

* use copied bloomkfilter

* add ByteBuffer methods to BloomKFilter to allow agg to use in place, simplify some things, more tests

* add methods to BloomKFilter to get number of set bits, use in comparator, fixes

* more docs

* fix

* fix style

* simplify bloomfilter bytebuffer merge, change methods to allow passing buffer offsets

* oof, more fixes

* more sane docs example

* fix it

* do the right thing in the right place

* formatting

* fix

* avoid conflict

* typo fixes, faster comparator, docs for comparator behavior

* unused imports

* use buffer comparator instead of deserializing

* striped readwrite lock for buffer agg, null handling comparator, other review changes

* style fixes

* style

* remove sync for now

* oops

* consistency

* inspect runtime shape of selector instead of selector plus, static comparator, add inner exception on serde exception

* CardinalityBufferAggregator inspect selectors instead of selectorPluses

* fix style

* refactor away from using ColumnSelectorPlus and ColumnSelectorStrategyFactory to instead use specialized aggregators for each supported column type, other review comments

* adjustment

* fix teamcity error?

* rename nil aggs to empty, change empty agg constructor signature, add comments

* use stringutils base64 stuff to be chill with master

* add aggregate combiner, comment
2019-01-29 20:05:17 +07: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
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
Clint Wylie 66f64cd8bd fix long/float/double dimension filtering for columns with nulls (#6906)
* fix long,float, double dimension filtering when sql compatible null handling is enabled and the column has null values

* revert unintended change

* fix tests
2019-01-23 22:36:52 -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
Jihoon Son cc06e7e2df Fix fallback to cursor-based plan in UseIndexesStrategy (#6875)
* Fix fallback to cursor-based plan in UseIndexesStrategy

* fix build

* add a comment
2019-01-18 10:41:01 +08:00
Jonathan Wei 68f744ec0a
Fixed buckets histogram aggregator (#6638)
* Fixed buckets histogram aggregator

* PR comments

* More PR comments

* Checkstyle

* TeamCity

* More TeamCity

* PR comment

* PR comment

* Fix doc formatting
2019-01-17 14:51:16 -08:00
Dayue Gao 5b8a221713 Add SQL id, request logs, and metrics (#6302)
* use SqlLifecyle to manage sql execution, add sqlId

* add sql request logger

* fix UT

* rename sqlId to sqlQueryId, sql/time to sqlQuery/time, etc

* add docs and more sql request logger impls

* add UT for http and jdbc

* fix forbidden use of com.google.common.base.Charsets

* fix UT in QuantileSqlAggregatorTest, supressed unused warning of getSqlQueryId

* do not use default method in QueryMetrics interface

* capitalize 'sql' everywhere in the non-property parts of the docs

* use RequestLogger interface to log sql query

* minor bugfixes and add switching request logger

* add filePattern configs for FileRequestLogger

* address review comments, adjust sql request log format

* fix inspection error

* try SuppressWarnings("RedundantThrows") to fix inspection error on ComposingRequestLoggerProvider
2019-01-15 23:12:59 -08:00