Merge commit 'apache/master^^^' into quidem-record

This commit is contained in:
Zoltan Haindrich 2024-07-17 13:27:54 +00:00
commit 2a590eb3ae
647 changed files with 19161 additions and 21952 deletions

View File

@ -113,7 +113,7 @@ jobs:
- name: Collect service logs on failure
if: ${{ failure() && steps.run-it.conclusion == 'failure' }}
run: |
tar cvzf ./service-logs.tgz ~/shared/logs
tar cvzf ./service-logs.tgz ~/shared/logs ~/shared/tasklogs
- name: Upload Druid service logs to GitHub
if: ${{ failure() && steps.run-it.conclusion == 'failure' }}

View File

@ -50,7 +50,7 @@ jobs:
matrix:
#jdk: [8, 11, 17]
jdk: [8]
it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Security]
it: [HighAvailability, MultiStageQuery, Catalog, BatchIndex, MultiStageQueryWithMM, InputSource, InputFormat, Security, Query]
#indexer: [indexer, middleManager]
indexer: [middleManager]
uses: ./.github/workflows/reusable-revised-its.yml

View File

@ -351,11 +351,11 @@ public class FrameChannelMergerBenchmark
channels.stream().map(BlockingQueueFrameChannel::readable).collect(Collectors.toList()),
frameReader,
outputChannel.writable(),
FrameWriters.makeFrameWriterFactory(
FrameType.ROW_BASED,
FrameWriters.makeRowBasedFrameWriterFactory(
new ArenaMemoryAllocatorFactory(1_000_000),
signature,
sortKey
sortKey,
false
),
sortKey,
null,

View File

@ -37,6 +37,7 @@ import org.apache.druid.math.expr.ExpressionProcessing;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.segment.AutoTypeColumnSchema;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndex;
@ -236,6 +237,14 @@ public class SqlExpressionBenchmark
})
private String schema;
@Param({
"singleString",
"fixedWidth",
"fixedWidthNonNumeric",
"always"
})
private String deferExpressionDimensions;
@Param({
// non-expression reference
"0",
@ -414,7 +423,8 @@ public class SqlExpressionBenchmark
{
final Map<String, Object> context = ImmutableMap.of(
QueryContexts.VECTORIZE_KEY, vectorize,
QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize
QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize,
GroupByQueryConfig.CTX_KEY_DEFER_EXPRESSION_DIMENSIONS, deferExpressionDimensions
);
final String sql = QUERIES.get(Integer.parseInt(query));
try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sql, context)) {

View File

@ -104,6 +104,7 @@ public class NewestSegmentFirstPolicyBenchmark
null,
null,
null,
null,
null
)
);
@ -140,7 +141,7 @@ public class NewestSegmentFirstPolicyBenchmark
@Benchmark
public void measureNewestSegmentFirstPolicy(Blackhole blackhole)
{
final CompactionSegmentIterator iterator = policy.reset(compactionConfigs, dataSources, Collections.emptyMap());
final CompactionSegmentIterator iterator = policy.createIterator(compactionConfigs, dataSources, Collections.emptyMap());
for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) {
blackhole.consume(iterator.next());
}

View File

@ -55,7 +55,7 @@ The installation of a MySQL metadata store is outside the scope of this document
Use of other databases such as Postgres or Derby are entirely reasonable, but doing so is left as an excercise to the reader.
## ZooKeeper
This also assumes you have [ZooKeeper](http://zookeeper.apache.org/releases.html) running locally, which usually just involves downloading the latst distribution of ZooKeeper, doing some minor configuration in ZooKeeper's `conf/` directory (most defaults are fine), then running `./bin/zkServer.sh start` in the ZooKeeper directory.
This also assumes you have [ZooKeeper](http://zookeeper.apache.org/releases.html) running locally, which usually just involves downloading the latest distribution of ZooKeeper, doing some minor configuration in ZooKeeper's `conf/` directory (most defaults are fine), then running `./bin/zkServer.sh start` in the ZooKeeper directory.
On macOS, you can also achieve this through the following commands

View File

@ -16,10 +16,9 @@
# limitations under the License.
import os
import requests
import subprocess
import sys
import requests
if len(sys.argv) != 5:
sys.stderr.write('usage: program <github-username> <previous-release-commit> <new-release-commit> <milestone-number-to-tag>\n')

View File

@ -176,7 +176,11 @@ Returns a list of all segments, overlapping with any of given intervals, for a d
`POST /druid/coordinator/v1/metadata/dataSourceInformation`
Returns information about the specified datasources, including the datasource schema.
Returns information about the specified datasources, including the datasource schema.
`POST /druid/coordinator/v1/metadata/bootstrapSegments`
Returns information about bootstrap segments for all datasources. The returned set includes all broadcast segments if broadcast rules are configured.
<a name="coordinator-datasources"></a>

View File

@ -22,7 +22,6 @@ title: "Extensions"
~ under the License.
-->
Druid implements an extension system that allows for adding functionality at runtime. Extensions
are commonly used to add support for deep storages (like HDFS and S3), metadata stores (like MySQL
and PostgreSQL), new aggregators, new input formats, and so on.
@ -55,7 +54,7 @@ Core extensions are maintained by Druid committers.
|druid-parquet-extensions|Support for data in Apache Parquet data format. Requires druid-avro-extensions to be loaded.|[link](../development/extensions-core/parquet.md)|
|druid-protobuf-extensions| Support for data in Protobuf data format.|[link](../development/extensions-core/protobuf.md)|
|druid-ranger-security|Support for access control through Apache Ranger.|[link](../development/extensions-core/druid-ranger-security.md)|
|druid-s3-extensions|Interfacing with data in AWS S3, and using S3 as deep storage.|[link](../development/extensions-core/s3.md)|
|druid-s3-extensions|Interfacing with data in Amazon S3, and using S3 as deep storage.|[link](../development/extensions-core/s3.md)|
|druid-ec2-extensions|Interfacing with AWS EC2 for autoscaling middle managers|UNDOCUMENTED|
|druid-aws-rds-extensions|Support for AWS token based access to AWS RDS DB Cluster.|[link](../development/extensions-core/druid-aws-rds.md)|
|druid-stats|Statistics related module including variance and standard deviation.|[link](../development/extensions-core/stats.md)|
@ -101,7 +100,7 @@ All of these community extensions can be downloaded using [pull-deps](../operati
|druid-momentsketch|Support for approximate quantile queries using the [momentsketch](https://github.com/stanford-futuredata/momentsketch) library|[link](../development/extensions-contrib/momentsketch-quantiles.md)|
|druid-tdigestsketch|Support for approximate sketch aggregators based on [T-Digest](https://github.com/tdunning/t-digest)|[link](../development/extensions-contrib/tdigestsketch-quantiles.md)|
|gce-extensions|GCE Extensions|[link](../development/extensions-contrib/gce-extensions.md)|
|prometheus-emitter|Exposes [Druid metrics](../operations/metrics.md) for Prometheus server collection (https://prometheus.io/)|[link](../development/extensions-contrib/prometheus.md)|
|prometheus-emitter|Exposes [Druid metrics](../operations/metrics.md) for Prometheus server collection (<https://prometheus.io/>)|[link](../development/extensions-contrib/prometheus.md)|
|druid-kubernetes-overlord-extensions|Support for launching tasks in k8s without Middle Managers|[link](../development/extensions-contrib/k8s-jobs.md)|
|druid-spectator-histogram|Support for efficient approximate percentile queries|[link](../development/extensions-contrib/spectator-histogram.md)|
|druid-rabbit-indexing-service|Support for creating and managing [RabbitMQ](https://www.rabbitmq.com/) indexing tasks|[link](../development/extensions-contrib/rabbit-stream-ingestion.md)|
@ -111,7 +110,6 @@ All of these community extensions can be downloaded using [pull-deps](../operati
Please post on [dev@druid.apache.org](https://lists.apache.org/list.html?dev@druid.apache.org) if you'd like an extension to be promoted to core.
If we see a community extension actively supported by the community, we can promote it to core based on community feedback.
For information how to create your own extension, please see [here](../development/modules.md).
## Loading extensions

View File

@ -668,14 +668,12 @@ Store task logs in S3. Note that the `druid-s3-extensions` extension must be loa
##### Azure Blob Store task logs
Store task logs in Azure Blob Store.
Store task logs in Azure Blob Store. To enable this feature, load the `druid-azure-extensions` extension, and configure deep storage for Azure. Druid uses the same authentication method configured for deep storage and stores task logs in the same storage account (set in `druid.azure.account`).
Note: The `druid-azure-extensions` extension must be loaded, and this uses the same storage account as the deep storage module for azure.
|Property|Description|Default|
|--------|-----------|-------|
|`druid.indexer.logs.container`|The Azure Blob Store container to write logs to|none|
|`druid.indexer.logs.prefix`|The path to prepend to logs|none|
| Property | Description | Default |
|---|---|---|
| `druid.indexer.logs.container` | The Azure Blob Store container to write logs to. | Must be set. |
| `druid.indexer.logs.prefix` | The path to prepend to logs. | Must be set. |
##### Google Cloud Storage task logs
@ -714,7 +712,7 @@ You can configure Druid API error responses to hide internal information like th
|`druid.server.http.showDetailedJettyErrors`|When set to true, any error from the Jetty layer / Jetty filter includes the following fields in the JSON response: `servlet`, `message`, `url`, `status`, and `cause`, if it exists. When set to false, the JSON response only includes `message`, `url`, and `status`. The field values remain unchanged.|true|
|`druid.server.http.errorResponseTransform.strategy`|Error response transform strategy. The strategy controls how Druid transforms error responses from Druid services. When unset or set to `none`, Druid leaves error responses unchanged.|`none`|
##### Error response transform strategy
#### Error response transform strategy
You can use an error response transform strategy to transform error responses from within Druid services to hide internal information.
When you specify an error response transform strategy other than `none`, Druid transforms the error responses from Druid services as follows:
@ -723,12 +721,12 @@ When you specify an error response transform strategy other than `none`, Druid t
* For any SQL query API that fails, for example `POST /druid/v2/sql/...`, Druid sets the fields `errorClass` and `host` to null. Druid applies the transformation strategy to the `errorMessage` field.
* For any JDBC related exceptions, Druid will turn all checked exceptions into `QueryInterruptedException` otherwise druid will attempt to keep the exception as the same type. For example if the original exception isn't owned by Druid it will become `QueryInterruptedException`. Druid applies the transformation strategy to the `errorMessage` field.
###### No error response transform strategy
##### No error response transform strategy
In this mode, Druid leaves error responses from underlying services unchanged and returns the unchanged errors to the API client.
This is the default Druid error response mode. To explicitly enable this strategy, set `druid.server.http.errorResponseTransform.strategy` to `none`.
###### Allowed regular expression error response transform strategy
##### Allowed regular expression error response transform strategy
In this mode, Druid validates the error responses from underlying services against a list of regular expressions. Only error messages that match a configured regular expression are returned. To enable this strategy, set `druid.server.http.errorResponseTransform.strategy` to `allowedRegex`.
@ -774,7 +772,7 @@ This config is used to find the [Coordinator](../design/coordinator.md) using Cu
You can configure how to announce and unannounce Znodes in ZooKeeper (using Curator). For normal operations you do not need to override any of these configs.
##### Batch data segment announcer
#### Batch data segment announcer
In current Druid, multiple data segments may be announced under the same Znode.
@ -1562,7 +1560,7 @@ For most types of tasks, `SegmentWriteOutMediumFactory` can be configured per-ta
|`druid.worker.capacity`|Maximum number of tasks the Indexer can accept.|Number of available processors - 1|
|`druid.worker.baseTaskDirs`|List of base temporary working directories, one of which is assigned per task in a round-robin fashion. This property can be used to allow usage of multiple disks for indexing. This property is recommended in place of and takes precedence over `${druid.indexer.task.baseTaskDir}`. If this configuration is not set, `${druid.indexer.task.baseTaskDir}` is used. Example: `druid.worker.baseTaskDirs=[\"PATH1\",\"PATH2\",...]`.|null|
|`druid.worker.baseTaskDirSize`|The total amount of bytes that can be used by tasks on any single task dir. This value is treated symmetrically across all directories, that is, if this is 500 GB and there are 3 `baseTaskDirs`, then each of those task directories is assumed to allow for 500 GB to be used and a total of 1.5 TB will potentially be available across all tasks. The actual amount of memory assigned to each task is discussed in [Configuring task storage sizes](../ingestion/tasks.md#configuring-task-storage-sizes)|`Long.MAX_VALUE`|
|`druid.worker.globalIngestionHeapLimitBytes`|Total amount of heap available for ingestion processing. This is applied by automatically setting the `maxBytesInMemory` property on tasks.|60% of configured JVM heap|
|`druid.worker.globalIngestionHeapLimitBytes`|Total amount of heap available for ingestion processing. This is applied by automatically setting the `maxBytesInMemory` property on tasks.|Configured max JVM heap size / 6|
|`druid.worker.numConcurrentMerges`|Maximum number of segment persist or merge operations that can run concurrently across all tasks.|`druid.worker.capacity` / 2, rounded down|
|`druid.indexer.task.baseDir`|Base temporary working directory.|`System.getProperty("java.io.tmpdir")`|
|`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|`${druid.indexer.task.baseDir}/persistent/tasks`|
@ -2037,7 +2035,7 @@ A simple in-memory LRU cache. Local cache resides in JVM heap memory, so if you
|Property|Description|Default|
|--------|-----------|-------|
|`druid.cache.sizeInBytes`|Maximum cache size in bytes. Zero disables caching.|0|
|`druid.cache.initialSize`|Initial size of the hashtable backing the cache.|500000|
|`druid.cache.initialSize`|Initial size of the hash table backing the cache.|500000|
|`druid.cache.logEvictionCount`|If non-zero, log cache eviction every `logEvictionCount` items.|0|
#### Caffeine cache

View File

@ -33,7 +33,7 @@ make sure it has `/master/` in the URL.
##### Installing Java and Maven
- JDK 8, 8u92+ or JDK 11. See our [Java documentation](../operations/java.md) for information about obtaining a JDK.
- See our [Java documentation](../operations/java.md) for information about obtaining a supported JDK
- [Maven version 3.x](http://maven.apache.org/download.cgi)
##### Other dependencies

View File

@ -34,8 +34,7 @@ Druid docs contributors:
Druid docs contributors can open an issue about documentation, or contribute a change with a pull request (PR).
The open source Druid docs are located here:
https://druid.apache.org/docs/latest/design/index.html
<https://druid.apache.org/docs/latest/design/index.html>
If you need to update a Druid doc, locate and update the doc in the Druid repo following the instructions below.
@ -48,19 +47,22 @@ See [`CONTRIBUTING.md`](https://github.com/apache/incubator-druid/blob/master/CO
## Before you begin
Before you can contribute to the Druid docs for the first time, you must complete the following steps:
1. Fork the [Druid repo](https://github.com/apache/druid). Your fork will be the `origin` remote.
2. Clone your fork:
2. Clone your fork:
```bash
git clone git@github.com:GITHUB_USERNAME/druid.git
```
Replace `GITHUB_USERNAME` with your GitHub username.
3. In the directory where you cloned your fork, set up `apache/druid` as your your remote `upstream` repo:
```bash
git remote add upstream https://github.com/apache/druid.git
```
4. Confirm that your fork shows up as the origin repo and `apache/druid` shows up as the upstream repo:
4. Confirm that your fork shows up as the origin repo and `apache/druid` shows up as the upstream repo:
```bash
git remote -v
@ -71,11 +73,11 @@ Before you can contribute to the Druid docs for the first time, you must complet
```bash
git config user.email
```
If you need to set your email, see the [GitHub instructions](https://docs.github.com/en/github-ae@latest/account-and-profile/setting-up-and-managing-your-github-user-account/managing-email-preferences/setting-your-commit-email-address#setting-your-commit-email-address-in-git).
5. Install Docusaurus so that you can build the site locally. Run either `npm install` or `yarn install` in the `website` directory.
6. Install Docusaurus so that you can build the site locally. Run either `npm install` or `yarn install` in the `website` directory.
## Contributing
Before you contribute, make sure your local branch of `master` and the upstream Apache branch are up-to-date and in sync. This can help you avoid merge conflicts. Run the following commands on your fork's `master` branch:
@ -104,13 +106,13 @@ Now you're up to date, and you can make your changes.
Provide a name for your feature branch in `MY-BRANCH`.
2. Find the file that you want to make changes to. All the source files for the docs are written in Markdown and located in the `docs` directory. The URL for the page includes the subdirectory the source file is in. For example, the SQL-based ingestion tutorial found at `https://druid.apache.org/docs/latest/tutorials/tutorial-msq-extern.html` is in the `tutorials` subdirectory.
If you're adding a page, create a new Markdown file in the appropriate subdirectory. Then, copy the front matter and Apache license from an existing file. Update the `title` and `id` fields. Don't forget to add it to `website/sidebars.json` so that your new page shows up in the navigation.
4. Test changes locally by building the site and navigating to your changes. In the `website` directory, run `docusaurus-start`. By default, this starts the site on `localhost:3000`. If port `3000` is already in use, it'll increment the port number from there.
5. Use the following commands to run the link and spellcheckers locally:
3. Test changes locally by building the site and navigating to your changes. In the `website` directory, run `npm run start`. By default, this starts the site on `localhost:3000`. If port `3000` is already in use, it'll increment the port number from there.
4. Use the following commands to run the link and spellcheckers locally:
```bash
cd website
# You only need to install once
@ -123,7 +125,7 @@ Now you're up to date, and you can make your changes.
This step can save you time during the review process since they'll run faster than the GitHub Action version of the checks and warn you of issues before you create a PR.
5. Push your changes to your fork:
5. Push your changes to your fork:
```bash
git push --set-upstream origin MY-BRANCH
@ -133,7 +135,7 @@ Now you're up to date, and you can make your changes.
The pull request template is extensive. You may not need all the information there, so feel free to delete unneeded sections as you fill it out. Once you create the pull request, GitHub automatically labels the issue so that reviewers can take a look.
The docs go through a review process similar to the code where community members will offer feedback. Once the review process is complete and your changes are merged, they'll be available on the live site when the site gets republished.
The docs go through a review process similar to the code where community members will offer feedback. Once the review process is complete and your changes are merged, they'll be available on the live site when the site gets republished.
## Style guide
@ -146,6 +148,7 @@ The style guide should serve as a point of reference to enable contributors and
In some cases, Google Style might make the Druid docs more difficult to read and understand. This section highlights those exceptions.
#### SQL keyword syntax
For SQL keywords and functions, use all caps, but do not use code font.
:::tip
@ -159,7 +162,6 @@ The UNNEST clause unnests array values.
The \`UNNEST\` clause unnests array values.
:::
#### Optional parameters and arguments
For optional parameters and arguments, enclose the optional parameter and leading command in brackets.

View File

@ -40,7 +40,280 @@ The extension uses `druid.indexer.runner.capacity` to limit the number of k8s jo
Other configurations required are:
`druid.indexer.runner.type: k8s` and `druid.indexer.task.encapsulatedTask: true`
## Pod Adapters
### Dynamic config
Druid operators can dynamically tune certain features within this extension. You don't need to restart the Overlord
service for these changes to take effect.
Druid can dynamically tune [pod template selection](#pod-template-selection), which allows you to configure the pod
template based on the task to be run. To enable dynamic pod template selection, first configure the
[custom template pod adapter](#custom-template-pod-adapter).
Use the following APIs to view and update the dynamic configuration for the Kubernetes task runner.
To use these APIs, ensure you have read and write permissions for the CONFIG resource type with the resource name
"CONFIG". For more information on permissions, see
[User authentication and authorization](../../operations/security-user-auth.md#config).
#### Get dynamic configuration
Retrieves the current dynamic execution config for the Kubernetes task runner.
Returns a JSON object with the dynamic configuration properties.
##### URL
`GET` `/druid/indexer/v1/k8s/taskrunner/executionconfig`
##### Responses
<Tabs>
<TabItem value="1" label="200 SUCCESS">
*Successfully retrieved dynamic configuration*
</TabItem>
</Tabs>
---
##### Sample request
<Tabs>
<TabItem value="2" label="cURL">
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/k8s/taskrunner/executionconfig"
```
</TabItem>
<TabItem value="3" label="HTTP">
```HTTP
GET /druid/indexer/v1/k8s/taskrunner/executionconfig HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
</TabItem>
</Tabs>
##### Sample response
<details>
<summary>View the response</summary>
```json
{
"type": "default",
"podTemplateSelectStrategy":
{
"type": "selectorBased",
"selectors": [
{
"selectionKey": "podSpec1",
"context.tags": {
"userProvidedTag": ["tag1", "tag2"]
},
"dataSource": ["wikipedia"]
},
{
"selectionKey": "podSpec2",
"type": ["index_kafka"]
}
]
}
}
```
</details>
#### Update dynamic configuration
Updates the dynamic configuration for the Kubernetes Task Runner
##### URL
`POST` `/druid/indexer/v1/k8s/taskrunner/executionconfig`
##### Header parameters
The endpoint supports the following optional header parameters to populate the `author` and `comment` fields in the configuration history.
* `X-Druid-Author`
* Type: String
* Author of the configuration change.
* `X-Druid-Comment`
* Type: String
* Description for the update.
##### Responses
<Tabs>
<TabItem value="4" label="200 SUCCESS">
*Successfully updated dynamic configuration*
</TabItem>
</Tabs>
---
##### Sample request
<Tabs>
<TabItem value="5" label="cURL">
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/k8s/taskrunner/executionconfig" \
--header 'Content-Type: application/json' \
--data '{
"type": "default",
"podTemplateSelectStrategy":
{
"type": "selectorBased",
"selectors": [
{
"selectionKey": "podSpec1",
"context.tags":
{
"userProvidedTag": ["tag1", "tag2"]
},
"dataSource": ["wikipedia"]
},
{
"selectionKey": "podSpec2",
"type": ["index_kafka"]
}
]
}
}'
```
</TabItem>
<TabItem value="6" label="HTTP">
```HTTP
POST /druid/indexer/v1/k8s/taskrunner/executionconfig HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
Content-Type: application/json
{
"type": "default",
"podTemplateSelectStrategy":
{
"type": "selectorBased",
"selectors": [
{
"selectionKey": "podSpec1",
"context.tags":
{
"userProvidedTag": ["tag1", "tag2"]
},
"dataSource": ["wikipedia"]
},
{
"selectionKey": "podSpec2",
"type": ["index_kafka"]
}
]
}
}
```
</TabItem>
</Tabs>
##### Sample response
A successful request returns an HTTP `200 OK` message code and an empty response body.
#### Get dynamic configuration history
Retrieves the history of changes to Kubernetes task runner's dynamic execution config over an interval of time. Returns
an empty array if there are no history records available.
##### URL
`GET` `/druid/indexer/v1/k8s/taskrunner/executionconfig/history`
##### Query parameters
The endpoint supports the following optional query parameters to filter results.
* `interval`
* Type: String
* Limit the results to the specified time interval in ISO 8601 format delimited with `/`. For example, `2023-07-13/2023-07-19`. The default interval is one week. You can change this period by setting `druid.audit.manager.auditHistoryMillis` in the `runtime.properties` file for the Coordinator.
* `count`
* Type: Integer
* Limit the number of results to the last `n` entries.
##### Responses
<Tabs>
<TabItem value="1" label="200 SUCCESS">
*Successfully retrieved dynamic configuration*
</TabItem>
</Tabs>
---
##### Sample request
<Tabs>
<TabItem value="2" label="cURL">
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/k8s/taskrunner/executionconfig/history"
```
</TabItem>
<TabItem value="3" label="HTTP">
```HTTP
GET /druid/indexer/v1/k8s/taskrunner/executionconfig/history HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
</TabItem>
</Tabs>
##### Sample response
<details>
<summary>View the response</summary>
```json
[
{
"key": "k8s.taskrunner.config",
"type": "k8s.taskrunner.config",
"auditInfo": {
"author": "",
"comment": "",
"ip": "127.0.0.1"
},
"payload": "{\"type\": \"default\",\"podTemplateSelectStrategy\":{\"type\": \"taskType\"}",
"auditTime": "2024-06-13T20:59:51.622Z"
}
]
```
</details>
## Pod adapters
The logic defining how the pod template is built for your Kubernetes Job depends on which pod adapter you have specified.
### Overlord Single Container Pod Adapter/Overlord Multi Container Pod Adapter
@ -65,7 +338,7 @@ and in your sidecar specs:
That will not work, because we cannot decipher what your command is, the extension needs to know it explicitly.
**Even for sidecars like Istio which are dynamically created by the service mesh, this needs to happen.*
Instead do the following:
Instead, do the following:
You can keep your Dockerfile the same but you must have a sidecar spec like so:
``` container:
name: foo
@ -90,13 +363,10 @@ The custom template pod adapter allows you to specify a pod template file per ta
The base pod template must be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.base: /path/to/basePodSpec.yaml`
Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{taskType}: /path/to/taskSpecificPodSpec.yaml` where {taskType} is the name of the task type i.e `index_parallel`.
<details>
<summary>Example Pod Template that uses the regular druid docker image</summary>
If you are trying to use the default image's environment variable parsing feature to set runtime properties, you need to add a extra escape underscore when specifying pod templates.
e.g. set the environment variable `druid_indexer_runner_k8s_podTemplate_index__parallel` when setting `druid.indxer.runner.k8s.podTemplate.index_parallel`
The following is an example Pod Template that uses the regular druid docker image.
```
```yaml
apiVersion: "v1"
kind: "PodTemplate"
template:
@ -164,6 +434,7 @@ template:
- emptyDir: {}
name: deepstorage-volume
```
</details>
The below runtime properties need to be passed to the Job's peon process.
@ -177,6 +448,10 @@ druid.indexer.task.encapsulatedTask=true
```
Any runtime property or JVM config used by the peon process can also be passed. E.G. below is a example of a ConfigMap that can be used to generate the `nodetype-config-volume` mount in the above template.
<details>
<summary>Example ConfigMap</summary>
```
kind: ConfigMap
metadata:
@ -217,59 +492,112 @@ data:
druid.peon.mode=remote
druid.indexer.task.encapsulatedTask=true
```
#### Dynamic Pod Template Selection Config
The Dynamic Pod Template Selection feature enhances the K8s extension by enabling more flexible and dynamic selection of pod templates based on task properties. This process is governed by the `PodTemplateSelectStrategy`. Below are the two strategies implemented:
</details>
|Property|Description|Default|
|--------|-----------|-------|
|`TaskTypePodTemplateSelectStrategy`| This strategy selects pod templates based on task type for execution purposes, implementing the behavior that maps templates to specific task types. | true |
|`SelectorBasedPodTemplateSelectStrategy`| This strategy evaluates a series of selectors, known as `selectors`, which are aligned with potential task properties. | false |
#### Pod template selection
The pod template adapter can select which pod template should be used for a task using the [task runner execution config](#dynamic-config)
`SelectorBasedPodTemplateSelectStrategy`, the strategy implementing this new feature, is based on conditional `selectors` that match against top-level keys from the task payload. Currently, it supports matching based on task context tags, task type, and dataSource. These selectors are ordered in the dynamic configuration, with the first selector given the highest priority during the evaluation process. This means that the selection process uses these ordered conditions to determine a tasks Pod template. The first matching condition immediately determines the Pod template, thereby prioritizing certain configurations over others. If no selector matches, it will fall back to an optional `defaultKey` if configured; if there is still no match, it will use the `base` template.
##### Select based on task type
Example Configuration:
The `TaskTypePodTemplateSelectStrategy` strategy selects pod templates based on task type for execution purposes,
implementing the behavior that maps templates to specific task types. This is the default pod template selection
strategy. To explicitly select this strategy, set the `podTemplateSelectStrategy` in the dynamic execution config to
We define two template keys in the configuration—`low-throughput` and `medium-throughput`—each associated with specific task conditions and arranged in a priority order.
- Low Throughput Template: This is the first template evaluated and has the highest priority. Tasks that have a context tag `billingCategory=streaming_ingestion` and a datasource of `wikipedia` will be classified under the `low-throughput` template. This classification directs such tasks to utilize a predefined pod template optimized for low throughput requirements.
- Medium Throughput Template: If a task does not meet the low-throughput criteria, the system will then evaluate it against the next selector in order. In this example, if the task type is index_kafka, it will fall into the `medium-throughput` template.
```json
{ "type": "default" }
```
Task specific pod templates can be specified as the runtime property
`druid.indexer.runner.k8s.podTemplate.{taskType}: /path/to/taskSpecificPodSpec.yaml` where {taskType} is the name of the
task type. For example, `index_parallel`.
If you are trying to use the default image's environment variable parsing feature to set runtime properties, you need to add a extra escape underscore when specifying pod templates.
For example, set the environment variable `druid_indexer_runner_k8s_podTemplate_index__kafka` when you set the runtime property `druid.indexer.runner.k8s.podTemplate.index_kafka`
The following example shows a configuration for task-based pod template selection:
```properties
druid.indexer.runner.k8s.podTemplate.base=/path/to/basePodSpec.yaml
druid.indexer.runner.k8s.podTemplate.index_kafka=/path/to/kafkaPodSpec.yaml
```
##### Select based on one or more conditions
The `SelectorBasedPodTemplateSelectStrategy` strategy evaluates a series of criteria within `selectors` to determine
which pod template to use to run the task. Pod templates are configured in the runtime properties like
`druid.indexer.runner.k8s.podTemplate.<selectionKey>=...`.
```json
{
"type": "selectorBased",
"selectors": [
{
"selectionKey": "podSpec1",
"context.tags":
{
"userProvidedTag": ["tag1", "tag2"]
},
"dataSource": ["wikipedia"]
},
{
"selectionKey": "podSpec2",
"type": ["index_kafka"]
}
]
}
```
Selectors are processed in order. Druid selects the template based on the first matching selector. If a task does not
match any selector in the list, it will use the `base` pod template.
For a task to match a selector, all the conditions within the selector must match. A selector can match on
- `type`: Type of the task
- `dataSource`: Destination datasource of the task.
- `context.tags`: Tags passed in the task's context.
##### Example
Set the following runtime properties to define the pod specs that can be used by Druid.
```properties
druid.indexer.runner.k8s.podTemplate.base=/path/to/basePodSpec.yaml
druid.indexer.runner.k8s.podTemplate.podSpec1=/path/to/podSpecWithHighMemRequests.yaml
druid.indexer.runner.k8s.podTemplate.podSpec2=/path/to/podSpecWithLowCpuRequests.yaml
```
Set the dynamic execution config to define the pod template selection strategy.
```json
{
"type": "default",
"podTemplateSelectStrategy":
{
"podTemplateSelectStrategy": {
"type": "selectorBased",
"selectors": [
{
"selectionKey": "low-throughput",
"context.tags":
{
"billingCategory": ["streaming_ingestion"]
},
"selectionKey": "podSpec1",
"context.tags": { "userProvidedTag": ["tag1", "tag2"] },
"dataSource": ["wikipedia"]
},
{
"selectionKey": "medium-throughput",
"selectionKey": "podSpec2",
"type": ["index_kafka"]
}
],
"defaultKey"" "base"
]
}
}
```
Task specific pod templates can be specified as the runtime property `druid.indexer.runner.k8s.podTemplate.{template}: /path/to/taskSpecificPodSpec.yaml` where {template} is the matched `selectionKey` of the `podTemplateSelectStrategy` i.e low-throughput.
Similar to Overlord dynamic configuration, the following API endpoints are defined to retrieve and manage dynamic configurations of Pod Template Selection config:
Druid selects the pod templates as follows:
1. Use `podSpecWithHighMemRequests.yaml` when both of the following conditions are met:
1. The task context contains a tag with the key `userProvidedTag` that has the value `tag1` or `tag2`.
2. The task targets the `wikipedia` datasource.
2. Use `podSpecWithLowCpuRequests.yaml` when the task type is `index_kafka`.
3. Use the `basePodSpec.yaml` for all other tasks.
- Get dynamic configuration:
`POST` `/druid/indexer/v1/k8s/taskRunner/executionConfig`
- Update dynamic configuration:
`GET` `/druid/indexer/v1/k8s/taskRunner/executionConfig`
- Get dynamic configuration history:
`GET` `/druid/indexer/v1/k8s/taskRunner/executionConfig/history`
In this example, if there is an `index_kafka` task for the `wikipedia` datasource with the tag `userProvidedTag: tag1`,
Druid selects the pod template `podSpecWithHighMemRequests.yaml`.
### Properties
|Property| Possible Values | Description |Default|required|
@ -302,7 +630,8 @@ Similar to Overlord dynamic configuration, the following API endpoints are defin
- All Druid Pods belonging to one Druid cluster must be inside the same Kubernetes namespace.
- You must have a role binding for the overlord's service account that provides the needed permissions for interacting with Kubernetes. An example spec could be:
```
```yaml
kind: Role
apiVersion: rbac.authorization.k8s.io/v1
metadata:

View File

@ -22,25 +22,75 @@ title: "Microsoft Azure"
~ under the License.
-->
## Azure extension
This extension allows you to do the following:
* [Ingest data](#ingest-data-from-azure) from objects stored in Azure Blob Storage.
* [Write segments](#store-segments-in-azure) to Azure Blob Storage for deep storage.
* [Persist task logs](#persist-task-logs-in-azure) to Azure Blob Storage for long-term storage.
:::info
To use this Apache Druid extension, [include](../../configuration/extensions.md#loading-extensions) `druid-azure-extensions` in the extensions load list.
## Deep Storage
:::
[Microsoft Azure Storage](http://azure.microsoft.com/en-us/services/storage/) is another option for deep storage. This requires some additional Druid configuration.
### Ingest data from Azure
|Property|Description|Possible Values|Default|
|--------|---------------|-----------|-------|
|`druid.storage.type`|azure||Must be set.|
|`druid.azure.account`||Azure Storage account name.|Must be set.|
|`druid.azure.key`||Azure Storage account key.|Optional. Set one of key, sharedAccessStorageToken or useAzureCredentialsChain.|
|`druid.azure.sharedAccessStorageToken`||Azure Shared Storage access token|Optional. Set one of key, sharedAccessStorageToken or useAzureCredentialsChain..|
|`druid.azure.useAzureCredentialsChain`|Use [DefaultAzureCredential](https://learn.microsoft.com/en-us/java/api/overview/azure/identity-readme?view=azure-java-stable) for authentication|Optional. Set one of key, sharedAccessStorageToken or useAzureCredentialsChain.|False|
|`druid.azure.managedIdentityClientId`|If you want to use managed identity authentication in the `DefaultAzureCredential`, `useAzureCredentialsChain` must be true.||Optional.|
|`druid.azure.container`||Azure Storage container name.|Must be set.|
|`druid.azure.prefix`|A prefix string that will be prepended to the blob names for the segments published to Azure deep storage| |""|
|`druid.azure.protocol`|the protocol to use|http or https|https|
|`druid.azure.maxTries`|Number of tries before canceling an Azure operation.| |3|
|`druid.azure.maxListingLength`|maximum number of input files matching a given prefix to retrieve at a time| |1024|
|`druid.azure.storageAccountEndpointSuffix`| The endpoint suffix to use. Use this config instead of `druid.azure.endpointSuffix`. Override the default value to connect to [Azure Government](https://learn.microsoft.com/en-us/azure/azure-government/documentation-government-get-started-connect-to-storage#getting-started-with-storage-api). This config supports storage accounts enabled for [AzureDNSZone](https://learn.microsoft.com/en-us/azure/dns/dns-getstarted-portal). Note: do not include the storage account name prefix in this config value. | Examples: `ABCD1234.blob.storage.azure.net`, `blob.core.usgovcloudapi.net`| `blob.core.windows.net`|
See [Azure Services](http://azure.microsoft.com/en-us/pricing/free-trial/) for more information.
Ingest data using either [MSQ](../../multi-stage-query/index.md) or a native batch [parallel task](../../ingestion/native-batch.md) with an [Azure input source](../../ingestion/input-sources.md#azure-input-source) (`azureStorage`) to read objects directly from Azure Blob Storage.
### Store segments in Azure
:::info
To use Azure for deep storage, set `druid.storage.type=azure`.
:::
#### Configure location
Configure where to store segments using the following properties:
| Property | Description | Default |
|---|---|---|
| `druid.azure.account` | The Azure Storage account name. | Must be set. |
| `druid.azure.container` | The Azure Storage container name. | Must be set. |
| `druid.azure.prefix` | A prefix string that will be prepended to the blob names for the segments published. | "" |
| `druid.azure.maxTries` | Number of tries before canceling an Azure operation. | 3 |
| `druid.azure.protocol` | The protocol to use to connect to the Azure Storage account. Either `http` or `https`. | `https` |
| `druid.azure.storageAccountEndpointSuffix` | The Storage account endpoint to use. Override the default value to connect to [Azure Government](https://learn.microsoft.com/en-us/azure/azure-government/documentation-government-get-started-connect-to-storage#getting-started-with-storage-api) or storage accounts with [Azure DNS zone endpoints](https://learn.microsoft.com/en-us/azure/storage/common/storage-account-overview#azure-dns-zone-endpoints-preview).<br/><br/>Do _not_ include the storage account name prefix in this config value.<br/><br/>Examples: `ABCD1234.blob.storage.azure.net`, `blob.core.usgovcloudapi.net`. | `blob.core.windows.net` |
#### Configure authentication
Authenticate access to Azure Blob Storage using one of the following methods:
* [SAS token](https://learn.microsoft.com/en-us/azure/storage/common/storage-sas-overview)
* [Shared Key](https://learn.microsoft.com/en-us/rest/api/storageservices/authorize-with-shared-key)
* Default Azure credentials chain ([`DefaultAzureCredential`](https://learn.microsoft.com/en-us/java/api/overview/azure/identity-readme#defaultazurecredential)).
Configure authentication using the following properties:
| Property | Description | Default |
|---|---|---|
| `druid.azure.sharedAccessStorageToken` | The SAS (Shared Storage Access) token. | |
| `druid.azure.key` | The Shared Key. | |
| `druid.azure.useAzureCredentialsChain` | If `true`, use `DefaultAzureCredential` for authentication. | `false` |
| `druid.azure.managedIdentityClientId` | To use managed identity authentication in the `DefaultAzureCredential`, set `useAzureCredentialsChain` to `true` and provide the client ID here. | |
### Persist task logs in Azure
:::info
To persist task logs in Azure Blob Storage, set `druid.indexer.logs.type=azure`.
:::
Druid stores task logs using the storage account and authentication method configured for storing segments. Use the following configuration to set up where to store the task logs:
| Property | Description | Default |
|---|---|---|
| `druid.indexer.logs.container` | The Azure Blob Store container to write logs to. | Must be set. |
| `druid.indexer.logs.prefix` | The path to prepend to logs. | Must be set. |
For general options regarding task retention, see [Log retention policy](../../configuration/index.md#log-retention-policy).

View File

@ -22,7 +22,6 @@ title: "HDFS"
~ under the License.
-->
To use this Apache Druid extension, [include](../../configuration/extensions.md#loading-extensions) `druid-hdfs-storage` in the extensions load list and run druid processes with `GOOGLE_APPLICATION_CREDENTIALS=/path/to/service_account_keyfile` in the environment.
## Deep Storage
@ -44,11 +43,11 @@ If you want to eagerly authenticate against a secured hadoop/hdfs cluster you mu
### Configuration for Cloud Storage
You can also use the AWS S3 or the Google Cloud Storage as the deep storage via HDFS.
You can also use the Amazon S3 or the Google Cloud Storage as the deep storage via HDFS.
#### Configuration for AWS S3
#### Configuration for Amazon S3
To use the AWS S3 as the deep storage, you need to configure `druid.storage.storageDirectory` properly.
To use the Amazon S3 as the deep storage, you need to configure `druid.storage.storageDirectory` properly.
|Property|Possible Values|Description|Default|
|--------|---------------|-----------|-------|

View File

@ -25,6 +25,7 @@ title: "S3-compatible"
## S3 extension
This extension allows you to do 2 things:
* [Ingest data](#reading-data-from-s3) from files stored in S3.
* Write segments to [deep storage](#deep-storage) in S3.
@ -41,7 +42,7 @@ To read objects from S3, you must supply [connection information](#configuration
### Deep Storage
S3-compatible deep storage means either AWS S3 or a compatible service like Google Storage which exposes the same API as S3.
S3-compatible deep storage means either Amazon S3 or a compatible service like Google Storage which exposes the same API as S3.
S3 deep storage needs to be explicitly enabled by setting `druid.storage.type=s3`. **Only after setting the storage type to S3 will any of the settings below take effect.**
@ -97,19 +98,19 @@ Note that this setting only affects Druid's behavior. Changing S3 to use Object
If you're using ACLs, Druid needs the following permissions:
- `s3:GetObject`
- `s3:PutObject`
- `s3:DeleteObject`
- `s3:GetBucketAcl`
- `s3:PutObjectAcl`
* `s3:GetObject`
* `s3:PutObject`
* `s3:DeleteObject`
* `s3:GetBucketAcl`
* `s3:PutObjectAcl`
#### Object Ownership permissions
If you're using Object Ownership, Druid needs the following permissions:
- `s3:GetObject`
- `s3:PutObject`
- `s3:DeleteObject`
* `s3:GetObject`
* `s3:PutObject`
* `s3:DeleteObject`
### AWS region
@ -117,8 +118,8 @@ The AWS SDK requires that a target region be specified. You can set these by us
For example, to set the region to 'us-east-1' through system properties:
- Add `-Daws.region=us-east-1` to the `jvm.config` file for all Druid services.
- Add `-Daws.region=us-east-1` to `druid.indexer.runner.javaOpts` in [Middle Manager configuration](../../configuration/index.md#middlemanager-configuration) so that the property will be passed to Peon (worker) processes.
* Add `-Daws.region=us-east-1` to the `jvm.config` file for all Druid services.
* Add `-Daws.region=us-east-1` to `druid.indexer.runner.javaOpts` in [Middle Manager configuration](../../configuration/index.md#middlemanager-configuration) so that the property will be passed to Peon (worker) processes.
### Connecting to S3 configuration
@ -146,6 +147,6 @@ For example, to set the region to 'us-east-1' through system properties:
You can enable [server-side encryption](https://docs.aws.amazon.com/AmazonS3/latest/dev/serv-side-encryption) by setting
`druid.storage.sse.type` to a supported type of server-side encryption. The current supported types are:
- s3: [Server-side encryption with S3-managed encryption keys](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption)
- kms: [Server-side encryption with AWS KMSManaged Keys](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption)
- custom: [Server-side encryption with Customer-Provided Encryption Keys](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys)
* s3: [Server-side encryption with S3-managed encryption keys](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption)
* kms: [Server-side encryption with AWS KMSManaged Keys](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption)
* custom: [Server-side encryption with Customer-Provided Encryption Keys](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys)

View File

@ -148,7 +148,7 @@ For example, using the static input paths:
"paths" : "hdfs://path/to/data/is/here/data.gz,hdfs://path/to/data/is/here/moredata.gz,hdfs://path/to/data/is/here/evenmoredata.gz"
```
You can also read from cloud storage such as AWS S3 or Google Cloud Storage.
You can also read from cloud storage such as Amazon S3 or Google Cloud Storage.
To do so, you need to install the necessary library under Druid's classpath in _all MiddleManager or Indexer processes_.
For S3, you can run the below command to install the [Hadoop AWS module](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/).
@ -336,7 +336,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|logParseExceptions|Boolean|If true, log an error message when a parsing exception occurs, containing information about the row where the error occurred.|no(default = false)|
|maxParseExceptions|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overrides `ignoreInvalidRows` if `maxParseExceptions` is defined.|no(default = unlimited)|
|useYarnRMJobStatusFallback|Boolean|If the Hadoop jobs created by the indexing task are unable to retrieve their completion status from the JobHistory server, and this parameter is true, the indexing task will try to fetch the application status from `http://<yarn-rm-address>/ws/v1/cluster/apps/<application-id>`, where `<yarn-rm-address>` is the value of `yarn.resourcemanager.webapp.address` in your Hadoop configuration. This flag is intended as a fallback for cases where an indexing task's jobs succeed, but the JobHistory server is unavailable, causing the indexing task to fail because it cannot determine the job statuses.|no (default = true)|
|awaitSegmentAvailabilityTimeoutMillis|Long|Milliseconds to wait for the newly indexed segments to become available for query after ingestion completes. If `<= 0`, no wait will occur. If `> 0`, the task will wait for the Coordinator to indicate that the new segments are available for querying. If the timeout expires, the task will exit as successful, but the segments were not confirmed to have become available for query.|no (default = 0)|
|awaitSegmentAvailabilityTimeoutMillis|Long|Milliseconds to wait for the newly indexed segments to become available for query after ingestion completes. If `<= 0`, no wait will occur. If `> 0`, the task will wait for the Coordinator to indicate that the new segments are available for querying. If the timeout expires, the task will exit as successful, but the segments were not confirmed to have become available for query.|no (default = 0)|
### `jobProperties`

View File

@ -30,12 +30,15 @@ For general information on native batch indexing and parallel task indexing, see
## S3 input source
:::info
You need to include the [`druid-s3-extensions`](../development/extensions-core/s3.md) as an extension to use the S3 input source.
You need to include the [`druid-s3-extensions`](../development/extensions-core/s3.md) as an extension to use the S3 input source.
:::
The S3 input source reads objects directly from S3. You can specify either:
- a list of S3 URI strings
- a list of S3 location prefixes that attempts to list the contents and ingest
* a list of S3 URI strings
* a list of S3 location prefixes that attempts to list the contents and ingest
all objects contained within the locations.
The S3 input source is splittable. Therefore, you can use it with the [Parallel task](./native-batch.md). Each worker task of `index_parallel` reads one or multiple objects.
@ -76,7 +79,6 @@ Sample specs:
...
```
```json
...
"ioConfig": {
@ -210,13 +212,17 @@ Properties Object:
|assumeRoleExternalId|A unique identifier that might be required when you assume a role in another account [see](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_credentials_temp_request.html)|None|no|
:::info
**Note:** If `accessKeyId` and `secretAccessKey` are not given, the default [S3 credentials provider chain](../development/extensions-core/s3.md#s3-authentication-methods) is used.
If `accessKeyId` and `secretAccessKey` are not given, the default [S3 credentials provider chain](../development/extensions-core/s3.md#s3-authentication-methods) is used.
:::
## Google Cloud Storage input source
:::info
You need to include the [`druid-google-extensions`](../development/extensions-core/google.md) as an extension to use the Google Cloud Storage input source.
You need to include the [`druid-google-extensions`](../development/extensions-core/google.md) as an extension to use the Google Cloud Storage input source.
:::
The Google Cloud Storage input source is to support reading objects directly
@ -261,7 +267,6 @@ Sample specs:
...
```
```json
...
"ioConfig": {
@ -300,16 +305,18 @@ Google Cloud Storage object:
|path|The path where data is located.|None|yes|
|systemFields|JSON array of system fields to return as part of input rows. Possible values: `__file_uri` (Google Cloud Storage URI starting with `gs://`), `__file_bucket` (GCS bucket), and `__file_path` (GCS key).|None|no|
## Azure input source
## Azure input source
:::info
You need to include the [`druid-azure-extensions`](../development/extensions-core/azure.md) as an extension to use the Azure input source.
You need to include the [`druid-azure-extensions`](../development/extensions-core/azure.md) as an extension to use the Azure input source.
:::
The Azure input source (that uses the type `azureStorage`) reads objects directly from Azure Blob store or Azure Data Lake sources. You can
specify objects as a list of file URI strings or prefixes. You can split the Azure input source for use with [Parallel task](./native-batch.md) indexing and each worker task reads one chunk of the split data.
The `azureStorage` input source is a new schema for Azure input sources that allows you to specify which storage account files should be ingested from. We recommend that you update any specs that use the old `azure` schema to use the new `azureStorage` schema. The new schema provides more functionality than the older `azure` schema.
The `azureStorage` input source is a new schema for Azure input sources that allows you to specify which storage account files should be ingested from. We recommend that you update any specs that use the old `azure` schema to use the new `azureStorage` schema. The new schema provides more functionality than the older `azure` schema.
Sample specs:
@ -347,7 +354,6 @@ Sample specs:
...
```
```json
...
"ioConfig": {
@ -379,7 +385,7 @@ Sample specs:
|objects|JSON array of Azure objects to ingest.|None|One of the following must be set:`uris`, `prefixes`, or `objects`.|
|objectGlob|A glob for the object part of the Azure URI. In the URI `azureStorage://foo/bar/file.json`, the glob is applied to `bar/file.json`.<br /><br />The glob must match the entire object part, not just the filename. For example, the glob `*.json` does not match `azureStorage://foo/bar/file.json` because the object part is `bar/file.json`, and the`*` does not match the slash. To match all objects ending in `.json`, use `**.json` instead.<br /><br />For more information, refer to the documentation for [`FileSystem#getPathMatcher`](https://docs.oracle.com/javase/8/docs/api/java/nio/file/FileSystem.html#getPathMatcher-java.lang.String-).|None|no|
|systemFields|JSON array of system fields to return as part of input rows. Possible values: `__file_uri` (Azure blob URI starting with `azureStorage://`), `__file_bucket` (Azure bucket), and `__file_path` (Azure object path).|None|no|
|properties|Properties object for overriding the default Azure configuration. See below for more information.|None|No (defaults will be used if not given)
|properties|Properties object for overriding the default Azure configuration. See below for more information.|None|No (defaults will be used if not given)|
Note that the Azure input source skips all empty objects only when `prefixes` is specified.
@ -390,14 +396,12 @@ The `objects` property can one of the following:
|bucket|Name of the Azure Blob Storage or Azure Data Lake storage account|None|yes|
|path|The container and path where data is located.|None|yes|
The `properties` property can be one of the following:
- `sharedAccessStorageToken`
- `key`
- `appRegistrationClientId`, `appRegistrationClientSecret`, and `tenantId`
- empty
* `sharedAccessStorageToken`
* `key`
* `appRegistrationClientId`, `appRegistrationClientSecret`, and `tenantId`
* empty
|Property|Description|Default|Required|
|--------|-----------|-------|---------|
@ -407,8 +411,7 @@ The `properties` property can be one of the following:
|appRegistrationClientSecret|The client secret of the Azure App registration to authenticate as|None|Yes if `appRegistrationClientId` is provided|
|tenantId|The tenant ID of the Azure App registration to authenticate as|None|Yes if `appRegistrationClientId` is provided|
#### `azure` input source
### Legacy `azure` input source
The Azure input source that uses the type `azure` is an older version of the Azure input type and is not recommended. It doesn't support specifying which storage account to ingest from. We recommend using the [`azureStorage` input source schema](#azure-input-source) instead since it provides more functionality.
@ -448,7 +451,6 @@ Sample specs:
...
```
```json
...
"ioConfig": {
@ -487,11 +489,12 @@ The `objects` property is:
|bucket|Name of the Azure Blob Storage or Azure Data Lake container|None|yes|
|path|The path where data is located.|None|yes|
## HDFS input source
:::info
You need to include the [`druid-hdfs-storage`](../development/extensions-core/hdfs.md) as an extension to use the HDFS input source.
You need to include the [`druid-hdfs-storage`](../development/extensions-core/hdfs.md) as an extension to use the HDFS input source.
:::
The HDFS input source is to support reading files directly
@ -580,10 +583,12 @@ in `druid.ingestion.hdfs.allowedProtocols`. See [HDFS input source security conf
The HTTP input source is to support reading files directly from remote sites via HTTP.
:::info
**Security notes:** Ingestion tasks run under the operating system account that runs the Druid processes, for example the Indexer, Middle Manager, and Peon. This means any user who can submit an ingestion task can specify an input source referring to any location that the Druid process can access. For example, using `http` input source, users may have access to internal network servers.
:::info Security notes
Ingestion tasks run under the operating system account that runs the Druid processes, for example the Indexer, Middle Manager, and Peon. This means any user who can submit an ingestion task can specify an input source referring to any location that the Druid process can access. For example, using `http` input source, users may have access to internal network servers.
The `http` input source is not limited to the HTTP or HTTPS protocols. It uses the Java URI class that supports HTTP, HTTPS, FTP, file, and jar protocols by default.
The `http` input source is not limited to the HTTP or HTTPS protocols. It uses the Java URI class that supports HTTP, HTTPS, FTP, file, and jar protocols by default.
:::
For more information about security best practices, see [Security overview](../operations/security-overview.md#best-practices).
@ -725,7 +730,7 @@ Sample spec:
|filter|A wildcard filter for files. See [here](http://commons.apache.org/proper/commons-io/apidocs/org/apache/commons/io/filefilter/WildcardFileFilter) for more information. Files matching the filter criteria are considered for ingestion. Files not matching the filter criteria are ignored.|yes if `baseDir` is specified|
|baseDir|Directory to search recursively for files to be ingested. Empty files under the `baseDir` will be skipped.|At least one of `baseDir` or `files` should be specified|
|files|File paths to ingest. Some files can be ignored to avoid ingesting duplicate files if they are located under the specified `baseDir`. Empty files will be skipped.|At least one of `baseDir` or `files` should be specified|
|systemFields|JSON array of system fields to return as part of input rows. Possible values: `__file_uri` (File URI starting with `file:`) and `__file_path` (file path).|None|no|
|systemFields|JSON array of system fields to return as part of input rows. Possible values: `__file_uri` (File URI starting with `file:`) and `__file_path` (file path).|no|
## Druid input source
@ -744,9 +749,9 @@ no `inputFormat` field needs to be specified in the ingestion spec when using th
The Druid input source can be used for a variety of purposes, including:
- Creating new datasources that are rolled-up copies of existing datasources.
- Changing the [partitioning or sorting](./partitioning.md) of a datasource to improve performance.
- Updating or removing rows using a [`transformSpec`](./ingestion-spec.md#transformspec).
* Creating new datasources that are rolled-up copies of existing datasources.
* Changing the [partitioning or sorting](./partitioning.md) of a datasource to improve performance.
* Updating or removing rows using a [`transformSpec`](./ingestion-spec.md#transformspec).
When using the Druid input source, the timestamp column shows up as a numeric field named `__time` set to the number
of milliseconds since the epoch (January 1, 1970 00:00:00 UTC). It is common to use this in the timestampSpec, if you
@ -813,16 +818,16 @@ rolled-up datasource `wikipedia_rollup` by grouping on hour, "countryName", and
```
:::info
Note: Older versions (0.19 and earlier) did not respect the timestampSpec when using the Druid input source. If you
have ingestion specs that rely on this and cannot rewrite them, set
[`druid.indexer.task.ignoreTimestampSpecForDruidInputSource`](../configuration/index.md#indexer-general-configuration)
to `true` to enable a compatibility mode where the timestampSpec is ignored.
Older versions (0.19 and earlier) did not respect the timestampSpec when using the Druid input source. If you have ingestion specs that rely on this and cannot rewrite them, set [`druid.indexer.task.ignoreTimestampSpecForDruidInputSource`](../configuration/index.md#indexer-general-configuration) to `true` to enable a compatibility mode where the timestampSpec is ignored.
:::
The [secondary partitioning method](native-batch.md#partitionsspec) determines the requisite number of concurrent worker tasks that run in parallel to complete ingestion with the Combining input source.
Set this value in `maxNumConcurrentSubTasks` in `tuningConfig` based on the secondary partitioning method:
- `range` or `single_dim` partitioning: greater than or equal to 1
- `hashed` or `dynamic` partitioning: greater than or equal to 2
* `range` or `single_dim` partitioning: greater than or equal to 1
* `hashed` or `dynamic` partitioning: greater than or equal to 2
For more information on the `maxNumConcurrentSubTasks` field, see [Implementation considerations](native-batch.md#implementation-considerations).
@ -866,7 +871,7 @@ The following is an example of an SQL input source spec:
The spec above will read all events from two separate SQLs for the interval `2013-01-01/2013-01-02`.
Each of the SQL queries will be run in its own sub-task and thus for the above example, there would be two sub-tasks.
**Recommended practices**
### Recommended practices
Compared to the other native batch input sources, SQL input source behaves differently in terms of reading the input data. Therefore, consider the following points before using this input source in a production environment:
@ -878,7 +883,6 @@ Compared to the other native batch input sources, SQL input source behaves diffe
* Similar to file-based input formats, any updates to existing data will replace the data in segments specific to the intervals specified in the `granularitySpec`.
## Combining input source
The Combining input source lets you read data from multiple input sources.
@ -928,7 +932,9 @@ The following is an example of a Combining input source spec:
## Iceberg input source
:::info
To use the Iceberg input source, load the extension [`druid-iceberg-extensions`](../development/extensions-contrib/iceberg.md).
:::
You use the Iceberg input source to read data stored in the Iceberg table format. For a given table, the input source scans up to the latest Iceberg snapshot from the configured Hive catalog. Druid ingests the underlying live data files using the existing input source formats.
@ -1133,13 +1139,15 @@ This input source provides the following filters: `and`, `equals`, `interval`, a
## Delta Lake input source
:::info
To use the Delta Lake input source, load the extension [`druid-deltalake-extensions`](../development/extensions-contrib/delta-lake.md).
:::
You can use the Delta input source to read data stored in a Delta Lake table. For a given table, the input source scans
the latest snapshot from the configured table. Druid ingests the underlying delta files from the table.
| Property|Description|Required|
| Property|Description|Required|
|---------|-----------|--------|
| type|Set this value to `delta`.|yes|
| tablePath|The location of the Delta table.|yes|
@ -1155,7 +1163,6 @@ on statistics collected when the non-partitioned table is created. In this scena
data that doesn't match the filter. To guarantee that the Delta Kernel prunes out unnecessary column values, only use
filters on partitioned columns.
`and` filter:
| Property | Description | Required |
@ -1217,7 +1224,6 @@ filters on partitioned columns.
| column | The table column to apply the filter on. | yes |
| value | The value to use in the filter. | yes |
The following is a sample spec to read all records from the Delta table `/delta-table/foo`:
```json

View File

@ -28,12 +28,14 @@ sidebar_label: JSON-based batch
:::
Apache Druid supports the following types of JSON-based batch indexing tasks:
- Parallel task indexing (`index_parallel`) that can run multiple indexing tasks concurrently. Parallel task works well for production ingestion tasks.
- Simple task indexing (`index`) that run a single indexing task at a time. Simple task indexing is suitable for development and test environments.
This topic covers the configuration for `index_parallel` ingestion specs.
For related information on batch indexing, see:
- [Batch ingestion method comparison table](./index.md#batch) for a comparison of batch ingestion methods.
- [Tutorial: Loading a file](../tutorials/tutorial-batch.md) for a tutorial on JSON-based batch ingestion.
- [Input sources](./input-sources.md) for possible input sources.
@ -97,7 +99,6 @@ By default, JSON-based batch ingestion replaces all data in the intervals in you
You can also perform concurrent append and replace tasks. For more information, see [Concurrent append and replace](./concurrent-append-replace.md)
#### Fully replacing existing segments using tombstones
:::info
@ -124,12 +125,12 @@ You want to re-ingest and overwrite with new data as follows:
Unless you set `dropExisting` to true, the result after ingestion with overwrite using the same `MONTH` `segmentGranularity` would be:
* **January**: 1 record
* **February**: 10 records
* **March**: 9 records
- **January**: 1 record
- **February**: 10 records
- **March**: 9 records
This may not be what it is expected since the new data has 0 records for January. Set `dropExisting` to true to replace the unneeded January segment with a tombstone.
## Parallel indexing example
The following example illustrates the configuration for a parallel indexing task.
@ -214,6 +215,7 @@ The following example illustrates the configuration for a parallel indexing task
}
}
```
</details>
## Parallel indexing configuration
@ -305,7 +307,7 @@ The segments split hint spec is used only for [`DruidInputSource`](./input-sourc
### `partitionsSpec`
The primary partition for Druid is time. You can define a secondary partitioning method in the partitions spec. Use the `partitionsSpec` type that applies for your [rollup](rollup.md) method.
The primary partition for Druid is time. You can define a secondary partitioning method in the partitions spec. Use the `partitionsSpec` type that applies for your [rollup](rollup.md) method.
For perfect rollup, you can use:
@ -366,7 +368,7 @@ In the `partial segment generation` phase, just like the Map phase in MapReduce,
the Parallel task splits the input data based on the split hint spec
and assigns each split to a worker task. Each worker task (type `partial_index_generate`) reads the assigned split, and partitions rows by the time chunk from `segmentGranularity` (primary partition key) in the `granularitySpec`
and then by the hash value of `partitionDimensions` (secondary partition key) in the `partitionsSpec`.
The partitioned data is stored in local storage of
The partitioned data is stored in local storage of
the [middleManager](../design/middlemanager.md) or the [indexer](../design/indexer.md).
The `partial segment merge` phase is similar to the Reduce phase in MapReduce.
@ -709,12 +711,14 @@ The returned result contains the worker task spec, a current task status if exis
"taskHistory": []
}
```
</details>
`http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}/history`
Returns the task attempt history of the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode.
## Segment pushing modes
While ingesting data using the parallel task indexing, Druid creates segments from the input data and pushes them. For segment pushing,
the parallel task index supports the following segment pushing modes based upon your type of [rollup](./rollup.md):
@ -743,10 +747,12 @@ This may help the higher priority tasks to finish earlier than lower priority ta
by assigning more task slots to them.
## Splittable input sources
Use the `inputSource` object to define the location where your index can read data. Only the native parallel task and simple task support the input source.
For details on available input sources see:
- [S3 input source](./input-sources.md#s3-input-source) (`s3`) reads data from AWS S3 storage.
- [S3 input source](./input-sources.md#s3-input-source) (`s3`) reads data from Amazon S3 storage.
- [Google Cloud Storage input source](./input-sources.md#google-cloud-storage-input-source) (`gs`) reads data from Google Cloud Storage.
- [Azure input source](./input-sources.md#azure-input-source) (`azure`) reads data from Azure Blob Storage and Azure Data Lake.
- [HDFS input source](./input-sources.md#hdfs-input-source) (`hdfs`) reads data from HDFS storage.

View File

@ -42,6 +42,7 @@ The following table outlines the high-level configuration options for a supervis
|`spec.dataSchema`|Object|The schema for the indexing task to use during ingestion. See [`dataSchema`](../ingestion/ingestion-spec.md#dataschema) for more information.|Yes|
|`spec.ioConfig`|Object|The I/O configuration object to define the connection and I/O-related settings for the supervisor and indexing tasks.|Yes|
|`spec.tuningConfig`|Object|The tuning configuration object to define performance-related settings for the supervisor and indexing tasks.|No|
|`suspended`|Boolean|Puts the supervisor in a suspended state|No|
### I/O configuration

View File

@ -22,15 +22,9 @@ title: "Tranquility"
~ under the License.
-->
[Tranquility](https://github.com/druid-io/tranquility/) is a separately distributed package for pushing
streams to Druid in real-time.
Tranquility has not been built against a version of Druid later than Druid 0.9.2
release. It may still work with the latest Druid servers, but not all features and functionality will be available
due to limitations of older Druid APIs on the Tranquility side.
[Tranquility](https://github.com/druid-io/tranquility/) was a separately distributed package for pushing
streams to Druid in real-time. It is not compatible with recent versions of Druid.
For new projects that require streaming ingestion, we recommend using Druid's native support for
[Apache Kafka](../ingestion/kafka-ingestion.md) or
[Amazon Kinesis](../ingestion/kinesis-ingestion.md).
For more details, check out the [Tranquility GitHub page](https://github.com/druid-io/tranquility/).

View File

@ -410,6 +410,7 @@ The following table lists the context parameters for the MSQ task engine:
| `skipTypeVerification` | INSERT or REPLACE<br /><br />During query validation, Druid validates that [string arrays](../querying/arrays.md) and [multi-value dimensions](../querying/multi-value-dimensions.md) are not mixed in the same column. If you are intentionally migrating from one to the other, use this context parameter to disable type validation.<br /><br />Provide the column list as comma-separated values or as a JSON array in string form.| empty list |
| `failOnEmptyInsert` | INSERT or REPLACE<br /><br /> When set to false (the default), an INSERT query generating no output rows will be no-op, and a REPLACE query generating no output rows will delete all data that matches the OVERWRITE clause. When set to true, an ingest query generating no output rows will throw an `InsertCannotBeEmpty` fault. | `false` |
| `storeCompactionState` | REPLACE<br /><br /> When set to true, a REPLACE query stores as part of each segment's metadata a `lastCompactionState` field that captures the various specs used to create the segment. Future compaction jobs skip segments whose `lastCompactionState` matches the desired compaction state. Works the same as [`storeCompactionState`](../ingestion/tasks.md#context-parameters) task context flag. | `false` |
| `removeNullBytes` | SELECT, INSERT or REPLACE<br /><br /> The MSQ engine cannot process null bytes in strings and throws `InvalidNullByteFault` if it encounters them in the source data. If the parameter is set to true, The MSQ engine will remove the null bytes in string fields when reading the data. | `false` |
## Joins

View File

@ -508,6 +508,19 @@ These metrics are only available if the `OshiSysMonitor` module is included.
|`sys/tcpv4/out/rsts`|Total "out reset" packets sent to reset the connection||Generally 0|
|`sys/tcpv4/retrans/segs`|Total segments re-transmitted||Varies|
## S3 multi-part upload
These metrics are only available if the `druid-s3-extensions` module is included and if certain specific features are being used: MSQ export to S3, durable intermediate storage on S3.
|Metric|Description|Dimensions|Normal value|
|------|-----------|----------|------------|
|`s3/upload/part/queueSize`|Number of items currently waiting in queue to be uploaded to S3. Each item in the queue corresponds to a single part in a multi-part upload.||Varies|
|`s3/upload/part/queuedTime`|Milliseconds spent by a single item (or part) in queue before it starts getting uploaded to S3.|`uploadId`, `partNumber`|Varies|
|`s3/upload/part/time`|Milliseconds taken to upload a single part of a multi-part upload to S3.|`uploadId`, `partNumber`|Varies|
|`s3/upload/total/time`|Milliseconds taken for uploading all parts of a multi-part upload to S3.|`uploadId`|Varies|
|`s3/upload/total/bytes`|Total bytes uploaded to S3 during a multi-part upload.|`uploadId`|Varies|
## Cgroup
These metrics are available on operating systems with the cgroup kernel feature. All the values are derived by reading from `/sys/fs/cgroup`.

View File

@ -385,7 +385,7 @@ Supported query contexts:
|`forceLimitPushDown`|When all fields in the orderby are part of the grouping key, the Broker will push limit application down to the Historical processes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.|false|
|`applyLimitPushDownToSegment`|If Broker pushes limit down to queryable nodes (historicals, peons) then limit results during segment scan. This context value can be used to override `druid.query.groupBy.applyLimitPushDownToSegment`.|true|
|`groupByEnableMultiValueUnnesting`|Safety flag to enable/disable the implicit unnesting on multi value column's as part of the grouping key. 'true' indicates multi-value grouping keys are unnested. 'false' returns an error if a multi value column is found as part of the grouping key.|true|
|`deferExpressionDimensions`|When an entry in `dimensions` references an `expression` virtual column, this property influences whether expression evaluation is deferred from cursor processing to the merge step. Options are:<ul><li>`fixedWidth`: Defer expressions with fixed-width inputs (numeric and dictionary-encoded string).</li><li>`fixedWidthNonNumeric`: Defer expressions with fixed-width inputs (numeric and dictionary-encoded string), unless the expression output and all inputs are numeric.</li><li>`singleString`: Defer string-typed expressions with a single dictionary-encoded string input.</li><li>`always`: Defer all expressions. May require building dictionaries for expression inputs.</li></ul><br />These properties only take effect when the `groupBy` query can be vectorized. Non-vectorized queries only defer string-typed expressions of single string inputs.|`fixedWidthNonNumeric`|
#### Array based result rows

View File

@ -242,5 +242,5 @@ For an example, see [Ingesting arrays: Native batch and streaming ingestion](../
* For SQL-based batch ingestion, include the [query context parameter](../multi-stage-query/reference.md#context-parameters) `"arrayIngestMode": "array"` and reference the relevant array type (`VARCHAR ARRAY`, `BIGINT ARRAY`, or `DOUBLE ARRAY`) in the [EXTEND clause](../multi-stage-query/reference.md#extern-function) that lists the column names and data types.
For examples, see [Ingesting arrays: SQL-based ingestion](../querying/arrays.md#sql-based-ingestion).
As a best practice, always use the ARRAY data type in your input schema. If you want to ingest MVDs, explicitly wrap the string array in [ARRAY_TO_MV](../querying/sql-functions.md#array_to_mv). For an example, see [Multi-value dimensions: SQL-based ingestion](/querying/multi-value-dimensions.md#sql-based-ingestion).
As a best practice, always use the ARRAY data type in your input schema. If you want to ingest MVDs, explicitly wrap the string array in [ARRAY_TO_MV](../querying/sql-functions.md#array_to_mv). For an example, see [Multi-value dimensions: SQL-based ingestion](../querying/multi-value-dimensions.md#sql-based-ingestion).

View File

@ -52,6 +52,10 @@ This property takes precedence over `maxSubqueryRows`.
You can set both `maxSubqueryRows` and `maxSubqueryBytes` at cluster level and override them in individual queries.
See [Overriding default query context values](../configuration#overriding-default-query-context-values) for more information.
Make sure you enable the Broker monitor `SubqueryCountStatsMonitor` so that Druid emits metrics for subquery statistics.
To do this, add `org.apache.druid.server.metrics.SubqueryCountStatsMonitor` to the `druid.monitoring.monitors` property in your Broker's `runtime.properties` configuration file.
See [Metrics monitors](../configuration/index.md#metrics-monitors) for more information.
## Learn more
See the following topics for more information:

View File

@ -28,7 +28,7 @@ In general, when we introduce new features and behaviors into Apache Druid, we m
The guides in this section outline breaking changes introduced in Druid 25.0.0 and later. Each guide provides instructions to migrate to new features.
## Migrate to arrays from multi-value dimensions
## Migrate from multi-value dimensions to arrays
Druid now supports SQL-compliant array types. Whenever possible, you should use the array type over multi-value dimensions. See [Migration guide: MVDs to arrays](migr-mvd-array.md).
@ -36,6 +36,6 @@ Druid now supports SQL-compliant array types. Whenever possible, you should use
Druid encodes string columns into dictionaries for better compression. Front-coded dictionary encoding reduces storage and improves performance by optimizing for strings that share similar beginning substrings. See [Migration guide: front-coded dictionaries](migr-front-coded-dict.md) for more information.
## Migrate to `maxSubqueryBytes` from `maxSubqueryRows`
## Migrate from `maxSubqueryRows` to `maxSubqueryBytes`
Druid allows you to set a byte-based limit on subquery size to prevent Brokers from running out of memory when handling large subqueries. The byte-based subquery limit overrides Druid's row-based subquery limit. We recommend that you move towards using byte-based limits starting in Druid 30.0.0. See [Migration guide: subquery limit](migr-subquery-limit.md) for more information.

View File

@ -97,6 +97,19 @@ This section contains detailed release notes separated by areas.
### Upgrade notes
#### Front-coded dictionaries
<!--Carry this forward until 32. Then move it to incompatible changes -->
In Druid 32.0.0, the front coded dictionaries feature will be turned on by default. Front-coded dictionaries reduce storage and improve performance by optimizing for strings where the front part looks similar.
Once this feature is on, you cannot easily downgrade to an earlier version that does not support the feature.
For more information, see [Migration guide: front-coded dictionaries](./migr-front-coded-dict.md).
If you're already using this feature, you don't need to take any action.
### Incompatible changes
### Developer notes

View File

@ -26,6 +26,18 @@ The upgrade notes assume that you are upgrading from the Druid version that imme
For the full release notes for a specific version, see the [releases page](https://github.com/apache/druid/releases).
## Announcements
#### Front-coded dictionaries
In Druid 32.0.0, the front coded dictionaries feature will be turned on by default. Front-coded dictionaries reduce storage and improve performance by optimizing for strings where the front part looks similar.
Once this feature is on, you cannot easily downgrade to an earlier version that does not support the feature.
For more information, see [Migration guide: front-coded dictionaries](./migr-front-coded-dict.md).
If you're already using this feature, you don't need to take any action.
## 30.0.0
### Upgrade notes

View File

@ -1,7 +1,7 @@
---
id: index
title: "Quickstart (local)"
sidebar_label: Quickstart (local)
title: "Local quickstart"
sidebar_label: Local quickstart
---
<!--
@ -53,7 +53,7 @@ dedicated user account for running Druid.
## Install Druid
Download the [{{DRUIDVERSION}} release](https://www.apache.org/dyn/closer.cgi?path=/druid/{{DRUIDVERSION}}/apache-druid-{{DRUIDVERSION}}-bin.tar.gz) from Apache Druid.
Download the [{{DRUIDVERSION}} release](https://druid.apache.org/downloads/) from Apache Druid.
In your terminal, extract the file and change directories to the distribution directory:

View File

@ -21,9 +21,16 @@ PWD="$(pwd)"
WHEREAMI="$(dirname "$0")"
WHEREAMI="$(cd "$WHEREAMI" && pwd)"
if [ -x "$(command -v python2)" ]
if [ -x "$(command -v python3)" ]
then
exec python2 "$WHEREAMI/dsql-main" "$@"
exec python3 "$WHEREAMI/dsql-main-py3" "$@"
elif [ -x "$(command -v python2)" ]
then
echo "Warning: Support for Python 2 will be removed in the future. Please consider upgrading to Python 3"
exec python2 "$WHEREAMI/dsql-main-py2" "$@"
elif [ -x "$(command -v python)" ]
then
exec python "$WHEREAMI/dsql-main-py3" "$@"
else
exec "$WHEREAMI/dsql-main" "$@"
echo "python interepreter not found"
fi

5
examples/bin/dsql-main → examples/bin/dsql-main-py2 Executable file → Normal file
View File

@ -17,6 +17,11 @@
# specific language governing permissions and limitations
# under the License.
# NOTE:
# Any feature updates to this script must also be reflected in
# `dsql-main-py3` so that intended changes work for users using
# Python 2 or 3.
from __future__ import print_function
import argparse

523
examples/bin/dsql-main-py3 Executable file
View File

@ -0,0 +1,523 @@
#!/usr/bin/env python
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
# NOTE:
# Any feature updates to this script must also be reflected in
# `dsql-main-py2` so that intended changes work for users using
# Python 2 or 3.
import argparse
import base64
import collections
import csv
import errno
import json
import numbers
import os
import re
import readline
import ssl
import sys
import time
import unicodedata
import urllib.request
import urllib.error
class DruidSqlException(Exception):
def friendly_message(self):
return getattr(self, 'message', 'Query failed')
def write_to(self, f):
f.write('\x1b[31m')
f.write(self.friendly_message())
f.write('\x1b[0m')
f.write('\n')
f.flush()
def do_query_with_args(url, sql, context, args):
return do_query(url, sql, context, args.timeout, args.user, args.ignore_ssl_verification, args.cafile, args.capath, args.certchain, args.keyfile, args.keypass)
def do_query(url, sql, context, timeout, user, ignore_ssl_verification, ca_file, ca_path, cert_chain, key_file, key_pass):
json_decoder = json.JSONDecoder(object_pairs_hook=collections.OrderedDict)
try:
if timeout <= 0:
timeout = None
query_context = context
elif int(context.get('timeout', 0)) / 1000. < timeout:
query_context = context.copy()
query_context['timeout'] = timeout * 1000
sql_json = json.dumps({'query' : sql, 'context' : query_context})
# SSL stuff
ssl_context = None
if ignore_ssl_verification or ca_file is not None or ca_path is not None or cert_chain is not None:
ssl_context = ssl.create_default_context()
if ignore_ssl_verification:
ssl_context.check_hostname = False
ssl_context.verify_mode = ssl.CERT_NONE
elif ca_path is not None:
ssl_context.load_verify_locations(cafile=ca_file, capath=ca_path)
else:
ssl_context.load_cert_chain(certfile=cert_chain, keyfile=key_file, password=key_pass)
req = urllib.request.Request(url, sql_json.encode('utf-8'), {'Content-Type' : 'application/json'})
if user:
req.add_header("Authorization", "Basic %s" % base64.b64encode(user.encode('utf-8')).decode('utf-8'))
response = urllib.request.urlopen(req, None, timeout, context=ssl_context)
first_chunk = True
eof = False
buf = ''
while not eof or len(buf) > 0:
while True:
try:
# Remove starting ','
buf = buf.lstrip(',')
obj, sz = json_decoder.raw_decode(buf)
yield obj
buf = buf[sz:]
except ValueError as e:
# Maybe invalid JSON, maybe partial object; it's hard to tell with this library.
if eof and buf.rstrip() == ']':
# Stream done and all objects read.
buf = ''
break
elif eof or len(buf) > 256 * 1024:
# If we read more than 256KB or if it's eof then report the parse error.
raise
else:
# Stop reading objects, get more from the stream instead.
break
# Read more from the http stream
if not eof:
chunk = response.read(8192).decode('utf-8')
if chunk:
buf = buf + chunk
if first_chunk:
# Remove starting '['
buf = buf.lstrip('[')
else:
# Stream done. Keep reading objects out of buf though.
eof = True
except urllib.error.URLError as e:
raise_friendly_error(e)
def raise_friendly_error(e):
if isinstance(e, urllib.error.HTTPError):
text = e.read().strip()
error_obj = {}
try:
error_obj = dict(json.loads(text))
except:
pass
if e.code == 500 and 'errorMessage' in error_obj:
error_text = ''
if error_obj['error'] != 'Unknown exception':
error_text = error_text + error_obj['error'] + ': '
if error_obj['errorClass']:
error_text = error_text + str(error_obj['errorClass']) + ': '
error_text = error_text + str(error_obj['errorMessage'])
if error_obj['host']:
error_text = error_text + ' (' + str(error_obj['host']) + ')'
raise DruidSqlException(error_text)
elif e.code == 405:
error_text = 'HTTP Error {0}: {1}\n{2}'.format(e.code, e.reason + " - Are you using the correct broker URL and " +\
"is druid.sql.enabled set to true on your broker?", text)
raise DruidSqlException(error_text)
else:
raise DruidSqlException("HTTP Error {0}: {1}\n{2}".format(e.code, e.reason, text))
else:
raise DruidSqlException(str(e))
def to_utf8(value):
if value is None:
return b""
elif isinstance(value, str):
return value.encode("utf-8")
else:
return str(value).encode("utf-8")
def to_tsv(values, delimiter):
return delimiter.join(to_utf8(v).replace(delimiter, '') for v in values)
def print_csv(rows, header):
csv_writer = csv.writer(sys.stdout)
first = True
for row in rows:
if first and header:
csv_writer.writerow(list(to_utf8(k) for k in row.keys()))
first = False
values = []
for key, value in row.iteritems():
values.append(to_utf8(value))
csv_writer.writerow(values)
def print_tsv(rows, header, tsv_delimiter):
first = True
for row in rows:
if first and header:
print(to_tsv(row.keys(), tsv_delimiter))
first = False
values = []
for key, value in row.iteritems():
values.append(value)
print(to_tsv(values, tsv_delimiter))
def print_json(rows):
for row in rows:
print(json.dumps(row))
def table_to_printable_value(value):
# Unicode string, trimmed with control characters removed
if value is None:
return u"NULL"
else:
return to_utf8(value).strip().decode('utf-8').translate(dict.fromkeys(range(32)))
def table_compute_string_width(v):
normalized = unicodedata.normalize('NFC', v)
width = 0
for c in normalized:
ccategory = unicodedata.category(c)
cwidth = unicodedata.east_asian_width(c)
if ccategory == 'Cf':
# Formatting control, zero width
pass
elif cwidth == 'F' or cwidth == 'W':
# Double-wide character, prints in two columns
width = width + 2
else:
# All other characters
width = width + 1
return width
def table_compute_column_widths(row_buffer):
widths = None
for values in row_buffer:
values_widths = [table_compute_string_width(v) for v in values]
if not widths:
widths = values_widths
else:
i = 0
for v in values:
widths[i] = max(widths[i], values_widths[i])
i = i + 1
return widths
def table_print_row(values, column_widths, column_types):
vertical_line = '\u2502'
for i in range(len(values)):
padding = ' ' * max(0, column_widths[i] - table_compute_string_width(values[i]))
if column_types and column_types[i] == 'n':
print(vertical_line + ' ' + padding + values[i] + ' ', end="")
else:
print(vertical_line + ' ' + values[i] + padding + ' ', end="")
print(vertical_line)
def table_print_header(values, column_widths):
# Line 1
left_corner = '\u250C'
horizontal_line = '\u2500'
top_tee = '\u252C'
right_corner = '\u2510'
print(left_corner, end="")
for i in range(0, len(column_widths)):
print(horizontal_line * max(0, column_widths[i] + 2), end="")
if i + 1 < len(column_widths):
print(top_tee, end="")
print(right_corner)
# Line 2
table_print_row(values, column_widths, None)
# Line 3
left_tee = '\u251C'
cross = '\u253C'
right_tee = '\u2524'
print(left_tee, end="")
for i in range(0, len(column_widths)):
print(horizontal_line * max(0, column_widths[i] + 2), end="")
if i + 1 < len(column_widths):
print(cross, end="")
print(right_tee)
def table_print_bottom(column_widths):
left_corner = '\u2514'
right_corner = '\u2518'
bottom_tee = '\u2534'
horizontal_line = '\u2500'
print(left_corner, end="")
for i in range(0, len(column_widths)):
print(horizontal_line * max(0, column_widths[i] + 2), end="")
if i + 1 < len(column_widths):
print(bottom_tee, end="")
print(right_corner)
def table_print_row_buffer(row_buffer, column_widths, column_types):
first = True
for values in row_buffer:
if first:
table_print_header(values, column_widths)
first = False
else:
table_print_row(values, column_widths, column_types)
def print_table(rows):
start = time.time()
nrows = 0
first = True
# Buffer some rows before printing.
rows_to_buffer = 500
row_buffer = []
column_types = []
column_widths = None
for row in rows:
nrows = nrows + 1
if first:
row_buffer.append([table_to_printable_value(k) for k in row.keys()])
for k in row.keys():
if isinstance(row[k], numbers.Number):
column_types.append('n')
else:
column_types.append('s')
first = False
values = [table_to_printable_value(v) for k, v in row.items()]
if rows_to_buffer > 0:
row_buffer.append(values)
rows_to_buffer = rows_to_buffer - 1
else:
if row_buffer:
column_widths = table_compute_column_widths(row_buffer)
table_print_row_buffer(row_buffer, column_widths, column_types)
del row_buffer[:]
table_print_row(values, column_widths, column_types)
if row_buffer:
column_widths = table_compute_column_widths(row_buffer)
table_print_row_buffer(row_buffer, column_widths, column_types)
if column_widths:
table_print_bottom(column_widths)
print("Retrieved {0:,d} row{1:s} in {2:.2f}s.".format(nrows, 's' if nrows != 1 else '', time.time() - start))
print("")
def display_query(url, sql, context, args):
rows = do_query_with_args(url, sql, context, args)
if args.format == 'csv':
print_csv(rows, args.header)
elif args.format == 'tsv':
print_tsv(rows, args.header, args.tsv_delimiter)
elif args.format == 'json':
print_json(rows)
elif args.format == 'table':
print_table(rows)
def sql_literal_escape(s):
if s is None:
return "''"
elif isinstance(s, str):
ustr = s
else:
ustr = str(s)
escaped = ["U&'"]
for c in ustr:
ccategory = unicodedata.category(c)
if ccategory.startswith('L') or ccategory.startswith('N') or c == ' ':
escaped.append(c)
else:
escaped.append(u'\\')
escaped.append('%04x' % ord(c))
escaped.append("'")
return ''.join(escaped)
def make_readline_completer(url, context, args):
starters = [
'EXPLAIN PLAN FOR',
'SELECT'
]
middlers = [
'FROM',
'WHERE',
'GROUP BY',
'ORDER BY',
'LIMIT'
]
def readline_completer(text, state):
if readline.get_begidx() == 0:
results = [x for x in starters if x.startswith(text.upper())] + [None]
else:
results = ([x for x in middlers if x.startswith(text.upper())] + [None])
return results[state] + " "
print("Connected to [" + args.host + "].")
print("")
return readline_completer
def main():
parser = argparse.ArgumentParser(description='Druid SQL command-line client.')
parser_cnn = parser.add_argument_group('Connection options')
parser_fmt = parser.add_argument_group('Formatting options')
parser_oth = parser.add_argument_group('Other options')
parser_cnn.add_argument('--host', '-H', type=str, default='http://localhost:8082/', help='Druid query host or url, like https://localhost:8282/')
parser_cnn.add_argument('--user', '-u', type=str, help='HTTP basic authentication credentials, like user:password')
parser_cnn.add_argument('--timeout', type=int, default=0, help='Timeout in seconds')
parser_cnn.add_argument('--cafile', type=str, help='Path to SSL CA file for validating server certificates. See load_verify_locations() in https://docs.python.org/3/library/ssl.html#ssl.SSLContext.')
parser_cnn.add_argument('--capath', type=str, help='SSL CA path for validating server certificates. See load_verify_locations() in https://docs.python.org/3/library/ssl.html#ssl.SSLContext.')
parser_cnn.add_argument('--ignore-ssl-verification', '-k', action='store_true', default=False, help='Skip verification of SSL certificates.')
parser_fmt.add_argument('--format', type=str, default='table', choices=('csv', 'tsv', 'json', 'table'), help='Result format')
parser_fmt.add_argument('--header', action='store_true', help='Include header row for formats "csv" and "tsv"')
parser_fmt.add_argument('--tsv-delimiter', type=str, default='\t', help='Delimiter for format "tsv"')
parser_oth.add_argument('--context-option', '-c', type=str, action='append', help='Set context option for this connection, see https://druid.apache.org/docs/latest/querying/sql.html#connection-context for options')
parser_oth.add_argument('--execute', '-e', type=str, help='Execute single SQL query')
parser_cnn.add_argument('--certchain', type=str, help='Path to SSL certificate used to connect to server. See load_cert_chain() in https://docs.python.org/3/library/ssl.html#ssl.SSLContext.')
parser_cnn.add_argument('--keyfile', type=str, help='Path to private SSL key used to connect to server. See load_cert_chain() in https://docs.python.org/3/library/ssl.html#ssl.SSLContext.')
parser_cnn.add_argument('--keypass', type=str, help='Password to private SSL key file used to connect to server. See load_cert_chain() in https://docs.python.org/3/library/ssl.html#ssl.SSLContext.')
args = parser.parse_args()
# Build broker URL
url = args.host.rstrip('/') + '/druid/v2/sql/'
if not url.startswith('http:') and not url.startswith('https:'):
url = 'http://' + url
# Build context
context = {}
if args.context_option:
for opt in args.context_option:
kv = opt.split("=", 1)
if len(kv) != 2:
raise ValueError('Invalid context option, should be key=value: ' + opt)
if re.match(r"^\d+$", kv[1]):
context[kv[0]] = int(kv[1])
else:
context[kv[0]] = kv[1]
if args.execute:
display_query(url, args.execute, context, args)
else:
# interactive mode
print("Welcome to dsql, the command-line client for Druid SQL.")
readline_history_file = os.path.expanduser("~/.dsql_history")
readline.parse_and_bind('tab: complete')
readline.set_history_length(500)
readline.set_completer(make_readline_completer(url, context, args))
try:
readline.read_history_file(readline_history_file)
except IOError:
# IOError can happen if the file doesn't exist.
pass
print("Type \"\\h\" for help.")
while True:
sql = ''
while not sql.endswith(';'):
prompt = "dsql> " if sql == '' else 'more> '
try:
more_sql = input(prompt)
except EOFError:
sys.stdout.write('\n')
sys.exit(1)
if sql == '' and more_sql.startswith('\\'):
# backslash command
dmatch = re.match(r'^\\d(S?)(\+?)(\s+.*?|)\s*$', more_sql)
if dmatch:
include_system = dmatch.group(1)
extra_info = dmatch.group(2)
arg = dmatch.group(3).strip()
if arg:
sql = "SELECT TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME, DATA_TYPE FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_NAME = " + sql_literal_escape(arg)
if not include_system:
sql = sql + " AND TABLE_SCHEMA = 'druid'"
# break to execute sql
break
else:
sql = "SELECT TABLE_SCHEMA, TABLE_NAME FROM INFORMATION_SCHEMA.TABLES"
if not include_system:
sql = sql + " WHERE TABLE_SCHEMA = 'druid'"
# break to execute sql
break
hmatch = re.match(r'^\\h\s*$', more_sql)
if hmatch:
print("Commands:")
print(" \\d show tables")
print(" \\dS show tables, including system tables")
print(" \\d table_name describe table")
print(" \\h show this help")
print(" \\q exit this program")
print("Or enter a SQL query ending with a semicolon (;).")
continue
qmatch = re.match(r'^\\q\s*$', more_sql)
if qmatch:
sys.exit(0)
print("No such command: " + more_sql)
else:
sql = (sql + ' ' + more_sql).strip()
try:
readline.write_history_file(readline_history_file)
display_query(url, sql.rstrip(';'), context, args)
except DruidSqlException as e:
e.write_to(sys.stdout)
except KeyboardInterrupt:
sys.stdout.write("Query interrupted\n")
sys.stdout.flush()
try:
main()
except DruidSqlException as e:
e.write_to(sys.stderr)
sys.exit(1)
except KeyboardInterrupt:
sys.exit(1)
except IOError as e:
if e.errno == errno.EPIPE:
sys.exit(1)
else:
raise

View File

@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query", "druid-parquet-extensions"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.

View File

@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query", "druid-parquet-extensions"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.

View File

@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query", "druid-parquet-extensions"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.

View File

@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query", "druid-parquet-extensions"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.

View File

@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query", "druid-parquet-extensions"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.

View File

@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query", "druid-parquet-extensions"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.

View File

@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query", "druid-parquet-extensions"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.

View File

@ -24,7 +24,7 @@
# If you specify `druid.extensions.loadList=[]`, Druid won't load any extension from file system.
# If you don't specify `druid.extensions.loadList`, Druid will load all the extensions under root extension directory.
# More info: https://druid.apache.org/docs/latest/operations/including-extensions.html
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query"]
druid.extensions.loadList=["druid-hdfs-storage", "druid-kafka-indexing-service", "druid-datasketches", "druid-multi-stage-query", "druid-parquet-extensions"]
# If you have a different version of Hadoop, place your Hadoop client jar files in your hadoop-dependencies directory
# and uncomment the line below to point to your directory.

View File

@ -32,7 +32,6 @@ import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.util.Optionality;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.sql.calcite.aggregation.Aggregation;
@ -130,7 +129,7 @@ public abstract class CompressedBigDecimalSqlAggregatorBase implements SqlAggreg
// create the factory
AggregatorFactory aggregatorFactory = factoryCreator.create(
StringUtils.format("%s:agg", name),
name,
sumColumnName,
size,
scale,

View File

@ -165,9 +165,9 @@ public abstract class CompressedBigDecimalSqlAggregatorTestBase extends BaseCalc
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.granularity(Granularities.ALL)
.aggregators(
factoryCreator.create("a0:agg", "m1", 9, 9, false),
factoryCreator.create("a1:agg", "m2", 9, 9, false),
factoryCreator.create("a2:agg", "dim1", 9, 9, false)
factoryCreator.create("a0", "m1", 9, 9, false),
factoryCreator.create("a1", "m2", 9, 9, false),
factoryCreator.create("a2", "dim1", 9, 9, false)
)
.context(QUERY_CONTEXT_DEFAULT)
@ -190,7 +190,7 @@ public abstract class CompressedBigDecimalSqlAggregatorTestBase extends BaseCalc
.dataSource(CalciteTests.DATASOURCE1)
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.granularity(Granularities.ALL)
.aggregators(factoryCreator.create("a0:agg", "dim1", 9, 9, true))
.aggregators(factoryCreator.create("a0", "dim1", 9, 9, true))
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
@ -218,9 +218,9 @@ public abstract class CompressedBigDecimalSqlAggregatorTestBase extends BaseCalc
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.granularity(Granularities.ALL)
.aggregators(
factoryCreator.create("a0:agg", "m1", 9, 3, false),
factoryCreator.create("a1:agg", "m2", 9, 3, false),
factoryCreator.create("a2:agg", "dim1", 9, 3, false)
factoryCreator.create("a0", "m1", 9, 3, false),
factoryCreator.create("a1", "m2", 9, 3, false),
factoryCreator.create("a2", "dim1", 9, 3, false)
)
.context(QUERY_CONTEXT_DEFAULT)
.build()
@ -249,9 +249,9 @@ public abstract class CompressedBigDecimalSqlAggregatorTestBase extends BaseCalc
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.granularity(Granularities.ALL)
.aggregators(
factoryCreator.create("a0:agg", "m1", 9, 9, false),
factoryCreator.create("a1:agg", "m2", 9, 9, false),
factoryCreator.create("a2:agg", "dim1", 9, 9, false)
factoryCreator.create("a0", "m1", 9, 9, false),
factoryCreator.create("a1", "m2", 9, 9, false),
factoryCreator.create("a2", "dim1", 9, 9, false)
)
.context(QUERY_CONTEXT_DEFAULT)
.build()
@ -275,9 +275,9 @@ public abstract class CompressedBigDecimalSqlAggregatorTestBase extends BaseCalc
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.granularity(Granularities.ALL)
.aggregators(
factoryCreator.create("a0:agg", "m1", 6, 9, false),
factoryCreator.create("a1:agg", "m2", 6, 9, false),
factoryCreator.create("a2:agg", "dim1", 6, 9, false)
factoryCreator.create("a0", "m1", 6, 9, false),
factoryCreator.create("a1", "m2", 6, 9, false),
factoryCreator.create("a2", "dim1", 6, 9, false)
)
.context(QUERY_CONTEXT_DEFAULT)
.build()

View File

@ -128,6 +128,7 @@
<dependency>
<groupId>org.apache.curator</groupId>
<artifactId>curator-client</artifactId>
<version>5.5.0</version>
<scope>provided</scope>
</dependency>
<dependency>

View File

@ -44,12 +44,17 @@
<artifactId>delta-kernel-api</artifactId>
<version>${delta-kernel.version}</version>
</dependency>
<dependency>
<groupId>io.delta</groupId>
<artifactId>delta-kernel-defaults</artifactId>
<version>${delta-kernel.version}</version>
</dependency>
<dependency>
<groupId>io.delta</groupId>
<artifactId>delta-storage</artifactId>
<version>${delta-kernel.version}</version>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client-api</artifactId>

View File

@ -42,6 +42,7 @@ import io.delta.kernel.types.StructField;
import io.delta.kernel.types.StructType;
import io.delta.kernel.utils.CloseableIterator;
import io.delta.kernel.utils.FileStatus;
import io.delta.storage.LogStore;
import org.apache.druid.data.input.ColumnsFilter;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputRowSchema;
@ -151,7 +152,8 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
}
} else {
final Table table = Table.forPath(engine, tablePath);
final Snapshot latestSnapshot = table.getLatestSnapshot(engine);
final Snapshot latestSnapshot = getLatestSnapshotForTable(table, engine);
final StructType fullSnapshotSchema = latestSnapshot.getSchema(engine);
final StructType prunedSchema = pruneSchema(
fullSnapshotSchema,
@ -207,7 +209,7 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
final Snapshot latestSnapshot;
final Table table = Table.forPath(engine, tablePath);
try {
latestSnapshot = table.getLatestSnapshot(engine);
latestSnapshot = getLatestSnapshotForTable(table, engine);
}
catch (TableNotFoundException e) {
throw InvalidInput.exception(e, "tablePath[%s] not found.", tablePath);
@ -331,6 +333,21 @@ public class DeltaInputSource implements SplittableInputSource<DeltaSplit>
);
}
private Snapshot getLatestSnapshotForTable(final Table table, final Engine engine)
{
// Setting the LogStore class loader before calling the Delta Kernel snapshot API is required as a workaround with
// the 3.2.0 Delta Kernel because the Kernel library cannot instantiate the LogStore class otherwise. Please see
// https://github.com/delta-io/delta/issues/3299 for details. This workaround can be removed once the issue is fixed.
final ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader();
try {
Thread.currentThread().setContextClassLoader(LogStore.class.getClassLoader());
return table.getLatestSnapshot(engine);
}
finally {
Thread.currentThread().setContextClassLoader(currCtxCl);
}
}
@VisibleForTesting
String getTablePath()
{

View File

@ -443,11 +443,17 @@ public class KubernetesTaskRunner implements TaskLogStreamer, TaskRunner
@Override
public TaskLocation getTaskLocation(String taskId)
{
final KubernetesWorkItem workItem = tasks.get(taskId);
if (workItem == null) {
try {
final KubernetesWorkItem workItem = tasks.get(taskId);
if (workItem == null) {
return TaskLocation.unknown();
} else {
return workItem.getLocation();
}
}
catch (Exception e) {
log.warn("Unable to find location for task [%s]", taskId);
return TaskLocation.unknown();
} else {
return workItem.getLocation();
}
}

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
import io.fabric8.kubernetes.api.model.PodTemplate;
import org.apache.druid.indexing.common.task.Task;
import javax.validation.constraints.NotNull;
import java.util.Map;
/**
@ -41,8 +42,7 @@ public interface PodTemplateSelectStrategy
* allows for customized resource allocation and management tailored to the task's specific requirements.
*
* @param task The task for which the Pod template is determined.
* @return The selected Pod template. If no matching template is found,
* the method falls back to a base template.
* @return The pod template that should be used to run the task.
*/
PodTemplate getPodTemplateForTask(Task task, Map<String, PodTemplate> templates);
@NotNull PodTemplate getPodTemplateForTask(Task task, Map<String, PodTemplate> templates);
}

View File

@ -25,7 +25,6 @@ import com.google.common.base.Preconditions;
import io.fabric8.kubernetes.api.model.PodTemplate;
import org.apache.druid.indexing.common.task.Task;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@ -36,19 +35,15 @@ import java.util.Objects;
*/
public class SelectorBasedPodTemplateSelectStrategy implements PodTemplateSelectStrategy
{
@Nullable
private String defaultKey;
private List<Selector> selectors;
private final List<Selector> selectors;
@JsonCreator
public SelectorBasedPodTemplateSelectStrategy(
@JsonProperty("selectors") List<Selector> selectors,
@JsonProperty("defaultKey") @Nullable String defaultKey
@JsonProperty("selectors") List<Selector> selectors
)
{
Preconditions.checkNotNull(selectors, "selectors");
this.selectors = selectors;
this.defaultKey = defaultKey;
}
/**
@ -64,7 +59,7 @@ public class SelectorBasedPodTemplateSelectStrategy implements PodTemplateSelect
.filter(selector -> selector.evaluate(task))
.findFirst()
.map(Selector::getSelectionKey)
.orElse(defaultKey);
.orElse("base");
return templates.getOrDefault(templateKey, templates.get("base"));
}
@ -75,13 +70,6 @@ public class SelectorBasedPodTemplateSelectStrategy implements PodTemplateSelect
return selectors;
}
@Nullable
@JsonProperty
public String getDefaultKey()
{
return defaultKey;
}
@Override
public boolean equals(Object o)
{
@ -92,13 +80,13 @@ public class SelectorBasedPodTemplateSelectStrategy implements PodTemplateSelect
return false;
}
SelectorBasedPodTemplateSelectStrategy that = (SelectorBasedPodTemplateSelectStrategy) o;
return Objects.equals(defaultKey, that.defaultKey) && Objects.equals(selectors, that.selectors);
return Objects.equals(selectors, that.selectors);
}
@Override
public int hashCode()
{
return Objects.hash(defaultKey, selectors);
return Objects.hash(selectors);
}
@Override
@ -106,7 +94,6 @@ public class SelectorBasedPodTemplateSelectStrategy implements PodTemplateSelect
{
return "SelectorBasedPodTemplateSelectStrategy{" +
"selectors=" + selectors +
", defaultKey=" + defaultKey +
'}';
}
}

View File

@ -39,7 +39,6 @@ import org.apache.druid.guice.IndexingServiceModuleHelper;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.k8s.overlord.KubernetesTaskRunnerConfig;
@ -141,10 +140,6 @@ public class PodTemplateTaskAdapter implements TaskAdapter
PodTemplate podTemplate = podTemplateSelectStrategy.getPodTemplateForTask(task, templates);
if (podTemplate == null) {
throw new ISE("Pod template spec not found for task type [%s]", task.getType());
}
return new JobBuilder()
.withNewMetadata()
.withName(new K8sTaskId(task).getK8sJobName())

View File

@ -654,6 +654,24 @@ public class KubernetesTaskRunnerTest extends EasyMockSupport
Assert.assertEquals(TaskLocation.create("host", 0, 1, false), taskLocation);
}
@Test
public void test_getTaskLocation_throws()
{
KubernetesWorkItem workItem = new KubernetesWorkItem(task, null)
{
@Override
public TaskLocation getLocation()
{
throw new RuntimeException();
}
};
runner.tasks.put(task.getId(), workItem);
TaskLocation taskLocation = runner.getTaskLocation(task.getId());
Assert.assertEquals(TaskLocation.unknown(), taskLocation);
}
@Test
public void test_getTaskLocation_noTaskFound()
{

View File

@ -64,8 +64,7 @@ public class KubernetesTaskRunnerDynamicConfigTest
+ " \"selectionKey\": \"medium-throughput\",\n"
+ " \"type\": [\"index_kafka\"]\n"
+ " }\n"
+ " ],\n"
+ " \"defaultKey\": \"base\"\n"
+ " ]\n"
+ " }\n"
+ "}";

View File

@ -88,14 +88,14 @@ public class SelectorBasedPodTemplateSelectStrategyTest
@Test(expected = NullPointerException.class)
public void shouldThrowNullPointerExceptionWhenSelectorsAreNull()
{
new SelectorBasedPodTemplateSelectStrategy(null, null);
new SelectorBasedPodTemplateSelectStrategy(null);
}
@Test
public void testGetPodTemplate_ForTask_emptySelectorsFallbackToBaseTemplate()
{
List<Selector> emptySelectors = Collections.emptyList();
SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(emptySelectors, null);
SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(emptySelectors);
Task task = NoopTask.create();
Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).getMetadata().getName());
}
@ -105,21 +105,11 @@ public class SelectorBasedPodTemplateSelectStrategyTest
{
Selector noMatchSelector = new MockSelector(false, "mock");
List<Selector> selectors = Collections.singletonList(noMatchSelector);
SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors, null);
SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors);
Task task = NoopTask.create();
Assert.assertEquals("base", strategy.getPodTemplateForTask(task, templates).getMetadata().getName());
}
@Test
public void testGetPodTemplate_ForTask_noMatchSelectorsFallbackToDefaultKeyTemplate()
{
Selector noMatchSelector = new MockSelector(false, "mock");
List<Selector> selectors = Collections.singletonList(noMatchSelector);
SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors, "match");
Task task = NoopTask.create();
Assert.assertEquals("match", strategy.getPodTemplateForTask(task, templates).getMetadata().getName());
}
@Test
public void testGetPodTemplate_ForTask_withMatchSelectors()
{
@ -132,7 +122,7 @@ public class SelectorBasedPodTemplateSelectStrategyTest
noMatchSelector,
matchSelector
);
SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors, null);
SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(selectors);
Task task = NoopTask.create();
Assert.assertEquals("match", strategy.getPodTemplateForTask(task, templates).getMetadata().getName());
}
@ -152,7 +142,7 @@ public class SelectorBasedPodTemplateSelectStrategyTest
);
SelectorBasedPodTemplateSelectStrategy strategy = new SelectorBasedPodTemplateSelectStrategy(
Collections.singletonList(selector), "default");
Collections.singletonList(selector));
SelectorBasedPodTemplateSelectStrategy strategy2 = objectMapper.readValue(
objectMapper.writeValueAsBytes(strategy),

View File

@ -620,7 +620,7 @@ public class PodTemplateTaskAdapterTest
dynamicConfigRef = () -> new DefaultKubernetesTaskRunnerDynamicConfig(new SelectorBasedPodTemplateSelectStrategy(
Collections.singletonList(
new Selector("lowThrougput", null, null, Sets.newSet(dataSource)
)), null));
))));
PodTemplateTaskAdapter adapter = new PodTemplateTaskAdapter(
taskRunnerConfig,

View File

@ -64,7 +64,6 @@ import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
@ -145,7 +144,7 @@ public class MaterializedViewSupervisorTest
}
@Test
public void testCheckSegments() throws IOException
public void testCheckSegments()
{
List<DataSegment> baseSegments = createBaseSegments();
Set<DataSegment> derivativeSegments = Sets.newHashSet(createDerivativeSegments());
@ -171,7 +170,7 @@ public class MaterializedViewSupervisorTest
}
@Test
public void testSubmitTasksDoesNotFailIfTaskAlreadyExists() throws IOException
public void testSubmitTasksDoesNotFailIfTaskAlreadyExists()
{
Set<DataSegment> baseSegments = Sets.newHashSet(createBaseSegments());
Set<DataSegment> derivativeSegments = Sets.newHashSet(createDerivativeSegments());
@ -193,7 +192,7 @@ public class MaterializedViewSupervisorTest
}
@Test
public void testSubmitTasksFailsIfTaskCannotBeAdded() throws IOException
public void testSubmitTasksFailsIfTaskCannotBeAdded()
{
Set<DataSegment> baseSegments = Sets.newHashSet(createBaseSegments());
Set<DataSegment> derivativeSegments = Sets.newHashSet(createDerivativeSegments());
@ -219,7 +218,7 @@ public class MaterializedViewSupervisorTest
}
@Test
public void testCheckSegmentsAndSubmitTasks() throws IOException
public void testCheckSegmentsAndSubmitTasks()
{
Set<DataSegment> baseSegments = Collections.singleton(createBaseSegments().get(0));
indexerMetadataStorageCoordinator.commitSegments(baseSegments, null);

View File

@ -178,13 +178,8 @@ public class DatasourceOptimizerTest extends CuratorTestBase
Lists.newArrayList("dim1", "dim2", "dim3", "dim4"),
1024 * 1024
);
try {
metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null);
announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper);
}
catch (IOException e) {
return false;
}
metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null);
announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper);
return true;
}
);
@ -203,13 +198,8 @@ public class DatasourceOptimizerTest extends CuratorTestBase
Lists.newArrayList("dim1", "dim2", "dim3"),
1024
);
try {
metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null);
announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper);
}
catch (IOException e) {
return false;
}
metadataStorageCoordinator.commitSegments(Sets.newHashSet(segment), null);
announceSegmentForServer(druidServer, segment, zkPathsConfig, jsonMapper);
return true;
}
);

View File

@ -27,7 +27,6 @@ import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.util.Optionality;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchAggregatorFactory;
import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchUtils;
@ -79,7 +78,6 @@ public class TDigestGenerateSketchSqlAggregator implements SqlAggregator
}
final AggregatorFactory aggregatorFactory;
final String aggName = StringUtils.format("%s:agg", name);
Integer compression = TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION;
if (aggregateCall.getArgList().size() > 1) {
@ -116,7 +114,7 @@ public class TDigestGenerateSketchSqlAggregator implements SqlAggregator
// No existing match found. Create a new one.
if (input.isDirectColumnAccess()) {
aggregatorFactory = new TDigestSketchAggregatorFactory(
aggName,
name,
input.getDirectColumn(),
compression
);
@ -125,7 +123,7 @@ public class TDigestGenerateSketchSqlAggregator implements SqlAggregator
input,
ColumnType.FLOAT
);
aggregatorFactory = new TDigestSketchAggregatorFactory(aggName, virtualColumnName, compression);
aggregatorFactory = new TDigestSketchAggregatorFactory(name, virtualColumnName, compression);
}
return Aggregation.create(aggregatorFactory);

View File

@ -135,7 +135,7 @@ public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.granularity(Granularities.ALL)
.aggregators(ImmutableList.of(
new TDigestSketchAggregatorFactory("a0:agg", "m1", 200)
new TDigestSketchAggregatorFactory("a0", "m1", 200)
))
.context(QUERY_CONTEXT_DEFAULT)
.build()
@ -205,7 +205,7 @@ public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.granularity(Granularities.ALL)
.aggregators(ImmutableList.of(
new TDigestSketchAggregatorFactory("a0:agg", "m1", 200)
new TDigestSketchAggregatorFactory("a0", "m1", 200)
))
.context(QUERY_CONTEXT_DEFAULT)
.build()
@ -242,7 +242,7 @@ public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
)
)
.aggregators(ImmutableList.of(
new TDigestSketchAggregatorFactory("a0:agg", "v0", 200)
new TDigestSketchAggregatorFactory("a0", "v0", 200)
))
.context(QUERY_CONTEXT_DEFAULT)
.build()
@ -275,7 +275,7 @@ public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
.granularity(Granularities.ALL)
.aggregators(ImmutableList.of(
new TDigestSketchAggregatorFactory("a0:agg", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION)
new TDigestSketchAggregatorFactory("a0", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION)
))
.context(QUERY_CONTEXT_DEFAULT)
.build()
@ -357,7 +357,7 @@ public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
.setDimensions(new DefaultDimensionSpec("dim1", "d0"))
.setAggregatorSpecs(
ImmutableList.of(
new TDigestSketchAggregatorFactory("a0:agg", "m1", 200)
new TDigestSketchAggregatorFactory("a0", "m1", 200)
)
)
.setContext(QUERY_CONTEXT_DEFAULT)
@ -368,7 +368,7 @@ public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
.setGranularity(Granularities.ALL)
.setAggregatorSpecs(
ImmutableList.of(
new TDigestSketchAggregatorFactory("_a0:agg", "a0:agg", 100)
new TDigestSketchAggregatorFactory("_a0:agg", "a0", 100)
)
)
.setPostAggregatorSpecs(
@ -534,7 +534,7 @@ public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
.filters(numericEquality("dim2", 0L, ColumnType.LONG))
.granularity(Granularities.ALL)
.aggregators(ImmutableList.of(
new TDigestSketchAggregatorFactory("a0:agg", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION),
new TDigestSketchAggregatorFactory("a0", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION),
new TDigestSketchAggregatorFactory("a1:agg", "qsketch_m1", 100)
))
.postAggregators(
@ -571,7 +571,7 @@ public class TDigestSketchSqlAggregatorTest extends BaseCalciteQueryTest
.setAggregatorSpecs(
aggregators(
new FilteredAggregatorFactory(
new TDigestSketchAggregatorFactory("a0:agg", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION),
new TDigestSketchAggregatorFactory("a0", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION),
equality("dim1", "nonexistent", ColumnType.STRING)
),
new FilteredAggregatorFactory(

View File

@ -139,11 +139,34 @@ public class SchemaRegistryBasedAvroBytesDecoder implements AvroBytesDecoder
ParsedSchema parsedSchema = registry.getSchemaById(id);
schema = parsedSchema instanceof AvroSchema ? ((AvroSchema) parsedSchema).rawSchema() : null;
}
catch (IOException | RestClientException ex) {
throw new ParseException(null, ex, "Failed to fetch Avro schema id[%s] from registry. Check if the schema "
+ "exists in the registry. Otherwise it could mean that there is "
+ "malformed data in the stream or data that doesn't conform to the schema "
+ "specified.", id);
catch (IOException ex1) {
throw new ParseException(
null,
ex1,
"Failed to fetch Avro schema id[%s] from registry. Check if the schema exists in the registry. Otherwise it"
+ " could mean that there is malformed data in the stream or data that doesn't conform to the schema"
+ " specified.",
id
);
}
catch (RestClientException ex2) {
if (ex2.getErrorCode() == 401) {
throw new ParseException(
null,
ex2,
"Failed to authenticate to schema registry for Avro schema id[%s]. Please check your credentials.",
id
);
}
// For all other errors, just include the code and message received from the library.
throw new ParseException(
null,
ex2,
"Failed to fetch Avro schema id[%s] from registry. Error code[%s] and message[%s].",
id,
ex2.getErrorCode(),
ex2.getMessage()
);
}
if (schema == null) {
throw new ParseException(null, "No Avro schema id[%s] in registry", id);

View File

@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import io.confluent.kafka.schemaregistry.ParsedSchema;
import io.confluent.kafka.schemaregistry.avro.AvroSchema;
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumWriter;
@ -61,15 +62,15 @@ public class SchemaRegistryBasedAvroBytesDecoderTest
@Test
public void testMultipleUrls() throws Exception
{
// Given
String json = "{\"urls\":[\"http://localhost\"],\"type\": \"schema_registry\"}";
ObjectMapper mapper = new DefaultObjectMapper();
mapper.setInjectableValues(
new InjectableValues.Std().addValue(ObjectMapper.class, new DefaultObjectMapper())
);
SchemaRegistryBasedAvroBytesDecoder decoder;
decoder = (SchemaRegistryBasedAvroBytesDecoder) mapper
.readerFor(AvroBytesDecoder.class)
.readValue(json);
// When
SchemaRegistryBasedAvroBytesDecoder decoder = mapper.readerFor(AvroBytesDecoder.class).readValue(json);
// Then
Assert.assertNotEquals(decoder.hashCode(), 0);
@ -78,15 +79,15 @@ public class SchemaRegistryBasedAvroBytesDecoderTest
@Test
public void testUrl() throws Exception
{
// Given
String json = "{\"url\":\"http://localhost\",\"type\": \"schema_registry\"}";
ObjectMapper mapper = new DefaultObjectMapper();
mapper.setInjectableValues(
new InjectableValues.Std().addValue(ObjectMapper.class, new DefaultObjectMapper())
);
SchemaRegistryBasedAvroBytesDecoder decoder;
decoder = (SchemaRegistryBasedAvroBytesDecoder) mapper
.readerFor(AvroBytesDecoder.class)
.readValue(json);
// When
SchemaRegistryBasedAvroBytesDecoder decoder = mapper.readerFor(AvroBytesDecoder.class).readValue(json);
// Then
Assert.assertNotEquals(decoder.hashCode(), 0);
@ -95,15 +96,15 @@ public class SchemaRegistryBasedAvroBytesDecoderTest
@Test
public void testConfig() throws Exception
{
// Given
String json = "{\"url\":\"http://localhost\",\"type\": \"schema_registry\", \"config\":{}}";
ObjectMapper mapper = new DefaultObjectMapper();
mapper.setInjectableValues(
new InjectableValues.Std().addValue(ObjectMapper.class, new DefaultObjectMapper())
);
SchemaRegistryBasedAvroBytesDecoder decoder;
decoder = (SchemaRegistryBasedAvroBytesDecoder) mapper
.readerFor(AvroBytesDecoder.class)
.readValue(json);
// When
SchemaRegistryBasedAvroBytesDecoder decoder = mapper.readerFor(AvroBytesDecoder.class).readValue(json);
// Then
Assert.assertNotEquals(decoder.hashCode(), 0);
@ -120,21 +121,33 @@ public class SchemaRegistryBasedAvroBytesDecoderTest
byte[] bytes = getAvroDatum(schema, someAvroDatum);
ByteBuffer bb = ByteBuffer.allocate(bytes.length + 5).put((byte) 0).putInt(1234).put(bytes);
bb.rewind();
// When
new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb);
GenericRecord parse = new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb);
// Then
Assert.assertEquals(schema, parse.getSchema());
}
@Test(expected = ParseException.class)
@Test
public void testParseCorruptedNotEnoughBytesToEvenGetSchemaInfo()
{
// Given
ByteBuffer bb = ByteBuffer.allocate(2).put((byte) 0).put(1, (byte) 1);
bb.rewind();
// When
new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb);
// When / Then
final ParseException e = Assert.assertThrows(
ParseException.class,
() -> new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb)
);
MatcherAssert.assertThat(
e.getMessage(),
CoreMatchers.containsString("Failed to decode avro message, not enough bytes to decode (2)")
);
}
@Test(expected = ParseException.class)
@Test
public void testParseCorruptedPartial() throws Exception
{
// Given
@ -145,19 +158,30 @@ public class SchemaRegistryBasedAvroBytesDecoderTest
byte[] bytes = getAvroDatum(schema, someAvroDatum);
ByteBuffer bb = ByteBuffer.allocate(4 + 5).put((byte) 0).putInt(1234).put(bytes, 5, 4);
bb.rewind();
// When
new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb);
// When / Then
final ParseException e = Assert.assertThrows(
ParseException.class,
() -> new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb)
);
MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(IOException.class));
MatcherAssert.assertThat(e.getMessage(), CoreMatchers.containsString("Failed to decode Avro message for schema id[1234]"));
}
@Test(expected = ParseException.class)
@Test
public void testParseWrongSchemaType() throws Exception
{
// Given
Mockito.when(registry.getSchemaById(ArgumentMatchers.eq(1234))).thenReturn(Mockito.mock(ParsedSchema.class));
ByteBuffer bb = ByteBuffer.allocate(5).put((byte) 0).putInt(1234);
bb.rewind();
// When
new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb);
// When / Then
final ParseException e = Assert.assertThrows(
ParseException.class,
() -> new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb)
);
MatcherAssert.assertThat(e.getMessage(), CoreMatchers.containsString("No Avro schema id[1234] in registry"));
}
@Test
@ -167,7 +191,8 @@ public class SchemaRegistryBasedAvroBytesDecoderTest
Mockito.when(registry.getSchemaById(ArgumentMatchers.anyInt())).thenThrow(new IOException("no pasaran"));
ByteBuffer bb = ByteBuffer.allocate(5).put((byte) 0).putInt(1234);
bb.rewind();
// When
// When / Then
final ParseException e = Assert.assertThrows(
ParseException.class,
() -> new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb)
@ -187,17 +212,20 @@ public class SchemaRegistryBasedAvroBytesDecoderTest
@Test
public void testParseHeader() throws JsonProcessingException
{
// Given
String json = "{\"url\":\"http://localhost\",\"type\":\"schema_registry\",\"config\":{},\"headers\":{\"druid.dynamic.config.provider\":{\"type\":\"mapString\", \"config\":{\"registry.header.prop.2\":\"value.2\", \"registry.header.prop.3\":\"value.3\"}},\"registry.header.prop.1\":\"value.1\",\"registry.header.prop.2\":\"value.4\"}}";
ObjectMapper mapper = new DefaultObjectMapper();
mapper.setInjectableValues(
new InjectableValues.Std().addValue(ObjectMapper.class, new DefaultObjectMapper())
);
SchemaRegistryBasedAvroBytesDecoder decoder;
decoder = (SchemaRegistryBasedAvroBytesDecoder) mapper
.readerFor(AvroBytesDecoder.class)
.readValue(json);
SchemaRegistryBasedAvroBytesDecoder decoder = mapper.readerFor(AvroBytesDecoder.class).readValue(json);
Map<String, String> header = DynamicConfigProviderUtils.extraConfigAndSetStringMap(decoder.getHeaders(), SchemaRegistryBasedAvroBytesDecoder.DRUID_DYNAMIC_CONFIG_PROVIDER_KEY, new DefaultObjectMapper());
// When
Map<String, String> header = DynamicConfigProviderUtils.extraConfigAndSetStringMap(
decoder.getHeaders(),
SchemaRegistryBasedAvroBytesDecoder.DRUID_DYNAMIC_CONFIG_PROVIDER_KEY,
new DefaultObjectMapper()
);
// Then
Assert.assertEquals(3, header.size());
@ -209,17 +237,20 @@ public class SchemaRegistryBasedAvroBytesDecoderTest
@Test
public void testParseConfig() throws JsonProcessingException
{
// Given
String json = "{\"url\":\"http://localhost\",\"type\":\"schema_registry\",\"config\":{\"druid.dynamic.config.provider\":{\"type\":\"mapString\", \"config\":{\"registry.config.prop.2\":\"value.2\", \"registry.config.prop.3\":\"value.3\"}},\"registry.config.prop.1\":\"value.1\",\"registry.config.prop.2\":\"value.4\"},\"headers\":{}}";
ObjectMapper mapper = new DefaultObjectMapper();
mapper.setInjectableValues(
new InjectableValues.Std().addValue(ObjectMapper.class, new DefaultObjectMapper())
);
SchemaRegistryBasedAvroBytesDecoder decoder;
decoder = (SchemaRegistryBasedAvroBytesDecoder) mapper
.readerFor(AvroBytesDecoder.class)
.readValue(json);
SchemaRegistryBasedAvroBytesDecoder decoder = mapper.readerFor(AvroBytesDecoder.class).readValue(json);
Map<String, ?> config = DynamicConfigProviderUtils.extraConfigAndSetStringMap(decoder.getConfig(), SchemaRegistryBasedAvroBytesDecoder.DRUID_DYNAMIC_CONFIG_PROVIDER_KEY, new DefaultObjectMapper());
// When
Map<String, ?> config = DynamicConfigProviderUtils.extraConfigAndSetStringMap(
decoder.getConfig(),
SchemaRegistryBasedAvroBytesDecoder.DRUID_DYNAMIC_CONFIG_PROVIDER_KEY,
new DefaultObjectMapper()
);
// Then
Assert.assertEquals(3, config.size());
@ -227,4 +258,59 @@ public class SchemaRegistryBasedAvroBytesDecoderTest
Assert.assertEquals("value.2", config.get("registry.config.prop.2"));
Assert.assertEquals("value.3", config.get("registry.config.prop.3"));
}
@Test
public void testParseWhenUnauthenticatedException() throws IOException, RestClientException
{
// Given
Mockito.when(registry.getSchemaById(ArgumentMatchers.eq(1234)))
.thenThrow(new RestClientException("unauthenticated", 401, 401));
GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum();
Schema schema = SomeAvroDatum.getClassSchema();
byte[] bytes = getAvroDatum(schema, someAvroDatum);
ByteBuffer bb = ByteBuffer.allocate(4 + 5).put((byte) 0).putInt(1234).put(bytes, 5, 4);
bb.rewind();
// When / Then
final ParseException e = Assert.assertThrows(
ParseException.class,
() -> new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb)
);
MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RestClientException.class));
MatcherAssert.assertThat(e.getCause().getMessage(), CoreMatchers.containsString("unauthenticated"));
MatcherAssert.assertThat(
e.getMessage(),
CoreMatchers.containsString(
"Failed to authenticate to schema registry for Avro schema id[1234]. Please check your credentials"
)
);
}
@Test
public void testParseWhenResourceNotFoundException() throws IOException, RestClientException
{
// Given
Mockito.when(registry.getSchemaById(ArgumentMatchers.eq(1234)))
.thenThrow(new RestClientException("resource doesn't exist", 404, 404));
GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum();
Schema schema = SomeAvroDatum.getClassSchema();
byte[] bytes = getAvroDatum(schema, someAvroDatum);
ByteBuffer bb = ByteBuffer.allocate(4 + 5).put((byte) 0).putInt(1234).put(bytes, 5, 4);
bb.rewind();
// When / Then
final ParseException e = Assert.assertThrows(
ParseException.class,
() -> new SchemaRegistryBasedAvroBytesDecoder(registry).parse(bb)
);
MatcherAssert.assertThat(e.getCause(), CoreMatchers.instanceOf(RestClientException.class));
MatcherAssert.assertThat(e.getCause().getMessage(), CoreMatchers.containsString("resource doesn't exist"));
MatcherAssert.assertThat(
e.getMessage(),
CoreMatchers.containsString(
"Failed to fetch Avro schema id[1234] from registry."
+ " Error code[404] and message[resource doesn't exist; error code: 404]."
)
);
}
}

View File

@ -27,7 +27,6 @@ import org.apache.calcite.sql.SqlAggFunction;
import org.apache.calcite.sql.SqlFunctionCategory;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.SketchQueryContext;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory;
@ -89,7 +88,6 @@ public class DoublesSketchObjectSqlAggregator implements SqlAggregator
}
final AggregatorFactory aggregatorFactory;
final String histogramName = StringUtils.format("%s:agg", name);
final int k;
if (aggregateCall.getArgList().size() >= 2) {
@ -108,7 +106,7 @@ public class DoublesSketchObjectSqlAggregator implements SqlAggregator
// No existing match found. Create a new one.
if (input.isDirectColumnAccess()) {
aggregatorFactory = new DoublesSketchAggregatorFactory(
histogramName,
name,
input.getDirectColumn(),
k,
DoublesSketchApproxQuantileSqlAggregator.getMaxStreamLengthFromQueryContext(plannerContext.queryContext()),
@ -120,7 +118,7 @@ public class DoublesSketchObjectSqlAggregator implements SqlAggregator
ColumnType.FLOAT
);
aggregatorFactory = new DoublesSketchAggregatorFactory(
histogramName,
name,
virtualColumnName,
k,
DoublesSketchApproxQuantileSqlAggregator.getMaxStreamLengthFromQueryContext(plannerContext.queryContext()),

View File

@ -24,8 +24,11 @@ import com.google.common.collect.ImmutableMap;
import com.google.inject.Injector;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.Druids;
import org.apache.druid.query.JoinDataSource;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryDataSource;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
@ -33,6 +36,7 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.aggregation.FilteredAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.aggregation.cardinality.CardinalityAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.SketchQueryContext;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule;
@ -43,6 +47,7 @@ import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchTo
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToRankPostAggregator;
import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToStringPostAggregator;
import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchSqlAggregatorTest.DoublesSketchComponentSupplier;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator;
import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
@ -53,6 +58,7 @@ import org.apache.druid.segment.IndexBuilder;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.join.JoinType;
import org.apache.druid.segment.join.JoinableFactoryWrapper;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
@ -263,6 +269,143 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
);
}
@Test
public void testSubqueryWithNestedGroupBy()
{
final List<Object[]> expectedResults = ImmutableList.of(
new Object[]{946684800000L, "", 1L, "val1"},
new Object[]{946684800000L, "1", 1L, "val1"},
new Object[]{946684800000L, "10.1", 1L, "val1"},
new Object[]{946684800000L, "2", 1L, "val1"},
new Object[]{946684800000L, "abc", 1L, "val1"},
new Object[]{946684800000L, "def", 1L, "val1"}
);
testQuery(
"SELECT\n"
+ " MILLIS_TO_TIMESTAMP(946684800000) AS __time,\n"
+ " alias.\"user\",\n"
+ " alias.days,\n"
+ " (CASE WHEN alias.days < quantiles.first_quartile THEN 'val2' \n"
+ " WHEN alias.days >= quantiles.first_quartile AND alias.days < quantiles.third_quartile THEN 'val3' \n"
+ " WHEN alias.days >= quantiles.third_quartile THEN 'val1' END) AS val4\n"
+ "FROM (\n"
+ " SELECT\n"
+ " APPROX_QUANTILE_DS(alias.days, 0.25) AS first_quartile,\n"
+ " APPROX_QUANTILE_DS(alias.days, 0.75) AS third_quartile\n"
+ " FROM (\n"
+ " SELECT\n"
+ " dim1 \"user\",\n"
+ " COUNT(DISTINCT __time) AS days\n"
+ " FROM \"foo\"\n"
+ " GROUP BY 1\n"
+ " ) AS alias\n"
+ ") AS quantiles, (\n"
+ " SELECT\n"
+ " dim1 \"user\",\n"
+ " COUNT(DISTINCT __time) AS days\n"
+ " FROM \"foo\"\n"
+ " GROUP BY 1\n"
+ ") AS alias\n",
ImmutableMap.<String, Object>builder()
.putAll(QUERY_CONTEXT_DEFAULT)
.put(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000")
// Disallows the fallback to row based limiting
.put(QueryContexts.MAX_SUBQUERY_ROWS_KEY, "10")
.build(),
ImmutableList.of(
newScanQueryBuilder()
.dataSource(
JoinDataSource.create(
new QueryDataSource(
GroupByQuery.builder()
.setDataSource(
new QueryDataSource(
GroupByQuery.builder()
.setDataSource("foo")
.setQuerySegmentSpec(querySegmentSpec(Intervals.ETERNITY))
.setGranularity(Granularities.ALL)
.addDimension(new DefaultDimensionSpec(
"dim1",
"d0",
ColumnType.STRING
))
.addAggregator(new CardinalityAggregatorFactory(
"a0:a",
null,
Collections.singletonList(new DefaultDimensionSpec(
"__time",
"__time",
ColumnType.LONG
)),
false,
true
))
.setPostAggregatorSpecs(new HyperUniqueFinalizingPostAggregator(
"a0",
"a0:a"
))
.build()
)
)
.setQuerySegmentSpec(querySegmentSpec(Intervals.ETERNITY))
.setGranularity(Granularities.ALL)
.addAggregator(new DoublesSketchAggregatorFactory("_a0:agg", "a0", 128))
.setPostAggregatorSpecs(
new DoublesSketchToQuantilePostAggregator(
"_a0",
new FieldAccessPostAggregator("_a0:agg", "_a0:agg"),
0.25
),
new DoublesSketchToQuantilePostAggregator(
"_a1",
new FieldAccessPostAggregator("_a0:agg", "_a0:agg"),
0.75
)
)
.build()
),
new QueryDataSource(
GroupByQuery.builder()
.setDataSource("foo")
.setQuerySegmentSpec(querySegmentSpec(Intervals.ETERNITY))
.setGranularity(Granularities.ALL)
.addDimension(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))
.addAggregator(new CardinalityAggregatorFactory(
"a0",
null,
Collections.singletonList(new DefaultDimensionSpec(
"__time",
"__time",
ColumnType.LONG
)),
false,
true
))
.build()
),
"j0.",
"1",
JoinType.INNER,
null,
TestExprMacroTable.INSTANCE,
null
)
)
.intervals(querySegmentSpec(Intervals.ETERNITY))
.virtualColumns(
new ExpressionVirtualColumn("v0", "946684800000", ColumnType.LONG, TestExprMacroTable.INSTANCE),
new ExpressionVirtualColumn("v1", "case_searched((\"j0.a0\" < \"_a0\"),'val2',((\"j0.a0\" >= \"_a0\") && (\"j0.a0\" < \"_a1\")),'val3',(\"j0.a0\" >= \"_a1\"),'val1',null)", ColumnType.STRING, TestExprMacroTable.INSTANCE)
)
.columns("j0.a0", "j0.d0", "v0", "v1")
.build()
),
expectedResults
);
}
@Test
public void testQuantileOnCastedString()
{
@ -539,8 +682,8 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
.aggregators(ImmutableList.of(
new LongSumAggregatorFactory("a0", "cnt"),
new DoublesSketchAggregatorFactory("a1:agg", "cnt", 128),
new DoublesSketchAggregatorFactory("a2:agg", "cnt", 128, null, false),
new DoublesSketchAggregatorFactory("a3:agg", "v0", 128, null, false)
new DoublesSketchAggregatorFactory("a2", "cnt", 128, null, false),
new DoublesSketchAggregatorFactory("a3", "v0", 128, null, false)
))
.postAggregators(
new DoublesSketchToQuantilePostAggregator(
@ -557,7 +700,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
"p2",
new FieldAccessPostAggregator(
"p1",
"a2:agg"
"a2"
),
0.5f
),
@ -570,7 +713,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
"p5",
new FieldAccessPostAggregator(
"p4",
"a3:agg"
"a3"
),
0.5f
),
@ -583,7 +726,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
"p8",
new FieldAccessPostAggregator(
"p7",
"a2:agg"
"a2"
),
0.5f
),
@ -592,7 +735,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
"p11",
new FieldAccessPostAggregator(
"p10",
"a2:agg"
"a2"
),
new double[]{0.5d, 0.8d}
),
@ -600,7 +743,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
"p13",
new FieldAccessPostAggregator(
"p12",
"a2:agg"
"a2"
),
new double[]{0.5d, 0.8d}
),
@ -608,7 +751,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
"p15",
new FieldAccessPostAggregator(
"p14",
"a2:agg"
"a2"
),
new double[]{0.2d, 0.6d},
null
@ -617,7 +760,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
"p17",
new FieldAccessPostAggregator(
"p16",
"a2:agg"
"a2"
),
3.0d
),
@ -625,7 +768,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
"p19",
new FieldAccessPostAggregator(
"p18",
"a2:agg"
"a2"
),
new double[]{0.2d, 0.6d}
),
@ -633,7 +776,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
"p21",
new FieldAccessPostAggregator(
"p20",
"a2:agg"
"a2"
)
),
expressionPostAgg(
@ -697,24 +840,24 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
.granularity(Granularities.ALL)
.aggregators(
ImmutableList.of(
new DoublesSketchAggregatorFactory("a0:agg", "m1", 128, null, false)
new DoublesSketchAggregatorFactory("a0", "m1", 128, null, false)
)
)
.postAggregators(
ImmutableList.of(
new DoublesSketchToQuantilePostAggregator(
"p1",
new FieldAccessPostAggregator("p0", "a0:agg"),
new FieldAccessPostAggregator("p0", "a0"),
0.5
),
new DoublesSketchToQuantilePostAggregator(
"s1",
new FieldAccessPostAggregator("s0", "a0:agg"),
new FieldAccessPostAggregator("s0", "a0"),
0.5
),
new DoublesSketchToQuantilePostAggregator(
"s3",
new FieldAccessPostAggregator("s2", "a0:agg"),
new FieldAccessPostAggregator("s2", "a0"),
0.9800000190734863
)
)
@ -750,8 +893,8 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
.aggregators(ImmutableList.of(
new DoublesSketchAggregatorFactory("a0:agg", "m1", null),
new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null),
new DoublesSketchAggregatorFactory("a2:agg", "m1", null, null, false),
new DoublesSketchAggregatorFactory("a3:agg", "qsketch_m1", null, null, false)
new DoublesSketchAggregatorFactory("a2", "m1", null, null, false),
new DoublesSketchAggregatorFactory("a3", "qsketch_m1", null, null, false)
))
.postAggregators(
new DoublesSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.01f),
@ -797,8 +940,8 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
.aggregators(ImmutableList.of(
new DoublesSketchAggregatorFactory("a0:agg", "m1", null),
new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null),
new DoublesSketchAggregatorFactory("a2:agg", "m1", null, null, true),
new DoublesSketchAggregatorFactory("a3:agg", "qsketch_m1", null, null, true)
new DoublesSketchAggregatorFactory("a2", "m1", null, null, true),
new DoublesSketchAggregatorFactory("a3", "qsketch_m1", null, null, true)
))
.postAggregators(
new DoublesSketchToQuantilePostAggregator("a0", makeFieldAccessPostAgg("a0:agg"), 0.01f),
@ -848,11 +991,11 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
equality("dim1", "nonexistent", ColumnType.STRING)
),
new FilteredAggregatorFactory(
new DoublesSketchAggregatorFactory("a2:agg", "m1", null, null, false),
new DoublesSketchAggregatorFactory("a2", "m1", null, null, false),
equality("dim1", "nonexistent", ColumnType.STRING)
),
new FilteredAggregatorFactory(
new DoublesSketchAggregatorFactory("a3:agg", "qsketch_m1", null, null, false),
new DoublesSketchAggregatorFactory("a3", "qsketch_m1", null, null, false),
equality("dim1", "nonexistent", ColumnType.STRING)
)
)
@ -919,11 +1062,11 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
equality("dim1", "nonexistent", ColumnType.STRING)
),
new FilteredAggregatorFactory(
new DoublesSketchAggregatorFactory("a2:agg", "m1", null, null, true),
new DoublesSketchAggregatorFactory("a2", "m1", null, null, true),
equality("dim1", "nonexistent", ColumnType.STRING)
),
new FilteredAggregatorFactory(
new DoublesSketchAggregatorFactory("a3:agg", "qsketch_m1", null, null, true),
new DoublesSketchAggregatorFactory("a3", "qsketch_m1", null, null, true),
equality("dim1", "nonexistent", ColumnType.STRING)
)
)

View File

@ -81,7 +81,6 @@ public class BloomFilterSqlAggregator implements SqlAggregator
}
final AggregatorFactory aggregatorFactory;
final String aggName = StringUtils.format("%s:agg", name);
final RexNode maxNumEntriesOperand = inputAccessor.getField(aggregateCall.getArgList().get(1));
if (!maxNumEntriesOperand.isA(SqlKind.LITERAL)) {
@ -157,7 +156,7 @@ public class BloomFilterSqlAggregator implements SqlAggregator
}
aggregatorFactory = new BloomFilterAggregatorFactory(
aggName,
name,
spec,
maxNumEntries
);

View File

@ -49,7 +49,6 @@ import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;

View File

@ -148,7 +148,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
.aggregators(
ImmutableList.of(
new BloomFilterAggregatorFactory(
"a0:agg",
"a0",
new DefaultDimensionSpec("dim1", "a0:dim1"),
TEST_NUM_ENTRIES
)
@ -207,12 +207,12 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
.aggregators(
ImmutableList.of(
new BloomFilterAggregatorFactory(
"a0:agg",
"a0",
new DefaultDimensionSpec("dim1", "a0:dim1"),
TEST_NUM_ENTRIES
),
new BloomFilterAggregatorFactory(
"a1:agg",
"a1",
new DefaultDimensionSpec("dim2", "a1:dim2"),
TEST_NUM_ENTRIES
)
@ -258,7 +258,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
.aggregators(
ImmutableList.of(
new BloomFilterAggregatorFactory(
"a0:agg",
"a0",
new ExtractionDimensionSpec(
"dim1",
"a0:dim1",
@ -308,7 +308,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
.aggregators(
ImmutableList.of(
new BloomFilterAggregatorFactory(
"a0:agg",
"a0",
new DefaultDimensionSpec("l1", "a0:l1", ColumnType.LONG),
TEST_NUM_ENTRIES
)
@ -361,7 +361,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
.aggregators(
ImmutableList.of(
new BloomFilterAggregatorFactory(
"a0:agg",
"a0",
new DefaultDimensionSpec("v0", "a0:v0"),
TEST_NUM_ENTRIES
)
@ -415,7 +415,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
.aggregators(
ImmutableList.of(
new BloomFilterAggregatorFactory(
"a0:agg",
"a0",
new DefaultDimensionSpec("v0", "a0:v0"),
TEST_NUM_ENTRIES
)
@ -469,7 +469,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
.aggregators(
ImmutableList.of(
new BloomFilterAggregatorFactory(
"a0:agg",
"a0",
new DefaultDimensionSpec("v0", "a0:v0"),
TEST_NUM_ENTRIES
)
@ -508,12 +508,12 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
.aggregators(
ImmutableList.of(
new BloomFilterAggregatorFactory(
"a0:agg",
"a0",
new DefaultDimensionSpec("dim1", "a0:dim1"),
TEST_NUM_ENTRIES
),
new BloomFilterAggregatorFactory(
"a1:agg",
"a1",
new DefaultDimensionSpec("l1", "a1:l1", ColumnType.LONG),
TEST_NUM_ENTRIES
)
@ -559,7 +559,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
aggregators(
new FilteredAggregatorFactory(
new BloomFilterAggregatorFactory(
"a0:agg",
"a0",
new DefaultDimensionSpec("dim1", "a0:dim1"),
TEST_NUM_ENTRIES
),
@ -567,7 +567,7 @@ public class BloomFilterSqlAggregatorTest extends BaseCalciteQueryTest
),
new FilteredAggregatorFactory(
new BloomFilterAggregatorFactory(
"a1:agg",
"a1",
new DefaultDimensionSpec("l1", "a1:l1", ColumnType.LONG),
TEST_NUM_ENTRIES
),

View File

@ -41,7 +41,6 @@ import javax.servlet.ServletRequest;
import javax.servlet.ServletResponse;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import java.io.IOException;
import java.util.Collection;

View File

@ -31,7 +31,6 @@ import org.apache.druid.indexing.seekablestream.SettableByteEntity;
import org.apache.druid.java.util.common.CloseableIterators;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.java.util.common.parsers.ParseException;
import org.joda.time.DateTime;
import javax.annotation.Nullable;
@ -171,21 +170,8 @@ public class KafkaInputReader implements InputEntityReader
{
return valueParser.read().map(
r -> {
final MapBasedInputRow valueRow;
try {
// Return type for the value parser should be of type MapBasedInputRow
// Parsers returning other types are not compatible currently.
valueRow = (MapBasedInputRow) r;
}
catch (ClassCastException e) {
throw new ParseException(
null,
"Unsupported input format in valueFormat. KafkaInputFormat only supports input format that return MapBasedInputRow rows"
);
}
final Map<String, Object> event = buildBlendedEventMap(valueRow.getEvent(), headerKeyList);
final HashSet<String> newDimensions = new HashSet<>(valueRow.getDimensions());
final HashSet<String> newDimensions = new HashSet<>(r.getDimensions());
final Map<String, Object> event = buildBlendedEventMap(r::getRaw, newDimensions, headerKeyList);
newDimensions.addAll(headerKeyList.keySet());
// Remove the dummy timestamp added in KafkaInputFormat
newDimensions.remove(KafkaInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
@ -244,25 +230,18 @@ public class KafkaInputReader implements InputEntityReader
}
List<InputRow> newInputRows = Lists.newArrayListWithCapacity(rowAndValues.getInputRows().size());
List<Map<String, Object>> newRawRows = Lists.newArrayListWithCapacity(rowAndValues.getRawValues().size());
ParseException parseException = null;
for (Map<String, Object> raw : rowAndValues.getRawValuesList()) {
newRawRows.add(buildBlendedEventMap(raw, headerKeyList));
newRawRows.add(buildBlendedEventMap(raw::get, raw.keySet(), headerKeyList));
}
for (InputRow r : rowAndValues.getInputRows()) {
MapBasedInputRow valueRow = null;
try {
valueRow = (MapBasedInputRow) r;
}
catch (ClassCastException e) {
parseException = new ParseException(
null,
"Unsupported input format in valueFormat. KafkaInputFormat only supports input format that return MapBasedInputRow rows"
if (r != null) {
final HashSet<String> newDimensions = new HashSet<>(r.getDimensions());
final Map<String, Object> event = buildBlendedEventMap(
r::getRaw,
newDimensions,
headerKeyList
);
}
if (valueRow != null) {
final Map<String, Object> event = buildBlendedEventMap(valueRow.getEvent(), headerKeyList);
final HashSet<String> newDimensions = new HashSet<>(valueRow.getDimensions());
newDimensions.addAll(headerKeyList.keySet());
// Remove the dummy timestamp added in KafkaInputFormat
newDimensions.remove(KafkaInputFormat.DEFAULT_AUTO_TIMESTAMP_STRING);
@ -279,7 +258,7 @@ public class KafkaInputReader implements InputEntityReader
);
}
}
return InputRowListPlusRawValues.ofList(newRawRows, newInputRows, parseException);
return InputRowListPlusRawValues.ofList(newRawRows, newInputRows, null);
}
);
}
@ -302,22 +281,31 @@ public class KafkaInputReader implements InputEntityReader
/**
* Builds a map that blends two {@link Map}, presenting the combined keyset of both maps, and preferring to read
* from the first map and falling back to the second map if the value is not present.
*
* <p>
* This strategy is used rather than just copying the values of the keyset into a new map so that any 'flattening'
* machinery (such as {@link Map} created by {@link org.apache.druid.java.util.common.parsers.ObjectFlatteners}) is
* still in place to be lazily evaluated instead of eagerly copying.
*/
private static Map<String, Object> buildBlendedEventMap(Map<String, Object> map, Map<String, Object> fallback)
private static Map<String, Object> buildBlendedEventMap(
Function<String, Object> getRowValue,
Set<String> rowDimensions,
Map<String, Object> fallback
)
{
final Set<String> keySet = new HashSet<>(fallback.keySet());
keySet.addAll(map.keySet());
keySet.addAll(rowDimensions);
return new AbstractMap<String, Object>()
{
@Override
public Object get(Object key)
{
return map.getOrDefault((String) key, fallback.get(key));
final String skey = (String) key;
final Object val = getRowValue.apply(skey);
if (val == null) {
return fallback.get(skey);
}
return val;
}
@Override

View File

@ -24,14 +24,17 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.ColumnsFilter;
import org.apache.druid.data.input.InputEntityReader;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputRowSchema;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.data.input.kafka.KafkaRecordEntity;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.seekablestream.SettableByteEntity;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.StringUtils;
@ -57,9 +60,29 @@ import java.util.Optional;
public class KafkaInputFormatTest
{
private KafkaRecordEntity inputEntity;
private final long timestamp = DateTimes.of("2021-06-24").getMillis();
static {
NullHandling.initializeForTests();
}
private static final long TIMESTAMP_MILLIS = DateTimes.of("2021-06-24").getMillis();
private static final String TOPIC = "sample";
private static final byte[] SIMPLE_JSON_KEY_BYTES = StringUtils.toUtf8(
TestUtils.singleQuoteToStandardJson(
"{'key': 'sampleKey'}"
)
);
private static final byte[] SIMPLE_JSON_VALUE_BYTES = StringUtils.toUtf8(
TestUtils.singleQuoteToStandardJson(
"{"
+ " 'timestamp': '2021-06-25',"
+ " 'bar': null,"
+ " 'foo': 'x',"
+ " 'baz': 4,"
+ " 'o': {'mg': 1}"
+ "}"
)
);
private static final Iterable<Header> SAMPLE_HEADERS = ImmutableList.of(
new Header()
{
@ -177,26 +200,9 @@ public class KafkaInputFormatTest
@Test
public void testWithHeaderKeyAndValue() throws IOException
{
final byte[] key = StringUtils.toUtf8(
"{\n"
+ " \"key\": \"sampleKey\"\n"
+ "}"
);
final byte[] payload = StringUtils.toUtf8(
"{\n"
+ " \"timestamp\": \"2021-06-25\",\n"
+ " \"bar\": null,\n"
+ " \"foo\": \"x\",\n"
+ " \"baz\": 4,\n"
+ " \"o\": {\n"
+ " \"mg\": 1\n"
+ " }\n"
+ "}"
);
Headers headers = new RecordHeaders(SAMPLE_HEADERS);
inputEntity = makeInputEntity(key, payload, headers);
KafkaRecordEntity inputEntity =
makeInputEntity(SIMPLE_JSON_KEY_BYTES, SIMPLE_JSON_VALUE_BYTES, headers);
final InputEntityReader reader = format.createReader(
new InputRowSchema(
@ -248,21 +254,7 @@ public class KafkaInputFormatTest
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
// Header verification
Assert.assertEquals("application/json", Iterables.getOnlyElement(row.getDimension("kafka.newheader.encoding")));
Assert.assertEquals("pkc-bar", Iterables.getOnlyElement(row.getDimension("kafka.newheader.kafkapkc")));
Assert.assertEquals(
String.valueOf(DateTimes.of("2021-06-24").getMillis()),
Iterables.getOnlyElement(row.getDimension("kafka.newts.timestamp"))
);
Assert.assertEquals(
TOPIC,
Iterables.getOnlyElement(row.getDimension("kafka.newtopic.topic"))
);
Assert.assertEquals(
"2021-06-25",
Iterables.getOnlyElement(row.getDimension("timestamp"))
);
verifyHeader(row);
// Key verification
Assert.assertEquals("sampleKey", Iterables.getOnlyElement(row.getDimension("kafka.newkey.key")));
@ -282,20 +274,8 @@ public class KafkaInputFormatTest
//Headers cannot be null, so testing only no key use case!
public void testWithOutKey() throws IOException
{
final byte[] payload = StringUtils.toUtf8(
"{\n"
+ " \"timestamp\": \"2021-06-24\",\n"
+ " \"bar\": null,\n"
+ " \"foo\": \"x\",\n"
+ " \"baz\": 4,\n"
+ " \"o\": {\n"
+ " \"mg\": 1\n"
+ " }\n"
+ "}"
);
Headers headers = new RecordHeaders(SAMPLE_HEADERS);
inputEntity = makeInputEntity(null, payload, headers);
KafkaRecordEntity inputEntity = makeInputEntity(null, SIMPLE_JSON_VALUE_BYTES, headers);
final InputEntityReader reader = format.createReader(
new InputRowSchema(
@ -338,7 +318,7 @@ public class KafkaInputFormatTest
@Test
public void testTimestampFromHeader() throws IOException
{
Iterable<Header> sample_header_with_ts = Iterables.unmodifiableIterable(
final Iterable<Header> sampleHeaderWithTs = Iterables.unmodifiableIterable(
Iterables.concat(
SAMPLE_HEADERS,
ImmutableList.of(
@ -359,26 +339,9 @@ public class KafkaInputFormatTest
)
)
);
final byte[] key = StringUtils.toUtf8(
"{\n"
+ " \"key\": \"sampleKey\"\n"
+ "}"
);
final byte[] payload = StringUtils.toUtf8(
"{\n"
+ " \"timestamp\": \"2021-06-24\",\n"
+ " \"bar\": null,\n"
+ " \"foo\": \"x\",\n"
+ " \"baz\": 4,\n"
+ " \"o\": {\n"
+ " \"mg\": 1\n"
+ " }\n"
+ "}"
);
Headers headers = new RecordHeaders(sample_header_with_ts);
inputEntity = makeInputEntity(key, payload, headers);
Headers headers = new RecordHeaders(sampleHeaderWithTs);
KafkaRecordEntity inputEntity =
makeInputEntity(SIMPLE_JSON_KEY_BYTES, SIMPLE_JSON_VALUE_BYTES, headers);
final InputEntityReader reader = format.createReader(
new InputRowSchema(
@ -417,21 +380,7 @@ public class KafkaInputFormatTest
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
// Header verification
Assert.assertEquals("application/json", Iterables.getOnlyElement(row.getDimension("kafka.newheader.encoding")));
Assert.assertEquals("pkc-bar", Iterables.getOnlyElement(row.getDimension("kafka.newheader.kafkapkc")));
Assert.assertEquals(
String.valueOf(DateTimes.of("2021-06-24").getMillis()),
Iterables.getOnlyElement(row.getDimension("kafka.newts.timestamp"))
);
Assert.assertEquals(
"2021-06-24",
Iterables.getOnlyElement(row.getDimension("kafka.newheader.headerTs"))
);
Assert.assertEquals(
"2021-06-24",
Iterables.getOnlyElement(row.getDimension("timestamp"))
);
verifyHeader(row);
// Key verification
Assert.assertEquals("sampleKey", Iterables.getOnlyElement(row.getDimension("kafka.newkey.key")));
@ -450,20 +399,9 @@ public class KafkaInputFormatTest
@Test
public void testWithOutKeyAndHeaderSpecs() throws IOException
{
final byte[] payload = StringUtils.toUtf8(
"{\n"
+ " \"timestamp\": \"2021-06-24\",\n"
+ " \"bar\": null,\n"
+ " \"foo\": \"x\",\n"
+ " \"baz\": 4,\n"
+ " \"o\": {\n"
+ " \"mg\": 1\n"
+ " }\n"
+ "}"
);
Headers headers = new RecordHeaders(SAMPLE_HEADERS);
inputEntity = makeInputEntity(null, payload, headers);
KafkaRecordEntity inputEntity =
makeInputEntity(null, SIMPLE_JSON_VALUE_BYTES, headers);
KafkaInputFormat localFormat = new KafkaInputFormat(
null,
@ -590,7 +528,7 @@ public class KafkaInputFormatTest
for (int i = 0; i < keys.length; i++) {
headers = headers.add(new RecordHeader("indexH", String.valueOf(i).getBytes(StandardCharsets.UTF_8)));
inputEntity = makeInputEntity(keys[i], values[i], headers);
KafkaRecordEntity inputEntity = makeInputEntity(keys[i], values[i], headers);
settableByteEntity.setEntity(inputEntity);
final int numExpectedIterations = 1;
@ -612,7 +550,6 @@ public class KafkaInputFormatTest
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
Assert.assertEquals(String.valueOf(i), Iterables.getOnlyElement(row.getDimension("index")));
// Header verification
Assert.assertEquals(
"application/json",
@ -652,26 +589,9 @@ public class KafkaInputFormatTest
@Test
public void testMissingTimestampThrowsException() throws IOException
{
final byte[] key = StringUtils.toUtf8(
"{\n"
+ " \"key\": \"sampleKey\"\n"
+ "}"
);
final byte[] payload = StringUtils.toUtf8(
"{\n"
+ " \"timestamp\": \"2021-06-25\",\n"
+ " \"bar\": null,\n"
+ " \"foo\": \"x\",\n"
+ " \"baz\": 4,\n"
+ " \"o\": {\n"
+ " \"mg\": 1\n"
+ " }\n"
+ "}"
);
Headers headers = new RecordHeaders(SAMPLE_HEADERS);
inputEntity = makeInputEntity(key, payload, headers);
KafkaRecordEntity inputEntity =
makeInputEntity(SIMPLE_JSON_KEY_BYTES, SIMPLE_JSON_VALUE_BYTES, headers);
final InputEntityReader reader = format.createReader(
new InputRowSchema(
@ -696,11 +616,9 @@ public class KafkaInputFormatTest
try (CloseableIterator<InputRow> iterator = reader.read()) {
while (iterator.hasNext()) {
Throwable t = Assert.assertThrows(ParseException.class, () -> iterator.next());
Assert.assertEquals(
"Timestamp[null] is unparseable! Event: {kafka.newtopic.topic=sample, foo=x, kafka.newts"
+ ".timestamp=1624492800000, kafka.newkey.key=sampleKey...",
t.getMessage()
Throwable t = Assert.assertThrows(ParseException.class, iterator::next);
Assert.assertTrue(
t.getMessage().startsWith("Timestamp[null] is unparseable! Event: {")
);
}
}
@ -709,27 +627,9 @@ public class KafkaInputFormatTest
@Test
public void testWithSchemaDiscovery() throws IOException
{
// testWithHeaderKeyAndValue + schemaless
final byte[] key = StringUtils.toUtf8(
"{\n"
+ " \"key\": \"sampleKey\"\n"
+ "}"
);
final byte[] payload = StringUtils.toUtf8(
"{\n"
+ " \"timestamp\": \"2021-06-25\",\n"
+ " \"bar\": null,\n"
+ " \"foo\": \"x\",\n"
+ " \"baz\": 4,\n"
+ " \"o\": {\n"
+ " \"mg\": 1\n"
+ " }\n"
+ "}"
);
Headers headers = new RecordHeaders(SAMPLE_HEADERS);
inputEntity = makeInputEntity(key, payload, headers);
KafkaRecordEntity inputEntity =
makeInputEntity(SIMPLE_JSON_KEY_BYTES, SIMPLE_JSON_VALUE_BYTES, headers);
final InputEntityReader reader = format.createReader(
new InputRowSchema(
@ -777,21 +677,7 @@ public class KafkaInputFormatTest
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
// Header verification
Assert.assertEquals("application/json", Iterables.getOnlyElement(row.getDimension("kafka.newheader.encoding")));
Assert.assertEquals("pkc-bar", Iterables.getOnlyElement(row.getDimension("kafka.newheader.kafkapkc")));
Assert.assertEquals(
String.valueOf(DateTimes.of("2021-06-24").getMillis()),
Iterables.getOnlyElement(row.getDimension("kafka.newts.timestamp"))
);
Assert.assertEquals(
TOPIC,
Iterables.getOnlyElement(row.getDimension("kafka.newtopic.topic"))
);
Assert.assertEquals(
"2021-06-25",
Iterables.getOnlyElement(row.getDimension("timestamp"))
);
verifyHeader(row);
// Key verification
Assert.assertEquals("sampleKey", Iterables.getOnlyElement(row.getDimension("kafka.newkey.key")));
@ -808,29 +694,102 @@ public class KafkaInputFormatTest
}
@Test
public void testWithPartialDeclarationSchemaDiscovery() throws IOException
public void testValueInCsvFormat() throws IOException
{
// testWithHeaderKeyAndValue + partial-schema + schema discovery
final byte[] key = StringUtils.toUtf8(
"{\n"
+ " \"key\": \"sampleKey\"\n"
+ "}"
);
final byte[] payload = StringUtils.toUtf8(
"{\n"
+ " \"timestamp\": \"2021-06-25\",\n"
+ " \"bar\": null,\n"
+ " \"foo\": \"x\",\n"
+ " \"baz\": 4,\n"
+ " \"o\": {\n"
+ " \"mg\": 1\n"
+ " }\n"
+ "}"
format = new KafkaInputFormat(
new KafkaStringHeaderFormat(null),
// Key Format
new JsonInputFormat(
new JSONPathSpec(true, ImmutableList.of()),
null,
null,
false,
false
),
// Value Format
new CsvInputFormat(
Arrays.asList("foo", "bar", "timestamp", "baz"),
null,
false,
false,
0
),
"kafka.newheader.",
"kafka.newkey.key",
"kafka.newts.timestamp",
"kafka.newtopic.topic"
);
Headers headers = new RecordHeaders(SAMPLE_HEADERS);
inputEntity = makeInputEntity(key, payload, headers);
KafkaRecordEntity inputEntity =
makeInputEntity(SIMPLE_JSON_KEY_BYTES, StringUtils.toUtf8("x,,2021-06-25,4"), headers);
final InputEntityReader reader = format.createReader(
new InputRowSchema(
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(
ImmutableList.of(
"bar",
"foo",
"kafka.newheader.encoding",
"kafka.newheader.kafkapkc",
"kafka.newts.timestamp",
"kafka.newtopic.topic"
)
)
),
ColumnsFilter.all()
),
newSettableByteEntity(inputEntity),
null
);
final int numExpectedIterations = 1;
try (CloseableIterator<InputRow> iterator = reader.read()) {
int numActualIterations = 0;
while (iterator.hasNext()) {
final InputRow row = iterator.next();
Assert.assertEquals(
Arrays.asList(
"bar",
"foo",
"kafka.newheader.encoding",
"kafka.newheader.kafkapkc",
"kafka.newts.timestamp",
"kafka.newtopic.topic"
),
row.getDimensions()
);
// Payload verifications
// this isn't super realistic, since most of these columns are not actually defined in the dimensionSpec
// but test reading them anyway since it isn't technically illegal
Assert.assertEquals(DateTimes.of("2021-06-25"), row.getTimestamp());
Assert.assertEquals("x", Iterables.getOnlyElement(row.getDimension("foo")));
Assert.assertEquals("4", Iterables.getOnlyElement(row.getDimension("baz")));
Assert.assertTrue(row.getDimension("bar").isEmpty());
verifyHeader(row);
// Key verification
Assert.assertEquals("sampleKey", Iterables.getOnlyElement(row.getDimension("kafka.newkey.key")));
numActualIterations++;
}
Assert.assertEquals(numExpectedIterations, numActualIterations);
}
}
@Test
public void testWithPartialDeclarationSchemaDiscovery() throws IOException
{
// testWithHeaderKeyAndValue + partial-schema + schema discovery
Headers headers = new RecordHeaders(SAMPLE_HEADERS);
KafkaRecordEntity inputEntity =
makeInputEntity(SIMPLE_JSON_KEY_BYTES, SIMPLE_JSON_VALUE_BYTES, headers);
final InputEntityReader reader = format.createReader(
new InputRowSchema(
@ -881,21 +840,7 @@ public class KafkaInputFormatTest
Assert.assertEquals("1", Iterables.getOnlyElement(row.getDimension("jq_omg")));
Assert.assertEquals(ImmutableMap.of("mg", 1L), row.getRaw("o"));
// Header verification
Assert.assertEquals("application/json", Iterables.getOnlyElement(row.getDimension("kafka.newheader.encoding")));
Assert.assertEquals("pkc-bar", Iterables.getOnlyElement(row.getDimension("kafka.newheader.kafkapkc")));
Assert.assertEquals(
String.valueOf(DateTimes.of("2021-06-24").getMillis()),
Iterables.getOnlyElement(row.getDimension("kafka.newts.timestamp"))
);
Assert.assertEquals(
TOPIC,
Iterables.getOnlyElement(row.getDimension("kafka.newtopic.topic"))
);
Assert.assertEquals(
"2021-06-25",
Iterables.getOnlyElement(row.getDimension("timestamp"))
);
verifyHeader(row);
// Key verification
Assert.assertEquals("sampleKey", Iterables.getOnlyElement(row.getDimension("kafka.newkey.key")));
@ -918,7 +863,7 @@ public class KafkaInputFormatTest
TOPIC,
0,
0,
timestamp,
TIMESTAMP_MILLIS,
null,
0,
0,
@ -930,6 +875,23 @@ public class KafkaInputFormatTest
);
}
private void verifyHeader(InputRow row)
{
Assert.assertEquals("application/json", Iterables.getOnlyElement(row.getDimension("kafka.newheader.encoding")));
Assert.assertEquals("pkc-bar", Iterables.getOnlyElement(row.getDimension("kafka.newheader.kafkapkc")));
Assert.assertEquals(
String.valueOf(DateTimes.of("2021-06-24").getMillis()),
Iterables.getOnlyElement(row.getDimension("kafka.newts.timestamp"))
);
Assert.assertEquals(
TOPIC,
Iterables.getOnlyElement(row.getDimension("kafka.newtopic.topic"))
);
Assert.assertEquals(
"2021-06-25",
Iterables.getOnlyElement(row.getDimension("timestamp"))
);
}
private SettableByteEntity<KafkaRecordEntity> newSettableByteEntity(KafkaRecordEntity kafkaRecordEntity)
{

View File

@ -367,7 +367,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
// Wait for task to exit
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3));
Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone());
Assert.assertTrue(task.getRunner().getSegmentGenerationMetrics().isProcessingDone());
// Check published metadata and segments in deep storage
assertEqualsExceptVersion(
@ -505,7 +505,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
// Wait for task to exit
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3));
Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone());
Assert.assertTrue(task.getRunner().getSegmentGenerationMetrics().isProcessingDone());
// Check published metadata and segments in deep storage
assertEqualsExceptVersion(
@ -554,7 +554,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
// Wait for task to exit
Assert.assertEquals(TaskState.SUCCESS, future.get().getStatusCode());
verifyTaskMetrics(task, RowMeters.with().bytes(getTotalSizeOfRecords(2, 5)).totalProcessed(3));
Assert.assertTrue(task.getRunner().getFireDepartmentMetrics().isProcessingDone());
Assert.assertTrue(task.getRunner().getSegmentGenerationMetrics().isProcessingDone());
// Check published metadata and segments in deep storage
assertEqualsExceptVersion(
@ -1634,8 +1634,8 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
List<String> expectedMessages = Arrays.asList(
"Unable to parse value[notanumber] for field[met1]",
"could not convert value [notanumber] to float",
"could not convert value [notanumber] to long",
"Could not convert value [notanumber] to float for dimension [dimFloat].",
"Could not convert value [notanumber] to long for dimension [dimLong].",
"Unable to parse [] as the intermediateRow resulted in empty input row (Record: 1)",
"Unable to parse row [unparseable] (Record: 1)",
"Encountered row with timestamp[246140482-04-24T15:36:27.903Z] that cannot be represented as a long: [{timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}] (Record: 1)"

View File

@ -40,7 +40,7 @@ import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.TaskInfoProvider;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.task.RealtimeIndexTask;
import org.apache.druid.indexing.common.task.NoopTask;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.kafka.KafkaConsumerConfigs;
import org.apache.druid.indexing.kafka.KafkaDataSourceMetadata;
@ -89,9 +89,7 @@ import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.ParseExceptionReport;
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.RealtimeIOConfig;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.realtime.FireDepartment;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.Resource;
@ -1194,15 +1192,13 @@ public class KafkaSupervisorTest extends EasyMockSupport
addSomeEvents(1);
// non KafkaIndexTask (don't kill)
Task id2 = new RealtimeIndexTask(
Task id2 = new NoopTask(
"id2",
null,
new FireDepartment(
dataSchema,
new RealtimeIOConfig(null, null),
null
),
null
dataSchema.getDataSource(),
100,
100,
ImmutableMap.of()
);
List<Task> existingTasks = ImmutableList.of(id2);

View File

@ -1201,8 +1201,8 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
List<String> expectedMessages = Arrays.asList(
"Unable to parse value[notanumber] for field[met1]",
"could not convert value [notanumber] to float",
"could not convert value [notanumber] to long",
"Could not convert value [notanumber] to float for dimension [dimFloat].",
"Could not convert value [notanumber] to long for dimension [dimLong].",
"Timestamp[null] is unparseable! Event: {} (Record: 1)",
"Unable to parse [] as the intermediateRow resulted in empty input row (Record: 1)",
"Unable to parse row [unparseable] (Record: 1)",

View File

@ -37,7 +37,7 @@ import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.TaskInfoProvider;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.task.RealtimeIndexTask;
import org.apache.druid.indexing.common.task.NoopTask;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.kinesis.KinesisDataSourceMetadata;
import org.apache.druid.indexing.kinesis.KinesisIndexTask;
@ -84,9 +84,7 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.RealtimeIOConfig;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.realtime.FireDepartment;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.Resource;
@ -940,15 +938,13 @@ public class KinesisSupervisorTest extends EasyMockSupport
EasyMock.expectLastCall().anyTimes();
// non KinesisIndexTask (don't kill)
Task id2 = new RealtimeIndexTask(
Task id2 = new NoopTask(
"id2",
null,
new FireDepartment(
dataSchema,
new RealtimeIOConfig(null, null),
null
),
null
dataSchema.getDataSource(),
100,
100,
ImmutableMap.of()
);
List<Task> existingTasks = ImmutableList.of(id2);

View File

@ -45,7 +45,6 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BooleanSupplier;
@ManageLifecycle
@ -116,7 +115,7 @@ public class K8sDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider
return nodeTypeWatchers.computeIfAbsent(
nodeType,
nType -> {
LOGGER.info("Creating NodeRoleWatcher for nodeRole [%s].", nType);
LOGGER.info("Creating NodeRoleWatcher for role[%s].", nType);
NodeRoleWatcher nodeRoleWatcher = new NodeRoleWatcher(
listenerExecutor,
nType,
@ -128,7 +127,7 @@ public class K8sDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider
if (startAfterCreation) {
nodeRoleWatcher.start();
}
LOGGER.info("Created NodeRoleWatcher for nodeRole [%s].", nType);
LOGGER.info("Created NodeRoleWatcher for role[%s].", nType);
return nodeRoleWatcher;
}
);
@ -188,7 +187,6 @@ public class K8sDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider
private final LifecycleLock lifecycleLock = new LifecycleLock();
private final AtomicReference<Closeable> watchRef = new AtomicReference<>();
private static final Closeable STOP_MARKER = () -> {};
private final NodeRole nodeRole;
@ -221,7 +219,7 @@ public class K8sDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider
boolean cacheInitialized = false;
if (!lifecycleLock.awaitStarted()) {
LOGGER.error("Lifecycle not started, Exited Watch for NodeRole [%s].", nodeRole);
LOGGER.error("Lifecycle not started, Exited Watch for role[%s].", nodeRole);
return;
}
@ -236,23 +234,22 @@ public class K8sDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider
}
keepWatching(
podInfo.getPodNamespace(),
labelSelector,
list.getResourceVersion()
);
}
catch (Throwable ex) {
LOGGER.error(ex, "Expection while watching for NodeRole [%s].", nodeRole);
LOGGER.error(ex, "Exception while watching for role[%s].", nodeRole);
// Wait a little before trying again.
sleep(watcherErrorRetryWaitMS);
}
}
LOGGER.info("Exited Watch for NodeRole [%s].", nodeRole);
LOGGER.info("Exited Watch for role[%s].", nodeRole);
}
private void keepWatching(String namespace, String labelSelector, String resourceVersion)
private void keepWatching(String labelSelector, String resourceVersion)
{
String nextResourceVersion = resourceVersion;
while (lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)) {
@ -285,7 +282,7 @@ public class K8sDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider
} else {
// Try again by starting the watch from the beginning. This can happen if the
// watch goes bad.
LOGGER.debug("Received NULL item while watching node type [%s]. Restarting watch.", this.nodeRole);
LOGGER.debug("Received NULL item while watching role[%s]. Restarting watch.", this.nodeRole);
return;
}
}
@ -301,7 +298,7 @@ public class K8sDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider
sleep(watcherErrorRetryWaitMS);
}
catch (Throwable ex) {
LOGGER.error(ex, "Error while watching node type [%s]", this.nodeRole);
LOGGER.error(ex, "Error while watching role[%s]", this.nodeRole);
sleep(watcherErrorRetryWaitMS);
}
}
@ -324,11 +321,11 @@ public class K8sDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider
}
try {
LOGGER.info("Starting NodeRoleWatcher for [%s]...", nodeRole);
LOGGER.info("Starting NodeRoleWatcher for role[%s]...", nodeRole);
this.watchExecutor = Execs.singleThreaded(this.getClass().getName() + nodeRole.getJsonName());
watchExecutor.submit(this::watch);
lifecycleLock.started();
LOGGER.info("Started NodeRoleWatcher for [%s].", nodeRole);
LOGGER.info("Started NodeRoleWatcher for role[%s].", nodeRole);
}
finally {
lifecycleLock.exitStart();
@ -343,18 +340,18 @@ public class K8sDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider
}
try {
LOGGER.info("Stopping NodeRoleWatcher for [%s]...", nodeRole);
LOGGER.info("Stopping NodeRoleWatcher for role[%s]...", nodeRole);
// STOP_MARKER cannot throw exceptions on close(), so this is OK.
CloseableUtils.closeAndSuppressExceptions(STOP_MARKER, e -> {});
watchExecutor.shutdownNow();
if (!watchExecutor.awaitTermination(15, TimeUnit.SECONDS)) {
LOGGER.warn("Failed to stop watchExecutor for NodeRoleWatcher[%s]", nodeRole);
LOGGER.warn("Failed to stop watchExecutor for role[%s]", nodeRole);
}
LOGGER.info("Stopped NodeRoleWatcher for [%s].", nodeRole);
LOGGER.info("Stopped NodeRoleWatcher for role[%s].", nodeRole);
}
catch (Exception ex) {
LOGGER.error(ex, "Failed to stop NodeRoleWatcher for [%s].", nodeRole);
LOGGER.error(ex, "Failed to stop NodeRoleWatcher for role[%s].", nodeRole);
}
}

View File

@ -55,7 +55,12 @@ public class ChannelCounters implements QueryCounter
public void incrementRowCount()
{
add(NO_PARTITION, 1, 0, 0, 0);
incrementRowCount(NO_PARTITION);
}
public void incrementRowCount(int partition)
{
add(partition, 1, 0, 0, 0);
}
public void incrementBytes(long bytes)

View File

@ -20,15 +20,15 @@
package org.apache.druid.msq.counters;
import org.apache.druid.msq.indexing.processor.SegmentGeneratorFrameProcessor;
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
import org.apache.druid.segment.realtime.SegmentGenerationMetrics;
/**
* Wrapper around {@link FireDepartmentMetrics} which updates the progress counters while updating its metrics. This
* Wrapper around {@link SegmentGenerationMetrics} which updates the progress counters while updating its metrics. This
* is necessary as the {@link org.apache.druid.segment.realtime.appenderator.BatchAppenderator} used by the
* {@link SegmentGeneratorFrameProcessor} is not part of the MSQ extension, and hence,
* cannot update the counters used in MSQ reports as it persists and pushes segments to deep storage.
*/
public class SegmentGeneratorMetricsWrapper extends FireDepartmentMetrics
public class SegmentGeneratorMetricsWrapper extends SegmentGenerationMetrics
{
private final SegmentGenerationProgressCounter segmentGenerationProgressCounter;

View File

@ -101,9 +101,7 @@ import org.apache.druid.msq.indexing.MSQTuningConfig;
import org.apache.druid.msq.indexing.WorkerCount;
import org.apache.druid.msq.indexing.client.ControllerChatHandler;
import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination;
import org.apache.druid.msq.indexing.destination.ExportMSQDestination;
import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination;
import org.apache.druid.msq.indexing.error.CanceledFault;
import org.apache.druid.msq.indexing.error.CannotParseExternalDataFault;
import org.apache.druid.msq.indexing.error.FaultsExceededChecker;
@ -119,6 +117,7 @@ import org.apache.druid.msq.indexing.error.MSQFault;
import org.apache.druid.msq.indexing.error.MSQWarningReportLimiterPublisher;
import org.apache.druid.msq.indexing.error.QueryNotSupportedFault;
import org.apache.druid.msq.indexing.error.TooManyBucketsFault;
import org.apache.druid.msq.indexing.error.TooManySegmentsInTimeChunkFault;
import org.apache.druid.msq.indexing.error.TooManyWarningsFault;
import org.apache.druid.msq.indexing.error.UnknownFault;
import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault;
@ -194,6 +193,7 @@ import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.DruidNode;
import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
import org.apache.druid.sql.calcite.planner.ColumnMappings;
import org.apache.druid.sql.calcite.rel.DruidQuery;
import org.apache.druid.sql.http.ResultFormat;
@ -582,7 +582,8 @@ public class ControllerImpl implements Controller
queryId(),
makeQueryControllerToolKit(),
querySpec,
context.jsonMapper()
context.jsonMapper(),
resultsContext
);
if (log.isDebugEnabled()) {
@ -961,6 +962,14 @@ public class ControllerImpl implements Controller
final Granularity segmentGranularity = destination.getSegmentGranularity();
// Compute & validate partitions by bucket (time chunk) if there is a maximum number of segments to be enforced per time chunk
if (querySpec.getTuningConfig().getMaxNumSegments() != null) {
final Map<DateTime, List<Pair<Integer, ClusterByPartition>>> partitionsByBucket =
getPartitionsByBucket(partitionBoundaries, segmentGranularity, keyReader);
validateNumSegmentsPerBucketOrThrow(partitionsByBucket, segmentGranularity);
}
String previousSegmentId = null;
segmentReport = new MSQSegmentReport(
@ -1028,6 +1037,43 @@ public class ControllerImpl implements Controller
return retVal;
}
/**
* Return partition ranges by bucket (time chunk).
*/
private Map<DateTime, List<Pair<Integer, ClusterByPartition>>> getPartitionsByBucket(
final ClusterByPartitions partitionBoundaries,
final Granularity segmentGranularity,
final RowKeyReader keyReader
)
{
final Map<DateTime, List<Pair<Integer, ClusterByPartition>>> partitionsByBucket = new HashMap<>();
for (int i = 0; i < partitionBoundaries.ranges().size(); i++) {
final ClusterByPartition partitionBoundary = partitionBoundaries.ranges().get(i);
final DateTime bucketDateTime = getBucketDateTime(partitionBoundary, segmentGranularity, keyReader);
partitionsByBucket.computeIfAbsent(bucketDateTime, ignored -> new ArrayList<>())
.add(Pair.of(i, partitionBoundary));
}
return partitionsByBucket;
}
private void validateNumSegmentsPerBucketOrThrow(
final Map<DateTime, List<Pair<Integer, ClusterByPartition>>> partitionsByBucket,
final Granularity segmentGranularity
)
{
final Integer maxNumSegments = querySpec.getTuningConfig().getMaxNumSegments();
if (maxNumSegments == null) {
// Return early because a null value indicates no maximum, i.e., a time chunk can have any number of segments.
return;
}
for (final Map.Entry<DateTime, List<Pair<Integer, ClusterByPartition>>> bucketEntry : partitionsByBucket.entrySet()) {
final int numSegmentsInTimeChunk = bucketEntry.getValue().size();
if (numSegmentsInTimeChunk > maxNumSegments) {
throw new MSQException(new TooManySegmentsInTimeChunkFault(bucketEntry.getKey(), numSegmentsInTimeChunk, maxNumSegments, segmentGranularity));
}
}
}
/**
* Used by {@link #generateSegmentIdsWithShardSpecs}.
*
@ -1071,13 +1117,11 @@ public class ControllerImpl implements Controller
}
// Group partition ranges by bucket (time chunk), so we can generate shardSpecs for each bucket independently.
final Map<DateTime, List<Pair<Integer, ClusterByPartition>>> partitionsByBucket = new HashMap<>();
for (int i = 0; i < partitionBoundaries.ranges().size(); i++) {
ClusterByPartition partitionBoundary = partitionBoundaries.ranges().get(i);
final DateTime bucketDateTime = getBucketDateTime(partitionBoundary, segmentGranularity, keyReader);
partitionsByBucket.computeIfAbsent(bucketDateTime, ignored -> new ArrayList<>())
.add(Pair.of(i, partitionBoundary));
}
final Map<DateTime, List<Pair<Integer, ClusterByPartition>>> partitionsByBucket =
getPartitionsByBucket(partitionBoundaries, segmentGranularity, keyReader);
// Validate the buckets.
validateNumSegmentsPerBucketOrThrow(partitionsByBucket, segmentGranularity);
// Process buckets (time chunks) one at a time.
for (final Map.Entry<DateTime, List<Pair<Integer, ClusterByPartition>>> bucketEntry : partitionsByBucket.entrySet()) {
@ -1089,6 +1133,7 @@ public class ControllerImpl implements Controller
}
final List<Pair<Integer, ClusterByPartition>> ranges = bucketEntry.getValue();
String version = null;
final List<TaskLock> locks = context.taskActionClient().submit(new LockListAction());
@ -1513,7 +1558,7 @@ public class ControllerImpl implements Controller
if (!destination.isReplaceTimeChunks()) {
// Store compaction state only for replace queries.
log.warn(
"storeCompactionState flag set for a non-REPLACE query [%s]. Ignoring the flag for now.",
"Ignoring storeCompactionState flag since it is set for a non-REPLACE query[%s].",
queryDef.getQueryId()
);
} else {
@ -1613,9 +1658,11 @@ public class ControllerImpl implements Controller
GranularitySpec granularitySpec = new UniformGranularitySpec(
segmentGranularity,
dataSchema.getGranularitySpec().getQueryGranularity(),
QueryContext.of(querySpec.getQuery().getContext())
.getGranularity(DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY, jsonMapper),
dataSchema.getGranularitySpec().isRollup(),
dataSchema.getGranularitySpec().inputIntervals()
// Not using dataSchema.getGranularitySpec().inputIntervals() as that always has ETERNITY
((DataSourceMSQDestination) querySpec.getDestination()).getReplaceTimeChunks()
);
DimensionsSpec dimensionsSpec = dataSchema.getDimensionsSpec();
@ -1627,9 +1674,9 @@ public class ControllerImpl implements Controller
List<Object> metricsSpec = dataSchema.getAggregators() == null
? null
: jsonMapper.convertValue(
dataSchema.getAggregators(), new TypeReference<List<Object>>()
{
});
dataSchema.getAggregators(),
new TypeReference<List<Object>>() {}
);
IndexSpec indexSpec = tuningConfig.getIndexSpec();
@ -1673,7 +1720,8 @@ public class ControllerImpl implements Controller
final String queryId,
@SuppressWarnings("rawtypes") final QueryKit toolKit,
final MSQSpec querySpec,
final ObjectMapper jsonMapper
final ObjectMapper jsonMapper,
final ResultsContext resultsContext
)
{
final MSQTuningConfig tuningConfig = querySpec.getTuningConfig();
@ -1781,9 +1829,9 @@ public class ControllerImpl implements Controller
);
return builder.build();
} else if (querySpec.getDestination() instanceof TaskReportMSQDestination) {
} else if (MSQControllerTask.writeFinalResultsToTaskReport(querySpec)) {
return queryDef;
} else if (querySpec.getDestination() instanceof DurableStorageMSQDestination) {
} else if (MSQControllerTask.writeFinalStageResultsToDurableStorage(querySpec)) {
// attaching new query results stage if the final stage does sort during shuffle so that results are ordered.
StageDefinition finalShuffleStageDef = queryDef.getFinalStageDefinition();
@ -1836,7 +1884,8 @@ public class ControllerImpl implements Controller
queryId,
exportStorageProvider,
resultFormat,
columnMappings
columnMappings,
resultsContext
))
);
return builder.build();
@ -2885,12 +2934,12 @@ public class ControllerImpl implements Controller
final InputChannelFactory inputChannelFactory;
if (queryKernelConfig.isDurableStorage() || MSQControllerTask.writeResultsToDurableStorage(querySpec)) {
if (queryKernelConfig.isDurableStorage() || MSQControllerTask.writeFinalStageResultsToDurableStorage(querySpec)) {
inputChannelFactory = DurableStorageInputChannelFactory.createStandardImplementation(
queryId(),
MSQTasks.makeStorageConnector(context.injector()),
closer,
MSQControllerTask.writeResultsToDurableStorage(querySpec)
MSQControllerTask.writeFinalStageResultsToDurableStorage(querySpec)
);
} else {
inputChannelFactory = new WorkerInputChannelFactory(netClient, () -> taskIds);
@ -2911,7 +2960,8 @@ public class ControllerImpl implements Controller
inputChannelFactory,
() -> ArenaMemoryAllocator.createOnHeap(5_000_000),
resultReaderExec,
cancellationId
cancellationId,
MultiStageQueryContext.removeNullBytes(querySpec.getQuery().context())
);
resultsChannel = ReadableConcatFrameChannel.open(

View File

@ -19,6 +19,8 @@
package org.apache.druid.msq.exec;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.sql.calcite.run.SqlResults;
@ -34,21 +36,24 @@ public class ResultsContext
private final List<SqlTypeName> sqlTypeNames;
private final SqlResults.Context sqlResultsContext;
@JsonCreator
public ResultsContext(
final List<SqlTypeName> sqlTypeNames,
final SqlResults.Context sqlResultsContext
@JsonProperty("sqlTypeNames") final List<SqlTypeName> sqlTypeNames,
@JsonProperty("sqlResultsContext") final SqlResults.Context sqlResultsContext
)
{
this.sqlTypeNames = sqlTypeNames;
this.sqlResultsContext = sqlResultsContext;
}
@JsonProperty("sqlTypeNames")
@Nullable
public List<SqlTypeName> getSqlTypeNames()
{
return sqlTypeNames;
}
@JsonProperty("sqlResultsContext")
@Nullable
public SqlResults.Context getSqlResultsContext()
{

View File

@ -36,7 +36,6 @@ import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.SettableFuture;
import it.unimi.dsi.fastutil.bytes.ByteArrays;
import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.frame.FrameType;
import org.apache.druid.frame.allocation.ArenaMemoryAllocator;
import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory;
import org.apache.druid.frame.channel.BlockingQueueFrameChannel;
@ -184,6 +183,8 @@ public class WorkerImpl implements Worker
private final ByteTracker intermediateSuperSorterLocalStorageTracker;
private final boolean durableStageStorageEnabled;
private final WorkerStorageParameters workerStorageParameters;
private final boolean isRemoveNullBytes;
/**
* Only set for select jobs.
*/
@ -229,6 +230,7 @@ public class WorkerImpl implements Worker
QueryContext queryContext = QueryContext.of(task.getContext());
this.durableStageStorageEnabled = MultiStageQueryContext.isDurableStorageEnabled(queryContext);
this.selectDestination = MultiStageQueryContext.getSelectDestinationOrNull(queryContext);
this.isRemoveNullBytes = MultiStageQueryContext.removeNullBytes(queryContext);
this.workerStorageParameters = workerStorageParameters;
long maxBytes = workerStorageParameters.isIntermediateStorageLimitConfigured()
@ -1112,7 +1114,8 @@ public class WorkerImpl implements Worker
inputChannelFactory,
() -> ArenaMemoryAllocator.createOnHeap(frameContext.memoryParameters().getStandardFrameSize()),
exec,
cancellationId
cancellationId,
MultiStageQueryContext.removeNullBytes(QueryContext.of(task.getContext()))
);
inputSliceReader = new MapInputSliceReader(
@ -1206,7 +1209,8 @@ public class WorkerImpl implements Worker
frameContext,
parallelism,
counterTracker,
e -> warningPublisher.publishException(kernel.getStageDefinition().getStageNumber(), e)
e -> warningPublisher.publishException(kernel.getStageDefinition().getStageNumber(), e),
isRemoveNullBytes
);
final ProcessorManager<ProcessorReturnType, ManagerReturnType> processorManager = processors.getProcessorManager();
@ -1543,7 +1547,8 @@ public class WorkerImpl implements Worker
memoryParameters.getSuperSorterMaxChannelsPerProcessor(),
-1,
cancellationId,
counterTracker.sortProgress()
counterTracker.sortProgress(),
isRemoveNullBytes
);
return FutureUtils.transform(
@ -1575,11 +1580,11 @@ public class WorkerImpl implements Worker
outputChannels.stream().map(OutputChannel::getWritableChannel).collect(Collectors.toList()),
kernel.getStageDefinition().getFrameReader(),
kernel.getStageDefinition().getClusterBy().getColumns().size(),
FrameWriters.makeFrameWriterFactory(
FrameType.ROW_BASED,
FrameWriters.makeRowBasedFrameWriterFactory(
new ArenaMemoryAllocatorFactory(frameContext.memoryParameters().getStandardFrameSize()),
kernel.getStageDefinition().getSignature(),
kernel.getStageDefinition().getSortKey()
kernel.getStageDefinition().getSortKey(),
isRemoveNullBytes
)
);
@ -1672,7 +1677,8 @@ public class WorkerImpl implements Worker
// Tracker is not actually tracked, since it doesn't quite fit into the way we report counters.
// There's a single SuperSorterProgressTrackerCounter per worker, but workers that do local
// sorting have a SuperSorter per partition.
new SuperSorterProgressTracker()
new SuperSorterProgressTracker(),
isRemoveNullBytes
);
return FutureUtils.transform(sorter.run(), r -> Iterables.getOnlyElement(r.getAllChannels()));
@ -1766,6 +1772,7 @@ public class WorkerImpl implements Worker
@Override
public void onSuccess(final ClusterByStatisticsCollector result)
{
result.logSketches();
kernelManipulationQueue.add(
holder ->
holder.getStageKernelMap().get(stageDefinition.getId())

View File

@ -20,6 +20,7 @@
package org.apache.druid.msq.guice;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
@ -29,6 +30,7 @@ import org.apache.druid.msq.counters.CounterSnapshotsSerializer;
import org.apache.druid.msq.counters.SegmentGenerationProgressCounter;
import org.apache.druid.msq.counters.SuperSorterProgressTrackerCounter;
import org.apache.druid.msq.counters.WarningCounters;
import org.apache.druid.msq.indexing.MSQCompactionRunner;
import org.apache.druid.msq.indexing.MSQControllerTask;
import org.apache.druid.msq.indexing.MSQWorkerTask;
import org.apache.druid.msq.indexing.error.BroadcastTablesTooLargeFault;
@ -59,6 +61,7 @@ import org.apache.druid.msq.indexing.error.TooManyColumnsFault;
import org.apache.druid.msq.indexing.error.TooManyInputFilesFault;
import org.apache.druid.msq.indexing.error.TooManyPartitionsFault;
import org.apache.druid.msq.indexing.error.TooManyRowsWithSameKeyFault;
import org.apache.druid.msq.indexing.error.TooManySegmentsInTimeChunkFault;
import org.apache.druid.msq.indexing.error.TooManyWarningsFault;
import org.apache.druid.msq.indexing.error.TooManyWorkersFault;
import org.apache.druid.msq.indexing.error.UnknownFault;
@ -126,6 +129,7 @@ public class MSQIndexingModule implements DruidModule
TooManyInputFilesFault.class,
TooManyPartitionsFault.class,
TooManyRowsWithSameKeyFault.class,
TooManySegmentsInTimeChunkFault.class,
TooManyWarningsFault.class,
TooManyWorkersFault.class,
TooManyAttemptsForJob.class,
@ -190,6 +194,8 @@ public class MSQIndexingModule implements DruidModule
NilInputSource.class
);
module.registerSubtypes(new NamedType(MSQCompactionRunner.class, MSQCompactionRunner.TYPE));
FAULT_CLASSES.forEach(module::registerSubtypes);
module.addSerializer(new CounterSnapshotsSerializer());
return Collections.singletonList(module);

View File

@ -263,12 +263,14 @@ public class IndexerControllerContext implements ControllerContext
{
final ImmutableMap.Builder<String, Object> taskContextOverridesBuilder = ImmutableMap.builder();
final long maxParseExceptions = MultiStageQueryContext.getMaxParseExceptions(querySpec.getQuery().context());
final boolean removeNullBytes = MultiStageQueryContext.removeNullBytes(querySpec.getQuery().context());
taskContextOverridesBuilder
.put(MultiStageQueryContext.CTX_DURABLE_SHUFFLE_STORAGE, queryKernelConfig.isDurableStorage())
.put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, maxParseExceptions)
.put(MultiStageQueryContext.CTX_IS_REINDEX, MSQControllerTask.isReplaceInputDataSourceTask(querySpec))
.put(MultiStageQueryContext.CTX_MAX_CONCURRENT_STAGES, queryKernelConfig.getMaxConcurrentStages());
.put(MultiStageQueryContext.CTX_MAX_CONCURRENT_STAGES, queryKernelConfig.getMaxConcurrentStages())
.put(MultiStageQueryContext.CTX_REMOVE_NULL_BYTES, removeNullBytes);
// Put the lookup loading info in the task context to facilitate selective loading of lookups.
if (controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE) != null) {

View File

@ -20,7 +20,6 @@
package org.apache.druid.msq.indexing;
import com.google.common.collect.Iterables;
import org.apache.druid.frame.FrameType;
import org.apache.druid.frame.allocation.MemoryAllocator;
import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory;
import org.apache.druid.frame.channel.BlockingQueueFrameChannel;
@ -58,6 +57,7 @@ public class InputChannelsImpl implements InputChannels
private final FrameProcessorExecutor exec;
private final String cancellationId;
private final Map<StagePartition, ReadablePartition> readablePartitionMap;
private final boolean removeNullBytes;
public InputChannelsImpl(
final QueryDefinition queryDefinition,
@ -65,7 +65,8 @@ public class InputChannelsImpl implements InputChannels
final InputChannelFactory channelFactory,
final Supplier<MemoryAllocator> allocatorMaker,
final FrameProcessorExecutor exec,
final String cancellationId
final String cancellationId,
final boolean removeNullBytes
)
{
this.queryDefinition = queryDefinition;
@ -74,6 +75,7 @@ public class InputChannelsImpl implements InputChannels
this.allocatorMaker = allocatorMaker;
this.exec = exec;
this.cancellationId = cancellationId;
this.removeNullBytes = removeNullBytes;
for (final ReadablePartition readablePartition : readablePartitions) {
readablePartitionMap.put(
@ -128,13 +130,13 @@ public class InputChannelsImpl implements InputChannels
channels,
stageDefinition.getFrameReader(),
queueChannel.writable(),
FrameWriters.makeFrameWriterFactory(
FrameType.ROW_BASED,
FrameWriters.makeRowBasedFrameWriterFactory(
new SingleMemoryAllocatorFactory(allocatorMaker.get()),
stageDefinition.getFrameReader().signature(),
// No sortColumns, because FrameChannelMerger generates frames that are sorted all on its own
Collections.emptyList()
Collections.emptyList(),
removeNullBytes
),
stageDefinition.getSortKey(),
null,

View File

@ -0,0 +1,524 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.msq.indexing;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.inject.Injector;
import org.apache.druid.client.indexing.ClientCompactionRunnerInfo;
import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SecondaryPartitionType;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.task.CompactionRunner;
import org.apache.druid.indexing.common.task.CompactionTask;
import org.apache.druid.indexing.common.task.CurrentSubTaskHolder;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.AllGranularity;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
import org.apache.druid.msq.util.MultiStageQueryContext;
import org.apache.druid.query.Druids;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContext;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.expression.TimestampFloorExprMacro;
import org.apache.druid.query.expression.TimestampParseExprMacro;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
import org.apache.druid.server.coordinator.CompactionConfigValidationResult;
import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
import org.apache.druid.sql.calcite.planner.ColumnMapping;
import org.apache.druid.sql.calcite.planner.ColumnMappings;
import org.joda.time.Interval;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.stream.Collectors;
public class MSQCompactionRunner implements CompactionRunner
{
private static final Logger log = new Logger(MSQCompactionRunner.class);
public static final String TYPE = "msq";
private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL;
private final ObjectMapper jsonMapper;
private final Injector injector;
// Needed as output column name while grouping in the scenario of:
// a) no query granularity -- to specify an output name for the time dimension column since __time is a reserved name.
// b) custom query granularity -- to create a virtual column containing the rounded-off row timestamp.
// In both cases, the new column is converted back to __time later using columnMappings.
public static final String TIME_VIRTUAL_COLUMN = "__vTime";
@JsonIgnore
private final CurrentSubTaskHolder currentSubTaskHolder = new CurrentSubTaskHolder(
(taskObject, config) -> {
final MSQControllerTask msqControllerTask = (MSQControllerTask) taskObject;
msqControllerTask.stopGracefully(config);
});
@JsonCreator
public MSQCompactionRunner(@JacksonInject ObjectMapper jsonMapper, @JacksonInject Injector injector)
{
this.jsonMapper = jsonMapper;
this.injector = injector;
}
/**
* Checks if the provided compaction config is supported by MSQ. The same validation is done at
* {@link ClientCompactionRunnerInfo#compactionConfigSupportedByMSQEngine}
* The following configs aren't supported:
* <ul>
* <li>partitionsSpec of type HashedParititionsSpec.</li>
* <li>maxTotalRows in DynamicPartitionsSpec.</li>
* <li>rollup set to false in granularitySpec when metricsSpec is specified. Null is treated as true.</li>
* <li>queryGranularity set to ALL in granularitySpec.</li>
* <li>Each metric has output column name same as the input name.</li>
* </ul>
*/
@Override
public CompactionConfigValidationResult validateCompactionTask(
CompactionTask compactionTask
)
{
List<CompactionConfigValidationResult> validationResults = new ArrayList<>();
if (compactionTask.getTuningConfig() != null) {
validationResults.add(ClientCompactionRunnerInfo.validatePartitionsSpecForMSQ(
compactionTask.getTuningConfig().getPartitionsSpec())
);
}
if (compactionTask.getGranularitySpec() != null) {
validationResults.add(ClientCompactionRunnerInfo.validateRollupForMSQ(
compactionTask.getMetricsSpec(),
compactionTask.getGranularitySpec().isRollup()
));
}
validationResults.add(ClientCompactionRunnerInfo.validateMaxNumTasksForMSQ(compactionTask.getContext()));
validationResults.add(ClientCompactionRunnerInfo.validateMetricsSpecForMSQ(compactionTask.getMetricsSpec()));
return validationResults.stream()
.filter(result -> !result.isValid())
.findFirst()
.orElse(new CompactionConfigValidationResult(true, null));
}
@Override
public CurrentSubTaskHolder getCurrentSubTaskHolder()
{
return currentSubTaskHolder;
}
@Override
public TaskStatus runCompactionTasks(
CompactionTask compactionTask,
Map<Interval, DataSchema> intervalDataSchemas,
TaskToolbox taskToolbox
) throws Exception
{
List<MSQControllerTask> msqControllerTasks = createMsqControllerTasks(compactionTask, intervalDataSchemas);
if (msqControllerTasks.isEmpty()) {
String msg = StringUtils.format(
"Can't find segments from inputSpec[%s], nothing to do.",
compactionTask.getIoConfig().getInputSpec()
);
return TaskStatus.failure(compactionTask.getId(), msg);
}
return runSubtasks(
msqControllerTasks,
taskToolbox,
currentSubTaskHolder,
compactionTask.getId()
);
}
public List<MSQControllerTask> createMsqControllerTasks(
CompactionTask compactionTask,
Map<Interval, DataSchema> intervalDataSchemas
) throws JsonProcessingException
{
final List<MSQControllerTask> msqControllerTasks = new ArrayList<>();
for (Map.Entry<Interval, DataSchema> intervalDataSchema : intervalDataSchemas.entrySet()) {
Query<?> query;
Interval interval = intervalDataSchema.getKey();
DataSchema dataSchema = intervalDataSchema.getValue();
if (isGroupBy(dataSchema)) {
query = buildGroupByQuery(compactionTask, interval, dataSchema);
} else {
query = buildScanQuery(compactionTask, interval, dataSchema);
}
QueryContext compactionTaskContext = new QueryContext(compactionTask.getContext());
MSQSpec msqSpec = MSQSpec.builder()
.query(query)
.columnMappings(getColumnMappings(dataSchema))
.destination(buildMSQDestination(compactionTask, dataSchema))
.assignmentStrategy(MultiStageQueryContext.getAssignmentStrategy(compactionTaskContext))
.tuningConfig(buildMSQTuningConfig(compactionTask, compactionTaskContext))
.build();
Map<String, Object> msqControllerTaskContext = createMSQTaskContext(compactionTask, dataSchema);
MSQControllerTask controllerTask = new MSQControllerTask(
compactionTask.getId(),
msqSpec.withOverriddenContext(msqControllerTaskContext),
null,
msqControllerTaskContext,
null,
null,
null,
msqControllerTaskContext,
injector
);
msqControllerTasks.add(controllerTask);
}
return msqControllerTasks;
}
private static DataSourceMSQDestination buildMSQDestination(
CompactionTask compactionTask,
DataSchema dataSchema
)
{
final Interval replaceInterval = compactionTask.getIoConfig()
.getInputSpec()
.findInterval(compactionTask.getDataSource());
return new DataSourceMSQDestination(
dataSchema.getDataSource(),
dataSchema.getGranularitySpec().getSegmentGranularity(),
null,
ImmutableList.of(replaceInterval)
);
}
private static MSQTuningConfig buildMSQTuningConfig(CompactionTask compactionTask, QueryContext compactionTaskContext)
{
// Transfer MSQ-related context params, if any, from the compaction context itself.
final int maxNumTasks = MultiStageQueryContext.getMaxNumTasks(compactionTaskContext);
// This parameter is used internally for the number of worker tasks only, so we subtract 1
final int maxNumWorkers = maxNumTasks - 1;
// We don't consider maxRowsInMemory coming via CompactionTuningConfig since it always sets a default value if no
// value specified by user.
final int maxRowsInMemory = MultiStageQueryContext.getRowsInMemory(compactionTaskContext);
final Integer maxNumSegments = MultiStageQueryContext.getMaxNumSegments(compactionTaskContext);
Integer rowsPerSegment = getRowsPerSegment(compactionTask);
return new MSQTuningConfig(
maxNumWorkers,
maxRowsInMemory,
rowsPerSegment,
maxNumSegments,
compactionTask.getTuningConfig() != null ? compactionTask.getTuningConfig().getIndexSpec() : null
);
}
private static Integer getRowsPerSegment(CompactionTask compactionTask)
{
Integer rowsPerSegment = PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT;
if (compactionTask.getTuningConfig() != null) {
PartitionsSpec partitionsSpec = compactionTask.getTuningConfig().getPartitionsSpec();
if (partitionsSpec instanceof DynamicPartitionsSpec) {
rowsPerSegment = partitionsSpec.getMaxRowsPerSegment();
} else if (partitionsSpec instanceof DimensionRangePartitionsSpec) {
DimensionRangePartitionsSpec dimensionRangePartitionsSpec = (DimensionRangePartitionsSpec) partitionsSpec;
rowsPerSegment = dimensionRangePartitionsSpec.getTargetRowsPerSegment() != null
? dimensionRangePartitionsSpec.getTargetRowsPerSegment()
: dimensionRangePartitionsSpec.getMaxRowsPerSegment();
}
}
return rowsPerSegment;
}
private static RowSignature getRowSignature(DataSchema dataSchema)
{
RowSignature.Builder rowSignatureBuilder = RowSignature.builder();
rowSignatureBuilder.add(dataSchema.getTimestampSpec().getTimestampColumn(), ColumnType.LONG);
if (!isQueryGranularityEmptyOrNone(dataSchema)) {
// A virtual column for query granularity would have been added. Add corresponding column type.
rowSignatureBuilder.add(TIME_VIRTUAL_COLUMN, ColumnType.LONG);
}
for (DimensionSchema dimensionSchema : dataSchema.getDimensionsSpec().getDimensions()) {
rowSignatureBuilder.add(dimensionSchema.getName(), ColumnType.fromString(dimensionSchema.getTypeName()));
}
return rowSignatureBuilder.build();
}
private static List<DimensionSpec> getAggregateDimensions(DataSchema dataSchema)
{
List<DimensionSpec> dimensionSpecs = new ArrayList<>();
if (isQueryGranularityEmptyOrNone(dataSchema)) {
// Dimensions in group-by aren't allowed to have time column name as the output name.
dimensionSpecs.add(new DefaultDimensionSpec(ColumnHolder.TIME_COLUMN_NAME, TIME_VIRTUAL_COLUMN, ColumnType.LONG));
} else {
// The changed granularity would result in a new virtual column that needs to be aggregated upon.
dimensionSpecs.add(new DefaultDimensionSpec(TIME_VIRTUAL_COLUMN, TIME_VIRTUAL_COLUMN, ColumnType.LONG));
}
dimensionSpecs.addAll(dataSchema.getDimensionsSpec().getDimensions().stream()
.map(dim -> new DefaultDimensionSpec(
dim.getName(),
dim.getName(),
dim.getColumnType()
))
.collect(Collectors.toList()));
return dimensionSpecs;
}
private static ColumnMappings getColumnMappings(DataSchema dataSchema)
{
List<ColumnMapping> columnMappings = dataSchema.getDimensionsSpec()
.getDimensions()
.stream()
.map(dim -> new ColumnMapping(
dim.getName(), dim.getName()))
.collect(Collectors.toList());
columnMappings.addAll(Arrays.stream(dataSchema.getAggregators())
.map(agg -> new ColumnMapping(agg.getName(), agg.getName()))
.collect(
Collectors.toList()));
if (isGroupBy(dataSchema) || !isQueryGranularityEmptyOrNone(dataSchema)) {
// For scan queries, a virtual column is created from __time if a custom query granularity is provided. For
// group-by queries, as insert needs __time, it will always be one of the dimensions. Since dimensions in groupby
// aren't allowed to have time column as the output name, we map time dimension to TIME_VIRTUAL_COLUMN in
// dimensions, and map it back to the time column here.
columnMappings.add(new ColumnMapping(TIME_VIRTUAL_COLUMN, ColumnHolder.TIME_COLUMN_NAME));
} else {
columnMappings.add(new ColumnMapping(ColumnHolder.TIME_COLUMN_NAME, ColumnHolder.TIME_COLUMN_NAME));
}
return new ColumnMappings(columnMappings);
}
private static List<OrderByColumnSpec> getOrderBySpec(PartitionsSpec partitionSpec)
{
if (partitionSpec.getType() == SecondaryPartitionType.RANGE) {
List<String> dimensions = ((DimensionRangePartitionsSpec) partitionSpec).getPartitionDimensions();
return dimensions.stream()
.map(dim -> new OrderByColumnSpec(dim, OrderByColumnSpec.Direction.ASCENDING))
.collect(Collectors.toList());
}
return Collections.emptyList();
}
private static Query<?> buildScanQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema)
{
RowSignature rowSignature = getRowSignature(dataSchema);
return new Druids.ScanQueryBuilder().dataSource(dataSchema.getDataSource())
.columns(rowSignature.getColumnNames())
.virtualColumns(getVirtualColumns(dataSchema, interval))
.columnTypes(rowSignature.getColumnTypes())
.intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(interval)))
.legacy(false)
.filters(dataSchema.getTransformSpec().getFilter())
.context(compactionTask.getContext())
.build();
}
private static boolean isGroupBy(DataSchema dataSchema)
{
if (dataSchema.getGranularitySpec() != null) {
// If rollup is true without any metrics, all columns are treated as dimensions and
// duplicate rows are removed in line with native compaction.
return dataSchema.getGranularitySpec().isRollup();
}
// If no rollup specified, decide based on whether metrics are present.
return dataSchema.getAggregators().length > 0;
}
private static boolean isQueryGranularityEmptyOrNone(DataSchema dataSchema)
{
return dataSchema.getGranularitySpec() == null
|| dataSchema.getGranularitySpec().getQueryGranularity() == null
|| Objects.equals(
dataSchema.getGranularitySpec().getQueryGranularity(),
Granularities.NONE
);
}
/**
* Creates a virtual timestamp column to create a new __time field according to the provided queryGranularity, as
* queryGranularity field itself is mandated to be ALL in MSQControllerTask.
*/
private static VirtualColumns getVirtualColumns(DataSchema dataSchema, Interval interval)
{
if (isQueryGranularityEmptyOrNone(dataSchema)) {
return VirtualColumns.EMPTY;
}
String virtualColumnExpr;
if (dataSchema.getGranularitySpec()
.getQueryGranularity()
.equals(Granularities.ALL)) {
// For ALL query granularity, all records in a segment are assigned the interval start timestamp of the segment.
// It's the same behaviour in native compaction.
virtualColumnExpr = StringUtils.format("timestamp_parse('%s')", interval.getStart());
} else {
PeriodGranularity periodQueryGranularity = (PeriodGranularity) dataSchema.getGranularitySpec()
.getQueryGranularity();
// Round of the __time column according to the required granularity.
virtualColumnExpr =
StringUtils.format(
"timestamp_floor(\"%s\", '%s')",
ColumnHolder.TIME_COLUMN_NAME,
periodQueryGranularity.getPeriod().toString()
);
}
return VirtualColumns.create(new ExpressionVirtualColumn(
TIME_VIRTUAL_COLUMN,
virtualColumnExpr,
ColumnType.LONG,
new ExprMacroTable(ImmutableList.of(new TimestampFloorExprMacro(), new TimestampParseExprMacro()))
));
}
private static Query<?> buildGroupByQuery(CompactionTask compactionTask, Interval interval, DataSchema dataSchema)
{
DimFilter dimFilter = dataSchema.getTransformSpec().getFilter();
GroupByQuery.Builder builder = new GroupByQuery.Builder()
.setDataSource(new TableDataSource(compactionTask.getDataSource()))
.setVirtualColumns(getVirtualColumns(dataSchema, interval))
.setDimFilter(dimFilter)
.setGranularity(new AllGranularity())
.setDimensions(getAggregateDimensions(dataSchema))
.setAggregatorSpecs(Arrays.asList(dataSchema.getAggregators()))
.setContext(compactionTask.getContext())
.setInterval(interval);
if (compactionTask.getTuningConfig() != null && compactionTask.getTuningConfig().getPartitionsSpec() != null) {
getOrderBySpec(compactionTask.getTuningConfig().getPartitionsSpec()).forEach(builder::addOrderByColumn);
}
return builder.build();
}
private String serializeGranularity(Granularity granularity, ObjectMapper jsonMapper) throws JsonProcessingException
{
if (granularity != null) {
// AllGranularity by default gets deserialized into {"type": "all"} since there is no custom serialize impl -- as
// is there for PeriodGranularity. Not implementing the serializer itself to avoid things breaking elsewhere.
return granularity.equals(Granularities.ALL) ? "ALL" : jsonMapper.writeValueAsString(granularity);
}
return null;
}
private Map<String, Object> createMSQTaskContext(CompactionTask compactionTask, DataSchema dataSchema)
throws JsonProcessingException
{
Map<String, Object> context = new HashMap<>(compactionTask.getContext());
context.put(
DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY,
serializeGranularity(dataSchema.getGranularitySpec() != null
? dataSchema.getGranularitySpec()
.getSegmentGranularity()
: DEFAULT_SEGMENT_GRANULARITY, jsonMapper)
);
if (!isQueryGranularityEmptyOrNone(dataSchema)) {
context.put(
DruidSqlInsert.SQL_INSERT_QUERY_GRANULARITY,
serializeGranularity(dataSchema.getGranularitySpec().getQueryGranularity(), jsonMapper)
);
}
// Similar to compaction using the native engine, don't finalize aggregations.
context.putIfAbsent(MultiStageQueryContext.CTX_FINALIZE_AGGREGATIONS, false);
// Only scalar or array-type dimensions are allowed as grouping keys.
context.putIfAbsent(GroupByQueryConfig.CTX_KEY_ENABLE_MULTI_VALUE_UNNESTING, false);
return context;
}
private static TaskStatus runSubtasks(
List<MSQControllerTask> tasks,
TaskToolbox toolbox,
CurrentSubTaskHolder currentSubTaskHolder,
String compactionTaskId
) throws JsonProcessingException
{
final int totalNumSpecs = tasks.size();
log.info("Generated [%d] MSQControllerTask specs", totalNumSpecs);
int failCnt = 0;
for (MSQControllerTask eachTask : tasks) {
final String json = toolbox.getJsonMapper().writerWithDefaultPrettyPrinter().writeValueAsString(eachTask);
if (!currentSubTaskHolder.setTask(eachTask)) {
String errMsg = "Task was asked to stop. Finish as failed.";
log.info(errMsg);
return TaskStatus.failure(compactionTaskId, errMsg);
}
try {
if (eachTask.isReady(toolbox.getTaskActionClient())) {
log.info("Running MSQControllerTask: " + json);
final TaskStatus eachResult = eachTask.run(toolbox);
if (!eachResult.isSuccess()) {
failCnt++;
log.warn("Failed to run MSQControllerTask: [%s].\nTrying the next MSQControllerTask.", json);
}
} else {
failCnt++;
log.warn("MSQControllerTask is not ready: [%s].\nTrying the next MSQControllerTask.", json);
}
}
catch (Exception e) {
failCnt++;
log.warn(e, "Failed to run MSQControllerTask: [%s].\nTrying the next MSQControllerTask.", json);
}
}
String msg = StringUtils.format(
"Ran [%d] MSQControllerTasks, [%d] succeeded, [%d] failed",
totalNumSpecs,
totalNumSpecs - failCnt,
failCnt
);
log.info(msg);
return failCnt == 0 ? TaskStatus.success(compactionTaskId) : TaskStatus.failure(compactionTaskId, msg);
}
}

View File

@ -52,6 +52,7 @@ import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination;
import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination;
import org.apache.druid.msq.indexing.destination.ExportMSQDestination;
import org.apache.druid.msq.indexing.destination.MSQDestination;
import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination;
import org.apache.druid.msq.util.MultiStageQueryContext;
import org.apache.druid.query.QueryContext;
import org.apache.druid.rpc.ServiceClientFactory;
@ -143,6 +144,22 @@ public class MSQControllerTask extends AbstractTask implements ClientTaskQuery,
addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true);
}
public MSQControllerTask(
@Nullable String id,
MSQSpec querySpec,
@Nullable String sqlQuery,
@Nullable Map<String, Object> sqlQueryContext,
@Nullable SqlResults.Context sqlResultsContext,
@Nullable List<SqlTypeName> sqlTypeNames,
@Nullable List<ColumnType> nativeTypeNames,
@Nullable Map<String, Object> context,
Injector injector
)
{
this(id, querySpec, sqlQuery, sqlQueryContext, sqlResultsContext, sqlTypeNames, nativeTypeNames, context);
this.injector = injector;
}
@Override
public String getType()
{
@ -305,16 +322,38 @@ public class MSQControllerTask extends AbstractTask implements ClientTaskQuery,
return querySpec.getDestination().getDestinationResource();
}
/**
* Checks whether the task is an ingestion into a Druid datasource.
*/
public static boolean isIngestion(final MSQSpec querySpec)
{
return querySpec.getDestination() instanceof DataSourceMSQDestination;
}
/**
* Checks whether the task is an export into external files.
*/
public static boolean isExport(final MSQSpec querySpec)
{
return querySpec.getDestination() instanceof ExportMSQDestination;
}
/**
* Checks whether the task is an async query which writes frame files containing the final results into durable storage.
*/
public static boolean writeFinalStageResultsToDurableStorage(final MSQSpec querySpec)
{
return querySpec.getDestination() instanceof DurableStorageMSQDestination;
}
/**
* Checks whether the task is an async query which writes frame files containing the final results into durable storage.
*/
public static boolean writeFinalResultsToTaskReport(final MSQSpec querySpec)
{
return querySpec.getDestination() instanceof TaskReportMSQDestination;
}
/**
* Returns true if the task reads from the same table as the destination. In this case, we would prefer to fail
* instead of reading any unused segments to ensure that old data is not read.
@ -330,11 +369,6 @@ public class MSQControllerTask extends AbstractTask implements ClientTaskQuery,
}
}
public static boolean writeResultsToDurableStorage(final MSQSpec querySpec)
{
return querySpec.getDestination() instanceof DurableStorageMSQDestination;
}
@Override
public LookupLoadingSpec getLookupLoadingSpec()
{

View File

@ -56,6 +56,9 @@ public class MSQTuningConfig
@Nullable
private final Integer rowsPerSegment;
@Nullable
private final Integer maxNumSegments;
@Nullable
private final IndexSpec indexSpec;
@ -63,18 +66,20 @@ public class MSQTuningConfig
@JsonProperty("maxNumWorkers") @Nullable final Integer maxNumWorkers,
@JsonProperty("maxRowsInMemory") @Nullable final Integer maxRowsInMemory,
@JsonProperty("rowsPerSegment") @Nullable final Integer rowsPerSegment,
@JsonProperty("maxNumSegments") @Nullable final Integer maxNumSegments,
@JsonProperty("indexSpec") @Nullable final IndexSpec indexSpec
)
{
this.maxNumWorkers = maxNumWorkers;
this.maxRowsInMemory = maxRowsInMemory;
this.rowsPerSegment = rowsPerSegment;
this.maxNumSegments = maxNumSegments;
this.indexSpec = indexSpec;
}
public static MSQTuningConfig defaultConfig()
{
return new MSQTuningConfig(null, null, null, null);
return new MSQTuningConfig(null, null, null, null, null);
}
@JsonProperty("maxNumWorkers")
@ -98,6 +103,13 @@ public class MSQTuningConfig
return rowsPerSegment;
}
@JsonProperty("maxNumSegments")
@JsonInclude(JsonInclude.Include.NON_NULL)
Integer getMaxNumSegmentsForSerialization()
{
return maxNumSegments;
}
@JsonProperty("indexSpec")
@JsonInclude(JsonInclude.Include.NON_NULL)
IndexSpec getIndexSpecForSerialization()
@ -120,6 +132,12 @@ public class MSQTuningConfig
return rowsPerSegment != null ? rowsPerSegment : PartitionsSpec.DEFAULT_MAX_ROWS_PER_SEGMENT;
}
@Nullable
public Integer getMaxNumSegments()
{
return maxNumSegments;
}
public IndexSpec getIndexSpec()
{
return indexSpec != null ? indexSpec : IndexSpec.DEFAULT;
@ -138,13 +156,14 @@ public class MSQTuningConfig
return Objects.equals(maxNumWorkers, that.maxNumWorkers)
&& Objects.equals(maxRowsInMemory, that.maxRowsInMemory)
&& Objects.equals(rowsPerSegment, that.rowsPerSegment)
&& Objects.equals(maxNumSegments, that.maxNumSegments)
&& Objects.equals(indexSpec, that.indexSpec);
}
@Override
public int hashCode()
{
return Objects.hash(maxNumWorkers, maxRowsInMemory, rowsPerSegment, indexSpec);
return Objects.hash(maxNumWorkers, maxRowsInMemory, rowsPerSegment, maxNumSegments, indexSpec);
}
@Override
@ -154,6 +173,7 @@ public class MSQTuningConfig
"maxNumWorkers=" + maxNumWorkers +
", maxRowsInMemory=" + maxRowsInMemory +
", rowsPerSegment=" + rowsPerSegment +
", maxNumSegments=" + maxNumSegments +
", indexSpec=" + indexSpec +
'}';
}

View File

@ -64,7 +64,8 @@ public class InvalidNullByteFault extends BaseMSQFault
super(
CODE,
"Invalid null byte at source[%s], rowNumber[%d], column[%s], value[%s], position[%d]. "
+ "Consider sanitizing the input string column using REPLACE(\"%s\", U&'\\0000', '') AS %s",
+ "Consider sanitizing the input string column using \"REPLACE(\"%s\", U&'\\0000', '') AS %s\" or setting 'removeNullBytes' "
+ "to true in the query context.",
source,
rowNumber,
column,

View File

@ -0,0 +1,130 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.msq.indexing.error;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.granularity.GranularityType;
import org.apache.druid.msq.util.MultiStageQueryContext;
import org.joda.time.DateTime;
import java.util.Objects;
@JsonTypeName(TooManySegmentsInTimeChunkFault.CODE)
public class TooManySegmentsInTimeChunkFault extends BaseMSQFault
{
public static final String CODE = "TooManySegmentsInTimeChunk";
private final DateTime timeChunk;
private final int numSegments;
private final int maxNumSegments;
private final Granularity segmentGranularity;
@JsonCreator
public TooManySegmentsInTimeChunkFault(
@JsonProperty("timeChunk") final DateTime timeChunk,
@JsonProperty("numSegments") final int numSegments,
@JsonProperty("maxNumSegments") final int maxNumSegments,
@JsonProperty("segmentGranularity") final Granularity segmentGranularity
)
{
super(
CODE,
"Too many segments requested to be generated in time chunk[%s] with granularity[%s]"
+ " (requested = [%,d], maximum = [%,d]). Please try breaking up your query or change the maximum using"
+ " the query context parameter[%s].",
timeChunk,
convertToGranularityString(segmentGranularity),
numSegments,
maxNumSegments,
MultiStageQueryContext.CTX_MAX_NUM_SEGMENTS
);
this.timeChunk = timeChunk;
this.numSegments = numSegments;
this.maxNumSegments = maxNumSegments;
this.segmentGranularity = segmentGranularity;
}
/**
* Convert the given granularity to a more user-friendly granularity string, when possible.
*/
private static String convertToGranularityString(final Granularity granularity)
{
// If it's a "standard" granularity, we get a nicer string from the GranularityType enum. For any other
// granularity, we just fall back to the toString(). See GranularityType#isStandard().
for (GranularityType value : GranularityType.values()) {
if (value.getDefaultGranularity().equals(granularity)) {
return value.name();
}
}
return granularity.toString();
}
@JsonProperty
public DateTime getTimeChunk()
{
return timeChunk;
}
@JsonProperty
public int getNumSegments()
{
return numSegments;
}
@JsonProperty
public int getMaxNumSegments()
{
return maxNumSegments;
}
@JsonProperty
public Granularity getSegmentGranularity()
{
return segmentGranularity;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
if (!super.equals(o)) {
return false;
}
TooManySegmentsInTimeChunkFault that = (TooManySegmentsInTimeChunkFault) o;
return numSegments == that.numSegments
&& maxNumSegments == that.maxNumSegments
&& Objects.equals(timeChunk, that.timeChunk)
&& Objects.equals(segmentGranularity, that.segmentGranularity);
}
@Override
public int hashCode()
{
return Objects.hash(super.hashCode(), timeChunk, numSegments, maxNumSegments, segmentGranularity);
}
}

View File

@ -124,7 +124,8 @@ public class SegmentGeneratorFrameProcessorFactory
FrameContext frameContext,
int maxOutstandingProcessors,
CounterTracker counters,
Consumer<Throwable> warningPublisher
Consumer<Throwable> warningPublisher,
boolean removeNullBytes
)
{
if (extra == null || extra.isEmpty()) {

View File

@ -74,7 +74,8 @@ public interface FrameProcessorFactory<T, R, ExtraInfoType>
FrameContext frameContext,
int maxOutstandingProcessors,
CounterTracker counters,
Consumer<Throwable> warningPublisher
Consumer<Throwable> warningPublisher,
boolean removeNullBytes
) throws IOException;
@Nullable

View File

@ -100,6 +100,7 @@ public class GlobalSortMaxCountShuffleSpec implements GlobalSortShuffleSpec
} else if (maxPartitions > maxNumPartitions) {
return Either.error((long) maxPartitions);
} else {
collector.logSketches();
final ClusterByPartitions generatedPartitions = collector.generatePartitionsWithMaxCount(maxPartitions);
if (generatedPartitions.size() <= maxNumPartitions) {
return Either.value(generatedPartitions);

View File

@ -99,6 +99,7 @@ public class GlobalSortTargetSizeShuffleSpec implements GlobalSortShuffleSpec
if (expectedPartitions > maxNumPartitions) {
return Either.error(expectedPartitions);
} else {
collector.logSketches();
final ClusterByPartitions generatedPartitions = collector.generatePartitionsWithTargetWeight(targetSize);
if (generatedPartitions.size() <= maxNumPartitions) {
return Either.value(generatedPartitions);

View File

@ -27,7 +27,6 @@ import com.google.common.base.Suppliers;
import it.unimi.dsi.fastutil.ints.IntAVLTreeSet;
import it.unimi.dsi.fastutil.ints.IntSet;
import it.unimi.dsi.fastutil.ints.IntSets;
import org.apache.druid.frame.FrameType;
import org.apache.druid.frame.allocation.MemoryAllocator;
import org.apache.druid.frame.allocation.MemoryAllocatorFactory;
import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory;
@ -355,17 +354,17 @@ public class StageDefinition
*
* Calls {@link MemoryAllocatorFactory#newAllocator()} for each frame.
*/
public FrameWriterFactory createFrameWriterFactory(final MemoryAllocatorFactory memoryAllocatorFactory)
public FrameWriterFactory createFrameWriterFactory(final MemoryAllocatorFactory memoryAllocatorFactory, final boolean removeNullBytes)
{
return FrameWriters.makeFrameWriterFactory(
FrameType.ROW_BASED,
return FrameWriters.makeRowBasedFrameWriterFactory(
memoryAllocatorFactory,
signature,
// Main processor does not sort when there is a hash going on, even if isSort = true. This is because
// FrameChannelHashPartitioner is expected to be attached to the processor and do the sorting. We don't
// want to double-sort.
doesShuffle() && !shuffleSpec.kind().isHash() ? getClusterBy().getColumns() : Collections.emptyList()
doesShuffle() && !shuffleSpec.kind().isHash() ? getClusterBy().getColumns() : Collections.emptyList(),
removeNullBytes
);
}
@ -374,9 +373,9 @@ public class StageDefinition
*
* Re-uses the same {@link MemoryAllocator} for each frame.
*/
public FrameWriterFactory createFrameWriterFactory(final MemoryAllocator allocator)
public FrameWriterFactory createFrameWriterFactory(final MemoryAllocator allocator, final boolean removeNullBytes)
{
return createFrameWriterFactory(new SingleMemoryAllocatorFactory(allocator));
return createFrameWriterFactory(new SingleMemoryAllocatorFactory(allocator), removeNullBytes);
}
public FrameReader getFrameReader()

View File

@ -87,7 +87,8 @@ public abstract class BaseLeafFrameProcessorFactory extends BaseFrameProcessorFa
FrameContext frameContext,
int maxOutstandingProcessors,
CounterTracker counters,
Consumer<Throwable> warningPublisher
Consumer<Throwable> warningPublisher,
final boolean removeNullBytes
) throws IOException
{
// BaseLeafFrameProcessorFactory is used for native Druid queries, where the following input cases can happen:
@ -125,7 +126,7 @@ public abstract class BaseLeafFrameProcessorFactory extends BaseFrameProcessorFa
final OutputChannel outputChannel = outputChannelFactory.openChannel(0 /* Partition number doesn't matter */);
outputChannels.add(outputChannel);
channelQueue.add(outputChannel.getWritableChannel());
frameWriterFactoryQueue.add(stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator()));
frameWriterFactoryQueue.add(stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator(), removeNullBytes));
}

View File

@ -119,7 +119,8 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
FrameContext frameContext,
int maxOutstandingProcessors,
CounterTracker counters,
Consumer<Throwable> warningPublisher
Consumer<Throwable> warningPublisher,
final boolean removeNullBytes
)
{
// Expecting a single input slice from some prior stage.
@ -152,7 +153,7 @@ public class WindowOperatorQueryFrameProcessorFactory extends BaseFrameProcessor
query,
readableInput.getChannel(),
outputChannel.getWritableChannel(),
stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator()),
stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator(), removeNullBytes),
readableInput.getChannelFrameReader(),
frameContext.jsonMapper(),
operatorList,

View File

@ -95,7 +95,8 @@ public class OffsetLimitFrameProcessorFactory extends BaseFrameProcessorFactory
FrameContext frameContext,
int maxOutstandingProcessors,
CounterTracker counters,
Consumer<Throwable> warningPublisher
Consumer<Throwable> warningPublisher,
boolean removeNullBytes
) throws IOException
{
if (workerNumber > 0) {
@ -126,7 +127,7 @@ public class OffsetLimitFrameProcessorFactory extends BaseFrameProcessorFactory
ReadableConcatFrameChannel.open(Iterators.transform(readableInputs.iterator(), ReadableInput::getChannel)),
outputChannel.getWritableChannel(),
readableInputs.frameReader(),
stageDefinition.createFrameWriterFactory(HeapMemoryAllocator.unlimited()),
stageDefinition.createFrameWriterFactory(HeapMemoryAllocator.unlimited(), removeNullBytes),
offset,
// Limit processor will add limit + offset at various points; must avoid overflow
limit == null ? Long.MAX_VALUE - offset : limit

View File

@ -134,7 +134,8 @@ public class SortMergeJoinFrameProcessorFactory extends BaseFrameProcessorFactor
FrameContext frameContext,
int maxOutstandingProcessors,
CounterTracker counters,
Consumer<Throwable> warningPublisher
Consumer<Throwable> warningPublisher,
boolean removeNullBytes
) throws IOException
{
if (inputSlices.size() != 2 || !inputSlices.stream().allMatch(slice -> slice instanceof StageInputSlice)) {
@ -180,7 +181,7 @@ public class SortMergeJoinFrameProcessorFactory extends BaseFrameProcessorFactor
readableInputs.get(LEFT),
readableInputs.get(RIGHT),
outputChannel.getWritableChannel(),
stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator()),
stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator(), removeNullBytes),
rightPrefix,
keyColumns,
requiredNonNullKeyParts,

View File

@ -81,7 +81,8 @@ public class GroupByPostShuffleFrameProcessorFactory extends BaseFrameProcessorF
FrameContext frameContext,
int maxOutstandingProcessors,
CounterTracker counters,
Consumer<Throwable> warningPublisher
Consumer<Throwable> warningPublisher,
boolean removeNullBytes
)
{
// Expecting a single input slice from some prior stage.
@ -116,7 +117,7 @@ public class GroupByPostShuffleFrameProcessorFactory extends BaseFrameProcessorF
engine,
readableInput.getChannel(),
outputChannel.getWritableChannel(),
stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator()),
stageDefinition.createFrameWriterFactory(outputChannel.getFrameMemoryAllocator(), removeNullBytes),
readableInput.getChannelFrameReader(),
frameContext.jsonMapper()
);

View File

@ -185,13 +185,14 @@ public class GroupByQueryKit implements QueryKit<GroupByQuery>
);
if (doLimitOrOffset) {
final ShuffleSpec finalShuffleSpec = resultShuffleSpecFactory.build(resultClusterBy, false);
final DefaultLimitSpec limitSpec = (DefaultLimitSpec) queryToRun.getLimitSpec();
queryDefBuilder.add(
StageDefinition.builder(firstStageNumber + 2)
.inputs(new StageInputSpec(firstStageNumber + 1))
.signature(resultSignature)
.maxWorkerCount(1)
.shuffleSpec(null) // no shuffling should be required after a limit processor.
.shuffleSpec(finalShuffleSpec)
.processorFactory(
new OffsetLimitFrameProcessorFactory(
limitSpec.getOffset(),
@ -224,12 +225,13 @@ public class GroupByQueryKit implements QueryKit<GroupByQuery>
);
if (doLimitOrOffset) {
final DefaultLimitSpec limitSpec = (DefaultLimitSpec) queryToRun.getLimitSpec();
final ShuffleSpec finalShuffleSpec = resultShuffleSpecFactory.build(resultClusterBy, false);
queryDefBuilder.add(
StageDefinition.builder(firstStageNumber + 2)
.inputs(new StageInputSpec(firstStageNumber + 1))
.signature(resultSignature)
.maxWorkerCount(1)
.shuffleSpec(null)
.shuffleSpec(finalShuffleSpec)
.processorFactory(
new OffsetLimitFrameProcessorFactory(
limitSpec.getOffset(),

View File

@ -36,6 +36,7 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.msq.counters.ChannelCounters;
import org.apache.druid.msq.exec.ResultsContext;
import org.apache.druid.msq.util.SequenceUtils;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
@ -44,6 +45,7 @@ import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.planner.ColumnMapping;
import org.apache.druid.sql.calcite.planner.ColumnMappings;
import org.apache.druid.sql.calcite.run.SqlResults;
import org.apache.druid.sql.http.ResultFormat;
import org.apache.druid.storage.StorageConnector;
@ -64,6 +66,8 @@ public class ExportResultsFrameProcessor implements FrameProcessor<Object>
private final String exportFilePath;
private final Object2IntMap<String> outputColumnNameToFrameColumnNumberMap;
private final RowSignature exportRowSignature;
private final ResultsContext resultsContext;
private final int partitionNum;
private volatile ResultFormat.Writer exportWriter;
@ -75,7 +79,9 @@ public class ExportResultsFrameProcessor implements FrameProcessor<Object>
final ObjectMapper jsonMapper,
final ChannelCounters channelCounter,
final String exportFilePath,
final ColumnMappings columnMappings
final ColumnMappings columnMappings,
final ResultsContext resultsContext,
final int partitionNum
)
{
this.inputChannel = inputChannel;
@ -85,6 +91,8 @@ public class ExportResultsFrameProcessor implements FrameProcessor<Object>
this.jsonMapper = jsonMapper;
this.channelCounter = channelCounter;
this.exportFilePath = exportFilePath;
this.resultsContext = resultsContext;
this.partitionNum = partitionNum;
this.outputColumnNameToFrameColumnNumberMap = new Object2IntOpenHashMap<>();
final RowSignature inputRowSignature = frameReader.signature();
@ -130,13 +138,13 @@ public class ExportResultsFrameProcessor implements FrameProcessor<Object>
return ReturnOrAwait.awaitAll(1);
}
if (exportWriter == null) {
createExportWriter();
}
if (inputChannel.isFinished()) {
exportWriter.writeResponseEnd();
return ReturnOrAwait.returnObject(exportFilePath);
} else {
if (exportWriter == null) {
createExportWriter();
}
exportFrame(inputChannel.read());
return ReturnOrAwait.awaitAll(1);
}
@ -167,9 +175,23 @@ public class ExportResultsFrameProcessor implements FrameProcessor<Object>
for (int j = 0; j < exportRowSignature.size(); j++) {
String columnName = exportRowSignature.getColumnName(j);
BaseObjectColumnValueSelector<?> selector = selectors.get(outputColumnNameToFrameColumnNumberMap.getInt(columnName));
exportWriter.writeRowField(columnName, selector.getObject());
if (resultsContext == null) {
throw DruidException.forPersona(DruidException.Persona.OPERATOR)
.ofCategory(DruidException.Category.RUNTIME_FAILURE)
.build("Received null resultsContext from the controller. This is due to a version mismatch between the controller and the worker. Please ensure that the worker and the controller are on the same version before retrying the query.");
}
exportWriter.writeRowField(
columnName,
SqlResults.coerce(
jsonMapper,
resultsContext.getSqlResultsContext(),
selector.getObject(),
resultsContext.getSqlTypeNames().get(j),
columnName
)
);
}
channelCounter.incrementRowCount();
channelCounter.incrementRowCount(partitionNum);
exportWriter.writeRowEnd();
cursor.advance();
}

View File

@ -36,6 +36,7 @@ import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.msq.counters.ChannelCounters;
import org.apache.druid.msq.counters.CounterNames;
import org.apache.druid.msq.counters.CounterTracker;
import org.apache.druid.msq.exec.ResultsContext;
import org.apache.druid.msq.input.InputSlice;
import org.apache.druid.msq.input.InputSliceReader;
import org.apache.druid.msq.input.ReadableInput;
@ -63,19 +64,22 @@ public class ExportResultsFrameProcessorFactory implements FrameProcessorFactory
private final ExportStorageProvider exportStorageProvider;
private final ResultFormat exportFormat;
private final ColumnMappings columnMappings;
private final ResultsContext resultsContext;
@JsonCreator
public ExportResultsFrameProcessorFactory(
@JsonProperty("queryId") String queryId,
@JsonProperty("exportStorageProvider") ExportStorageProvider exportStorageProvider,
@JsonProperty("exportFormat") ResultFormat exportFormat,
@JsonProperty("columnMappings") @Nullable ColumnMappings columnMappings
@JsonProperty("columnMappings") @Nullable ColumnMappings columnMappings,
@JsonProperty("resultsContext") @Nullable ResultsContext resultsContext
)
{
this.queryId = queryId;
this.exportStorageProvider = exportStorageProvider;
this.exportFormat = exportFormat;
this.columnMappings = columnMappings;
this.resultsContext = resultsContext;
}
@JsonProperty("queryId")
@ -105,6 +109,14 @@ public class ExportResultsFrameProcessorFactory implements FrameProcessorFactory
return columnMappings;
}
@JsonProperty("resultsContext")
@JsonInclude(JsonInclude.Include.NON_NULL)
@Nullable
public ResultsContext getResultsContext()
{
return resultsContext;
}
@Override
public ProcessorsAndChannels<Object, Object> makeProcessors(
StageDefinition stageDefinition,
@ -116,7 +128,8 @@ public class ExportResultsFrameProcessorFactory implements FrameProcessorFactory
FrameContext frameContext,
int maxOutstandingProcessors,
CounterTracker counters,
Consumer<Throwable> warningPublisher
Consumer<Throwable> warningPublisher,
boolean removeNullBytes
)
{
final StageInputSlice slice = (StageInputSlice) CollectionUtils.getOnlyElement(
@ -132,7 +145,7 @@ public class ExportResultsFrameProcessorFactory implements FrameProcessorFactory
);
}
ChannelCounters channelCounter = counters.channel(CounterNames.outputChannel());
final ChannelCounters channelCounter = counters.channel(CounterNames.outputChannel());
final Sequence<ReadableInput> readableInputs =
Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher));
@ -145,7 +158,9 @@ public class ExportResultsFrameProcessorFactory implements FrameProcessorFactory
frameContext.jsonMapper(),
channelCounter,
getExportFilePath(queryId, workerNumber, readableInput.getStagePartition().getPartitionNumber(), exportFormat),
columnMappings
columnMappings,
resultsContext,
readableInput.getStagePartition().getPartitionNumber()
)
);

View File

@ -68,7 +68,8 @@ public class QueryResultFrameProcessorFactory extends BaseFrameProcessorFactory
FrameContext frameContext,
int maxOutstandingProcessors,
CounterTracker counters,
Consumer<Throwable> warningPublisher
Consumer<Throwable> warningPublisher,
boolean removeNullBytes
)
{
// Expecting a single input slice from some prior stage.

Some files were not shown because too many files have changed in this diff Show More