Commit Graph

1696 Commits

Author SHA1 Message Date
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
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
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
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
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 f540216931 fix InputFormat serde issue with SeekableStream based supervisors (#9136) 2020-01-07 16:18:54 -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
Jonathan Wei 4e8368a5d9 Set version to 0.18.0-SNAPSHOT (#9109) 2020-01-02 17:55:10 -05:00
Suneet Saldanha dec619ebf4 Optimize CachingLocalSegmentAllocator#getSequenceName (#8909)
* Optimize CachingLocalSegmentAllocator#getSequenceName

Replace StringUtils#format with string addition to generate the sequence
name for an interval and partition. This is faster because format uses a
Matcher under the covers to replace the string format with the variables.

* fix imports and add test

* Add comment about optimization

* Use renamed function for TaskToolbox

* Move tests after refactor

* Rename tests
2019-12-23 18:33:22 -08:00
Chi Cao Minh 6178f05da6 Fail superbatch range partition multi dim values (#9058)
* Fail superbatch range partition multi dim values

Change the behavior of parallel indexing range partitioning to fail
ingestion if any row had multiple values for the partition dimension.
After this change, the behavior matches that of hadoop indexing.
(Previously, rows with multiple dimension values would be skipped.)

* Improve err msg, rename method, rename test class
2019-12-18 10:14:03 -08:00
Himanshu 45101183bc
HRTR: make pending task execution handling to go through all tasks on not finding worker slots (#8697)
* HRTR: make pending task execution handling to go through all tasks on
not finding worker slots

* make HRTR methods package private that are meant to be used only in HttpRemoteTaskRunnerResource

* mark HttpRemoteTaskRunnerWorkItem.State global variables final

* hrtr: move immutableWorker NULL check outside of try-catch or finally block could have NPE

* add some explanatory comments

* add comment on explaining mechanics around hand off of pending tasks from submission to it getting picked up by a task execution thread

* fix spelling
2019-12-12 14:58:52 -08:00
Jonathan Wei 8af41d7cd0 Update version to 0.18.0-incubating-SNAPSHOT (#9009) 2019-12-11 14:04:03 -08:00
Chi Cao Minh 3de7ab8523 DataSketches jars in core (#9003)
Having DataSketches jars in core will allow potential improvements, for
example:
- Provide an alternative implementation of HLL:
  https://datasketches.github.io/docs/HLL/HllSketchVsDruidHyperLogLogCollector.html
- Range partitioning for native parallel batch indexing without having
  the user load extensions on the classpath

Dev mailing list discussion:
https://lists.apache.org/thread.html/301410d71ff799cf616bf17c4ebcf9999fc30829f5fa62909f403e6c%40%3Cdev.druid.apache.org%3E
2019-12-10 14:02:34 -08:00
Chi Cao Minh bab78fc80e Parallel indexing single dim partitions (#8925)
* Parallel indexing single dim partitions

Implements single dimension range partitioning for native parallel batch
indexing as described in #8769. This initial version requires the
druid-datasketches extension to be loaded.

The algorithm has 5 phases that are orchestrated by the supervisor in
`ParallelIndexSupervisorTask#runRangePartitionMultiPhaseParallel()`.
These phases and the main classes involved are described below:

1) In parallel, determine the distribution of dimension values for each
   input source split.

   `PartialDimensionDistributionTask` uses `StringSketch` to generate
   the approximate distribution of dimension values for each input
   source split. If the rows are ungrouped,
   `PartialDimensionDistributionTask.UngroupedRowDimensionValueFilter`
   uses a Bloom filter to skip rows that would be grouped. The final
   distribution is sent back to the supervisor via
   `DimensionDistributionReport`.

2) The range partitions are determined.

   In `ParallelIndexSupervisorTask#determineAllRangePartitions()`, the
   supervisor uses `StringSketchMerger` to merge the individual
   `StringSketch`es created in the preceding phase. The merged sketch is
   then used to create the range partitions.

3) In parallel, generate partial range-partitioned segments.

   `PartialRangeSegmentGenerateTask` uses the range partitions
   determined in the preceding phase and
   `RangePartitionCachingLocalSegmentAllocator` to generate
   `SingleDimensionShardSpec`s.  The partition information is sent back
   to the supervisor via `GeneratedGenericPartitionsReport`.

4) The partial range segments are grouped.

   In `ParallelIndexSupervisorTask#groupGenericPartitionLocationsPerPartition()`,
   the supervisor creates the `PartialGenericSegmentMergeIOConfig`s
   necessary for the next phase.

5) In parallel, merge partial range-partitioned segments.

   `PartialGenericSegmentMergeTask` uses `GenericPartitionLocation` to
   retrieve the partial range-partitioned segments generated earlier and
   then merges and publishes them.

* Fix dependencies & forbidden apis

* Fixes for integration test

* Address review comments

* Fix docs, strict compile, sketch check, rollup check

* Fix first shard spec, partition serde, single subtask

* Fix first partition check in test

* Misc rewording/refactoring to address code review

* Fix doc link

* Split batch index integration test

* Do not run parallel-batch-index twice

* Adjust last partition

* Split ITParallelIndexTest to reduce runtime

* Rename test class

* Allow null values in range partitions

* Indicate which phase failed

* Improve asserts in tests
2019-12-09 23:05:49 -08:00
Roman Leventov 1c62987783
Add SelfDiscoveryResource; rename org.apache.druid.discovery.No… (#6702)
* Add SelfDiscoveryResource

* Rename org.apache.druid.discovery.NodeType to NodeRole. Refactor CuratorDruidNodeDiscoveryProvider. Make SelfDiscoveryResource to listen to updates only about a single node (itself).

* Extended docs

* Fix brace

* Remove redundant throws in Lifecycle.Handler.stop()

* Import order

* Remove unresolvable link

* Address comments

* tmp

* tmp

* Rollback docker changes

* Remove extra .sh files

* Move filter

* Fix SecurityResourceFilterTest
2019-12-08 18:47:58 +03:00
Jonathan Wei c949a25210
Add DruidInputSource (replacement for IngestSegmentFirehose) (#8982)
* Add Druid input source and format

* Inherit dims/metrics from segment

* Add ingest segment firehose reindexing test

* Remove unnecessary module

* Fix unit tests, checkstyle

* Add doc entry

* Fix dimensionExclusions handling, add parallel index integration test

* Add spelling exclusion

* Address some PR comments

* Checkstyle

* wip

* Address rest of PR comments

* Address PR comments
2019-12-05 16:50:00 -08:00
Chi Cao Minh af74acaa85 Address security vulnerabilities CVSS >= 7 (#8980)
* Address security vulnerabilities CVSS >= 7

Update dependencies to address security vulnerabilities with CVSS scores
of 7 or higher. A new Travis CI job is added to prevent new
high/critical security vulnerabilities from being added.

Updated dependencies:
- api-util 1.0.0 -> 1.0.3
- jackson 2.9.10 -> 2.10.1
- kafka 2.1.0 -> 2.1.1
- libthrift 0.10.0 -> 0.13.0
- protobuf 3.2.0 -> 3.11.0

The following high/critical security vulnerabilities are currently
suppressed (so that the new Travis CI job can be added now) and are left
as future work to fix:
- hibernate-validator:5.2.5
- jackson-mapper-asl:1.9.13
- libthrift:0.6.1
- netty:3.10.6
- nimbus-jose-jwt:4.41.1

* Rename EDL1 license file

* Fix inspection errors
2019-12-05 14:34:35 -08: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
Jonathan Wei 00ce18a0ea
Additional Kinesis resharding fixes (#8870)
* Additional Kinesis resharding fixes

* Address PR comments

* Remove unused method

* Adjust SegmentTransactionalInsertAction null handling

* Check for unchanged metadata on empty publish

* Add logs for empty publish

* Fix javadoc

* Clear offset when invalid endOffsets are seen

* Fix LGTM alert

* Fix build

* Add resharding note to Kinesis docs

* Checkstyle

* Spelling

* Address PR comments

* Checkstyle
2019-11-28 12:59:01 -08:00
Jihoon Son 86e8903523
Support orc format for native batch ingestion (#8950)
* Support orc format for native batch ingestion

* fix pom and remove wrong comment

* fix unnecessary condition check

* use flatMap back to handle exception properly

* move exceptionThrowingIterator to intermediateRowParsingReader

* runtime
2019-11-28 12:45:24 -08:00
jon-wei dfbc066163 Revert "[maven-release-plugin] prepare release druid-0.16.1-incubating-rc1"
This reverts commit a0f21d9b07.
2019-11-27 23:22:43 -08:00
jon-wei 0402ff85b8 Revert "[maven-release-plugin] prepare for next development iteration"
This reverts commit 8ffa71e7e6.
2019-11-27 23:22:32 -08:00
jon-wei 8ffa71e7e6 [maven-release-plugin] prepare for next development iteration 2019-11-27 23:18:48 -08:00
jon-wei a0f21d9b07 [maven-release-plugin] prepare release druid-0.16.1-incubating-rc1 2019-11-27 23:18:37 -08:00
Jihoon Son a2e6de4b16 Fix the potential race between SplittableInputSource.getNumSplits() and SplittableInputSource.createSplits() in TaskMonitor (#8924)
* Fix the potential race SplittableInputSource.getNumSplits() and SplittableInputSource.createSplits() in TaskMonitor

* Fix docs and javadoc

* Add unit tests for large or small estimated num splits

* add override
2019-11-23 01:38:08 -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
Rye 0514e5686e add TsvInputFormat (#8915)
* add TsvInputFormat

* refactor code

* fix grammar

* use enum replace string literal

* code refactor

* code refactor

* mark abstract for base class meant not to be instantiated

* remove constructor for test
2019-11-22 18:01:40 -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
Surekha d628bebbd7 Make supervisor API similar to submit task API (#8810)
* accept spec or dataSchema, tuningConfig, ioConfig while submitting task json

* fix test

* update docs

* lgtm warning

* Add original constructor back to IndexTask to minimize changes

* fix indentation in docs

* Allow spec to be specified in supervisor schema

* undo IndexTask spec changes

* update docs

* Add Nullable and deprecated annotations

* remove deprecated configs from SeekableStreamSupervisorSpec

* remove nullable annotation
2019-11-20 10:04:41 -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
Jihoon Son baefc65f80 Retrying with a backward compatible task type on unknown task type error in parallel indexing (#8905)
* Retrying with a backward compatible task type on unknown task type error in parallel indexing

* Register legacy class; add a serde test
2019-11-19 19:29:25 -08:00
Rye d0913475b7 sampler returns nulls in CSV (#8871)
* sampler returns nulls in CSV

* fixed kafka sampler test

* fix Kinesis test

* sql compatibility fix

* remove null to empty string conversion, use null

* fix sql compatibility
2019-11-19 13:59:44 -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
Surekha cf6643eb9a add sequenceName and currentCheckPoint for backwards compatibility (#8864)
* add sequenceName and currentCheckPoint for backwards compatibility

* Add serde unit test in kafka

* fix checkstyle

* add hashcode

* update javadoc
2019-11-19 13:11:31 -08:00
Chi Cao Minh 8365bdf62a Address security vulnerabilities (#8878)
* Address security vulnerabilities

Security vulnerabilities addressed by upgrading 3rd party libs:

- Upgrade avro-ipc to 1.9.1
  - sonatype-2019-0115
- Upgrade caffeine to 2.8.0
  - sonatype-2019-0282
- Upgrade commons-beanutils to 1.9.4
  - CVE-2014-0114
- Upgrade commons-codec to 1.13
  - sonatype-2012-0050
- Upgrade commons-compress to 1.19
  - CVE-2019-12402
  - sonatype-2018-0293
- Upgrade hadoop-common to 2.8.5
  - CVE-2018-11767
- Upgrade hadoop-mapreduce-client-core to 2.8.5
  - CVE-2017-3166
- Upgrade hibernate-validator to 5.2.5
  - CVE-2017-7536
- Upgrade httpclient to 4.5.10
  - sonatype-2017-0359
- Upgrade icu4j to 55.1
  - CVE-2014-8147
- Upgrade jackson-databind to 2.6.7.3:
  - CVE-2017-7525
- Upgrade jetty-http to 9.4.12:
  - CVE-2017-7657
  - CVE-2017-7658
  - CVE-2017-7656
  - CVE-2018-12545
- Upgrade log4j-core to 2.8.2
  - CVE-2017-5645:
- Upgrade netty to 3.10.6
  - CVE-2015-2156
- Upgrade netty-common to 4.1.42
  - CVE-2019-9518
- Upgrade netty-codec-http to 4.1.42
  - CVE-2019-16869
- Upgrade nimbus-jose-jwt to 4.41.1
  - CVE-2017-12972
  - CVE-2017-12974
- Upgrade plexus-utils to 3.0.24
  - CVE-2017-1000487
  - sonatype-2015-0173
  - sonatype-2016-0398
- Upgrade postgresql to 42.2.8
  - CVE-2018-10936

Note that if users are using JDBC lookups with postgres, they may need
to update the JDBC jar used by the lookup extension.

* Fix license for postgresql
2019-11-19 09:14:33 -08:00
Atul Mohan 8515a03c6b Modify batch index task naming to accomodate simultaneous tasks (#8612)
* Change hadoop task naming

* Remove unused

* Add timestamp

* Fix build
2019-11-18 15:07:16 -08:00
Rye ea8e4066f6 Use earliest offset on kafka newly discovered partitions (#8748)
* Use earliest offset on kafka newly discovered partitions

* resolve conflicts

* remove redundant check cases

* simplified unit tests

* change test case

* rewrite comments

* add regression test

* add junit ignore annotation

* minor modifications

* indent

* override testableKafkaSupervisor and KafkaRecordSupplier to make the test runable

* modified test constructor of kafkaRecordSupplier

* simplify

* delegated constructor
2019-11-18 11:05:31 -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
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
chencb cc2bdb5f51 Fix hadoop task jdk11 compatible (#8799)
* Fix hadoop task jdk11 compatible

* Fix HadoopTaskTest
2019-11-13 02:32:46 -08:00
Roman Leventov 5c0fc0a13a Fix ambiguity about IndexerSQLMetadataStorageCoordinator.getUsedSegmentsForInterval() returning only non-overshadowed or all used segments (#8564)
* IndexerSQLMetadataStorageCoordinator.getTimelineForIntervalsWithHandle() don't fetch abutting intervals; simplify getUsedSegmentsForIntervals()

* Add VersionedIntervalTimeline.findNonOvershadowedObjectsInInterval() method; Propagate the decision about whether only visible segmetns or visible and overshadowed segments should be returned from IndexerMetadataStorageCoordinator's methods to the user logic; Rename SegmentListUsedAction to RetrieveUsedSegmentsAction, SegmetnListUnusedAction to RetrieveUnusedSegmentsAction, and UsedSegmentLister to UsedSegmentsRetriever

* Fix tests

* More fixes

* Add javadoc notes about returning Collection instead of Set. Add JacksonUtils.readValue() to reduce boilerplate code

* Fix KinesisIndexTaskTest, factor out common parts from KinesisIndexTaskTest and KafkaIndexTaskTest into SeekableStreamIndexTaskTestBase

* More test fixes

* More test fixes

* Add a comment to VersionedIntervalTimelineTestBase

* Fix tests

* Set DataSegment.size(0) in more tests

* Specify DataSegment.size(0) in more places in tests

* Fix more tests

* Fix DruidSchemaTest

* Set DataSegment's size in more tests and benchmarks

* Fix HdfsDataSegmentPusherTest

* Doc changes addressing comments

* Extended doc for visibility

* Typo

* Typo 2

* Address comment
2019-11-06 11:07:04 -08:00
Giuseppe Martino 9c171e2b1f Message rejection absolute date (#8656)
* Add option lateMessageRejectionStartDate

* Use option lateMessageRejectionStartDate

* Fix tests

* Add lateMessageRejectionStartDate to kafka indexing service

* Update tests kafka indexing service

* Fix tests for KafkaSupervisorTest

* Add lateMessageRejectionStartDate to KinesisSupervisorIOConfig

* Fix var name

* Update documentation

* Add check lateMessageRejectionStartDateTime and lateMessageRejectionPeriod, fails if both were specified.
2019-10-31 15:13:02 -07:00
Jihoon Son 2363b61983
Asynchronous file copy in the shuffle of parallel indexing task (#8783) 2019-10-30 18:00:05 -07:00
Xiaobao b9d10473a5 fix typo (#8745) 2019-10-25 19:21:56 +08:00
Jihoon Son 094936ca03 Remove commit() method Firehose (#8688)
* Remove commit() method Firehose

* fix javadoc
2019-10-23 16:52:02 -07:00