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
* 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
* 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
* Add extension for supporting kerberos security
- This PR adds an extension for supporting druid authentication via
Kerberos.
- Working on the docs.
* Add docs
* review comments
* more review comments
* Block all paths by default
* more review comments - use proper Oid
* Allow extensions to override httpclient for integration tests
* Add kerberos lock to prevent multithreaded issues.
* review comment - remove enabled flag and fix router injection
* Add Cookie Handling and more detailed docs
* review comment - rename DruidKerberosConfig -> AuthKerberosConfig
* review comments
* fix travis failure on jdk7
* 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
* 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.