Commit Graph

383 Commits

Author SHA1 Message Date
Gian Merlino d5abd06b96
Fix flaky KafkaIndexTaskTest. (#12657)
* Fix flaky KafkaIndexTaskTest.

The testRunTransactionModeRollback case had many race conditions. Most notably,
it would commit a transaction and then immediately check to see that the results
were *not* indexed. This is racey because it relied on the indexing thread being
slower than the test thread.

Now, the case waits for the transaction to be processed by the indexing thread
before checking the results.

* Changes from review.
2022-06-24 13:53:51 -07:00
Abhishek Agarwal 2fe053c5cb
Bump up the versions (#12480) 2022-04-27 14:28:20 +05:30
Jihoon Son b6eeef31e5
Store null columns in the segments (#12279)
* Store null columns in the segments

* fix test

* remove NullNumericColumn and unused dependency

* fix compile failure

* use guava instead of apache commons

* split new tests

* unused imports

* address comments
2022-03-23 16:54:04 -07:00
Parag Jain 2efb74ff1e
fix supervisor auto scaler config serde bug (#12317) 2022-03-09 16:17:12 -08:00
Abhishek Agarwal 6346b9561d
Reuse the InputEntityReader in SettableByteEntityReader (#12269)
* Reuse the InputEntityReader in SettableByteEntityReader

* Fix logic

* Fix kafka streaming ingestion

* Add Tests for kafka input format change

* Address review comments
2022-03-09 14:38:31 -08:00
Laksh Singla 3f709db173
Make ParseExceptions more informative (#12259)
This PR aims to make the ParseExceptions in Druid more informative, by adding additional information (metadata) to the ParseException, which can contain additional information about the exception. For example - the path of the file generating the issue, the line number (where it can be easily fetched - like CsvReader)

Following changes are addressed in this PR:

A new class CloseableIteratorWithMetadata has been created which is like CloseableIterator but also has a metadata method that returns a context Map<String, Object> about the current element returned by next().
IntermediateRowParsingReader#read() now attaches the InputEntity and the "record number" which created the exception (while parsing them), and IntermediateRowParsingReader#sample attaches the InputEntity (but not the "record number").
TextReader (and its subclasses), which is a specific implementation of the IntermediateRowParsingReader also include the line number which caused the generation of the error.
This will also help in triaging the issues when InputSourceReader generates ParseException because it can point to the specific InputEntity which caused the exception (while trying to read it).
2022-02-28 22:31:15 +05:30
Jihoon Son e5ad862665
A new includeAllDimension flag for dimensionsSpec (#12276)
* includeAllDimensions in dimensionsSpec

* doc

* address comments

* unused import and doc spelling
2022-02-25 18:27:48 -08:00
Xavier Léauté 009dd9e09a
upgrade core Apache Kafka dependencies to 3.1.0 (#12203)
Announcement: https://blogs.apache.org/kafka/entry/what-s-new-in-apache7
Release notes: https://dist.apache.org/repos/dist/release/kafka/3.1.0/RELEASE_NOTES.html

* upgrade core Apache Kafka dependencies to 3.1.0
* fix use of private Kafka APIs
* remove deprecated test rules
* remove mock calls that weren't verified in the first place
* remove the need for powermock in KafkaLookupExtractorFactoryTest
* align curator-test version with curator itself
* update easymock to 4.3.0
2022-02-23 18:42:51 -08:00
Abhishek Agarwal 53c0e489c2
Fix infinite retrying during task pausing (#12167)
This fixes a bug that causes TaskClient in overlord to continuously retry to pause tasks. This can happen when a task is not responding to the pause command. Ideally, in such a case when the task is unresponsive, the overlord would have given up after a few retries and would have killed the task. However, due to this bug, retries go on forever.
2022-01-19 09:03:36 +05:30
Jonathan Wei 229f82a6f0
Add parse error list API for stream supervisors, use structured object for parse exceptions, simplify parse exception message (#11961)
* Add parse error list API for stream supervisors, simplify parse exception message

* Add input string to parse exception

* Use structured ParseExceptionReport

* Fix tests

* Add test

* PR comments, add ParseExceptionReport equals verifier

* Fix test
2021-12-09 15:42:55 -06:00
Gian Merlino e0e05aad99
Enhancements to IndexTaskClient. (#12011)
* Enhancements to IndexTaskClient.

1) Ability to use handlers other than StringFullResponseHandler. This
   functionality is not used in production code yet, but is useful
   because it will allow tasks to communicate with each other in
   non-string-based formats and in streaming fashion. In the future,
   we'll be able to use this to make task-to-task communication
   more efficient.

2) Truncate server errors at 1KB, so long errors do not pollute logs.

3) Change error log level for retryable errors from WARN to INFO. (The
   final error is still WARN.)

4) Harmonize log and exception messages to have a more consistent format.

* Additional tests and improvements.
2021-12-03 09:14:32 -08:00
Frank Chen c2cea25a6b
Improve exception message when loading data from web-console (#11723)
* Improve exception handling

* Revert some changes

* Resolve comments

* Update indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerExceptionMapper.java

Co-authored-by: Karan Kumar <karankumar1100@gmail.com>

* Update indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/SamplerExceptionMapper.java

Co-authored-by: Karan Kumar <karankumar1100@gmail.com>

* Address review comments

Co-authored-by: Karan Kumar <karankumar1100@gmail.com>
2021-12-03 21:33:49 +08:00
lokesh-lingarajan ad6609a606
Kafka Input Format for headers, key and payload parsing (#11630)
### Description

Today we ingest a number of high cardinality metrics into Druid across dimensions. These metrics are rolled up on a per minute basis, and are very useful when looking at metrics on a partition or client basis. Events is another class of data that provides useful information about a particular incident/scenario inside a Kafka cluster. Events themselves are carried inside kafka payload, but nonetheless there are some very useful metadata that is carried in kafka headers that can serve as useful dimension for aggregation and in turn bringing better insights.

PR(https://github.com/apache/druid/pull/10730) introduced support of Kafka headers in InputFormats.

We still need an input format to parse out the headers and translate those into relevant columns in Druid. Until that’s implemented, none of the information available in the Kafka message headers would be exposed. So first there is a need to write an input format that can parse headers in any given format(provided we support the format) like we parse payloads today. Apart from headers there is also some useful information present in the key portion of the kafka record. We also need a way to expose the data present in the key as druid columns. We need a generic way to express at configuration time what attributes from headers, key and payload need to be ingested into druid. We need to keep the design generic enough so that users can specify different parsers for headers, key and payload.

This PR is designed to solve the above by providing wrapper around any existing input formats and merging the data into a single unified Druid row.

Lets look at a sample input format from the above discussion

"inputFormat":
{
    "type": "kafka",     // New input format type
    "headerLabelPrefix": "kafka.header.",   // Label prefix for header columns, this will avoid collusions while merging columns
    "recordTimestampLabelPrefix": "kafka.",  // Kafka record's timestamp is made available in case payload does not carry timestamp
    "headerFormat":  // Header parser specifying that values are of type string
    {
        "type": "string"
    },
    "valueFormat": // Value parser from json parsing
    {
        "type": "json",
        "flattenSpec": {
          "useFieldDiscovery": true,
          "fields": [...]
        }
    },
    "keyFormat":  // Key parser also from json parsing
    {
        "type": "json"
    }
}

Since we have independent sections for header, key and payload, it will enable parsing each section with its own parser, eg., headers coming in as string and payload as json. 

KafkaInputFormat will be the uber class extending inputFormat interface and will be responsible for creating individual parsers for header, key and payload, blend the data resolving conflicts in columns and generating a single unified InputRow for Druid ingestion. 

"headerFormat" will allow users to plug parser type for the header values and will add default header prefix as "kafka.header."(can be overridden) for attributes to avoid collision while merging attributes with payload.

Kafka payload parser will be responsible for parsing the Value portion of the Kafka record. This is where most of the data will come from and we should be able to plugin existing parser. One thing to note here is that if batching is performed, then the code is augmenting header and key values to every record in the batch.

Kafka key parser will handle parsing Key portion of the Kafka record and will ingest the Key with dimension name as "kafka.key".

## KafkaInputFormat Class: 
This is the class that orchestrates sending the consumerRecord to each parser, retrieve rows, merge the columns into one final row for Druid consumption. KafkaInputformat should make sure to release the resources that gets allocated as a part of reader in CloseableIterator<InputRow> during normal and exception cases.

During conflicts in dimension/metrics names, the code will prefer dimension names from payload and ignore the dimension either from headers/key. This is done so that existing input formats can be easily migrated to this new format without worrying about losing information.
2021-10-07 08:56:27 -07:00
Xavier Léauté bc3b038712
Update Apache Kafka client libraries to 3.0.0 (#11735)
Release notes:
https://downloads.apache.org/kafka/3.0.0/RELEASE_NOTES.html
https://blogs.apache.org/kafka/entry/what-s-new-in-apache6
2021-10-05 10:23:19 -07:00
Clint Wylie fe1d8c206a
bump version to 0.23.0-SNAPSHOT (#11670) 2021-09-08 15:56:04 -07:00
Agustin Gonzalez 9efa6cc9c8
Make persists concurrent with adding rows in batch ingestion (#11536)
* Make persists concurrent with ingestion

* Remove semaphore but keep concurrent persists (with add) and add push in the backround as well

* Go back to documented default persists (zero)

* Move to debug

* Remove unnecessary Atomics

* Comments on synchronization (or not) for sinks & sinkMetadata

* Some cleanup for unit tests but they still need further work

* Shutdown & wait for persists and push on close

* Provide support for three existing batch appenderators using batchProcessingMode flag

* Fix reference to wrong appenderator

* Fix doc typos

* Add BatchAppenderators class test coverage

* Add log message to batchProcessingMode final value, fix typo in enum name

* Another typo and minor fix to log message

* LEGACY->OPEN_SEGMENTS, Edit docs

* Minor update legacy->open segments log message

* More code comments, mostly small adjustments to naming etc

* fix spelling

* Exclude BtachAppenderators from Jacoco since it is fully tested but Jacoco still refuses to ack coverage

* Coverage for Appenderators & BatchAppenderators, name change of a method that was still using "legacy" rather than "openSegments"

Co-authored-by: Clint Wylie <cjwylie@gmail.com>
2021-09-08 13:31:52 -07:00
Agustin Gonzalez a2da407b70
Add error msg to parallel task's TaskStatus (#11486)
* Add error msg to parallel task's TaskStatus

* Consolidate failure block

* Add failure test

* Make it fail

* Add fail while stopped

* Simplify hash task test using a runner that fails after so many runs (parameter)

* Remove unthrown exception

* Use runner names to identify phase

* Added range partition kill test & fixed a timing bug with the custom runner

* Forbidden api

* Style

* Unit test code cleanup

* Added message to invalid state exception and improved readability  of the phase error messages for the parallel task failure unit tests
2021-08-02 12:11:28 -07:00
Abhishek Agarwal 94c1671eaf
Split SegmentLoader into SegmentLoader and SegmentCacheManager (#11466)
This PR splits current SegmentLoader into SegmentLoader and SegmentCacheManager.

SegmentLoader - this class is responsible for building the segment object but does not expose any methods for downloading, cache space management, etc. Default implementation delegates the download operations to SegmentCacheManager and only contains the logic for building segments once downloaded. . This class will be used in SegmentManager to construct Segment objects.

SegmentCacheManager - this class manages the segment cache on the local disk. It fetches the segment files to the local disk, can clean up the cache, and in the future, support reserve and release on cache space. [See https://github.com/Make SegmentLoader extensible and customizable #11398]. This class will be used in ingestion tasks such as compaction, re-indexing where segment files need to be downloaded locally.
2021-07-21 00:14:19 +05:30
frank chen 2236cf2234
eliminate extra object instantiation (#11345) 2021-07-12 18:31:39 -07:00
Abhishek Agarwal 03a6a6d6e1
Replace Processing ExecutorService with QueryProcessingPool (#11382)
This PR refactors the code for QueryRunnerFactory#mergeRunners to accept a new interface called QueryProcessingPool instead of ExecutorService for concurrent execution of query runners. This interface will let custom extensions inject their own implementation for deciding which query-runner to prioritize first. The default implementation is the same as today that takes the priority of query into account. QueryProcessingPool can also be used as a regular executor service. It has a dedicated method for accepting query execution work so implementations can differentiate between regular async tasks and query execution tasks. This dedicated method also passes the QueryRunner object as part of the task information. This hook will let custom extensions carry any state from QuerySegmentWalker to QueryProcessingPool#mergeRunners which is not possible currently.
2021-07-01 16:03:08 +05:30
Yi Yuan de8daf8139
Delete buildV9Directly in Kafka and Kinesis Indexing Service (#11351)
* delete_buildV9Directly_in_kafka_and_kinesis_indexing_service

* delete

* delete them from server

* delete buildV9Directly from hadoop indexing

* bug fixed

Co-authored-by: yuanyi <yuanyi@freewheel.tv>
2021-06-23 16:36:46 -07:00
Xavier Léauté b517c3339b
remove ZooKeeper 3.4 support + pass tests with Java 15 (#11073)
With this change, Druid will only support ZooKeeper 3.5.x and later.

In order to support Java 15 we need to switch to ZK 3.5.x client libraries and drop support for ZK 3.4.x
(see #10780 for the detailed reasons) 

* remove ZooKeeper 3.4.x compatibility
* exclude additional ZK 3.5.x netty dependencies to ensure we use our version
* keep ZooKeeper version used for integration tests in sync with client library version
* remove the need to specify ZK version at runtime for docker
* add support to run integration tests with JDK 15
* build and run unit tests with Java 15 in travis
2021-05-25 12:49:49 -07:00
Agustin Gonzalez 8e5048e643
Avoid memory mapping hydrants after they are persisted & after they are merged for native batch ingestion (#11123)
* Avoid mapping hydrants in create segments phase for native ingestion

* Drop queriable indices after a given sink is fully merged

* Do not drop memory mappings for realtime ingestion

* Style fixes

* Renamed to match use case better

* Rollback memoization code and use the real time flag instead

* Null ptr fix in FireHydrant toString plus adjustments to memory pressure tracking calculations

* Style

* Log some count stats

* Make sure sinks size is obtained at the right time

* BatchAppenderator unit test

* Fix comment typos

* Renamed methods to make them more readable

* Move persisted metadata from FireHydrant class to AppenderatorImpl. Removed superfluous differences and fix comment typo. Removed custom comparator

* Missing dependency

* Make persisted hydrant metadata map concurrent and better reflect the fact that keys are Java references. Maintain persisted metadata when dropping/closing segments.

* Replaced concurrent variables with normal ones

* Added   batchMemoryMappedIndex "fallback" flag with default "false". Set this to "true" make code fallback to previous code path.

* Style fix.

* Added note to new setting in doc, using Iterables.size (and removing a dependency), and fixing a typo in a comment.

* Forgot to commit this edited documentation message
2021-05-11 14:34:26 -07:00
Yuanli Han 8647040f4d
Allow user to set group.id for Kafka ingestion task (#11147)
* allow user to set group.id for Kafka ingestion task

* fix test coverage by removing deprecated code and add doc

* fix typo

* Update docs/development/extensions-core/kafka-ingestion.md

Co-authored-by: frank chen <frankchen@apache.org>

Co-authored-by: frank chen <frankchen@apache.org>
2021-05-09 11:56:19 +08:00
Xavier Léauté 0296f20551
upgrade Apache Kafka to 2.8.0 (#11139)
* upgrade to Apache Kafka 2.8.0 (release notes:
  https://downloads.apache.org/kafka/2.8.0/RELEASE_NOTES.html)
* pass Kafka version as a Docker argument in integration tests
  to keep in sync with maven version
* fix use of internal Kafka APIs in integration tests
2021-04-24 08:27:07 -07:00
Lucas Capistrant 8264203cee
Allow client to configure batch ingestion task to wait to complete until segments are confirmed to be available by other (#10676)
* Add ability to wait for segment availability for batch jobs

* IT updates

* fix queries in legacy hadoop IT

* Fix broken indexing integration tests

* address an lgtm flag

* spell checker still flagging for hadoop doc. adding under that file header too

* fix compaction IT

* Updates to wait for availability method

* improve unit testing for patch

* fix bad indentation

* refactor waitForSegmentAvailability

* Fixes based off of review comments

* cleanup to get compile after merging with master

* fix failing test after previous logic update

* add back code that must have gotten deleted during conflict resolution

* update some logging code

* fixes to get compilation working after merge with master

* reset interrupt flag in catch block after code review pointed it out

* small changes following self-review

* fixup some issues brought on by merge with master

* small changes after review

* cleanup a little bit after merge with master

* Fix potential resource leak in AbstractBatchIndexTask

* syntax fix

* Add a Compcation TuningConfig type

* add docs stipulating the lack of support by Compaction tasks for the new config

* Fixup compilation errors after merge with master

* Remove erreneous newline
2021-04-08 21:03:00 -07:00
Gian Merlino bf20f9e979
DruidInputSource: Fix issues in column projection, timestamp handling. (#10267)
* DruidInputSource: Fix issues in column projection, timestamp handling.

DruidInputSource, DruidSegmentReader changes:

1) Remove "dimensions" and "metrics". They are not necessary, because we
   can compute which columns we need to read based on what is going to
   be used by the timestamp, transform, dimensions, and metrics.
2) Start using ColumnsFilter (see below) to decide which columns we need
   to read.
3) Actually respect the "timestampSpec". Previously, it was ignored, and
   the timestamp of the returned InputRows was set to the `__time` column
   of the input datasource.

(1) and (2) together fix a bug in which the DruidInputSource would not
properly read columns that are used as inputs to a transformSpec.

(3) fixes a bug where the timestampSpec would be ignored if you attempted
to set the column to something other than `__time`.

(1) and (3) are breaking changes.

Web console changes:

1) Remove "Dimensions" and "Metrics" from the Druid input source.
2) Set timestampSpec to `{"column": "__time", "format": "millis"}` for
   compatibility with the new behavior.

Other changes:

1) Add ColumnsFilter, a new class that allows input readers to determine
   which columns they need to read. Currently, it's only used by the
   DruidInputSource, but it could be used by other columnar input sources
   in the future.
2) Add a ColumnsFilter to InputRowSchema.
3) Remove the metric names from InputRowSchema (they were unused).
4) Add InputRowSchemas.fromDataSchema method that computes the proper
   ColumnsFilter for given timestamp, dimensions, transform, and metrics.
5) Add "getRequiredColumns" method to TransformSpec to support the above.

* Various fixups.

* Uncomment incorrectly commented lines.

* Move TransformSpecTest to the proper module.

* Add druid.indexer.task.ignoreTimestampSpecForDruidInputSource setting.

* Fix.

* Fix build.

* Checkstyle.

* Misc fixes.

* Fix test.

* Move config.

* Fix imports.

* Fixup.

* Fix ShuffleResourceTest.

* Add import.

* Smarter exclusions.

* Fixes based on tests.

Also, add TIME_COLUMN constant in the web console.

* Adjustments for tests.

* Reorder test data.

* Update docs.

* Update docs to say Druid 0.22.0 instead of 0.21.0.

* Fix test.

* Fix ITAutoCompactionTest.

* Changes from review & from merging.
2021-03-25 10:32:21 -07:00
Maytas Monsereenusorn 4dd22a850b
Fix streaming ingestion fails if it encounters empty rows (Regression) (#10962)
* Fix streaming ingestion fails and halt if it  encounters empty rows

* address comments
2021-03-09 12:11:58 -08:00
zhangyue19921010 bddacbb1c3
Dynamic auto scale Kafka-Stream ingest tasks (#10524)
* druid task auto scale based on kafka lag

* fix kafkaSupervisorIOConfig and KinesisSupervisorIOConfig

* druid task auto scale based on kafka lag

* fix kafkaSupervisorIOConfig and KinesisSupervisorIOConfig

* test dynamic auto scale done

* auto scale tasks tested on prd cluster

* auto scale tasks tested on prd cluster

* modify code style to solve 29055.10 29055.9 29055.17 29055.18 29055.19 29055.20

* rename test fiel function

* change codes and add docs based on capistrant reviewed

* midify test docs

* modify docs

* modify docs

* modify docs

* merge from master

* Extract the autoScale logic out of SeekableStreamSupervisor to minimize putting more stuff inside there &&  Make autoscaling algorithm configurable and scalable.

* fix ci failed

* revert msic.xml

* add uts to test autoscaler create && scale out/in and kafka ingest with scale enable

* add more uts

* fix inner class check

* add IT for kafka ingestion with autoscaler

* add new IT in groups=kafka-index named testKafkaIndexDataWithWithAutoscaler

* review change

* code review

* remove unused imports

* fix NLP

* fix docs and UTs

* revert misc.xml

* use jackson to build autoScaleConfig with default values

* add uts

* use jackson to init AutoScalerConfig in IOConfig instead of Map<>

* autoscalerConfig interface and provide a defaultAutoScalerConfig

* modify uts

* modify docs

* fix checkstyle

* revert misc.xml

* modify uts

* reviewed code change

* reviewed code change

* code reviewed

* code review

* log changed

* do StringUtils.encodeForFormat when create allocationExec

* code review && limit taskCountMax to partitionNumbers

* modify docs

* code review

Co-authored-by: yuezhang <yuezhang@freewheel.tv>
2021-03-06 14:36:52 +05:30
Abhishek Agarwal 96d26e5338
Fix kinesis ingestion bugs (#10761)
* add offsetFetchPeriod to kinesis ingestion doc

* Remove jackson dependencies from extensions

* Use fixed delay for lag collection

* Metrics reset after finishing processing

* comments

* Broaden the list of exceptions to retry for

* Unit tests

* Add more tests

* Refactoring

* re-order metrics

* Doc suggestions

Co-authored-by: Charles Smith <38529548+techdocsmith@users.noreply.github.com>

* Add tests

Co-authored-by: Charles Smith <38529548+techdocsmith@users.noreply.github.com>
2021-02-05 02:49:58 -08:00
Maytas Monsereenusorn a46d561bd7
Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead (#10740)
* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* fix checkstyle

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* fix test

* fix test

* add log

* Fix byte calculation for maxBytesInMemory to take into account of Sink/Hydrant Object overhead

* address comments

* fix checkstyle

* fix checkstyle

* add config to skip overhead memory calculation

* add test for the skipBytesInMemoryOverheadCheck config

* add docs

* fix checkstyle

* fix checkstyle

* fix spelling

* address comments

* fix travis

* address comments
2021-01-27 00:34:56 -08:00
Jihoon Son 95065bdf1a
Bump dev version to 0.22.0-SNAPSHOT (#10759) 2021-01-15 13:16:23 -08:00
Xavier Léauté 118b50195e
Introduce KafkaRecordEntity to support Kafka headers in InputFormats (#10730)
Today Kafka message support in streaming indexing tasks is limited to
message values, and does not provide a way to expose Kafka headers,
timestamps, or keys, which may be of interest to more specialized
Druid input formats. For instance, Kafka headers may be used to indicate
payload format/encoding or additional metadata, and timestamps are often
omitted from values in Kafka streams applications, since they are
included in the record.

This change proposes to introduce KafkaRecordEntity as InputEntity,
which would give input formats full access to the underlying Kafka record,
including headers, key, timestamps. It would also open access to low-level
information such as topic, partition, offset if needed.

KafkaEntity is a subclass of ByteEntity for backwards compatibility with
existing input formats, and to avoid introducing unnecessary complexity
for Kinesis indexing tasks.
2021-01-08 16:04:37 -08:00
Jonathan Wei 769c21cc87
Add sample method to IndexingServiceClient (#10729)
* Add sample method to IndexingServiceClient

* Add unit test

* Fix LGTM
2021-01-05 15:02:44 -08:00
Xavier Léauté b7a16d08a6
Update Apache Kafka to 2.7.0 (#10701)
- align scala versions to match Kafka
2020-12-22 13:56:00 -08:00
zhangyue19921010 229b5f359f
Remove hard limitation that druid(after 0.15.0) only can consume Kafka version 0.11.x or better (#10551)
* remove build in kafka consumer config :

* modify druid docs of kafka indexing service

* yuezhang

* modify doc

* modify docs

* fix kafkaindexTaskTest.java

* revert uncessary change

* add more logs and modify docs

* revert jdk version

* modify docs

* modify-kafka-version v2

* modify docs

* modify docs

* modify docs

* modify docs

* modify docs

* done

* remove useless import

* change code and add UT

Co-authored-by: yuezhang <yuezhang@freewheel.tv>
2020-12-03 17:37:59 -08:00
Himanshu 7e9522870f
introduce DynamicConfigProvider interface and make kafka consumer props extensible (#10309)
* introduce DynamicConfigProvider interface and make kafka consumer props extensible

* fix intellij inspection error

* make DynamicConfigProvider generic

Change-Id: I2e3e89f8617b6fe7fc96859deca4011f609dc5a3

* deprecate PasswordProvider
2020-12-02 16:38:27 -08:00
frank chen e83d5cb59e
Fix ingestion failure of pretty-formatted JSON message (#10383)
* support multi-line text

* add test cases

* split json text into lines case by case

* improve exception handle

* fix CI

* use IntermediateRowParsingReader as base of JsonReader

* update doc

* ignore the non-immutable field in test case

* add more test cases

* mark `lineSplittable` as final

* fix testcases

* fix doc

* add a test case for SqlReader

* return all raw columns when exception occurs

* fix CI

* fix test cases

* resolve review comments

* handle ParseException returned by index.add

* apply Iterables.getOnlyElement

* fix CI

* fix test cases

* improve code in more graceful way

* fix test cases

* fix test cases

* add a test case to check multiple json string in one text block

* fix inspection check
2020-11-13 13:59:23 -08:00
Liran Funaro f3a2903218
Configurable Index Type (#10335)
* Introduce a Configurable Index Type

* Change to @UnstableApi

* Add AppendableIndexSpecTest

* Update doc

* Add spelling exception

* Add tests coverage

* Revert some of the changes to reduce diff

* Minor fixes

* Update getMaxBytesInMemoryOrDefault() comment

* Fix typo, remove redundant interface

* Remove off-heap spec (postponed to a later PR)

* Add javadocs to AppendableIndexSpec

* Describe testCreateTask()

* Add tests for AppendableIndexSpec within TuningConfig

* Modify hashCode() to conform with equals()

* Add comment where building incremental-index

* Add "EqualsVerifier" tests

* Revert some of the API back to AppenderatorConfig

* Don't use multi-line comments

* Remove knob documentation (deferred)
2020-10-23 18:34:26 -07:00
Jonathan Wei 65c0d64676
Update version to 0.21.0-SNAPSHOT (#10450)
* [maven-release-plugin] prepare release druid-0.21.0

* [maven-release-plugin] prepare for next development iteration

* Update web-console versions
2020-10-03 16:08:34 -07:00
Jihoon Son 8f14ac814e
More structured way to handle parse exceptions (#10336)
* More structured way to handle parse exceptions

* checkstyle; add more tests

* forbidden api; test

* address comment; new test

* address review comments

* javadoc for parseException; remove redundant parseException in streaming ingestion

* fix tests

* unnecessary catch

* unused imports

* appenderator test

* unused import
2020-09-11 16:31:10 -07:00
Gian Merlino 8ab1979304
Remove implied profanity from error messages. (#10270)
i.e. WTF, WTH.
2020-08-28 11:38:50 -07:00
Jihoon Son f82fd22fa7
Move tools for indexing to TaskToolbox instead of injecting them in constructor (#10308)
* Move tools for indexing to TaskToolbox instead of injecting them in constructor

* oops, other changes

* fix test

* unnecessary new file

* fix test

* fix build
2020-08-26 17:08:12 -07:00
Xavier Léauté 225490474d
Update Kafka dependencies to 2.6.0 (#10286)
* update Kafka dependencies to Kafka 2.6.0
* switch to Scala 2.13 build of Kafka
* update integration tests
* update Kafka tutorial
2020-08-15 07:56:40 -07:00
Suneet Saldanha e6c9142129
Add validation for authenticator and authorizer name (#10106)
* Add validation for authorizer name

* fix deps

* add javadocs

* Do not use resource filters

* Fix BasicAuthenticatorResource as well

* Add integration tests

* fix test

* fix
2020-07-13 21:15:54 -07:00
Clint Wylie c86e7ce30b
bump version to 0.20.0-SNAPSHOT (#10124) 2020-07-06 15:08:32 -07:00
Suneet Saldanha 363d0d86be
QueryCountStatsMonitor can be injected in the Peon (#10092)
* QueryCountStatsMonitor can be injected in the Peon

This change fixes a dependency injection bug where there is a circular
dependency on getting the MonitorScheduler when a user configures the
QueryCountStatsMonitor to be used.

* fix tests

* Actually fix the tests this time
2020-06-29 21:03:07 -07:00
Aleksey Plekhanov 2c384b61ff
IntelliJ inspection and checkstyle rule for "Collection.EMPTY_* field accesses replaceable with Collections.empty*()" (#9690)
* IntelliJ inspection and checkstyle rule for "Collection.EMPTY_* field accesses replaceable with Collections.empty*()"

* Reverted checkstyle rule

* Added tests to pass CI

* Codestyle
2020-06-18 09:47:07 -07:00
Jonathan Wei 771870ae2d
Load broadcast datasources on broker and tasks (#9971)
* Load broadcast datasources on broker and tasks

* Add javadocs

* Support HTTP segment management

* Fix indexer maxSize

* inspection fix

* Make segment cache optional on non-historicals

* Fix build

* Fix inspections, some coverage, failed tests

* More tests

* Add CliIndexer to MainTest

* Fix inspection

* Rename UnprunedDataSegment to LoadableDataSegment

* Address PR comments

* Fix
2020-06-08 20:15:59 -07:00
Xavier Léauté a934b2664c
remove ListenableFutures and revert to using the Guava implementation (#9944)
This change removes ListenableFutures.transformAsync in favor of the
existing Guava Futures.transform implementation. Our own implementation
had a bug which did not fail the future if the applied function threw an
exception, resulting in the future never completing.

An attempt was made to fix this bug, however when running againts Guava's own
tests, our version failed another half dozen tests, so it was decided to not
continue down that path and scrap our own implementation.

Explanation for how was this bug manifested itself:

An exception thrown in BaseAppenderatorDriver.publishInBackground when
invoked via transformAsync in StreamAppenderatorDriver.publish will
cause the resulting future to never complete.

This explains why when encountering https://github.com/apache/druid/issues/9845
the task will never complete, forever waiting for the publishFuture to
register the handoff. As a result, the corresponding "Error while
publishing segments ..." message only gets logged once the index task
times out and is forcefully shutdown when the future is force-cancelled
by the executor.
2020-06-03 10:46:03 -07:00
Xavier Léauté 65280a6953
update kafka client version to 2.5.0 (#9902)
- remove dependency on deprecated internal Kafka classes
- keep LZ4 version in line with the version shipped with Kafka
2020-05-27 13:20:32 -07:00
Clint Wylie 2e9548d93d
refactor SeekableStreamSupervisor usage of RecordSupplier (#9819)
* refactor SeekableStreamSupervisor usage of RecordSupplier to reduce contention between background threads and main thread, refactor KinesisRecordSupplier, refactor Kinesis lag metric collection and emitting

* fix style and test

* cleanup, refactor, javadocs, test

* fixes

* keep collecting current offsets and lag if unhealthy in background reporting thread

* review stuffs

* add comment
2020-05-16 14:09:39 -07:00
mcbrewster 28be107a1c
add flag to flattenSpec to keep null columns (#9814)
* add flag to flattenSpec to keep null columns

* remove changes to inputFormat interface

* add comment

* change comment message

* update web console e2e test

* move keepNullColmns to JSONParseSpec

* fix merge conflicts

* fix tests

* set keepNullColumns to false by default

* fix lgtm

* change Boolean to boolean, add keepNullColumns to hash, add tests for keepKeepNullColumns false + true with no nuulul columns

* Add equals verifier tests
2020-05-08 21:53:39 -07:00
Jihoon Son 964a1fc9df
Remove ParseSpec.toInputFormat() (#9815)
* Remove toInputFormat() from ParseSpec

* fix test
2020-05-05 11:17:57 -07:00
Maytas Monsereenusorn 8b78eebdbd
Test reading from empty kafka/kinesis partitions (#9729)
* add test for stream sequence number returns null

* fix checkstyle

* add index test for when stream returns null

* retrigger test
2020-04-27 10:23:56 -07:00
Clint Wylie d267b1c414
check paths used for shuffle intermediary data manager get and delete (#9630)
* check paths used for shuffle intermediary data manager get and delete

* add test

* newline

* meh
2020-04-07 09:47:18 -07:00
Jihoon Son 0da8ffc3ff
Bump up development version to 0.19.0-SNAPSHOT (#9586) 2020-03-30 16:24:04 -07:00
Clint Wylie 142742f291
add kinesis lag metric (#9509)
* add kinesis lag metric

* fixes

* heh

* do it right this time

* more test

* split out supervisor report lags into lagMillis, remove latest offsets from kinesis supervisor report since always null, review stuffs
2020-03-16 21:39:53 -07:00
Roman Leventov b9186f8f9f Reconcile terminology and method naming to 'used/unused segments'; Rename MetadataSegmentManager to MetadataSegmentsManager (#7306)
* Reconcile terminology and method naming to 'used/unused segments'; Don't use terms 'enable/disable data source'; Rename MetadataSegmentManager to MetadataSegments; Make REST API methods which mark segments as used/unused to return server error instead of an empty response in case of error

* Fix brace

* Import order

* Rename withKillDataSourceWhitelist to withSpecificDataSourcesToKill

* Fix tests

* Fix tests by adding proper methods without interval parameters to IndexerMetadataStorageCoordinator instead of hacking with Intervals.ETERNITY

* More aligned names of DruidCoordinatorHelpers, rename several CoordinatorDynamicConfig parameters

* Rename ClientCompactTaskQuery to ClientCompactionTaskQuery for consistency with CompactionTask; ClientCompactQueryTuningConfig to ClientCompactionTaskQueryTuningConfig

* More variable and method renames

* Rename MetadataSegments to SegmentsMetadata

* Javadoc update

* Simplify SegmentsMetadata.getUnusedSegmentIntervals(), more javadocs

* Update Javadoc of VersionedIntervalTimeline.iterateAllObjects()

* Reorder imports

* Rename SegmentsMetadata.tryMark... methods to mark... and make them to return boolean and the numbers of segments changed and relay exceptions to callers

* Complete merge

* Add CollectionUtils.newTreeSet(); Refactor DruidCoordinatorRuntimeParams creation in tests

* Remove MetadataSegmentManager

* Rename millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments to leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments

* Fix tests, refactor DruidCluster creation in tests into DruidClusterBuilder

* Fix inspections

* Fix SQLMetadataSegmentManagerEmptyTest and rename it to SqlSegmentsMetadataEmptyTest

* Rename SegmentsAndMetadata to SegmentsAndCommitMetadata to reduce the similarity with SegmentsMetadata; Rename some methods

* Rename DruidCoordinatorHelper to CoordinatorDuty, refactor DruidCoordinator

* Unused import

* Optimize imports

* Rename IndexerSQLMetadataStorageCoordinator.getDataSourceMetadata() to retrieveDataSourceMetadata()

* Unused import

* Update terminology in datasource-view.tsx

* Fix label in datasource-view.spec.tsx.snap

* Fix lint errors in datasource-view.tsx

* Doc improvements

* Another attempt to please TSLint

* Another attempt to please TSLint

* Style fixes

* Fix IndexerSQLMetadataStorageCoordinator.createUsedSegmentsSqlQueryForIntervals() (wrong merge)

* Try to fix docs build issue

* Javadoc and spelling fixes

* Rename SegmentsMetadata to SegmentsMetadataManager, address other comments

* Address more comments
2020-01-27 11:24:29 -08:00
Clint Wylie c6c8b80644 fix build by updating kafka client to 2.2.2 for CVE-2019-12399 (#9259)
* fix build by updating kafka client to 2.2.2 for CVE-2019-12399

* one kafka version to rule them all

* notice
2020-01-27 11:07:02 -08:00
Gian Merlino 19b427e8f3
Add JoinableFactory interface and use it in the query stack. (#9247)
* Add JoinableFactory interface and use it in the query stack.

Also includes InlineJoinableFactory, which enables joining against
inline datasources. This is the first patch where a basic join query
actually works. It includes integration tests.

* Fix test issues.

* Adjustments from code review.
2020-01-24 13:10:01 -08:00
Gian Merlino d21054f7c5
Remove the deprecated interval-chunking stuff. (#9216)
* Remove the deprecated interval-chunking stuff.

See https://github.com/apache/druid/pull/6591, https://github.com/apache/druid/pull/4004#issuecomment-284171911 for details.

* Remove unused import.

* Remove chunkInterval too.
2020-01-19 17:14:23 -08:00
Clint Wylie f540216931 fix InputFormat serde issue with SeekableStream based supervisors (#9136) 2020-01-07 16:18:54 -06:00
Jonathan Wei aa539177ec De-incubation cleanup in code, docs, packaging (#9108)
* De-incubation cleanup in code, docs, packaging

* remove unused docs script
2020-01-03 12:33:19 -05:00
Jonathan Wei 4e8368a5d9 Set version to 0.18.0-SNAPSHOT (#9109) 2020-01-02 17:55:10 -05:00
Jonathan Wei 8af41d7cd0 Update version to 0.18.0-incubating-SNAPSHOT (#9009) 2019-12-11 14:04:03 -08:00
Jonathan Wei 00ce18a0ea
Additional Kinesis resharding fixes (#8870)
* Additional Kinesis resharding fixes

* Address PR comments

* Remove unused method

* Adjust SegmentTransactionalInsertAction null handling

* Check for unchanged metadata on empty publish

* Add logs for empty publish

* Fix javadoc

* Clear offset when invalid endOffsets are seen

* Fix LGTM alert

* Fix build

* Add resharding note to Kinesis docs

* Checkstyle

* Spelling

* Address PR comments

* Checkstyle
2019-11-28 12:59:01 -08:00
jon-wei dfbc066163 Revert "[maven-release-plugin] prepare release druid-0.16.1-incubating-rc1"
This reverts commit a0f21d9b07.
2019-11-27 23:22:43 -08:00
jon-wei 0402ff85b8 Revert "[maven-release-plugin] prepare for next development iteration"
This reverts commit 8ffa71e7e6.
2019-11-27 23:22:32 -08:00
jon-wei 8ffa71e7e6 [maven-release-plugin] prepare for next development iteration 2019-11-27 23:18:48 -08:00
jon-wei a0f21d9b07 [maven-release-plugin] prepare release druid-0.16.1-incubating-rc1 2019-11-27 23:18:37 -08:00
Gian Merlino e0eb85ace7 Add FileUtils.createTempDir() and enforce its usage. (#8932)
* Add FileUtils.createTempDir() and enforce its usage.

The purpose of this is to improve error messages. Previously, the error
message on a nonexistent or unwritable temp directory would be
"Failed to create directory within 10,000 attempts".

* Further updates.

* Another update.

* Remove commons-io from benchmark.

* Fix tests.
2019-11-22 19:48:49 -08:00
Jihoon Son ac6d703814 Support inputFormat and inputSource for sampler (#8901)
* Support inputFormat and inputSource for sampler

* Cleanup javadocs and names

* fix style

* fix timed shutoff input source reader

* fix timed shutoff input source reader again

* tidy up timed shutoff reader

* unused imports

* fix tc
2019-11-20 14:51:25 -08:00
Surekha d628bebbd7 Make supervisor API similar to submit task API (#8810)
* accept spec or dataSchema, tuningConfig, ioConfig while submitting task json

* fix test

* update docs

* lgtm warning

* Add original constructor back to IndexTask to minimize changes

* fix indentation in docs

* Allow spec to be specified in supervisor schema

* undo IndexTask spec changes

* update docs

* Add Nullable and deprecated annotations

* remove deprecated configs from SeekableStreamSupervisorSpec

* remove nullable annotation
2019-11-20 10:04:41 -08:00
Clint Wylie 3fcaa1a61b
fix sql compatible null handling config work with runtime.properties (#8876)
* fix sql compatible null handling config work with runtime.properties

* fix npe

* fix tests

* add friendly error

* comment, and friendlier still

* fix compile

* fix from merges
2019-11-20 03:55:29 -08:00
Rye d0913475b7 sampler returns nulls in CSV (#8871)
* sampler returns nulls in CSV

* fixed kafka sampler test

* fix Kinesis test

* sql compatibility fix

* remove null to empty string conversion, use null

* fix sql compatibility
2019-11-19 13:59:44 -08:00
Gian Merlino c44452f0c1 Tidy up lifecycle, query, and ingestion logging. (#8889)
* Tidy up lifecycle, query, and ingestion logging.

The goal of this patch is to improve the clarity and usefulness of
Druid's logging for cluster operators. For more information, see
https://twitter.com/cowtowncoder/status/1195469299814555648.

Concretely, this patch does the following:

- Changes a lot of INFO logs to DEBUG, and DEBUG to TRACE, with the
  goal of reducing redundancy and improving clarity by avoiding
  showing rarely-useful log messages. This includes most "starting"
  and "stopping" messages, and most messages related to individual
  columns.
- Adds new log4j2 templates that show operators how to enabled DEBUG
  logging for certain important packages.
- Eliminate stack traces for query errors, unless log level is DEBUG
  or more. This is useful because query errors often indicate user
  error rather than system error, but dumping stack trace often gave
  operators the impression that there was a system failure.
- Adds task id to Appenderator, AppenderatorDriver thread names. In
  the default log4j2 configuration, this will put them in log lines
  as well. It's very useful if a user is using the Indexer, where
  multiple tasks run in the same JVM.
- More consistent terminology when it comes to "sequences" (sets of
  segments that are handed-off together by Kafka ingestion) and
  "offsets" (cursors in partitions). These terms had been confused in
  some log messages due to the fact that Kinesis calls offsets
  "sequence numbers".
- Replaces some ugly toString calls with either the JSONification or
  something more operator-accessible (like a URL or segment identifier,
  instead of JSON object representing the same).

* Adjustments.

* Adjust integration test.
2019-11-19 13:57:58 -08:00
Surekha cf6643eb9a add sequenceName and currentCheckPoint for backwards compatibility (#8864)
* add sequenceName and currentCheckPoint for backwards compatibility

* Add serde unit test in kafka

* fix checkstyle

* add hashcode

* update javadoc
2019-11-19 13:11:31 -08:00
Atul Mohan 8515a03c6b Modify batch index task naming to accomodate simultaneous tasks (#8612)
* Change hadoop task naming

* Remove unused

* Add timestamp

* Fix build
2019-11-18 15:07:16 -08:00
Rye ea8e4066f6 Use earliest offset on kafka newly discovered partitions (#8748)
* Use earliest offset on kafka newly discovered partitions

* resolve conflicts

* remove redundant check cases

* simplified unit tests

* change test case

* rewrite comments

* add regression test

* add junit ignore annotation

* minor modifications

* indent

* override testableKafkaSupervisor and KafkaRecordSupplier to make the test runable

* modified test constructor of kafkaRecordSupplier

* simplify

* delegated constructor
2019-11-18 11:05:31 -08:00
Jihoon Son 1611792855
Add InputSource and InputFormat interfaces (#8823)
* Add InputSource and InputFormat interfaces

* revert orc dependency

* fix dimension exclusions and failing unit tests

* fix tests

* fix test

* fix test

* fix firehose and inputSource for parallel indexing task

* fix tc

* fix tc: remove unused method

* Formattable

* add needsFormat(); renamed to ObjectSource; pass metricsName for reader

* address comments

* fix closing resource

* fix checkstyle

* fix tests

* remove verify from csv

* Revert "remove verify from csv"

This reverts commit 1ea7758489.

* address comments

* fix import order and javadoc

* flatMap

* sampleLine

* Add IntermediateRowParsingReader

* Address comments

* move csv reader test

* remove test for verify

* adjust comments

* Fix InputEntityIteratingReader

* rename source -> entity

* address comments
2019-11-15 09:22:09 -08:00
Roman Leventov 5c0fc0a13a Fix ambiguity about IndexerSQLMetadataStorageCoordinator.getUsedSegmentsForInterval() returning only non-overshadowed or all used segments (#8564)
* IndexerSQLMetadataStorageCoordinator.getTimelineForIntervalsWithHandle() don't fetch abutting intervals; simplify getUsedSegmentsForIntervals()

* Add VersionedIntervalTimeline.findNonOvershadowedObjectsInInterval() method; Propagate the decision about whether only visible segmetns or visible and overshadowed segments should be returned from IndexerMetadataStorageCoordinator's methods to the user logic; Rename SegmentListUsedAction to RetrieveUsedSegmentsAction, SegmetnListUnusedAction to RetrieveUnusedSegmentsAction, and UsedSegmentLister to UsedSegmentsRetriever

* Fix tests

* More fixes

* Add javadoc notes about returning Collection instead of Set. Add JacksonUtils.readValue() to reduce boilerplate code

* Fix KinesisIndexTaskTest, factor out common parts from KinesisIndexTaskTest and KafkaIndexTaskTest into SeekableStreamIndexTaskTestBase

* More test fixes

* More test fixes

* Add a comment to VersionedIntervalTimelineTestBase

* Fix tests

* Set DataSegment.size(0) in more tests

* Specify DataSegment.size(0) in more places in tests

* Fix more tests

* Fix DruidSchemaTest

* Set DataSegment's size in more tests and benchmarks

* Fix HdfsDataSegmentPusherTest

* Doc changes addressing comments

* Extended doc for visibility

* Typo

* Typo 2

* Address comment
2019-11-06 11:07:04 -08:00
Giuseppe Martino 9c171e2b1f Message rejection absolute date (#8656)
* Add option lateMessageRejectionStartDate

* Use option lateMessageRejectionStartDate

* Fix tests

* Add lateMessageRejectionStartDate to kafka indexing service

* Update tests kafka indexing service

* Fix tests for KafkaSupervisorTest

* Add lateMessageRejectionStartDate to KinesisSupervisorIOConfig

* Fix var name

* Update documentation

* Add check lateMessageRejectionStartDateTime and lateMessageRejectionPeriod, fails if both were specified.
2019-10-31 15:13:02 -07:00
Jihoon Son 2518478b20 Remove deprecated parameter for Checkpoint request (#8707)
* Remove deprecated parameter for Checkpoint request

* fix wrong doc
2019-10-23 16:51:16 -07:00
Surekha 98f59ddd7e Add `sys.supervisors` table to system tables (#8547)
* Add supervisors table to SystemSchema

* Add docs

* fix checkstyle

* fix test

* fix CI

* Add comments

* Fix javadoc teamcity error

* comments

* fix links in docs

* fix links

* rename fullStatus query param to system and remove it from docs
2019-10-18 15:16:42 -07:00
Jonathan Wei 89ce6384f5
More Kinesis resharding adjustments (#8671)
* More Kinesis resharding adjustments

* Fix TC inspection

* Fix comment'

* Adjust comment, small refactor

* Make repartition transition time configurable

* Add spellcheck exclusion

* Spelling fix
2019-10-15 23:19:17 -07:00
Jihoon Son 4046c86d62
Stateful auto compaction (#8573)
* Stateful auto compaction

* javaodc

* add removed test back

* fix test

* adding indexSpec to compactionState

* fix build

* add lastCompactionState

* address comments

* extract CompactionState

* fix doc

* fix build and test

* Add a task context to store compaction state; add javadoc

* fix it test
2019-10-15 22:57:42 -07:00
elloooooo 7f2b6577ef get active task by datasource when supervisor discover tasks (#8450)
* get active task by datasource when supervisor discover tasks

* fix ut

* fix ut

* fix ut

* remove unnecessary condition check

* fix ut

* remove stream in hot loop
2019-09-26 16:15:24 -07:00
Rye f2a444321b Added live reports for Kafka and Native batch task (#8557)
* Added live reports for Kafka and Native batch task

* Removed unused local variables

* Added the missing unit test

* Refine unit test logic, add implementation for HttpRemoteTaskRunner

* checksytle fixes

* Update doc descriptions for updated API

* remove unnecessary files

* Fix spellcheck complaints

* More details for api descriptions
2019-09-23 21:08:36 -07:00
Kamal Gurala 61761bd0b1 kafka version update (#8525) 2019-09-12 18:56:47 -07:00
Chi Cao Minh 5f61374cb3 Fix dependency analyze warnings (#8230)
* Fix dependency analyze warnings

Update the maven dependency plugin to the latest version and fix all
warnings for unused declared and used undeclared dependencies in the
compile scope. Added new travis job to add the check to CI. Also fixed
some source code files to use the correct packages for their imports and
updated druid-forbidden-apis to prevent regressions.

* Address review comments

* Adjust scope for org.glassfish.jaxb:jaxb-runtime

* Fix dependencies for hdfs-storage

* Consolidate netty4 versions
2019-09-09 14:37:21 -07:00
Clint Wylie c73a489335
bump master version to 0.17.0-incubating-SNAPSHOT (#8421) 2019-08-28 01:58:36 -07:00
Himanshu 4d87a19547
Logging emitter to publish query and other metric events as valid json objects (#8359)
* LoggingEmitter: print event as json

* use DefaultRequestLogEventBuilderFactory in emitting request logger by default

* print context in query metric as json

* removed unused jsonMapper from DefaultQueryMetrics

* add comment

* remove change to DefaultRequestLogEventBuilderFactory.java
2019-08-27 15:00:23 -07:00
Jihoon Son e5ef5ddafa Fix the shuffle with TLS enabled for parallel indexing; add an integration test; improve unit tests (#8350)
* Fix shuffle with tls enabled; add an integration test; improve unit tests

* remove debug log

* fix tests

* unused import

* add javadoc

* rename to getContent
2019-08-26 19:27:41 -07:00
SandishKumarHN 33f0753a70 Add Checkstyle for constant name static final (#8060)
* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* check ctyle for constant field name

* merging with upstream

* review-1

* unknow changes

* unknow changes

* review-2

* merging with master

* review-2 1 changes

* review changes-2 2

* bug fix
2019-08-23 13:13:54 +03:00
Jihoon Son fba92ae469 Fix to always use end sequenceNumber for reset (#8305)
* Fix to always use end sequenceNumber for reset

* fix checkstyle

* fix style and add log
2019-08-22 16:51:25 -05:00
Aaron Bossert a4d1219184 Removed hard-coded Kafka Deserializer in Web-UI Kafka data import such that users can supply a custom deserializer in the UI as well as in hand-built ingestion specs. (#8364) 2019-08-21 18:03:57 -07:00
Jihoon Son 22d6384d36
Fix unrealistic test variables in KafkaSupervisorTest and tidy up unused variable in checkpointing process (#7319)
* Fix unrealistic test arguments in KafkaSupervisorTest

* remove currentCheckpoint from checkpoint action

* rename variable
2019-08-21 10:58:22 -07:00
Jihoon Son 5dac6375f3
Add support for parallel native indexing with shuffle for perfect rollup (#8257)
* Add TaskResourceCleaner; fix a couple of concurrency bugs in batch tasks

* kill runner when it's ready

* add comment

* kill run thread

* fix test

* Take closeable out of Appenderator

* add javadoc

* fix test

* fix test

* update javadoc

* add javadoc about killed task

* address comment

* Add support for parallel native indexing with shuffle for perfect rollup.

* Add comment about volatiles

* fix test

* fix test

* handling missing exceptions

* more clear javadoc for stopGracefully

* unused import

* update javadoc

* Add missing statement in javadoc

* address comments; fix doc

* add javadoc for isGuaranteedRollup

* Rename confusing variable name and fix typos

* fix typos; move fetch() to a better home; fix the expiration time

* add support https
2019-08-15 17:43:35 -07:00
Jihoon Son 8a16a8e97f
Teach tasks what machine they are running on (#8190)
* Teach the middleManager port to tasks

* parent annotation

* Bind parent for indexer
2019-08-02 15:34:44 -07:00