* 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.
* 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
* SQL: EARLIEST, LATEST aggregators.
I chose these names instead of FIRST, LAST because those are already
reserved functions in Calcite that mean something different. I think
these are also better names anyway.
* Finalify.
* SQL updates.
* Adjust aggregator calls.
* Validations, test updates.
* Review docs.
* 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
* remove select query
* thanks teamcity
* oops
* oops
* add back a SelectQuery class that throws RuntimeExceptions linking to docs
* adjust text
* update docs per review
* deprecated
* 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
* LoggingEmitter: print event as json
* use DefaultRequestLogEventBuilderFactory in emitting request logger by default
* print context in query metric as json
* removed unused jsonMapper from DefaultQueryMetrics
* add comment
* remove change to DefaultRequestLogEventBuilderFactory.java
* 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 IPv4 SQL functions
New SQL functions for filtering IPv4 addresses:
- IPV4_MATCH: Check if IP address belongs to a subnet
- IPV4_PARSE: Convert string IP address to integer
- IPV4_STRINGIFY: Convert integer IP address to string
These are the SQL analogs of the druid expressions with the same name.
Filtering is more efficient when operating on IP addresses as integers
instead of strings.
* Refactor operator conversions into named constants
* 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.
* 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
* 1. Added TimestampExtractExprMacro.Unit for MILLISECOND 2. expr eval for MILLISECOND 3. Added a test case to test extracting millisecond from expression. #7935
* 1. Adding DATASOURCE4 in tests. 2. Adding test TimeExtractWithMilliseconds
* Fixing testInformationSchemaTables test
* Fixing failing tests in DruidAvaticaHandlerTest
* Adding cannotVectorize() call before the test
* Extract time function - Adding support for MICROSECOND, ISODOW, ISOYEAR and CENTURY time units, documentation changes.
* Adding MILLISECOND in test case
* Adding support DECADE and MILLENNIUM, updating test case and documentation
* Fixing expression eval for DECADE and MILLENIUM
* add CachingClusteredClient benchmark, refactor some stuff
* revert WeightedServerSelectorStrategy to ConnectionCountServerSelectorStrategy and remove getWeight since felt artificial, default mergeResults in toolchest implementation for topn, search, select
* adjust javadoc
* adjustments
* oops
* use it
* use BinaryOperator, remove CombiningFunction, use Comparator instead of Ordering, other review adjustments
* rename createComparator to createResultComparator, fix typo, firstNonNull nullable parameters
* Benchmarks: New SqlBenchmark, add caching & vectorization to some others.
- Introduce a new SqlBenchmark geared towards benchmarking a wide
variety of SQL queries. Rename the old SqlBenchmark to
SqlVsNativeBenchmark.
- Add (optional) caching to SegmentGenerator to enable easier
benchmarking of larger segments.
- Add vectorization to FilteredAggregatorBenchmark and GroupByBenchmark.
* Query vectorization.
This patch includes vectorized timeseries and groupBy engines, as well
as some analogs of your favorite Druid classes:
- VectorCursor is like Cursor. (It comes from StorageAdapter.makeVectorCursor.)
- VectorColumnSelectorFactory is like ColumnSelectorFactory, and it has
methods to create analogs of the column selectors you know and love.
- VectorOffset and ReadableVectorOffset are like Offset and ReadableOffset.
- VectorAggregator is like BufferAggregator.
- VectorValueMatcher is like ValueMatcher.
There are some noticeable differences between vectorized and regular
execution:
- Unlike regular cursors, vector cursors do not understand time
granularity. They expect query engines to handle this on their own,
which a new VectorCursorGranularizer class helps with. This is to
avoid too much batch-splitting and to respect the fact that vector
selectors are somewhat more heavyweight than regular selectors.
- Unlike FilteredOffset, FilteredVectorOffset does not leverage indexes
for filters that might partially support them (like an OR of one
filter that supports indexing and another that doesn't). I'm not sure
that this behavior is desirable anyway (it is potentially too eager)
but, at any rate, it'd be better to harmonize it between the two
classes. Potentially they should both do some different thing that
is smarter than what either of them is doing right now.
- When vector cursors are created by QueryableIndexCursorSequenceBuilder,
they use a morphing binary-then-linear search to find their start and
end rows, rather than linear search.
Limitations in this patch are:
- Only timeseries and groupBy have vectorized engines.
- GroupBy doesn't handle multi-value dimensions yet.
- Vector cursors cannot handle virtual columns or descending order.
- Only some filters have vectorized matchers: "selector", "bound", "in",
"like", "regex", "search", "and", "or", and "not".
- Only some aggregators have vectorized implementations: "count",
"doubleSum", "floatSum", "longSum", "hyperUnique", and "filtered".
- Dimension specs other than "default" don't work yet (no extraction
functions or filtered dimension specs).
Currently, the testing strategy includes adding vectorization-enabled
tests to TimeseriesQueryRunnerTest, GroupByQueryRunnerTest,
GroupByTimeseriesQueryRunnerTest, CalciteQueryTest, and all of the
filtering tests that extend BaseFilterTest. In all of those classes,
there are some test cases that don't support vectorization. They are
marked by special function calls like "cannotVectorize" or "skipVectorize"
that tell the test harness to either expect an exception or to skip the
test case.
Testing should be expanded in the future -- a project in and of itself.
Related to #3011.
* WIP
* Adjustments for unused things.
* Adjust javadocs.
* DimensionDictionarySelector adjustments.
* Add "clone" to BatchIteratorAdapter.
* ValueMatcher javadocs.
* Fix benchmark.
* Fixups post-merge.
* Expect exception on testGroupByWithStringVirtualColumn for IncrementalIndex.
* BloomDimFilterSqlTest: Tag two non-vectorizable tests.
* Minor adjustments.
* Update surefire, bump up Xmx in Travis.
* Some more adjustments.
* Javadoc adjustments
* AggregatorAdapters adjustments.
* Additional comments.
* Remove switching search.
* Only missiles.
* more sql support for expression array functions
* prepend/slice
* doc fixes
* fix imports
* fix tests
* add null numeric expr for proper conversions between ExprEval and Expr and back to ExprEval
* re-arrange
* imports :(
* add append/prepend test
* 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
Recently we've been talking about using SegmentIds as map keys rather than
DataSegments, because its sense of equality is more well-defined. This is
a refactor that does this in the druid-sql module, which mostly involves
DruidSchema and some related classes. It should have no user-visible effects.
* SQL: Allow select-sort-project query shapes.
Fixes#7768.
Design changes:
- In PartialDruidQuery, allow projection after select + sort by removing
the SELECT_SORT query stage and instead allowing the SORT and
SORT_PROJECT stages to apply either after aggregation or after a plain
non-aggregating select. This is different from prior behavior, where
SORT and SORT_PROJECT were only considered valid after aggregation
stages. This logic change is in the "canAccept" method.
- In DruidQuery, represent either kind of sorting with a single "Sorting"
class (instead of DefaultLimitSpec). The Sorting class is still
convertible into a DefaultLimitSpec, but is also convertible into the
sorting parameters accepted by a Scan query.
- In DruidQuery, represent post-select and post-sorting projections with
a single "Projection" class. This obsoletes the SortProject and
SelectProjection classes, and simplifies the DruidQuery by allowing us
to move virtual-column and post-aggregator-creation logic into the
new Projection class.
- Split "DruidQuerySignature" into RowSignature and VirtualColumnRegistry.
This effectively means that instead of having mutable and immutable
versions of DruidQuerySignature, we instead of RowSignature (always
immutable) and VirtualColumnRegistry (always mutable, but sometimes
null). This change wasn't required, but IMO it this makes the logic
involving them easier to follow, and makes it more clear when the
virtual column registry is active and when it's not.
Other changes:
- ConvertBoundsToSelectors now just accepts a RowSignature, but we
use the VirtualColumnRegistry.getFullRowSignature() method to get
a signature that includes all columns, and therefore allows us to
simplify the logic (no need to special-case virtual columns).
- Add `__time` to the Scan column list if the query is ordering by time.
* Remove unused import.
* SQL: Allow NULLs in place of optional arguments in many functions.
Also adjust SQL docs to describe how to make time literals using
TIME_PARSE (which is now possible in a nicer way).
* Be less forbidden.
* SQL: Respect default timezone for TIME_PARSE and TIME_SHIFT.
They were inadvertently using UTC rather than the default timezone.
Also, harmonize how time functions handle their parameters.
* Fix tests
* Add another TIME_SHIFT test.
* update sys.servers table to show all servers
* update docs
* Fix integration test
* modify test query for batch integration test
* fix case in test queries
* make the server_type lowercase
* Apply suggestions from code review
Co-Authored-By: Himanshu <g.himanshu@gmail.com>
* Fix compilation from git suggestion
* fix unit test
This feature allows Calcite's Bindable interpreter to be bolted on
top of Druid queries and table scans. I think it should be removed for
a few reasons:
1. It is not recommended for production anyway, because it generates
unscalable query plans (e.g. it will plan a join into two table scans
and then try to do the entire join in memory on the broker).
2. It doesn't work with Druid-specific SQL functions, like TIME_FLOOR,
REGEXP_EXTRACT, APPROX_COUNT_DISTINCT, etc.
3. It makes the SQL planning code needlessly complicated.
With SQL coming out of experimental status soon, it's a good opportunity
to remove this feature.
* now with 100% more buffer
* there can be only 1
* simplify
* javadoc
* clean up unused test method
* fix exception message
* style
* why does style hate javadocs
* review stuff
* style :(
* Implement round function in druid-sql
* Return value according to the type of argument
* Fix codes for abnoraml inputs, updated math-expr.md
* Fix assert text
* Fix error messages and refactor codes
* Fix compile error, update sql.md, refactor codes and format tests
PR #6902 introduced the ability to use virtual columns for filters, but they
were being omitted from "scan" queries, so filters would refer to a null column
instead of the intended virtual column.
* 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
* Moved Scan Builder to Druids class and started on Scan Benchmark setup
* Need to form queries
* It runs.
* Stuff for time-ordered scan query
* Move ScanResultValue timestamp comparator to a separate class for testing
* Licensing stuff
* Change benchmark
* Remove todos
* Added TimestampComparator tests
* Change number of benchmark iterations
* Added time ordering to the scan benchmark
* Changed benchmark params
* More param changes
* Benchmark param change
* Made Jon's changes and removed TODOs
* Broke some long lines into two lines
* nit
* Decrease segment size for less memory usage
* Wrote tests for heapsort scan result values and fixed bug where iterator
wasn't returning elements in correct order
* Wrote more tests for scan result value sort
* Committing a param change to kick teamcity
* Fixed codestyle and forbidden API errors
* .
* Improved conciseness
* nit
* Created an error message for when someone tries to time order a result
set > threshold limit
* Set to spaces over tabs
* Fixing tests WIP
* Fixed failing calcite tests
* Kicking travis with change to benchmark param
* added all query types to scan benchmark
* Fixed benchmark queries
* Renamed sort function
* Added javadoc on ScanResultValueTimestampComparator
* Unused import
* Added more javadoc
* improved doc
* Removed unused import to satisfy PMD check
* Small changes
* Changes based on Gian's comments
* Fixed failing test due to null resultFormat
* Added config and get # of segments
* Set up time ordering strategy decision tree
* Refactor and pQueue works
* Cleanup
* Ordering is correct on n-way merge -> still need to batch events into
ScanResultValues
* WIP
* Sequence stuff is so dirty :(
* Fixed bug introduced by replacing deque with list
* Wrote docs
* Multi-historical setup works
* WIP
* Change so batching only occurs on broker for time-ordered scans
Restricted batching to broker for time-ordered queries and adjusted
tests
Formatting
Cleanup
* Fixed mistakes in merge
* Fixed failing tests
* Reset config
* Wrote tests and added Javadoc
* Nit-change on javadoc
* Checkstyle fix
* Improved test and appeased TeamCity
* Sorry, checkstyle
* Applied Jon's recommended changes
* Checkstyle fix
* Optimization
* Fixed tests
* Updated error message
* Added error message for UOE
* Renaming
* Finish rename
* Smarter limiting for pQueue method
* Optimized n-way merge strategy
* Rename segment limit -> segment partitions limit
* Added a bit of docs
* More comments
* Fix checkstyle and test
* Nit comment
* Fixed failing tests -> allow usage of all types of segment spec
* Fixed failing tests -> allow usage of all types of segment spec
* Revert "Fixed failing tests -> allow usage of all types of segment spec"
This reverts commit ec470288c7.
* Revert "Merge branch '6088-Time-Ordering-On-Scans-N-Way-Merge' of github.com:justinborromeo/incubator-druid into 6088-Time-Ordering-On-Scans-N-Way-Merge"
This reverts commit 57033f36df, reversing
changes made to 8f01d8dd16.
* Check type of segment spec before using for time ordering
* Fix bug in numRowsScanned
* Fix bug messing up count of rows
* Fix docs and flipped boolean in ScanQueryLimitRowIterator
* Refactor n-way merge
* Added test for n-way merge
* Refixed regression
* Checkstyle and doc update
* Modified sequence limit to accept longs and added test for long limits
* doc fix
* Implemented Clint's recommendations
* 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
* 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.
* refactor sql planning to re-use expression virtual columns when possible when constructing a DruidQuery, allowing virtual columns to be defined in filter expressions, and making resulting native druid queries more concise. also minor refactor of built-in sql aggregators to maximize code re-use
* fix it
* fix it in the right place
* fixup for base64 stuff
* fixup tests
* fix merge conflict on import order
* fixup
* fix imports
* fix tests
* review comments
* refactor
* re-arrange
* better javadoc
* fixup merge
* fixup tests
* fix accidental changes
* 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
* Fix the bug with num_rows in sys.segments
* Fix segmentMetadataInfo update in DruidSchema
* Add numRows to SegmentMetadataHolder builder's constructor, so it's not overwritten
* Rename SegSegmentSignature to setSegmentMetadataHolder and fix it so nested map is appended instead of recreated
* Replace Map<String, Set<String>> segmentServerMap with Set<String> for num_replica
* Remove unnecessary code and update test
* Add unit test for num_rows
* PR comments
* change access modifier to default package level
* minor changes to comments
* PR comments
* Moved Scan Builder to Druids class and started on Scan Benchmark setup
* Need to form queries
* It runs.
* Remove todos
* Change number of benchmark iterations
* Changed benchmark params
* More param changes
* Made Jon's changes and removed TODOs
* Broke some long lines into two lines
* Decrease segment size for less memory usage
* Committing a param change to kick teamcity
* Prohibit assigning concurrent maps into Map-types variables and fields; Fix a race condition in CoordinatorRuleManager; improve logic in DirectDruidClient and ResourcePool
* Enforce that if compute(), computeIfAbsent(), computeIfPresent() or merge() is called on a ConcurrentHashMap, it's stored in a ConcurrentHashMap-typed variable, not ConcurrentMap; add comments explaining get()-before-computeIfAbsent() optimization; refactor Counters; fix a race condition in Intialization.java
* Remove unnecessary comment
* Checkstyle
* Fix getFromExtensions()
* Add a reference to the comment about guarded computeIfAbsent() optimization; IdentityHashMap optimization
* Fix UriCacheGeneratorTest
* Workaround issue with MaterializedViewQueryQueryToolChest
* Strengthen Appenderator's contract regarding concurrency
* Add published segment cache in broker
* Change the DataSegment interner so it's not based on DataSEgment's equals only and size is preserved if set
* Added a trueEquals to DataSegment class
* Use separate interner for realtime and historical segments
* Remove trueEquals as it's not used anymore, change log message
* PR comments
* PR comments
* Fix tests
* PR comments
* Few more modification to
* change the coordinator api
* removeall segments at once from MetadataSegmentView in order to serve a more consistent view of published segments
* Change the poll behaviour to avoid multiple poll execution at same time
* minor changes
* PR comments
* PR comments
* Make the segment cache in broker off by default
* Added a config to PlannerConfig
* Moved MetadataSegmentView to sql module
* Add doc for new planner config
* Update documentation
* PR comments
* some more changes
* PR comments
* fix test
* remove unintentional change, whether to synchronize on lifecycleLock is still in discussion in PR
* minor changes
* some changes to initialization
* use pollPeriodInMS
* Add boolean cachePopulated to check if first poll succeeds
* Remove poll from start()
* take the log message out of condition in stop()
* add "bloom_filter_test" druid expression to support bloom filters in ExpressionVirtualColumn and ExpressionDimFilter and sql expressions
* more docs
* use java.util.Base64, doc fixes
* use SqlLifecyle to manage sql execution, add sqlId
* add sql request logger
* fix UT
* rename sqlId to sqlQueryId, sql/time to sqlQuery/time, etc
* add docs and more sql request logger impls
* add UT for http and jdbc
* fix forbidden use of com.google.common.base.Charsets
* fix UT in QuantileSqlAggregatorTest, supressed unused warning of getSqlQueryId
* do not use default method in QueryMetrics interface
* capitalize 'sql' everywhere in the non-property parts of the docs
* use RequestLogger interface to log sql query
* minor bugfixes and add switching request logger
* add filePattern configs for FileRequestLogger
* address review comments, adjust sql request log format
* fix inspection error
* try SuppressWarnings("RedundantThrows") to fix inspection error on ComposingRequestLoggerProvider
* Fix num_replicas count from sys.segments
* Adjust unit test for num_replica > 1
* Pass named arguments instead of passing boolean constants
* Address PR comments
* PR comments
* Use multi-guava version friendly direct executor implementation
* Don't use a singleton
* Fix strict compliation complaints
* Copy Guava's DirectExecutor
* Fix javadoc
* Imports are the devil
* Set is_available to false by default for published segment
* Address comments
Fix the is_published value for segments not in metadata store
* Remove unused import
* Use non-null sharSpec for a segment in test
* Fix checkstyle
* Modify comment
* 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.
* SQL: Add support for queries with project-after-semijoin.
These didn't work before, since the top Project rel wasn't getting
merged into the DruidSemiJoin rel. This patch allows that to happen.
* Null handling
* Null handling
* Null handling
* Add checkstyle rules about imports and empty lines between members
* Add suppressions
* Update Eclipse import order
* Add empty line
* Fix StatsDEmitter
* SystemSchema: Fix data types for various fields.
- segments: start, end, partition_num
- servers: plaintext_port, tls_port
- tasks: plaintext_port, tls_port
The declared and actual types did not match, but they must or
else queries may generate ClassCastExceptions.
Also adjusted some of the code for generating values to be more
robust in the face of nulls or malformed strings.
* Fix style.
* SQL: Set INFORMATION_SCHEMA catalog name to "druid".
Some third party tools ignore catalogs with empty names. So using
the name "druid" for the catalog makes integration easier.
* Update tests.
Also add a "fromIndex" argument to the strpos expression function. There
are some -1 and +1 adjustment terms due to the fact that the strpos
expression behaves like Java indexOf (0-indexed), but the POSITION SQL
function is 1-indexed.
* Use current coordinator leader instead of cached one (#6551)
Check the response status and throw exception if not OK
* Modify tests
* PR comment
* Add the correct check for status of BytesAccumulatingResponseHandler
* Move the status check into JsonParserIterator so sql query outputs meaningful message on failure
* Fix tests
* Prohibit some guava collection APIs and use JDK APIs directly
* reset files that changed by accident
* sort codestyle/druid-forbidden-apis.txt alphabetically
* SQL: Support for selecting multi-value dimensions.
Fixes#4637. Doesn't completely address everything mentioned in #4638,
but at least fixes one issue on the way there.
* Fix null cases in tests.
* Fix inconsistent segment size(#6448)
* Fix the segment size for published segments
* Changes to get numReplicas
* Make coordinator segments API truly streaming
* Changes to store partial segment data
* Simplify SegmentMetadataHolder
* Store partial the columns from available segments
* Address comments
* Added SystemSchema with following tables (#5989)
* SEGMENTS table provides details on served and published segments
* SERVERS table provides details on data servers
* SERVERSEGMETS table is the JOIN of SEGMENTS and SERVERS
* TASKS table provides details on tasks
* Add documentation for system schema
* Fix static-analysis warnings
* Address PR comments
*Add unit tests
* Fix a test
* Try to fix a test
* Fix a bug around replica count
* rename io.druid to org.apache.druid
* Major change is to make tasks and segment queries streaming
* Made tasks/segments stream to calcite instead of storing it in memory
* Add num_rows to segments table
* Refactor JsonParserIterator
* Replace with closeable iterator
* Fix docs, make num_rows column nullable, some unit test changes
* make num_rows column type long, allow it to be null
fix a compile error after merge, add TrafficCop param to InputStreamResponseHandler
* Filter null rows for segments table from Linq4j enumerable
* change num_replicas datatype to long in segments table
* Fix some tests and address comments
* Doc updates, other PR comments
* Update tests
* Address comments
* Add auth check
* Update docs
* Refactoring
* Fix teamcity warning, change the getQueryableServer in TimelineServerView
* Fix compilation after rebase
* Use the stream API from AuthorizationUtils
* Added LeaderClient interface and NoopDruidLeaderClient class
* Revert "Added LeaderClient interface and NoopDruidLeaderClient class"
This reverts commit 100fa46e39.
* Make the naming consistent to server_segments for the join table
* Add ForbiddenException on auth check failure
* Remove static block from SystemSchema
* Try to fix a test in CalciteQueryTest due to rename of server_segments
* Fix the json output format in the coordinator API
* Add auth check in the segments API
* Add null check to avoid NPE
* Use annonymous class object instead of mock for DruidLeaderClient in SqlBenchmark
* Fix test failures, type long/BIGINT can be nullable
* Revert long nullability to fix tests
* Fix style for tests
* PR comments
* Address PR comments
* Add the missing BytesAccumulatingResponseHandler class
* Use Sequences.withBaggage in DruidPlanner
* Fix docs, add comments
* Close the iterator if hasNext returns false
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`
* SQL: Update to Calcite 1.17.0.
Other than keeping things fresh, another motivation is that
this fixes CALCITE-1436 (AggregateNode NPE for aggregators other
than SUM/COUNT), which affects aggregate functions on our system
tables.
Also sets shouldConvertRaggedUnionTypesToVarying = true, a new
type system parameter that prefers VARCHAR over CHAR. This is
better for Druid, because we don't really have support for a
true CHAR type.
* Remove unused import.
The "Duplicate field name" check on inputRowSignature is too strict:
it is actually fine for a row signature to have the same field name
twice. It happens when the same expression is selected twice, and
both selections map to the same Druid object (dimension, aggregator,
etc).
I did not succeed in writing a test that triggers this, but I did see
it occur in production for a complex query with hundreds of aggregators.
* 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.
* add subtotalsSpec attribute to groupBy query
* dont sent subtotalsSpec to downstream nodes from broker and other updates
* address review comment
* fix checkstyle issues after merge to master
* add docs for subtotalsSpec feature
* address doc review comments
The old code assumes that post-aggregator prefixes are one character
long followed by numbers. This isn't always true (we may pad with
underscores to avoid conflicts). Instead, the new code uses a different
base prefix for sort-project postaggregators ("s" instead of "p") and
uses the usual Calcites.findUnusedPrefix function to avoid conflicts.
* SQL: Support more result formats, add columns header.
- Add result formats for line-based JSON and CSV.
- Add X-Druid-Sql-Columns header with a list of all columns that
the response will contain.
- Add more comprehensive documentation on what callers should expect
when making Druid SQL queries.
* Fix some tests.
* Adjust tests.
* Adjust trailer, add types header.
* Fix trailers.
* 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
False failures on Travis due to spurious timeout (in turn due to noisy
neighbors) is a bigger problem than legitimate failures taking too long
to time out. So it makes sense to extend timeouts.
* Adding decoration method to proxy servlet
Change-Id: I872f9282fb60bfa20524271535980a36a87b9621
* moving the proxy request decoration to authenticators
Change-Id: I7f94b9ff5ecf08e8abf7169b58bc410f33148448
* added docs
Change-Id: I901543e52f0faf4666bfea6256a7c05593b1ae70
* use the authentication result to decorate request
Change-Id: I052650de9cd02b4faefdbcdaf2332dd3b2966af5
* adding authenticated by name
Change-Id: I074d2933460165feeddb19352eac9bd0f96f42ca
* ensure that authenticator is not null
Change-Id: Idb58e308f90db88224a06f3759114872165b24f5
* fix types and minor bug
Change-Id: I6801d49a05d5d8324406fc0280286954eb66db10
* fix typo
Change-Id: I390b12af74f44d760d0812a519125fbf0df4e97b
* use actual type names
Change-Id: I62c3ee763363781e52809ec912aafd50b8486b8e
* set authenitcatedBy to null for AutheticationResults created by
Escalator.
Change-Id: I4a675c372f59ebd8a8d19c61b85a1e4bf227a8ba
* 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
Druid stores timestamps down to the millisecond, so we should use
precision = 3. Setting this wrong sometimes caused milliseconds
to be ignored in timestamp literals.
Fixes#5337.
Fixes test failures reported in -
https://github.com/druid-io/druid/issues/4909
Issue is that If some test skips setting up Calcite system properties
with proper encoding and loads calcite classes that use that property,
All subsequent tests in the same JVM fails.
To reproduce the issue - ExpressionsTest and CalciteQueryTest from IDE
in this order.
A better fix would be to not use System Properties in calcite, This
will work for now.
All new Calcite Unit tests that are added need to inherit
CalciteTestBase.
* Use both Joad Ids and Java IDs as Timezone to string readers
Change-Id: Ieb5c18559879f3f3a0104912ce2f0a354ad0aac3
* move the function to DateTimes and add org.joda.time.DateTimeZone#forID as part of forbidden api
Change-Id: Iff97fa044758019ed0c231587d10e31a9cc18da0
* exclude class and remove other usage
Change-Id: Ib458c2caaa1865535767e1009fbf017a92c8f615
* remove it from test classes
Change-Id: I9b576324f6c7e17a74bd8b13879232c9a8cd40b4
* remove unused
Change-Id: If1c5b70c26c2b7c83c20434cb72b2060653f5052
* timewarp and timezones
changes:
* `TimewarpOperator` will now compensate for daylight savings time shifts between date translation ranges for queries using a `PeriodGranularity` with a timezone defined
* introduces a new abstract query type `TimeBucketedQuery` for all queries which have a `Granularity` (100% not attached to this name). `GroupByQuery`, `SearchQuery`, `SelectQuery`, `TimeseriesQuery`, and `TopNQuery` all extend `TimeBucke
tedQuery`, cutting down on some duplicate code and providing a mechanism for `TimewarpOperator` (and anything else) that needs to be aware of granularity
* move precondition check to TimeBucketedQuery, add Granularities.nullToAll, add getTimezone to TimeBucketQuery
* formatting
* more formatting
* unused import
* changes:
* add 'getGranularity' and 'getTimezone' to 'Query' interface
* merge 'TimeBucketedQuery' into 'BaseQuery'
* fixup tests from resulting serialization changes
* dedupe
* fix after merge
* suppress warning
* 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
* SQL: Improve translation of time floor expressions.
The main change is to TimeFloorOperatorConversion.applyTimestampFloor.
- Prefer timestamp_floor expressions to timeFormat extractionFns, to
avoid turning things into strings when it isn't necessary.
- Collapse CAST(FLOOR(X TO Y) AS DATE) to FLOOR(X TO Y) if appropriate.
* Fix tests.
* ExpressionSelectors: Add caching selectors.
- SingleLongInputCaching selector for expressions on the __time column,
using a similar optimization to SingleScanTimeDimSelector
- SingleStringInputDimensionSelector for expressions on string columns
that return strings, using a similar optimization to ExtractionFn
based DimensionSelectors.
- SingleStringInputCaching selector for expressions on string columns
that return primitives.
Also, in the SQL planner, prefer expressions for time operations
rather than extractionFns.
* Code review comments.
i.e., aggregations like COUNT(DISTINCT CASE WHEN x THEN y END). This
patch also changes complex columns to report as nullable, which
is required for them to type-check properly when used in these kinds
of filtered aggregations.
* maxQueryTimeout property in runtime properties.
* extra line
* move withTimeoutAndMaxScatterGatherBytes method to QueryLifeCycle.
* Fix initialize method.
* remove unused import.
* doc update.
* some more details in doc about query failure..
* minor fix.
* decorating QueryRunner to set and verify context. Added by servers.
* remove whitespace.
* SQL: Improved behavior when implicitly casting strings to date/time literals.
- Handle all flavors of ISO8601 and SQL literals.
- Throw errors on other literals instead of silently transforming them to 0.
* Respect timeZone when format is null.
* Fix havingSpec on complex aggregators.
- Uses the technique from #4883 on DimFilterHavingSpec too.
- Also uses Transformers from #4890, necessitating a move of that and other
related classes from druid-server to druid-processing. They probably make
more sense there anyway.
- Adds a SQL query test.
Fixes#4957.
* Remove unused import.