* RealtimeIndexTask to support alertTimeout in context and raise alert if task process exists after the timeout
* move alertTimeout config to tuningConfig and document
It wasn't doing anything useful (the sequences were being concatted, and
cursor.getTime() wasn't being called) and it defaulted to Granularities.NONE.
Changing it to Granularities.ALL gave me a 700x+ performance boost on a
small dataset I was reindexing (2m27s to 365ms). Most of that was from avoiding
making a lot of unnecessary column selectors.
* 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
* 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
* 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.
GroupBy v2 doesn't cache on the broker, so it isn't actually testing
what the test was supposed to be testing. Also, the test failed due
to mismatched expectations.
* fix cache populate incorrect content when numBackgroundThreads>1
* simplify code by using Futures.allAsList and use CountDownLatch in UT
* fix test code style and assert countDownLatch.await()
* 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
* modified "end" column to `end`. "end" is interpretted as a string rather than dereferencing the column value
* SQLMetadataConnector.getQuoteString defines the string that should be used to quote string fields
* positional arguments for String.format
* for Connectors that use " need to include the \ escape as well
* Add PostAggregators to generator cache keys for top-n queries
* Add tests for strings
* Remove debug comments
* Add type keys and list sizes to cache key
* Make post aggregators used for sort are considered for cache key generation
* Use assertArrayEquals()
* Improve findPostAggregatorsForSort()
* Address comments
* fix test failure
* address comments
* Specialize LoadBalancingPool as MemcacheClientPool, reduce locking and don't override Object.finalize()
* Remove locking and don't override Object.finalize() in ReferenceCountingResourceHolder
* Add leak counts in ReferenceCountingResourceHolder and MemcacheClientPool. Add tests for ReferenceCountingResourceHolder and MemcacheClientPool
* Fix a race condition in ReferenceCountingResourceHolder.increment()
* Introduce SegmentizerFactory
- that knows how to deserialize specific type of segment
- Default implementation is MMappedQueryableSegmentizerFactory which creates QueryableIndexSegment
- Unit test for the default behavior
* review comments
* auto reset option for Kafka Indexing service in case message at the offset being fetched is not present anymore at kafka brokers
* review comments
* review comments
* reverted last change
* review comments
* review comments
* fix typo
* 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
* 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
* 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.
* 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.
* Add metrics for Query Count statistics
This PR adds a new metrics monitor “QueryCountStatsMonitor” which emits
three new metrics -
1) query/success/count - number of successful queries
2) query/failed/count - number of failed queries
3) query/interrupted/count - number of interrupted/timedout queries
fix bindings
* make fields final
* fix imports
* AsyncQueryForwardingServlet implement QueryStatsProvider
* remove unused import
* Enable parallel test
* Remove unnecessary NotThreadSafe annocation
* Randomize the start port when finding available ports
* Fix test failure
* Change to handle all negatives
* report message gap, source gap and sink count in RealtimePlumber
* report message gap, sink count in Appenderator
* add ingest/events/sourceGap in metrics.md
* remove source gap
* Fix PathChildrenCache's executorService leak in Announcer, CuratorInventoryManager and RemoteTaskRunner
* Use a single ExecutorService for all workerStatusPathChildrenCaches in RemoteTaskRunner
* make sure CliCoordinator initializes and starts DerbyMetadataStorage first if configured
* Revert "make sure CliCoordinator initializes and starts DerbyMetadataStorage first if configured"
This reverts commit 54f5644054626d4a9e2448bb4bd5e6ce9a9fca1d.
* make sure CliCoordinator initializes and starts DerbyMetadataStorage first if configured
* Normalized Cost Balancer
* Adding documentation and renaming to use diskNormalizedCostBalancer
* Remove balancer from the strings
* Update docs and include random cost balancer
* Fix checkstyle issues
This retries when the start condition is met but SELECT -> INSERT/UPDATE
fails, which indicates a race. If the start condition isn't met, there
won't be any retrying.
* Update emitter dependency to 0.4.0 and emit "version" dimension for all druid metrics, not only query metrics
* Remove unused imports
* Use empty string instead of "testing-version" as a version placeholder
Excludes tests from AvoidStaticImport, since those are used often there and
I didn't want to make this changeset too large. Production code use was minimal
and I switched those to non-static imports.
This is useful for the insert-segment-to-db tool, which would otherwise
potentially insert a lot of overshadowed segments as "used", causing
load and drop churn in the cluster.
* print exception details from QueryInterruptedException
* in QueryResource.java, set thread name to include dataSource names and not whole query string e.g. from QueryDataSource
* Add support for timezone in segment granularity
* CR feedback. Handle null timezone during equals check.
* Include timezone in docs.
Add timezone for ArbitraryGranularitySpec.
* Show candidate hosts for the given query
* Added test cases & minor changes to address comments
* Changed path-param to query-pram for intervals/numCandidates
* support renaming of outputName for cached select and search queries
* rebase and resolve conflicts
* rollback CacheStrategy interface change
* updated based on review comments
* introducing lists of existing columns in the fields of select queries' output
* rebase master
* address the comment. add test code for select query caching
* change the cache code in SelectQueryQueryToolChest to 0x16
Follow-up to #1773, which meant to add more useful query errors but
did not actually do so. Since that patch, any error other than
interrupt/cancel/timeout was reported as `{"error":"Unknown exception"}`.
With this patch, the error fields are:
- error, one of the specific strings "Query interrupted", "Query timeout",
"Query cancelled", or "Unknown exception" (same behavior as before).
- errorMessage, the message of the topmost non-QueryInterruptedException
in the causality chain.
- errorClass, the class of the topmost non-QueryInterruptedException
in the causality chain.
- host, the host that failed the query.
* ability to not rollup at index time, make pre aggregation an option
* rename getRowIndexForRollup to getPriorIndex
* fix doc misspelling
* test query using no-rollup indexes
* fix benchmark fail due to jmh bug
* Add numeric StringComparator
* Only use direct long comparison for numeric ordering in BoundFilter, add time filtering benchmark query
* Address PR comments, add multithreaded BoundDimFilter test
* Add comment on strlen tie handling
* Add timeseries interval filter benchmark
* Adjust docs
* Use jackson for StringComparator, address PR comments
* Add new TopNMetricSpec and SearchSortSpec with tests (WIP)
* More TopNMetricSpec and SearchSortSpec tests
* Fix NewSearchSortSpec serde
* Update docs for new DimensionTopNMetricSpec
* Delete NumericDimensionTopNMetricSpec
* Delete old SearchSortSpec
* Rename NewSearchSortSpec to SearchSortSpec
* Add TopN numeric comparator benchmark, address PR comments
* Refactor OrderByColumnSpec
* Add null checks to NumericComparator and String->BigDecimal conversion function
* Add more OrderByColumnSpec serde tests
This reverts commit a931debf79.
Fixes#3283
The core issue here is that realtime nodes announce their size as 0, so a coordinator which interns the realtime version of the data segment will not be able to see the new sized announcement when handoff occurs.
This is caused by the `eauals` method on a `DataSegment` only evaluating the identifier. the `eauals` method *should* be correct for object equivalence, and things which need to check equivalence of some sub-portion of the object should do so explicitly.
Fixes inconsistent metric handling between the two implementations. Formerly,
RealtimePlumber only emitted query/segmentAndCache/time and query/wait and
Appenderator only emitted query/partial/time and query/wait (all per sink).
Now they both do the same thing:
- query/segmentAndCache/time, query/segment/time are the time spent per sink.
- query/cpu/time is the CPU time spent per query.
- query/wait/time is the executor waiting time per sink.
These generally match historical metrics, except segmentAndCache & segment
mean the same thing here, because one Sink may be partially cached and
partially uncached and we aren't splitting that out.
All query metrics now start with toolChest.makeMetricBuilder, and all of
*those* now start with DruidMetrics.makePartialQueryTimeMetric. Also, "id"
moved to common code, since all query metrics added it anyway.
In particular this will add query-type specific dimensions like "threshold"
and "numDimensions" to servlet-originated metrics like query/time.
* Migrate IndexerSQLMetadataStorageCoordinator.getUnusedSegmentsForInterval to streaming
* Missed query from #2859
* Make inReadOnlyTransaction part of SQLMetadataConnector
This patch introduces a GroupByStrategy concept and two strategies: "v1"
is the current groupBy strategy and "v2" is a new one. It also introduces
a merge buffers concept in DruidProcessingModule, to try to better
manage memory used for merging.
Both of these are described in more detail in #2987.
There are two goals of this patch:
1. Make it possible for historical/realtime nodes to return larger groupBy
result sets, faster, with better memory management.
2. Make it possible for brokers to merge streams when there are no order-by
columns, avoiding materialization.
This patch does not do anything to help with memory management on the broker
when there are order-by columns or when there are nested queries. That could
potentially be done in a future patch.
* add get dimension rangeset to filters
* add get domain to ShardSpec and added chunk filter in caching clustered client
* add null check and modified not filter, started with unit test
* add filter test with caching
* refactor and some comments
* extract filtershard to helper function
* fixup
* minor changes
* update javadoc
* fix caching for search results
properly read count when reading from cache.
* fix NPE during merging search count and add test
* Update cache key to invalidate prev results
* validate X-Druid-Task-Id header in request and add header to response
* modify KafkaIndexTaskClient to take a TaskLocationProvider as the TaskLocation may not remain constant
* Allow dynamically setting of shutoffTime for EventReceiverFirehose
Allow dynamically setting shutoffTime for EventReceiverFirehose
review comments and tests
* shut down exec on close
* Datasource as lookup tier
* Adds an option to let indexing service tasks pull their lookup tier from the datasource they are working for.
* Fix bad docs for lookups lookupTier
* Add Datasource name holder
* Move task and datasource to be pulled from Task file
* Make LookupModule pull from bound dataSource
* Fix test
* Fix code style on imports
* Fix formatting
* Make naming better
* Address code comments about naming
* new interval based cost function
Addresses issues with balancing of segments in the existing cost function
- `gapPenalty` led to clusters of segments ~30 days apart
- `recencyPenalty` caused imbalance among recent segments
- size-based cost could be skewed by compression
New cost function is purely based on segment intervals:
- assumes each time-slice of a partition is a constant cost
- cost is additive, i.e. cost(A, B union C) = cost(A, B) + cost(A, C)
- cost decays exponentially based on distance between time-slices
* comments and formatting
* add more comments to explain the calculation
* Allow user to set cost balancer threads more than the number of cores.
Allow user to set cost balancer threads more than the number of cores.
* modify test
* Optimize CostBalancerStrategy
Ignore benchmark test in normal run
fix test
review comments
fix compilation
fix test
* review comments
* review comment
* Make URI Exctraction Namespace take more sane arguments
* Fixes https://github.com/druid-io/druid/issues/2669
* Update docs
* Rename error message
* Undo overzealous deletion of docs
* Explain caching mechanism a bit more in docs
- Introduce `AuthorizationInfo` interface, specific implementations of which would be provided by extensions
- If the `druid.auth.enabled` is set to `true` then the `isAuthorized` method of `AuthorizationInfo` will be called to perform authorization checks
- `AuthorizationInfo` object will be created in the servlet filters of specific extension and will be passed as a request attribute with attribute name as `AuthConfig.DRUID_AUTH_TOKEN`
- As per the scope of this PR, all resources that needs to be secured are divided into 3 types - `DATASOURCE`, `CONFIG` and `STATE`. For any type of resource, possible actions are - `READ` or `WRITE`
- Specific ResourceFilters are used to perform auth checks for all endpoints that corresponds to a specific resource type. This prevents duplication of logic and need to inject HttpServletRequest inside each endpoint. For example
- `DatasourceResourceFilter` is used for endpoints where the datasource information is present after "datasources" segment in the request Path such as `/druid/coordinator/v1/datasources/`, `/druid/coordinator/v1/metadata/datasources/`, `/druid/v2/datasources/`
- `RulesResourceFilter` is used where the datasource information is present after "rules" segment in the request Path such as `/druid/coordinator/v1/rules/`
- `TaskResourceFilter` is used for endpoints is used where the datasource information is present after "task" segment in the request Path such as `druid/indexer/v1/task`
- `ConfigResourceFilter` is used for endpoints like `/druid/coordinator/v1/config`, `/druid/indexer/v1/worker`, `/druid/worker/v1` etc
- `StateResourceFilter` is used for endpoints like `/druid/broker/v1/loadstatus`, `/druid/coordinator/v1/leader`, `/druid/coordinator/v1/loadqueue`, `/druid/coordinator/v1/rules` etc
- For endpoints where a list of resources is returned like `/druid/coordinator/v1/datasources`, `/druid/indexer/v1/completeTasks` etc. the list is filtered to return only the resources to which the requested user has access. In these cases, `HttpServletRequest` instance needs to be injected in the endpoint method.
Note -
JAX-RS specification provides an interface called `SecurityContext`. However, we did not use this but provided our own interface `AuthorizationInfo` mainly because it provides more flexibility. For example, `SecurityContext` has a method called `isUserInRole(String role)` which would be used for auth checks and if used then the mapping of what roles can access what resource needs to be modeled inside Druid either using some convention or some other means which is not very flexible as Druid has dynamic resources like datasources. Fixes#2355 with PR #2424
* Avoids fetching all segment records into heap by JDBC driver
* Set connection to read-only to help database optimize queries
* Update JDBC drivers (MySQL has fixes for streaming results)
* Add back FilteredServerView removed in a32906c7fd to reduce memory usage using watched tiers.
* Add functionality to specify "druid.broker.segment.watchedDataSources"
segment creation deterministic.
This means that each segment will contain data from just one Kafka
partition. So, users will probably not want to have a super high number
of Kafka partitions...
Fixes#2703.
Reverts "Update com.maxmind.geoip2 to 2.6.0" and exclude the google http client
from com.maxmind.geoip2. This should satisfy the original need from #2646 (wanting
to run Druid along with an upgraded com.google.http-client) while preventing
Jackson conflicts pointed out in #2717.
Fixes#2717.
This reverts commit 21b7572533.
Fixes#2682
IndexingService helpers are added according to the settings in runtime.properties.
Rather than having all the config.isXXX checks there, it makes sense to have a pluggable
approach for allowing the dynamic configuration to bring in implementations for helpers
without having to have hard-coded sets of available helpers. Plus, it will also make it possible for extensions to plug helpers in.
With https://github.com/druid-io/druid-api/pull/76, we could conditionally bind a helper to Coordinator's runlist.
The condition is driven by the value set in the runtime.properties.
I believe that the instanceof chain in Filters exists because in the past, Filter
and DimFilter were in different packages (DimFilter was in druid-client and Filter
was in druid-processing). And since druid-client didn't depend on druid-processing,
DimFilter couldn't have a toFilter method. But now it can.
com.maxmind.geoip2 2.6.0 depends on com.google.http-client 1.15.0-rc (3 years old).
When trying to include other libraries in Druid that require an up to date version of com.google.http-client this causes a problem.
Geared towards supporting transactional inserts of new segments. This involves an
interface "DataSourceMetadata" that allows combining of partially specified metadata
(useful for partitioned ingestion).
DataSource metadata is stored in a new "dataSource" table.
Appenderators are a way of getting more control over the ingestion process
than a Plumber allows. The idea is that existing Plumbers could be implemented
using Appenderators, but you could also implement things that Plumbers can't do.
FiniteAppenderatorDrivers help simplify indexing a finite stream of data.
Also:
- Sink: Ability to consider itself "finished" vs "still writable".
- Sink: Ability to return the number of rows contained within the sink.
The incremental indexes handle that now so it's not necessary.
Also, add debug logging and more detailed exceptions to the incremental
indexes for the case where there are parse exceptions during aggregation.
After finding the FireChief for a specific partition, Druid will need to find the specific queryRunner for each segment being queried by passing the query to FireChief. Currently Druid is passing the original query that contains all the segments need to be queried, it's possible that fireChief.getQueryRunner(query) returns more than 1 queryRunner because query.getIntervals() is not specific to a single segment.
In this patch, for each segment being queried, Druid will update the query with its corresponding SpecificSegmentSpec.
See stack traces here, from current master: https://gist.github.com/gianm/bd9a66c826995f97fc8f
1. The thread "qtp925672150-62" holds the lock on InternalInjectorCreator.class,
used by Scopes.SINGLETON, and wants the lock on "handlers" in Lifecycle.addMaybeStartHandler
called by DiscoveryModule.getServiceAnnouncer.
2. The main thread holds the lock on "handlers" in Lifecycle.addMaybeStartHandler, which it
took because it's trying to add the ExecutorLifecycle to the lifecycle. main is trying
to get the InternalInjectorCreator.class lock because it's running ExecutorLifecycle.start,
which does some Jackson deserialization, and Jackson needs that lock in order to inject
stuff into the Task it's deserializing.
This patch eagerly instantiates ChatHandlerResource (which I believe is what's trying to
create the ServiceAnnouncer in the qtp925672150-62 jetty thread) and the ExecutorLifecycle.
To bring consistency to docs and source this commit changes the default
values for maxRowsInMemory and rowFlushBoundary to 75000 after
discussion in PR https://github.com/druid-io/druid/pull/2457.
The previous default was 500000 and it's lower now on the grounds that
it's better for a default to be somewhat less efficient, and work,
than to reach for the stars and possibly result in
"OutOfMemoryError: java heap space" errors.
Add tests that verify whether RealtimeManager is querying the correct FireChief for a specific partition
make FireChief static and package private, add latches in the UT
Two changes:
- Allow IncrementalIndex to suppress ParseExceptions on "aggregate".
- Add "reportParseExceptions" option to realtime tuning configs. By default this is "false".
Behavior of the counters should now be:
- processed: Number of rows indexed, including rows where some fields could be parsed and some could not.
- thrownAway: Number of rows thrown away due to rejection policy.
- unparseable: Number of rows thrown away due to being completely unparseable (no fields salvageable at all).
If "reportParseExceptions" is true then "unparseable" will always be zero (because a parse error would
cause an exception to be thrown). In addition, "processed" will only include fully parseable rows
(because even partial parse failures will cause exceptions to be thrown).
Fixes#2510.
- Add druid.indexer.server.maxChatRequests, which sets up a QoSFilter on the main Jetty server.
- Deprecate druid.indexer.runner.separateIngestionEndpoint
- Deprecate druid.indexer.server.chathandler.*