Commit Graph

9307 Commits

Author SHA1 Message Date
Qi Shu ed2beb6ba5 Add is_overshadowed column to segment table (#7555) 2019-05-01 12:28:53 -07:00
Surekha 15d19f3059 Add is_overshadowed column to sys.segments table (#7425)
* Add is_overshadowed column to sys.segments table

* update docs

* Rename class and variables

* PR comments

* PR comments

* remove unused variables in MetadataResource

* move constants together

* add getFullyOvershadowedSegments method to ImmutableDruidDataSource

* Fix compareTo of SegmentWithOvershadowedStatus

* PR comment

* PR comments

* PR comments

* PR comments

* PR comments

* fix issue with already consumed stream

* minor refactoring

* PR comments
2019-05-01 18:00:57 +02:00
Vadim Ogievetsky d97c0d19a0 add url baser (#7585) 2019-05-01 01:53:03 -07:00
Xavier Léauté 6d4181191f replace jdk internal exceptions with closest publicly available one 2019-04-30 14:21:45 -07:00
Xavier Léauté c2263a339e Remove direct references to unsafe for Java 9+ 2019-04-30 14:21:45 -07:00
Gian Merlino f776b94089 AggregatorFactory: Clarify methods that return other AggregatorFactories. (#7293) 2019-04-29 19:27:30 +02:00
Gian Merlino 7b8bc9a5ef EmitterModule: Throw an error on invalid emitter types. (#7328)
* EmitterModule: Throw an error on invalid emitter types.

The current behavior of silently using the "noop" emitter is unhelpful
and makes it difficult to debug config typos.

* Add comments.
2019-04-29 19:23:53 +02:00
Qi Shu 20755f4ca0 Add supervisor card in home view (#7551)
* Add supervisor card in home view

* hide the text under conditions
2019-04-29 10:15:10 -07:00
Gian Merlino ce7298b51e BaseAppenderatorDriver: Fix potentially overeager segment cleanup. (#7558)
* BaseAppenderatorDriver: Fix potentially overeager segment cleanup.

Here is a thing that I think can go wrong:

1. We push some segments, then try to publish them transactionally.
2. The segments are actually published, but the 200 OK response gets
   lost (connection dropped, whatever).
3. We try again, and on the second try, the publish fails (because
   the transaction baseline start metadata no longer matches).
4. Because the publish failed, we delete the pushed segments.
5. But this is bad, because the publish didn't really fail, it actually
   succeeded in step 2.

I haven't seen this in the wild, but thought about it while
reviewing #7537.

This patch also cleans up logging a bit, making it more accurate and
somewhat less chatty.

* Avoid wrapping exceptions when not necessary.
2019-04-29 09:55:04 -07: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
Gian Merlino c648775b5b SQL: Remove "useFallback" feature. (#7567)
This feature allows Calcite's Bindable interpreter to be bolted on
top of Druid queries and table scans. I think it should be removed for
a few reasons:

1. It is not recommended for production anyway, because it generates
unscalable query plans (e.g. it will plan a join into two table scans
and then try to do the entire join in memory on the broker).
2. It doesn't work with Druid-specific SQL functions, like TIME_FLOOR,
REGEXP_EXTRACT, APPROX_COUNT_DISTINCT, etc.
3. It makes the SQL planning code needlessly complicated.

With SQL coming out of experimental status soon, it's a good opportunity
to remove this feature.
2019-04-28 18:26:44 -07:00
Eyal Yurman f02251ab2d Contributing Moving-Average Query to open source. (#6430)
* Contributing Moving-Average Query to open source.

* Fix failing code inspections.

* See if explicit types will invoke the correct comparison function.

* Explicitly remove support for druid.generic.useDefaultValueForNull configuration parameter.

* Update styling and headers for complience.

* Refresh code with latest master changes:

* Remove NullDimensionSelector.
* Apply changes of RequestLogger.
* Apply changes of TimelineServerView.

* Small checkstyle fix.

* Checkstyle fixes.

* Fixing rat errors; Teamcity errors.

* Removing support theta sketches. Will be added back in this pr or a following once DI conflicts with datasketches are resolved.

* Implements some of the review fixes.

* Contributing Moving-Average Query to open source.

* Fix failing code inspections.

* See if explicit types will invoke the correct comparison function.

* Explicitly remove support for druid.generic.useDefaultValueForNull configuration parameter.

* Update styling and headers for complience.

* Refresh code with latest master changes:

* Remove NullDimensionSelector.
* Apply changes of RequestLogger.
* Apply changes of TimelineServerView.

* Small checkstyle fix.

* Checkstyle fixes.

* Fixing rat errors; Teamcity errors.

* Removing support theta sketches. Will be added back in this pr or a following once DI conflicts with datasketches are resolved.

* Implements some of the review fixes.

* More fixes for review.

* More fixes from review.

* MapBasedRow is Unmodifiable. Create new rows instead of modifying existing ones.

* Remove more changes related to datasketches support.

* Refactor BaseAverager startFrom field and add a comment.

* fakeEvents field: Refactor initialization and add comment.

* Rename parameters (tiny change).

* Fix variable name typo in test (JAN_4).

* Fix styling of non camelCase fields.

* Fix Preconditions.checkArgument for cycleSize.

* Add more documentation to RowBucketIterable and other classes.

* key/value comment on in MovingAverageIterable.

* Fix anonymous makeColumnValueSelector returning null.

* Replace IdentityYieldingAccumolator with Yielders.each().

* * internalNext() should return null instead of throwing exception.
* Remove unused variables/prarameters.

* Harden MovingAverageIterableTest (Switch anyOf to exact match).

* Change internalNext() from recursion to iteration; Simplify next() and hasNext().

* Remove unused imports.

* Address review comments.

* Rename fakeEvents to emptyEvents.

* Remove redundant parameter key from computeMovingAverage.

* Check yielder as well in RowBucketIterable#hasNext()

* Fix javadoc.
2019-04-26 17:07:48 -07: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
Adam Peck ebdf07b69f Add reload by interval API (#7490)
* Add reload by interval API
Implements the reload proposal of #7439
Added tests and updated docs

* PR updates

* Only build timeline with required segments
Use 404 with message when a segmentId is not found
Fix typo in doc
Return number of segments modified.

* Fix checkstyle errors

* Replace String.format with StringUtils.format

* Remove return value

* Expand timeline to segments that overlap for intervals
Restrict update call to only segments that need updating.

* Only add overlapping enabled segments to the timeline

* Some renames for clarity
Added comments

* Don't rely on cached poll data
Only fetch required information from DB

* Match error style

* Merge and cleanup doc

* Fix String.format call

* Add unit tests

* Fix unit tests that check for overshadowing
2019-04-26 16:01:50 -07:00
Clint Wylie 09b7700d13 fix docs (#7556) 2019-04-25 22:00:37 -07:00
Clint Wylie 2a65431b08 fix LICENSE.BINARY to have correct datasketches version (#7557) 2019-04-25 22:00:13 -07:00
Justin Borromeo 012ab02bf4 Update select doc disclaimer (#7554) 2019-04-25 19:23:39 -07:00
Alexander Saydakov 9d8f934e68 handle empty sketches (#7526)
* handle empty sketches

* return array of NaN in case of empty sketch

* noinspection ForLoopReplaceableByForEach in tests

* style fixes
2019-04-25 14:28:41 -07:00
Surekha 8308ffef1f API to drop data by interval (#7494)
* Add api to drop data by interval

* update to address comments

* unused imports

* PR comments + add tests in SQLMetadataSegmentManagerTest

*  update tests and docs
2019-04-25 14:24:40 -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 Shu 3ec9fbaa47 Fix history dialog styling (#7536)
* Fix history dialog

* more styling
2019-04-25 07:59:34 -07:00
Jonathan Wei 658fb2b062 Fix bugs in milestone contributor script (#7545)
* Only check PRs in milestone contributor script

* Fix no-pagination bug
2019-04-24 22:11:57 -07:00
es1220 3e25b75c3f Fix aggregatorFactory meta merge exception (#7504) 2019-04-24 14:08:46 -07:00
Qi Shu 11a7e91a73 No SQL mode in web console (#7493)
* Added no sql mode

* Use status code

* Add no sql mode to server view

* add sql broker check to decide if no sql mode should be enabled

* Fix historicals in home view

* Name change

* Add types for query result; improved functions

* Fixed a conflict/bug

* Fixed a bug

* multiple fix

* removed unused imports

* terminate query manager

* fix wording
2019-04-23 16:15:02 -07:00
Jonathan Wei 8b1a4e18dd Additional Apache branding doc updates (#7524) 2019-04-23 14:39:16 -07:00
Xue Yu 2c8a71f883 Support LPAD and RPAD sql function (#7388)
* lpad and rpad sql function

* feedback address

* feedback address

* add doc and format

* update docs
2019-04-22 14:51:32 -07:00
Jihoon Son c60e7feab8 Fix encoded taskId check in chatHandlerResource (#7520)
* Fix encoded taskId check in chatHandlerResource

* fix tests
2019-04-20 18:08:34 -07:00
Qi Chen b59b9ef8c7 Fix too many dentry cache slab objs#7508. (#7509) 2019-04-19 20:39:50 -07:00
Jonathan Wei 3487663de9 Adjust approx agg deprecation wording (#7518) 2019-04-19 19:31:50 -07:00
Jonathan Wei 5939fc28ea Fix some bundled license files (#7516) 2019-04-19 16:07:44 -07:00
Jonathan Wei 74960e82bf Add more Apache branding to docs (#7515) 2019-04-19 15:52:26 -07:00
Qi Shu 9929f8b022 Fixed filter for status in task table (#7507)
* Fixed filter for Task view status

* Refactored code

* Fixed a bug for SQL filter by not converting input to lower case since the comparison is done through SQL
2019-04-19 15:51:45 -07:00
Gian Merlino 1fb5ec3989 Adds backwards-compatible serde for SeekableStreamStartSequenceNumbers. (#7512)
This allows them to be deserialized by older Druid versions as
KafkaPartitions objects.

Fixes #7470.
2019-04-19 13:19:45 -07:00
Slim Bouguerra 5463ecb979 Fix broken link due to Typo. (#7513)
Change-Id: I5792f89ed6afe945f386058edd44f0400998460a
2019-04-19 09:58:54 -07:00
Roman Leventov 0a0fd63f4e
Introduce Non-TeamCity Warning inspection severity in IntelliJ's profile (#7499)
In the IDE interface, "Non-TeamCity Warning" looks exactly like an ordinary warning, but TeamCity should be unaware of it.

This may help to workaround these issues: https://youtrack.jetbrains.com/issue/IDEA-209789 and https://youtrack.jetbrains.com/issue/IDEA-209791, that block the upgrade of IntelliJ engine used in the TeamCity build. It seems like there may be a bug that leads to false positive error and the build fail in this PR: https://github.com/apache/incubator-druid/pull/6702.

Removed the comment regarding "StaticPseudoFunctionalStyleMethod" inspection because the IntelliJ keeps removing it, see this issue: https://youtrack.jetbrains.com/issue/IDEA-211087
2019-04-19 15:20:08 +02: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
Clint Wylie be65cca248 refactor druid-bloom-filter aggregators (#7496)
* now with 100% more buffer

* there can be only 1

* simplify

* javadoc

* clean up unused test method

* fix exception message

* style

* why does style hate javadocs

* review stuff

* style :(
2019-04-18 11:54:06 -07:00
Qi Shu 99ddce1277 Capitalize keywords in SQL auto complete (#7462)
* Split commit; keep auto complete only

* Add comment
2019-04-17 23:12:55 -07:00
Qi Shu f06f0c3ad8 Add created_time as tie breaker when sorting by status in task table (#7460)
* Add created_time as tie breaker when sorting by status

* Fixed bug

* Fixed created_time tie breaker

* Use locale compare
2019-04-17 23:11:45 -07:00
Qi Shu c8bffd9351 Add exact match filtering to console table (#7448)
* Add exact filtering

* Bug fix

* Extract filter as a function

* change code position
2019-04-17 23:05:44 -07:00
Samarth Jain 9732e04c60 Pass in segmentTable correctly (#7492) 2019-04-17 20:07:22 -07:00
Jonathan Wei 8078f567aa Update kafka version in tutorials (#7500) 2019-04-17 14:56:29 -07:00
Xavier Léauté 4322ce3303 Java 9 compatible cleaner operations (#7487)
Java 9 removed support for sun.misc.Cleaner in favor of
java.lang.ref.Cleaner. This change adds a thin abstraction to switch
between Cleaner implementations based on JDK version at runtime
2019-04-17 08:04:52 -07:00
Roman Leventov 9d385274e3
Move dev-related files and instructions to dev/ directory; add committer's instructions (#7279)
* Add committer_readme.md

* Clean up

* Update, add PR merge action checklist

* Move dev-related docs and files except CONTRIBUTING.md to dev/ directory; More committer's intructions

* Add some accents

* Move TeamCity instruction images to teamcity-images/, edit CONTRIBUTING.md

* Add links to tags
2019-04-17 15:27:14 +02:00
Kazuhito Takeuchi 7c19c92a81 Add ROUND function in druid-sql. (#7224)
* Implement round function in druid-sql

* Return value according to the type of argument

* Fix codes for abnoraml inputs, updated math-expr.md

* Fix assert text

* Fix error messages and refactor codes

* Fix compile error, update sql.md, refactor codes and format tests
2019-04-16 11:15:39 -07:00
Clint Wylie 7385dbc9e8 fix Lifecycle javadoc typo (#7489) 2019-04-16 10:23:01 -07:00
Jihoon Son 625c030744 Fix ParallelIndexTuningConfig constructor (#7479)
* Fix ParallelIndexTuningConfig constructor

* unused import
2019-04-15 17:41:53 -07:00
Lucas Capistrant 8acad27d99 Enhance the Http Firehose to work with URIs requiring basic authentication (#7145)
* Enhnace the HttpFirehose to work with both insecure URIs and URIs requiring basic authentication

* Improve security of enhanced HttpFirehoseFactory by not logging auth credentials

* Fix checkstyle failure in HttpFirehoseFactory.java

* Update docs and fix TeamCity build with required noinspection

* Indentation cleanup and logic modification for HttpFirehose object stream

* Remove default Empty string password provider in http firehose

* Add JavaDoc for MixIn describing its intended use

* Reverting documentation notation for json code to be inline with rest of doc

* Improve instantiation of ObjectMappers that require MixIn for redacting password from task logs

* Add comment to clarify fully qualified references of Objects in SQLMetadataStorageActionHandler
2019-04-15 14:29:01 -07:00
Surekha 4654e1e851 Remove unnecessary collection (#7350)
From the discussion [here](https://github.com/apache/incubator-druid/pull/6901#discussion_r265741002)

Remove the collection and filter datasources from the stream. 
Also remove StreamingOutput and JsonFactory constructs.
2019-04-15 19:49:21 +02:00
Xavier Léauté 530a378fea
Java 9 compatible ByteBuffer unmap operation (#7466)
The current bytebuffer unmap operations are not compatible with Java 9
and above. This change adapts the code from Apache Kafka to perform the
appropriate unmap operation based on JDK version at runtime.

see e554dc518e
2019-04-15 09:03:18 -07:00