* Use NodeType enum instead of Strings
* Make NodeType constants uppercase
* Fix CommonCacheNotifier and NodeType/ServerType comments
* Reconsidering comment
* Fix import
* Add a comment to CommonCacheNotifier.NODE_TYPES
This PR accumulates many refactorings and small improvements that I did while preparing the next change set of https://github.com/druid-io/druid/projects/2. I finally decided to make them a separate PR to minimize the volume of the main PR.
Some of the changes:
- Renamed confusing "Generic Column" term to "Numeric Column" (what it actually implies) in many class names.
- Generified `ComplexMetricExtractor`
* Adding licenses and enable apache-rat-plugi.
Change-Id: I4685a2d9f1e147855dba69329b286f2d5bee3c18
* restore the copywrite of demo_table and add it to the list of allowed ones
Change-Id: I2a9efde6f4b984bc1ac90483e90d98e71f818a14
* revirew comments
Change-Id: I0256c930b7f9a5bb09b44b5e7a149e6ec48cb0ca
* more fixup
Change-Id: I1355e8a2549e76cd44487abec142be79bec59de2
* align
Change-Id: I70bc47ecb577bdf6b91639dd91b6f5642aa6b02f
* Rename io.druid to org.apache.druid.
* Fix META-INF files and remove some benchmark results.
* MonitorsConfig update for metrics package migration.
* Reorder some dimensions in inner queries for some reason.
* Fix protobuf tests.
* Native parallel indexing without shuffle
* fix build
* fix ci
* fix ingestion without intervals
* fix retry
* fix retry
* add it test
* use chat handler
* fix build
* add docs
* fix ITUnionQueryTest
* fix failures
* disable metrics reporting
* working
* Fix split of static-s3 firehose
* Add endpoints to supervisor task and a unit test for endpoints
* increase timeout in test
* Added doc
* Address comments
* Fix overlapping locks
* address comments
* Fix static s3 firehose
* Fix test
* fix build
* fix test
* fix typo in docs
* add missing maxBytesInMemory to doc
* address comments
* fix race in test
* fix test
* Rename to ParallelIndexSupervisorTask
* fix teamcity
* address comments
* Fix license
* addressing comments
* addressing comments
* indexTaskClient-based segmentAllocator instead of CountingActionBasedSegmentAllocator
* Fix race in TaskMonitor and move HTTP endpoints to supervisorTask from runner
* Add more javadocs
* use StringUtils.nonStrictFormat for logging
* fix typo and remove unused class
* fix tests
* change package
* fix strict build
* tmp
* Fix overlord api according to the recent change in master
* Fix it test
* Various changes about druid-services module
* Patch improvements from reviewer
* Add ToArrayCallWithZeroLengthArrayArgument & ArraysAsListWithZeroOrOneArgument into inspection profile
* Fix ArraysAsListWithZeroOrOneArgument
* Fix conflict
* Fix ToArrayCallWithZeroLengthArrayArgument
* Fix AliEqualsAvoidNull
* Remove blank line
* Remove unused import clauses
* Fix code style in TopNQueryRunnerTest
* Fix conflict
* Don't use Collections.singletonList when converting the type of array type
* Add argLine into maven-surefire-plugin in druid-process module & increase the timeout value for testMoveSegment testcase
* Roll back the latest commit
* Add java.io.File#toURL() into druid-forbidden-apis
* Using Boolean.parseBoolean instead of Boolean.valueOf for CliCoordinator#isOverlord
* Add a new regexp element into stylecode xml file
* Fix style error for new regexp
* Set the level of ArraysAsListWithZeroOrOneArgument as WARNING
* Fix style error for new regexp
* Add option BY_LEVEL for ToArrayCallWithZeroLengthArrayArgument in inspection profile
* Roll back the level as ToArrayCallWithZeroLengthArrayArgument as ERROR
* Add toArray(new Object[0]) regexp into checkstyle config file & fix them
* Set the level of ArraysAsListWithZeroOrOneArgument as ERROR & Roll back the level of ToArrayCallWithZeroLengthArrayArgument as WARNING until Youtrack fix it
* Add a comment for string equals regexp in checkstyle config
* Fix code format
* Add RedundantTypeArguments as ERROR level inspection
* Fix cannot resolve symbol datasource
* This commit introduces a new tuning config called 'maxBytesInMemory' for ingestion tasks
Currently a config called 'maxRowsInMemory' is present which affects how much memory gets
used for indexing.If this value is not optimal for your JVM heap size, it could lead
to OutOfMemoryError sometimes. A lower value will lead to frequent persists which might
be bad for query performance and a higher value will limit number of persists but require
more jvm heap space and could lead to OOM.
'maxBytesInMemory' is an attempt to solve this problem. It limits the total number of bytes
kept in memory before persisting.
* The default value is 1/3(Runtime.maxMemory())
* To maintain the current behaviour set 'maxBytesInMemory' to -1
* If both 'maxRowsInMemory' and 'maxBytesInMemory' are present, both of them
will be respected i.e. the first one to go above threshold will trigger persist
* Fix check style and remove a comment
* Add overlord unsecured paths to coordinator when using combined service (#5579)
* Add overlord unsecured paths to coordinator when using combined service
* PR comment
* More error reporting and stats for ingestion tasks (#5418)
* Add more indexing task status and error reporting
* PR comments, add support in AppenderatorDriverRealtimeIndexTask
* Use TaskReport instead of metrics/context
* Fix tests
* Use TaskReport uploads
* Refactor fire department metrics retrieval
* Refactor input row serde in hadoop task
* Refactor hadoop task loader names
* Truncate error message in TaskStatus, add errorMsg to task report
* PR comments
* Allow getDomain to return disjointed intervals (#5570)
* Allow getDomain to return disjointed intervals
* Indentation issues
* Adding feature thetaSketchConstant to do some set operation in PostAgg (#5551)
* Adding feature thetaSketchConstant to do some set operation in PostAggregator
* Updated review comments for PR #5551 - Adding thetaSketchConstant
* Fixed CI build issue
* Updated review comments 2 for PR #5551 - Adding thetaSketchConstant
* Fix taskDuration docs for KafkaIndexingService (#5572)
* With incremental handoff the changed line is no longer true.
* Add doc for automatic pendingSegments (#5565)
* Add missing doc for automatic pendingSegments
* address comments
* Fix indexTask to respect forceExtendableShardSpecs (#5509)
* Fix indexTask to respect forceExtendableShardSpecs
* add comments
* Deprecate spark2 profile in pom.xml (#5581)
Deprecated due to https://github.com/druid-io/druid/pull/5382
* CompressionUtils: Add support for decompressing xz, bz2, zip. (#5586)
Also switch various firehoses to the new method.
Fixes#5585.
* This commit introduces a new tuning config called 'maxBytesInMemory' for ingestion tasks
Currently a config called 'maxRowsInMemory' is present which affects how much memory gets
used for indexing.If this value is not optimal for your JVM heap size, it could lead
to OutOfMemoryError sometimes. A lower value will lead to frequent persists which might
be bad for query performance and a higher value will limit number of persists but require
more jvm heap space and could lead to OOM.
'maxBytesInMemory' is an attempt to solve this problem. It limits the total number of bytes
kept in memory before persisting.
* The default value is 1/3(Runtime.maxMemory())
* To maintain the current behaviour set 'maxBytesInMemory' to -1
* If both 'maxRowsInMemory' and 'maxBytesInMemory' are present, both of them
will be respected i.e. the first one to go above threshold will trigger persist
* Address code review comments
* Fix the coding style according to druid conventions
* Add more javadocs
* Rename some variables/methods
* Other minor issues
* Address more code review comments
* Some refactoring to put defaults in IndexTaskUtils
* Added check for maxBytesInMemory in AppenderatorImpl
* Decrement bytes in abandonSegment
* Test unit test for multiple sinks in single appenderator
* Fix some merge conflicts after rebase
* Fix some style checks
* Merge conflicts
* Fix failing tests
Add back check for 0 maxBytesInMemory in OnHeapIncrementalIndex
* Address PR comments
* Put defaults for maxRows and maxBytes in TuningConfig
* Change/add javadocs
* Refactoring and renaming some variables/methods
* Fix TeamCity inspection warnings
* Added maxBytesInMemory config to HadoopTuningConfig
* Updated the docs and examples
* Added maxBytesInMemory config in docs
* Removed references to maxRowsInMemory under tuningConfig in examples
* Set maxBytesInMemory to 0 until used
Set the maxBytesInMemory to 0 if user does not set it as part of tuningConfing
and set to part of max jvm memory when ingestion task starts
* Update toString in KafkaSupervisorTuningConfig
* Use correct maxBytesInMemory value in AppenderatorImpl
* Update DEFAULT_MAX_BYTES_IN_MEMORY to 1/6 max jvm memory
Experimenting with various defaults, 1/3 jvm memory causes OOM
* Update docs to correct maxBytesInMemory default value
* Minor to rename and add comment
* Add more details in docs
* Address new PR comments
* Address PR comments
* Fix spelling typo
* Update defaultHadoopCoordinates in documentation.
To match changes applied in #5382.
* Remove a parameter with defaults from example configuration file.
If it has reasonable defaults, then why would it be in an example config file?
Also, it is yet another place that has been forgotten to be updated and will be forgotten in the future.
Also, if someone is running different hadoop version, then there's much more work to be done than just changing this property, so why give users false hopes?
* Fix typo in documentation.
* Add config to allow setting up custom unsecured paths for druid nodes.
* return all resources for Unsecured paths
* review comment - Add test
* fix tests
* fix test
* Add support for task reports, upload reports to deep storage
* PR comments
* Better name for method
* Fix report file upload
* Use TaskReportFileWriter
* Checkstyle
* More PR comments
* Add graceful shutdown timeout
* Handle interruptedException
* Incorporate code review comments
* Address code review comments
* Poll for activeConnections to be zero
* Use statistics handler to get active requests
* Use native jetty shutdown gracefully
* Move log line back to where it was
* Add unannounce wait time
* Make the default retain prior behavior
* Update docs with new config defaults
* Make duration handling on jetty shutdown more consistent
* StatisticsHandler is a wrapper
* Move jetty lifecycle error logging to error
The behavior is configurable through druid.extensions.useExtensionClassloaderFirst.
It is useful when extensions want to load a dependency different from one provided
by Druid, for example a different version of geoip or protobuf.
Also remove Guice annotations from LoadQueueTaskMaster, since it is
provided by CliCoordinator, so Guice does not need to know how to
build one directly.
* just renaming of SegmentChangeRequestHistory etc
* additional change history refactoring changes
* WorkerTaskManager a replica of WorkerTaskMonitor
* HttpServerInventoryView refactoring to extract sync code and robustification
* Introducing HttpRemoteTaskRunner
* Additional Worker side updates
* 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
* Make Errorprone the default compiler
* Address comments
* Make Error Prone's ClassCanBeStatic rule a error
* Preconditions allow only %s pattern
* Fix DruidCoordinatorBalancerTester
* Try to give the compiler more memory
* Remove distribution module activation on jdk 1.8 because only jdk 1.8 is used now
* Don't show compiler warnings
* Try different travis script
* Fix travis.yml
* Make Error Prone optional again
* For error-prone compiler
* Increase compiler's maxmem
* Don't run Error Prone for benchmarks because of OOM
* Skip install step in Travis
* Remove MetricHolder.writeToChannel()
* In travis.yml, check compilation before tests, because it may fail faster
* Add QueryPlus. Add QueryRunner.run(QueryPlus, Map) method with default implementation, to replace QueryRunner.run(Query, Map).
* Fix GroupByMergingQueryRunnerV2
* Fix QueryResourceTest
* Expand the comment to Query.run(walker, context)
* Remove legacy version of BySegmentSkippingQueryRunner.doRun()
* Add LegacyApiQueryRunnerTest and be more specific about legacy API removal plans in Druid 0.11 in Javadocs
* coordinator lookups mgmt improvements
* revert replaces removal, deprecate it instead
* convert and use older specs stored in db
* more tests and updates
* review comments
* add behavior for 0.10.0 to 0.9.2 downgrade
* incorporating more review comments
* remove explicit lock and use LifecycleLock in LookupReferencesManager. use LifecycleLock in LookupCoordinatorManager as well
* wip on LookupCoordinatorManager
* lifecycle lock
* refactor thread creation into utility method
* more review comments addressed
* support smooth roll back of lookup snapshots from 0.10.0 to 0.9.2
* correctly use LifecycleLock in LookupCoordinatorManager and remove synchronization from start/stop
* run lookup mgmt on leader coordinator only
* wip: changes to do multiple start() and stop() on LookupCoordinatorManager
* lifecycleLock fix usage in LookupReferencesManagerTest
* add LifecycleLock back
* fix license hdr
* some fixes
* make LookupReferencesManager.getAllLookupsState() consistent while still being lockless
* address review comments
* addressing leventov's comments
* address charle's comments
* add IOE.java
* for safety in LookupReferencesManager mainThread check for lifecycle started state on each loop in addition to interrupt
* move thread creation utility method to Execs
* fix names
* add tests for LookupCoordinatorManager.lookupManagementLoop()
* add further tests for figuring out toBeLoaded and toBeDropped on LookupCoordinatorManager
* address leventov comments
* remove LookupsStateWithMap and parameterize LookupsState
* address review comments
* address more review comments
* misc fixes
* RealtimeIndexTask to support alertTimeout in context and raise alert if task process exists after the timeout
* move alertTimeout config to tuningConfig and document
* In IndexMerger and IndexMergerV9, create temporary files under the output directory/tmpPeonFiles, instead of java.io.tmpdir
* Use FileUtils.forceMkdir() across the codebase and remove some unused code
* Fix test
* Fix PullDependencies.run()
* Unused import
* NN optimization for hdfs data segments.
* HdfsDataSegmentKiller, HdfsDataSegment finder changes to use new storage
format.Docs update.
* Common utility function in DataSegmentPusherUtil.
* new static method `makeSegmentOutputPathUptoVersionForHdfs` in JobHelper
* reuse getHdfsStorageDirUptoVersion in
DataSegmentPusherUtil.getHdfsStorageDir()
* Addressed comments.
* Review comments.
* HdfsDataSegmentKiller requested changes.
* extra newline
* Add maprfs.
* No more singleton. Reduce iterations
* Granularities
* Fix the delay in the test
* Add license header
* Remove unused imports
* Lot more unused imports from all the rearranging
* CR feedback
* Move javadoc to constructor
* Refactor Segment Granularity
* Beginning of one granularity
* Copy the fix for custom periods in segment-grunalrity over here.
* Remove the custom serialization for now.
* Compilation cleanup
* Reformat code
* Fixing unit tests
* Unify to use a single iterable
* Backward compatibility for rolling upgrade
* Minor check style. Cosmetic changes.
* Rename length and millis to duration
* CR feedback
* Minor changes.
* Require Java 8 and include some Java 8 dependencies.
- Upgrade Jetty to 9.3.16.v20170120.
- Upgrade DataSketches to 0.8.4.
- Bundle caffeine-cache by default.
- Still target Java 7 when compiling base Druid classes.
* Update cluster, quickstart docs.
* Remove oraclejdk7 from travis.yml.
* auto reset option for Kafka Indexing service in case message at the offset being fetched is not present anymore at kafka brokers
* review comments
* review comments
* reverted last change
* review comments
* review comments
* fix typo
* Allow users to specify additional command line args for creating tar balls
This PR allows users to specify additional command line options to the
pull deps command while creating druid distribution.
e.g. To also package graphite-emitter in druid tarball one can run -
mvn package -Ddruid.distribution.pulldeps.opts='-c
io.druid.extensions.contrib:graphite-emitter'
* Set default to --clean instead of blank value
* Add metrics for Query Count statistics
This PR adds a new metrics monitor “QueryCountStatsMonitor” which emits
three new metrics -
1) query/success/count - number of successful queries
2) query/failed/count - number of failed queries
3) query/interrupted/count - number of interrupted/timedout queries
fix bindings
* make fields final
* fix imports
* AsyncQueryForwardingServlet implement QueryStatsProvider
* remove unused import
* make sure CliCoordinator initializes and starts DerbyMetadataStorage first if configured
* Revert "make sure CliCoordinator initializes and starts DerbyMetadataStorage first if configured"
This reverts commit 54f5644054626d4a9e2448bb4bd5e6ce9a9fca1d.
* make sure CliCoordinator initializes and starts DerbyMetadataStorage first if configured
* Normalized Cost Balancer
* Adding documentation and renaming to use diskNormalizedCostBalancer
* Remove balancer from the strings
* Update docs and include random cost balancer
* Fix checkstyle issues
* support finding segments from a AWS S3 storage.
* add more Uts
* address comments and add a document for the feature.
* update docs indentation
* update docs indentation
* address comments.
1. add a Ut for json ser/deser for the config object.
2. more informant error message in a Ut.
* address comments.
1. use @Min to validate the configuration object
2. change updateDescriptor to a string as it does not take an argument otherwise
* fix a Ut failure - delete a Ut for testing default max length.
* Show candidate hosts for the given query
* Added test cases & minor changes to address comments
* Changed path-param to query-pram for intervals/numCandidates
1) Modify CliHadoopIndexer to share constant from `TaskConfig.DEFAULT_DEFAULT_HADOOP_COORDINATES`
2) add comment to pom.xml as discussed in
https://github.com/druid-io/druid/pull/3044
fix name
All query metrics now start with toolChest.makeMetricBuilder, and all of
*those* now start with DruidMetrics.makePartialQueryTimeMetric. Also, "id"
moved to common code, since all query metrics added it anyway.
In particular this will add query-type specific dimensions like "threshold"
and "numDimensions" to servlet-originated metrics like query/time.
* Datasource as lookup tier
* Adds an option to let indexing service tasks pull their lookup tier from the datasource they are working for.
* Fix bad docs for lookups lookupTier
* Add Datasource name holder
* Move task and datasource to be pulled from Task file
* Make LookupModule pull from bound dataSource
* Fix test
* Fix code style on imports
* Fix formatting
* Make naming better
* Address code comments about naming
Fixes#2682
IndexingService helpers are added according to the settings in runtime.properties.
Rather than having all the config.isXXX checks there, it makes sense to have a pluggable
approach for allowing the dynamic configuration to bring in implementations for helpers
without having to have hard-coded sets of available helpers. Plus, it will also make it possible for extensions to plug helpers in.
With https://github.com/druid-io/druid-api/pull/76, we could conditionally bind a helper to Coordinator's runlist.
The condition is driven by the value set in the runtime.properties.
Geared towards supporting transactional inserts of new segments. This involves an
interface "DataSourceMetadata" that allows combining of partially specified metadata
(useful for partitioned ingestion).
DataSource metadata is stored in a new "dataSource" table.
* Eliminate exclusion groups from pull-deps
* Only consider dependency nodes in pull-deps if they are not in the following scopes
* provided
* test
* system
* Fix a bunch of `<scope>provided</scope>` missing tags
* Better exclusions for a couple of problematic libs
See stack traces here, from current master: https://gist.github.com/gianm/bd9a66c826995f97fc8f
1. The thread "qtp925672150-62" holds the lock on InternalInjectorCreator.class,
used by Scopes.SINGLETON, and wants the lock on "handlers" in Lifecycle.addMaybeStartHandler
called by DiscoveryModule.getServiceAnnouncer.
2. The main thread holds the lock on "handlers" in Lifecycle.addMaybeStartHandler, which it
took because it's trying to add the ExecutorLifecycle to the lifecycle. main is trying
to get the InternalInjectorCreator.class lock because it's running ExecutorLifecycle.start,
which does some Jackson deserialization, and Jackson needs that lock in order to inject
stuff into the Task it's deserializing.
This patch eagerly instantiates ChatHandlerResource (which I believe is what's trying to
create the ServiceAnnouncer in the qtp925672150-62 jetty thread) and the ExecutorLifecycle.
* Moves last run task state information to Worker
* Makes WorkerTaskRunner a TaskRunner which has interfaces to help with getting information about a Worker
- Add druid.indexer.server.maxChatRequests, which sets up a QoSFilter on the main Jetty server.
- Deprecate druid.indexer.runner.separateIngestionEndpoint
- Deprecate druid.indexer.server.chathandler.*
* Don't put druid****selfcontained.jar at the end of the hadoop isolated classpath
* Add `<scope>provided</scope>` to prevent repeated dependency inclusion in the extension directories
Historical will drop a segment that shouldn't be dropped in the following scenario:
Historical node tried to load segmentA, but failed with SegmentLoadingException,
then ZkCoordinator called removeSegment(segmentA, blah) to schedule a runnable that would drop segmentA by deleting its files. Now, before that runnable executed, another LOAD request was sent to this historical, this time historical actually succeeded on loading segmentA and announced it. But later on, the scheduled drop-of-segment runnable started executing and removed the segment files, while historical is still announcing segmentA.
`insert-segment-to-db` is a tool that can insert segments into Druid metadata storage. It is intended to be used
to update the segment table in metadata storage after people manually migrate segments from one place to another.
It can also be used to insert missing segment into Druid, or even recover metadata storage by telling it where the
segments are stored.
Note: This tool expects users to have Druid cluster running in a "safe" mode, where there are no active tasks to interfere
the segments being inserted. Users can optionally bring down the cluster to make 100% sure nothing is interfering.
- fixes#1970
- extracted out segment handoff callbacks in SegmentHandoffNotifier
which is responsible for tracking segment handoffs and doing callbacks
when handoff is complete.
- Coordinator now maintains a view of segments in the cluster, this
will affect the jam heap requirements for the overlord for large
clusters.
realtime index task and nodes now use HTTP end points exposed by the
coordinator to get serverView
review comment
fix realtime node guide injection
review comments
make test not rely on scheduled exec
fix compilation
fix import
review comment
introduce immutableSegmentLoadInfo
fix son reading
remove unnecessary logging
This is a feature meant to allow realtime tasks to work without being told upfront
what shardSpec they should use (so we can potentially publish a variable number
of segments per interval).
The idea is that there is a "pendingSegments" table in the metadata store that
tracks allocated segments. Each one has a segment id (the same segment id we know
and love) and is also part of a sequence.
The sequences are an idea from @cheddar that offers a way of doing replication.
If there are N tasks reading exactly the same data with exactly the same logic
(think Kafka tasks reading a fixed range of offsets) then you can place them
in the same sequence, and they will generate the same sequence of segments.
1) Remove maven client from downloading extensions at runtime.
2) Provide a way to load Druid extensions and hadoop dependencies through file system.
3) Refactor pull-deps so that it can download extensions into extension directories.
4) Add documents on how to use this new extension loading mechanism.
5) Change the way how Druid tarball is generated. Now all the extensions + hadoop-client 2.3.0
are packaged within the Druid tarball.
- move assembly out of druid-services into a 'distribution' module
- create separate 'extensions-distribution' module and assembly to
package extensions and their dependencies into a local maven
repository
- include this extensions maven repository in the binaries tarball
review comments
more refactoring and cleaning of redundant code
add UT + docs + more refactoring
fixes + review comments
more cleanup
end points to fetch history
review comments
remove unnecessary changes
review comments rename header name
review comments + add test for MetadataRulesManager
review comments docs
- Uses method overrides instead of modified Jetty code, now that
ProxyServlet provides enough method hooks for proper overrides.
This means we may also benefit from any Jetty ProxyServlet fixes
- Adds test for async proxy servlet to make sure gzip encoding is
properly proxied.
- Router now proxies POST requests for requests that are not Druid
queries, by only treating /druid/v2/* endpoints as queries.