* Add metrics to the native queries underpinning SQL.
This is done by factoring out the metrics and request log emitting
code from QueryResource into a new QueryLifecycle class. That class
is used by both QueryResource and the SQL DruidSchema and QueryMaker.
Also fixes a couple of bugs in QueryResource:
- RequestLogLine start time was set to `TimeUnit.NANOSECONDS.toMillis(startNs)`,
which is incorrect since absolute nanos cannot be converted to millis.
- DruidMetrics.makeRequestMetrics was called with null `query` on
unparseable queries, which led to spurious "Unable to log query"
errors.
Partial fix for #4047.
* Code style
* Remove unused imports.
* Fix tests.
* Remove unused import.
* Fixes and improvements to SQL metadata caching.
Also adds support for MultipleSpecificSegmentSpec to CachingClusteredClient.
SQL changes:
- Cache metadata on a per-segment level, in addition to per-dataSource, so
we don't need to re-query all segments whenever a single new one appears.
This should lower the load placed on the cluster by metadata queries.
- Fix race condition in DruidSchema that can cause us to miss metadata. It was
possible to notice new segments, then issue a query, and have that query
not actually hit those segments, and not notice that it didn't hit those segments.
Then, the metadata from those segments would be ignored.
- Fix assumption in DruidSchema that all segments are immutable. Now, mutable
segments are periodically re-queried.
- Fix inappropriate re-use of SchemaPlus. Now we create one for each planning
cycle, rather than sharing one. It caches table objects, which we want to
avoid, since it can cause stale metadata. We do the caching in DruidSchema
so we don't need the SchemaPlus caching.
Server changes:
- Add a TimelineCallback to TimelineServerView, for callers that want to get updates
when the timeline has been modified.
- Change CachingClusteredClient from a QueryRunner to a QuerySegmentWalker. This
allows it to accept queries that are segment-descriptor-based rather than
intervals-based. In particular it will now support MultipleSpecificSegmentSpec.
* Fix DruidSchema, and unused imports.
* Remove unused import.
* Fix SqlBenchmark.
* Remove some unnecessary use of boxed types.
* Fix some incorrect format strings.
* Enable IDEA's MalformedFormatString inspection.
* Add a Checkstyle check for finding uses of incorrect logging packages.
* Fix some incorrect usages of the metamx logger.
* Bypass incorrect logger Checkstyle check where using the correct logger is not simple.
* Fix some more places where the wrong number of arguments are provided to format strings.
* Suppress `MalformedFormatString` inspection on legacy logging test.
* Use @SuppressWarnings rather than a noinspection suppression comment.
* Fix some more incorrect format strings.
* Suppress some more incorrect format string warnings where the incorrect string is intentional.
* Log the aggregator when closing it fails.
* Remove some unneeded log lines.
* Early publishing segments in the middle of data ingestion
* Remove unnecessary logs
* Address comments
* Refactoring the patch according to #4292 and address comments
* Set the total shard number of NumberedShardSpec to 0
* refactoring
* Address comments
* Fix tests
* Address comments
* Fix sync problem of committer and retry push only
* Fix doc
* Fix build failure
* Address comments
* Fix compilation failure
* Fix transient test failure
* SQL + Expressions = Best friends forever.
- Use expressions as a projection layer for anything that can't be
expressed using traditional Druid extractionFns. Sometimes they're
embedded directly (like "expression" filters, builtin aggregators,
or "expression" post-aggregators). Sometimes they're referenced
through virtual columns (like dimensionSpecs, which can't innately
reference functions of more than one column without the virtual
column layer).
- Add many new functions and operators, taking advantage of the
expression capability (see the querying/sql.md doc).
- Improve consistency of constant reduction and of casting by
using Druid expressions for this instead of Calcite's RexExecutor.
* Fix casting bug, and other code review comments.
* Fix docs.
* Use Double.NEGATIVE_INFINITY and Double.POSITIVE_INFINITY instead of Double.MIN_VALUE and Double.MAX_VALUE, same for Float
* Replace usages in comments
* Fix RTree
* Remove commented code
* Add tests
* Avoid usages of Default system Locale and printing to System.out or System.err in production code
* Fix Charset in DruidKerberosUtil
* Remove redundant string format in GenericIndexed
* Rename StringUtils.safeFormat() to unimportantSafeFormat(); add StringUtils.format() which fails as well as String.format()
* Fix testSafeFormat()
* More fixes of redundant StringUtils.format() inside ISE
* Rename unimportantSafeFormat() to nonStrictFormat()
* Add some new expression functions and macros.
See misc/math-expr.md for the list of added functions, except for
"like", which previously existed but was not documented.
* Add easymock to datasketches tests.
* Add easymock to distinctcount tests.
* Add easymock to virtual-columns tests.
* Code review comments.
* Clean up code a bit.
* Add easymock to scan-query tests.
* Rework ExprMacros that have multiple impls.
* Improve test coverage.
* Remove ability to create segments in v8 format
* Fix IndexGeneratorJobTest
* Fix parameterized test name in IndexMergerTest
* Remove extra legacy merging stuff
* Remove legacy serializer builders
* Remove ConciseBitmapIndexMergerTest and RoaringBitmapIndexMergerTest
* Adding a flag to indicate when ObjectCachingColumnSelectorFactory need not be threadsafe.
* - Use of computeIfAbsent over putIfAbsent
- Replace Maps.newXXXMap() with normal instantiation
- Documentations on when is thread-safe required.
- Use Builders for On/OffheapIncrementalIndex
* - Optimization on computeIfAbsent
- Constant EMPTY DimensionsSpec
- Improvement on IncrementalIndexSchema.Builder
- Remove setting of default values
- Use var args for metrics
- Correction on On/OffheapIncrementalIndex Builders
- Combine On/OffheapIncrementalIndex Builders
* - Removing unused imports.
* - Helper method for testing with IncrementalIndex.Builder
* - Correction on javadoc.
* Style fix
The tests go from 5 minutes to about 10 seconds. 1000 maxRowCount is still
low enough to get a few merges, so we're still exercising that functionality.
* Make PolyBind to fail if property value is not found
* Fix test
* Add onHeap option in NamespaceExtractionModule
* Add PolyBind.createChoiceWithDefaultNoScope()
* Fix NPE
* Fix
* Configure MetadataStorageProvider option for MySQL, PostgreSQL and SQLServer
* Deprecate PolyBind.createChoiceWithDefault form with unused defaultKey
* Fix NPE
* Expressions: Add ExprMacros, which have the same syntax as functions, but
can convert themselves to any kind of Expr at parse-time.
ExprMacroTable is an extension point for adding new ExprMacros. Anything
that might need to parse expressions needs an ExprMacroTable, which can
be injected through Guice.
* Address code review comments.
* Enable most IntelliJ 'Probable bugs' inspections
* Fix in RemoteTestNG
* Fix IndexSpec's equals() and hashCode() to include longEncoding
* Fix inspection errors
* Extract global isntance of natural().nullsFirst(); address comments
* Fix
* Use noinspection comments instead of SuppressWarnings on method for IntelliJ-specific inspections
* Prohibit Ordering.natural().nullsFirst() using Checkstyle
* Make using implicit system charset an error
* Use StringUtils.toUtf8() and fromUtf8() instead of String.getBytes() and new String()
* Use English locale in StringUtils.safeFormat()
* Restore comment
* refactor lag reporting and report lag at status endpoint
* refactor offset reporting logic to fetch offsets periodically vs. at request time
* remove JavaCompatUtils
* code review changes
* code review changes
* Adding s3a schema and s3a implem to hdfs storage module.
* use 2.7.3
* use segment pusher to make loadspec
* move getStorageDir and makeLoad spec under DataSegmentPusher
* fix uts
* fix comment part1
* move to hadoop 2.8
* inject deep storage properties
* set version to 2.7.3
* fix build issue about static class
* fix comments
* fix default hadoop default coordinate
* fix create filesytem
* downgrade aws sdk
* bump the version
* fix TestKafkaExtractionCluster fail due to port already used
* explicitly unmap hydrant files when abandonSegment to recyle mmap memory
* address the comments
* apply to AppenderatorImpl
* Refactoring Appenderator
1) Added publishExecutor and handoffExecutor for background publishing and handing segments off
2) Change add() to not move segments out in it
* Address comments
1) Remove publishTimeout for KafkaIndexTask
2) Simplifying registerHandoff()
3) Add increamental handoff test
* Remove unused variable
* Add persist() to Appenderator and more tests for AppenderatorDriver
* Remove unused imports
* Fix strict build
* Address comments
* move ProtoBufInputRowParser from processing module to protobuf extensions
* Ported PR #3509
* add DynamicMessage
* fix local test stuff that slipped in
* add license header
* removed redundant type name
* removed commented code
* fix code style
* rename ProtoBuf -> Protobuf
* pom.xml: shade protobuf classes, handle .desc resource file as binary file
* clean up error messages
* pick first message type from descriptor if not specified
* fix protoMessageType null check. add test case
* move protobuf-extension from contrib to core
* document: add new configuration keys, and descriptions
* update document. add examples
* move protobuf-extension from contrib to core (2nd try)
* touch
* include protobuf extensions in the distribution
* fix whitespace
* include protobuf example in the distribution
* example: create new pb obj everytime
* document: use properly quoted json
* fix whitespace
* bump parent version to 0.10.1-SNAPSHOT
* ignore Override check
* touch
Disables canonicalization for simpleJson, where expect field names to be unique
anyway. Keeps canonicalization enabled for customJson, but avoids sharing the
table with the global ObjectMapper.
* Timeout and maxScatterGatherBytes handling for queries run by Druid SQL
* Address PR comments
* Fix contexts in CalciteQueryTest
* Fix contexts in QuantileSqlAggregatorTest
* Optional long-polling based segment announcement via HTTP instead of Zookeeper
* address review comments
* make endpoint /druid-internal/v1 instead of /druid/internal so that jetty qos filters can be configured easily when needed
* update segment callback initialization to be called only after first segment list fetch has been succeeded from all servers
* address review comments
* remove size check not required anymore as only segment servers announce themselves and not all peon processes
* annouce segment server on historical only after cached segments are loaded
* fix checkstyle errors
* Monomorphic processing of topN queries with simple double aggregators and historical segments
* Add CalledFromHotLoop annocations to specialized methods in SimpleDoubleBufferAggregator
* Fix a bug in Historical1SimpleDoubleAggPooledTopNScannerPrototype
* Fix a bug in SpecializationService
* In SpecializationService, emit maxSpecializations warning only once
* Make GenericIndexed.theBuffer final
* Address comments
* Newline
* Reapply 439c906 (Make GenericIndexed.theBuffer final)
* Remove extra PooledTopNAlgorithm.capabilities field
* Improve CachingIndexed.inspectRuntimeShape()
* Fix CompressedVSizeIntsIndexedSupplier.inspectRuntimeShape()
* Don't override inspectRuntimeShape() in subclasses of CompressedVSizeIndexedInts
* Annotate methods in specializations of DimensionSelector and FloatColumnSelector with @CalledFromHotLoop
* Make ValueMatcher to implement HotLoopCallee
* Doc fix
* Fix inspectRuntimeShape() impl in ExpressionSelectors
* INFO logging of specialization events
* Remove modificator
* Fix OrFilter
* Fix AndFilter
* Refactor PooledTopNAlgorithm.scanAndAggregate()
* Small refactoring
* Add 'nothing to inspect' messages in empty HotLoopCallee.inspectRuntimeShape() implementations
* Don't care about runtime shape in tests
* Fix accessor bugs in Historical1SimpleDoubleAggPooledTopNScannerPrototype and HistoricalSingleValueDimSelector1SimpleDoubleAggPooledTopNScannerPrototype, cover them with tests
* Doc wording
* Address comments
* Remove MagicAccessorBridge and ensure Offset subclasses are public
* Attach error message to element
From the kafka-schema-registry-client in the avro extension slf4j
will be packaged into the distribution. We don't want this as it
will conflict and throw a slf4j multiple bindings warning.
This will cause slf4j to fall back to no-operation (NOP) binding.
* Make Errorprone the default compiler
* Address comments
* Make Error Prone's ClassCanBeStatic rule a error
* Preconditions allow only %s pattern
* Fix DruidCoordinatorBalancerTester
* Try to give the compiler more memory
* Remove distribution module activation on jdk 1.8 because only jdk 1.8 is used now
* Don't show compiler warnings
* Try different travis script
* Fix travis.yml
* Make Error Prone optional again
* For error-prone compiler
* Increase compiler's maxmem
* Don't run Error Prone for benchmarks because of OOM
* Skip install step in Travis
* Remove MetricHolder.writeToChannel()
* In travis.yml, check compilation before tests, because it may fail faster
* Add QueryPlus. Add QueryRunner.run(QueryPlus, Map) method with default implementation, to replace QueryRunner.run(Query, Map).
* Fix GroupByMergingQueryRunnerV2
* Fix QueryResourceTest
* Expand the comment to Query.run(walker, context)
* Remove legacy version of BySegmentSkippingQueryRunner.doRun()
* Add LegacyApiQueryRunnerTest and be more specific about legacy API removal plans in Druid 0.11 in Javadocs
These objects get constructed semi-frequently (any time a parser is
deserialized) and so info logs are spammy. They'll still appear in
task logs at least once, since they're part of the task definition and
will get logged due to that.
* 1. Collections.singletonList instand of Arrays.asList; 2. close FSDataInputStream/ByteBufferInputStream for releasing resource; 3. convert com.google.common.base.Function into java.util.function.Function; 4. others code refactor
* Put each param on its own line for code style
* Revert GenericRecordAsMap back about `Function`
* Fix lz4 library incompatibility in kafka-indexing-service extension #3266
* Bumped Kafka version to 0.10.2.0 for : Fix lz4 library incompatibility in kafka-indexing-service extension #3266
* Replaced Lists.newArrayList() with Collections.singletonList() For Fix lz4 library incompatibility in kafka-indexing-service extension #4115
* Minor bug fixes in GenericIndexed; Refactor and optimize GenericIndexed; Remove some unnecessary ByteBuffer duplications in some deserialization paths; Add ZeroCopyByteArrayOutputStream
* Fixes
* Move GenericIndexedWriter.writeLongValueToOutputStream() and writeIntValueToOutputStream() to SerializerUtils
* Move constructors
* Add GenericIndexedBenchmark
* Comments
* Typo
* Note in Javadoc that IntermediateLongSupplierSerializer, LongColumnSerializer and LongMetricColumnSerializer are thread-unsafe
* Use primitive collections in IntermediateLongSupplierSerializer instead of BiMap
* Optimize TableLongEncodingWriter
* Add checks to SerializerUtils methods
* Don't restrict byte order in SerializerUtils.writeLongToOutputStream() and writeIntToOutputStream()
* Update GenericIndexedBenchmark
* SerializerUtils.writeIntToOutputStream() and writeLongToOutputStream() separate for big-endian and native-endian
* Add GenericIndexedBenchmark.indexOf()
* More checks in methods in SerializerUtils
* Use helperBuffer.arrayOffset()
* Optimizations in SerializerUtils
* Monomorphic processing: add HotLoopCallee, CalledFromHotLoop, RuntimeShapeInspector, SpecializationService. Specialize topN queries with 1 or 2 aggregators. Add Cursor.advanceUninterruptibly() and isDoneOrInterrupted() for exception-free query processing.
* Use Execs.singleThreaded()
* RuntimeShapeInspector to support nullable fields
* Make CalledFromHotLoop annotation Inherited
* Remove unnecessary conversion of array of ColumnSelectorPluses to list and back to array in CardinalityAggregatorFactory
* Close InputStream in SpecializationService
* Formatting
* Test specialized PooledTopNScanners
* Set flags in PooledTopNAlgorithm directly
* Fix tests, dependent on CountAggragatorFactory toString() form
* Fix
* Revert CountAggregatorFactory changes
* Implement inspectRuntimeShape() for LongWrappingDimensionSelector and FloatWrappingDimensionSelector
* Remove duplicate RoaringBitmap dependency in the extendedset pom.xml
* Fix
* Treat ByteBuffers specially in StringRuntimeShape
* Doc fix
* Annotate BufferAggregator.init() with CalledFromHotLoop
* Make triggerSpecializationIterationsThreshold an int
* Remove SpecializationService.PerPrototypeClassState.of()
* Add comments
* Limit the amount of specializations that SpecializationService could make
* Add default implementation for BufferAggregator.inspectRuntimeShape(), for compatibility with extensions
* Use more efficient ConcurrentMap's idioms in SpecializationService
* Allow compilation as Java8 source and target for everything except API
* Remove conditions in tests which assume that we may run with Java 7
* Update easymock to 3.4
* Make Animal Sniffer to check Java 1.8 usage; remove redundant druid-caffeine-cache configuration
* Use try-with-resources in LargeColumnSupportedComplexColumnSerializerTest.testSanity()
* Remove java7 special for druid-api
task.pause(0) can return early before the task is actually paused.
Exception for failure -
java.lang.AssertionError: expected:<PAUSED> but was:<READING>
at org.junit.Assert.fail(Assert.java:88)
at org.junit.Assert.failNotEquals(Assert.java:743)
at org.junit.Assert.assertEquals(Assert.java:118)
at org.junit.Assert.assertEquals(Assert.java:144)
at
io.druid.indexing.kafka.KafkaIndexTaskTest.testRunWithOffsetOutOfRangeEx
ceptionAndPause(KafkaIndexTaskTest.java:1229)
To reproduce add Thread.sleep(10000) in beginning of
KafkaIndexTask.possiblypause method.
* Ignore chunkPeriod for groupBy v2, fix chunkPeriod for irregular periods.
Includes two fixes:
- groupBy v2 now ignores chunkPeriod, since it wouldn't have helped anyway (its mergeResults
returns a lazy sequence) and it generates incorrect results.
- Fix chunkPeriod handling for periods of irregular length, like "P1M" or "P1Y".
Also includes doc and test fixes:
- groupBy v1 was no longer being tested by GroupByQueryRunnerTest since #3953, now it
is once again.
- chunkPeriod documentation was misleading due to its checkered past. Updated it to
be more accurate.
* Remove unused import.
* Restore buffer size.
* In IndexMerger and IndexMergerV9, create temporary files under the output directory/tmpPeonFiles, instead of java.io.tmpdir
* Use FileUtils.forceMkdir() across the codebase and remove some unused code
* Fix test
* Fix PullDependencies.run()
* Unused import
* NN optimization for hdfs data segments.
* HdfsDataSegmentKiller, HdfsDataSegment finder changes to use new storage
format.Docs update.
* Common utility function in DataSegmentPusherUtil.
* new static method `makeSegmentOutputPathUptoVersionForHdfs` in JobHelper
* reuse getHdfsStorageDirUptoVersion in
DataSegmentPusherUtil.getHdfsStorageDir()
* Addressed comments.
* Review comments.
* HdfsDataSegmentKiller requested changes.
* extra newline
* Add maprfs.
* No more singleton. Reduce iterations
* Granularities
* Fix the delay in the test
* Add license header
* Remove unused imports
* Lot more unused imports from all the rearranging
* CR feedback
* Move javadoc to constructor
* Refactor Segment Granularity
* Beginning of one granularity
* Copy the fix for custom periods in segment-grunalrity over here.
* Remove the custom serialization for now.
* Compilation cleanup
* Reformat code
* Fixing unit tests
* Unify to use a single iterable
* Backward compatibility for rolling upgrade
* Minor check style. Cosmetic changes.
* Rename length and millis to duration
* CR feedback
* Minor changes.
* initial commits for finalizeFieldAccess #2433
* fix some bugs to run a query
* change name of method Queries.verifyAggregations to Queries.prepareAggregations
* add Uts
* fix Ut failures
* rebased to master
* address comments and add a Ut for arithmetic post aggregators
* rebased to the master
* address the comment of injection within arithmetic post aggregator
* address comments and introduce decorate() in the PostAggregator interface.
* Address comments. 1. Implements getComparator in FinalizingFieldAccessPostAggregator and add Uts for it 2. Some minor changes like renaming a method name.
* Fix a code style mismatch.
* Rebased to the master
* SQL: Add context and contextual functions to planner.
Added support for context parameters specified as JDBC connection properties
or a JSON object for SQL-over-JSON-over-HTTP.
Also added features that depend on context functionality:
- Added CURRENT_DATE, CURRENT_TIME, CURRENT_TIMESTAMP functions.
- Added support for time zones other than UTC via a "timeZone" context.
- Pass down query context to Druid queries too.
Also some bug fixes:
- Fix DATE handling, it was largely done incorrectly before.
- Fix CAST(__time TO DATE) which should do a floor-to-day.
- Fix non-equality comparisons to FLOOR(__time TO X).
- Fix maxQueryCount property.
* Pass down context to nested queries too.
* Require Java 8 and include some Java 8 dependencies.
- Upgrade Jetty to 9.3.16.v20170120.
- Upgrade DataSketches to 0.8.4.
- Bundle caffeine-cache by default.
- Still target Java 7 when compiling base Druid classes.
* Update cluster, quickstart docs.
* Remove oraclejdk7 from travis.yml.
* Less use of File.deleteOnExit()
* removed deleteOnExit from most of the tests/benchmarks/iopeon
* Made IOpeon closable
* Formatting.
* Revert DeterminePartitionsJobTest, remove cleanup method from IOPeon