#### `EventReceiverFirehoseFactory`
Fixed several concurrency bugs in `EventReceiverFirehoseFactory`:
- Race condition over putting an entry into `producerSequences` in `checkProducerSequence()`.
- `Stopwatch` used to measure time across threads, but it's a non-thread-safe class.
- Use `System.nanoTime()` instead of `System.currentTimeMillis()` because the latter are [not suitable](https://stackoverflow.com/a/351571/648955) for measuring time intervals.
- `close()` was not synchronized by could be called from multiple threads concurrently.
Removed unnecessary `readLock` (protecting `hasMore()` and `nextRow()` which are always called from a single thread). Removed unnecessary `volatile` modifiers.
Documented threading model and concurrent control flow of `EventReceiverFirehose` instances.
**Important:** please read the updated Javadoc for `EventReceiverFirehose.addAll()`. It allows events from different requests (batches) to be interleaved in the buffer. Is this OK?
#### `TimedShutoffFirehoseFactory`
- Fixed a race condition that was possible because `close()` that was not properly synchronized.
Documented threading model and concurrent control flow of `TimedShutoffFirehose` instances.
#### `Firehose`
Refined concurrency contract of `Firehose` based on `EventReceiverFirehose` implementation. Importantly, now it states that `close()` doesn't affect `hasMore()` and `nextRow()` and could be called concurrently with them. In other words, specified that `close()` is for "row supply" side rather than "row consume" side. However, I didn't check that other `Firehose` implementatations adhere to this contract.
<hr>
This issue is the result of reviewing `EventReceiverFirehose` and `TimedShutoffFirehose` using [this checklist](https://medium.com/@leventov/code-review-checklist-java-concurrency-49398c326154).
* 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
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.
* Remove DataSegmentFinder, InsertSegmentToDb, and descriptor.json file
* delete descriptor.file when killing segments
* fix test
* Add doc for ha
* improve warning
* maintenance mode for Historical
forbidden api fix, config deserialization fix
logging fix, unit tests
* addressed comments
* addressed comments
* a style fix
* addressed comments
* a unit-test fix due to recent code-refactoring
* docs & refactoring
* addressed comments
* addressed a LoadRule drop flaw
* post merge cleaning up
* 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
* Add published segment cache in broker
* Change the DataSegment interner so it's not based on DataSEgment's equals only and size is preserved if set
* Added a trueEquals to DataSegment class
* Use separate interner for realtime and historical segments
* Remove trueEquals as it's not used anymore, change log message
* PR comments
* PR comments
* Fix tests
* PR comments
* Few more modification to
* change the coordinator api
* removeall segments at once from MetadataSegmentView in order to serve a more consistent view of published segments
* Change the poll behaviour to avoid multiple poll execution at same time
* minor changes
* PR comments
* PR comments
* Make the segment cache in broker off by default
* Added a config to PlannerConfig
* Moved MetadataSegmentView to sql module
* Add doc for new planner config
* Update documentation
* PR comments
* some more changes
* PR comments
* fix test
* remove unintentional change, whether to synchronize on lifecycleLock is still in discussion in PR
* minor changes
* some changes to initialization
* use pollPeriodInMS
* Add boolean cachePopulated to check if first poll succeeds
* Remove poll from start()
* take the log message out of condition in stop()
* 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
* * 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
* use SqlLifecyle to manage sql execution, add sqlId
* add sql request logger
* fix UT
* rename sqlId to sqlQueryId, sql/time to sqlQuery/time, etc
* add docs and more sql request logger impls
* add UT for http and jdbc
* fix forbidden use of com.google.common.base.Charsets
* fix UT in QuantileSqlAggregatorTest, supressed unused warning of getSqlQueryId
* do not use default method in QueryMetrics interface
* capitalize 'sql' everywhere in the non-property parts of the docs
* use RequestLogger interface to log sql query
* minor bugfixes and add switching request logger
* add filePattern configs for FileRequestLogger
* address review comments, adjust sql request log format
* fix inspection error
* try SuppressWarnings("RedundantThrows") to fix inspection error on ComposingRequestLoggerProvider
* Fix num_replicas count from sys.segments
* Adjust unit test for num_replica > 1
* Pass named arguments instead of passing boolean constants
* Address PR comments
* PR comments
* 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
* Handoff should ignore segments that are dropped by drop rules
* fix travis-ci
* fix tests
* address comments
* remove line added by accident
* address comments
* add javadoc and logging the full stack trace of exception
* add error message
* Fix issue that tasks failed because of no sink for identifier
* make find sinks to persist run in one callable together with the actual persist work
* Revert "make find sinks to persist run in one callable together with the actual persist work"
This reverts commit a24a2d80ae.
* Broker: Await initialization before finishing startup.
In particular, hold off on announcing the service and starting the
HTTP server until the server view and SQL metadata cache are finished
initializing. This closes a window of time where a Broker could return
partial results shortly after startup.
As part of this, some simplification of server-lifecycle service
announcements. This helps ensure that the two different kinds of
announcements we do (legacy and new-style) stay in sync.
* Remove unused imports.
* Fix NPE in ServerRunnable.
* 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
* 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.
* 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
* remove AbstractResourceFilter.isApplicable because it is not, add tests for OverlordResource.doShutdown and OverlordResource.shutdownTasksForDatasource
* cleanup