* Add ability to wait for segment availability for batch jobs
* IT updates
* fix queries in legacy hadoop IT
* Fix broken indexing integration tests
* address an lgtm flag
* spell checker still flagging for hadoop doc. adding under that file header too
* fix compaction IT
* Updates to wait for availability method
* improve unit testing for patch
* fix bad indentation
* refactor waitForSegmentAvailability
* Fixes based off of review comments
* cleanup to get compile after merging with master
* fix failing test after previous logic update
* add back code that must have gotten deleted during conflict resolution
* update some logging code
* fixes to get compilation working after merge with master
* reset interrupt flag in catch block after code review pointed it out
* small changes following self-review
* fixup some issues brought on by merge with master
* small changes after review
* cleanup a little bit after merge with master
* Fix potential resource leak in AbstractBatchIndexTask
* syntax fix
* Add a Compcation TuningConfig type
* add docs stipulating the lack of support by Compaction tasks for the new config
* Fixup compilation errors after merge with master
* Remove erreneous newline
* JavaScript script engine support was removed in JDK 15: skip those tests for JDKs without it
* Fix flaky HTTP client tests with Java 15
* Switch from CMS to G1GC in integration tests, since CMS is no longer available in JDK 15
* DruidInputSource: Fix issues in column projection, timestamp handling.
DruidInputSource, DruidSegmentReader changes:
1) Remove "dimensions" and "metrics". They are not necessary, because we
can compute which columns we need to read based on what is going to
be used by the timestamp, transform, dimensions, and metrics.
2) Start using ColumnsFilter (see below) to decide which columns we need
to read.
3) Actually respect the "timestampSpec". Previously, it was ignored, and
the timestamp of the returned InputRows was set to the `__time` column
of the input datasource.
(1) and (2) together fix a bug in which the DruidInputSource would not
properly read columns that are used as inputs to a transformSpec.
(3) fixes a bug where the timestampSpec would be ignored if you attempted
to set the column to something other than `__time`.
(1) and (3) are breaking changes.
Web console changes:
1) Remove "Dimensions" and "Metrics" from the Druid input source.
2) Set timestampSpec to `{"column": "__time", "format": "millis"}` for
compatibility with the new behavior.
Other changes:
1) Add ColumnsFilter, a new class that allows input readers to determine
which columns they need to read. Currently, it's only used by the
DruidInputSource, but it could be used by other columnar input sources
in the future.
2) Add a ColumnsFilter to InputRowSchema.
3) Remove the metric names from InputRowSchema (they were unused).
4) Add InputRowSchemas.fromDataSchema method that computes the proper
ColumnsFilter for given timestamp, dimensions, transform, and metrics.
5) Add "getRequiredColumns" method to TransformSpec to support the above.
* Various fixups.
* Uncomment incorrectly commented lines.
* Move TransformSpecTest to the proper module.
* Add druid.indexer.task.ignoreTimestampSpecForDruidInputSource setting.
* Fix.
* Fix build.
* Checkstyle.
* Misc fixes.
* Fix test.
* Move config.
* Fix imports.
* Fixup.
* Fix ShuffleResourceTest.
* Add import.
* Smarter exclusions.
* Fixes based on tests.
Also, add TIME_COLUMN constant in the web console.
* Adjustments for tests.
* Reorder test data.
* Update docs.
* Update docs to say Druid 0.22.0 instead of 0.21.0.
* Fix test.
* Fix ITAutoCompactionTest.
* Changes from review & from merging.
* druid task auto scale based on kafka lag
* fix kafkaSupervisorIOConfig and KinesisSupervisorIOConfig
* druid task auto scale based on kafka lag
* fix kafkaSupervisorIOConfig and KinesisSupervisorIOConfig
* test dynamic auto scale done
* auto scale tasks tested on prd cluster
* auto scale tasks tested on prd cluster
* modify code style to solve 29055.10 29055.9 29055.17 29055.18 29055.19 29055.20
* rename test fiel function
* change codes and add docs based on capistrant reviewed
* midify test docs
* modify docs
* modify docs
* modify docs
* merge from master
* Extract the autoScale logic out of SeekableStreamSupervisor to minimize putting more stuff inside there && Make autoscaling algorithm configurable and scalable.
* fix ci failed
* revert msic.xml
* add uts to test autoscaler create && scale out/in and kafka ingest with scale enable
* add more uts
* fix inner class check
* add IT for kafka ingestion with autoscaler
* add new IT in groups=kafka-index named testKafkaIndexDataWithWithAutoscaler
* review change
* code review
* remove unused imports
* fix NLP
* fix docs and UTs
* revert misc.xml
* use jackson to build autoScaleConfig with default values
* add uts
* use jackson to init AutoScalerConfig in IOConfig instead of Map<>
* autoscalerConfig interface and provide a defaultAutoScalerConfig
* modify uts
* modify docs
* fix checkstyle
* revert misc.xml
* modify uts
* reviewed code change
* reviewed code change
* code reviewed
* code review
* log changed
* do StringUtils.encodeForFormat when create allocationExec
* code review && limit taskCountMax to partitionNumbers
* modify docs
* code review
Co-authored-by: yuezhang <yuezhang@freewheel.tv>
* add query granularity to compaction task
* fix checkstyle
* fix checkstyle
* fix test
* fix test
* add tests
* fix test
* fix test
* cleanup
* rename class
* fix test
* fix test
* add test
* fix test
* Granularity: Introduce primitive-typed bucketStart, increment methods.
Saves creation of unnecessary DateTime objects in timestamp_floor and
timestamp_ceil expressions.
* Fix style.
* Amp up the test coverage.
* Keep query granularity of compacted segments after compaction
* Protect against null isRollup
* Fix bugspot check RC_REF_COMPARISON_BAD_PRACTICE_BOOLEAN & edit an existing comment
* Make sure that NONE is also included when comparing for the finer granularity
* Update integration test check for segment size due to query granularity propagation affecting size
* Minor code cleanup
* Added functional test to verify queryGranlarity after compaction
* Minor style fix
* Update unit tests
* Support segmentGranularity for auto-compaction
* Support segmentGranularity for auto-compaction
* Support segmentGranularity for auto-compaction
* Support segmentGranularity for auto-compaction
* resolve conflict
* Support segmentGranularity for auto-compaction
* Support segmentGranularity for auto-compaction
* fix tests
* fix more tests
* fix checkstyle
* add unit tests
* fix checkstyle
* fix checkstyle
* fix checkstyle
* add unit tests
* add integration tests
* fix checkstyle
* fix checkstyle
* fix failing tests
* address comments
* address comments
* fix tests
* fix tests
* fix test
* fix test
* fix test
* fix test
* fix test
* fix test
* fix test
* fix test
* add offsetFetchPeriod to kinesis ingestion doc
* Remove jackson dependencies from extensions
* Use fixed delay for lag collection
* Metrics reset after finishing processing
* comments
* Broaden the list of exceptions to retry for
* Unit tests
* Add more tests
* Refactoring
* re-order metrics
* Doc suggestions
Co-authored-by: Charles Smith <38529548+techdocsmith@users.noreply.github.com>
* Add tests
Co-authored-by: Charles Smith <38529548+techdocsmith@users.noreply.github.com>
* Prevent interval materialization for UniformGranularitySpec inside the overlord
* Change API of bucketIntervals in GranularitySpec to return an Iterable<Interval>
* Javadoc update, respect inputIntervals contract
* Eliminate dependency on wrappedspec (i.e. ArbitraryGranularity) in UniformGranularitySpec
* Added one boundary condition test to UniformGranularityTest and fixed Travis forbidden method errors in IntervalsByGranularity
* Fix Travis style & other checks
* Refactor TreeSet to facilitate re-use in UniformGranularitySpec
* Make sure intervals are unique when there is no segment granularity
* Style/bugspot fixes...
* More travis checks
* Add condensedIntervals method to GranularitySpec and pass it as needed to the lock method
* Style & PR feedback
* Fixed failing test
* Fixed bug in IntervalsByGranularity iterator that it would return repeated elements (see added unit tests that were broken before this change)
* Refactor so that we can get the condensed buckets without materializing the intervals
* Get rid of GranularitySpec::condensedInputIntervals ... not needed
* Travis failures fixes
* Travis checkstyle fix
* Edited/added javadoc comments and a method name (code review feedback)
* Fixed jacoco coverage by moving class and adding more coverage
* Avoid materializing the condensed intervals when locking
* Deal with overlapping intervals
* Remove code and use library code instead
* Refactor intervals by granularity using the FluentIterable, add sanity checks
* Change !hasNext() to inputIntervals().isEmpty()
* Remove redundant lambda
* Use materialized intervals here since this is outside the overlord (for performance)
* Name refactor to reflect the fact that bucket intervals are sorted.
* Style fixes
* Removed redundant method and have condensedIntervalIterator throw IAE when element is null for consistency with other methods in this class (as well that null interval when condensing does not make sense)
* Remove forbidden api
* Move helper class inside common base class to reduce public space pollution
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* fix checkstyle
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* fix test
* fix test
* add log
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead
* address comments
* fix checkstyle
* fix checkstyle
* add config to skip overhead memory calculation
* add test for the skipBytesInMemoryOverheadCheck config
* add docs
* fix checkstyle
* fix checkstyle
* fix spelling
* address comments
* fix travis
* address comments
* ready to test
* tested on dev cluster
* tested
* code review
* add UTs
* add UTs
* ut passed
* ut passed
* opti imports
* done
* done
* fix checkstyle
* modify uts
* modify logs
* changing the package of SegmentLazyLoadFailCallback.java to org.apache.druid.segment
* merge from master
* modify import orders
* merge from master
* merge from master
* modify logs
* modify docs
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
* modify logs to rerun ci
Co-authored-by: yuezhang <yuezhang@freewheel.tv>
Today Kafka message support in streaming indexing tasks is limited to
message values, and does not provide a way to expose Kafka headers,
timestamps, or keys, which may be of interest to more specialized
Druid input formats. For instance, Kafka headers may be used to indicate
payload format/encoding or additional metadata, and timestamps are often
omitted from values in Kafka streams applications, since they are
included in the record.
This change proposes to introduce KafkaRecordEntity as InputEntity,
which would give input formats full access to the underlying Kafka record,
including headers, key, timestamps. It would also open access to low-level
information such as topic, partition, offset if needed.
KafkaEntity is a subclass of ByteEntity for backwards compatibility with
existing input formats, and to avoid introducing unnecessary complexity
for Kinesis indexing tasks.
* Remove redundant IncrementalIndex.Builder
* Parametrize incremental index tests and benchmarks
- Reveal and fix a bug in OffheapIncrementalIndex
* Fix forbiddenapis error: Forbidden method invocation: java.lang.String#format(java.lang.String,java.lang.Object[]) [Uses default locale]
* Fix Intellij errors: declared exception is never thrown
* Add documentation and validate before closing objects on tearDown.
* Add documentation to OffheapIncrementalIndexTestSpec
* Doc corrections and minor changes.
* Add logging for generated rows.
* Refactor new tests/benchmarks.
* Improve IncrementalIndexCreator documentation
* Add required tests for DataGenerator
* Revert "rollupOpportunity" to be a string
* Multiphase merge for IndexMergerV9
* JSON fix
* Cleanup temp files
* Docs
* Address logging and add IT
* Fix spelling and test unloader datasource name
* integration test for coordinator and overlord leadership, added sys.servers is_leader column
* docs
* remove not needed
* fix comments
* fix compile heh
* oof
* revert unintended
* fix tests, split out docker-compose file selection from starting cluster, use docker-compose down to stop cluster
* fixes
* style
* dang
* heh
* scripts are hard
* fix spelling
* fix thing that must not matter since was already wrong ip, log when test fails
* needs more heap
* fix merge
* less aggro
* Fixes and tests related to the Indexer process.
Three bugs fixed:
1) Indexers would not announce themselves as segment servers if they
did not have storage locations defined. This used to work, but was
broken in #9971. Fixed this by adding an "isSegmentServer" method
to ServerType and updating SegmentLoadDropHandler to always announce
if this method returns true.
2) Certain batch task types were written in a way that assumed "isReady"
would be called before "run", which is not guaranteed. In particular,
they relied on it in order to initialize "taskLockHelper". Fixed this
by updating AbstractBatchIndexTask to ensure "isReady" is called
before "run" for these tasks.
3) UnifiedIndexerAppenderatorsManager did not properly handle complex
datasources. Introduced DataSourceAnalysis in order to fix this.
Test changes:
1) Add a new "docker-compose.cli-indexer.yml" config that spins up an
Indexer instead of a MiddleManager.
2) Introduce a "USE_INDEXER" environment variable that determines if
docker-compose will start up an Indexer or a MiddleManager.
3) Duplicate all the jdk8 tests and run them in both MiddleManager and
Indexer mode.
4) Various adjustments to encourage fail-fast errors in the Docker
build scripts.
5) Various adjustments to speed up integration tests and reduce memory
usage.
6) Add another Mac-specific approach to determining a machine's own IP.
This was useful on my development machine.
7) Update segment-count check in ITCompactionTaskTest to eliminate a
race condition (it was looking for 6 segments, which only exist
together briefly, until the older 4 are marked unused).
Javadoc updates:
1) AbstractBatchIndexTask: Added javadocs to determineLockGranularityXXX
that make it clear when taskLockHelper will be initialized as a side
effect. (Related to the second bug above.)
2) Task: Clarified that "isReady" is not guaranteed to be called before
"run". It was already implied, but now it's explicit.
3) ZkCoordinator: Clarified deprecation message.
4) DataSegmentServerAnnouncer: Clarified deprecation message.
* Fix stop_cluster script.
* Fix sanity check in script.
* Fix hashbang lines.
* Test and doc adjustments.
* Additional tests, and adjustments for tests.
* Split ITs back out.
* Revert change to druid_coordinator_period_indexingPeriod.
* Set Indexer capacity to match MM.
* Bump up Historical memory.
* Bump down coordinator, overlord memory.
* Bump up Broker memory.
This is useful because it groups up segments for the same time chunk
into the same splits, which in turn is useful because it minimizes the
number of time chunks that each task will have to deal with.
* Two fixes related to encoding of % symbols.
1) TaskResourceFilter: Don't double-decode task ids. request.getPathSegments()
returns already-decoded strings. Applying StringUtils.urlDecode on
top of that causes erroneous behavior with '%' characters.
2) Update various ThreadFactoryBuilder name formats to escape '%'
characters. This fixes situations where substrings starting with '%'
are erroneously treated as format specifiers.
ITs are updated to include a '%' in extra.datasource.name.suffix.
* Avoid String.replace.
* Work around surefire bug.
* Fix xml encoding.
* Another try at the proper encoding.
* Give up on the emojis.
* Less ambitious testing.
* Fix an additional problem.
* Adjust encodeForFormat to return null if the input is null.
These caused certain APIs to not actually be properly forbidden.
Also removed two MoreExecutors entries for methods that don't exist in
our version of Guava.
* support multi-line text
* add test cases
* split json text into lines case by case
* improve exception handle
* fix CI
* use IntermediateRowParsingReader as base of JsonReader
* update doc
* ignore the non-immutable field in test case
* add more test cases
* mark `lineSplittable` as final
* fix testcases
* fix doc
* add a test case for SqlReader
* return all raw columns when exception occurs
* fix CI
* fix test cases
* resolve review comments
* handle ParseException returned by index.add
* apply Iterables.getOnlyElement
* fix CI
* fix test cases
* improve code in more graceful way
* fix test cases
* fix test cases
* add a test case to check multiple json string in one text block
* fix inspection check
* Introduce a Configurable Index Type
* Change to @UnstableApi
* Add AppendableIndexSpecTest
* Update doc
* Add spelling exception
* Add tests coverage
* Revert some of the changes to reduce diff
* Minor fixes
* Update getMaxBytesInMemoryOrDefault() comment
* Fix typo, remove redundant interface
* Remove off-heap spec (postponed to a later PR)
* Add javadocs to AppendableIndexSpec
* Describe testCreateTask()
* Add tests for AppendableIndexSpec within TuningConfig
* Modify hashCode() to conform with equals()
* Add comment where building incremental-index
* Add "EqualsVerifier" tests
* Revert some of the API back to AppenderatorConfig
* Don't use multi-line comments
* Remove knob documentation (deferred)
* Add shuffle metrics for parallel indexing
* javadoc and concurrency test
* concurrency
* fix javadoc
* Feature flag
* doc
* fix doc and add a test
* checkstyle
* add tests
* fix build and address comments
* Adding more worker metrics to Druid Overlord
* Changing the nomenclature from worker to peon as that represents the metrics that we want to monitor better
* Few more instance of worker usage replaced with peon
* Modifying the peon idle count logic to only use eligible workers available capacity
* Changing the naming to task slot count instead of peon
* Adding some unit test coverage for the new test runner apis
* Addressing Review Comments
* Modifying the TaskSlotCountStatsProvider apis so that overlords which are not leader do not emit these metrics
* Fixing the spelling issue in the docs
* Setting the annotation Nullable on the TaskSlotCountStatsProvider methods
* Store hash partition function in dataSegment and allow segment pruning only when hash partition function is provided
* query context
* fix tests; add more test
* javadoc
* docs and more tests
* remove default and hadoop tests
* consistent name and fix javadoc
* spelling and field name
* default function for partitionsSpec
* other comments
* address comments
* fix tests and spelling
* test
* doc
* Move tools for indexing to TaskToolbox instead of injecting them in constructor
* oops, other changes
* fix test
* unnecessary new file
* fix test
* fix build
* Fix handling of 'join' on top of 'union' datasources.
The problem is that unions are typically rewritten into a series of
individual queries on the underlying tables, but this isn't done when
the union is wrapped in a join.
The main changes are in UnionQueryRunner:
1) Replace an instanceof UnionQueryRunner check with DataSourceAnalysis.
2) Replace a "query.withDataSource" call with a new function, "Queries.withBaseDataSource".
Together, these enable UnionQueryRunner to "see through" a join.
* Tests.
* Adjust heap sizes for integration tests.
* Different approach, more tests.
* Tweak.
* Styling.
* Add support for all partitioing schemes for auto compaction
* annotate last compaction state for multi phase parallel indexing
* fix build and tests
* test
* better home
* better type tracking: add typed postaggs, finalized types for agg factories
* more javadoc
* adjustments
* transition to getTypeName to be used exclusively for complex types
* remove unused fn
* adjust
* more better
* rename getTypeName to getComplexTypeName
* setup expression post agg for type inference existing
* more javadocs
* fixup
* oops
* more test
* more test
* more comments/javadoc
* nulls
* explicitly handle only numeric and complex aggregators for incremental index
* checkstyle
* more tests
* adjust
* more tests to showcase difference in behavior
* timeseries longsum array
* Add "offset" parameter to the Scan query.
It works by doing the query as normal and then throwing away the first
"offset" number of rows on the broker.
* Fix constructor call.
* Fix up JSONs.
* Fix call to ScanQuery.
* Doc update.
* Fix javadocs.
* Spotbugs, LGTM suppressions.
* Javadocs.
* Fix suppression.
* Stabilize Scan query result order, add tests.
* Update LGTM comment.
* Fixup.
* Test different batch sizes too.
* Nicer tests.
* Fix comment.
* support unit suffix on byte-related properties
* add doc
* change default value of byte-related properites in example files
* fix coding style
* fix doc
* fix CI
* suppress spelling errors
* improve code according to comments
* rename Bytes to HumanReadableBytes
* add getBytesInInt to get value safely
* improve doc
* fix problem reported by CI
* fix problem reported by CI
* resolve code review comments
* improve error message
* improve code & doc according to comments
* fix CI problem
* improve doc
* suppress spelling check errors
* Add validation for authorizer name
* fix deps
* add javadocs
* Do not use resource filters
* Fix BasicAuthenticatorResource as well
* Add integration tests
* fix test
* fix
* Add availability and consistency docs.
Describes transactional ingestion and atomic replacement. Also, this patch
deletes some bad advice from the javadocs for SegmentTransactionalInsertAction.
* Fix missing word.
* QueryCountStatsMonitor can be injected in the Peon
This change fixes a dependency injection bug where there is a circular
dependency on getting the MonitorScheduler when a user configures the
QueryCountStatsMonitor to be used.
* fix tests
* Actually fix the tests this time
* Fix native batch range partition segment sizing
Fixes#10057.
Native batch range partitioning was only considering the partition
dimension value when grouping rows instead of using all of the row's
partition values. Thus, for schemas with multiple dimensions, the rollup
was overestimated, which would cause too many dimension values to be
packed into the same range partition. The resulting segments would then
be overly large (and not honor the target or max partition sizes).
Main changes:
- PartialDimensionDistributionTask: Consider all dimension values when
grouping row
- RangePartitionMultiPhaseParallelIndexingTest: Regression test by
having input with rows that should roll up and rows that should not
roll up
* Use hadoop & native hash ingestion row group key
* Fix missing temp dir for native single_dim
Native single dim indexing throws a file not found exception from
InputEntityIteratingReader.java:81. This MR creates the required
temporary directory when setting up the
PartialDimensionDistributionTask. The change was tested on a Druid
cluster. After installing the change native single_dim indexing
completes successfully.
* Fix indentation
* Use SinglePhaseSubTask as example for creating the temp dir
* Move temporary indexing dir creation in to TaskToolbox
* Remove unused dependency
Co-authored-by: Morri Feldman <morri@appsflyer.com>
* Fill in the core partition set size properly for batch ingestion with
dynamic partitioning
* incomplete javadoc
* Address comments
* fix tests
* fix json serde, add tests
* checkstyle
* Set core partition set size for hash-partitioned segments properly in
batch ingestion
* test for both parallel and single-threaded task
* unused variables
* fix test
* unused imports
* add hash/range buckets
* some test adjustment and missing json serde
* centralized partition id allocation in parallel and simple tasks
* remove string partition chunk
* revive string partition chunk
* fill numCorePartitions for hadoop
* clean up hash stuffs
* resolved todos
* javadocs
* Fix tests
* add more tests
* doc
* unused imports
* Allow append to existing datasources when dynamic partitioing is used
* fix test
* checkstyle
* checkstyle
* fix test
* fix test
* fix other tests..
* checkstyle
* hansle unknown core partitions size in overlord segment allocation
* fail to append when numCorePartitions is unknown
* log
* fix comment; rename to be more intuitive
* double append test
* cleanup complete(); add tests
* fix build
* add tests
* address comments
* checkstyle
* Fill in the core partition set size properly for batch ingestion with
dynamic partitioning
* incomplete javadoc
* Address comments
* fix tests
* fix json serde, add tests
* checkstyle
* Set core partition set size for hash-partitioned segments properly in
batch ingestion
* test for both parallel and single-threaded task
* unused variables
* fix test
* unused imports
* add hash/range buckets
* some test adjustment and missing json serde
* centralized partition id allocation in parallel and simple tasks
* remove string partition chunk
* revive string partition chunk
* fill numCorePartitions for hadoop
* clean up hash stuffs
* resolved todos
* javadocs
* Fix tests
* add more tests
* doc
* unused imports
* IntelliJ inspection and checkstyle rule for "Collection.EMPTY_* field accesses replaceable with Collections.empty*()"
* Reverted checkstyle rule
* Added tests to pass CI
* Codestyle
* move benchmark data generator into druid-processing, add a GeneratorInputSource to fill up a cluster with data
* newlines
* make test coverage not fail maybe
* remove useless test
* Update pom.xml
* Update GeneratorInputSourceTest.java
* less passive aggressive test names
* Empty partitionDimension has less rollup compared to the case when it is explicitly specified
* Adding a unit test for the empty partitionDimension scenario. Fixing another test which was failing
* Fixing CI Build Inspection Issue
* Addressing all review comments
* Updating the javadocs for the hash method in HashBasedNumberedShardSpec
This change removes ListenableFutures.transformAsync in favor of the
existing Guava Futures.transform implementation. Our own implementation
had a bug which did not fail the future if the applied function threw an
exception, resulting in the future never completing.
An attempt was made to fix this bug, however when running againts Guava's own
tests, our version failed another half dozen tests, so it was decided to not
continue down that path and scrap our own implementation.
Explanation for how was this bug manifested itself:
An exception thrown in BaseAppenderatorDriver.publishInBackground when
invoked via transformAsync in StreamAppenderatorDriver.publish will
cause the resulting future to never complete.
This explains why when encountering https://github.com/apache/druid/issues/9845
the task will never complete, forever waiting for the publishFuture to
register the handoff. As a result, the corresponding "Error while
publishing segments ..." message only gets logged once the index task
times out and is forcefully shutdown when the future is force-cancelled
by the executor.
* refactor SeekableStreamSupervisor usage of RecordSupplier to reduce contention between background threads and main thread, refactor KinesisRecordSupplier, refactor Kinesis lag metric collection and emitting
* fix style and test
* cleanup, refactor, javadocs, test
* fixes
* keep collecting current offsets and lag if unhealthy in background reporting thread
* review stuffs
* add comment
* add flag to flattenSpec to keep null columns
* remove changes to inputFormat interface
* add comment
* change comment message
* update web console e2e test
* move keepNullColmns to JSONParseSpec
* fix merge conflicts
* fix tests
* set keepNullColumns to false by default
* fix lgtm
* change Boolean to boolean, add keepNullColumns to hash, add tests for keepKeepNullColumns false + true with no nuulul columns
* Add equals verifier tests
* IntelliJ inspections cleanup
* Standard Charset object can be used
* Redundant Collection.addAll() call
* String literal concatenation missing whitespace
* Statement with empty body
* Redundant Collection operation
* StringBuilder can be replaced with String
* Type parameter hides visible type
* fix warnings in test code
* more test fixes
* remove string concatenation inspection error
* fix extra curly brace
* cleanup AzureTestUtils
* fix charsets for RangerAdminClient
* review comments
* WIP integration tests
* Add integration test for ingestion with transformSpec
* WIP almost working tests
* Add ignored tests
* checkstyle stuff
* remove newPage from index task ingestion spec
* more test cleanup
* still not quite working
* Actually disable the tests
* working tests
* fix codestyle
* dont use junit in integration tests
* actually fix the bug
* fix checkstyle
* bring index tests closer to reindex tests
* fix nullhandling exceptions related to test ordering
Tests might get executed in different order depending on the maven
version and the test environment. This may lead to "NullHandling module
not initialized" errors for some tests where we do not initialize
null-handling explicitly.
* use InitializedNullHandlingTest
* DruidSegmentReader should work if timestamp is specified as a dimension
* Add integration tests
Tests for compaction and re-indexing a datasource with the timestamp column
* Instructions to run integration tests against quickstart
* address pr
* add kinesis lag metric
* fixes
* heh
* do it right this time
* more test
* split out supervisor report lags into lagMillis, remove latest offsets from kinesis supervisor report since always null, review stuffs
* Use the iterator instead of higherKey(); use the iterator API instead of stream
* Fix tests; fix a concurrency bug in timeline
* fix test
* add tests for findNonOvershadowedObjectsInInterval
* fix test
* add missing tests; fix a bug in QueueEntry
* equals tests
* fix test
* Fix superbatch merge last partition boundaries
A bug in the computation for the last parallel merge partition could
cause an IndexOutOfBoundsException or precondition failure due to an
empty partition.
* Improve comments and tests
* Create splits of multiple files for parallel indexing
* fix wrong import and npe in test
* use the single file split in tests
* rename
* import order
* Remove specific local input source
* Update docs/ingestion/native-batch.md
Co-Authored-By: sthetland <steve.hetland@imply.io>
* Update docs/ingestion/native-batch.md
Co-Authored-By: sthetland <steve.hetland@imply.io>
* doc and error msg
* fix build
* fix a test and address comments
Co-authored-by: sthetland <steve.hetland@imply.io>
More functional tests to cover handling of input data that has a
partition dimension that contains:
1) Null values: Should be in first partition
2) Multi values: Should cause superbatch task to abort
* Codestyle - use java style array declaration
Replaced C-style array declarations with java style declarations and marked
the intelliJ inspection as an error
* cleanup test code
* Refactoring codes around ingestion:
- Parallel index task and simple task now use the same segment allocator implementation. This is reusable for the future implementation as well.
- Added PartitionAnalysis to store the analysis of the partitioning
- Move some util methods to SegmentLockHelper and rename it to TaskLockHelper
* fix build
* fix SingleDimensionShardSpecFactory
* optimize SingledimensionShardSpecFactory
* fix test
* shard spec builder
* import order
* shardSpecBuilder -> partialShardSpec
* build -> complete
* fix comment; add unit tests for partitionBoundaries
* add more tests and fix javadoc
* fix toString(); add serde tests for HashBasedNumberedPartialShardSpec and SegmentAllocateAction
* fix test
* add equality test for hash and range partial shard specs
* Forbid easily misused HashSet and HashMap constructors
* Add two LinkedHashMap constructors to forbidden-apis and create utility method as replacement for them
* Fix visibility of constant in CollectionUtils.java
* Make an exception for an instance of LinkedHashMap#<init>(int) because proper sizing is used
* revert changes to sql module tests that should be in separate PR
* Finish reverting changes to sql module tests that were flagged in checkstyle during CI
* Add netty dependency resulting from SupressForbidden
* intelliJ inspections cleanup
- remove redundant escapes
- performance warnings
- access static member via instance reference
- static method declared final
- inner class may be static
Most of these changes are aesthetic, however, they will allow inspections to
be enabled as part of CI checks going forward
The valuable changes in this delta are:
- using StringBuilder instead of string addition in a loop
indexing-hadoop/.../Utils.java
processing/.../ByteBufferMinMaxOffsetHeap.java
- Use class variables instead of static variables for parameterized test
processing/src/.../ScanQueryLimitRowIteratorTest.java
* Add intelliJ inspection warnings as errors to druid profile
* one more static inner class
* Reconcile terminology and method naming to 'used/unused segments'; Don't use terms 'enable/disable data source'; Rename MetadataSegmentManager to MetadataSegments; Make REST API methods which mark segments as used/unused to return server error instead of an empty response in case of error
* Fix brace
* Import order
* Rename withKillDataSourceWhitelist to withSpecificDataSourcesToKill
* Fix tests
* Fix tests by adding proper methods without interval parameters to IndexerMetadataStorageCoordinator instead of hacking with Intervals.ETERNITY
* More aligned names of DruidCoordinatorHelpers, rename several CoordinatorDynamicConfig parameters
* Rename ClientCompactTaskQuery to ClientCompactionTaskQuery for consistency with CompactionTask; ClientCompactQueryTuningConfig to ClientCompactionTaskQueryTuningConfig
* More variable and method renames
* Rename MetadataSegments to SegmentsMetadata
* Javadoc update
* Simplify SegmentsMetadata.getUnusedSegmentIntervals(), more javadocs
* Update Javadoc of VersionedIntervalTimeline.iterateAllObjects()
* Reorder imports
* Rename SegmentsMetadata.tryMark... methods to mark... and make them to return boolean and the numbers of segments changed and relay exceptions to callers
* Complete merge
* Add CollectionUtils.newTreeSet(); Refactor DruidCoordinatorRuntimeParams creation in tests
* Remove MetadataSegmentManager
* Rename millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments to leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments
* Fix tests, refactor DruidCluster creation in tests into DruidClusterBuilder
* Fix inspections
* Fix SQLMetadataSegmentManagerEmptyTest and rename it to SqlSegmentsMetadataEmptyTest
* Rename SegmentsAndMetadata to SegmentsAndCommitMetadata to reduce the similarity with SegmentsMetadata; Rename some methods
* Rename DruidCoordinatorHelper to CoordinatorDuty, refactor DruidCoordinator
* Unused import
* Optimize imports
* Rename IndexerSQLMetadataStorageCoordinator.getDataSourceMetadata() to retrieveDataSourceMetadata()
* Unused import
* Update terminology in datasource-view.tsx
* Fix label in datasource-view.spec.tsx.snap
* Fix lint errors in datasource-view.tsx
* Doc improvements
* Another attempt to please TSLint
* Another attempt to please TSLint
* Style fixes
* Fix IndexerSQLMetadataStorageCoordinator.createUsedSegmentsSqlQueryForIntervals() (wrong merge)
* Try to fix docs build issue
* Javadoc and spelling fixes
* Rename SegmentsMetadata to SegmentsMetadataManager, address other comments
* Address more comments
* Add JoinableFactory interface and use it in the query stack.
Also includes InlineJoinableFactory, which enables joining against
inline datasources. This is the first patch where a basic join query
actually works. It includes integration tests.
* Fix test issues.
* Adjustments from code review.
Builds on #9235, using the datasource analysis functionality to replace various ad-hoc
approaches. The most interesting changes are in ClientQuerySegmentWalker (brokers),
ServerManager (historicals), and SinkQuerySegmentWalker (indexing tasks).
Other changes related to improving how we analyze queries:
1) Changes TimelineServerView to return an Optional timeline, which I thought made
the analysis changes cleaner to implement.
2) Added QueryToolChest#canPerformSubquery, which is now used by query entry points to
determine whether it is safe to pass a subquery dataSource to the query toolchest.
Fixes an issue introduced in #5471 where subqueries under non-groupBy-typed queries
were silently ignored, since neither the query entry point nor the toolchest did
anything special with them.
3) Removes the QueryPlus.withQuerySegmentSpec method, which was mostly being used in
error-prone ways (ignoring any potential subqueries, and not verifying that the
underlying data source is actually a table). Replaces with a new function,
Queries.withSpecificSegments, that includes sanity checks.
* Add join-related DataSource types, and analysis functionality.
Builds on #9111 and implements the datasource analysis mentioned in #8728. Still can't
handle join datasources, but we're a step closer.
Join-related DataSource types:
1) Add "join", "lookup", and "inline" datasources.
2) Add "getChildren" and "withChildren" methods to DataSource, which will be used
in the future for query rewriting (e.g. inlining of subqueries).
DataSource analysis functionality:
1) Add DataSourceAnalysis class, which breaks down datasources into three components:
outer queries, a base datasource (left-most of the highest level left-leaning join
tree), and other joined-in leaf datasources (the right-hand branches of the
left-leaning join tree).
2) Add "isConcrete", "isGlobal", and "isCacheable" methods to DataSource in order to
support analysis.
Other notes:
1) Renamed DataSource#getNames to DataSource#getTableNames, which I think is clearer.
Also, made it a Set, so implementations don't need to worry about duplicates.
2) The addition of "isCacheable" should work around #8713, since UnionDataSource now
returns false for cacheability.
* Remove javadoc comment.
* Updates reflecting code review.
* Add comments.
* Add more comments.
Add more unit tests for range partition native batch parallel indexing.
Also, fix a bug where ParallelIndexPhaseRunner incorrectly thinks that
identical collected DimensionDistributionReports are not equal due to
not overriding equals() in DimensionDistributionReport.
* Doc update for new input source and input format.
- The input source and input format are promoted in all docs under docs/ingestion
- All input sources including core extension ones are located in docs/ingestion/native-batch.md
- All input formats and parsers including core extension ones are localted in docs/ingestion/data-formats.md
- New behavior of the parallel task with different partitionsSpecs are documented in docs/ingestion/native-batch.md
* parquet
* add warning for range partitioning with sequential mode
* hdfs + s3, gs
* add fs impl for gs
* address comments
* address comments
* gcs
* Optimize CachingLocalSegmentAllocator#getSequenceName
Replace StringUtils#format with string addition to generate the sequence
name for an interval and partition. This is faster because format uses a
Matcher under the covers to replace the string format with the variables.
* fix imports and add test
* Add comment about optimization
* Use renamed function for TaskToolbox
* Move tests after refactor
* Rename tests
* Fail superbatch range partition multi dim values
Change the behavior of parallel indexing range partitioning to fail
ingestion if any row had multiple values for the partition dimension.
After this change, the behavior matches that of hadoop indexing.
(Previously, rows with multiple dimension values would be skipped.)
* Improve err msg, rename method, rename test class
* HRTR: make pending task execution handling to go through all tasks on
not finding worker slots
* make HRTR methods package private that are meant to be used only in HttpRemoteTaskRunnerResource
* mark HttpRemoteTaskRunnerWorkItem.State global variables final
* hrtr: move immutableWorker NULL check outside of try-catch or finally block could have NPE
* add some explanatory comments
* add comment on explaining mechanics around hand off of pending tasks from submission to it getting picked up by a task execution thread
* fix spelling
* Parallel indexing single dim partitions
Implements single dimension range partitioning for native parallel batch
indexing as described in #8769. This initial version requires the
druid-datasketches extension to be loaded.
The algorithm has 5 phases that are orchestrated by the supervisor in
`ParallelIndexSupervisorTask#runRangePartitionMultiPhaseParallel()`.
These phases and the main classes involved are described below:
1) In parallel, determine the distribution of dimension values for each
input source split.
`PartialDimensionDistributionTask` uses `StringSketch` to generate
the approximate distribution of dimension values for each input
source split. If the rows are ungrouped,
`PartialDimensionDistributionTask.UngroupedRowDimensionValueFilter`
uses a Bloom filter to skip rows that would be grouped. The final
distribution is sent back to the supervisor via
`DimensionDistributionReport`.
2) The range partitions are determined.
In `ParallelIndexSupervisorTask#determineAllRangePartitions()`, the
supervisor uses `StringSketchMerger` to merge the individual
`StringSketch`es created in the preceding phase. The merged sketch is
then used to create the range partitions.
3) In parallel, generate partial range-partitioned segments.
`PartialRangeSegmentGenerateTask` uses the range partitions
determined in the preceding phase and
`RangePartitionCachingLocalSegmentAllocator` to generate
`SingleDimensionShardSpec`s. The partition information is sent back
to the supervisor via `GeneratedGenericPartitionsReport`.
4) The partial range segments are grouped.
In `ParallelIndexSupervisorTask#groupGenericPartitionLocationsPerPartition()`,
the supervisor creates the `PartialGenericSegmentMergeIOConfig`s
necessary for the next phase.
5) In parallel, merge partial range-partitioned segments.
`PartialGenericSegmentMergeTask` uses `GenericPartitionLocation` to
retrieve the partial range-partitioned segments generated earlier and
then merges and publishes them.
* Fix dependencies & forbidden apis
* Fixes for integration test
* Address review comments
* Fix docs, strict compile, sketch check, rollup check
* Fix first shard spec, partition serde, single subtask
* Fix first partition check in test
* Misc rewording/refactoring to address code review
* Fix doc link
* Split batch index integration test
* Do not run parallel-batch-index twice
* Adjust last partition
* Split ITParallelIndexTest to reduce runtime
* Rename test class
* Allow null values in range partitions
* Indicate which phase failed
* Improve asserts in tests
* Address security vulnerabilities CVSS >= 7
Update dependencies to address security vulnerabilities with CVSS scores
of 7 or higher. A new Travis CI job is added to prevent new
high/critical security vulnerabilities from being added.
Updated dependencies:
- api-util 1.0.0 -> 1.0.3
- jackson 2.9.10 -> 2.10.1
- kafka 2.1.0 -> 2.1.1
- libthrift 0.10.0 -> 0.13.0
- protobuf 3.2.0 -> 3.11.0
The following high/critical security vulnerabilities are currently
suppressed (so that the new Travis CI job can be added now) and are left
as future work to fix:
- hibernate-validator:5.2.5
- jackson-mapper-asl:1.9.13
- libthrift:0.6.1
- netty:3.10.6
- nimbus-jose-jwt:4.41.1
* Rename EDL1 license file
* Fix inspection errors
* Support orc format for native batch ingestion
* fix pom and remove wrong comment
* fix unnecessary condition check
* use flatMap back to handle exception properly
* move exceptionThrowingIterator to intermediateRowParsingReader
* runtime
* Fix the potential race SplittableInputSource.getNumSplits() and SplittableInputSource.createSplits() in TaskMonitor
* Fix docs and javadoc
* Add unit tests for large or small estimated num splits
* add override
* 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.
* add TsvInputFormat
* refactor code
* fix grammar
* use enum replace string literal
* code refactor
* code refactor
* mark abstract for base class meant not to be instantiated
* remove constructor for test
* Refactor parallel indexing perfect rollup partitioning
Refactoring to make it easier to later add range partitioning for
perfect rollup parallel indexing. This is accomplished by adding several
new base classes (e.g., PerfectRollupWorkerTask) and new classes for
encapsulating logic that needs to be changed for different partitioning
strategies (e.g., IndexTaskInputRowIteratorBuilder).
The code is functionally equivalent to before except for the following
small behavior changes:
1) PartialSegmentMergeTask: Previously, this task had a priority of
DEFAULT_TASK_PRIORITY. It now has a priority of
DEFAULT_BATCH_INDEX_TASK_PRIORITY (via the new PerfectRollupWorkerTask
base class), since it is a batch index task.
2) ParallelIndexPhaseRunner: A decorator was added to
subTaskSpecIterator to ensure the subtasks are generated with unique
ids. Previously, only tests (i.e., MultiPhaseParallelIndexingTest)
would have this decorator, but this behavior is desired for non-test
code as well.
* Fix forbidden apis and pmd warnings
* Fix analyze dependencies warnings
* Fix IndexTask json and add IT diags
* Fix parallel index supervisor<->worker serde
* Fix TeamCity inspection errors/warnings
* Fix TeamCity inspection errors/warnings again
* Integrate changes with those from #8823
* Address review comments
* Address more review comments
* Fix forbidden apis
* Address more review comments
* 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.
* Use earliest offset on kafka newly discovered partitions
* resolve conflicts
* remove redundant check cases
* simplified unit tests
* change test case
* rewrite comments
* add regression test
* add junit ignore annotation
* minor modifications
* indent
* override testableKafkaSupervisor and KafkaRecordSupplier to make the test runable
* modified test constructor of kafkaRecordSupplier
* simplify
* delegated constructor
* transformSpec + array expressions
changes:
* added array expression support to transformSpec
* removed ParseSpec.verify since its only use afaict was preventing transform expr that did not replace their input from functioning
* hijacked index task test to test changes
* remove docs about being unsupported
* re-arrange test assert
* unused imports
* imports
* fix tests
* preserve types
* suppress warning, fixes, add test
* formatting
* cleanup
* better list to array type conversion and tests
* fix oops
* 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
* Add option lateMessageRejectionStartDate
* Use option lateMessageRejectionStartDate
* Fix tests
* Add lateMessageRejectionStartDate to kafka indexing service
* Update tests kafka indexing service
* Fix tests for KafkaSupervisorTest
* Add lateMessageRejectionStartDate to KinesisSupervisorIOConfig
* Fix var name
* Update documentation
* Add check lateMessageRejectionStartDateTime and lateMessageRejectionPeriod, fails if both were specified.
* 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
* IOConfig for compaction task
* add javadoc, doc, unit test
* fix webconsole test
* add spelling
* address comments
* fix build and test
* address comments
* Make more package EverythingIsNonnullByDefault by default
* Fixed additional voilations after pulling in master
* Change iterator to list.addAll
* Fix annotations
* get active task by datasource when supervisor discover tasks
* fix ut
* fix ut
* fix ut
* remove unnecessary condition check
* fix ut
* remove stream in hot loop
* Added live reports for Kafka and Native batch task
* Removed unused local variables
* Added the missing unit test
* Refine unit test logic, add implementation for HttpRemoteTaskRunner
* checksytle fixes
* Update doc descriptions for updated API
* remove unnecessary files
* Fix spellcheck complaints
* More details for api descriptions
* Adjust defaults for hashed partitioning
If neither the partition size nor the number of shards are specified,
default to partitions of 5,000,000 rows (similar to the behavior of
dynamic partitions). Previously, both could be null and cause incorrect
behavior.
Specifying both a partition size and a number of shards now results in
an error instead of ignoring the partition size in favor of using the
number of shards. This is a behavior change that makes it more apparent
to the user that only one of the two properties will be honored
(previously, a message was just logged when the specified partition size
was ignored).
* Fix test
* Handle -1 as null
* Add -1 as null tests for single dim partitioning
* Simplify logic to handle -1 as null
* Address review comments
* 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
* Fallback to parsing classpath for hadoop task in Java 9+
In Java 9 and above we cannot assume that the system classloader is an
instance of URLClassLoader. This change adds a fallback method to parse
the system classpath in that case, and adds a unit test to validate it matches
what JDK8 would do.
Note: This has not been tested in an actual hadoop setup, so this is mostly
to help us pass unit tests.
* Remove granularity test of dubious value
One of our granularity tests relies on system classloader being a URLClassLoaders to
catch a bug related to class initialization and static initializers using a subclass (see
#2979)
This test was added to catch a potential regression, but it assumes we would add back
the same type of static initializers to this specific class, so it seems to be of dubious value
as a unit test and mostly serves to illustrate the bug.
relates to #5589
* 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
* Add group_id to overlord tasks API and sys.tasks table
* adjust test
* modify docs
* Make groupId nullable
* fix integration test
* fix toString
* Remove groupId from TaskInfo
* Modify docs and tests
* modify TaskMonitorTest
* Add TaskResourceCleaner; fix a couple of concurrency bugs in batch tasks
* kill runner when it's ready
* add comment
* kill run thread
* fix test
* Take closeable out of Appenderator
* add javadoc
* fix test
* fix test
* update javadoc
* add javadoc about killed task
* address comment
* Add support for parallel native indexing with shuffle for perfect rollup.
* Add comment about volatiles
* fix test
* fix test
* handling missing exceptions
* more clear javadoc for stopGracefully
* unused import
* update javadoc
* Add missing statement in javadoc
* address comments; fix doc
* add javadoc for isGuaranteedRollup
* Rename confusing variable name and fix typos
* fix typos; move fetch() to a better home; fix the expiration time
* add support https
* Fix race between canHandle() and addSegment() in StorageLocation
* add comment
* Add shuffleSegmentPusher which is a dataSegmentPusher used for writing shuffle data in local storage.
* add comments
* unused import
* add comments
* fix test
* address comments
* remove <p> tag from javadoc
* address comments
* comparingLong
* Address comments
* fix test
* 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
* GroupBy array-based result rows.
Fixes#8118; see that proposal for details.
Other than the GroupBy changes, the main other "interesting" classes are:
- ResultRow: The array-based result type.
- BaseQuery: T is no longer required to be Comparable.
- QueryToolChest: Adds "decorateObjectMapper" to enable query-aware serialization
and deserialization of result rows (necessary due to their positional nature).
- QueryResource: Uses the new decoration functionality.
- DirectDruidClient: Also uses the new decoration functionality.
- QueryMaker (in Druid SQL): Modifications to read ResultRows.
These classes weren't changed, but got some new javadocs:
- BySegmentQueryRunner
- FinalizeResultsQueryRunner
- Query
* Adjustments for TC stuff.
* Use partitionsSpec for all task types
* fix doc
* fix typos and revert to use isPushRequired
* address comments
* move partitionsSpec to core
* remove hadoopPartitionsSpec
* 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
* 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
* 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
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.
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)
* 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..
* 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
* 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
* Add checkstyle for "Local variable names shouldn't start with capital"
* Adjust some local variables to constants
* Replace StringUtils.LINE_SEPARATOR with System.lineSeparator()
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.
* 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
* 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
* 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
* 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
* 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
* 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
* 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
* 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
* 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.
* 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.
* 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
* 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
* 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.
* 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
* 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
* 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
* 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
* 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
* Improper equals override is fixed to prevent NullPointerException
* Fixed curly brace indentation.
* Test method is added for equals method of TaskLockPosse class.
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.
* 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
* 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
* 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
* 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
* 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