Commit Graph

1645 Commits

Author SHA1 Message Date
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
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
Roman Leventov ceb969903f
Refactor SQLMetadataSegmentManager; Change contract of REST met… (#7653)
* Refactor SQLMetadataSegmentManager; Change contract of REST methods in DataSourcesResource

* Style fixes

* Unused imports

* Fix tests

* Fix style

* Comments

* Comment fix

* Remove unresolvable Javadoc references; address comments

* Add comments to ImmutableDruidDataSource

* Merge with master

* Fix bad web-console merge

* Fixes in api-reference.md

* Rename in DruidCoordinatorRuntimeParams

* Fix compilation

* Residual changes
2019-07-17 17:18:48 +03: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
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
Gian Merlino 338b8b3fef
SupervisorManager: Add authorization checks to bulk endpoints. (#8044)
The endpoints added in #6272 were missing authorization checks. This patch removes the bulk
methods from SupervisorManager, and instead has SupervisorResource run the full list through
filterAuthorizedSupervisorIds before calling resume/suspend/terminate one by one.
2019-07-09 13:16:54 -07:00
Sashidhar Thallam 6701dc08fe Making StatusResponseHandler singleton and fixing all its instantiation invocations (#7969)
* Making StatusResponseHandler singleton and fixing all its instantiation invocations

* Using StatusResponseHandler.getInstance() where applicable
2019-07-08 13:33:00 +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
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
Fokko Driesprong 4508d3ff55 Enable RCN_REDUNDANT_NULLCHECK_OF_NULL_VALUE (#7972) 2019-07-01 11:25:06 +05:30
Fokko Driesprong e8dedb3de4 Enable MS_MUTABLE_COLLECTION_PKGPROTECT (#7971) 2019-07-01 11:24:05 +05:30
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
Himanshu 417fcef385 WorkerTaskManager to create disk files atomically and ignore task file corruption (#7917)
* WorkerTaskManager to create disk files atomically and ignore task file
corruptions

* fixing weird checkstyle lambda indentation issues
2019-06-18 09:18:43 -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
litao91 daed1bc141 Fix HttpRemoteTaskRunnerConfig not loading (#7902) 2019-06-16 00:31:42 -07:00
Sashidhar Thallam 3bee6adcf7 Use map.putIfAbsent() or map.computeIfAbsent() as appropriate instead of containsKey() + put() (#7764)
* https://github.com/apache/incubator-druid/issues/7316 Use Map.putIfAbsent() instead of containsKey() + put()

* fixing indentation

* Using map.computeIfAbsent() instead of map.putIfAbsent() where appropriate

* fixing checkstyle

* Changing the recommendation text

* Reverting auto changes made by IDE

* Implementing recommendation: A ConcurrentHashMap on which computeIfAbsent() is called should be assigned into variables of ConcurrentHashMap type, not ConcurrentMap

* Removing unused import
2019-06-14 17:59:36 +02: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
Surekha ea752ef562 Optimize overshadowed segments computation (#7595)
* Move the overshadowed segment computation to SQLMetadataSegmentManager's poll

* rename method in MetadataSegmentManager

* Fix tests

* PR comments

* PR comments

* PR comments

* fix indentation

* fix tests

*  fix test

*  add test for SegmentWithOvershadowedStatus serde format

* PR comments

* PR comments

* fix test

* remove snapshot updates outside poll

* PR comments

* PR comments

* PR comments

*  removed unused import
2019-06-07 19:15:54 +02:00
Jihoon Son 61ec521135
Remove keepSegmentGranularity option for compaction (#7747)
* Remove keepSegmentGranularity option from compaction

* fix it test

* clean up

* remove from web console

* fix test
2019-06-03 12:59:15 -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
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
Jonathan Wei c97eb93e16 Fix dead code in IndexTask.collectIntervalsAndShardSpecs() (#7746) 2019-05-25 10:12:14 -07:00
Bartosz Ługowski cbdac49ab3 Web console - add enable/disable actions for middle manager workers (#7642)
* Overlord console - add enable/disable button for remote workers.

* Overlord console - add proxy for remote workers API.

* WorkerResourceTest - revert newline change.

* Remote worker proxy tests - remove empty line.

* Refactor remote worker proxy for readability and security

* Rename method in remote task runner tests for readability

* Remove enable/disable button for remote workers from old web console

* Add enable/disable actions for middle manager worker in new web console

* Fix variable type

* Add worker task runner query adapter

* Fix web console tests: segments-view, servers-view

* Fix overlord resource tests
2019-05-23 16:47:23 -07:00
Himanshu daf20b4b86 use memoized supplier for lazy singleton in SeekableStreamIndexTask.java (#7740) 2019-05-23 15:58:38 -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
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
Xavier Léauté b3c7463059
fix test pattern matching for jdk11 (#7602)
This change makes the error pattern more generic to account for
differences between JDK versions.
2019-05-06 21:02:44 -07:00
David Lim ec8562c885 Data loader (sampler component) (#7531)
* sampler initial check-in
fix checkstyle issues
add sampler fix to process CSV files from cache properly
change to composition and rename some classes
add tests and report num rows read and indexed
remove excludedByFilter flag and don't send filtered out data
fix tests to handle both settings for druid.generic.useDefaultValueForNull

* wrap sampler firehose in TimedShutoffFirehoseFactory to support timeouts

* code review changes - add additional comments, limit maxRows
2019-05-01 22:37:14 -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
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
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
Mingming Qiu 7a38d28cf3 run pending tasks when assigned a task that is already pending (#6991)
* run pending tasks when assigned a task that is already pending

* add unit test

* fix pending tasks aren't going to run

* address comments
2019-04-14 23:37:15 +08:00
Eugene Sevastyanov c57e62bcd6 7153: Isolated a supplier's close by removing its invocation (#7459)
from a non-notice-processing thread
2019-04-12 19:13:08 -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
lxqfy 6b741f0800 Overwrite index task maxTotalRows with computed maxRowsPerSegments (#7338)
fixed identation

set maxTotalRows to higher value in case mutiple segments in appendrator
with comments
2019-04-03 11:47:59 -07:00
David Glasser 4e23c11345 Make IngestSegmentFirehoseFactory splittable for parallel ingestion (#7048)
* Make IngestSegmentFirehoseFactory splittable for parallel ingestion

* Code review feedback

- Get rid of WindowedSegment
- Don't document 'segments' parameter or support splitting firehoses that use it
- Require 'intervals' in WindowedSegmentId (since it won't be written by hand)

* Add missing @JsonProperty

* Integration test passes

* Add unit test

* Remove two FIXME comments from CompactionTask

I'd like to leave this PR in a potentially mergeable state, but I still would
appreciate reviewer eyes on the questions I'm removing here.

* Updates from code review
2019-04-02 14:59:17 -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
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 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
Surekha e170203876 Consolidate kafka consumer configs (#7249)
* Consolidate kafka consumer configs

* change the order of adding properties

* Add consumer properties to fix test

it seems kafka consumer does not reveive any message without these configs

* Use KafkaConsumerConfigs in integration test

* Update zookeeper and kafka versions in the setup.sh for the base druid image

*  use version 0.2 of base druid image

* Try to fix tests in KafkaRecordSupplierTest

* unused import

* Fix tests in KafkaSupervisorTest
2019-03-21 11:19:49 -07:00
Roman Leventov dfd27e00c0
Avoid many unnecessary materializations of collections of 'all segments in cluster' cardinality (#7185)
* Avoid many  unnecessary materializations of collections of 'all segments in cluster' cardinality

* Fix DruidCoordinatorTest; Renamed DruidCoordinator.getReplicationStatus() to computeUnderReplicationCountsPerDataSourcePerTier()

* More Javadocs, typos, refactor DruidCoordinatorRuntimeParams.createAvailableSegmentsSet()

* Style

* typo

* Disable StaticPseudoFunctionalStyleMethod inspection because of too much false positives

* Fixes
2019-03-19 18:22:56 -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
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
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
Jihoon Son 873232954f Fix log level and throw NPE on null currOffset in SeekableStreamIndexTaskRunner (#7253) 2019-03-13 10:20:43 -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
Gian Merlino dcfca03718
More accurate RealtimeMetricsMonitor messages. (#7230)
The old messages did not reflect the full range of reasons why messages
could be thrown away.
2019-03-11 19:50:32 -04:00
Jihoon Son e48a9c138e Reduce default max # of subTasks to 1 for native parallel task (#7181)
* Reduce # of max subTasks to 2

* fix typo and add more doc

* add more doc and link

* change default and add warning

* fix doc

* add test

* fix it test
2019-03-05 22:06:36 -08:00
Gian Merlino fa218f5160 Fix two SeekableStream serde issues. (#7176)
* Fix two SeekableStream serde issues.

1) Fix backwards-compatibility serde for SeekableStreamPartitions. It is needed
   for split 0.13 / 0.14 clusters to work properly during a rolling update.
2) Abstract classes don't need JsonCreator constructors; remove them.

* Comment fixes.
2019-03-01 22:27:08 -08:00
Jihoon Son 06c8229c08
Kill all running tasks when the supervisor task is killed (#7041)
* Kill all running tasks when the supervisor task is killed

* add some docs and simplify

* address comment
2019-03-01 11:28:03 -08:00
Jihoon Son cacdc83cad Improve error message for integer overflow in compaction task (#7131)
* improve error message for integer overflow in compaction task

* fix build
2019-02-28 11:07:37 +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
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
David Glasser a81b1b8c9c index_parallel: support !appendToExisting with no explicit intervals (#7046)
* index_parallel: support !appendToExisting with no explicit intervals

This enables ParallelIndexSupervisorTask to dynamically request locks at runtime
if it is run without explicit intervals in the granularity spec and with
appendToExisting set to false.  Previously, it behaved as if appendToExisting
was set to true, which was undocumented and inconsistent with IndexTask and
Hadoop indexing.

Also, when ParallelIndexSupervisorTask allocates segments in the explicit
interval case, fail if its locks on the interval have been revoked.

Also make a few other additions/clarifications to native ingestion docs.

Fixes #6989.

* Review feedback.

PR description on GitHub updated to match.

* Make native batch ingestion partitions start at 0

* Fix to previous commit

* Unit test. Verified to fail without the other commits on this branch.

* Another round of review

* Slightly scarier warning
2019-02-20 10:54:26 -08:00
Clint Wylie cadb6c5280 Missing Overlord and MiddleManager api docs (#7042)
* document middle manager api

* re-arrange

* correction

* document more missing overlord api calls, minor re-arrange of some code i was referencing

* fix it

* this will fix it

* fixup

* link to other docs
2019-02-19 10:52:05 -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
Jihoon Son 1701fbcad3
Improve error message for revoked locks (#7035)
* Improve error message for revoked locks

* fix test

* fix test

* fix test

* fix toString
2019-02-13 11:22:48 -08:00
Mingming Qiu d0abf5c20a fix kafka index task doesn't resume when recieve duplicate request (#6990)
* fix kafka index task doesn't resume when recieve duplicate request

* add unit test
2019-02-12 13:24:28 -08:00
Ankit Kothari 16a4a50e91 [Issue #6967] NoClassDefFoundError when using druid-hdfs-storage (#7015)
* Fix:
  1. hadoop-common dependency for druid-hdfs and druid-kerberos extensions
 Refactoring:
  2. Hadoop config call in the inner static class to avoid class path conflicts for stopGracefully kill

* Fix:
  1. hadoop-common test dependency

* Fix:
  1. Avoid issue of kill command once the job is actually completed
2019-02-08 18:26:37 -08:00
Jonathan Wei fafbc4a80e
Set version to 0.15.0-incubating-SNAPSHOT (#7014) 2019-02-07 14:02:52 -08:00
Furkan KAMACI 58f9507ccf Improper equals override is fixed to prevent NullPointerException (#6938)
* Improper equals override is fixed to prevent NullPointerException

* Fixed curly brace indentation.

* Test method is added for equals method of TaskLockPosse class.
2019-02-06 14:08:50 -08:00
Jonathan Wei 8bc5eaa908
Set version to 0.14.0-incubating-SNAPSHOT (#7003) 2019-02-04 19:36:20 -08:00
David Glasser 7e48593b57 ParallelIndexSupervisorTask: don't warn about a default value (#6987)
Native batch indexing doesn't yet support the maxParseExceptions,
maxSavedParseExceptions, and logParseExceptions tuning config options, so
ParallelIndexSupervisorTask logs if these are set. But the default value for
maxParseExceptions is Integer.MAX_VALUE, which means that you'll get the
maxParseExceptions flavor of this warning even if you don't configure
maxParseExceptions.

This PR changes all three warnings to occur if you change the settings from the
default; this mostly affects the maxParseExceptions warning.
2019-02-04 12:00:26 -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
Vadim Ogievetsky 7f1b19bfb1 Adding a Unified web console. (#6923)
* Adding new web console.

* fixed css

* fix form height

* fix typo

* do import custom react-table css

* added repo field so npm does not complain

* ask travis for node 10

* move indexing-service/src/main/resources/indexer_static into web-console

* fix resource names and paths

* add licenses

* fix exclude file

* add licenses to misc files and tidy up

* remove rebase marker

* fix link

* updated env variable name

* tidy up licenses and surface errors

* cleanup

* remove unused code, fix missing await

* TeamCity does not like the name aux

* add more links to tasks view

* rm pages

* update gitignore

* update readme to be accurate

* make clean script

* removed old console dependancy

* update Jetty routes

* add a comment for welcome files for coordinator

* do not show inital notifaction for now

* renamed overlord console back to console.html

* fix coordinator console

* rename coordinator-console.html to index.html
2019-01-31 17:26:41 -08: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
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
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
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
Clint Wylie 8ba33b2505 add 'init' lifecycle stage for finer control over startup and shutdown (#6864)
* add Lifecycle.Stage.INIT, put log shutter downer in init stage, tests, rad startup banner

* log cleanup

* log changes

* add task-master lifecycle to module lifecycle to gracefully stop task-master stuff

* fix it the right way

* remove announce spam

* unused import

* one more log

* updated comments

* wrap leadership lifecycle stop to prevent exceptions from wrecking rest of task master stop

* add precondition check
2019-01-21 09:01:36 -08:00
Jonathan Wei 8537a771b0 Some fixes and tests for spaces/non-ASCII chars in datasource names (#6761)
* Fixes and tests for spaces/non-ASCII datasource names

* Some unit test fixes

* Fix ITRealtimeIndexTaskTest

* Checkstyle

* TeamCity

* PR comments
2019-01-15 08:33:31 -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 39780d914c Propagate errors to RemoteTaskActionClient (#6837)
* Propagate errors to RemoteTaskActionClient

* style
2019-01-10 20:53:18 -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
Jihoon Son 934c83bca6 Fix TaskLockbox when there are multiple intervals of the same start but differerent end (#6822)
* Fix TaskLockbox when there are multiple intervals of the same start but differernt end

* fix build

* fix npe
2019-01-09 19:38:27 -08:00
Jihoon Son c4716d1639 Fix ParallelIndexTask when publishing empty segments (#6807)
* Fix ParallelIndexTask when publishing empty segments

* unused import
2019-01-08 17:15:16 -08:00
Jihoon Son 9ad6a733a5 Add support segmentGranularity for CompactionTask (#6758)
* Add support segmentGranularity

* add doc and fix combination of options

* improve doc
2019-01-03 17:50:45 -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
Clint Wylie 9505074530 fix log typo (#6755)
* fix log typo, add DataSegmentUtils.getIdentifiersString util method

* fix indecisive oops
2018-12-18 15:10:25 -08:00
Clint Wylie 486c6f3cf9 emit logs that are only useful for debugging at debug level (#6741)
* make logs that are only useful for debugging be at debug level so log volume is much more chill

* info level messages for total merge buffer allocated/free

* more chill compaction logs
2018-12-17 14:20:28 +08:00
Gian Merlino 04e7c7fbdc FilteredRequestLogger: Fix start/stop, invalid delegate behavior. (#6637)
* FilteredRequestLogger: Fix start/stop, invalid delegate behavior.

Fixes two bugs:

1) FilteredRequestLogger did not start/stop the delegate.

2) FilteredRequestLogger would ignore an invalid delegate type, and
instead silently substitute the "noop" logger. This was due to a larger
problem with RequestLoggerProvider setup in general; the fix here is
to remove "defaultImpl" from the RequestLoggerProvider interface, and
instead have JsonConfigurator be responsible for creating the
default implementations. It is stricter about things than the old system
was, and is only willing to make a noop logger if it doesn't see any
request logger configs. Otherwise, it'll raise a provision error.

* Remove unneeded annotations.
2018-12-14 16:55:44 +08:00
dongyifeng 91e3cf7196 add charset UTF-8 to log api (#6709)
When I retrieve the task log in browser, the Chinese characters all end up as garbage.
![image](https://user-images.githubusercontent.com/1322134/49502749-bd614080-f8b0-11e8-839e-07f7117eebfd.png)
After adding charset UTF-8, it was correct.
![image](https://user-images.githubusercontent.com/1322134/49502804-dc5fd280-f8b0-11e8-916b-bda8f1e7f318.png)
2018-12-12 16:31:04 +01:00
Jihoon Son f727333b70 Fix shutdownAllTasks API for non-existing dataSource (#6706) 2018-12-08 09:54:01 -08:00
Mingming Qiu 607339003b Add TaskCountStatsMonitor to monitor task count stats (#6657)
* Add TaskCountStatsMonitor to monitor task count stats

* address comments

* add file header

* tweak test
2018-12-04 13:37:17 -08:00
Clint Wylie a1c9d0add2 autosize processing buffers based on direct memory sizing by default (#6588)
* autosize processing buffers based on direct memory sizing

* remove oops, more test

* max 1gb autosize buffers, test, start of docs

* fix oops

* revert accidental change

* print buffer size in exception

* change the things
2018-12-03 18:40:02 -07:00
Clint Wylie 43adb391c2 remove AbstractResourceFilter.isApplicable because it is not (#6691)
* remove AbstractResourceFilter.isApplicable because it is not, add tests for OverlordResource.doShutdown and OverlordResource.shutdownTasksForDatasource

* cleanup
2018-12-01 21:52:31 +08:00
Roman Leventov ec38df7575
Simplify DruidNodeDiscoveryProvider; add DruidNodeDiscovery.Listener.nodeViewInitialized() (#6606)
* Simplify DruidNodeDiscoveryProvider; add DruidNodeDiscovery.Listener.nodeViewInitialized() method; prohibit and eliminate some suboptimal Java 8 patterns

* Fix style

* Fix HttpEmitterTest.timeoutEmptyQueue()

* Add DruidNodeDiscovery.Listener.nodeViewInitialized() calls in tests

* Clarify code
2018-12-01 01:12:56 +01:00
Jihoon Son d6539abd0a Fix overlord api and console (#6686)
* Fix overlord APIs and console

* remove getRunningTasksByDataSource

* add missing path to isApplicable
2018-11-29 23:45:28 -08:00
Mingming Qiu c81cb94226 fix cannot resolve param at OverlordResource#getTasks (#6679) 2018-11-29 10:07:21 +08:00
David Glasser d150483fd3 indexing-service: fix HTML title on overlord console (#6671)
This follows a similar fix to the body of the page done in #5627.
2018-11-27 22:34:26 -08:00
Jihoon Son 422b76b33c Fix IndexTaskClient to retry on ChannelException (#6649)
* Fix IndexTaskClient to retry on ChannelException

* fix travis and add javadoc

* address comment
2018-11-27 15:54:38 -08:00