* Add FileUtils.createTempDir() and enforce its usage.
The purpose of this is to improve error messages. Previously, the error
message on a nonexistent or unwritable temp directory would be
"Failed to create directory within 10,000 attempts".
* Further updates.
* Another update.
* Remove commons-io from benchmark.
* Fix tests.
* Tidy up lifecycle, query, and ingestion logging.
The goal of this patch is to improve the clarity and usefulness of
Druid's logging for cluster operators. For more information, see
https://twitter.com/cowtowncoder/status/1195469299814555648.
Concretely, this patch does the following:
- Changes a lot of INFO logs to DEBUG, and DEBUG to TRACE, with the
goal of reducing redundancy and improving clarity by avoiding
showing rarely-useful log messages. This includes most "starting"
and "stopping" messages, and most messages related to individual
columns.
- Adds new log4j2 templates that show operators how to enabled DEBUG
logging for certain important packages.
- Eliminate stack traces for query errors, unless log level is DEBUG
or more. This is useful because query errors often indicate user
error rather than system error, but dumping stack trace often gave
operators the impression that there was a system failure.
- Adds task id to Appenderator, AppenderatorDriver thread names. In
the default log4j2 configuration, this will put them in log lines
as well. It's very useful if a user is using the Indexer, where
multiple tasks run in the same JVM.
- More consistent terminology when it comes to "sequences" (sets of
segments that are handed-off together by Kafka ingestion) and
"offsets" (cursors in partitions). These terms had been confused in
some log messages due to the fact that Kinesis calls offsets
"sequence numbers".
- Replaces some ugly toString calls with either the JSONification or
something more operator-accessible (like a URL or segment identifier,
instead of JSON object representing the same).
* Adjustments.
* Adjust integration test.
* first steps
* clean licenses
* fix capabilities
* fix specs
* more tests
* new web console on coordinator and overlord, remove setup for old consoles, old configs
* better message
* update licenses
* sync license files
* more button
* fix tslint issue
* jetty-rewrite dependency to add redirects for old console paths
* put dependency in the right place
* fix overlord detection
* fix notices, dedupe licenses
* make segment timeline work in no SQL mode
* update license
* revert hard coded coordinator mode from testing
* update restricted mode copy
* IndexerSQLMetadataStorageCoordinator.getTimelineForIntervalsWithHandle() don't fetch abutting intervals; simplify getUsedSegmentsForIntervals()
* Add VersionedIntervalTimeline.findNonOvershadowedObjectsInInterval() method; Propagate the decision about whether only visible segmetns or visible and overshadowed segments should be returned from IndexerMetadataStorageCoordinator's methods to the user logic; Rename SegmentListUsedAction to RetrieveUsedSegmentsAction, SegmetnListUnusedAction to RetrieveUnusedSegmentsAction, and UsedSegmentLister to UsedSegmentsRetriever
* Fix tests
* More fixes
* Add javadoc notes about returning Collection instead of Set. Add JacksonUtils.readValue() to reduce boilerplate code
* Fix KinesisIndexTaskTest, factor out common parts from KinesisIndexTaskTest and KafkaIndexTaskTest into SeekableStreamIndexTaskTestBase
* More test fixes
* More test fixes
* Add a comment to VersionedIntervalTimelineTestBase
* Fix tests
* Set DataSegment.size(0) in more tests
* Specify DataSegment.size(0) in more places in tests
* Fix more tests
* Fix DruidSchemaTest
* Set DataSegment's size in more tests and benchmarks
* Fix HdfsDataSegmentPusherTest
* Doc changes addressing comments
* Extended doc for visibility
* Typo
* Typo 2
* Address comment
* Support assign tasks to run on different tiers of MiddleManagers
* address comments
* address comments
* rename tier to category and docs
* doc
* fix doc
* fix spelling errors
* docs
* Stateful auto compaction
* javaodc
* add removed test back
* fix test
* adding indexSpec to compactionState
* fix build
* add lastCompactionState
* address comments
* extract CompactionState
* fix doc
* fix build and test
* Add a task context to store compaction state; add javadoc
* fix it test
* Fix dependency analyze warnings
Update the maven dependency plugin to the latest version and fix all
warnings for unused declared and used undeclared dependencies in the
compile scope. Added new travis job to add the check to CI. Also fixed
some source code files to use the correct packages for their imports and
updated druid-forbidden-apis to prevent regressions.
* Address review comments
* Adjust scope for org.glassfish.jaxb:jaxb-runtime
* Fix dependencies for hdfs-storage
* Consolidate netty4 versions
* check ctyle for constant field name
* check ctyle for constant field name
* check ctyle for constant field name
* check ctyle for constant field name
* check ctyle for constant field name
* check ctyle for constant field name
* check ctyle for constant field name
* check ctyle for constant field name
* check ctyle for constant field name
* merging with upstream
* review-1
* unknow changes
* unknow changes
* review-2
* merging with master
* review-2 1 changes
* review changes-2 2
* bug fix
* handle exception thrown while trying to call sun.misc.VM.maxDirectMemory() which is not available in Java 11
* fixup String.format -> StringUtils.format
* Add a cluster-wide configuration to force timeChunk lock and add a doc for segment locking
* add more test
* javadoc for missingIntervalsInOverwriteMode
* Fix test
* Address comments
* avoid spotbugs
* Fix dependency analyze warnings
Update the maven dependency plugin to the latest version and fix all
warnings for unused declared and used undeclared dependencies in the
compile scope. Added new travis job to add the check to CI. Also fixed
some source code files to use the correct packages for their imports.
* Fix licenses and dependencies
* Fix licenses and dependencies again
* Fix integration test dependency
* Address review comments
* Fix unit test dependencies
* Fix integration test dependency
* Fix integration test dependency again
* Fix integration test dependency third time
* Fix integration test dependency fourth time
* Fix compile error
* Fix assert package
* 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
* 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
This includes the router, overlord, middleManager, and coordinator.
Does the following things:
- Loads LookupSerdeModule on MM, overlord, and coordinator.
- Adds LookupExprMacro to LookupSerdeModule, which allows these node
types to understand that the 'lookup' function exists.
- Adds a test to make sure that LookupSerdeModule works for virtual
columns, filters, transforms, and dimension specs.
This is implementing the technique discussed on these two issues:
- https://github.com/apache/incubator-druid/issues/7724#issuecomment-494723333
- https://github.com/apache/incubator-druid/pull/7082#discussion_r264888771
* 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
* V1 - improve parallelism of zookeeper based segment change processing
* Create zk nodes in batches. Address code review comments.
Introduce various configs.
* Add documentation for the newly added configs
* Fix test failures
* Fix more test failures
* Remove prinstacktrace statements
* Address code review comments
* Use a single queue
* Address code review comments
Since we have a separate load peon for every historical, just having a single SegmentChangeProcessor
task per historical is enough. This commit also gets rid of the associated config druid.coordinator.loadqueuepeon.curator.numCreateThreads
* Resolve merge conflict
* Fix compilation failure
* Remove batching since we already have a dynamic config maxSegmentsInNodeLoadingQueue that provides that control
* Fix NPE in test
* Remove documentation for configs that are no longer needed
* Address code review comments
* Address more code review comments
* Fix checkstyle issue
* Address code review comments
* Code review comments
* Add back monitor node remove executor
* Cleanup code to isolate null checks and minor refactoring
* Change param name since it conflicts with member variable name
* 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
* refactor lookups to be more chill to router
* remove accidental change
* fix and combine LookupIntrospectionResourceTest
* fix inspection
* rename RouterLookupModule to LookupSerdeModule and RouterLookupExtractorFactoryContainerProvider to NoopLookupExtractorFactoryContainerProvider
* make comment generic
* use ConfigResourceFilter instead of StateResourceFilter
* fix indentation
* unused import
* another unused import
* refactor some stuff into processing module, split up LookupModule.java classes into their own files
* 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.
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
* 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
* 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
* 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.
* 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
Not putting this to 0.13 milestone because the found bugs are not critical (one is a harmless DI config duplicate, and another is in a benchmark.
Change in `DumpSegment` is just an indentation change.
* Add checkstyle rules about imports and empty lines between members
* Add suppressions
* Update Eclipse import order
* Add empty line
* Fix StatsDEmitter
* Prohibit some guava collection APIs and use JDK APIs directly
* reset files that changed by accident
* sort codestyle/druid-forbidden-apis.txt alphabetically
* 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