* Properly set "identity" in query metrics.
This patch adds an "identity" field to QueryPlus and sets it in
QueryLifecycle when the query starts executing. This is important
because it allows it to be used for future QueryMetrics created
by that QueryPlus object.
We also add "identity" to the request-level QueryMetrics object
created in emitLogsAndMetrics.
* Remove unused method.
* Change RealtimeIndexTask to use AppenderatorDriver instead of RealtimePlumber. Related to #4774
* Remove unused throwableDuringPublishing
* Fix usage of forbidden API
* Update realtime index IT to account for not skipping older data any more
* Separate out waiting on publish futures and handoff futures to avoid a race condition where the handoff timeout expires before the segment is published
* #5261 Add separate AppenderatorDriverRealtimeIndexTask and revert changes to RealtimeIndexTask
* #5261 Add separate AppenderatorDriverRealtimeIndexTask and revert changes to RealtimeIndexTask
* #5261 Readability improvements in AppenderatorDriverRealtimeIndexTask. Combine publish and handoff futures in to single future
* #5261 Add separate tuningConfig for RealtimeAppenderatorIndexTask. Revert changes to RealtimeTuningConfig
* #5261 Change JSON type to realtime_appenderator to keep the same naming pattern as RealtimeIndexTask
* Fix races in LookupSnapshotTaker, CoordinatorPollingBasicAuthenticatorCacheManager.
Both were susceptible to the following conditions:
1. Two JVMs on the same machine (perhaps two peons) could conflict by one reading while the
other was writing, or by writing to the file at the same time.
2. One JVM could partially write a file, then crash, leaving a truncated file.
* Use StringUtils.format
* 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
The behavior is configurable through druid.extensions.useExtensionClassloaderFirst.
It is useful when extensions want to load a dependency different from one provided
by Druid, for example a different version of geoip or protobuf.
Also remove Guice annotations from LoadQueueTaskMaster, since it is
provided by CliCoordinator, so Guice does not need to know how to
build one directly.
Code changes:
- In the lookup-based extractionFns, inherit injective property from
the lookup itself if not specified.
Doc changes:
- Add a "Query execution" section to the lookups doc explaining how
injective lookups and their optimizations work.
- Remove scary warnings against using registeredLookup extractionFns.
They are necessary and important since they work with filters and
function cascades -- two things that the dimension specs do not do.
They deserve to be first class citizens.
- Move the "registeredLookup" fn above the "lookup" fn. It's probably
more commonly used, so the docs read better this way.
* just renaming of SegmentChangeRequestHistory etc
* additional change history refactoring changes
* WorkerTaskManager a replica of WorkerTaskMonitor
* HttpServerInventoryView refactoring to extract sync code and robustification
* Introducing HttpRemoteTaskRunner
* Additional Worker side updates
* Fix broken KafkaEmitterConfig parsing
This was a regression introduced in
https://github.com/druid-io/druid/pull/4722
KafkaEmitterConfig property names have dot(.) in the name of properties
and JsonConfigurator behavior was changed to not support that.
Added a test and fixed parsing of properties that have dot(.) in
property names
* Fix test failure
* Add freeSpacePercent config in segment location config to enforce free space while storing segments
* address review comments
* address review comments: more doc on freeSpacePercent and use Double for freeSpacePercent
* FilteredHttpServerInventoryViewProvider to start with always false predicate for each segment discovered
* update HttpServerInventoryViewTest to ensure that predicates are honored
* add docs for HttpServerInventoryView.defaultFilter
* change to javadoc style comment
* 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
* Kafka Index Task that supports Incremental handoffs
- Incrementally handoff segments when they hit maxRowsPerSegment limit
- Decouple segment partitioning from Kafka partitioning, all records from consumed partitions go to a single druid segment
- Support for restoring task on middle manager restarts by check pointing end offsets for segments
* take care of review comments
* make getCurrentOffsets call async, keep track of publishing sequence, review comments
* fix setEndoffset duplicate request handling, formatting
* fix unit test
* backward compatibility
* make AppenderatorDriverMetadata backwards compatible
* add unit test
* fix deadlock between persist and push executors in AppenderatorImpl
* fix formatting
* use persist dir instead of work dir
* review comments
* fix deadlock
* actually fix deadlock
* 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.
* use ImmutableDruidDataSource for map and set
* address comments
* unused import
* allow returning only ImmutableDruidDataSource in MetadataSegmentManager
* address comments
* remove TreeSet
* revert to use TreeSet
* use default brokerServiceName when priority is not valid
* use AtomicInteger for NodesHolder.roundRobinIndex
* revert inspectionProfiles change
* adjust TieredBrokerHostSelectorTest
* combine if statements and ensure index does not become negative
* set next index with mod if overflows
* fix codestyle
* use nextIndex
* extract the while loop to a method
* Add retries for coordinator fetch and lookup start in LookupReferencesManager
* Fix LookupConfigTest
* Address comments
* Address more comments
* And address more comments
* Address comms
* Recognize 'not found' lookups in LookupReferencesManager.tryGetLookupListFromCoordinator(), by @egor-ryashin
* 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.
* Introduce "transformSpec" at ingest-time.
It accepts a "filter" (standard query filter object) and "transforms" (a
list of objects with "name" and "expression"). These can be used to do
filtering and single-row transforms without need for a separate data
processing job.
The "expression" fields use the same expression language as other
expression-based feature.
* Remove forbidden api.
* Fix compile error.
* Fix tests.
* Some more changes.
- Add nullable annotation to Firehose.nextRow.
- Add tests for index task, realtime task, kafka task, hadoop mapper,
and ingestSegment firehose.
* Fix bad merge.
* Adjust imports.
* Adjust whitespace.
* Make Transform into an interface.
* Add missing annotation.
* Switch logger.
* Switch logger.
* Adjust test.
* Adjustment to handling for DatasourceIngestionSpec.
* Fix test.
* CR comments.
* Remove unused method.
* Add javadocs.
* More javadocs, and always decorate.
* Fix bug in TransformingStringInputRowParser.
* Fix bad merge.
* Fix ISFF tests.
* Fix DORC test.
* Fix binary serialization in caching
The previous caching code just concatenated a list of objects into a
byte array -- this is actually not valid because jackson-databind uses
enumerated references to strings internally, and concatenating multiple
binary serialized objects can throw off the references.
This change uses a single JsonGenerator to serialize the object list
rather than concatenating byte arrays.
* remove unused imports
* introducing CuratorLoadQueuePeon
* HttpLoadQueuePeon based off of current code
* Revert "Remove SegmentLoaderConfig.numLoadingThreads config (#4829)"
This reverts commit d8b3bfa63c.
* SegmentLoadDropHandler copy/pasted from ZkCoordinator
* Revert "1-based counts in ZkCoordinator (#4917)"
This reverts commit e725ff4146.
* remove non-zk part from ZkCoordinator
* remove zk part from SegmentLoadDropHandler
* additional changes for segment load/drop management with http
* address review comments
* add some more logs
* Execs class is moved
* Only consider loaded replicants when computing replication status.
This affects the computation of segment/underReplicated/count and
segment/unavailable/count, as well as the loadstatus?simple and
loadstatus?full APIs.
I'm not sure why they currently consider segments in the load
queues, but it would make more sense to me if they only considered
segments that are actually loaded.
* Fix tests.
* Fix imports.
* Changes for lookup synchronization
* Refactor of Lookup classes
* Minor refactors and doc update
* Change coordinator instance to be retrieved by DruidLeaderClient
* Wait before thread shutdown
* Make disablelookups flag true by default
* Update docs
* Rename flag
* Move executorservice shutdown to finally block
* Update LookupConfig
* Refactoring and doc changes
* Remove lookup config constructor
* Revert Lookupconfig constructor changes
* Add tests to LookupConfig
* Make executorservice local
* Update LRM
* Move ListeningScheduledExecutorService to ExecutorCompletionService
* Move exception to outer block
* Remove check to see future is done
* Remove unnecessary assignment
* Add logging
* Add ability to optionally specify a sequence identifier to reduce the possibility of duplicate events entering the event receiver firehose
* Add ability to optionally specify a sequence identifier to reduce the possibility of duplicate events entering the event receiver firehose
* Add a hard coded limit to the maximum number of possible producer IDs to prevent a malicious (or uninformed) client from overflowing the heap
* add configs to enable fast request failure on broker
* address review comments
* fix styling error
* fix style error
* have enableRequestLimit config instead of having user specify max limit
* add comment
* fix style error
* add UT fo LimitRequestsFilter
* address review comments
* fix test
* make LimitRequestsFilterTest more robust
* fix JettyQosTest
* Fix#4647
* NPE protect bucketInterval as well
* Add test to verify timezone as well
* Also handle case when intervals are already present
* Fix checkstyle error
* Use factory method instead for Datetime
* Use Intervals factory method
* Priority on loading for primary replica
* Simplicity fixes
* Fix on skipping drop for quick return.
* change to debug logging for no replicants.
* Fix on filter logic
* swapping if-else
* Fix on wrong "hasTier" logic
* Refactoring of LoadRule
* Rename createPredicate to createLoadQueueSizeLimitingPredicate
* Rename getHolderList to getFilteredHolders
* remove varargs
* extract out currentReplicantsInTier
* rename holders to holdersInTier
* don't do temporary removal of tier.
* rename primaryTier to tierToSkip
* change LinkedList to ArrayList
* Change MinMaxPriorityQueue in DruidCluster to TreeSet.
* Adding some comments.
* Modify log messages in light of predicates.
* Add in-method comments
* Don't create new Object2IntOpenHashMap for each run() call.
* Cache result from strategy call in the primary assignment to be reused during the same run.
* Spelling mistake
* Cleaning up javadoc.
* refactor out loading in progress check.
* Removed redundant comment.
* Removed forbidden API
* Correct non-forbidden API.
* Precision in variable type for NavigableSet.
* Obsolete comment.
* Clarity in method call and moving retrieval of ServerHolder into method call.
* Comment on mutability of CoordinatoorStats.
* Added auxiliary fixture for dropping.
* Add identity to query metrics, logs.
Also fix a bug where unauthorized requests would not emit any logs or metrics,
and instead would log a "Tried to emit logs and metrics twice" warning.
Also rename QueryResource's "getServer" to "cancelQuery", because that's what
it does.
* Do not emit identity by default.
* remove ServerConfig from DruidNode as all information needs to be present in DruidNode serialized form
* sanitize output of /druid/coordinator/v1/cluster endpoint
* Added org.joda.time.DateTime#(java.lang.String) to forbidden API.
* Added org.joda.time.DateTime#(java.lang.String, org.joda.time.format.DateTimeFormatter) to forbidden API.
* Add additional APIs that may create DateTime with default time zone
* Add helper function that accepts formatter to parse String.
* Add additional forbidden APIs
* Replace existing usage of forbidden APIs
* Use wrapper class to enforce Chronology on DateTimeFormatter.
* Creates constant UtcFormatter for constant ISODateTimeFormat.
* Move caffeine out of extension.
* Remove `JsonTypeName` from the class itself
* Fix bad docs
* Fix distribution pom
* Fix unused import
* Make caffeine default
* Address code comments
* Add more description around the jre version in the readme
* Add suggested comments
* Move emitters from io.druid.server.initialization to the dedicated io.druid.server.emitter package; Update emitter library to 0.6.0; Add support for ParametrizedUriEmitter; Support hierarical properties in JsonConfigurator (was needed for ParametrizedUriEmitter)
* Log created RequestLoggers
* Fix forbidden API
* Test fix
* More Http and Parametrized Http Emitter docs
* Switch to debug level
* fixes HttpServerInventoryView to call server/segment callbacks correctly and Unit Tests for the class
* fix checkstyle and forbidden-api errors
* HttpServerInventoryView to finish start() only after server inventory is initialized
* fix compilation errors
* address review comments
* add exponential backoff instead of fixed 5 secs on successive failures
* update test to exercise server fail scenarios
* use AtomicInteger for requestNum and increment only once
* Move scan-query from a contrib extension into core.
Based on a proposal at: https://groups.google.com/d/topic/druid-development/ME_OatUDnbk/discussion
This patch also adds support for virtual columns to the Scan query,
and updates Druid SQL to use Scan instead of Select.
This patch also makes some behavioral changes to handling of the __time
column. In particular, it is now is returned as "__time" rather than
"timestamp"; it is no longer included if you do not specifically ask for
it in your "columns"; and it is returned as a long rather than a string.
Users can revert time handling to the legacy extension behavior by
setting "legacy" : true in their queries, or setting the property
druid.query.scan.legacy = true. This is meant to provide a migration
path for users that were formerly using the contrib extension.
* Adjustments from review.
* Add back Select query.
* Adjust SQL docs.
* Restore SelectQuery link.
* SQL: Full TRIM support.
- Support trimming arbitrary characters
- Support BOTH, LEADING, and TRAILING
* Remove unused import.
* Fix tests, add RTRIM / LTRIM.
* Remove unused imports.
* BTRIM and docs.
* Replace for with foreach.
* Use internal-discovery and http for talking to overlord/coordinator leaders
* CuratorDruidNodeDiscovery.getAllNodes() best effort 30 sec wait for cache initialization
* DruidLeaderClientProvider to eagerly instantiate DruidNodeDiscovery when needed so that DruidNodeDiscovery impl cache gets initialized well in time
* Revert "DruidLeaderClientProvider to eagerly instantiate DruidNodeDiscovery when needed so that DruidNodeDiscovery impl cache gets initialized well in time"
This reverts commit f1a2432614ba56ddc2d55fe47e990d17fcfd6129.
* add lifecycle to DruidLeaderClient to early initialize DruidNodeDiscovery so that it has its cache update well in time
Expose version property for so that it may serialize/deserialize correctly
Expose version property for `CustomVersioningPolicy` so that it may serialize/deserialize correctly
Expose version property for CustomVersioningPolicy so that it may serialize/deserialize correctly
Expose version property for `CustomVersioningPolicy` so that it may serialize/deserialize correctly
* DruidLeaderSelector interface for leader election and Curator based impl. DruidCoordinator/TaskMaster are updated to use the new interface.
* add fake DruidNode binding in integration-tests module
* add docs on DruidLeaderSelector interface
* remove start/stop and keep register/unregister Listener in DruidLeaderSelector interface
* updated comments on DruidLeaderSelector
* cache the listener executor in CuratorDruidLeaderSelector
* use same latch owner name that was used before
* remove stuff related to druid.zk.paths.indexer.leaderLatchPath config
* randomize the delay when giving up leadership and restarting leader latch
* Factor QueryableIndexColumnSelectorFactory and IncrementalIndexColumnSelectorFactory out of QueryableIndexStorageAdapter and IncrementalIndexStorageAdapter; Add Offset.getBaseReadableOffset(); Remove OffsetHolder interface; Replace Cursor extends ColumnSelectorFactory with composition; Reduce indirection in ColumnValueSelectors created by QueryableIndexColumnSelectorFactory
* Don't override clone() in FilteredOffset (the prev. implementation was broken); Some warnings fixed
* Simplify Cursors in QueryableIndexStorageAdapter
* Address comments
* Remove unused and unimplemented methods from GenericColumn interface
* Comments
* Add @ExtensionPoint and @PublicApi annotations.
* Clean up wording.
* Remove unused import.
* Remove unused imports.
* Only types can be extension points.
* Adjust annotations some more.
* Remove unused import.
* Make ServletFilterHolder an extension point.
* Add a couple extension points, and update docs.
* Do not remove segment that should not be moved from currentlyMovingSegments (segments are removed by callbacks or not inserted)
* Mark segments that are going to be dropped from server and use this information in CostBalancerStrategy
* Fix tests
* update LookupCoordinatorManager to use internal discovery to discover lookup nodes
* router:use internal-discovery to discover brokers
* minor [Curator]DruidDiscoveryProvider refactoring
* add initialized() method to DruidNodeDiscovery.Listener
* update HttpServerInventoryView to use initialized() and call segment callback initialization asynchronously
* Revert "update HttpServerInventoryView to use initialized() and call segment callback initialization asynchronously"
This reverts commit f796e441221fe8b0e9df87fdec6c9f47bcedd890.
* Revert "add initialized() method to DruidNodeDiscovery.Listener"
This reverts commit f0661541d073683f28fce2dd4f30ec37db90deb0.
* minor refactoring removing synchronized from DruidNodeDiscoveryProvider
* updated DruidNodeDiscovery.Listener contract to take List of nodes and first call marks initialization
* update HttpServerInventoryView to handle new contract and handle initialization
* router update to handle updated listener contract
* document DruidNodeDiscovery.Listener contract
* fix forbidden-api error
* change log level to info for unknown path children cache events in CuratorDruidNodeDiscoveryProvider
* announce broker only after segment inventory is initialized
* internal-discovery: interfaces for announcement/discovery, curator impls
* more tests
* address some review comments
* more fixes
* address more review comments
* simplify ObjectMapper setup in CuratorDruidNodeAnnouncerAndDiscoveryTest
* fix KafkaIndexTaskTest
* make lookupTier overridable via RealtimeIndexTask and KafkaIndexTask context
* make teamcity build happy
* Do not remove segment that should not be moved from currentlyMovingSegments (segments are removed by callbacks or not inserted)
* Replace putIfAbsent with computeIfAbsent in DruidBalancer
* Refactoring
* make BrokerQueryResource instantiation singleton
* fix druid.router.http.* handling so that they are actually used and introduce numRequestsQueued for jetty http client at router
* address comments
* address review comment
* Use ConcurrentHashMap to store segment servers or else getInventory() would need to clone the values list
* introduce unstableTimeout for segment servers
* address review comment
* add HttpServerInventoryViewConfigTest
* 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.
* Rename ResourceManagementStrategy to ProvisioningStrategy, similarly for related classes. Make ProvisioningService non-global, created per RemoteTaskRunner instead. Add OverlordBlinkLeadershipTest.
* Fix RemoteTaskRunnerFactoryTest.testExecNotSharedBetweenRunners()
* Small fix
* Make SimpleProvisioner and PendingProvisioner more similar in details
* Fix executor name
* Style fixes
* Use LifecycleLock in RemoteTaskRunner
* 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
* 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 DruidProcessingModule, QueryableModule and QueryRunnerFactoryModule from DI for coordinator, overlord, middle-manager. Add RouterDruidProcessing not to allocate processing resources on router
* Fix examples
* Fixes
* Revert Peon configs and add comments
* Remove qualifier
* Add maxSegmentsInQueue parameter to CoordinatorDinamicConfig and use it in LoadRule to improve segments loading and replication time
* Rename maxSegmentsInQueue to maxSegmentsInNodeLoadingQueue
* Make CoordinatorDynamicConfig constructor private; add/fix tests; set default maxSegmentsInNodeLoadingQueue to 0 (unbounded)
* Docs added for maxSegmentsInNodeLoadingQueue parameter in CoordinatorDynamicConfig
* More docs for maxSegmentsInNodeLoadingQueue and style fixes
* 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
* 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
* 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
* Do not re-create prioritized servers on each call in server selector and extend TierSelectorStrategy interface with a method to pick multiple elements at once
* Fix compilation
* Replaces use of CountingMap with Object2LongMap from fastutil.
* Remove CountingMap classes and minor fixes
* Added additional test cases for DatasourceInputFormat.
* Added additional test cases for CoordinatorStats.
* Not materializing segment list.
* Put in this fix because it is failing the test on its expected behavior.
* Added missing header.
* Improve concurrency of SegmentManager
* Fix SegmentManager and add more tests
* Add more tests
* Add null check to TimelineEntry
* Remove empty data source and check null in getTimeline()
* Add a comment for returning null in compute()
* Make SegmentManager LazySingleton