* Deduplicate DataSegments contents (loadSpec's keys, dimensions and metrics lists as a whole) more aggressively; use ArrayMap instead of default LinkedHashMap for DataSegment.loadSpec, because they have only 3 entries on average; prune DataSegment.loadSpec on brokers
* Fix DataSegmentTest
* Refinements
* Try to fix
* Fix the second DataSegmentTest
* Nullability
* Fix tests
* Fix tests, unify to use TestHelper.getJsonMapper()
* Revert TestUtil as ServerTestHelper, fix tests
* Add newline
* Fix indexing tests
* Fix s3 tests
* Try to fix tests, remove lazy caching of ObjectMapper in TestHelper, rename TestHelper.getJsonMapper() to makeJsonMapper()
* Fix HDFS tests
* Fix HdfsDataSegmentPusherTest
* Capitalize constant names
* use ImmutableDruidDataSource for map and set
* address comments
* unused import
* allow returning only ImmutableDruidDataSource in MetadataSegmentManager
* address comments
* remove TreeSet
* revert to use TreeSet
* Introduce "transformSpec" at ingest-time.
It accepts a "filter" (standard query filter object) and "transforms" (a
list of objects with "name" and "expression"). These can be used to do
filtering and single-row transforms without need for a separate data
processing job.
The "expression" fields use the same expression language as other
expression-based feature.
* Remove forbidden api.
* Fix compile error.
* Fix tests.
* Some more changes.
- Add nullable annotation to Firehose.nextRow.
- Add tests for index task, realtime task, kafka task, hadoop mapper,
and ingestSegment firehose.
* Fix bad merge.
* Adjust imports.
* Adjust whitespace.
* Make Transform into an interface.
* Add missing annotation.
* Switch logger.
* Switch logger.
* Adjust test.
* Adjustment to handling for DatasourceIngestionSpec.
* Fix test.
* CR comments.
* Remove unused method.
* Add javadocs.
* More javadocs, and always decorate.
* Fix bug in TransformingStringInputRowParser.
* Fix bad merge.
* Fix ISFF tests.
* Fix DORC test.
* introducing CuratorLoadQueuePeon
* HttpLoadQueuePeon based off of current code
* Revert "Remove SegmentLoaderConfig.numLoadingThreads config (#4829)"
This reverts commit d8b3bfa63c.
* SegmentLoadDropHandler copy/pasted from ZkCoordinator
* Revert "1-based counts in ZkCoordinator (#4917)"
This reverts commit e725ff4146.
* remove non-zk part from ZkCoordinator
* remove zk part from SegmentLoadDropHandler
* additional changes for segment load/drop management with http
* address review comments
* add some more logs
* Execs class is moved
* add configs to enable fast request failure on broker
* address review comments
* fix styling error
* fix style error
* have enableRequestLimit config instead of having user specify max limit
* add comment
* fix style error
* add UT fo LimitRequestsFilter
* address review comments
* fix test
* make LimitRequestsFilterTest more robust
* fix JettyQosTest
* Make AutoScaler, ProvisioningStrategy and BaseWorkerBehaviorConfig extension points; More logging in PendingTaskBasedWorkerProvisioningStrategy
* Address comments and fix a bug
* Extract method
* debug logging
* Rename BaseWorkerBehaviorConfig to WorkerBehaviorConfig and WorkerBehaviorConfig to DefaultWorkerBehaviorConfig
* Fixes
* remove ServerConfig from DruidNode as all information needs to be present in DruidNode serialized form
* sanitize output of /druid/coordinator/v1/cluster endpoint
* fixes HttpServerInventoryView to call server/segment callbacks correctly and Unit Tests for the class
* fix checkstyle and forbidden-api errors
* HttpServerInventoryView to finish start() only after server inventory is initialized
* fix compilation errors
* address review comments
* add exponential backoff instead of fixed 5 secs on successive failures
* update test to exercise server fail scenarios
* use AtomicInteger for requestNum and increment only once
* Use internal-discovery and http for talking to overlord/coordinator leaders
* CuratorDruidNodeDiscovery.getAllNodes() best effort 30 sec wait for cache initialization
* DruidLeaderClientProvider to eagerly instantiate DruidNodeDiscovery when needed so that DruidNodeDiscovery impl cache gets initialized well in time
* Revert "DruidLeaderClientProvider to eagerly instantiate DruidNodeDiscovery when needed so that DruidNodeDiscovery impl cache gets initialized well in time"
This reverts commit f1a2432614ba56ddc2d55fe47e990d17fcfd6129.
* add lifecycle to DruidLeaderClient to early initialize DruidNodeDiscovery so that it has its cache update well in time
* Factor QueryableIndexColumnSelectorFactory and IncrementalIndexColumnSelectorFactory out of QueryableIndexStorageAdapter and IncrementalIndexStorageAdapter; Add Offset.getBaseReadableOffset(); Remove OffsetHolder interface; Replace Cursor extends ColumnSelectorFactory with composition; Reduce indirection in ColumnValueSelectors created by QueryableIndexColumnSelectorFactory
* Don't override clone() in FilteredOffset (the prev. implementation was broken); Some warnings fixed
* Simplify Cursors in QueryableIndexStorageAdapter
* Address comments
* Remove unused and unimplemented methods from GenericColumn interface
* Comments
* update LookupCoordinatorManager to use internal discovery to discover lookup nodes
* router:use internal-discovery to discover brokers
* minor [Curator]DruidDiscoveryProvider refactoring
* add initialized() method to DruidNodeDiscovery.Listener
* update HttpServerInventoryView to use initialized() and call segment callback initialization asynchronously
* Revert "update HttpServerInventoryView to use initialized() and call segment callback initialization asynchronously"
This reverts commit f796e441221fe8b0e9df87fdec6c9f47bcedd890.
* Revert "add initialized() method to DruidNodeDiscovery.Listener"
This reverts commit f0661541d073683f28fce2dd4f30ec37db90deb0.
* minor refactoring removing synchronized from DruidNodeDiscoveryProvider
* updated DruidNodeDiscovery.Listener contract to take List of nodes and first call marks initialization
* update HttpServerInventoryView to handle new contract and handle initialization
* router update to handle updated listener contract
* document DruidNodeDiscovery.Listener contract
* fix forbidden-api error
* change log level to info for unknown path children cache events in CuratorDruidNodeDiscoveryProvider
* announce broker only after segment inventory is initialized
* internal-discovery: interfaces for announcement/discovery, curator impls
* more tests
* address some review comments
* more fixes
* address more review comments
* simplify ObjectMapper setup in CuratorDruidNodeAnnouncerAndDiscoveryTest
* fix KafkaIndexTaskTest
* make lookupTier overridable via RealtimeIndexTask and KafkaIndexTask context
* make teamcity build happy
* make BrokerQueryResource instantiation singleton
* fix druid.router.http.* handling so that they are actually used and introduce numRequestsQueued for jetty http client at router
* address comments
* address review comment
* Rename ResourceManagementStrategy to ProvisioningStrategy, similarly for related classes. Make ProvisioningService non-global, created per RemoteTaskRunner instead. Add OverlordBlinkLeadershipTest.
* Fix RemoteTaskRunnerFactoryTest.testExecNotSharedBetweenRunners()
* Small fix
* Make SimpleProvisioner and PendingProvisioner more similar in details
* Fix executor name
* Style fixes
* Use LifecycleLock in RemoteTaskRunner
* Avoid usages of Default system Locale and printing to System.out or System.err in production code
* Fix Charset in DruidKerberosUtil
* Remove redundant string format in GenericIndexed
* Rename StringUtils.safeFormat() to unimportantSafeFormat(); add StringUtils.format() which fails as well as String.format()
* Fix testSafeFormat()
* More fixes of redundant StringUtils.format() inside ISE
* Rename unimportantSafeFormat() to nonStrictFormat()
* Remove DruidProcessingModule, QueryableModule and QueryRunnerFactoryModule from DI for coordinator, overlord, middle-manager. Add RouterDruidProcessing not to allocate processing resources on router
* Fix examples
* Fixes
* Revert Peon configs and add comments
* Remove qualifier
* Make PolyBind to fail if property value is not found
* Fix test
* Add onHeap option in NamespaceExtractionModule
* Add PolyBind.createChoiceWithDefaultNoScope()
* Fix NPE
* Fix
* Configure MetadataStorageProvider option for MySQL, PostgreSQL and SQLServer
* Deprecate PolyBind.createChoiceWithDefault form with unused defaultKey
* Fix NPE
* Make using implicit system charset an error
* Use StringUtils.toUtf8() and fromUtf8() instead of String.getBytes() and new String()
* Use English locale in StringUtils.safeFormat()
* Restore comment
* Adding s3a schema and s3a implem to hdfs storage module.
* use 2.7.3
* use segment pusher to make loadspec
* move getStorageDir and makeLoad spec under DataSegmentPusher
* fix uts
* fix comment part1
* move to hadoop 2.8
* inject deep storage properties
* set version to 2.7.3
* fix build issue about static class
* fix comments
* fix default hadoop default coordinate
* fix create filesytem
* downgrade aws sdk
* bump the version
* Improve concurrency of SegmentManager
* Fix SegmentManager and add more tests
* Add more tests
* Add null check to TimelineEntry
* Remove empty data source and check null in getTimeline()
* Add a comment for returning null in compute()
* Make SegmentManager LazySingleton
* Add a ServerType for peons
* Add toString() method, toString() test, unsupported type check
* Use ServerType enum in DruidServer and DruidServerMetadata
* Optional long-polling based segment announcement via HTTP instead of Zookeeper
* address review comments
* make endpoint /druid-internal/v1 instead of /druid/internal so that jetty qos filters can be configured easily when needed
* update segment callback initialization to be called only after first segment list fetch has been succeeded from all servers
* address review comments
* remove size check not required anymore as only segment servers announce themselves and not all peon processes
* annouce segment server on historical only after cached segments are loaded
* fix checkstyle errors