Commit Graph

47 Commits

Author SHA1 Message Date
Jihoon Son 22d6384d36
Fix unrealistic test variables in KafkaSupervisorTest and tidy up unused variable in checkpointing process (#7319)
* Fix unrealistic test arguments in KafkaSupervisorTest

* remove currentCheckpoint from checkpoint action

* rename variable
2019-08-21 10:58:22 -07:00
Jihoon Son 5dac6375f3
Add support for parallel native indexing with shuffle for perfect rollup (#8257)
* Add TaskResourceCleaner; fix a couple of concurrency bugs in batch tasks

* kill runner when it's ready

* add comment

* kill run thread

* fix test

* Take closeable out of Appenderator

* add javadoc

* fix test

* fix test

* update javadoc

* add javadoc about killed task

* address comment

* Add support for parallel native indexing with shuffle for perfect rollup.

* Add comment about volatiles

* fix test

* fix test

* handling missing exceptions

* more clear javadoc for stopGracefully

* unused import

* update javadoc

* Add missing statement in javadoc

* address comments; fix doc

* add javadoc for isGuaranteedRollup

* Rename confusing variable name and fix typos

* fix typos; move fetch() to a better home; fix the expiration time

* add support https
2019-08-15 17:43:35 -07:00
Jihoon Son 8a16a8e97f
Teach tasks what machine they are running on (#8190)
* Teach the middleManager port to tasks

* parent annotation

* Bind parent for indexer
2019-08-02 15:34:44 -07:00
Jonathan Wei 640b7afc1c Add CliIndexer process type and initial task runner implementation (#8107)
* Add CliIndexer process type and initial task runner implementation

* Fix HttpRemoteTaskRunnerTest

* Remove batch sanity check on PeonAppenderatorsManager

* Fix paralle index tests

* PR comments

* Adjust Jersey resource logging

* Additional cleanup

* Fix SystemSchemaTest

* Add comment to LocalDataSegmentPusherTest absolute path test

* More PR comments

* Use Server annotated with RemoteChatHandler

* More PR comments

* Checkstyle

* PR comments

* Add task shutdown to stopGracefully

* Small cleanup

* Compile fix

* Address PR comments

* Adjust TaskReportFileWriter and fix nits

* Remove unnecessary closer

* More PR comments

* Minor adjustments

* PR comments

* ThreadingTaskRunner: cancel  task run future not shutdownFuture and remove thread from workitem
2019-07-29 17:06:33 -07:00
Chi Cao Minh ab71a2e1e4 Revert "Fix dependency analyze warnings (#8128)" (#8189)
This reverts commit 5dd0d8e873.
2019-07-29 11:42:16 -07:00
Chi Cao Minh 5dd0d8e873 Fix dependency analyze warnings (#8128)
* Fix dependency analyze warnings

Update the maven dependency plugin to the latest version and fix all
warnings for unused declared and used undeclared dependencies in the
compile scope. Added new travis job to add the check to CI. Also fixed
some source code files to use the correct packages for their imports.

* Fix licenses and dependencies

* Fix licenses and dependencies again

* Fix integration test dependency

* Address review comments

* Fix unit test dependencies

* Fix integration test dependency

* Fix integration test dependency again

* Fix integration test dependency third time

* Fix integration test dependency fourth time

* Fix compile error

* Fix assert package
2019-07-26 10:49:03 -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
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
Jihoon Son c7eb7cd018
Add intermediary data server for shuffle (#8088)
* Add intermediary data server for shuffle

* javadoc

* adjust timeout

* resolved todo

* fix test

* style

* address comments

* rename to shuffleDataLocations

* Address comments

* bit adjustment StorageLocation

* fix test

* address comment & fix test

* handle interrupted exception
2019-07-18 14:46:47 -07:00
Himanshu 14aec7fcec
add config to optionally disable all compression in intermediate segment persists while ingestion (#7919)
* disable all compression in intermediate segment persists while ingestion

* more changes and build fix

* by default retain existing indexingSpec for intermediate persisted segments

* document indexSpecForIntermediatePersists index tuning config

* fix build issues

* update serde tests
2019-07-10 12:22:24 -07:00
Jihoon Son fcf56f2330 Add IS_INCREMENTAL_HANDOFF_SUPPORTED for KIS backward compatibility (#8050)
* Add IS_INCREMENTAL_HANDOFF_SUPPORTED for KIS backward compatibility

* do it for kafka only

* fix test
2019-07-10 08:29:37 -07:00
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
Clint Wylie 42a7b8849a remove FirehoseV2 and realtime node extensions (#8020)
* remove firehosev2 and realtime node extensions

* revert intellij stuff

* rat exclusion
2019-07-04 15:40:22 -07:00
Clint Wylie 3fbb0a5e00 Supervisor list api with states and health (#7839)
* allow optionally listing all supervisors with their state and health

* docs

* add state to full

* clean

* casing

* format

* spelling
2019-06-07 16:26:33 -07:00
Gian Merlino 1de1a02e49 Kinesis: Fix getPartitionIds, should be checking isHasMoreShards. (#7830) 2019-06-04 16:26:22 -07:00
Justin Borromeo 8032c4add8 Add errors and state to stream supervisor status API endpoint (#7428)
* Add state and error tracking for seekable stream supervisors

* Fixed nits in docs

* Made inner class static and updated spec test with jackson inject

* Review changes

* Remove redundant config param in supervisor

* Style

* Applied some of Jon's recommendations

* Add transience field

* write test

* implement code review changes except for reconsidering logic of markRunFinishedAndEvaluateHealth()

* remove transience reporting and fix SeekableStreamSupervisorStateManager impl

* move call to stateManager.markRunFinished() from RunNotice to runInternal() for tests

* remove stateHistory because it wasn't adding much value, some fixes, and add more tests

* fix tests

* code review changes and add HTTP health check status

* fix test failure

* refactor to split into a generic SupervisorStateManager and a specific SeekableStreamSupervisorStateManager

* fixup after merge

* code review changes - add additional docs

* cleanup KafkaIndexTaskTest

* add additional documentation for Kinesis indexing

* remove unused throws class
2019-05-31 17:16:01 -07:00
Jihoon Son 7abfbb066a Bump up snapshot version to 0.16.0 (#7802) 2019-05-30 17:17:33 -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
Jihoon Son eff2be4f8f Remove LegacyKafkaIndexTaskRunner (#7735) 2019-05-23 09:25:35 -07:00
Gian Merlino 53b6467fc8 SeekableStreamIndexTaskRunner: Lazy init of runner. (#7729)
The main motivation is that this fixes #7724, by making it so the overlord
doesn't try to create a task runner and parser when all it really wants to
do is create a task object and serialize it.
2019-05-22 21:13:57 -07:00
David Lim d38457933f Data loader (sampler component) - Kafka/Kinesis samplers (#7566)
* implement Kafka/Kinesis sampler

* add KafkaSamplerSpecTest and KinesisSamplerSpecTest

* code review changes
2019-05-16 20:26:23 -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
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
Justin Borromeo 2771ed50b0 Support Kafka supervisor adopting running tasks between versions (#7212)
* Recompute hash in isTaskCurrent() and added tests

* Fixed checkstyle stuff

* Fixed failing tests

* Make TestableKafkaSupervisorWithCustomIsTaskCurrent static

* Add doc

* baseSequenceName change

* Added comment

* WIP

* Fixed imports

* Undid lambda change for diff sake

* Cleanup

* Added comment

* Reinsert Kafka tests

* Readded kinesis test

* Readd bad partition assignment in kinesis supervisor test

* Nit

* Misnamed var
2019-04-10 18:16:38 -07:00
Jonathan Wei 09abc4a856 Fix comments in KinesisIndexTaskIOConfig (#7430) 2019-04-09 08:40:08 -07:00
Jihoon Son e87d6e32b3 Support kinesis compatibility (#7351) 2019-04-08 19:19:34 -07:00
Charles Allen eeb3dbe79d Move GCP to a core extension (#6953)
* Move GCP to a core extension

* Don't provide druid-core >.<

* Keep AWS and GCP modules separate

* Move AWSModule to its own module

* Add aws ec2 extension and more modules in more places

* Fix bad imports

* Fix test jackson module

* Include AWS and GCP core in server

* Add simple empty method comment

* Update version to 15

* One more 0.13.0-->0.15.0 change

* Fix multi-binding problem

* Grep for s3-extensions and update docs

* Update extensions.md
2019-03-27 09:00:43 -07:00
Jihoon Son 5294277cb4
Fix exclusive start partitions for sequenceMetadata (#7339)
* Fix exclusvie start partitions for sequenceMetadata

* add empty check
2019-03-26 14:39:07 -07:00
Jihoon Son 0c5dcf5586 Fix exclusivity for start offset in kinesis indexing service & check exclusivity properly in IndexerSQLMetadataStorageCoordinator (#7291)
* Fix exclusivity for start offset in kinesis indexing service

* some adjustment

* Fix SeekableStreamDataSourceMetadata

* Add missing javadocs

* Add missing comments and unit test

* fix SeekableStreamStartSequenceNumbers.plus and add comments

* remove extra exclusivePartitions in KafkaIOConfig and fix downgrade issue

* Add javadocs

* fix compilation

* fix test

* remove unused variable
2019-03-21 13:12:22 -07:00
Gian Merlino a8c7132482 Logic adjustments to SeekableStreamIndexTaskRunner. (#7267)
* Logic adjustments to SeekableStreamIndexTaskRunner.

A mix of simplifications and bug fixes. They are intermingled because
some of the bugs were made difficult to fix, and also more likely to
happen in the first place, by how the code was structured. I tried to
keep restructuring to a minimum. The changes are:

- Remove "initialOffsetsSnapshot", which was used to determine when to
  skip start offsets. Replace it with "lastReadOffsets", which I hope
  is more intuitive. (There is a connection: start offsets must be
  skipped if and only if they have already been read, either by a
  previous task or by a previous sequence in the same task, post-restoring.)
- Remove "isStartingSequenceOffsetsExclusive", because it should always
  be the opposite of isEndOffsetExclusive. The reason is that starts are
  exclusive exactly when the prior ends are inclusive: they must match
  up in that way for adjacent reads to link up properly.
- Don't call "seekToStartingSequence" after the initial seek. There is
  no reason to, since we expect to read continuous message streams
  throughout the task. And calling it makes offset-tracking logic
  trickier, so better to avoid the need for trickiness. I believe the
  call being here was causing a bug in Kinesis ingestion where a
  message might get double-read.
- Remove the "continue" calls in the main read loop. They are bad
  because they prevent keeping currOffsets and lastReadOffsets up to
  date, and prevent us from detecting that we have finished reading.
- Rework "verifyInitialRecordAndSkipExclusivePartition" into
  "verifyRecordInRange". It no longer has side effects. It does a sanity
  check on the message offset and also makes sure that it is not past
  the endOffsets.
- Rework "assignPartitions" to replace inline comparisons with
  "isRecordAlreadyRead" and "isMoreToReadBeforeReadingRecord" calls. I
  believe this fixes an off-by-one error with Kinesis where the last
  record would not get read. It also makes the logic easier to read.
- When doing the final publish, only adjust end offsets of the final
  sequence, rather than potentially adjusting any unpublished sequence.
  Adjusting sequences other than the last one is a mistake since it
  will extend their endOffsets beyond what they actually read. (I'm not
  sure if this was an issue in practice, since I'm not sure if real
  world situations would have more than one unpublished sequence.)
- Rename "isEndSequenceOffsetsExclusive" to "isEndOffsetExclusive". It's
  shorter and more clear, I think.
- Add equals/hashCode/toString methods to OrderedSequenceNumber.

Kafka test changes:

- Added a Kafka "testRestoreAtEndOffset" test to verify that restores at
  the very end of the task lifecycle still work properly.

Kinesis test changes:

- Renamed "testRunOnNothing" to "testRunOnSingletonRange". I think that
  given Kinesis semantics, the right behavior when start offset equals
  end offset (and there aren't exclusive partitions set) is to read that
  single offset. This is because they are both meant to be treated as
  inclusive.
- Adjusted "testRestoreAfterPersistingSequences" to expect one more
  message read. I believe the old test was wrong; it expected the task
  not to read message number 5.
- Adjusted "testRunContextSequenceAheadOfStartingOffsets" to use a
  checkpoint starting from 1 rather than 2. I believe the old test was
  wrong here too; it was expecting the task to start reading from the
  checkpointed offset, but it actually should have started reading from
  one past the checkpointed offset.
- Adjusted "testIncrementalHandOffReadsThroughEndOffsets" to expect
  11 messages read instead of 12. It's starting at message 0 and reading
  up to 10, which should be 11 messages.

* Changes from code review.
2019-03-15 00:22:42 -07:00
Jihoon Son 69a6f1154a Fix testIncrementalHandOffReadsThroughEndOffsets in Kafka/KinesisIndexTaskTest (#7264)
* Fix testIncrementalHandOffReadsThroughEndOffsets in Kafka/KinesisIndexTaskTest

* revert unnecessary change

* fix test

* remove debug log
2019-03-14 23:31:08 -04: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
Clint Wylie fb1489d313 fix SequenceMetadata deserialization (#7256)
* wip

* fix tests, stop reading if we are at end offset

* fix build

* remove restore at end offsets fix in favor of a separate PR

* use typereference from method for serialization too
2019-03-13 17:29:39 -04:00
Clint Wylie 3895914aa2 consolidate CompressionUtils.java since now in the same jar (#6908) 2019-03-13 11:02:44 -04:00
Jihoon Son 32e86ea75e Fix record validation in SeekableStreamIndexTaskRunner (#7246)
* Fix record validation in SeekableStreamIndexTaskRunner

* add kinesis test
2019-03-12 21:12:21 -07: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
David Glasser 1c2753ab90 ParallelIndexSubTask: support ingestSegment in delegating factories (#7089)
IndexTask had special-cased code to properly send a TaskToolbox to a
IngestSegmentFirehoseFactory that's nested inside a CombiningFirehoseFactory,
but ParallelIndexSubTask didn't.

This change refactors IngestSegmentFirehoseFactory so that it doesn't need a
TaskToolbox; it instead gets a CoordinatorClient and a SegmentLoaderFactory
directly injected into it.

This also refactors SegmentLoaderFactory so it doesn't depend on
an injectable SegmentLoaderConfig, since its only method always
replaces the preconfigured SegmentLoaderConfig anyway.
This makes it possible to use SegmentLoaderFactory without setting
druid.segmentCaches.locations to some dummy value.

Another goal of this PR is to make it possible for IngestSegmentFirehoseFactory
to list data segments outside of connect() --- specifically, to make it a
FiniteFirehoseFactory which can query the coordinator in order to calculate its
splits. See #7048.

This also adds missing datasource name URL-encoding to an API used by
CoordinatorBasedSegmentHandoffNotifier.
2019-02-23 17:02:56 -08:00
Jihoon Son 4e2b085201
Remove DataSegmentFinder, InsertSegmentToDb, and descriptor.json file in deep storage (#6911)
* Remove DataSegmentFinder, InsertSegmentToDb, and descriptor.json file

* delete descriptor.file when killing segments

* fix test

* Add doc for ha

* improve warning
2019-02-20 15:10:29 -08:00
Surekha 80a2ef7be4 Support kafka transactional topics (#5404) (#6496)
* Support kafka transactional topics

* update kafka to version 2.0.0
* Remove the skipOffsetGaps option since it's not used anymore
* Adjust kafka consumer to use transactional semantics
* Update tests

* Remove unused import from test

* Fix compilation

* Invoke transaction api to fix a unit test

* temporary modification of travis.yml for debugging

* another attempt to get travis tasklogs

* update kafka to 2.0.1 at all places

* Remove druid-kafka-eight dependency from integration-tests, remove the kafka firehose test and deprecate kafka-eight classes

* Add deprecated in docs for kafka-eight and kafka-simple extensions

* Remove skipOffsetGaps and code changes for transaction support

* Fix indentation

* remove skipOffsetGaps from kinesis

* Add transaction api to KafkaRecordSupplierTest

* Fix indent

* Fix test

* update kafka version to 2.1.0
2019-02-18 11:50:08 -08:00
Jonathan Wei fafbc4a80e
Set version to 0.15.0-incubating-SNAPSHOT (#7014) 2019-02-07 14:02:52 -08:00
Jonathan Wei 8bc5eaa908
Set version to 0.14.0-incubating-SNAPSHOT (#7003) 2019-02-04 19:36:20 -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
Gian Merlino ba33bdc497 Add exclusions to limit doubling up on jars. (#6927) 2019-01-28 11:06:30 -08:00
Ankit Kothari 8492d94f59 Kill Hadoop MR task on kill of Hadoop ingestion task (#6828)
* KillTask from overlord UI now makes sure that it terminates the underlying MR job, thus saving unnecessary compute

Run in jobby is now split into 2
 1. submitAndGetHadoopJobId followed by 2. run
  submitAndGetHadoopJobId is responsible for submitting the job and returning the jobId as a string, run monitors this job for completion

JobHelper writes this jobId in the path provided by HadoopIndexTask which in turn is provided by the ForkingTaskRunner

HadoopIndexTask reads this path when kill task is clicked to get hte jobId and fire the kill command via the yarn api. This is taken care in the stopGracefully method which is called in SingleTaskBackgroundRunner. Have enabled `canRestore` method to return `true` for HadoopIndexTask in order for the stopGracefully method to be called

Hadoop*Job files have been changed to incorporate the changes to jobby

* Addressing PR comments

* Addressing PR comments - Fix taskDir

* Addressing PR comments - For changing the contract of Task.stopGracefully()
`SingleTaskBackgroundRunner` calls stopGracefully in stop() and then checks for canRestore condition to return the status of the task

* Addressing PR comments
 1. Formatting
 2. Removing `submitAndGetHadoopJobId` from `Jobby` and calling writeJobIdToFile in the job itself

* Addressing PR comments
 1. POM change. Moving hadoop dependency to indexing-hadoop

* Addressing PR comments
 1. stopGracefully now accepts TaskConfig as a param
     Handling isRestoreOnRestart in stopGracefully for `AppenderatorDriverRealtimeIndexTask, RealtimeIndexTask, SeekableStreamIndexTask`
     Changing tests to make TaskConfig param isRestoreOnRestart to true
2019-01-25 15:43:06 -08:00
Charles Allen 5d2947cd52 Use Guava Compatible immediate executor service (#6815)
* Use multi-guava version friendly direct executor implementation

* Don't use a singleton

* Fix strict compliation complaints

* Copy Guava's DirectExecutor

* Fix javadoc

* Imports are the devil
2019-01-11 10:42:19 -08:00
Jihoon Son c35a39d70b
Add support maxRowsPerSegment for auto compaction (#6780)
* Add support maxRowsPerSegment for auto compaction

* fix build

* fix build

* fix teamcity

* add test

* fix test

* address comment
2019-01-10 09:50:14 -08:00
Joshua Sun 7c7997e8a1 Add Kinesis Indexing Service to core Druid (#6431)
* created seekablestream classes

* created seekablestreamsupervisor class

* first attempt to integrate kafa indexing service to use SeekableStream

* seekablestream bug fixes

* kafkarecordsupplier

* integrated kafka indexing service with seekablestream

* implemented resume/suspend and refactored some package names

* moved kinesis indexing service into core druid extensions

* merged some changes from kafka supervisor race condition

* integrated kinesis-indexing-service with seekablestream

* unite tests for kinesis-indexing-service

* various bug fixes for kinesis-indexing-service

* refactored kinesisindexingtask

* finished up more kinesis unit tests

* more bug fixes for kinesis-indexing-service

* finsihed refactoring kinesis unit tests

* removed KinesisParititons and KafkaPartitions to use SeekableStreamPartitions

* kinesis-indexing-service code cleanup and docs

* merge #6291

merge #6337

merge #6383

* added more docs and reordered methods

* fixd kinesis tests after merging master and added docs in seekablestream

* fix various things from pr comment

* improve recordsupplier and add unit tests

* migrated to aws-java-sdk-kinesis

* merge changes from master

* fix pom files and forbiddenapi checks

* checkpoint JavaType bug fix

* fix pom and stuff

* disable checkpointing in kinesis

* fix kinesis sequence number null in closed shard

* merge changes from master

* fixes for kinesis tasks

* capitalized <partitionType, sequenceType>

* removed abstract class loggers

* conform to guava api restrictions

* add docker for travis other modules test

* address comments

* improve RecordSupplier to supply records in batch

* fix strict compile issue

* add test scope for localstack dependency

* kinesis indexing task refactoring

* comments

* github comments

* minor fix

* removed unneeded readme

* fix deserialization bug

* fix various bugs

* KinesisRecordSupplier unable to catch up to earliest position in stream bug fix

* minor changes to kinesis

* implement deaggregate for kinesis

* Merge remote-tracking branch 'upstream/master' into seekablestream

* fix kinesis offset discrepancy with kafka

* kinesis record supplier disable getPosition

* pr comments

* mock for kinesis tests and remove docker dependency for unit tests

* PR comments

* avg lag in kafkasupervisor #6587

* refacotred SequenceMetadata in taskRunners

* small fix

* more small fix

* recordsupplier resource leak

* revert .travis.yml formatting

* fix style

* kinesis docs

* doc part2

* more docs

* comments

* comments*2

* revert string replace changes

* comments

* teamcity

* comments part 1

* comments part 2

* comments part 3

* merge #6754

* fix injection binding

* comments

* KinesisRegion refactor

* comments part idk lol

* can't think of a commit msg anymore

* remove possiblyResetDataSourceMetadata() for IncrementalPublishingTaskRunner

* commmmmmmmmmments

* extra error handling in KinesisRecordSupplier getRecords

* comments

* quickfix

* typo

* oof
2018-12-21 12:49:24 -07:00