* Add virtual column types, holder serde, and safety features.
Virtual columns:
- add long, float, dimension selectors
- put cache IDs in VirtualColumnCacheHelper
- adjust serde so VirtualColumns can be the holder object for Jackson
- add fail-fast validation for cycle detection and duplicates
- add expression virtual column in core
Storage adapters:
- move virtual column hooks before checking base columns, to prevent surprises
when a new base column is added that happens to have the same name as a
virtual column.
* Fix ExtractionDimensionSpecs with virtual dimensions.
* Fix unused imports.
* CR comments
* Merge one more time, with feeling.
* SQL: Add resolution parameter to quantile agg, rename to APPROX_QUANTILE.
* Fix bug with re-use of filtered approximate histogram aggregators.
Also add APPROX_QUANTILE tests for filtering and running on complex columns.
Includes some slight refactoring to allow tests to make DruidTables that
include complex columns.
* Remove unused import
* * Add DimensionSelector.idLookup() and nameLookupPossibleInAdvance() to allow better inspection of features DimensionSelectors supports, and safer code working with DimensionSelectors in BaseTopNAlgorithm, BaseFilteredDimensionSpec, DimensionSelectorUtils;
* Add PredicateFilteringDimensionSelector, to make BaseFilteredDimensionSpec to be able to decorate DimensionSelectors with unknown cardinality;
* Add DimensionSelector.makeValueMatcher() (two kinds) for DimensionSelector-side specifics-aware optimization of ValueMatchers;
* Optimize getRow() in BaseFilteredDimensionSpec's DimensionSelector, StringDimensionIndexer's DimensionSelector and SingleScanTimeDimSelector;
* Use two static singletons, TrueValueMatcher and FalseValueMatcher, instead of BooleanValueMatcher;
* Add NullStringObjectColumnSelector singleton and use it in MapVirtualColumn
* Rename DimensionSelectorUtils.makeNonDictionaryEncodedIndexedIntsBasedValueMatcher to makeNonDictionaryEncodedRowBasedValueMatcher
* Make ArrayBasedIndexedInts constructor private, replace it's usages with of() static factory method
* Cache baseIdLookup in ForwardingFilteredDimensionSelector
* Fix a bug in DimensionSelectorUtils.makeRowBasedValueMatcher(selector, predicate, matchNull)
* Employ precomputed BitSet optimization in DimensionSelector.makeValueMatcher(value, matchNull) when lookupId() is not available, but cardinality is known and lookupName() is available
* Doc fixes
* Addressed comments
* Fix
* Fix
* Adjust javadoc of DimensionSelector.nameLookupPossibleInAdvance() for SingleScanTimeDimSelector
* throw UnsupportedOperationException instead of IAE in BaseTopNAlgorithm
* remove wait between sends, wait for ingestion to complete before querying
send fixed number of events
more fixes
* handle interrupted exception
* remove while
* review comments
* Removing unused code from io.druid.java.util.common.guava package; fix#3563 (more consistent and paranoiac resource handing in Sequences subsystem); Add Sequences.wrap() for DRY in MetricsEmittingQueryRunner, CPUTimeMetricQueryRunner and SpecificSegmentQueryRunner; Catch MissingSegmentsException in SpecificSegmentQueryRunner's yielder.next() method (follow up on #3617)
* Make Sequences.withEffect() execute the effect if the wrapped sequence throws exception from close()
* Fix strange code in MetricsEmittingQueryRunner
* Add comment on why YieldingSequenceBase is used in Sequences.withEffect()
* Use Closer in OrderedMergeSequence and MergeSequence to close multiple yielders
This simplifies DruidSemiJoin, which no longer needs to add aggregation back
in. It also allows some more kinds of queries to plan properly, like the one
added in "testTopNFilterJoin".
* SQL: Ditch CalciteConnection layer and add DruidMeta, extension aggregators.
Switched from CalciteConnection to Planner, bringing benefits:
- CalciteConnection's JDBC interface no longer sits between the SQL server
(HTTP/Avatica) and Druid's query layer. Instead, the SQL servers can use
Druid Sequence objects directly, reducing overhead in the query return path.
- Implemented our own Planner-based Avatica Meta, letting us control
connection timeouts and connection / statement limits. The previous
CalciteConnection-based implementation didn't have any limits or timeouts.
- The Planner interface lets us override the operator table, opening up
SQL language extensions. This patch includes two: APPROX_COUNT_DISTINCT
in core, and a QUANTILE aggregator in the druid-histogram extension.
Also:
- Added INFORMATION_SCHEMA metadata schema.
- Added tests for Unicode literals and escapes.
* Verify statement is actually open before closing it.
* More detailed INFORMATION_SCHEMA docs.
* streaming version of select query
* use columns instead of dimensions and metrics;prepare for valueVector;remove granularity
* respect query limit within historical
* use constant
* fix thread name corrupted bug when using jetty qtp thread rather than processing thread while working with SpecificSegmentQueryRunner
* add some test for scan query
* add scan query document
* fix merge conflicts
* add compactedList resultFormat, this format is better for json ser/der
* respect query timeout
* respect query limit on broker
* use static consts and remove unused code
* Exclude the transitive dependency LGPL jar since it is not needed
* add reason why exclude
* exclude from the root dependency
* add banning tool to enforce exclusions
* SQL support for nested groupBys.
Allows, for example, doing exact count distinct by writing:
SELECT COUNT(*) FROM (SELECT DISTINCT col FROM druid.foo)
Contrast with approximate count distinct, which is:
SELECT COUNT(DISTINCT col) FROM druid.foo
* Add deeply-nested groupBy docs, tests, and maxQueryCount config.
* Extract magic constants into statics.
* Rework rules to put preconditions in the "matches" method.
* Add an option to SearchQuery to choose a search query execution strategy.
Supported strategies are
1) Index-only query execution
2) Cursor-based scan
3) Auto: choose an efficient strategy for a given query
* Add SearchStrategy and SearchQueryExecutor
* Address comments
* Rename strategies and set UseIndexesStrategy as the default strategy
* Add a cost-based planner for auto strategy
* Add document
* Fix code style
* apply code style
* apply comments
* Filters: Use ColumnSelectorFactory directly for building row-based matchers.
* Adjustments based on code review.
- BoundDimFilter: fewer volatiles, rename matchesAnything to !matchesNothing.
- HavingSpecs: Clarify that they are not thread-safe, and make DimFilterHavingSpec
not thread safe.
- Renamed rowType to rowSignature.
- Added specializations for time-based vs non-time-based DimensionSelector in RBCSF.
- Added convenience method DimensionHanderUtils.createColumnSelectorPlus.
- Added singleton ZeroIndexedInts.
- Added test cases for DimFilterHavingSpec.
* Make ValueMatcherColumnSelectorStrategy actually use the associated selector.
* Add RangeIndexedInts.
* DimFilterHavingSpec: Fix concurrent usage guard on jdk7.
* Add assertion to ZeroIndexedInts.
* Rename no-longer-volatile members.
* Allow users to specify additional command line args for creating tar balls
This PR allows users to specify additional command line options to the
pull deps command while creating druid distribution.
e.g. To also package graphite-emitter in druid tarball one can run -
mvn package -Ddruid.distribution.pulldeps.opts='-c
io.druid.extensions.contrib:graphite-emitter'
* Set default to --clean instead of blank value
* allow JsonConfigTesterBase to treat the fields of collections
* [Feature] Exhibitor Support (#3664)
This patch provides the integration of Druid & Netflix Exhibitor. Druid
currently use Apache Curator as ZooKeeper client. Curator can be
integrated with Exhibitor to achieve a live/updating list of the
ZooKeeper ensemble. This patch enables Druid to use this features.
* Lookup cache refactoring (the main part of druid-io/druid#3667)
* Use PowerMock's static methods in NamespaceLookupExtractorFactoryTest
* Fix KafkaLookupExtractorFactoryTest
* Use VisibleForTesting annotation instead of Javadoc comment
* Create a NamespaceExtractionCacheManager separately for each test in NamespaceExtractionCacheManagersTest
* Rename CacheScheduler.NoCache.ENTRY_DISPOSED to ENTRY_CLOSED
* Reduce visibility of NamespaceExtractionCacheManager.cacheCount() and monitor() implementations, and don't run NamespaceExtractionCacheManagerExecutorsTest with off-heap cache (it didn't before)
* In NamespaceLookupExtractorFactory, use safer idiom to check if CacheState is NoCache or VersionedCache
* More logging in CacheHandler constructor and close(), VersionedCache.close()
* PR comments addressed
* Make CacheScheduler.EntryImpl AutoCloseable, avoid 'dispose' verb in comments, logging and naming in CacheScheduler in favor of 'close'
* More Javadoc comments to CacheScheduler
* Fix NPE
* Remove logging in OnHeapNamespaceExtractionCacheManager.expungeCollectedCaches()
* Make NamespaceExtractionCacheManagersTest.testRacyCreation() to have similar load to what it be before the refactoring
* Unwrap NamespaceExtractionCacheManager.scheduledExecutorService from unneeded MoreExecutors.listeningDecorator() and specify that this is ScheduledThreadPoolExecutor, which ensures happens-before between periodic runs of the tasks
* More comments on MapDbCacheDisposer.disposed
* Replace concat with Long.toString()
* Comment on why NamespaceExtractionCacheManager.scheduledExecutorService() returns ScheduledThreadPoolExecutor
* Place logging statements in VersionedCache.close() and CacheHandler.close() after actual closing logic, because logging may fail
* Make JDBCExtractionNamespaceCacheFactory and StaticMapExtractionNamespaceCacheFactory to try to close newly created VersionedCache if population has failed, as it is done already in URIExtractionNamespaceCacheFactory
* Don't close the whole CacheScheduler.Entry, if the cache update task failed
* Replace AtomicLong updateCounter and firstRunLatch with Phaser-based UpdateCounter in CacheScheduler.EntryImpl
* Fix#3795 (Java 7 compatibility).
Also introduce Animal Sniffer checks during build, which would
have caught the original problems.
* Add Animal Sniffer on caffeine-cache for JDK8.