Commit Graph

10065 Commits

Author SHA1 Message Date
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
Roman Leventov b9186f8f9f Reconcile terminology and method naming to 'used/unused segments'; Rename MetadataSegmentManager to MetadataSegmentsManager (#7306)
* Reconcile terminology and method naming to 'used/unused segments'; Don't use terms 'enable/disable data source'; Rename MetadataSegmentManager to MetadataSegments; Make REST API methods which mark segments as used/unused to return server error instead of an empty response in case of error

* Fix brace

* Import order

* Rename withKillDataSourceWhitelist to withSpecificDataSourcesToKill

* Fix tests

* Fix tests by adding proper methods without interval parameters to IndexerMetadataStorageCoordinator instead of hacking with Intervals.ETERNITY

* More aligned names of DruidCoordinatorHelpers, rename several CoordinatorDynamicConfig parameters

* Rename ClientCompactTaskQuery to ClientCompactionTaskQuery for consistency with CompactionTask; ClientCompactQueryTuningConfig to ClientCompactionTaskQueryTuningConfig

* More variable and method renames

* Rename MetadataSegments to SegmentsMetadata

* Javadoc update

* Simplify SegmentsMetadata.getUnusedSegmentIntervals(), more javadocs

* Update Javadoc of VersionedIntervalTimeline.iterateAllObjects()

* Reorder imports

* Rename SegmentsMetadata.tryMark... methods to mark... and make them to return boolean and the numbers of segments changed and relay exceptions to callers

* Complete merge

* Add CollectionUtils.newTreeSet(); Refactor DruidCoordinatorRuntimeParams creation in tests

* Remove MetadataSegmentManager

* Rename millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments to leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments

* Fix tests, refactor DruidCluster creation in tests into DruidClusterBuilder

* Fix inspections

* Fix SQLMetadataSegmentManagerEmptyTest and rename it to SqlSegmentsMetadataEmptyTest

* Rename SegmentsAndMetadata to SegmentsAndCommitMetadata to reduce the similarity with SegmentsMetadata; Rename some methods

* Rename DruidCoordinatorHelper to CoordinatorDuty, refactor DruidCoordinator

* Unused import

* Optimize imports

* Rename IndexerSQLMetadataStorageCoordinator.getDataSourceMetadata() to retrieveDataSourceMetadata()

* Unused import

* Update terminology in datasource-view.tsx

* Fix label in datasource-view.spec.tsx.snap

* Fix lint errors in datasource-view.tsx

* Doc improvements

* Another attempt to please TSLint

* Another attempt to please TSLint

* Style fixes

* Fix IndexerSQLMetadataStorageCoordinator.createUsedSegmentsSqlQueryForIntervals() (wrong merge)

* Try to fix docs build issue

* Javadoc and spelling fixes

* Rename SegmentsMetadata to SegmentsMetadataManager, address other comments

* Address more comments
2020-01-27 11:24:29 -08:00
Clint Wylie c6c8b80644 fix build by updating kafka client to 2.2.2 for CVE-2019-12399 (#9259)
* fix build by updating kafka client to 2.2.2 for CVE-2019-12399

* one kafka version to rule them all

* notice
2020-01-27 11:07:02 -08:00
Yamada Koji 20eb201d00 Fix DRUID_CONFIG to DRUID_CONFIG_COMMON (#9193) 2020-01-27 02:52: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
Caroline1000 3daf0f8e12 Update ingestion-view.tsx (#9250)
Grammar and accuracy updates.
2020-01-24 01:21:55 -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
Zhenxiao Luo 479c09751c Add MostAvailableSizeStorageLocationSelectorStrategy (#8879)
* Add MostAvailableSize LocationSelectorStrategy

* Add doc for mostAvailableSize strategy

* Fix docs for mostAvailableSize
2020-01-23 13:42:03 -08:00
sthetland 83ddc8de1e Update data-formats.md (#9238)
* Update data-formats.md

Field error and light rewording of new Avro material (and working through the doc authoring process).

* Update data-formats.md

Make default statements consistent. Future change: s/=/is.
2020-01-22 15:00:53 -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
Jihoon Son d541cbe436 Support both IndexTuningConfig and ParallelIndexTuningConfig for compaction task (#9222)
* Support both IndexTuningConfig and ParallelIndexTuningConfig for compaction task

* tuningConfig module

* fix tests
2020-01-21 13:56:54 -08:00
Chi Cao Minh 0b0056b77f More tests for range partition parallel indexing (#9232)
Add more unit tests for range partition native batch parallel indexing.

Also, fix a bug where ParallelIndexPhaseRunner incorrectly thinks that
identical collected DimensionDistributionReports are not equal due to
not overriding equals() in DimensionDistributionReport.
2020-01-21 12:59:43 -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
Gian Merlino f511af1306 Fix DOCKER_HOST_IP handling for multihomed machines. (#9225)
By picking one. Otherwise, when a machine has multiple IP addresses, DOCKER_HOST_IP
would have a newline in the middle, causing havoc in configuration files.
2020-01-21 09:01:19 -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
Suneet Saldanha 180c622e0f Minor doc updates (#9217)
* update string first last aggs

* update kafka ingestion specs in docs

* remove unnecessary parser spec
2020-01-20 11:34:37 -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
Suneet Saldanha d64bed79f0 Update docs for extensions (#9218)
* Update docs for s3 and avro extensions

* More doc updates - google + cleanup
2020-01-19 12:55:45 -08:00
Suneet Saldanha df3c1075a8 Update docs for extensions (#9218)
* Update docs for s3 and avro extensions

* More doc updates - google + cleanup
2020-01-19 12:55:01 -08:00
Suneet Saldanha bade2c802b Update docs for extensions (#9218)
* Update docs for s3 and avro extensions

* More doc updates - google + cleanup
2020-01-19 12:53:21 -08:00
Suneet Saldanha f98b664bb0 Update docs for extensions (#9218)
* Update docs for s3 and avro extensions

* More doc updates - google + cleanup
2020-01-19 12:52:49 -08:00
Suneet Saldanha de231d3c80 Update docs for extensions (#9218)
* Update docs for s3 and avro extensions

* More doc updates - google + cleanup
2020-01-19 12:50:05 -08:00
Suneet Saldanha 93167188ea Update docs for extensions (#9218)
* Update docs for s3 and avro extensions

* More doc updates - google + cleanup
2020-01-19 12:49:33 -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
Jihoon Son 153495068b Doc update for the new input source and the new input format (#9171)
* Doc update for new input source and input format.

- The input source and input format are promoted in all docs under docs/ingestion
- All input sources including core extension ones are located in docs/ingestion/native-batch.md
- All input formats and parsers including core extension ones are localted in docs/ingestion/data-formats.md
- New behavior of the parallel task with different partitionsSpecs are documented in docs/ingestion/native-batch.md

* parquet

* add warning for range partitioning with sequential mode

* hdfs + s3, gs

* add fs impl for gs

* address comments

* address comments

* gcs
2020-01-17 15:52:05 -08:00
Jihoon Son 84ff0d2352
Fix TSV bugs (#9199)
* working

* - support multi-char delimiter for tsv
- respect "delimiter" property for tsv

* default value check for findColumnsFromHeader

* remove CSVParser to have a true and only CSVParser

* fix tests

* fix another test
2020-01-17 15:35:14 -08:00
singh 936b9bdfd0 add deets about the keyfile (#9209) 2020-01-17 11:24:49 -08:00
Fokko Driesprong 12b84cfb33
Bump Jackson to 2.10.2 (#9173) 2020-01-17 11:39:32 +01:00
Vadim Ogievetsky ab2672514b allow empty values to be set in the auto form (#9198) 2020-01-16 21:06:51 -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
Fokko Driesprong 486c0fd149 Bump Apache Parquet to 1.11.0 (#9129)
* Bump Parquet to 1.11.0

* Update licenses.yaml

* Add parquet-format-structures
2020-01-16 16:24:25 -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
Vadim Ogievetsky 09efd20b42
fix refresh button (#9195) 2020-01-16 10:13:47 -08:00
Suneet Saldanha 92ac22d060 Link javaOpts to middlemanager runtime.properties docs (#9101)
* Link javaOpts to middlemanager runtime.properties docs

* fix broken link

* reword config links
2020-01-15 21:22:49 -08:00
Suneet Saldanha 85a3d416b0 Tutorials use new ingestion spec where possible (#9155)
* Tutorials use new ingestion spec where possible

There are 2 main changes
  * Use task type index_parallel instead of index
  * Remove the use of parser + firehose in favor of inputFormat + inputSource

index_parallel is the preferred method starting in 0.17. Setting the job to
index_parallel with the default maxNumConcurrentSubTasks(1) is the equivalent
of an index task

Instead of using a parserSpec, dimensionSpec and timestampSpec have been
promoted to the dataSchema. The format is described in the ioConfig as the
inputFormat.

There are a few cases where the new format is not supported
 * Hadoop must use firehoses instead of the inputSource and inputFormat
 * There is no equivalent of a combining firehose as an inputSource
 * A Combining firehose does not support index_parallel

* fix typo
2020-01-15 14:08:29 -08:00
Lucas Capistrant 4716e0b585 Fix concurrency of ComplexMetrics.java (#9134) 2020-01-15 17:19:45 +03:00
Chi Cao Minh b2877119d0 Suppress CVE-2019-20330 for htrace-core-4.0.1 (#9189)
CVE-2019-20330 was updated on 14 Jan 2020, which now gets flagged by the
security vulnerability scan. Since the CVE is for jackson-databind, via
htrace-core-4.0.1, it can be added to the existing list of security
vulnerability suppressions for that dependency.
2020-01-14 21:15:24 -08:00
Chi Cao Minh 1fd05bef9a Add jackson-mapper-asl for hdfs-storage extension (#9178)
Previously jackson-mapper-asl was excluded to remove a security
vulnerability; however, it is required for functionality (e.g.,
org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator).
2020-01-14 09:50:45 -08:00
Atul Mohan ea51bc45bf Fix nullhandling in tests (#9119) 2020-01-12 20:19:12 -08:00
Atul Mohan b642b1aa5b Fix deserialization of maxBytesInMemory (#9092)
* Fix deserialization of maxBytesInMemory

* Add maxBytes check
2020-01-12 20:08:07 -08: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 f245292e5d add middle manager and indexer worker category to tier column of services view (#9158) 2020-01-09 12:20:42 -08:00
Jihoon Son e27a1e8604
Fix handling nullable writableComparable in OrcStructConverter (#9138)
* Handle nullable writableComparable in OrcStructConverter

* add missing dependency
2020-01-08 13:40:24 -08:00