diff --git a/.github/dependabot.yml b/.github/dependabot.yml
index 48a0e0540db0..78005a722336 100644
--- a/.github/dependabot.yml
+++ b/.github/dependabot.yml
@@ -4,6 +4,7 @@ updates:
directory: "/"
schedule:
interval: "daily"
+ open-pull-requests-limit: 20
ignore:
- dependency-name: "com.google.guava:guava"
# pin ZooKeeper dependencies to 3.5.x
@@ -18,3 +19,6 @@ updates:
# Even then this will involve significant effort.
# See https://github.com/apache/druid/pull/12258
- dependency-name: "org.apache.calcite"
+ # jclouds 2.1 needs Guava 18+
+ - dependency-name: "org.apache.jclouds"
+ versions: "[2.1,)"
diff --git a/.github/labeler.yml b/.github/labeler.yml
index b84f335b12b2..a9bfc45a86ec 100644
--- a/.github/labeler.yml
+++ b/.github/labeler.yml
@@ -18,9 +18,50 @@
#
#
-# Pull Request Labeler Github Action Configuration: https://github.com/marketplace/actions/labeler
+# Pull Request Labeler GitHub Action Configuration: https://github.com/marketplace/actions/labeler
-"Area - Documentation":
- - "docs/**/*"
- - "website/**"
- - "examples/quickstart/jupyter-notebooks/*"
\ No newline at end of file
+'Area - Batch Ingestion':
+ - 'indexing-hadoop/**'
+ - 'extensions-core/multi-stage-query/**'
+
+'Area - Dependencies':
+ - '**/pom.xml'
+ - 'licenses.yaml'
+
+'Area - Documentation':
+ - 'docs/**/*'
+ - 'website/**'
+ - 'examples/quickstart/jupyter-notebooks/**'
+
+'Area - Ingestion':
+ - 'indexing-service/**'
+
+'Area - Lookups':
+ - 'extensions-core/lookups-cached-global/**'
+ - 'extensions-core/lookups-cached-single/**'
+ - 'extensions-core/kafka-extraction-namespace/**'
+
+'Area - Metrics/Event Emitting':
+ - 'processing/src/main/java/org/apache/druid/java/util/metrics/**'
+ - 'processing/src/main/java/org/apache/druid/java/util/emitter/**'
+ - 'extensions-contrib/*-emitter/**'
+
+'Area - MSQ':
+ - 'extensions-core/multi-stage-query/**'
+
+'Area - Querying':
+ - 'sql/**'
+ - 'extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/**'
+
+'Area - Segment Format and Ser/De':
+ - 'processing/src/main/java/org/apache/druid/segment/**'
+
+'Area - Streaming Ingestion':
+ - 'extensions-core/kafka-indexing-service/**'
+ - 'extensions-core/kinesis-indexing-service/**'
+
+'Area - Web Console':
+ - 'web-console/**'
+
+'Kubernetes':
+ - 'extensions-contrib/kubernetes-overlord-extensions/**'
diff --git a/.github/scripts/analyze_dependencies_script.sh b/.github/scripts/analyze_dependencies_script.sh
index c92d90030f71..1212dbd1cd2b 100755
--- a/.github/scripts/analyze_dependencies_script.sh
+++ b/.github/scripts/analyze_dependencies_script.sh
@@ -15,7 +15,7 @@
#!bin/bash
-${MVN} ${MAVEN_SKIP} dependency:analyze -DoutputXML=true -DignoreNonCompile=true -DfailOnWarning=true ${HADOOP_PROFILE} ||
+${MVN} ${MAVEN_SKIP} dependency:analyze -DoutputXML=true -DignoreNonCompile=true -DfailOnWarning=true ||
{ echo "
The dependency analysis has found a dependency that is either:
diff --git a/.github/scripts/license_checks_script.sh b/.github/scripts/license_checks_script.sh
index 410ac60375fa..163214f82988 100755
--- a/.github/scripts/license_checks_script.sh
+++ b/.github/scripts/license_checks_script.sh
@@ -20,7 +20,7 @@ set -e
./.github/scripts/setup_generate_license.sh
${MVN} apache-rat:check -Prat --fail-at-end \
-Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn \
--Drat.consoleOutput=true ${HADOOP_PROFILE}
+-Drat.consoleOutput=true
# Generate dependency reports and checks they are valid.
mkdir -p target
distribution/bin/generate-license-dependency-reports.py . target --clean-maven-artifact-transfer --parallel 2
diff --git a/.github/scripts/unit_tests_script.sh b/.github/scripts/unit_tests_script.sh
index 1f5407b95844..569811a93940 100755
--- a/.github/scripts/unit_tests_script.sh
+++ b/.github/scripts/unit_tests_script.sh
@@ -20,7 +20,7 @@ set -e
unset _JAVA_OPTIONS
# Set MAVEN_OPTS for Surefire launcher.
-MAVEN_OPTS='-Xmx2500m' ${MVN} test -pl ${MAVEN_PROJECTS} \
+MAVEN_OPTS='-Xmx2500m' ${MVN} test -B -pl ${MAVEN_PROJECTS} -Dmaven.test.failure.ignore=true \
${MAVEN_SKIP} -Ddruid.generic.useDefaultValueForNull=${DRUID_USE_DEFAULT_VALUE_FOR_NULL} \
-DjfrProfilerArgLine="${JFR_PROFILER_ARG_LINE}"
sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0"
diff --git a/.github/workflows/codeql.yml b/.github/workflows/codeql.yml
index 4e13d31de0a8..262d7ad80f85 100644
--- a/.github/workflows/codeql.yml
+++ b/.github/workflows/codeql.yml
@@ -29,6 +29,12 @@ jobs:
- name: Checkout repository
uses: actions/checkout@v3
+ - uses: actions/setup-java@v3
+ with:
+ distribution: 'zulu'
+ java-version: '8'
+ cache: 'maven'
+
# Initializes the CodeQL tools for scanning.
- name: Initialize CodeQL
uses: github/codeql-action/init@v2
@@ -42,7 +48,6 @@ jobs:
# Details on CodeQL's query packs refer to : https://docs.github.com/en/code-security/code-scanning/automatically-scanning-your-code-for-vulnerabilities-and-errors/configuring-code-scanning#using-queries-in-ql-packs
queries: +security-and-quality
-
- run: |
echo "Building using custom commands"
mvn clean package -f "pom.xml" -B -V -e -Dfindbugs.skip -Dcheckstyle.skip -Dpmd.skip=true -Denforcer.skip -Dmaven.javadoc.skip -DskipTests -Dmaven.test.skip.exec -Dlicense.skip=true -Dweb.console.skip=true -Dcyclonedx.skip=true
diff --git a/.github/workflows/cron-job-its.yml b/.github/workflows/cron-job-its.yml
index 3752b6c60eae..65471ad81b05 100644
--- a/.github/workflows/cron-job-its.yml
+++ b/.github/workflows/cron-job-its.yml
@@ -111,8 +111,6 @@ jobs:
name: security vulnerabilities
strategy:
fail-fast: false
- matrix:
- HADOOP_PROFILE: [ '', '-Phadoop2' ]
runs-on: ubuntu-latest
steps:
- name: Checkout branch
@@ -129,10 +127,8 @@ jobs:
run: mvn clean install dependency:go-offline -P dist -P skip-static-checks,skip-tests -Dmaven.javadoc.skip=true -Dcyclonedx.skip=true -Dweb.console.skip=true
- name: security vulnerabilities check
- env:
- HADOOP_PROFILE: ${{ matrix.HADOOP_PROFILE }}
run: |
- mvn dependency-check:purge dependency-check:check ${HADOOP_PROFILE} || { echo "
+ mvn dependency-check:purge dependency-check:check || { echo "
The OWASP dependency check has found security vulnerabilities. Please use a newer version
of the dependency that does not have vulnerabilities. To see a report run
`mvn dependency-check:check`
diff --git a/.github/workflows/reusable-revised-its.yml b/.github/workflows/reusable-revised-its.yml
index 49887db14c59..e59284b0c3b0 100644
--- a/.github/workflows/reusable-revised-its.yml
+++ b/.github/workflows/reusable-revised-its.yml
@@ -133,3 +133,35 @@ jobs:
- name: Run IT
run: ${{ inputs.script }}
+
+ - name: Collect docker logs on failure
+ if: ${{ failure() && steps.run-it.conclusion == 'failure' }}
+ run: |
+ mkdir docker-logs
+ for c in $(docker ps -a --format="{{.Names}}")
+ do
+ docker logs $c > ./docker-logs/$c.log
+ done
+
+ - name: Tar docker logs
+ if: ${{ failure() && steps.run-it.conclusion == 'failure' }}
+ run: tar cvzf ./docker-logs.tgz ./docker-logs
+
+ - name: Upload docker logs to GitHub
+ if: ${{ failure() && steps.run-it.conclusion == 'failure' }}
+ uses: actions/upload-artifact@master
+ with:
+ name: IT-${{ inputs.it }} docker logs (Compile=jdk${{ inputs.build_jdk }}, Run=jdk${{ inputs.runtime_jdk }}, Indexer=${{ inputs.use_indexer }}, Mysql=${{ inputs.mysql_driver }})
+ path: docker-logs.tgz
+
+ - name: Collect service logs on failure
+ if: ${{ failure() && steps.run-it.conclusion == 'failure' }}
+ run: |
+ tar cvzf ./service-logs.tgz ~/shared/logs
+
+ - name: Upload Druid service logs to GitHub
+ if: ${{ failure() && steps.run-it.conclusion == 'failure' }}
+ uses: actions/upload-artifact@master
+ with:
+ name: IT-${{ inputs.it }} service logs (Compile=jdk${{ inputs.build_jdk }}, Run=jdk${{ inputs.runtime_jdk }}, Indexer=${{ inputs.use_indexer }}, Mysql=${{ inputs.mysql_driver }})
+ path: service-logs.tgz
diff --git a/.github/workflows/reusable-standard-its.yml b/.github/workflows/reusable-standard-its.yml
index d4b0de1c0cfd..b96bb4b4c495 100644
--- a/.github/workflows/reusable-standard-its.yml
+++ b/.github/workflows/reusable-standard-its.yml
@@ -88,13 +88,34 @@ jobs:
echo "${MVN} verify -pl integration-tests -P integration-tests ${{ inputs.testing_groups }} -Djvm.runtime=${{ inputs.runtime_jdk }} -Dit.indexer=${{ inputs.use_indexer }} ${MAVEN_SKIP} -Doverride.config.path=${{ inputs.override_config_path }}"
${MVN} verify -pl integration-tests -P integration-tests ${{ inputs.testing_groups }} -Djvm.runtime=${{ inputs.runtime_jdk }} -Dit.indexer=${{ inputs.use_indexer }} ${MAVEN_SKIP} -Doverride.config.path=${{ inputs.override_config_path }}
- - name: Debug IT
+ - name: Collect docker logs on failure
if: ${{ failure() && steps.run-it.conclusion == 'failure' }}
run: |
- for v in broker router ${{ inputs.use_indexer }} historical coordinator overlord; do
- echo "=======================druid-"$v"========================";
- echo "-----------------------docker logs-----------------------";
- sudo docker logs druid-"$v" 2>&1 | tail -1000 ||:;
- echo "-----------------------service logs----------------------";
- sudo docker exec druid-"$v" tail -1000 /shared/logs/"$v".log 2>&1 ||:;
+ mkdir docker-logs
+ for c in $(docker ps -a --format="{{.Names}}")
+ do
+ docker logs $c > ./docker-logs/$c.log
done
+
+ - name: Tar docker logs
+ if: ${{ failure() && steps.run-it.conclusion == 'failure' }}
+ run: tar cvzf ./docker-logs.tgz ./docker-logs
+
+ - name: Upload docker logs to GitHub
+ if: ${{ failure() && steps.run-it.conclusion == 'failure' }}
+ uses: actions/upload-artifact@master
+ with:
+ name: IT-${{ inputs.group }} docker logs (Compile=jdk${{ inputs.build_jdk }}, Run=jdk${{ inputs.runtime_jdk }}, Indexer=${{ inputs.use_indexer }}, Mysql=${{ inputs.mysql_driver }})
+ path: docker-logs.tgz
+
+ - name: Collect service logs on failure
+ if: ${{ failure() && steps.run-it.conclusion == 'failure' }}
+ run: |
+ tar cvzf ./service-logs.tgz ./shared/logs
+
+ - name: Upload Druid service logs to GitHub
+ if: ${{ failure() && steps.run-it.conclusion == 'failure' }}
+ uses: actions/upload-artifact@master
+ with:
+ name: IT-${{ inputs.group }} service logs (Compile=jdk${{ inputs.build_jdk }}, Run=jdk${{ inputs.runtime_jdk }}, Indexer=${{ inputs.use_indexer }}, Mysql=${{ inputs.mysql_driver }})
+ path: service-logs.tgz
diff --git a/.github/workflows/reusable-unit-tests.yml b/.github/workflows/reusable-unit-tests.yml
index 34d992c397c2..8769673b1e89 100644
--- a/.github/workflows/reusable-unit-tests.yml
+++ b/.github/workflows/reusable-unit-tests.yml
@@ -111,6 +111,15 @@ jobs:
MAVEN_PROJECTS: ${{ inputs.maven_projects }}
run: ./.github/scripts/unit_tests_script.sh
+ - name: Test Report
+ uses: dorny/test-reporter@v1
+ if: always()
+ with:
+ name: Maven Tests
+ path: '**/target/**/surefire-reports/TEST-*.xml'
+ reporter: java-junit
+ fail-on-error: true
+
- name: set outputs on failure
id: set_outputs
if: ${{ failure() }}
diff --git a/.github/workflows/standard-its.yml b/.github/workflows/standard-its.yml
index f44dee04bdf8..2648dc0993b6 100644
--- a/.github/workflows/standard-its.yml
+++ b/.github/workflows/standard-its.yml
@@ -159,6 +159,7 @@ jobs:
with:
path: ~/.m2/repository
key: maven-${{ runner.os }}-8-${{ github.sha }}
+ restore-keys: setup-java-Linux-maven-${{ hashFiles('**/pom.xml') }}
- name: Maven build
if: steps.maven-restore.outputs.cache-hit != 'true'
diff --git a/.github/workflows/static-checks.yml b/.github/workflows/static-checks.yml
index 8fc05d35bd87..4b1c4db0c68d 100644
--- a/.github/workflows/static-checks.yml
+++ b/.github/workflows/static-checks.yml
@@ -41,20 +41,17 @@ jobs:
strategy:
fail-fast: false
matrix:
- java: [ 'jdk8', 'jdk11', 'jdk17' ]
+ java: [ '8', '11', '17' ]
runs-on: ubuntu-latest
steps:
- name: checkout branch
uses: actions/checkout@v3
- - name: set java version
- run: |
- export jdk=${{ matrix.java }}
- echo "java_version=${jdk:3}" >> $GITHUB_ENV
-
- - name: setup ${{ matrix.java }}
- run: |
- echo "JAVA_HOME=$JAVA_HOME_${{ env.java_version }}_X64" >> $GITHUB_ENV
+ - uses: actions/setup-java@v3
+ with:
+ distribution: 'zulu'
+ java-version: ${{ matrix.java }}
+ cache: 'maven'
- name: packaging check
run: |
@@ -66,60 +63,53 @@ jobs:
- name: script checks
# who watches the watchers?
- if: ${{ matrix.java == 'jdk8' }}
+ if: ${{ matrix.java == '8' }}
run: ./check_test_suite_test.py
- name: (openjdk17) strict compilation
- if: ${{ matrix.java == 'jdk17' }}
+ if: ${{ matrix.java == '17' }}
# errorprone requires JDK 11+
# Strict compilation requires more than 2 GB
run: ${MVN} clean -DstrictCompile compile test-compile --fail-at-end ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS}
- name: maven install
- if: ${{ matrix.java == 'jdk8' }}
+ if: ${{ matrix.java == '8' }}
run: |
echo 'Running Maven install...' &&
${MVN} clean install -q -ff -pl '!distribution,!:druid-it-image,!:druid-it-cases' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS} -T1C &&
${MVN} install -q -ff -pl 'distribution' ${MAVEN_SKIP} ${MAVEN_SKIP_TESTS}
- name: checkstyle
- if: ${{ matrix.java == 'jdk8' }}
+ if: ${{ matrix.java == '8' }}
run: ${MVN} checkstyle:checkstyle --fail-at-end
- name: license checks
- if: ${{ matrix.java == 'jdk8' }}
+ if: ${{ matrix.java == '8' }}
run: ./.github/scripts/license_checks_script.sh
- name: analyze dependencies
- if: ${{ matrix.java == 'jdk8' }}
- run: |
- ./.github/scripts/analyze_dependencies_script.sh
-
- - name: analyze dependencies for hadoop2
- if: ${{ matrix.java == 'jdk8' }}
- env:
- HADOOP_PROFILE: -Phadoop2
+ if: ${{ matrix.java == '8' }}
run: |
./.github/scripts/analyze_dependencies_script.sh
- name: animal sniffer checks
- if: ${{ matrix.java == 'jdk8' }}
+ if: ${{ matrix.java == '8' }}
run: ${MVN} animal-sniffer:check --fail-at-end
- name: enforcer checks
- if: ${{ matrix.java == 'jdk8' }}
+ if: ${{ matrix.java == '8' }}
run: ${MVN} enforcer:enforce --fail-at-end
- name: forbidden api checks
- if: ${{ matrix.java == 'jdk8' }}
+ if: ${{ matrix.java == '8' }}
run: ${MVN} forbiddenapis:check forbiddenapis:testCheck --fail-at-end
- name: pmd checks
- if: ${{ matrix.java == 'jdk8' }}
+ if: ${{ matrix.java == '8' }}
run: ${MVN} pmd:check --fail-at-end # TODO: consider adding pmd:cpd-check
- name: spotbugs checks
- if: ${{ matrix.java == 'jdk8' }}
+ if: ${{ matrix.java == '8' }}
run: ${MVN} spotbugs:check --fail-at-end -pl '!benchmarks'
intellij-inspections:
@@ -130,9 +120,11 @@ jobs:
- name: checkout branch
uses: actions/checkout@v3
- - name: setup JDK8
- run: |
- echo "JAVA_HOME=$JAVA_HOME_8_X64" >> $GITHUB_ENV
+ - uses: actions/setup-java@v3
+ with:
+ distribution: 'zulu'
+ java-version: '8'
+ cache: 'maven'
- name: maven install
run: |
@@ -160,9 +152,11 @@ jobs:
- name: checkout branch
uses: actions/checkout@v3
- - name: setup JDK17
- run: |
- echo "JAVA_HOME=$JAVA_HOME_17_X64" >> $GITHUB_ENV
+ - uses: actions/setup-java@v3
+ with:
+ distribution: 'zulu'
+ java-version: '17'
+ cache: 'maven'
- name: setup node
uses: actions/setup-node@v3
@@ -173,7 +167,7 @@ jobs:
run: |
(cd website && npm install)
cd website
- npm run link-lint
+ npm run build
npm run spellcheck
- name: web console
diff --git a/.github/workflows/unit-and-integration-tests-unified.yml b/.github/workflows/unit-and-integration-tests-unified.yml
index ff963dac7718..6ff6c8bd6500 100644
--- a/.github/workflows/unit-and-integration-tests-unified.yml
+++ b/.github/workflows/unit-and-integration-tests-unified.yml
@@ -72,6 +72,7 @@ jobs:
with:
path: ~/.m2/repository
key: maven-${{ runner.os }}-${{ matrix.jdk }}-${{ github.sha }}
+ restore-keys: setup-java-Linux-maven-${{ hashFiles('**/pom.xml') }}
- name: Cache targets
id: target
diff --git a/.gitignore b/.gitignore
index cd33e6271a1d..7d7cf0d5bd24 100644
--- a/.gitignore
+++ b/.gitignore
@@ -29,6 +29,13 @@ integration-tests/gen-scripts/
/bin/
*.hprof
**/.ipynb_checkpoints/
+website/.yarn/
+website/node_modules/
+website/.docusaurus/
+website/build/
+
+# Local Netlify folder
+.netlify
*.pyc
**/.ipython/
**/.jupyter/
diff --git a/README.md b/README.md
index 6a2448228986..cfd87db66728 100644
--- a/README.md
+++ b/README.md
@@ -87,7 +87,11 @@ Use the built-in query workbench to prototype [DruidSQL](https://druid.apache.or
See the [latest documentation](https://druid.apache.org/docs/latest/) for the documentation for the current official release. If you need information on a previous release, you can browse [previous releases documentation](https://druid.apache.org/docs/).
-Make documentation and tutorials updates in [`/docs`](https://github.com/apache/druid/tree/master/docs) using [MarkDown](https://www.markdownguide.org/) and contribute them using a pull request.
+Make documentation and tutorials updates in [`/docs`](https://github.com/apache/druid/tree/master/docs) using [Markdown](https://www.markdownguide.org/) or extended Markdown [(MDX)](https://mdxjs.com/). Then, open a pull request.
+
+To build the site locally, you need Node 16.14 or higher and to install Docusaurus 2 with `npm|yarn install` in the `website` directory. Then you can run `npm|yarn start` to launch a local build of the docs.
+
+If you're looking to update non-doc pages like Use Cases, those files are in the [`druid-website-src`](https://github.com/apache/druid-website-src/tree/master) repo.
### Community
diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml
index 0d6319c3202b..fdada96d98a6 100644
--- a/benchmarks/pom.xml
+++ b/benchmarks/pom.xml
@@ -261,7 +261,7 @@
maven-source-plugin
- 2.2.1
+ 3.3.0maven-surefire-plugin
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FrontCodedIndexedBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FrontCodedIndexedBenchmark.java
index fe0e717ab18e..d09d501a050e 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/FrontCodedIndexedBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FrontCodedIndexedBenchmark.java
@@ -88,10 +88,10 @@ public class FrontCodedIndexedBenchmark
@Param({
"generic",
- "front-coded-4",
- "front-coded-16",
- "front-coded-incremental-buckets-4",
- "front-coded-incremental-buckets-16"
+ "front-coded-v0-4",
+ "front-coded-v0-16",
+ "front-coded-v1-4",
+ "front-coded-v1-16"
})
public String indexType;
@@ -138,7 +138,7 @@ public void createIndex() throws IOException
FrontCodedIndexedWriter frontCodedIndexedWriter = new FrontCodedIndexedWriter(
new OnHeapMemorySegmentWriteOutMedium(),
ByteOrder.nativeOrder(),
- "front-coded-4".equals(indexType) ? 4 : 16,
+ "front-coded-v0-4".equals(indexType) ? 4 : 16,
FrontCodedIndexed.V0
);
frontCodedIndexedWriter.open();
@@ -146,7 +146,7 @@ public void createIndex() throws IOException
FrontCodedIndexedWriter frontCodedIndexedWriterIncrementalBuckets = new FrontCodedIndexedWriter(
new OnHeapMemorySegmentWriteOutMedium(),
ByteOrder.nativeOrder(),
- "front-coded-incremental-buckets-4".equals(indexType) ? 4 : 16,
+ "front-coded-v1-4".equals(indexType) ? 4 : 16,
FrontCodedIndexed.V1
);
frontCodedIndexedWriterIncrementalBuckets.open();
@@ -166,11 +166,11 @@ public void createIndex() throws IOException
fileGeneric = File.createTempFile("genericIndexedBenchmark", "meta");
smooshDirFrontCodedIncrementalBuckets = FileUtils.createTempDir();
- fileFrontCodedIncrementalBuckets = File.createTempFile("frontCodedIndexedBenchmarkIncrementalBuckets", "meta");
+ fileFrontCodedIncrementalBuckets = File.createTempFile("frontCodedIndexedBenchmarkv1Buckets", "meta");
EncodingSizeProfiler.encodedSize = (int) ("generic".equals(indexType)
? genericIndexedWriter.getSerializedSize()
- : indexType.startsWith("front-coded-incremental-buckets")
+ : indexType.startsWith("front-coded-v1")
? frontCodedIndexedWriterIncrementalBuckets.getSerializedSize()
: frontCodedIndexedWriter.getSerializedSize());
try (
@@ -286,7 +286,7 @@ public void createIndex() throws IOException
}
if ("generic".equals(indexType)) {
indexed = genericIndexed.singleThreaded();
- } else if (indexType.startsWith("front-coded-incremental-buckets")) {
+ } else if (indexType.startsWith("front-coded-v1")) {
indexed = frontCodedIndexedIncrementalBuckets;
} else {
indexed = frontCodedIndexed;
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
index 87a421df5fcb..2d902c12163f 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
@@ -52,13 +52,10 @@
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
-import org.apache.druid.query.groupby.GroupByQueryEngine;
import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
+import org.apache.druid.query.groupby.GroupingEngine;
import org.apache.druid.query.groupby.ResultRow;
-import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
-import org.apache.druid.query.groupby.strategy.GroupByStrategyV1;
-import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.IndexIO;
@@ -128,9 +125,6 @@ public class GroupByTypeInterfaceBenchmark
@Param({"100000"})
private int rowsPerSegment;
- @Param({"v2"})
- private String defaultStrategy;
-
@Param({"all"})
private String queryGranularity;
@@ -346,11 +340,6 @@ public void setup() throws IOException
);
final GroupByQueryConfig config = new GroupByQueryConfig()
{
- @Override
- public String getDefaultStrategy()
- {
- return defaultStrategy;
- }
@Override
public int getBufferGrouperInitialBuckets()
@@ -365,8 +354,6 @@ public HumanReadableBytes getMaxOnDiskStorage()
}
};
config.setSingleThreaded(false);
- config.setMaxIntermediateRows(Integer.MAX_VALUE);
- config.setMaxResults(Integer.MAX_VALUE);
DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig()
{
@@ -385,27 +372,19 @@ public String getFormatString()
};
final Supplier configSupplier = Suppliers.ofInstance(config);
- final GroupByStrategySelector strategySelector = new GroupByStrategySelector(
+ final GroupingEngine groupingEngine = new GroupingEngine(
+ druidProcessingConfig,
configSupplier,
- new GroupByStrategyV1(
- configSupplier,
- new GroupByQueryEngine(configSupplier, bufferPool),
- QueryBenchmarkUtil.NOOP_QUERYWATCHER
- ),
- new GroupByStrategyV2(
- druidProcessingConfig,
- configSupplier,
- bufferPool,
- mergePool,
- TestHelper.makeJsonMapper(),
- new ObjectMapper(new SmileFactory()),
- QueryBenchmarkUtil.NOOP_QUERYWATCHER
- )
+ bufferPool,
+ mergePool,
+ TestHelper.makeJsonMapper(),
+ new ObjectMapper(new SmileFactory()),
+ QueryBenchmarkUtil.NOOP_QUERYWATCHER
);
factory = new GroupByQueryRunnerFactory(
- strategySelector,
- new GroupByQueryQueryToolChest(strategySelector)
+ groupingEngine,
+ new GroupByQueryQueryToolChest(groupingEngine)
);
}
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java
index 4b0d55c2c62a..5adfcee0de8e 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java
@@ -54,12 +54,13 @@
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.query.BrokerParallelMergeConfig;
import org.apache.druid.query.BySegmentQueryRunner;
import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.Druids;
import org.apache.druid.query.FinalizeResultsQueryRunner;
-import org.apache.druid.query.FluentQueryRunnerBuilder;
+import org.apache.druid.query.FluentQueryRunner;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryPlus;
@@ -76,14 +77,11 @@
import org.apache.druid.query.expression.TestExprMacroTable;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
-import org.apache.druid.query.groupby.GroupByQueryEngine;
import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
+import org.apache.druid.query.groupby.GroupingEngine;
import org.apache.druid.query.groupby.ResultRow;
-import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
-import org.apache.druid.query.groupby.strategy.GroupByStrategyV1;
-import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
import org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
@@ -103,7 +101,6 @@
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.generator.SegmentGenerator;
-import org.apache.druid.segment.join.JoinableFactoryWrapperTest;
import org.apache.druid.server.QueryStackTests;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.metrics.NoopServiceEmitter;
@@ -258,12 +255,6 @@ public int getNumThreads()
{
return numProcessingThreads;
}
-
- @Override
- public boolean useParallelMergePool()
- {
- return true;
- }
};
conglomerate = new DefaultQueryRunnerFactoryConglomerate(
@@ -293,11 +284,6 @@ public boolean useParallelMergePool()
GroupByQueryRunnerTest.DEFAULT_MAPPER,
new GroupByQueryConfig()
{
- @Override
- public String getDefaultStrategy()
- {
- return GroupByStrategySelector.STRATEGY_V2;
- }
},
processingConfig
)
@@ -339,10 +325,15 @@ public > QueryToolChest getToolChest
new ForegroundCachePopulator(JSON_MAPPER, new CachePopulatorStats(), 0),
new CacheConfig(),
new DruidHttpClientConfig(),
- processingConfig,
+ new BrokerParallelMergeConfig() {
+ @Override
+ public boolean useParallelMergePool()
+ {
+ return true;
+ }
+ },
forkJoinPool,
QueryStackTests.DEFAULT_NOOP_SCHEDULER,
- JoinableFactoryWrapperTest.NOOP_JOINABLE_FACTORY_WRAPPER,
new NoopServiceEmitter()
);
}
@@ -365,25 +356,17 @@ private static GroupByQueryRunnerFactory makeGroupByQueryRunnerFactory(
bufferSupplier,
processingConfig.getNumMergeBuffers()
);
- final GroupByStrategySelector strategySelector = new GroupByStrategySelector(
+ final GroupingEngine groupingEngine = new GroupingEngine(
+ processingConfig,
configSupplier,
- new GroupByStrategyV1(
- configSupplier,
- new GroupByQueryEngine(configSupplier, bufferPool),
- QueryRunnerTestHelper.NOOP_QUERYWATCHER
- ),
- new GroupByStrategyV2(
- processingConfig,
- configSupplier,
- bufferPool,
- mergeBufferPool,
- mapper,
- mapper,
- QueryRunnerTestHelper.NOOP_QUERYWATCHER
- )
+ bufferPool,
+ mergeBufferPool,
+ mapper,
+ mapper,
+ QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
- final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(strategySelector);
- return new GroupByQueryRunnerFactory(strategySelector, toolChest);
+ final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine);
+ return new GroupByQueryRunnerFactory(groupingEngine, toolChest);
}
@TearDown(Level.Trial)
@@ -480,8 +463,11 @@ public void groupByQuery(Blackhole blackhole)
private List runQuery()
{
//noinspection unchecked
- QueryRunner theRunner = new FluentQueryRunnerBuilder(toolChestWarehouse.getToolChest(query))
- .create(cachingClusteredClient.getQueryRunnerForIntervals(query, query.getIntervals()))
+ QueryRunner theRunner = FluentQueryRunner
+ .create(
+ cachingClusteredClient.getQueryRunnerForIntervals(query, query.getIntervals()),
+ toolChestWarehouse.getToolChest(query)
+ )
.applyPreMergeDecoration()
.mergeResults()
.applyPostMergeDecoration();
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
index 08c51b9edf2b..d355dd2d0058 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
@@ -63,15 +63,12 @@
import org.apache.druid.query.filter.BoundDimFilter;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
-import org.apache.druid.query.groupby.GroupByQueryEngine;
import org.apache.druid.query.groupby.GroupByQueryQueryToolChest;
import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
+import org.apache.druid.query.groupby.GroupingEngine;
import org.apache.druid.query.groupby.ResultRow;
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
-import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
-import org.apache.druid.query.groupby.strategy.GroupByStrategyV1;
-import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
@@ -139,9 +136,6 @@ public class GroupByBenchmark
@Param({"basic.A", "basic.nested"})
private String schemaAndQuery;
- @Param({"v1", "v2"})
- private String defaultStrategy;
-
@Param({"all", "day"})
private String queryGranularity;
@@ -461,11 +455,6 @@ public void setup()
);
final GroupByQueryConfig config = new GroupByQueryConfig()
{
- @Override
- public String getDefaultStrategy()
- {
- return defaultStrategy;
- }
@Override
public int getBufferGrouperInitialBuckets()
@@ -480,8 +469,6 @@ public HumanReadableBytes getMaxOnDiskStorage()
}
};
config.setSingleThreaded(false);
- config.setMaxIntermediateRows(Integer.MAX_VALUE);
- config.setMaxResults(Integer.MAX_VALUE);
DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig()
{
@@ -500,27 +487,19 @@ public String getFormatString()
};
final Supplier configSupplier = Suppliers.ofInstance(config);
- final GroupByStrategySelector strategySelector = new GroupByStrategySelector(
+ final GroupingEngine groupingEngine = new GroupingEngine(
+ druidProcessingConfig,
configSupplier,
- new GroupByStrategyV1(
- configSupplier,
- new GroupByQueryEngine(configSupplier, bufferPool),
- QueryBenchmarkUtil.NOOP_QUERYWATCHER
- ),
- new GroupByStrategyV2(
- druidProcessingConfig,
- configSupplier,
- bufferPool,
- mergePool,
- TestHelper.makeJsonMapper(),
- new ObjectMapper(new SmileFactory()),
- QueryBenchmarkUtil.NOOP_QUERYWATCHER
- )
+ bufferPool,
+ mergePool,
+ TestHelper.makeJsonMapper(),
+ new ObjectMapper(new SmileFactory()),
+ QueryBenchmarkUtil.NOOP_QUERYWATCHER
);
factory = new GroupByQueryRunnerFactory(
- strategySelector,
- new GroupByQueryQueryToolChest(strategySelector)
+ groupingEngine,
+ new GroupByQueryQueryToolChest(groupingEngine)
);
}
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java
index 7733281908f0..498a9c2bdacd 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java
@@ -84,7 +84,7 @@ public class SqlExpressionBenchmark
static {
NullHandling.initializeForTests();
- ExpressionProcessing.initializeForStrictBooleansTests(true);
+ ExpressionProcessing.initializeForTests();
}
private static final DruidProcessingConfig PROCESSING_CONFIG = new DruidProcessingConfig()
@@ -106,13 +106,6 @@ public int getNumThreads()
{
return 1;
}
-
- @Override
- public boolean useParallelMergePoolConfigured()
- {
- return true;
- }
-
@Override
public String getFormatString()
{
@@ -204,7 +197,7 @@ public String getFormatString()
"SELECT TIME_SHIFT(MILLIS_TO_TIMESTAMP(long4), 'PT1H', 1), string2, SUM(long1 * double4) FROM foo GROUP BY 1,2 ORDER BY 3",
// 37: time shift + expr agg (group by), uniform distribution high cardinality
"SELECT TIME_SHIFT(MILLIS_TO_TIMESTAMP(long5), 'PT1H', 1), string2, SUM(long1 * double4) FROM foo GROUP BY 1,2 ORDER BY 3",
- // 38: LATEST aggregator
+ // 38: LATEST aggregator long
"SELECT LATEST(long1) FROM foo",
// 39: LATEST aggregator double
"SELECT LATEST(double4) FROM foo",
@@ -214,7 +207,13 @@ public String getFormatString()
"SELECT LATEST(float3), LATEST(long1), LATEST(double4) FROM foo",
// 42,43: filter numeric nulls
"SELECT SUM(long5) FROM foo WHERE long5 IS NOT NULL",
- "SELECT string2, SUM(long5) FROM foo WHERE long5 IS NOT NULL GROUP BY 1"
+ "SELECT string2, SUM(long5) FROM foo WHERE long5 IS NOT NULL GROUP BY 1",
+ // 44: EARLIEST aggregator long
+ "SELECT EARLIEST(long1) FROM foo",
+ // 45: EARLIEST aggregator double
+ "SELECT EARLIEST(double4) FROM foo",
+ // 46: EARLIEST aggregator float
+ "SELECT EARLIEST(float3) FROM foo"
);
@Param({"5000000"})
@@ -272,7 +271,11 @@ public String getFormatString()
"40",
"41",
"42",
- "43"
+ "43",
+ "44",
+ "45",
+ "46",
+ "47"
})
private String query;
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java
index 98514512e9ab..e1d866a4ba36 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java
@@ -34,8 +34,8 @@
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.expression.TestExprMacroTable;
+import org.apache.druid.segment.AutoTypeColumnSchema;
import org.apache.druid.segment.IndexSpec;
-import org.apache.druid.segment.NestedDataDimensionSchema;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.column.StringEncodingStrategy;
import org.apache.druid.segment.data.FrontCodedIndexed;
@@ -90,7 +90,7 @@ public class SqlNestedDataBenchmark
static {
NullHandling.initializeForTests();
- ExpressionProcessing.initializeForStrictBooleansTests(true);
+ ExpressionProcessing.initializeForTests();
}
private static final DruidProcessingConfig PROCESSING_CONFIG = new DruidProcessingConfig()
@@ -112,13 +112,7 @@ public int getNumThreads()
{
return 1;
}
-
- @Override
- public boolean useParallelMergePoolConfigured()
- {
- return true;
- }
-
+
@Override
public String getFormatString()
{
@@ -304,7 +298,7 @@ public void setup()
);
List dims = ImmutableList.builder()
.addAll(schemaInfo.getDimensionsSpec().getDimensions())
- .add(new NestedDataDimensionSchema("nested"))
+ .add(new AutoTypeColumnSchema("nested"))
.build();
DimensionsSpec dimsSpec = new DimensionsSpec(dims);
diff --git a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java
index 531003352123..52a6e0d97570 100644
--- a/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java
@@ -24,9 +24,9 @@
import org.apache.druid.client.DataSourcesSnapshot;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.server.coordinator.duty.CompactionSegmentIterator;
-import org.apache.druid.server.coordinator.duty.CompactionSegmentSearchPolicy;
-import org.apache.druid.server.coordinator.duty.NewestSegmentFirstPolicy;
+import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator;
+import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy;
+import org.apache.druid.server.coordinator.compact.NewestSegmentFirstPolicy;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentTimeline;
import org.apache.druid.timeline.partition.NumberedShardSpec;
@@ -142,8 +142,7 @@ public void measureNewestSegmentFirstPolicy(Blackhole blackhole)
{
final CompactionSegmentIterator iterator = policy.reset(compactionConfigs, dataSources, Collections.emptyMap());
for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) {
- final List segments = iterator.next();
- blackhole.consume(segments);
+ blackhole.consume(iterator.next());
}
}
}
diff --git a/cloud/aws-common/pom.xml b/cloud/aws-common/pom.xml
index 61bc6720016c..4bad5332ceef 100644
--- a/cloud/aws-common/pom.xml
+++ b/cloud/aws-common/pom.xml
@@ -76,8 +76,8 @@
aws-java-sdk-core
- javax.validation
- validation-api
+ jakarta.validation
+ jakarta.validation-api
diff --git a/cloud/aws-common/src/main/java/org/apache/druid/common/aws/AWSClientUtil.java b/cloud/aws-common/src/main/java/org/apache/druid/common/aws/AWSClientUtil.java
index c587ac6580aa..939cdcbe5ecb 100644
--- a/cloud/aws-common/src/main/java/org/apache/druid/common/aws/AWSClientUtil.java
+++ b/cloud/aws-common/src/main/java/org/apache/druid/common/aws/AWSClientUtil.java
@@ -22,11 +22,50 @@
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
import com.amazonaws.retry.RetryUtils;
+import com.amazonaws.services.s3.model.DeleteObjectsRequest;
+import com.amazonaws.services.s3.model.MultiObjectDeleteException;
+import com.google.common.collect.ImmutableSet;
import java.io.IOException;
+import java.util.Set;
public class AWSClientUtil
{
+ /**
+ * This list of error code come from {@link RetryUtils}, and
+ * .... At the moment, aws sdk
+ * does not expose a good way of retrying
+ * {@link com.amazonaws.services.s3.AmazonS3#deleteObjects(DeleteObjectsRequest)} requests. This request is used in
+ * org.apache.druid.storage.s3.S3DataSegmentKiller to delete a batch of segments from deep storage.
+ */
+ private static final Set RECOVERABLE_ERROR_CODES = ImmutableSet.of(
+ "503 SlowDown",
+ "AuthFailure",
+ "BandwidthLimitExceeded",
+ "EC2ThrottledException",
+ "IDPCommunicationError",
+ "InternalError",
+ "InvalidSignatureException",
+ "PriorRequestNotComplete",
+ "ProvisionedThroughputExceededException",
+ "RequestExpired",
+ "RequestInTheFuture",
+ "RequestLimitExceeded",
+ "RequestThrottled",
+ "RequestThrottledException",
+ "RequestTimeTooSkewed",
+ "RequestTimeout",
+ "RequestTimeoutException",
+ "ServiceUnavailable",
+ "SignatureDoesNotMatch",
+ "SlowDown",
+ "ThrottledException",
+ "ThrottlingException",
+ "TooManyRequestsException",
+ "TransactionInProgressException",
+ "Throttling"
+ );
+
/**
* Checks whether an exception can be retried or not. Implementation is copied
* from {@link com.amazonaws.retry.PredefinedRetryPolicies.SDKDefaultRetryCondition} except deprecated methods
@@ -54,6 +93,19 @@ public static boolean isClientExceptionRecoverable(AmazonClientException excepti
return true;
}
- return RetryUtils.isClockSkewError(exception);
+ if (RetryUtils.isClockSkewError(exception)) {
+ return true;
+ }
+
+ if (exception instanceof MultiObjectDeleteException) {
+ MultiObjectDeleteException multiObjectDeleteException = (MultiObjectDeleteException) exception;
+ for (MultiObjectDeleteException.DeleteError error : multiObjectDeleteException.getErrors()) {
+ if (RECOVERABLE_ERROR_CODES.contains(error.getCode())) {
+ return true;
+ }
+ }
+ }
+
+ return false;
}
}
diff --git a/cloud/aws-common/src/test/java/org/apache/druid/common/aws/AWSClientUtilTest.java b/cloud/aws-common/src/test/java/org/apache/druid/common/aws/AWSClientUtilTest.java
index bb1c4cdca796..3d1ff9a20c19 100644
--- a/cloud/aws-common/src/test/java/org/apache/druid/common/aws/AWSClientUtilTest.java
+++ b/cloud/aws-common/src/test/java/org/apache/druid/common/aws/AWSClientUtilTest.java
@@ -21,6 +21,8 @@
import com.amazonaws.AmazonClientException;
import com.amazonaws.AmazonServiceException;
+import com.amazonaws.services.s3.model.MultiObjectDeleteException;
+import com.google.common.collect.ImmutableList;
import org.junit.Assert;
import org.junit.Test;
@@ -82,6 +84,20 @@ public void testRecoverableException_ClockSkewedError()
Assert.assertTrue(AWSClientUtil.isClientExceptionRecoverable(ex));
}
+ @Test
+ public void testRecoverableException_MultiObjectDeleteException()
+ {
+ MultiObjectDeleteException.DeleteError retryableError = new MultiObjectDeleteException.DeleteError();
+ retryableError.setCode("RequestLimitExceeded");
+ MultiObjectDeleteException.DeleteError nonRetryableError = new MultiObjectDeleteException.DeleteError();
+ nonRetryableError.setCode("nonRetryableError");
+ MultiObjectDeleteException ex = new MultiObjectDeleteException(
+ ImmutableList.of(retryableError, nonRetryableError),
+ ImmutableList.of()
+ );
+ Assert.assertTrue(AWSClientUtil.isClientExceptionRecoverable(ex));
+ }
+
@Test
public void testNonRecoverableException_RuntimeException()
{
diff --git a/codestyle/guava16-forbidden-apis.txt b/codestyle/guava16-forbidden-apis.txt
deleted file mode 100644
index 274a12696fb2..000000000000
--- a/codestyle/guava16-forbidden-apis.txt
+++ /dev/null
@@ -1,3 +0,0 @@
-# Those signatures are only available in Guava 16:
-com.google.common.util.concurrent.MoreExecutors#sameThreadExecutor() @ Use org.apache.druid.java.util.common.concurrent.Execs#directExecutor()
-com.google.common.base.Objects#firstNonNull(java.lang.Object, java.lang.Object) @ Use org.apache.druid.common.guava.GuavaUtils#firstNonNull(java.lang.Object, java.lang.Object) instead (probably... the GuavaUtils method return object is nullable)
diff --git a/dev/druid_intellij_formatting.xml b/dev/druid_intellij_formatting.xml
index 6e0a33c9fd21..7771b2654721 100644
--- a/dev/druid_intellij_formatting.xml
+++ b/dev/druid_intellij_formatting.xml
@@ -74,6 +74,10 @@
+
+
+
+
diff --git a/distribution/asf-release-process-guide.md b/distribution/asf-release-process-guide.md
index 08ea5409c53b..d96047885373 100644
--- a/distribution/asf-release-process-guide.md
+++ b/distribution/asf-release-process-guide.md
@@ -258,7 +258,7 @@ It is also the release managers responsibility for correctly assigning all PRs m
| [get-milestone-contributors](bin/get-milestone-contributors.py) | lists github users who contributed to a milestone |
| [get-milestone-prs](bin/get-milestone-prs.py) | lists PRs between tags or commits and the milestone associated with them. |
| [tag-missing-milestones](bin/tag-missing-milestones.py) | Find pull requests which the milestone is missing and tag them properly. |
-| [find-missing-backports](bin/find-missing-backports.py) | Find PRs which have been back-ported to one release branch but not another. Useful if a bug fix release based on the previous release is required during a release cycle. |
+| [find-missing-backports](bin/find-missing-backports.py) | Find PRs which have been back-ported to one release branch but not another. Useful if a bug fix release based on the previous release is required during a release cycle. Make sure to fetch remote commits before running this command. |
| [make-linkable-release-notes](bin/make-linkable-release-notes.py) | given input of a version, input markdown file path, and output markdown file path, will rewrite markdown headers of the input file to have embedded links in the release notes style. |
@@ -370,20 +370,37 @@ $ svn commit -m 'add 0.17.0-rc3 artifacts'
### Update druid.staged.apache.org
+This repo is the source of truth for the Markdown files. The Markdown files get copied to `druid-website-src` and built there as part of the release process. It's all handled by a script in that repo called `do_all_things`.
+
+For more thorough instructions and a description of what the `do_all_things` script does, see the [`druid-website-src` README](https://github.com/apache/druid-website-src)
+
1. Pull https://github.com/apache/druid-website and https://github.com/apache/druid-website-src. These repositories should be in the same directory as your Druid repository that should have the release tag checked out.
-2. From druid-website, checkout branch `asf-staging`.
+2. From `druid-website`, checkout branch `asf-staging`.
-3. From druid-website-src, create a release branch from `master` and run `./release.sh 0.17.0 0.17.0`, replacing `0.17.0` where the first argument is the release version and 2nd argument is commit-ish. This script will:
+3. From `druid-website-src`, create a release branch from `master`, such as `27.0.0-docs`.
+ 1. Update the version list in `static/js/version`.js with the version you're releasing and the release date. The highest release version goes in position 0.
+ 1. In `scripts`, run:
+
+ ```python
+ # Include `--skip-install` if you already have Docusaurus 2 installed in druid-website-src.
+ # The script assumes you use `npm`. If you use `yarn`, include `--yarn`.
-* checkout the tag of the Druid release version
-* build the docs for that version into druid-website-src
-* build druid-website-src into druid-website
-* stage druid-website-src and druid-website repositories to git.
+ python do_all_things.py -v VERSION --source /my/path/to/apache/druid
+ ```
+
+
+4. Make a PR to the src repo (https://github.com/apache/druid-website-src) for the release branch. In the changed files, you should see the following:
+ - In `published_versions` directory: HTML files for `docs/VERSION` , `docs/latest`, and assorted HTML and non-HTML files
+ - In the `docs` directory at the root of the repo, the new Markdown files.
+
+ All these files should be part of your PR to `druid-website-src`.
+
+ Verify the site looks fine and that the versions on the homepage and Downloads page look correct. You can run `http-server` or something similar in `published_versions`.
-4. Make a PR to the src repo (https://github.com/apache/druid-website-src) for the release branch, such as `0.17.0-docs`.
-5. Make another PR to the website repo (https://github.com/apache/druid-website) for the `asf-staging` branch. Once the website PR is pushed to `asf-staging`, https://druid.staged.apache.org/ will be updated near immediately with the new docs.
+
+5. Make a PR to the website repo (https://github.com/apache/druid-website) for the `asf-staging` branch using the contents of `published_versions` in `druid-website-src`. Once the website PR is pushed to `asf-staging`, https://druid.staged.apache.org/ will be updated near immediately with the new docs.
### Create staged Maven repo
diff --git a/distribution/bin/check-licenses.py b/distribution/bin/check-licenses.py
index 5b059fd23afb..ae07e5a03733 100755
--- a/distribution/bin/check-licenses.py
+++ b/distribution/bin/check-licenses.py
@@ -251,7 +251,7 @@ def build_compatible_license_names():
compatible_licenses['BSD 3-Clause'] = 'BSD-3-Clause License'
compatible_licenses['BSD-3-Clause'] = 'BSD-3-Clause License'
- compatible_licenses['ICU License'] = 'ICU License'
+ compatible_licenses['Unicode/ICU License'] = 'Unicode/ICU License'
compatible_licenses['SIL Open Font License 1.1'] = 'SIL Open Font License 1.1'
diff --git a/distribution/bin/find-missing-backports.py b/distribution/bin/find-missing-backports.py
index c34677ed79eb..d7dcc143a9a2 100755
--- a/distribution/bin/find-missing-backports.py
+++ b/distribution/bin/find-missing-backports.py
@@ -22,37 +22,34 @@
import sys
+pr_number_pattern = r'\(#(\d+)\)'
+backport_pattern = r'\[Backport[^\]]*\]'
+
def extract_pr_title_from_commit_message(commit_msg):
# Extract commit message except the pr number
+ commit_msg = re.sub(backport_pattern, '', commit_msg)
pr_num_pos = commit_msg.find("(#")
if pr_num_pos < 0:
pr_num_pos = len(commit_msg)
- backport_pos = commit_msg.find("[Backport]")
- if backport_pos < 0:
- backport_pos = 0
- else:
- backport_pos = backport_pos + len("[Backport]")
- return commit_msg[backport_pos:pr_num_pos].strip()
-
-
-def extract_pr_title(pr_json):
- commit_url = pr_json['commits_url']
- resp = requests.get(commit_url, auth=(github_username, os.environ["GIT_TOKEN"]))
- title_candidates = [extract_pr_title_from_commit_message(pr_json['title'])]
- if len(resp.json()) == 1:
- title_candidates.append(extract_pr_title_from_commit_message(resp.json()[0]['commit']['message']))
- return title_candidates
+ return commit_msg[:pr_num_pos].strip()
+def extract_pr_numbers_from_commit_message(commit_msg):
+ extracted_numbers = re.findall(pr_number_pattern, commit_msg)
+ return extracted_numbers
-def find_missing_backports(pr_jsons, release_pr_subjects):
+def find_missing_backports(pr_jsons, release_pr_subjects, release_pr_numbers):
for pr in pr_jsons:
- if pr['milestone'] is not None:
- if pr['milestone']['number'] == milestone_number:
- for pr_title_candidate in extract_pr_title(pr):
- if pr_title_candidate in release_pr_subjects:
- return
- print("Missing backport found for PR {}, url: {}".format(pr['number'], pr['html_url']))
-
+ backport_found = False
+ for label in pr['labels']:
+ if label['name'] == 'Backport':
+ backport_found = True
+ pr_title_candidate = extract_pr_title_from_commit_message(pr['title'])
+ if pr_title_candidate in release_pr_subjects:
+ backport_found = True
+ if str(pr['number']) in release_pr_numbers:
+ backport_found = True
+ if backport_found == False:
+ print("Missing backport found for PR {}, url: {}".format(pr['number'], pr['html_url']))
def find_next_url(links):
for link in links:
@@ -95,15 +92,33 @@ def find_next_url(links):
all_release_commits = subprocess.check_output(command, shell=True).decode('UTF-8')
release_pr_subjects = set()
+release_pr_numbers = set()
for commit_msg in all_release_commits.splitlines():
title = extract_pr_title_from_commit_message(commit_msg)
+ pr_numbers = extract_pr_numbers_from_commit_message(commit_msg)
release_pr_subjects.add(title)
+ release_pr_numbers.update(pr_numbers)
+print("Number of release PR subjects: {}".format(len(release_pr_subjects)))
# Get all closed PRs and filter out with milestone
-next_url = "https://api.github.com/repos/apache/druid/pulls?state=closed"
-
-while next_url is not None:
- resp = requests.get(next_url, auth=(github_username, os.environ["GIT_TOKEN"]))
- find_missing_backports(resp.json(), release_pr_subjects)
- links = resp.headers['Link'].split(',')
- next_url = find_next_url(links)
\ No newline at end of file
+milestone_url = "https://api.github.com/repos/apache/druid/milestones/{}".format(milestone_number)
+resp = requests.get(milestone_url, auth=(github_username, os.environ["GIT_TOKEN"])).json()
+milestone_title = resp['title']
+pr_items = []
+page = 0
+while True:
+ page = page + 1
+ pr_url = "https://api.github.com/search/issues?per_page=50&page={}&q=milestone:{}+type:pr+is:merged+is:closed+repo:apache/druid".format(page,milestone_title)
+ pr_resp = requests.get(pr_url, auth=(github_username, os.environ["GIT_TOKEN"])).json()
+ if pr_resp['incomplete_results']:
+ sys.stderr.write('This script cannot handle incomplete results')
+ sys.exit(1)
+ pr_items.extend(pr_resp['items'])
+ if len(pr_resp['items']) < 50:
+ print("Total PRs for current milestone: {}".format(len(pr_items)))
+ print("Total expected count: {}".format(pr_resp['total_count']))
+ if pr_resp['total_count'] != len(pr_items):
+ sys.stderr.write('Expected PR count does not match with number of PRs fetched')
+ sys.exit(1)
+ break
+find_missing_backports(pr_items, release_pr_subjects, release_pr_numbers)
diff --git a/distribution/pom.xml b/distribution/pom.xml
index 1c7f2ca56c49..ecc00a9155d2 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -115,191 +115,6 @@
-
- dist-hadoop2
-
- false
-
- tar
-
-
-
-
-
- org.codehaus.mojo
- exec-maven-plugin
-
-
- generate-readme
- initialize
-
- exec
-
-
- ${project.basedir}/bin/build-textfile-readme.sh
-
- ${project.basedir}/../
- ${project.parent.version}
-
-
-
-
- generate-binary-license
- initialize
-
- exec
-
-
- ${project.basedir}/bin/generate-binary-license.py
-
- ${project.parent.basedir}/licenses/APACHE2
- ${project.parent.basedir}/licenses.yaml
- ${project.parent.basedir}/LICENSE.BINARY
-
-
-
-
- generate-binary-notice
- initialize
-
- exec
-
-
- ${project.basedir}/bin/generate-binary-notice.py
-
- ${project.parent.basedir}/NOTICE
- ${project.parent.basedir}/licenses.yaml
- ${project.parent.basedir}/NOTICE.BINARY
-
-
-
-
- pull-deps
- package
-
- exec
-
-
- ${project.parent.basedir}/examples/bin/run-java
-
- -classpath
-
- -Ddruid.extensions.loadList=[]
- -Ddruid.extensions.directory=${project.build.directory}/extensions
-
-
- -Ddruid.extensions.hadoopDependenciesDir=${project.build.directory}/hadoop-dependencies
-
- -Dhadoop2.enabled=true
- org.apache.druid.cli.Main
- tools
- pull-deps
- --clean
- --defaultVersion
- ${project.parent.version}
- -l
- ${settings.localRepository}
- -h
- org.apache.hadoop:hadoop-client:${hadoop.compile.version}
- -c
- org.apache.druid.extensions:druid-avro-extensions
- -c
- org.apache.druid.extensions:druid-azure-extensions
- -c
- org.apache.druid.extensions:druid-bloom-filter
- -c
- org.apache.druid.extensions:druid-datasketches
- -c
- org.apache.druid.extensions:druid-hdfs-storage
- -c
- org.apache.druid.extensions:druid-histogram
- -c
- org.apache.druid.extensions:druid-kafka-extraction-namespace
- -c
- org.apache.druid.extensions:druid-kafka-indexing-service
- -c
- org.apache.druid.extensions:druid-kinesis-indexing-service
- -c
- org.apache.druid.extensions:druid-lookups-cached-global
- -c
- org.apache.druid.extensions:druid-lookups-cached-single
- -c
- org.apache.druid.extensions:druid-multi-stage-query
- -c
- org.apache.druid.extensions:druid-protobuf-extensions
- -c
- org.apache.druid.extensions:mysql-metadata-storage
- -c
- org.apache.druid.extensions:druid-orc-extensions
- -c
- org.apache.druid.extensions:druid-parquet-extensions
- -c
- org.apache.druid.extensions:postgresql-metadata-storage
- -c
- org.apache.druid.extensions:druid-kerberos
- -c
- org.apache.druid.extensions:druid-s3-extensions
- -c
- org.apache.druid.extensions:druid-aws-rds-extensions
- -c
- org.apache.druid.extensions:druid-ec2-extensions
- -c
- org.apache.druid.extensions:druid-google-extensions
- -c
- org.apache.druid.extensions:druid-stats
- -c
- org.apache.druid.extensions:simple-client-sslcontext
- -c
- org.apache.druid.extensions:druid-basic-security
- -c
- org.apache.druid.extensions:druid-pac4j
- -c
- org.apache.druid.extensions:druid-ranger-security
- -c
- org.apache.druid.extensions:druid-kubernetes-extensions
- -c
- org.apache.druid.extensions:druid-catalog
- ${druid.distribution.pulldeps.opts}
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-assembly-plugin
-
-
- distro-assembly
- package
-
- single
-
-
- apache-druid-${project.parent.version}
- posix
-
- src/assembly/assembly.xml
-
-
-
-
-
-
- org.codehaus.mojo
- license-maven-plugin
-
-
- download-licenses
-
- download-licenses
-
-
-
-
-
-
- dist
diff --git a/doap_Druid.rdf b/doap_Druid.rdf
new file mode 100644
index 000000000000..4ac5f1ea999a
--- /dev/null
+++ b/doap_Druid.rdf
@@ -0,0 +1,42 @@
+
+
+
+
+
+ 2012-10-23
+
+ Apache Druid
+
+
+ High performance, real-time analytics database for sub-second queries at scale and under load.
+ Apache Druid is a high performance real-time analytics database. Druid's main value add is to reduce time to insight and action.
+
+Apache Druid is designed for workflows where fast queries and ingest really matter. Druid excels at powering UIs, running operational (ad-hoc) queries, or handling high concurrency. Consider Druid as an open source alternative to data warehouses for a variety of use cases.
+
+
+
+ Java
+
+
+
+
+
diff --git a/docs/api-reference/automatic-compaction-api.md b/docs/api-reference/automatic-compaction-api.md
index ea0f824190a0..d917cee42ebe 100644
--- a/docs/api-reference/automatic-compaction-api.md
+++ b/docs/api-reference/automatic-compaction-api.md
@@ -23,69 +23,766 @@ sidebar_label: Automatic compaction
~ under the License.
-->
-This document describes status and configuration API endpoints for [automatic compaction](../data-management/automatic-compaction.md) in Apache Druid.
+This topic describes the status and configuration API endpoints for [automatic compaction](../data-management/automatic-compaction.md) in Apache Druid. You can configure automatic compaction in the Druid web console or API.
-## Automatic compaction status
+In this topic, `http://ROUTER_IP:ROUTER_PORT` is a placeholder for your Router service address and port. Replace it with the information for your deployment. For example, use `http://localhost:8888` for quickstart deployments.
-`GET /druid/coordinator/v1/compaction/progress?dataSource={dataSource}`
+## Manage automatic compaction
-Returns the total size of segments awaiting compaction for the given dataSource. The specified dataSource must have [automatic compaction](../data-management/automatic-compaction.md) enabled.
+### Create or update automatic compaction configuration
-`GET /druid/coordinator/v1/compaction/status`
+Creates or updates the automatic compaction configuration for a datasource. Pass the automatic compaction as a JSON object in the request body.
-Returns the status and statistics from the auto-compaction run of all dataSources which have auto-compaction enabled in the latest run. The response payload includes a list of `latestStatus` objects. Each `latestStatus` represents the status for a dataSource (which has/had auto-compaction enabled).
+The automatic compaction configuration requires only the `dataSource` property. Druid fills all other properties with default values if not specified. See [Automatic compaction dynamic configuration](../configuration/index.md#automatic-compaction-dynamic-configuration) for configuration details.
-The `latestStatus` object has the following keys:
-* `dataSource`: name of the datasource for this status information
-* `scheduleStatus`: auto-compaction scheduling status. Possible values are `NOT_ENABLED` and `RUNNING`. Returns `RUNNING ` if the dataSource has an active auto-compaction config submitted. Otherwise, returns `NOT_ENABLED`.
-* `bytesAwaitingCompaction`: total bytes of this datasource waiting to be compacted by the auto-compaction (only consider intervals/segments that are eligible for auto-compaction)
-* `bytesCompacted`: total bytes of this datasource that are already compacted with the spec set in the auto-compaction config
-* `bytesSkipped`: total bytes of this datasource that are skipped (not eligible for auto-compaction) by the auto-compaction
-* `segmentCountAwaitingCompaction`: total number of segments of this datasource waiting to be compacted by the auto-compaction (only consider intervals/segments that are eligible for auto-compaction)
-* `segmentCountCompacted`: total number of segments of this datasource that are already compacted with the spec set in the auto-compaction config
-* `segmentCountSkipped`: total number of segments of this datasource that are skipped (not eligible for auto-compaction) by the auto-compaction
-* `intervalCountAwaitingCompaction`: total number of intervals of this datasource waiting to be compacted by the auto-compaction (only consider intervals/segments that are eligible for auto-compaction)
-* `intervalCountCompacted`: total number of intervals of this datasource that are already compacted with the spec set in the auto-compaction config
-* `intervalCountSkipped`: total number of intervals of this datasource that are skipped (not eligible for auto-compaction) by the auto-compaction
+Note that this endpoint returns an HTTP `200 OK` message code even if the datasource name does not exist.
-`GET /druid/coordinator/v1/compaction/status?dataSource={dataSource}`
+#### URL
-Similar to the API `/druid/coordinator/v1/compaction/status` above but filters response to only return information for the dataSource given.
-The dataSource must have auto-compaction enabled.
+POST/druid/coordinator/v1/config/compaction
-## Automatic compaction configuration
+#### Responses
-`GET /druid/coordinator/v1/config/compaction`
+
-Returns all automatic compaction configs.
+
-`GET /druid/coordinator/v1/config/compaction/{dataSource}`
+*Successfully submitted auto compaction configuration*
-Returns an automatic compaction config of a dataSource.
+
-`GET /druid/coordinator/v1/config/compaction/{dataSource}/history?interval={interval}&count={count}`
+---
+#### Sample request
+
+The following example creates an automatic compaction configuration for the datasource `wikipedia_hour`, which was ingested with `HOUR` segment granularity. This automatic compaction configuration performs compaction on `wikipedia_hour`, resulting in compacted segments that represent a day interval of data.
+
+In this example:
+
+* `wikipedia_hour` is a datasource with `HOUR` segment granularity.
+* `skipOffsetFromLatest` is set to `PT0S`, meaning that no data is skipped.
+* `partitionsSpec` is set to the default `dynamic`, allowing Druid to dynamically determine the optimal partitioning strategy.
+* `type` is set to `index_parallel`, meaning that parallel indexing is used.
+* `segmentGranularity` is set to `DAY`, meaning that each compacted segment is a day of data.
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/config/compaction"\
+--header 'Content-Type: application/json' \
+--data '{
+ "dataSource": "wikipedia_hour",
+ "skipOffsetFromLatest": "PT0S",
+ "tuningConfig": {
+ "partitionsSpec": {
+ "type": "dynamic"
+ },
+ "type": "index_parallel"
+ },
+ "granularitySpec": {
+ "segmentGranularity": "DAY"
+ }
+}'
+```
+
+
+
+```HTTP
+POST /druid/coordinator/v1/config/compaction HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+Content-Type: application/json
+Content-Length: 281
+
+{
+ "dataSource": "wikipedia_hour",
+ "skipOffsetFromLatest": "PT0S",
+ "tuningConfig": {
+ "partitionsSpec": {
+ "type": "dynamic"
+ },
+ "type": "index_parallel"
+ },
+ "granularitySpec": {
+ "segmentGranularity": "DAY"
+ }
+}
+```
+
+
+
+#### Sample response
+
+A successful request returns an HTTP `200 OK` message code and an empty response body.
+
+
+### Remove automatic compaction configuration
+
+Removes the automatic compaction configuration for a datasource. This updates the compaction status of the datasource to "Not enabled."
+
+#### URL
+
+DELETE/druid/coordinator/v1/config/compaction/:dataSource
+
+#### Responses
+
+
+
+
+
+*Successfully deleted automatic compaction configuration*
+
+
+
+*Datasource does not have automatic compaction or invalid datasource name*
+
+
+
+---
+
+
+#### Sample request
+
+
+
+
+
+```shell
+curl --request DELETE "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/config/compaction/wikipedia_hour"
+```
+
+
+
+```HTTP
+DELETE /druid/coordinator/v1/config/compaction/wikipedia_hour HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+#### Sample response
+
+A successful request returns an HTTP `200 OK` message code and an empty response body.
+
+### Update capacity for compaction tasks
+
+Updates the capacity for compaction tasks. The minimum number of compaction tasks is 1 and the maximum is 2147483647.
+
+Note that while the max compaction tasks can theoretically be set to 2147483647, the practical limit is determined by the available cluster capacity and is capped at 10% of the cluster's total capacity.
+
+#### URL
+
+POST/druid/coordinator/v1/config/compaction/taskslots
+
+#### Query parameters
+
+To limit the maximum number of compaction tasks, use the optional query parameters `ratio` and `max`:
+
+* `ratio` (optional)
+ * Type: Float
+ * Default: 0.1
+ * Limits the ratio of the total task slots to compaction task slots.
+* `max` (optional)
+ * Type: Int
+ * Default: 2147483647
+ * Limits the maximum number of task slots for compaction tasks.
+
+#### Responses
+
+
+
+
+
+*Successfully updated compaction configuration*
+
+
+
+*Invalid `max` value*
+
+
+
+---
+
+#### Sample request
+
+
+
+
+
+```shell
+curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/config/compaction/taskslots?ratio=0.2&max=250000"
+```
+
+
+
+```HTTP
+POST /druid/coordinator/v1/config/compaction/taskslots?ratio=0.2&max=250000 HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+#### Sample response
+
+A successful request returns an HTTP `200 OK` message code and an empty response body.
+
+## View automatic compaction configuration
+
+### Get all automatic compaction configurations
+
+Retrieves all automatic compaction configurations. Returns a `compactionConfigs` object containing the active automatic compaction configurations of all datasources.
+
+You can use this endpoint to retrieve `compactionTaskSlotRatio` and `maxCompactionTaskSlots` values for managing resource allocation of compaction tasks.
+
+#### URL
+
+GET/druid/coordinator/v1/config/compaction
+
+#### Responses
+
+
+
+
+
+*Successfully retrieved automatic compaction configurations*
+
+
+
+---
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/config/compaction"
+```
+
+
+
+```HTTP
+GET /druid/coordinator/v1/config/compaction HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+```json
+{
+ "compactionConfigs": [
+ {
+ "dataSource": "wikipedia_hour",
+ "taskPriority": 25,
+ "inputSegmentSizeBytes": 100000000000000,
+ "maxRowsPerSegment": null,
+ "skipOffsetFromLatest": "PT0S",
+ "tuningConfig": {
+ "maxRowsInMemory": null,
+ "appendableIndexSpec": null,
+ "maxBytesInMemory": null,
+ "maxTotalRows": null,
+ "splitHintSpec": null,
+ "partitionsSpec": {
+ "type": "dynamic",
+ "maxRowsPerSegment": 5000000,
+ "maxTotalRows": null
+ },
+ "indexSpec": null,
+ "indexSpecForIntermediatePersists": null,
+ "maxPendingPersists": null,
+ "pushTimeout": null,
+ "segmentWriteOutMediumFactory": null,
+ "maxNumConcurrentSubTasks": null,
+ "maxRetry": null,
+ "taskStatusCheckPeriodMs": null,
+ "chatHandlerTimeout": null,
+ "chatHandlerNumRetries": null,
+ "maxNumSegmentsToMerge": null,
+ "totalNumMergeTasks": null,
+ "maxColumnsToMerge": null,
+ "type": "index_parallel",
+ "forceGuaranteedRollup": false
+ },
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": null,
+ "rollup": null
+ },
+ "dimensionsSpec": null,
+ "metricsSpec": null,
+ "transformSpec": null,
+ "ioConfig": null,
+ "taskContext": null
+ },
+ {
+ "dataSource": "wikipedia",
+ "taskPriority": 25,
+ "inputSegmentSizeBytes": 100000000000000,
+ "maxRowsPerSegment": null,
+ "skipOffsetFromLatest": "PT0S",
+ "tuningConfig": {
+ "maxRowsInMemory": null,
+ "appendableIndexSpec": null,
+ "maxBytesInMemory": null,
+ "maxTotalRows": null,
+ "splitHintSpec": null,
+ "partitionsSpec": {
+ "type": "dynamic",
+ "maxRowsPerSegment": 5000000,
+ "maxTotalRows": null
+ },
+ "indexSpec": null,
+ "indexSpecForIntermediatePersists": null,
+ "maxPendingPersists": null,
+ "pushTimeout": null,
+ "segmentWriteOutMediumFactory": null,
+ "maxNumConcurrentSubTasks": null,
+ "maxRetry": null,
+ "taskStatusCheckPeriodMs": null,
+ "chatHandlerTimeout": null,
+ "chatHandlerNumRetries": null,
+ "maxNumSegmentsToMerge": null,
+ "totalNumMergeTasks": null,
+ "maxColumnsToMerge": null,
+ "type": "index_parallel",
+ "forceGuaranteedRollup": false
+ },
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": null,
+ "rollup": null
+ },
+ "dimensionsSpec": null,
+ "metricsSpec": null,
+ "transformSpec": null,
+ "ioConfig": null,
+ "taskContext": null
+ }
+ ],
+ "compactionTaskSlotRatio": 0.1,
+ "maxCompactionTaskSlots": 2147483647,
+ "useAutoScaleSlots": false
+}
+```
+
+
+### Get automatic compaction configuration
+
+Retrieves the automatic compaction configuration for a datasource.
-Returns the history of the automatic compaction config for a dataSource. Optionally accepts `interval` and `count`
-query string parameters to filter by interval and limit the number of results respectively. If the dataSource does not
-exist or there is no compaction history for the dataSource, an empty list is returned.
+#### URL
+
+GET/druid/coordinator/v1/config/compaction/:dataSource
+
+#### Responses
+
+
+
+
+
+*Successfully retrieved configuration for datasource*
+
+
+
+*Invalid datasource or datasource does not have automatic compaction enabled*
+
+
+
+---
+
+#### Sample request
+
+The following example retrieves the automatic compaction configuration for datasource `wikipedia_hour`.
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/config/compaction/wikipedia_hour"
+```
+
+
+
+```HTTP
+GET /druid/coordinator/v1/config/compaction/wikipedia_hour HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+```json
+{
+ "dataSource": "wikipedia_hour",
+ "taskPriority": 25,
+ "inputSegmentSizeBytes": 100000000000000,
+ "maxRowsPerSegment": null,
+ "skipOffsetFromLatest": "PT0S",
+ "tuningConfig": {
+ "maxRowsInMemory": null,
+ "appendableIndexSpec": null,
+ "maxBytesInMemory": null,
+ "maxTotalRows": null,
+ "splitHintSpec": null,
+ "partitionsSpec": {
+ "type": "dynamic",
+ "maxRowsPerSegment": 5000000,
+ "maxTotalRows": null
+ },
+ "indexSpec": null,
+ "indexSpecForIntermediatePersists": null,
+ "maxPendingPersists": null,
+ "pushTimeout": null,
+ "segmentWriteOutMediumFactory": null,
+ "maxNumConcurrentSubTasks": null,
+ "maxRetry": null,
+ "taskStatusCheckPeriodMs": null,
+ "chatHandlerTimeout": null,
+ "chatHandlerNumRetries": null,
+ "maxNumSegmentsToMerge": null,
+ "totalNumMergeTasks": null,
+ "maxColumnsToMerge": null,
+ "type": "index_parallel",
+ "forceGuaranteedRollup": false
+ },
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": null,
+ "rollup": null
+ },
+ "dimensionsSpec": null,
+ "metricsSpec": null,
+ "transformSpec": null,
+ "ioConfig": null,
+ "taskContext": null
+}
+```
+
+
+### Get automatic compaction configuration history
+
+Retrieves the history of the automatic compaction configuration for a datasource. Returns an empty list if the datasource does not exist or there is no compaction history for the datasource.
The response contains a list of objects with the following keys:
-* `globalConfig`: A json object containing automatic compaction config that applies to the entire cluster.
-* `compactionConfig`: A json object containing the automatic compaction config for the datasource.
-* `auditInfo`: A json object that contains information about the change made - like `author`, `comment` and `ip`.
+* `globalConfig`: A JSON object containing automatic compaction configuration that applies to the entire cluster.
+* `compactionConfig`: A JSON object containing the automatic compaction configuration for the datasource.
+* `auditInfo`: A JSON object containing information about the change made, such as `author`, `comment` or `ip`.
* `auditTime`: The date and time when the change was made.
-`POST /druid/coordinator/v1/config/compaction/taskslots?ratio={someRatio}&max={someMaxSlots}`
+#### URL
+
+GET/druid/coordinator/v1/config/compaction/:dataSource/history
+
+#### Query parameters
+* `interval` (optional)
+ * Type: ISO-8601
+ * Limits the results within a specified interval. Use `/` as the delimiter for the interval string.
+* `count` (optional)
+ * Type: Int
+ * Limits the number of results.
+
+#### Responses
+
+
+
+
+
+*Successfully retrieved configuration history*
+
+
+
+*Invalid `count` value*
+
+
+
+---
+
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/config/compaction/wikipedia_hour/history"
+```
+
+
+
+```HTTP
+GET /druid/coordinator/v1/config/compaction/wikipedia_hour/history HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+```json
+[
+ {
+ "globalConfig": {
+ "compactionTaskSlotRatio": 0.1,
+ "maxCompactionTaskSlots": 2147483647,
+ "useAutoScaleSlots": false
+ },
+ "compactionConfig": {
+ "dataSource": "wikipedia_hour",
+ "taskPriority": 25,
+ "inputSegmentSizeBytes": 100000000000000,
+ "maxRowsPerSegment": null,
+ "skipOffsetFromLatest": "P1D",
+ "tuningConfig": null,
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": null,
+ "rollup": null
+ },
+ "dimensionsSpec": null,
+ "metricsSpec": null,
+ "transformSpec": null,
+ "ioConfig": null,
+ "taskContext": null
+ },
+ "auditInfo": {
+ "author": "",
+ "comment": "",
+ "ip": "127.0.0.1"
+ },
+ "auditTime": "2023-07-31T18:15:19.302Z"
+ },
+ {
+ "globalConfig": {
+ "compactionTaskSlotRatio": 0.1,
+ "maxCompactionTaskSlots": 2147483647,
+ "useAutoScaleSlots": false
+ },
+ "compactionConfig": {
+ "dataSource": "wikipedia_hour",
+ "taskPriority": 25,
+ "inputSegmentSizeBytes": 100000000000000,
+ "maxRowsPerSegment": null,
+ "skipOffsetFromLatest": "PT0S",
+ "tuningConfig": {
+ "maxRowsInMemory": null,
+ "appendableIndexSpec": null,
+ "maxBytesInMemory": null,
+ "maxTotalRows": null,
+ "splitHintSpec": null,
+ "partitionsSpec": {
+ "type": "dynamic",
+ "maxRowsPerSegment": 5000000,
+ "maxTotalRows": null
+ },
+ "indexSpec": null,
+ "indexSpecForIntermediatePersists": null,
+ "maxPendingPersists": null,
+ "pushTimeout": null,
+ "segmentWriteOutMediumFactory": null,
+ "maxNumConcurrentSubTasks": null,
+ "maxRetry": null,
+ "taskStatusCheckPeriodMs": null,
+ "chatHandlerTimeout": null,
+ "chatHandlerNumRetries": null,
+ "maxNumSegmentsToMerge": null,
+ "totalNumMergeTasks": null,
+ "maxColumnsToMerge": null,
+ "type": "index_parallel",
+ "forceGuaranteedRollup": false
+ },
+ "granularitySpec": {
+ "segmentGranularity": "DAY",
+ "queryGranularity": null,
+ "rollup": null
+ },
+ "dimensionsSpec": null,
+ "metricsSpec": null,
+ "transformSpec": null,
+ "ioConfig": null,
+ "taskContext": null
+ },
+ "auditInfo": {
+ "author": "",
+ "comment": "",
+ "ip": "127.0.0.1"
+ },
+ "auditTime": "2023-07-31T18:16:16.362Z"
+ }
+]
+```
+
+
+## View automatic compaction status
+
+### Get segments awaiting compaction
+
+Returns the total size of segments awaiting compaction for a given datasource. Returns a 404 response if a datasource does not have automatic compaction enabled.
+
+#### URL
+
+GET/druid/coordinator/v1/compaction/progress?dataSource=:dataSource
+
+#### Query parameter
+* `dataSource` (required)
+ * Type: String
+ * Name of the datasource for this status information.
+
+#### Responses
+
+
+
+
+
+*Successfully retrieved segment size awaiting compaction*
+
+
+
+*Unknown datasource name or datasource does not have automatic compaction enabled*
+
+
+
+---
+
+#### Sample request
+
+The following example retrieves the remaining segments to be compacted for datasource `wikipedia_hour`.
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/compaction/progress?dataSource=wikipedia_hour"
+```
+
+
+
+```HTTP
+GET /druid/coordinator/v1/compaction/progress?dataSource=wikipedia_hour HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+```json
+{
+ "remainingSegmentSize": 7615837
+}
+```
+
+
+
+### Get compaction status and statistics
+
+Retrieves an array of `latestStatus` objects representing the status and statistics from the latest automatic compaction run for all datasources with automatic compaction enabled.
+
+The `latestStatus` object has the following properties:
+* `dataSource`: Name of the datasource for this status information.
+* `scheduleStatus`: Automatic compaction scheduling status. Possible values are `NOT_ENABLED` and `RUNNING`. Returns `RUNNING ` if the datasource has an active automatic compaction configuration submitted. Otherwise, returns `NOT_ENABLED`.
+* `bytesAwaitingCompaction`: Total bytes of this datasource waiting to be compacted by the automatic compaction (only consider intervals/segments that are eligible for automatic compaction).
+* `bytesCompacted`: Total bytes of this datasource that are already compacted with the spec set in the automatic compaction configuration.
+* `bytesSkipped`: Total bytes of this datasource that are skipped (not eligible for automatic compaction) by the automatic compaction.
+* `segmentCountAwaitingCompaction`: Total number of segments of this datasource waiting to be compacted by the automatic compaction (only consider intervals/segments that are eligible for automatic compaction).
+* `segmentCountCompacted`: Total number of segments of this datasource that are already compacted with the spec set in the automatic compaction configuration.
+* `segmentCountSkipped`: Total number of segments of this datasource that are skipped (not eligible for automatic compaction) by the automatic compaction.
+* `intervalCountAwaitingCompaction`: Total number of intervals of this datasource waiting to be compacted by the automatic compaction (only consider intervals/segments that are eligible for automatic compaction).
+* `intervalCountCompacted`: Total number of intervals of this datasource that are already compacted with the spec set in the automatic compaction configuration.
+* `intervalCountSkipped`: Total number of intervals of this datasource that are skipped (not eligible for automatic compaction) by the automatic compaction.
+
+#### URL
+
+GET/druid/coordinator/v1/compaction/status
+
+#### Query parameters
+* `dataSource` (optional)
+ * Type: String
+ * Filter the result by name of a specific datasource.
+
+#### Responses
+
+
+
+
+
+*Successfully retrieved `latestStatus` object*
+
+
+
+---
+#### Sample request
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/compaction/status"
+```
+
+
-Update the capacity for compaction tasks. `ratio` and `max` are used to limit the max number of compaction tasks.
-They mean the ratio of the total task slots to the compaction task slots and the maximum number of task slots for compaction tasks, respectively. The actual max number of compaction tasks is `min(max, ratio * total task slots)`.
-Note that `ratio` and `max` are optional and can be omitted. If they are omitted, default values (0.1 and unbounded)
-will be set for them.
+```HTTP
+GET /druid/coordinator/v1/compaction/status HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
-`POST /druid/coordinator/v1/config/compaction`
+
-Creates or updates the [automatic compaction](../data-management/automatic-compaction.md) config for a dataSource. See [Automatic compaction dynamic configuration](../configuration/index.md#automatic-compaction-dynamic-configuration) for configuration details.
+#### Sample response
-`DELETE /druid/coordinator/v1/config/compaction/{dataSource}`
+
+ Click to show sample response
-Removes the automatic compaction config for a dataSource.
+```json
+{
+ "latestStatus": [
+ {
+ "dataSource": "wikipedia_api",
+ "scheduleStatus": "RUNNING",
+ "bytesAwaitingCompaction": 0,
+ "bytesCompacted": 0,
+ "bytesSkipped": 64133616,
+ "segmentCountAwaitingCompaction": 0,
+ "segmentCountCompacted": 0,
+ "segmentCountSkipped": 8,
+ "intervalCountAwaitingCompaction": 0,
+ "intervalCountCompacted": 0,
+ "intervalCountSkipped": 1
+ },
+ {
+ "dataSource": "wikipedia_hour",
+ "scheduleStatus": "RUNNING",
+ "bytesAwaitingCompaction": 0,
+ "bytesCompacted": 5998634,
+ "bytesSkipped": 0,
+ "segmentCountAwaitingCompaction": 0,
+ "segmentCountCompacted": 1,
+ "segmentCountSkipped": 0,
+ "intervalCountAwaitingCompaction": 0,
+ "intervalCountCompacted": 1,
+ "intervalCountSkipped": 0
+ }
+ ]
+}
+```
+
diff --git a/docs/api-reference/data-management-api.md b/docs/api-reference/data-management-api.md
index d3fd75ac6980..c41f528cfc65 100644
--- a/docs/api-reference/data-management-api.md
+++ b/docs/api-reference/data-management-api.md
@@ -29,7 +29,9 @@ This document describes the data management API endpoints for Apache Druid. This
While segments may be enabled by issuing POST requests for the datasources, the Coordinator may again disable segments if they match any configured [drop rules](../operations/rule-configuration.md#drop-rules). Even if segments are enabled by these APIs, you must configure a [load rule](../operations/rule-configuration.md#load-rules) to load them onto Historical processes. If an indexing or kill task runs at the same time these APIs are invoked, the behavior is undefined. Some segments might be killed and others might be enabled. It's also possible that all segments might be disabled, but the indexing task can still read data from those segments and succeed.
-> Avoid using indexing or kill tasks and these APIs at the same time for the same datasource and time chunk.
+:::info
+ Avoid using indexing or kill tasks and these APIs at the same time for the same datasource and time chunk.
+:::
`POST /druid/coordinator/v1/datasources/{dataSourceName}`
diff --git a/docs/api-reference/dynamic-configuration-api.md b/docs/api-reference/dynamic-configuration-api.md
index 551c05300b8d..762895abb6cc 100644
--- a/docs/api-reference/dynamic-configuration-api.md
+++ b/docs/api-reference/dynamic-configuration-api.md
@@ -23,7 +23,7 @@ sidebar_label: Dynamic configuration
~ under the License.
-->
-This document describes the API endpoints to retrieve and manage the dynamic configurations for the [Coordinator](../configuration/index.html#overlord-dynamic-configuration) and [Overlord](../configuration/index.html#dynamic-configuration) in Apache Druid.
+This document describes the API endpoints to retrieve and manage the dynamic configurations for the [Coordinator](../configuration/index.md#overlord-dynamic-configuration) and [Overlord](../configuration/index.md#dynamic-configuration) in Apache Druid.
## Coordinator dynamic configuration
diff --git a/docs/api-reference/json-querying-api.md b/docs/api-reference/json-querying-api.md
index 54425dc1b1e1..633a04d1ac6a 100644
--- a/docs/api-reference/json-querying-api.md
+++ b/docs/api-reference/json-querying-api.md
@@ -3,8 +3,12 @@ id: json-querying-api
title: JSON querying API
sidebar_label: JSON querying
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
+
+
+
+
+
-
+*Successfully submitted query*
-*Successfully submitted query*
+
+
-
*Error thrown due to bad query. Returns a JSON object detailing the error with the following format:*
@@ -69,17 +76,19 @@ For additional properties based on your query type or use case, see [available n
```
For more information on possible error messages, see [query execution failures](../querying/querying.md#query-execution-failures).
-
+
+
---
### Example query: `topN`
-The following example shows a `topN` query. The query analyzes the `social_media` datasource to return the top five users from the `username` dimension with the highest number of views from the `views` metric.
+The following example shows a `topN` query. The query analyzes the `social_media` datasource to return the top five users from the `username` dimension with the highest number of views from the `views` metric.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/v2?pretty=null" \
@@ -103,7 +112,9 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/v2?pretty=null" \
]
}'
```
-
+
+
+
```HTTP
POST /druid/v2?pretty=null HTTP/1.1
@@ -131,7 +142,8 @@ Content-Length: 336
}
```
-
+
+
#### Example response: `topN`
@@ -179,9 +191,10 @@ In this query:
* The `upvoteToPostRatio` is a post-aggregation of the `upvoteSum` and the `postCount`, divided to calculate the ratio.
* The result is sorted based on the `upvoteToPostRatio` in descending order.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/v2" \
@@ -217,7 +230,9 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/v2" \
}'
```
-
+
+
+
```HTTP
POST /druid/v2?pretty=null HTTP/1.1
@@ -256,12 +271,14 @@ Content-Length: 817
}
```
-
+
+
#### Example response: `groupBy`
Click to show sample response
+
```json
[
{
@@ -280,7 +297,7 @@ Content-Length: 817
## Get segment information for query
-Retrieves an array that contains objects with segment information, including the server locations associated with the query provided in the request body.
+Retrieves an array that contains objects with segment information, including the server locations associated with the query provided in the request body.
### URL
@@ -292,13 +309,16 @@ Retrieves an array that contains objects with segment information, including the
### Responses
-
+
+
+
+
-
+*Successfully retrieved segment information*
-*Successfully retrieved segment information*
+
+
-
*Error thrown due to bad query. Returns a JSON object detailing the error with the following format:*
@@ -312,15 +332,17 @@ Retrieves an array that contains objects with segment information, including the
```
For more information on possible error messages, see [query execution failures](../querying/querying.md#query-execution-failures).
-
+
+
---
### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/candidates" \
@@ -345,7 +367,9 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/candidates" \
}'
```
-
+
+
+
```HTTP
POST /druid/v2/candidates HTTP/1.1
@@ -374,7 +398,8 @@ Content-Length: 336
}
```
-
+
+
### Sample response
@@ -895,4 +920,4 @@ Content-Length: 336
}
]
```
-
\ No newline at end of file
+
diff --git a/docs/api-reference/legacy-metadata-api.md b/docs/api-reference/legacy-metadata-api.md
index 609a7767f935..fe031c7348aa 100644
--- a/docs/api-reference/legacy-metadata-api.md
+++ b/docs/api-reference/legacy-metadata-api.md
@@ -99,8 +99,10 @@ If no used segments are found for the given inputs, this API returns `204 No Con
## Metadata store information
-> Note: Much of this information is available in a simpler, easier-to-use form through the Druid SQL
-> [`sys.segments`](../querying/sql-metadata-tables.md#segments-table) table.
+:::info
+ Note: Much of this information is available in a simpler, easier-to-use form through the Druid SQL
+ [`sys.segments`](../querying/sql-metadata-tables.md#segments-table) table.
+:::
`GET /druid/coordinator/v1/metadata/segments`
@@ -151,6 +153,11 @@ Returns a list of all segments for a datasource with the full segment metadata a
Returns full segment metadata for a specific segment as stored in the metadata store, if the segment is used. If the
segment is unused, or is unknown, a 404 response is returned.
+`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments/{segmentId}?includeUnused=true`
+
+Returns full segment metadata for a specific segment as stored in the metadata store. If the is unknown, a 404 response
+is returned.
+
`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments`
Returns a list of all segments, overlapping with any of given intervals, for a datasource as stored in the metadata store. Request body is array of string IS0 8601 intervals like `[interval1, interval2,...]`—for example, `["2012-01-01T00:00:00.000/2012-01-03T00:00:00.000", "2012-01-05T00:00:00.000/2012-01-07T00:00:00.000"]`.
@@ -279,10 +286,12 @@ This section documents the API endpoints for the processes that reside on Query
Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/`
as in `2016-06-27_2016-06-28`.
-> Note: Much of this information is available in a simpler, easier-to-use form through the Druid SQL
-> [`INFORMATION_SCHEMA.TABLES`](../querying/sql-metadata-tables.md#tables-table),
-> [`INFORMATION_SCHEMA.COLUMNS`](../querying/sql-metadata-tables.md#columns-table), and
-> [`sys.segments`](../querying/sql-metadata-tables.md#segments-table) tables.
+:::info
+ Note: Much of this information is available in a simpler, easier-to-use form through the Druid SQL
+ [`INFORMATION_SCHEMA.TABLES`](../querying/sql-metadata-tables.md#tables-table),
+ [`INFORMATION_SCHEMA.COLUMNS`](../querying/sql-metadata-tables.md#columns-table), and
+ [`sys.segments`](../querying/sql-metadata-tables.md#segments-table) tables.
+:::
`GET /druid/v2/datasources`
@@ -296,17 +305,21 @@ If no interval is specified, a default interval spanning a configurable period b
`GET /druid/v2/datasources/{dataSourceName}/dimensions`
-> This API is deprecated and will be removed in future releases. Please use [SegmentMetadataQuery](../querying/segmentmetadataquery.md) instead
-> which provides more comprehensive information and supports all dataSource types including streaming dataSources. It's also encouraged to use [INFORMATION_SCHEMA tables](../querying/sql-metadata-tables.md)
-> if you're using SQL.
->
+:::info
+ This API is deprecated and will be removed in future releases. Please use [SegmentMetadataQuery](../querying/segmentmetadataquery.md) instead
+ which provides more comprehensive information and supports all dataSource types including streaming dataSources. It's also encouraged to use [INFORMATION_SCHEMA tables](../querying/sql-metadata-tables.md)
+ if you're using SQL.
+:::
+
Returns the dimensions of the datasource.
`GET /druid/v2/datasources/{dataSourceName}/metrics`
-> This API is deprecated and will be removed in future releases. Please use [SegmentMetadataQuery](../querying/segmentmetadataquery.md) instead
-> which provides more comprehensive information and supports all dataSource types including streaming dataSources. It's also encouraged to use [INFORMATION_SCHEMA tables](../querying/sql-metadata-tables.md)
-> if you're using SQL.
+:::info
+ This API is deprecated and will be removed in future releases. Please use [SegmentMetadataQuery](../querying/segmentmetadataquery.md) instead
+ which provides more comprehensive information and supports all dataSource types including streaming dataSources. It's also encouraged to use [INFORMATION_SCHEMA tables](../querying/sql-metadata-tables.md)
+ if you're using SQL.
+:::
Returns the metrics of the datasource.
diff --git a/docs/api-reference/retention-rules-api.md b/docs/api-reference/retention-rules-api.md
index 42d5b8ecb292..0f41eba3e6d0 100644
--- a/docs/api-reference/retention-rules-api.md
+++ b/docs/api-reference/retention-rules-api.md
@@ -3,8 +3,12 @@ id: retention-rules-api
title: Retention rules API
sidebar_label: Retention rules
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
+
-This document describes the API endpoints for managing retention rules in Apache Druid.
+This topic describes the API endpoints for managing retention rules in Apache Druid. You can configure retention rules in the Druid web console or API.
+
+Druid uses retention rules to determine what data is retained in the cluster. Druid supports load, drop, and broadcast rules. For more information, see [Using rules to drop and retain data](../operations/rule-configuration.md).
+
+In this topic, `http://ROUTER_IP:ROUTER_PORT` is a placeholder for your Router service address and port. Replace it with the information for your deployment. For example, use `http://localhost:8888` for quickstart deployments.
+
+## Update retention rules for a datasource
+
+Updates one or more retention rules for a datasource. The request body takes an array of retention rule objects. For details on defining retention rules, see the following sources:
+
+* [Load rules](../operations/rule-configuration.md#load-rules)
+* [Drop rules](../operations/rule-configuration.md#drop-rules)
+* [Broadcast rules](../operations/rule-configuration.md#broadcast-rules)
+
+This request overwrites any existing rules for the datasource.
+Druid reads rules in the order in which they appear; for more information, see [rule structure](../operations/rule-configuration.md).
+
+Note that this endpoint returns an HTTP `200 OK` even if the datasource does not exist.
+
+### URL
+
+POST/druid/coordinator/v1/rules/:dataSource
+
+### Header parameters
+
+The endpoint supports a set of optional header parameters to populate the `author` and `comment` fields in the `auditInfo` property for audit history.
+
+* `X-Druid-Author` (optional)
+ * Type: String
+ * A string representing the author making the configuration change.
+* `X-Druid-Comment` (optional)
+ * Type: String
+ * A string describing the update.
+
+### Responses
+
+
+
+
+
+
+*Successfully updated retention rules for specified datasource*
+
+
+
+
+---
+
+### Sample request
+
+The following example sets a set of broadcast, load, and drop retention rules for the `kttm1` datasource.
+
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/rules/kttm1" \
+--header 'X-Druid-Author: doc intern' \
+--header 'X-Druid-Comment: submitted via api' \
+--header 'Content-Type: application/json' \
+--data '[
+ {
+ "type": "broadcastForever"
+ },
+ {
+ "type": "loadForever",
+ "tieredReplicants": {
+ "_default_tier": 2
+ },
+ "useDefaultTierForNull": true
+ },
+ {
+ "type": "dropByPeriod",
+ "period": "P1M"
+ }
+]'
+```
+
+
+
+
+
+```HTTP
+POST /druid/coordinator/v1/rules/kttm1 HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+X-Druid-Author: doc intern
+X-Druid-Comment: submitted via api
+Content-Type: application/json
+Content-Length: 273
+
+[
+ {
+ "type": "broadcastForever"
+ },
+ {
+ "type": "loadForever",
+ "tieredReplicants": {
+ "_default_tier": 1
+ },
+ "useDefaultTierForNull": true
+ },
+ {
+ "type": "dropByPeriod",
+ "period": "P1M"
+ }
+]
+```
+
+
+
+
+### Sample response
+
+A successful request returns an HTTP `200 OK` message code and an empty response body.
+
+## Update default retention rules for all datasources
+
+Updates one or more default retention rules for all datasources. Submit retention rules as an array of objects in the request body. For details on defining retention rules, see the following sources:
+
+* [Load rules](../operations/rule-configuration.md#load-rules)
+* [Drop rules](../operations/rule-configuration.md#drop-rules)
+* [Broadcast rules](../operations/rule-configuration.md#broadcast-rules)
+
+This request overwrites any existing rules for all datasources. To remove default retention rules for all datasources, submit an empty rule array in the request body. Rules are read in the order in which they appear; for more information, see [rule structure](../operations/rule-configuration.md).
+
+### URL
+
+POST/druid/coordinator/v1/rules/_default
+
+### Header parameters
+
+The endpoint supports a set of optional header parameters to populate the `author` and `comment` fields in the `auditInfo` property for audit history.
+
+* `X-Druid-Author` (optional)
+ * Type: String
+ * A string representing the author making the configuration change.
+* `X-Druid-Comment` (optional)
+ * Type: String
+ * A string describing the update.
+
+### Responses
+
+
+
+
+
+
+*Successfully updated default retention rules*
+
+
+
+
+
+*Error with request body*
+
+
+
+
+---
+
+### Sample request
+
+The following example updates the default retention rule for all datasources with a `loadByInterval` rule.
+
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/rules/_default" \
+--header 'Content-Type: application/json' \
+--data '[
+ {
+ "type": "loadByInterval",
+ "tieredReplicants": {},
+ "useDefaultTierForNull": false,
+ "interval": "2010-01-01/2020-01-01"
+ }
+]'
+```
+
+
+
+
+
+```HTTP
+POST /druid/coordinator/v1/rules/_default HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+Content-Type: application/json
+Content-Length: 205
+
+[
+ {
+ "type": "loadByInterval",
+ "tieredReplicants": {},
+ "useDefaultTierForNull": false,
+ "interval": "2010-01-01/2020-01-01"
+ }
+]
+```
+
+
+
+
+### Sample response
+
+A successful request returns an HTTP `200 OK` message code and an empty response body.
-## Retention rules
+## Get an array of all retention rules
-Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` as in `2016-06-27_2016-06-28`.
+Retrieves all current retention rules in the cluster including the default retention rule. Returns an array of objects for each datasource and their associated retention rules.
-`GET /druid/coordinator/v1/rules`
+### URL
-Returns all rules as JSON objects for all datasources in the cluster including the default datasource.
+GET/druid/coordinator/v1/rules
-`GET /druid/coordinator/v1/rules/{dataSourceName}`
+### Responses
-Returns all rules for a specified datasource.
+
-`GET /druid/coordinator/v1/rules/{dataSourceName}?full`
+
+
+
+*Successfully retrieved retention rules*
+
+
+
+
+---
+
+### Sample request
+
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/rules"
+```
+
+
+
+
+
+```HTTP
+GET /druid/coordinator/v1/rules HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+
+### Sample response
+
+
+ Click to show sample response
+
+ ```json
+{
+ "_default": [
+ {
+ "tieredReplicants": {
+ "_default_tier": 2
+ },
+ "type": "loadForever"
+ }
+ ],
+ "social_media": [
+ {
+ "interval": "2023-01-01T00:00:00.000Z/2023-02-01T00:00:00.000Z",
+ "type": "dropByInterval"
+ }
+ ],
+ "wikipedia_api": [],
+}
+ ```
+
+
+## Get an array of retention rules for a datasource
+
+Retrieves an array of rule objects for a single datasource. Returns an empty array if there are no retention rules.
+
+Note that this endpoint returns an HTTP `200 OK` message code even if the datasource does not exist.
+
+### URL
+
+GET/druid/coordinator/v1/rules/:dataSource
+
+### Query parameters
+
+* `full` (optional)
+ * Includes the default retention rule for the datasource in the response.
+
+### Responses
+
+
+
+
+
+
+*Successfully retrieved retention rules*
+
+
+
+
+---
+
+### Sample request
+
+The following example retrieves the custom retention rules and default retention rules for datasource with the name `social_media`.
+
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/rules/social_media?full=null"
+```
+
+
+
+
+
+```HTTP
+GET /druid/coordinator/v1/rules/social_media?full=null HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+
+### Sample response
+
+
+ Click to show sample response
+
+ ```json
+[
+ {
+ "interval": "2020-01-01T00:00:00.000Z/2022-02-01T00:00:00.000Z",
+ "type": "dropByInterval"
+ },
+ {
+ "interval": "2010-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z",
+ "tieredReplicants": {
+ "_default_tier": 2
+ },
+ "type": "loadByInterval"
+ },
+ {
+ "tieredReplicants": {
+ "_default_tier": 2
+ },
+ "type": "loadForever"
+ }
+]
+ ```
+
+
+
+## Get audit history for all datasources
+
+Retrieves the audit history of rules for all datasources over an interval of time. The default interval is 1 week. You can change this period by setting `druid.audit.manager.auditHistoryMillis` in the `runtime.properties` file for the Coordinator.
+
+### URL
+
+GET/druid/coordinator/v1/rules/history
+
+### Query parameters
+
+Note that the following query parameters cannot be chained.
+
+* `interval` (optional)
+ * Type: ISO 8601.
+ * Limits the number of results to the specified time interval. Delimit with `/`. For example, `2023-07-13/2023-07-19`.
+* `count` (optional)
+ * Type: Int
+ * Limits the number of results to the last `n` entries.
+
+### Responses
+
+
+
+
+
+
+*Successfully retrieved audit history*
+
+
+
+
+
+*Request in the incorrect format*
+
+
+
+
+
+*`count` query parameter too large*
+
+
+
+
+---
-Returns all rules for a specified datasource and includes default datasource.
+### Sample request
-`GET /druid/coordinator/v1/rules/history?interval=`
+The following example retrieves the audit history for all datasources from `2023-07-13` to `2023-07-19`.
-Returns audit history of rules for all datasources. Default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in Coordinator `runtime.properties`.
+
-`GET /druid/coordinator/v1/rules/history?count=`
+
-Returns last `n` entries of audit history of rules for all datasources.
-`GET /druid/coordinator/v1/rules/{dataSourceName}/history?interval=`
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/rules/history?interval=2023-07-13%2F2023-07-19"
+```
-Returns audit history of rules for a specified datasource. Default value of interval can be specified by setting `druid.audit.manager.auditHistoryMillis` (1 week if not configured) in Coordinator `runtime.properties`.
+
+
-`GET /druid/coordinator/v1/rules/{dataSourceName}/history?count=`
-Returns last `n` entries of audit history of rules for a specified datasource.
+```HTTP
+GET /druid/coordinator/v1/rules/history?interval=2023-07-13/2023-07-19 HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
-`POST /druid/coordinator/v1/rules/{dataSourceName}`
+
+
-POST with a list of rules in JSON form to update rules.
+### Sample response
-Optional Header Parameters for auditing the config change can also be specified.
+
+ Click to show sample response
-|Header Param Name| Description | Default |
-|----------|-------------|---------|
-|`X-Druid-Author`| Author making the config change|`""`|
-|`X-Druid-Comment`| Comment describing the change being done|`""`|
+ ```json
+[
+ {
+ "key": "social_media",
+ "type": "rules",
+ "auditInfo": {
+ "author": "console",
+ "comment": "test",
+ "ip": "127.0.0.1"
+ },
+ "payload": "[{\"interval\":\"2023-01-01T00:00:00.000Z/2023-02-01T00:00:00.000Z\",\"type\":\"dropByInterval\"}]",
+ "auditTime": "2023-07-13T18:05:33.066Z"
+ },
+ {
+ "key": "social_media",
+ "type": "rules",
+ "auditInfo": {
+ "author": "console",
+ "comment": "test",
+ "ip": "127.0.0.1"
+ },
+ "payload": "[]",
+ "auditTime": "2023-07-18T18:10:21.203Z"
+ },
+ {
+ "key": "wikipedia_api",
+ "type": "rules",
+ "auditInfo": {
+ "author": "console",
+ "comment": "test",
+ "ip": "127.0.0.1"
+ },
+ "payload": "[{\"tieredReplicants\":{\"_default_tier\":2},\"type\":\"loadForever\"}]",
+ "auditTime": "2023-07-18T18:10:44.519Z"
+ },
+ {
+ "key": "wikipedia_api",
+ "type": "rules",
+ "auditInfo": {
+ "author": "console",
+ "comment": "test",
+ "ip": "127.0.0.1"
+ },
+ "payload": "[]",
+ "auditTime": "2023-07-18T18:11:02.110Z"
+ },
+ {
+ "key": "social_media",
+ "type": "rules",
+ "auditInfo": {
+ "author": "console",
+ "comment": "test",
+ "ip": "127.0.0.1"
+ },
+ "payload": "[{\"interval\":\"2023-07-03T18:49:54.848Z/2023-07-03T18:49:55.861Z\",\"type\":\"dropByInterval\"}]",
+ "auditTime": "2023-07-18T18:32:50.060Z"
+ },
+ {
+ "key": "social_media",
+ "type": "rules",
+ "auditInfo": {
+ "author": "console",
+ "comment": "test",
+ "ip": "127.0.0.1"
+ },
+ "payload": "[{\"interval\":\"2020-01-01T00:00:00.000Z/2022-02-01T00:00:00.000Z\",\"type\":\"dropByInterval\"}]",
+ "auditTime": "2023-07-18T18:34:09.657Z"
+ },
+ {
+ "key": "social_media",
+ "type": "rules",
+ "auditInfo": {
+ "author": "console",
+ "comment": "test",
+ "ip": "127.0.0.1"
+ },
+ "payload": "[{\"interval\":\"2020-01-01T00:00:00.000Z/2022-02-01T00:00:00.000Z\",\"type\":\"dropByInterval\"},{\"tieredReplicants\":{\"_default_tier\":2},\"type\":\"loadForever\"}]",
+ "auditTime": "2023-07-18T18:38:37.223Z"
+ },
+ {
+ "key": "social_media",
+ "type": "rules",
+ "auditInfo": {
+ "author": "console",
+ "comment": "test",
+ "ip": "127.0.0.1"
+ },
+ "payload": "[{\"interval\":\"2020-01-01T00:00:00.000Z/2022-02-01T00:00:00.000Z\",\"type\":\"dropByInterval\"},{\"interval\":\"2010-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z\",\"tieredReplicants\":{\"_default_tier\":2},\"type\":\"loadByInterval\"}]",
+ "auditTime": "2023-07-18T18:49:43.964Z"
+ }
+]
+ ```
+
diff --git a/docs/api-reference/service-status-api.md b/docs/api-reference/service-status-api.md
index 88059b1450a3..0d2da88932ff 100644
--- a/docs/api-reference/service-status-api.md
+++ b/docs/api-reference/service-status-api.md
@@ -3,8 +3,12 @@ id: service-status-api
title: Service status API
sidebar_label: Service status
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
+
+
+
+
-
-*Successfully retrieved service information*
+*Successfully retrieved service information*
-
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/status"
```
-
+
+
+
```http
GET /status HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
{
"version": "26.0.0",
@@ -193,7 +203,7 @@ Retrieves the online status of the individual Druid service. It is a simple heal
Modify the host and port for the endpoint to match the service to query. Refer to the [default service ports](#common) for the port numbers.
-Additional checks for readiness should use the [Historical segment readiness](#get-segment-readiness) and [Broker query readiness](#get-broker-query-readiness) endpoints.
+Additional checks for readiness should use the [Historical segment readiness](#get-segment-readiness) and [Broker query readiness](#get-broker-query-readiness) endpoints.
#### URL
@@ -201,34 +211,40 @@ Additional checks for readiness should use the [Historical segment readiness](#g
#### Responses
-
+
+
+
-
-*Successfully retrieved service health*
+*Successfully retrieved service health*
-
+
+
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/status/health"
```
-
+
+
+
```http
GET /status/health HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -244,7 +260,7 @@ Host: http://ROUTER_IP:ROUTER_PORT
### Get configuration properties
-Retrieves the current configuration properties of the individual service queried.
+Retrieves the current configuration properties of the individual service queried.
Modify the host and port for the endpoint to match the service to query. Refer to the [default service ports](#common) for the port numbers.
@@ -254,34 +270,40 @@ Modify the host and port for the endpoint to match the service to query. Refer t
#### Responses
-
+
+
+
-
-*Successfully retrieved service configuration properties*
+*Successfully retrieved service configuration properties*
-
+
+
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/status/properties"
```
-
+
+
+
```http
GET /status/properties HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -390,7 +412,7 @@ Host: http://ROUTER_IP:ROUTER_PORT
### Get node discovery status and cluster integration confirmation
-Retrieves a JSON map of the form `{"selfDiscovered": true/false}`, indicating whether the node has received a confirmation from the central node discovery mechanism (currently ZooKeeper) of the Druid cluster that the node has been added to the cluster.
+Retrieves a JSON map of the form `{"selfDiscovered": true/false}`, indicating whether the node has received a confirmation from the central node discovery mechanism (currently ZooKeeper) of the Druid cluster that the node has been added to the cluster.
Only consider a Druid node "healthy" or "ready" in automated deployment/container management systems when this endpoint returns `{"selfDiscovered": true}`. Nodes experiencing network issues may become isolated and are not healthy.
For nodes that use Zookeeper segment discovery, a response of `{"selfDiscovered": true}` indicates that the node's Zookeeper client has started receiving data from the Zookeeper cluster, enabling timely discovery of segments and other nodes.
@@ -401,34 +423,40 @@ For nodes that use Zookeeper segment discovery, a response of `{"selfDiscovered"
#### Responses
-
+
+
+
-
-*Node was successfully added to the cluster*
+*Node was successfully added to the cluster*
-
+
+
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/status/selfDiscovered/status"
```
-
+
+
+
```http
GET /status/selfDiscovered/status HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -455,40 +483,48 @@ Use this endpoint for monitoring checks that are unable to examine the response
#### Responses
-
+
+
+
-
-*Successfully retrieved node status*
+*Successfully retrieved node status*
+
+
+
-
-*Unsuccessful node self-discovery*
+*Unsuccessful node self-discovery*
-
+
+
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/status/selfDiscovered"
```
-
+
+
+
```http
GET /status/selfDiscovered HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -498,7 +534,7 @@ A successful response to this endpoint results in an empty response body.
### Get Coordinator leader address
-Retrieves the address of the current leader Coordinator of the cluster. If any request is sent to a non-leader Coordinator, the request is automatically redirected to the leader Coordinator.
+Retrieves the address of the current leader Coordinator of the cluster. If any request is sent to a non-leader Coordinator, the request is automatically redirected to the leader Coordinator.
#### URL
@@ -506,36 +542,42 @@ Retrieves the address of the current leader Coordinator of the cluster. If any r
#### Responses
-
+
+
+
-
-*Successfully retrieved leader Coordinator address*
+*Successfully retrieved leader Coordinator address*
-
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/leader"
```
-
+
+
+
```http
GET /druid/coordinator/v1/leader HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -560,42 +602,50 @@ Use this endpoint as a load balancer status check when you only want the active
#### Responses
-
+
+
+
-
-*Current server is the leader*
+*Current server is the leader*
+
+
+
-
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://COORDINATOR_IP:COORDINATOR_PORT/druid/coordinator/v1/isLeader"
```
-
+
+
+
```http
GET /druid/coordinator/v1/isLeader HTTP/1.1
Host: http://COORDINATOR_IP:COORDINATOR_PORT
```
-
+
+
#### Sample response
@@ -622,36 +672,42 @@ Retrieves the address of the current leader Overlord of the cluster. In a cluste
#### Responses
-
+
+
+
-
-*Successfully retrieved leader Overlord address*
+*Successfully retrieved leader Overlord address*
-
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/leader"
```
-
+
+
+
```http
GET /druid/indexer/v1/leader HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -677,42 +733,50 @@ Use this endpoint as a load balancer status check when you only want the active
#### Responses
-
+
+
+
-
-*Current server is the leader*
+*Current server is the leader*
+
+
+
-
-*Current server is not the leader*
+*Current server is not the leader*
-
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://OVERLORD_IP:OVERLORD_PORT/druid/indexer/v1/isLeader"
```
-
+
+
+
```http
GET /druid/indexer/v1/isLeader HTTP/1.1
Host: http://OVERLORD_IP:OVERLORD_PORT
```
-
+
+
#### Sample response
@@ -740,42 +804,48 @@ Retrieves the enabled state of the MiddleManager. Returns JSON object keyed by t
#### Responses
-
+
+
+
-
-*Successfully retrieved MiddleManager state*
+*Successfully retrieved MiddleManager state*
-
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT/druid/worker/v1/enabled"
```
-
+
+
+
```http
GET /druid/worker/v1/enabled HTTP/1.1
Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
{
"localhost:8091": true
@@ -784,52 +854,58 @@ Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
-### Get active tasks
+### Get active tasks
Retrieves a list of active tasks being run on MiddleManager. Returns JSON list of task ID strings. Note that for normal usage, you should use the `/druid/indexer/v1/tasks` [Tasks API](./tasks-api.md) endpoint or one of the task state specific variants instead.
-#### URL
+#### URL
GET/druid/worker/v1/tasks
#### Responses
-
+
+
+
-
-*Successfully retrieved active tasks*
+*Successfully retrieved active tasks*
-
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT/druid/worker/v1/tasks"
```
-
+
+
+
```http
GET /druid/worker/v1/tasks HTTP/1.1
Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
[
"index_parallel_wikipedia_mgchefio_2023-06-13T22:18:05.360Z"
@@ -857,15 +933,17 @@ Shuts down a running task by ID. For normal usage, you should use the `/druid/in
POST/druid/worker/v1/task/:taskId/shutdown
#### Responses
-
+
+
+
-
-*Successfully shut down a task*
+*Successfully shut down a task*
-
+
+
---
@@ -873,28 +951,32 @@ Shuts down a running task by ID. For normal usage, you should use the `/druid/in
The following example shuts down a task with specified ID `index_kafka_wikiticker_f7011f8ffba384b_fpeclode`.
-
+
+
+
-
```shell
curl "http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT/druid/worker/v1/task/index_kafka_wikiticker_f7011f8ffba384b_fpeclode/shutdown"
```
-
+
+
+
```http
POST /druid/worker/v1/task/index_kafka_wikiticker_f7011f8ffba384b_fpeclode/shutdown HTTP/1.1
Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
{
"task":"index_kafka_wikiticker_f7011f8ffba384b_fpeclode"
@@ -914,40 +996,46 @@ keyed by the combined `druid.host` and `druid.port`.
#### Responses
-
+
+
+
-
-*Successfully disabled MiddleManager*
+*Successfully disabled MiddleManager*
-
+
+
#### Sample request
-
+
+
+
-
```shell
curl "http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT/druid/worker/v1/disable"
```
-
+
+
+
```http
POST /druid/worker/v1/disable HTTP/1.1
Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
{
"localhost:8091":"disabled"
@@ -967,40 +1055,46 @@ keyed by the combined `druid.host` and `druid.port`.
#### Responses
-
+
+
+
-
-*Successfully enabled MiddleManager*
+*Successfully enabled MiddleManager*
-
+
+
#### Sample request
-
+
+
+
-
```shell
curl "http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT/druid/worker/v1/enable"
```
-
+
+
+
```http
POST /druid/worker/v1/enable HTTP/1.1
Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
{
"localhost:8091":"enabled"
@@ -1009,11 +1103,11 @@ Host: http://MIDDLEMANAGER_IP:MIDDLEMANAGER_PORT
-## Historical
+## Historical
### Get segment load status
-Retrieves a JSON object of the form `{"cacheInitialized":value}`, where value is either `true` or `false` indicating if all segments in the local cache have been loaded.
+Retrieves a JSON object of the form `{"cacheInitialized":value}`, where value is either `true` or `false` indicating if all segments in the local cache have been loaded.
Use this endpoint to know when a Broker service is ready to accept queries after a restart.
@@ -1023,40 +1117,46 @@ Use this endpoint to know when a Broker service is ready to accept queries after
#### Responses
-
+
+
+
-
-*Successfully retrieved status*
+*Successfully retrieved status*
-
+
+
#### Sample request
-
+
+
+
-
```shell
curl "http://HISTORICAL_IP:HISTORICAL_PORT/druid/historical/v1/loadstatus"
```
-
+
+
+
```http
GET /druid/historical/v1/loadstatus HTTP/1.1
Host: http://HISTORICAL_IP:HISTORICAL_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
{
"cacheInitialized": true
@@ -1075,40 +1175,48 @@ Retrieves a status code to indicate if all segments in the local cache have been
#### Responses
-
+
+
+
-
-*Segments in local cache successfully loaded*
+*Segments in local cache successfully loaded*
+
+
+
-
-*Segments in local cache have not been loaded*
+*Segments in local cache have not been loaded*
-
+
+
#### Sample request
-
+
+
+
-
```shell
curl "http://HISTORICAL_IP:HISTORICAL_PORT/druid/historical/v1/readiness"
```
-
+
+
+
```http
GET /druid/historical/v1/readiness HTTP/1.1
Host: http://HISTORICAL_IP:HISTORICAL_PORT
```
-
+
+
#### Sample response
@@ -1126,34 +1234,40 @@ Retrieves a flag indicating if the Broker knows about all segments in the cluste
#### Responses
-
+
+
+
-
-*Segments successfully loaded*
+*Segments successfully loaded*
-
+
+
#### Sample request
-
+
+
+
-
```shell
curl "http://BROKER_IP:BROKER_PORT/druid/broker/v1/loadstatus"
```
-
+
+
+
```http
GET /druid/broker/v1/loadstatus HTTP/1.1
Host: http://:
```
-
+
+
#### Sample response
@@ -1178,40 +1292,48 @@ Retrieves a status code to indicate Broker readiness. Readiness signifies the Br
#### Responses
-
+
+
+
-
-*Segments successfully loaded*
+*Segments successfully loaded*
+
+
+
-
-*Segments have not been loaded*
+*Segments have not been loaded*
-
+
+
#### Sample request
-
+
+
+
-
```shell
curl "http://BROKER_IP:BROKER_PORT/druid/broker/v1/readiness"
```
-
+
+
+
```http
GET /druid/broker/v1/readiness HTTP/1.1
Host: http://BROKER_IP:BROKER_PORT
```
-
+
+
#### Sample response
diff --git a/docs/api-reference/sql-api.md b/docs/api-reference/sql-api.md
index aaaf499851d5..a800f4c8afdf 100644
--- a/docs/api-reference/sql-api.md
+++ b/docs/api-reference/sql-api.md
@@ -3,8 +3,12 @@ id: sql-api
title: Druid SQL API
sidebar_label: Druid SQL
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
-> Apache Druid supports two query languages: Druid SQL and [native queries](../querying/querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](../querying/querying.md).
+ This document describes the SQL language.
+:::
+
+In this topic, `http://ROUTER_IP:ROUTER_PORT` is a placeholder for your Router service address and port. Replace it with the information for your deployment. For example, use `http://localhost:8888` for quickstart deployments.
+
+## Query from Historicals
+
+### Submit a query
+
+Submits a SQL-based query in the JSON request body. Returns a JSON object with the query results and optional metadata for the results. You can also use this endpoint to query [metadata tables](../querying/sql-metadata-tables.md).
+
+Each query has an associated SQL query ID. You can set this ID manually using the SQL context parameter `sqlQueryId`. If not set, Druid automatically generates `sqlQueryId` and returns it in the response header for `X-Druid-SQL-Query-Id`. Note that you need the `sqlQueryId` to [cancel a query](#cancel-a-query) endpoint.
+
+#### URL
+
+POST/druid/v2/sql
+
+#### Request body
+
+The request body takes the following properties:
+
+* `query`: SQL query string.
+* `resultFormat`: String that indicates the format to return query results. Select one of the following formats:
+ * `object`: Returns a JSON array of JSON objects with the HTTP header `Content-Type: application/json`.
+ * `array`: Returns a JSON array of JSON arrays with the HTTP header `Content-Type: application/json`.
+ * `objectLines`: Returns newline-delimited JSON objects with a trailing blank line. Returns the HTTP header `Content-Type: text/plain`.
+ * `arrayLines`: Returns newline-delimited JSON arrays with a trailing blank line. Returns the HTTP header `Content-Type: text/plain`.
+ * `csv`: Returns a comma-separated values with one row per line and a trailing blank line. Returns the HTTP header `Content-Type: text/csv`.
+* `header`: Boolean value that determines whether to return information on column names. When set to `true`, Druid returns the column names as the first row of the results. To also get information on the column types, set `typesHeader` or `sqlTypesHeader` to `true`. For a comparative overview of data formats and configurations for the header, see the [Query output format](#query-output-format) table.
+* `typesHeader`: Adds Druid runtime type information in the header. Requires `header` to be set to `true`. Complex types, like sketches, will be reported as `COMPLEX` if a particular complex type name is known for that field, or as `COMPLEX` if the particular type name is unknown or mixed.
+* `sqlTypesHeader`: Adds SQL type information in the header. Requires `header` to be set to `true`.
+* `context`: JSON object containing optional [SQL query context parameters](../querying/sql-query-context.md), such as to set the query ID, time zone, and whether to use an approximation algorithm for distinct count.
+* `parameters`: List of query parameters for parameterized queries. Each parameter in the array should be a JSON object containing the parameter's SQL data type and parameter value. For a list of supported SQL types, see [Data types](../querying/sql-data-types.md).
+
+ For example:
+ ```
+ "parameters": [
+ {
+ "type": "VARCHAR",
+ "value": "bar"
+ }
+ ]
+ ```
+
+#### Responses
-You can submit and cancel [Druid SQL](../querying/sql.md) queries using the Druid SQL API.
-The Druid SQL API is available at `https://ROUTER:8888/druid/v2/sql`, where `ROUTER` is the IP address of the Druid Router.
+
-## Submit a query
+
-To use the SQL API to make Druid SQL queries, send your query to the Router using the POST method:
+
+*Successfully submitted query*
+
+
+
+
+
+*Error thrown due to bad query. Returns a JSON object detailing the error with the following format:*
+
+```json
+{
+ "error": "A well-defined error code.",
+ "errorMessage": "A message with additional details about the error.",
+ "errorClass": "Class of exception that caused this error.",
+ "host": "The host on which the error occurred."
+}
```
-POST https://ROUTER:8888/druid/v2/sql/
-```
+
+
+
+
+*Request not sent due to unexpected conditions. Returns a JSON object detailing the error with the following format:*
-Submit your query as the value of a "query" field in the JSON object within the request payload. For example:
```json
-{"query" : "SELECT COUNT(*) FROM data_source WHERE foo = 'bar'"}
+{
+ "error": "A well-defined error code.",
+ "errorMessage": "A message with additional details about the error.",
+ "errorClass": "Class of exception that caused this error.",
+ "host": "The host on which the error occurred."
+}
```
-### Request body
-
-|Property|Description|Default|
-|--------|----|-----------|
-|`query`|SQL query string.| none (required)|
-|`resultFormat`|Format of query results. See [Responses](#responses) for details.|`"object"`|
-|`header`|Whether or not to include a header row for the query result. See [Responses](#responses) for details.|`false`|
-|`typesHeader`|Whether or not to include type information in the header. Can only be set when `header` is also `true`. See [Responses](#responses) for details.|`false`|
-|`sqlTypesHeader`|Whether or not to include SQL type information in the header. Can only be set when `header` is also `true`. See [Responses](#responses) for details.|`false`|
-|`context`|JSON object containing [SQL query context parameters](../querying/sql-query-context.md).|`{}` (empty)|
-|`parameters`|List of query parameters for parameterized queries. Each parameter in the list should be a JSON object like `{"type": "VARCHAR", "value": "foo"}`. The type should be a SQL type; see [Data types](../querying/sql-data-types.md) for a list of supported SQL types.|`[]` (empty)|
+
+
+
+Older versions of Druid that support the `typesHeader` and `sqlTypesHeader` parameters return the HTTP header `X-Druid-SQL-Header-Included: yes` when you set `header` to `true`. Druid returns the HTTP response header for compatibility, regardless of whether `typesHeader` and `sqlTypesHeader` are set.
-You can use _curl_ to send SQL queries from the command-line:
+---
-```bash
-$ cat query.json
-{"query":"SELECT COUNT(*) AS TheCount FROM data_source"}
-$ curl -XPOST -H'Content-Type: application/json' http://ROUTER:8888/druid/v2/sql/ -d @query.json
-[{"TheCount":24433}]
+#### Sample request
+
+The following example retrieves all rows in the `wikipedia` datasource where the `user` is `BlueMoon2662`. The query is assigned the ID `request01` using the `sqlQueryId` context parameter. The optional properties `header`, `typesHeader`, and `sqlTypesHeader` are set to `true` to include type information to the response.
+
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql" \
+--header 'Content-Type: application/json' \
+--data '{
+ "query": "SELECT * FROM wikipedia WHERE user='\''BlueMoon2662'\''",
+ "context" : {"sqlQueryId" : "request01"},
+ "header" : true,
+ "typesHeader" : true,
+ "sqlTypesHeader" : true
+}'
+```
+
+
+
+
+
+```HTTP
+POST /druid/v2/sql HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+Content-Type: application/json
+Content-Length: 192
+
+{
+ "query": "SELECT * FROM wikipedia WHERE user='BlueMoon2662'",
+ "context" : {"sqlQueryId" : "request01"},
+ "header" : true,
+ "typesHeader" : true,
+ "sqlTypesHeader" : true
+}
```
-There are a variety of [SQL query context parameters](../querying/sql-query-context.md) you can provide by adding a "context" map,
-like:
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+```json
+[
+ {
+ "__time": {
+ "type": "LONG",
+ "sqlType": "TIMESTAMP"
+ },
+ "channel": {
+ "type": "STRING",
+ "sqlType": "VARCHAR"
+ },
+ "cityName": {
+ "type": "STRING",
+ "sqlType": "VARCHAR"
+ },
+ "comment": {
+ "type": "STRING",
+ "sqlType": "VARCHAR"
+ },
+ "countryIsoCode": {
+ "type": "STRING",
+ "sqlType": "VARCHAR"
+ },
+ "countryName": {
+ "type": "STRING",
+ "sqlType": "VARCHAR"
+ },
+ "isAnonymous": {
+ "type": "LONG",
+ "sqlType": "BIGINT"
+ },
+ "isMinor": {
+ "type": "LONG",
+ "sqlType": "BIGINT"
+ },
+ "isNew": {
+ "type": "LONG",
+ "sqlType": "BIGINT"
+ },
+ "isRobot": {
+ "type": "LONG",
+ "sqlType": "BIGINT"
+ },
+ "isUnpatrolled": {
+ "type": "LONG",
+ "sqlType": "BIGINT"
+ },
+ "metroCode": {
+ "type": "LONG",
+ "sqlType": "BIGINT"
+ },
+ "namespace": {
+ "type": "STRING",
+ "sqlType": "VARCHAR"
+ },
+ "page": {
+ "type": "STRING",
+ "sqlType": "VARCHAR"
+ },
+ "regionIsoCode": {
+ "type": "STRING",
+ "sqlType": "VARCHAR"
+ },
+ "regionName": {
+ "type": "STRING",
+ "sqlType": "VARCHAR"
+ },
+ "user": {
+ "type": "STRING",
+ "sqlType": "VARCHAR"
+ },
+ "delta": {
+ "type": "LONG",
+ "sqlType": "BIGINT"
+ },
+ "added": {
+ "type": "LONG",
+ "sqlType": "BIGINT"
+ },
+ "deleted": {
+ "type": "LONG",
+ "sqlType": "BIGINT"
+ }
+ },
+ {
+ "__time": "2015-09-12T00:47:53.259Z",
+ "channel": "#ja.wikipedia",
+ "cityName": "",
+ "comment": "/* 対戦通算成績と得失点 */",
+ "countryIsoCode": "",
+ "countryName": "",
+ "isAnonymous": 0,
+ "isMinor": 1,
+ "isNew": 0,
+ "isRobot": 0,
+ "isUnpatrolled": 0,
+ "metroCode": 0,
+ "namespace": "Main",
+ "page": "アルビレックス新潟の年度別成績一覧",
+ "regionIsoCode": "",
+ "regionName": "",
+ "user": "BlueMoon2662",
+ "delta": 14,
+ "added": 14,
+ "deleted": 0
+ }
+]
+```
+
+
+### Cancel a query
+
+Cancels a query on the Router or the Broker with the associated `sqlQueryId`. The `sqlQueryId` can be manually set when the query is submitted in the query context parameter, or if not set, Druid will generate one and return it in the response header when the query is successfully submitted. Note that Druid does not enforce a unique `sqlQueryId` in the query context. If you've set the same `sqlQueryId` for multiple queries, Druid cancels all requests with that query ID.
+
+When you cancel a query, Druid handles the cancellation in a best-effort manner. Druid immediately marks the query as canceled and aborts the query execution as soon as possible. However, the query may continue running for a short time after you make the cancellation request.
+
+Cancellation requests require READ permission on all resources used in the SQL query.
+
+#### URL
+
+DELETE/druid/v2/sql/:sqlQueryId
+
+#### Responses
+
+
+
+
+
+
+*Successfully deleted query*
+
+
+
+
+
+*Authorization failure*
+
+
+
+
+
+*Invalid `sqlQueryId` or query was completed before cancellation request*
+
+
+
+
+---
+
+#### Sample request
+
+The following example cancels a request with the set query ID `request01`.
+
+
+
+
+
+
+```shell
+curl --request DELETE "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/request01"
+```
+
+
+
+
+
+```HTTP
+DELETE /druid/v2/sql/request01 HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+
+#### Sample response
+
+A successful response results in an `HTTP 202` message code and an empty response body.
+
+### Query output format
+
+The following table shows examples of how Druid returns the column names and data types based on the result format and the type request. The examples includes the first row of results, where the value of `user` is `BlueMoon2662`.
+
+```
+| Format | typesHeader | sqlTypesHeader | Example Output |
+|--------|-------------|----------------|--------------------------------------------------------------------------------------------|
+| object | true | false | [ { "user" : { "type" : "STRING" } }, { "user" : "BlueMoon2662" } ] |
+| object | true | true | [ { "user" : { "type" : "STRING", "sqlType" : "VARCHAR" } }, { "user" : "BlueMoon2662" } ] |
+| object | false | true | [ { "user" : { "sqlType" : "VARCHAR" } }, { "user" : "BlueMoon2662" } ] |
+| object | false | false | [ { "user" : null }, { "user" : "BlueMoon2662" } ] |
+| array | true | false | [ [ "user" ], [ "STRING" ], [ "BlueMoon2662" ] ] |
+| array | true | true | [ [ "user" ], [ "STRING" ], [ "VARCHAR" ], [ "BlueMoon2662" ] ] |
+| array | false | true | [ [ "user" ], [ "VARCHAR" ], [ "BlueMoon2662" ] ] |
+| array | false | false | [ [ "user" ], [ "BlueMoon2662" ] ] |
+| csv | true | false | user STRING BlueMoon2662 |
+| csv | true | true | user STRING VARCHAR BlueMoon2662 |
+| csv | false | true | user VARCHAR BlueMoon2662 |
+| csv | false | false | user BlueMoon2662 |
+```
+
+## Query from deep storage
+
+> Query from deep storage is an [experimental feature](../development/experimental.md).
+
+You can use the `sql/statements` endpoint to query segments that exist only in deep storage and are not loaded onto your Historical processes as determined by your load rules.
+
+Note that at least one segment of a datasource must be available on a Historical process so that the Broker can plan your query. A quick way to check if this is true is whether or not a datasource is visible in the Druid console.
+
+
+For more information, see [Query from deep storage](../querying/query-from-deep-storage.md).
+
+### Submit a query
+
+Submit a query for data stored in deep storage. Any data ingested into Druid is placed into deep storage. The query is contained in the "query" field in the JSON object within the request payload.
+
+Note that at least part of a datasource must be available on a Historical process so that Druid can plan your query and only the user who submits a query can see the results.
+
+#### URL
+
+POST/druid/v2/sql/statements
+
+#### Request body
+
+Generally, the `sql` and `sql/statements` endpoints support the same response body fields with minor differences. For general information about the available fields, see [Submit a query to the `sql` endpoint](#submit-a-query).
+
+Keep the following in mind when submitting queries to the `sql/statements` endpoint:
+
+- There are additional context parameters for `sql/statements`:
+
+ - `executionMode` determines how query results are fetched. Druid currently only supports `ASYNC`. You must manually retrieve your results after the query completes.
+ - `selectDestination` determines where final results get written. By default, results are written to task reports. Set this parameter to `durableStorage` to instruct Druid to write the results from SELECT queries to durable storage, which allows you to fetch larger result sets. Note that this requires you to have [durable storage for MSQ enabled](../operations/durable-storage.md).
+
+- The only supported value for `resultFormat` is JSON LINES.
+
+#### Responses
+
+
+
+
+
+
+*Successfully queried from deep storage*
+
+
+
+
+
+*Error thrown due to bad query. Returns a JSON object detailing the error with the following format:*
```json
{
- "query" : "SELECT COUNT(*) FROM data_source WHERE foo = 'bar' AND __time > TIMESTAMP '2000-01-01 00:00:00'",
- "context" : {
- "sqlTimeZone" : "America/Los_Angeles"
- }
+ "error": "Summary of the encountered error.",
+ "errorClass": "Class of exception that caused this error.",
+ "host": "The host on which the error occurred.",
+ "errorCode": "Well-defined error code.",
+ "persona": "Role or persona associated with the error.",
+ "category": "Classification of the error.",
+ "errorMessage": "Summary of the encountered issue with expanded information.",
+ "context": "Additional context about the error."
}
```
-Parameterized SQL queries are also supported:
+
+
+
+---
+
+#### Sample request
+
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements" \
+--header 'Content-Type: application/json' \
+--data '{
+ "query": "SELECT * FROM wikipedia WHERE user='\''BlueMoon2662'\''",
+ "context": {
+ "executionMode":"ASYNC"
+ }
+}'
+```
+
+
+
+
+
+```HTTP
+POST /druid/v2/sql/statements HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+Content-Type: application/json
+Content-Length: 134
+
+{
+ "query": "SELECT * FROM wikipedia WHERE user='BlueMoon2662'",
+ "context": {
+ "executionMode":"ASYNC"
+ }
+}
+```
+
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+{
+ "queryId": "query-b82a7049-b94f-41f2-a230-7fef94768745",
+ "state": "ACCEPTED",
+ "createdAt": "2023-07-26T21:16:25.324Z",
+ "schema": [
+ {
+ "name": "__time",
+ "type": "TIMESTAMP",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "channel",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "cityName",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "comment",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "countryIsoCode",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "countryName",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "isAnonymous",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "isMinor",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "isNew",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "isRobot",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "isUnpatrolled",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "metroCode",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "namespace",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "page",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "regionIsoCode",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "regionName",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "user",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "delta",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "added",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "deleted",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ }
+ ],
+ "durationMs": -1
+}
+ ```
+
+
+### Get query status
+
+Retrieves information about the query associated with the given query ID. The response matches the response from the POST API if the query is accepted or running and the execution mode is `ASYNC`. In addition to the fields that this endpoint shares with `POST /sql/statements`, a completed query's status includes the following:
+
+- A `result` object that summarizes information about your results, such as the total number of rows and sample records.
+- A `pages` object that includes the following information for each page of results:
+ - `numRows`: the number of rows in that page of results.
+ - `sizeInBytes`: the size of the page.
+ - `id`: the page number that you can use to reference a specific page when you get query results.
+
+#### URL
+
+GET/druid/v2/sql/statements/:queryId
+
+#### Responses
+
+
+
+
+
+
+*Successfully retrieved query status*
+
+
+
+
+
+*Error thrown due to bad query. Returns a JSON object detailing the error with the following format:*
```json
{
- "query" : "SELECT COUNT(*) FROM data_source WHERE foo = ? AND __time > ?",
- "parameters": [
- { "type": "VARCHAR", "value": "bar"},
- { "type": "TIMESTAMP", "value": "2000-01-01 00:00:00" }
- ]
+ "error": "Summary of the encountered error.",
+ "errorCode": "Well-defined error code.",
+ "persona": "Role or persona associated with the error.",
+ "category": "Classification of the error.",
+ "errorMessage": "Summary of the encountered issue with expanded information.",
+ "context": "Additional context about the error."
}
```
-Metadata is available over HTTP POST by querying [metadata tables](../querying/sql-metadata-tables.md).
+
+
+
+#### Sample request
+
+The following example retrieves the status of a query with specified ID `query-9b93f6f7-ab0e-48f5-986a-3520f84f0804`.
+
+
+
+
+
+
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804"
+```
+
+
+
+
+
+```HTTP
+GET /druid/v2/sql/statements/query-9b93f6f7-ab0e-48f5-986a-3520f84f0804 HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+{
+ "queryId": "query-9b93f6f7-ab0e-48f5-986a-3520f84f0804",
+ "state": "SUCCESS",
+ "createdAt": "2023-07-26T22:57:46.620Z",
+ "schema": [
+ {
+ "name": "__time",
+ "type": "TIMESTAMP",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "channel",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "cityName",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "comment",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "countryIsoCode",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "countryName",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "isAnonymous",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "isMinor",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "isNew",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "isRobot",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "isUnpatrolled",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "metroCode",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "namespace",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "page",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "regionIsoCode",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "regionName",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "user",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ },
+ {
+ "name": "delta",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "added",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ },
+ {
+ "name": "deleted",
+ "type": "BIGINT",
+ "nativeType": "LONG"
+ }
+ ],
+ "durationMs": 25591,
+ "result": {
+ "numTotalRows": 1,
+ "totalSizeInBytes": 375,
+ "dataSource": "__query_select",
+ "sampleRecords": [
+ [
+ 1442018873259,
+ "#ja.wikipedia",
+ "",
+ "/* 対戦通算成績と得失点 */",
+ "",
+ "",
+ 0,
+ 1,
+ 0,
+ 0,
+ 0,
+ 0,
+ "Main",
+ "アルビレックス新潟の年度別成績一覧",
+ "",
+ "",
+ "BlueMoon2662",
+ 14,
+ 14,
+ 0
+ ]
+ ],
+ "pages": [
+ {
+ "id": 0,
+ "numRows": 1,
+ "sizeInBytes": 375
+ }
+ ]
+ }
+}
+ ```
+
+
-### Responses
+### Get query results
-#### Result formats
+Retrieves results for completed queries. Results are separated into pages, so you can use the optional `page` parameter to refine the results you get. Druid returns information about the composition of each page and its page number (`id`). For information about pages, see [Get query status](#get-query-status).
-Druid SQL's HTTP POST API supports a variety of result formats. You can specify these by adding a `resultFormat` parameter, like:
+If a page number isn't passed, all results are returned sequentially in the same response. If you have large result sets, you may encounter timeouts based on the value configured for `druid.router.http.readTimeout`.
+
+When getting query results, keep the following in mind:
+
+- JSON Lines is the only supported result format.
+- Getting the query results for an ingestion query returns an empty response.
+
+#### URL
+
+GET/druid/v2/sql/statements/:queryId/results
+
+#### Query parameters
+* `page` (optional)
+ * Type: Int
+ * Refine paginated results
+
+#### Responses
+
+
+
+
+
+
+*Successfully retrieved query results*
+
+
+
+
+
+*Query in progress. Returns a JSON object detailing the error with the following format:*
```json
{
- "query" : "SELECT COUNT(*) FROM data_source WHERE foo = 'bar' AND __time > TIMESTAMP '2000-01-01 00:00:00'",
- "resultFormat" : "array"
+ "error": "Summary of the encountered error.",
+ "errorCode": "Well-defined error code.",
+ "persona": "Role or persona associated with the error.",
+ "category": "Classification of the error.",
+ "errorMessage": "Summary of the encountered issue with expanded information.",
+ "context": "Additional context about the error."
}
```
-To request a header with information about column names, set `header` to true in your request.
-When you set `header` to true, you can optionally include `typesHeader` and `sqlTypesHeader` as well, which gives
-you information about [Druid runtime and SQL types](../querying/sql-data-types.md) respectively. You can request all these headers
-with a request like:
+
+
+
+
+*Query not found, failed or canceled*
+
+
+
+
+
+*Error thrown due to bad query. Returns a JSON object detailing the error with the following format:*
```json
{
- "query" : "SELECT COUNT(*) FROM data_source WHERE foo = 'bar' AND __time > TIMESTAMP '2000-01-01 00:00:00'",
- "resultFormat" : "array",
- "header" : true,
- "typesHeader" : true,
- "sqlTypesHeader" : true
+ "error": "Summary of the encountered error.",
+ "errorCode": "Well-defined error code.",
+ "persona": "Role or persona associated with the error.",
+ "category": "Classification of the error.",
+ "errorMessage": "Summary of the encountered issue with expanded information.",
+ "context": "Additional context about the error."
}
```
-The following table shows supported result formats:
+
+
-|Format|Description|Header description|Content-Type|
-|------|-----------|------------------|------------|
-|`object`|The default, a JSON array of JSON objects. Each object's field names match the columns returned by the SQL query, and are provided in the same order as the SQL query.|If `header` is true, the first row is an object where the fields are column names. Each field's value is either null (if `typesHeader` and `sqlTypesHeader` are false) or an object that contains the Druid type as `type` (if `typesHeader` is true) and the SQL type as `sqlType` (if `sqlTypesHeader` is true).|application/json|
-|`array`|JSON array of JSON arrays. Each inner array has elements matching the columns returned by the SQL query, in order.|If `header` is true, the first row is an array of column names. If `typesHeader` is true, the next row is an array of Druid types. If `sqlTypesHeader` is true, the next row is an array of SQL types.|application/json|
-|`objectLines`|Like `object`, but the JSON objects are separated by newlines instead of being wrapped in a JSON array. This can make it easier to parse the entire response set as a stream, if you do not have ready access to a streaming JSON parser. To make it possible to detect a truncated response, this format includes a trailer of one blank line.|Same as `object`.|text/plain|
-|`arrayLines`|Like `array`, but the JSON arrays are separated by newlines instead of being wrapped in a JSON array. This can make it easier to parse the entire response set as a stream, if you do not have ready access to a streaming JSON parser. To make it possible to detect a truncated response, this format includes a trailer of one blank line.|Same as `array`, except the rows are separated by newlines.|text/plain|
-|`csv`|Comma-separated values, with one row per line. Individual field values may be escaped by being surrounded in double quotes. If double quotes appear in a field value, they will be escaped by replacing them with double-double-quotes like `""this""`. To make it possible to detect a truncated response, this format includes a trailer of one blank line.|Same as `array`, except the lists are in CSV format.|text/csv|
+---
-If `typesHeader` is set to true, [Druid type](../querying/sql-data-types.md) information is included in the response. Complex types,
-like sketches, will be reported as `COMPLEX` if a particular complex type name is known for that field,
-or as `COMPLEX` if the particular type name is unknown or mixed. If `sqlTypesHeader` is set to true,
-[SQL type](../querying/sql-data-types.md) information is included in the response. It is possible to set both `typesHeader` and
-`sqlTypesHeader` at once. Both parameters require that `header` is also set.
+#### Sample request
-To aid in building clients that are compatible with older Druid versions, Druid returns the HTTP header
-`X-Druid-SQL-Header-Included: yes` if `header` was set to true and if the version of Druid the client is connected to
-understands the `typesHeader` and `sqlTypesHeader` parameters. This HTTP response header is present irrespective of
-whether `typesHeader` or `sqlTypesHeader` are set or not.
+The following example retrieves the status of a query with specified ID `query-f3bca219-173d-44d4-bdc7-5002e910352f`.
-Druid returns the SQL query identifier in the `X-Druid-SQL-Query-Id` HTTP header.
-This query id will be assigned the value of `sqlQueryId` from the [query context parameters](../querying/sql-query-context.md)
-if specified, else Druid will generate a SQL query id for you.
+
-#### Errors
+
-Errors that occur before the response body is sent will be reported in JSON, with an HTTP 500 status code, in the
-same format as [native Druid query errors](../querying/querying.md#query-errors). If an error occurs while the response body is
-being sent, at that point it is too late to change the HTTP status code or report a JSON error, so the response will
-simply end midstream and an error will be logged by the Druid server that was handling your request.
-As a caller, it is important that you properly handle response truncation. This is easy for the `object` and `array`
-formats, since truncated responses will be invalid JSON. For the line-oriented formats, you should check the
-trailer they all include: one blank line at the end of the result set. If you detect a truncated response, either
-through a JSON parsing error or through a missing trailing newline, you should assume the response was not fully
-delivered due to an error.
+```shell
+curl "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-f3bca219-173d-44d4-bdc7-5002e910352f/results"
+```
-## Cancel a query
+
+
-You can use the HTTP DELETE method to cancel a SQL query on either the Router or the Broker. When you cancel a query, Druid handles the cancellation in a best-effort manner. It marks the query canceled immediately and aborts the query execution as soon as possible. However, your query may run for a short time after your cancellation request.
-Druid SQL's HTTP DELETE method uses the following syntax:
+```HTTP
+GET /druid/v2/sql/statements/query-f3bca219-173d-44d4-bdc7-5002e910352f/results HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
```
-DELETE https://ROUTER:8888/druid/v2/sql/{sqlQueryId}
+
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+[
+ {
+ "__time": 1442018818771,
+ "channel": "#en.wikipedia",
+ "cityName": "",
+ "comment": "added project",
+ "countryIsoCode": "",
+ "countryName": "",
+ "isAnonymous": 0,
+ "isMinor": 0,
+ "isNew": 0,
+ "isRobot": 0,
+ "isUnpatrolled": 0,
+ "metroCode": 0,
+ "namespace": "Talk",
+ "page": "Talk:Oswald Tilghman",
+ "regionIsoCode": "",
+ "regionName": "",
+ "user": "GELongstreet",
+ "delta": 36,
+ "added": 36,
+ "deleted": 0
+ },
+ {
+ "__time": 1442018820496,
+ "channel": "#ca.wikipedia",
+ "cityName": "",
+ "comment": "Robot inserta {{Commonscat}} que enllaça amb [[commons:category:Rallicula]]",
+ "countryIsoCode": "",
+ "countryName": "",
+ "isAnonymous": 0,
+ "isMinor": 1,
+ "isNew": 0,
+ "isRobot": 1,
+ "isUnpatrolled": 0,
+ "metroCode": 0,
+ "namespace": "Main",
+ "page": "Rallicula",
+ "regionIsoCode": "",
+ "regionName": "",
+ "user": "PereBot",
+ "delta": 17,
+ "added": 17,
+ "deleted": 0
+ },
+ {
+ "__time": 1442018825474,
+ "channel": "#en.wikipedia",
+ "cityName": "Auburn",
+ "comment": "/* Status of peremptory norms under international law */ fixed spelling of 'Wimbledon'",
+ "countryIsoCode": "AU",
+ "countryName": "Australia",
+ "isAnonymous": 1,
+ "isMinor": 0,
+ "isNew": 0,
+ "isRobot": 0,
+ "isUnpatrolled": 0,
+ "metroCode": 0,
+ "namespace": "Main",
+ "page": "Peremptory norm",
+ "regionIsoCode": "NSW",
+ "regionName": "New South Wales",
+ "user": "60.225.66.142",
+ "delta": 0,
+ "added": 0,
+ "deleted": 0
+ },
+ {
+ "__time": 1442018828770,
+ "channel": "#vi.wikipedia",
+ "cityName": "",
+ "comment": "fix Lỗi CS1: ngày tháng",
+ "countryIsoCode": "",
+ "countryName": "",
+ "isAnonymous": 0,
+ "isMinor": 1,
+ "isNew": 0,
+ "isRobot": 1,
+ "isUnpatrolled": 0,
+ "metroCode": 0,
+ "namespace": "Main",
+ "page": "Apamea abruzzorum",
+ "regionIsoCode": "",
+ "regionName": "",
+ "user": "Cheers!-bot",
+ "delta": 18,
+ "added": 18,
+ "deleted": 0
+ },
+ {
+ "__time": 1442018831862,
+ "channel": "#vi.wikipedia",
+ "cityName": "",
+ "comment": "clean up using [[Project:AWB|AWB]]",
+ "countryIsoCode": "",
+ "countryName": "",
+ "isAnonymous": 0,
+ "isMinor": 0,
+ "isNew": 0,
+ "isRobot": 1,
+ "isUnpatrolled": 0,
+ "metroCode": 0,
+ "namespace": "Main",
+ "page": "Atractus flammigerus",
+ "regionIsoCode": "",
+ "regionName": "",
+ "user": "ThitxongkhoiAWB",
+ "delta": 18,
+ "added": 18,
+ "deleted": 0
+ },
+ {
+ "__time": 1442018833987,
+ "channel": "#vi.wikipedia",
+ "cityName": "",
+ "comment": "clean up using [[Project:AWB|AWB]]",
+ "countryIsoCode": "",
+ "countryName": "",
+ "isAnonymous": 0,
+ "isMinor": 0,
+ "isNew": 0,
+ "isRobot": 1,
+ "isUnpatrolled": 0,
+ "metroCode": 0,
+ "namespace": "Main",
+ "page": "Agama mossambica",
+ "regionIsoCode": "",
+ "regionName": "",
+ "user": "ThitxongkhoiAWB",
+ "delta": 18,
+ "added": 18,
+ "deleted": 0
+ },
+ {
+ "__time": 1442018837009,
+ "channel": "#ca.wikipedia",
+ "cityName": "",
+ "comment": "/* Imperi Austrohongarès */",
+ "countryIsoCode": "",
+ "countryName": "",
+ "isAnonymous": 0,
+ "isMinor": 0,
+ "isNew": 0,
+ "isRobot": 0,
+ "isUnpatrolled": 0,
+ "metroCode": 0,
+ "namespace": "Main",
+ "page": "Campanya dels Balcans (1914-1918)",
+ "regionIsoCode": "",
+ "regionName": "",
+ "user": "Jaumellecha",
+ "delta": -20,
+ "added": 0,
+ "deleted": 20
+ },
+ {
+ "__time": 1442018839591,
+ "channel": "#en.wikipedia",
+ "cityName": "",
+ "comment": "adding comment on notability and possible COI",
+ "countryIsoCode": "",
+ "countryName": "",
+ "isAnonymous": 0,
+ "isMinor": 0,
+ "isNew": 1,
+ "isRobot": 0,
+ "isUnpatrolled": 1,
+ "metroCode": 0,
+ "namespace": "Talk",
+ "page": "Talk:Dani Ploeger",
+ "regionIsoCode": "",
+ "regionName": "",
+ "user": "New Media Theorist",
+ "delta": 345,
+ "added": 345,
+ "deleted": 0
+ },
+ {
+ "__time": 1442018841578,
+ "channel": "#en.wikipedia",
+ "cityName": "",
+ "comment": "Copying assessment table to wiki",
+ "countryIsoCode": "",
+ "countryName": "",
+ "isAnonymous": 0,
+ "isMinor": 0,
+ "isNew": 0,
+ "isRobot": 1,
+ "isUnpatrolled": 0,
+ "metroCode": 0,
+ "namespace": "User",
+ "page": "User:WP 1.0 bot/Tables/Project/Pubs",
+ "regionIsoCode": "",
+ "regionName": "",
+ "user": "WP 1.0 bot",
+ "delta": 121,
+ "added": 121,
+ "deleted": 0
+ },
+ {
+ "__time": 1442018845821,
+ "channel": "#vi.wikipedia",
+ "cityName": "",
+ "comment": "clean up using [[Project:AWB|AWB]]",
+ "countryIsoCode": "",
+ "countryName": "",
+ "isAnonymous": 0,
+ "isMinor": 0,
+ "isNew": 0,
+ "isRobot": 1,
+ "isUnpatrolled": 0,
+ "metroCode": 0,
+ "namespace": "Main",
+ "page": "Agama persimilis",
+ "regionIsoCode": "",
+ "regionName": "",
+ "user": "ThitxongkhoiAWB",
+ "delta": 18,
+ "added": 18,
+ "deleted": 0
+ }
+]
+ ```
+
+
+### Cancel a query
+
+Cancels a running or accepted query.
+
+#### URL
+
+DELETE/druid/v2/sql/statements/:queryId
+
+#### Responses
+
+
+
+
+
+
+*A no op operation since the query is not in a state to be cancelled*
+
+
+
+
+
+*Successfully accepted query for cancellation*
+
+
+
+
+
+*Invalid query ID. Returns a JSON object detailing the error with the following format:*
+
+```json
+{
+ "error": "Summary of the encountered error.",
+ "errorCode": "Well-defined error code.",
+ "persona": "Role or persona associated with the error.",
+ "category": "Classification of the error.",
+ "errorMessage": "Summary of the encountered issue with expanded information.",
+ "context": "Additional context about the error."
+}
```
-The DELETE method requires the `sqlQueryId` path parameter. To predict the query id you must set it in the query context. Druid does not enforce unique `sqlQueryId` in the query context. If you issue a cancel request for a `sqlQueryId` active in more than one query context, Druid cancels all requests that use the query id.
+
+
-For example if you issue the following query:
-```bash
-curl --request POST 'https://ROUTER:8888/druid/v2/sql' \
---header 'Content-Type: application/json' \
---data-raw '{"query" : "SELECT sleep(CASE WHEN sum_added > 0 THEN 1 ELSE 0 END) FROM wikiticker WHERE sum_added > 0 LIMIT 15",
-"context" : {"sqlQueryId" : "myQuery01"}}'
+---
+
+#### Sample request
+
+The following example cancels a query with specified ID `query-945c9633-2fa2-49ab-80ae-8221c38c024da`.
+
+
+
+
+
+
+```shell
+curl --request DELETE "http://ROUTER_IP:ROUTER_PORT/druid/v2/sql/statements/query-945c9633-2fa2-49ab-80ae-8221c38c024da"
```
-You can cancel the query using the query id `myQuery01` as follows:
-```bash
-curl --request DELETE 'https://ROUTER:8888/druid/v2/sql/myQuery01' \
+
+
+
+
+
+```HTTP
+DELETE /druid/v2/sql/statements/query-945c9633-2fa2-49ab-80ae-8221c38c024da HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
```
-Cancellation requests require READ permission on all resources used in the SQL query.
+
+
-Druid returns an HTTP 202 response for successful deletion requests.
+#### Sample response
-Druid returns an HTTP 404 response in the following cases:
- - `sqlQueryId` is incorrect.
- - The query completes before your cancellation request is processed.
-
-Druid returns an HTTP 403 response for authorization failure.
\ No newline at end of file
+A successful request returns an HTTP `202 ACCEPTED` message code and an empty response body.
diff --git a/docs/api-reference/sql-ingestion-api.md b/docs/api-reference/sql-ingestion-api.md
index 3050f595e673..3daadfa5085f 100644
--- a/docs/api-reference/sql-ingestion-api.md
+++ b/docs/api-reference/sql-ingestion-api.md
@@ -3,6 +3,8 @@ id: sql-ingestion-api
title: SQL-based ingestion API
sidebar_label: SQL-based ingestion
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
-> This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
-> extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
-> ingestion method is right for you.
+:::info
+ This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
+ extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
+ ingestion method is right for you.
+:::
The **Query** view in the web console provides a friendly experience for the multi-stage query task engine (MSQ task
engine) and multi-stage query architecture. We recommend using the web console if you do not need a programmatic
@@ -52,9 +56,10 @@ As an experimental feature, this endpoint also accepts SELECT queries. SELECT qu
by the controller, and written into the [task report](#get-the-report-for-a-query-task) as an array of arrays. The
behavior and result format of plain SELECT queries (without INSERT or REPLACE) is subject to change.
-
+
+
+
-
```
POST /druid/v2/sql/task
@@ -69,7 +74,10 @@ POST /druid/v2/sql/task
}
```
-
+
+
+
+
```bash
# Make sure you replace `username`, `password`, `your-instance`, and `port` with the values for your deployment.
@@ -83,7 +91,10 @@ curl --location --request POST 'https://:@:
+
+
+
+
```python
import json
@@ -108,7 +119,9 @@ print(response.text)
```
-
+
+
+
#### Response
@@ -132,22 +145,29 @@ You can retrieve status of a query to see if it is still running, completed succ
#### Request
-
+
+
+
-
```
GET /druid/indexer/v1/task//status
```
-
+
+
+
+
```bash
# Make sure you replace `username`, `password`, `your-instance`, `port`, and `taskId` with the values for your deployment.
curl --location --request GET 'https://:@:/druid/indexer/v1/task//status'
```
-
+
+
+
+
```python
import requests
@@ -163,7 +183,9 @@ response = requests.get(url, headers=headers, data=payload, auth=('USER', 'PASSW
print(response.text)
```
-
+
+
+
#### Response
@@ -210,22 +232,29 @@ For an explanation of the fields in a report, see [Report response fields](#repo
#### Request
-
+
+
+
-
```
GET /druid/indexer/v1/task//reports
```
-
+
+
+
+
```bash
# Make sure you replace `username`, `password`, `your-instance`, `port`, and `taskId` with the values for your deployment.
curl --location --request GET 'https://:@:/druid/indexer/v1/task//reports'
```
-
+
+
+
+
```python
import requests
@@ -238,7 +267,9 @@ response = requests.get(url, headers=headers, auth=('USER', 'PASSWORD'))
print(response.text)
```
-
+
+
+
#### Response
@@ -257,7 +288,19 @@ The response shows an example report for a query.
"startTime": "2022-09-14T22:12:09.266Z",
"durationMs": 28227,
"pendingTasks": 0,
- "runningTasks": 2
+ "runningTasks": 2,
+ "segmentLoadStatus": {
+ "state": "SUCCESS",
+ "dataSource": "kttm_simple",
+ "startTime": "2022-09-14T23:12:09.266Z",
+ "duration": 15,
+ "totalSegments": 1,
+ "usedSegments": 1,
+ "precachedSegments": 0,
+ "onDemandSegments": 0,
+ "pendingSegments": 0,
+ "unknownSegments": 0
+ }
},
"stages": [
{
@@ -513,7 +556,7 @@ The response shows an example report for a query.
"0": 1,
"1": 1,
"2": 1
- },
+ },
"totalMergersForUltimateLevel": 1,
"progressDigest": 1
}
@@ -562,6 +605,16 @@ The following table describes the response fields when you retrieve a report for
| `multiStageQuery.payload.status.durationMs` | Milliseconds elapsed after the query has started running. -1 denotes that the query hasn't started running yet. |
| `multiStageQuery.payload.status.pendingTasks` | Number of tasks that are not fully started. -1 denotes that the number is currently unknown. |
| `multiStageQuery.payload.status.runningTasks` | Number of currently running tasks. Should be at least 1 since the controller is included. |
+| `multiStageQuery.payload.status.segmentLoadStatus` | Segment loading container. Only present after the segments have been published. |
+| `multiStageQuery.payload.status.segmentLoadStatus.state` | Either INIT, WAITING, SUCCESS, FAILED or TIMED_OUT. |
+| `multiStageQuery.payload.status.segmentLoadStatus.startTime` | Time since which the controller has been waiting for the segments to finish loading. |
+| `multiStageQuery.payload.status.segmentLoadStatus.duration` | The duration in milliseconds that the controller has been waiting for the segments to load. |
+| `multiStageQuery.payload.status.segmentLoadStatus.totalSegments` | The total number of segments generated by the job. This includes tombstone segments (if any). |
+| `multiStageQuery.payload.status.segmentLoadStatus.usedSegments` | The number of segments which are marked as used based on the load rules. Unused segments can be cleaned up at any time. |
+| `multiStageQuery.payload.status.segmentLoadStatus.precachedSegments` | The number of segments which are marked as precached and served by historicals, as per the load rules. |
+| `multiStageQuery.payload.status.segmentLoadStatus.onDemandSegments` | The number of segments which are not loaded on any historical, as per the load rules. |
+| `multiStageQuery.payload.status.segmentLoadStatus.pendingSegments` | The number of segments remaining to be loaded. |
+| `multiStageQuery.payload.status.segmentLoadStatus.unknownSegments` | The number of segments whose status is unknown. |
| `multiStageQuery.payload.status.errorReport` | Error object. Only present if there was an error. |
| `multiStageQuery.payload.status.errorReport.taskId` | The task that reported the error, if known. May be a controller task or a worker task. |
| `multiStageQuery.payload.status.errorReport.host` | The hostname and port of the task that reported the error, if known. |
@@ -589,22 +642,29 @@ The following table describes the response fields when you retrieve a report for
#### Request
-
+
+
+
-
```
POST /druid/indexer/v1/task//shutdown
```
-
+
+
+
+
```bash
# Make sure you replace `username`, `password`, `your-instance`, `port`, and `taskId` with the values for your deployment.
curl --location --request POST 'https://:@:/druid/indexer/v1/task//shutdown'
```
-
+
+
+
+
```python
import requests
@@ -620,7 +680,9 @@ response = requests.post(url, headers=headers, data=payload, auth=('USER', 'PASS
print(response.text)
```
-
+
+
+
#### Response
diff --git a/docs/api-reference/sql-jdbc.md b/docs/api-reference/sql-jdbc.md
index 9356e4f0d0cf..2963713e7096 100644
--- a/docs/api-reference/sql-jdbc.md
+++ b/docs/api-reference/sql-jdbc.md
@@ -23,27 +23,51 @@ sidebar_label: SQL JDBC driver
~ under the License.
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](../querying/querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](../querying/querying.md).
+ This document describes the SQL language.
+:::
-You can make [Druid SQL](../querying/sql.md) queries using the [Avatica JDBC driver](https://calcite.apache.org/avatica/downloads/). We recommend using Avatica JDBC driver version 1.17.0 or later. Note that as of the time of this writing, Avatica 1.17.0, the latest version, does not support passing connection string parameters from the URL to Druid, so you must pass them using a `Properties` object. Once you've downloaded the Avatica client jar, add it to your classpath and use the connect string `jdbc:avatica:remote:url=http://BROKER:8082/druid/v2/sql/avatica/`.
+You can make [Druid SQL](../querying/sql.md) queries using the [Avatica JDBC driver](https://calcite.apache.org/avatica/downloads/).
+We recommend using Avatica JDBC driver version 1.23.0 or later. Note that starting with Avatica 1.21.0, you may need to set the [`transparent_reconnection`](https://calcite.apache.org/avatica/docs/client_reference.html#transparent_reconnection) property to `true` if you notice intermittent query failures.
-When using the JDBC connector for the [examples](#examples) or in general, it's helpful to understand the parts of the connect string stored in the `url` variable:
+Once you've downloaded the Avatica client jar, add it to your classpath.
- - `jdbc:avatica:remote:url=` is prepended to the hostname and port.
- - The hostname and port number for your Druid deployment depends on whether you want to connect to the Router or a specific Broker. For more information, see [Connection stickiness](#connection-stickiness). In the case of the quickstart deployment, the hostname and port are `http://localhost:8888`, which connects to the Router running on your local machine.
- - The SQL endpoint in Druid for the Avatica driver is `/druid/v2/sql/avatica/`.
+Example connection string:
-Example code:
+```
+jdbc:avatica:remote:url=http://localhost:8888/druid/v2/sql/avatica/;transparent_reconnection=true
+```
+
+Or, to use the protobuf protocol instead of JSON:
+
+```
+jdbc:avatica:remote:url=http://localhost:8888/druid/v2/sql/avatica-protobuf/;transparent_reconnection=true;serialization=protobuf
+```
+
+The `url` is the `/druid/v2/sql/avatica/` endpoint on the Router, which routes JDBC connections to a consistent Broker.
+For more information, see [Connection stickiness](#connection-stickiness).
+
+Set `transparent_reconnection` to `true` so your connection is not interrupted if the pool of Brokers changes membership,
+or if a Broker is restarted.
+
+Set `serialization` to `protobuf` if using the protobuf endpoint.
+
+Note that as of the time of this writing, Avatica 1.23.0, the latest version, does not support passing
+[connection context parameters](../querying/sql-query-context.md) from the JDBC connection string to Druid. These context parameters
+must be passed using a `Properties` object instead. Refer to the Java code below for an example.
+
+Example Java code:
```java
// Connect to /druid/v2/sql/avatica/ on your Broker.
-String url = "jdbc:avatica:remote:url=http://localhost:8082/druid/v2/sql/avatica/";
+String url = "jdbc:avatica:remote:url=http://localhost:8888/druid/v2/sql/avatica/;transparent_reconnection=true";
-// Set any connection context parameters you need here
-// Or leave empty for default behavior.
+// Set any connection context parameters you need here.
+// Any property from https://druid.apache.org/docs/latest/querying/sql-query-context.html can go here.
Properties connectionProperties = new Properties();
+connectionProperties.setProperty("sqlTimeZone", "Etc/UTC");
try (Connection connection = DriverManager.getConnection(url, connectionProperties)) {
try (
@@ -62,10 +86,12 @@ For a runnable example that includes a query that you might run, see [Examples](
It is also possible to use a protocol buffers JDBC connection with Druid, this offer reduced bloat and potential performance
improvements for larger result sets. To use it apply the following connection URL instead, everything else remains the same
```
-String url = "jdbc:avatica:remote:url=http://localhost:8082/druid/v2/sql/avatica-protobuf/;serialization=protobuf";
+String url = "jdbc:avatica:remote:url=http://localhost:8888/druid/v2/sql/avatica-protobuf/;transparent_reconnection=true;serialization=protobuf";
```
-> The protobuf endpoint is also known to work with the official [Golang Avatica driver](https://github.com/apache/calcite-avatica-go)
+:::info
+ The protobuf endpoint is also known to work with the official [Golang Avatica driver](https://github.com/apache/calcite-avatica-go)
+:::
Table metadata is available over JDBC using `connection.getMetaData()` or by querying the
[INFORMATION_SCHEMA tables](../querying/sql-metadata-tables.md). For an example of this, see [Get the metadata for a datasource](#get-the-metadata-for-a-datasource).
@@ -130,11 +156,12 @@ public class JdbcListColumns {
{
// Connect to /druid/v2/sql/avatica/ on your Router.
// You can connect to a Broker but must configure connection stickiness if you do.
- String url = "jdbc:avatica:remote:url=http://localhost:8888/druid/v2/sql/avatica/";
+ String url = "jdbc:avatica:remote:url=http://localhost:8888/druid/v2/sql/avatica/;transparent_reconnection=true";
String query = "SELECT COLUMN_NAME,* FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_NAME = 'wikipedia' and TABLE_SCHEMA='druid'";
- // Set any connection context parameters you need here
- // Or leave empty for default behavior.
+
+ // Set any connection context parameters you need here.
+ // Any property from https://druid.apache.org/docs/latest/querying/sql-query-context.html can go here.
Properties connectionProperties = new Properties();
try (Connection connection = DriverManager.getConnection(url, connectionProperties)) {
@@ -169,12 +196,13 @@ public class JdbcCountryAndTime {
{
// Connect to /druid/v2/sql/avatica/ on your Router.
// You can connect to a Broker but must configure connection stickiness if you do.
- String url = "jdbc:avatica:remote:url=http://localhost:8888/druid/v2/sql/avatica/";
+ String url = "jdbc:avatica:remote:url=http://localhost:8888/druid/v2/sql/avatica/;transparent_reconnection=true";
//The query you want to run.
String query = "SELECT __time, isRobot, countryName, comment FROM wikipedia WHERE countryName='Japan'";
- // Set any connection context parameters you need here
- // Or leave empty for default behavior.
+
+ // Set any connection context parameters you need here.
+ // Any property from https://druid.apache.org/docs/latest/querying/sql-query-context.html can go here.
Properties connectionProperties = new Properties();
connectionProperties.setProperty("sqlTimeZone", "America/Los_Angeles");
diff --git a/docs/api-reference/supervisor-api.md b/docs/api-reference/supervisor-api.md
index 592a7ec73338..b315971ec207 100644
--- a/docs/api-reference/supervisor-api.md
+++ b/docs/api-reference/supervisor-api.md
@@ -3,8 +3,12 @@ id: supervisor-api
title: Supervisor API
sidebar_label: Supervisors
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
+
+
-
+
-*Successfully retrieved array of active supervisor IDs*
-
+*Successfully retrieved array of active supervisor IDs*
+
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/supervisor HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
[
"wikipedia_stream",
@@ -102,40 +112,46 @@ Retrieves an array of active supervisor objects. If there are no active supervis
#### Responses
-
+
-
+
-*Successfully retrieved supervisor objects*
-
+*Successfully retrieved supervisor objects*
+
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor?full=null"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/supervisor?full=null HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
[
{
@@ -764,40 +780,46 @@ Retrieves an array of objects representing active supervisors and their current
#### Responses
-
+
-
+
-*Successfully retrieved supervisor state objects*
-
+*Successfully retrieved supervisor state objects*
+
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor?state=true"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/supervisor?state=true HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
[
{
@@ -829,17 +851,21 @@ Retrieves the specification for a single supervisor. The returned specification
#### Responses
-
+
-
+
-*Successfully retrieved supervisor spec*
-
+*Successfully retrieved supervisor spec*
-*Invalid supervisor ID*
+
+
-
+
+*Invalid supervisor ID*
+
+
+
---
@@ -847,22 +873,26 @@ Retrieves the specification for a single supervisor. The returned specification
The following example shows how to retrieve the specification of a supervisor with the name `wikipedia_stream`.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/wikipedia_stream"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/supervisor/wikipedia_stream HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -1187,17 +1217,21 @@ For additional information about the status report, see the topic for each strea
#### Responses
-
+
-
+
-*Successfully retrieved supervisor status*
-
+*Successfully retrieved supervisor status*
-*Invalid supervisor ID*
+
+
-
+
+*Invalid supervisor ID*
+
+
+
---
@@ -1205,22 +1239,26 @@ For additional information about the status report, see the topic for each strea
The following example shows how to retrieve the status of a supervisor with the name `social_media`.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/status"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/supervisor/social_media/status HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -1287,34 +1325,40 @@ Retrieve an audit history of specs for all supervisors.
#### Responses
-
+
-
+
-*Successfully retrieved audit history*
-
+*Successfully retrieved audit history*
+
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/history"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/supervisor/history HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -1642,17 +1686,21 @@ Retrieves an audit history of specs for a single supervisor.
#### Responses
-
+
-
+
-*Successfully retrieved supervisor audit history*
-
+*Successfully retrieved supervisor audit history*
-*Invalid supervisor ID*
+
+
-
+
+*Invalid supervisor ID*
+
+
+
---
@@ -1660,22 +1708,26 @@ Retrieves an audit history of specs for a single supervisor.
The following example shows how to retrieve the audit history of a supervisor with the name `wikipedia_stream`.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/wikipedia_stream/history"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/supervisor/wikipedia_stream/history HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -1994,7 +2046,7 @@ Host: http://ROUTER_IP:ROUTER_PORT
### Create or update a supervisor
-Creates a new supervisor or updates an existing one for the same datasource with a new schema and configuration.
+Creates a new supervisor or updates an existing one for the same datasource with a new schema and configuration.
You can define a supervisor spec for [Apache Kafka](../development/extensions-core/kafka-ingestion.md#define-a-supervisor-spec) or [Amazon Kinesis](../development/extensions-core/kinesis-ingestion.md#supervisor-spec) streaming ingestion methods. Once created, the supervisor persists in the metadata database.
@@ -2006,27 +2058,32 @@ When you call this endpoint on an existing supervisor for the same datasource, t
#### Responses
-
+
-
+
-*Successfully created a new supervisor or updated an existing supervisor*
-
+*Successfully created a new supervisor or updated an existing supervisor*
-*Request body content type is not in JSON format*
+
+
-
+
+*Request body content type is not in JSON format*
+
+
+
---
#### Sample request
-The following example uses JSON input format to create a supervisor spec for Kafka with a `social_media` datasource and `social_media` topic.
+The following example uses JSON input format to create a supervisor spec for Kafka with a `social_media` datasource and `social_media` topic.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor" \
@@ -2083,7 +2140,9 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor" \
}'
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/supervisor HTTP/1.1
@@ -2143,7 +2202,8 @@ Content-Length: 1359
}
```
-
+
+
#### Sample response
@@ -2166,21 +2226,27 @@ Suspends a single running supervisor. Returns the updated supervisor spec, where
#### Responses
-
+
-
+
-*Successfully shut down supervisor*
-
+*Successfully shut down supervisor*
-*Supervisor already suspended*
+
+
-
-*Invalid supervisor ID*
+*Supervisor already suspended*
-
+
+
+
+
+*Invalid supervisor ID*
+
+
+
---
@@ -2188,22 +2254,26 @@ Suspends a single running supervisor. Returns the updated supervisor spec, where
The following example shows how to suspend a running supervisor with the name `social_media`.
-
+
+
+
-
```shell
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/suspend"
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/supervisor/social_media/suspend HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -2522,34 +2592,40 @@ Suspends all supervisors. Note that this endpoint returns an HTTP `200 Success`
#### Responses
-
+
-
+
-*Successfully suspended all supervisors*
-
+*Successfully suspended all supervisors*
+
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/suspendAll"
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/supervisor/suspendAll HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -2573,21 +2649,27 @@ Resumes indexing tasks for a supervisor. Returns an updated supervisor spec with
#### Responses
-
+
-
+
-*Successfully resumed supervisor*
-
+*Successfully resumed supervisor*
-*Supervisor already running*
+
+
-
-*Invalid supervisor ID*
+*Supervisor already running*
-
+
+
+
+
+*Invalid supervisor ID*
+
+
+
---
@@ -2595,22 +2677,26 @@ Resumes indexing tasks for a supervisor. Returns an updated supervisor spec with
The following example resumes a previously suspended supervisor with name `social_media`.
-
+
+
+
-
```shell
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/resume"
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/supervisor/social_media/resume HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -2930,34 +3016,40 @@ Resumes all supervisors. Note that this endpoint returns an HTTP `200 Success` c
#### Responses
-
+
-
+
-*Successfully resumed all supervisors*
-
+*Successfully resumed all supervisors*
+
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/resumeAll"
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/supervisor/resumeAll HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -2973,9 +3065,9 @@ Host: http://ROUTER_IP:ROUTER_PORT
### Reset a supervisor
-Resets the specified supervisor. This endpoint clears stored offsets in Kafka or sequence numbers in Kinesis, prompting the supervisor to resume data reading. The supervisor will start from the earliest or latest available position, depending on the platform (offsets in Kafka or sequence numbers in Kinesis). It kills and recreates active tasks to read from valid positions.
+Resets the specified supervisor. This endpoint clears _all_ stored offsets in Kafka or sequence numbers in Kinesis, prompting the supervisor to resume data reading. The supervisor will start from the earliest or latest available position, depending on the platform (offsets in Kafka or sequence numbers in Kinesis). It kills and recreates active tasks to read from valid positions.
-Use this endpoint to recover from a stopped state due to missing offsets in Kafka or sequence numbers in Kinesis. Use this endpoint with caution as it may result in skipped messages and lead to data loss or duplicate data.
+Use this endpoint to recover from a stopped state due to missing offsets in Kafka or sequence numbers in Kinesis. Use this endpoint with caution as it may result in skipped messages and lead to data loss or duplicate data.
#### URL
@@ -2983,39 +3075,145 @@ Use this endpoint to recover from a stopped state due to missing offsets in Kafk
#### Responses
+
+
+
+
+
+*Successfully reset supervisor*
+
+
+
+
+
+*Invalid supervisor ID*
+
+
+
+
+---
+
+#### Sample request
+
+The following example shows how to reset a supervisor with the name `social_media`.
+
+
+
+
+
+
+```shell
+curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/reset"
+```
+
+
+
+
+
+```HTTP
+POST /druid/indexer/v1/supervisor/social_media/reset HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+```
+
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+{
+ "id": "social_media"
+}
+ ```
+
+
+### Reset Offsets for a supervisor
+
+Resets the specified offsets for a supervisor. This endpoint clears _only_ the specified offsets in Kafka or sequence numbers in Kinesis, prompting the supervisor to resume data reading.
+If there are no stored offsets, the specified offsets will be set in the metadata store. The supervisor will start from the reset offsets for the partitions specified and for the other partitions from the stored offset.
+It kills and recreates active tasks pertaining to the partitions specified to read from valid offsets.
+
+Use this endpoint to selectively reset offsets for partitions without resetting the entire set.
+
+#### URL
+
+POST/druid/indexer/v1/supervisor/:supervisorId/resetOffsets
+
+#### Responses
+
-*Successfully reset supervisor*
+*Successfully reset offsets*
-*Invalid supervisor ID*
+*Invalid supervisor ID*
---
+#### Reset Offsets Metadata
+
+This section presents the structure and details of the reset offsets metadata payload.
+
+| Field | Type | Description | Required |
+|---------|---------|---------|---------|
+| `type` | String | The type of reset offsets metadata payload. It must match the supervisor's `type`. Possible values: `kafka` or `kinesis`. | Yes |
+| `partitions` | Object | An object representing the reset metadata. See below for details. | Yes |
+
+#### Partitions
+
+The following table defines the fields within the `partitions` object in the reset offsets metadata payload.
+
+| Field | Type | Description | Required |
+|---------|---------|---------|---------|
+| `type` | String | Must be set as `end`. Indicates the end sequence numbers for the reset offsets. | Yes |
+| `stream` | String | The stream to be reset. It must be a valid stream consumed by the supervisor. | Yes |
+| `partitionOffsetMap` | Object | A map of partitions to corresponding offsets for the stream to be reset.| Yes |
#### Sample request
-The following example shows how to reset a supervisor with the name `social_media`.
+The following example shows how to reset offsets for a kafka supervisor with the name `social_media`. Let's say the supervisor is reading
+from a kafka topic `ads_media_stream` and has the stored offsets: `{"0": 0, "1": 10, "2": 20, "3": 40}`.
```shell
-curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/reset"
+curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/resetOffsets"
+--header 'Content-Type: application/json'
+--data-raw '{"type":"kafka","partitions":{"type":"end","stream":"ads_media_stream","partitionOffsetMap":{"0":100, "2": 650}}}'
```
```HTTP
-POST /druid/indexer/v1/supervisor/social_media/reset HTTP/1.1
+POST /druid/indexer/v1/supervisor/social_media/resetOffsets HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
+Content-Type: application/json
+
+{
+ "type": "kafka",
+ "partitions": {
+ "type": "end",
+ "stream": "ads_media_stream",
+ "partitionOffsetMap": {
+ "0": 100,
+ "2": 650
+ }
+ }
+}
```
+The above operation will reset offsets only for partitions 0 and 2 to 100 and 650 respectively. After a successful reset,
+when the supervisor's tasks restart, they will resume reading from `{"0": 100, "1": 10, "2": 650, "3": 40}`.
+
#### Sample response
@@ -3032,48 +3230,56 @@ Host: http://ROUTER_IP:ROUTER_PORT
### Terminate a supervisor
-Terminates a supervisor and its associated indexing tasks, triggering the publishing of their segments. When terminated, a tombstone marker is placed in the database to prevent reloading on restart.
+Terminates a supervisor and its associated indexing tasks, triggering the publishing of their segments. When terminated, a tombstone marker is placed in the database to prevent reloading on restart.
The terminated supervisor still exists in the metadata store and its history can be retrieved.
-#### URL
+#### URL
POST/druid/indexer/v1/supervisor/:supervisorId/terminate
#### Responses
-
+
-
+
-*Successfully terminated a supervisor*
-
+*Successfully terminated a supervisor*
-*Invalid supervisor ID or supervisor not running*
+
+
-
+
+*Invalid supervisor ID or supervisor not running*
+
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/terminate"
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/supervisor/social_media/terminate HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -3097,34 +3303,40 @@ Terminates all supervisors. Terminated supervisors still exist in the metadata s
#### Responses
-
+
-
+
-*Successfully terminated all supervisors*
-
+*Successfully terminated all supervisors*
+
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/terminateAll"
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/supervisor/terminateAll HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -3140,8 +3352,8 @@ Host: http://ROUTER_IP:ROUTER_PORT
### Shut down a supervisor
-Shuts down a supervisor. This endpoint is deprecated and will be removed in future releases. Use the equivalent [terminate](#terminate-a-supervisor) endpoint instead.
+Shuts down a supervisor. This endpoint is deprecated and will be removed in future releases. Use the equivalent [terminate](#terminate-a-supervisor) endpoint instead.
#### URL
-POST/druid/indexer/v1/supervisor/:supervisorId/shutdown
\ No newline at end of file
+POST/druid/indexer/v1/supervisor/:supervisorId/shutdown
diff --git a/docs/api-reference/tasks-api.md b/docs/api-reference/tasks-api.md
index 4f7261a5b842..9ccde5bf9d74 100644
--- a/docs/api-reference/tasks-api.md
+++ b/docs/api-reference/tasks-api.md
@@ -3,8 +3,12 @@ id: tasks-api
title: Tasks API
sidebar_label: Tasks
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
+
-This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. Tasks are individual jobs performed by Druid to complete operations such as ingestion, querying, and compaction.
+This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid. Tasks are individual jobs performed by Druid to complete operations such as ingestion, querying, and compaction.
In this topic, `http://ROUTER_IP:ROUTER_PORT` is a placeholder for the Router service address and port. For example, on the quickstart configuration, use `http://localhost:8888`.
-## Task information and retrieval
+## Task information and retrieval
### Get an array of tasks
@@ -39,7 +43,7 @@ Retrieves an array of all tasks in the Druid cluster. Each task object includes
#### Query parameters
-The endpoint supports a set of optional query parameters to filter results.
+The endpoint supports a set of optional query parameters to filter results.
|Parameter|Type|Description|
|---|---|---|
@@ -51,21 +55,33 @@ The endpoint supports a set of optional query parameters to filter results.
#### Responses
-
+
+
+
+
+
+
+
+*Successfully retrieved list of tasks*
+
+
+
-
-*Successfully retrieved list of tasks*
+
-
+*Invalid `state` query parameter value*
-*Invalid `state` query parameter value*
+
+
-
-*Invalid query parameter*
+
-
+*Invalid query parameter*
+
+
+
---
@@ -75,31 +91,35 @@ The following example shows how to retrieve a list of tasks filtered with the fo
* State: `complete`
* Datasource: `wikipedia_api`
* Time interval: between `2015-09-12` and `2015-09-13`
-* Max entries returned: `10`
+* Max entries returned: `10`
* Task type: `query_worker`
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/tasks/?state=complete&datasource=wikipedia_api&createdTimeInterval=2015-09-12_2015-09-13&max=10&type=query_worker"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/tasks/?state=complete&datasource=wikipedia_api&createdTimeInterval=2015-09-12_2015-09-13&max=10&type=query_worker HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
[
{
@@ -171,7 +191,7 @@ Retrieves an array of completed tasks in the Druid cluster. This is functionally
#### Query parameters
-The endpoint supports a set of optional query parameters to filter results.
+The endpoint supports a set of optional query parameters to filter results.
|Parameter|Type|Description|
|---|---|---|
@@ -182,44 +202,56 @@ The endpoint supports a set of optional query parameters to filter results.
#### Responses
-
+
+
+
+
-
+
-*Successfully retrieved list of complete tasks*
+*Successfully retrieved list of complete tasks*
-
+
+
-*Request sent to incorrect service*
-
+
+
+*Request sent to incorrect service*
+
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/completeTasks"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/completeTasks HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
[
{
@@ -273,7 +305,7 @@ Retrieves an array of running task objects in the Druid cluster. It is functiona
#### Query parameters
-The endpoint supports a set of optional query parameters to filter results.
+The endpoint supports a set of optional query parameters to filter results.
|Parameter|Type|Description|
|---|---|---|
@@ -284,41 +316,49 @@ The endpoint supports a set of optional query parameters to filter results.
#### Responses
-
+
+
+
+
-
+
-*Successfully retrieved list of running tasks*
+*Successfully retrieved list of running tasks*
-
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/runningTasks"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/runningTasks HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
[
{
@@ -354,7 +394,7 @@ Retrieves an array of waiting tasks in the Druid cluster. It is functionally equ
#### Query parameters
-The endpoint supports a set of optional query parameters to filter results.
+The endpoint supports a set of optional query parameters to filter results.
|Parameter|Type|Description|
|---|---|---|
@@ -365,40 +405,48 @@ The endpoint supports a set of optional query parameters to filter results.
#### Responses
-
+
-
+
-*Successfully retrieved list of waiting tasks*
-
+
+
+*Successfully retrieved list of waiting tasks*
+
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/waitingTasks"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/waitingTasks HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
[
{
@@ -470,7 +518,7 @@ Retrieves an array of pending tasks in the Druid cluster. It is functionally equ
#### Query parameters
-The endpoint supports a set of optional query parameters to filter results.
+The endpoint supports a set of optional query parameters to filter results.
|Parameter|Type|Description|
|---|---|---|
@@ -481,40 +529,48 @@ The endpoint supports a set of optional query parameters to filter results.
#### Responses
-
+
+
+
+
-
+
-*Successfully retrieved list of pending tasks*
+*Successfully retrieved list of pending tasks*
-
+
+
---
#### Sample request
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/pendingTasks"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/pendingTasks HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
[
{
@@ -566,19 +622,27 @@ Retrieves the payload of a task given the task ID. It returns a JSON object with
GET/druid/indexer/v1/task/:taskId
-#### Responses
+#### Responses
+
+
+
+
-
-
+
-*Successfully retrieved payload of task*
+*Successfully retrieved payload of task*
-
+
+
-*Cannot find task with ID*
-
+
+
+*Cannot find task with ID*
+
+
+
---
@@ -586,29 +650,33 @@ Retrieves the payload of a task given the task ID. It returns a JSON object with
The following examples shows how to retrieve the task payload of a task with the specified ID `index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z`.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/task/index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/task/index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
{
"task": "index_parallel_wikipedia_short_iajoonnd_2023-07-07T17:53:12.174Z",
@@ -769,19 +837,27 @@ Retrieves the status of a task given the task ID. It returns a JSON object with
GET/druid/indexer/v1/task/:taskId/status
-#### Responses
+#### Responses
+
+
+
+
+
-
+
-
+*Successfully retrieved task status*
-*Successfully retrieved task status*
+
+
-
-*Cannot find task with ID*
+
-
+*Cannot find task with ID*
+
+
+
---
@@ -789,22 +865,26 @@ Retrieves the status of a task given the task ID. It returns a JSON object with
The following examples shows how to retrieve the status of a task with the specified ID `query-223549f8-b993-4483-b028-1b0d54713cad`.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/task/query-223549f8-b993-4483-b028-1b0d54713cad/status"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/task/query-223549f8-b993-4483-b028-1b0d54713cad/status HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -836,7 +916,9 @@ Host: http://ROUTER_IP:ROUTER_PORT
### Get task segments
-> This API is deprecated and will be removed in future releases.
+:::info
+ This API is deprecated and will be removed in future releases.
+:::
Retrieves information about segments generated by the task given the task ID. To hit this endpoint, make sure to enable the audit log config on the Overlord with `druid.indexer.auditLog.enabled = true`.
@@ -848,13 +930,17 @@ In addition to enabling audit logs, configure a cleanup strategy to prevent over
#### Responses
-
+
-
+
-*Successfully retrieved task segments*
-
+
+
+*Successfully retrieved task segments*
+
+
+
---
@@ -862,22 +948,26 @@ In addition to enabling audit logs, configure a cleanup strategy to prevent over
The following examples shows how to retrieve the task segment of the task with the specified ID `query-52a8aafe-7265-4427-89fe-dc51275cc470`.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -885,7 +975,7 @@ A successful request returns a `200 OK` response and an array of the task segmen
### Get task log
-Retrieves the event log associated with a task. It returns a list of logged events during the lifecycle of the task. The endpoint is useful for providing information about the execution of the task, including any errors or warnings raised.
+Retrieves the event log associated with a task. It returns a list of logged events during the lifecycle of the task. The endpoint is useful for providing information about the execution of the task, including any errors or warnings raised.
Task logs are automatically retrieved from the Middle Manager/Indexer or in long-term storage. For reference, see [Task logs](../ingestion/tasks.md#task-logs).
@@ -893,6 +983,10 @@ Task logs are automatically retrieved from the Middle Manager/Indexer or in long
GET/druid/indexer/v1/task/:taskId/log
+#### URL
+
+GET/druid/indexer/v1/task/:taskId/log
+
#### Query parameters
* `offset` (optional)
@@ -901,13 +995,17 @@ Task logs are automatically retrieved from the Middle Manager/Indexer or in long
#### Responses
-
+
+
+
+
-
+
-*Successfully retrieved task log*
+*Successfully retrieved task log*
-
+
+
---
@@ -915,28 +1013,32 @@ Task logs are automatically retrieved from the Middle Manager/Indexer or in long
The following examples shows how to retrieve the task log of a task with the specified ID `index_kafka_social_media_0e905aa31037879_nommnaeg`.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/task/index_kafka_social_media_0e905aa31037879_nommnaeg/log"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/task/index_kafka_social_media_0e905aa31037879_nommnaeg/log HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
2023-07-03T22:11:17,891 INFO [qtp1251996697-122] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Sequence[index_kafka_social_media_0e905aa31037879_0] end offsets updated from [{0=9223372036854775807}] to [{0=230985}].
2023-07-03T22:11:17,900 INFO [qtp1251996697-122] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Saved sequence metadata to disk: [SequenceMetadata{sequenceId=0, sequenceName='index_kafka_social_media_0e905aa31037879_0', assignments=[0], startOffsets={0=230985}, exclusiveStartPartitions=[], endOffsets={0=230985}, sentinel=false, checkpointed=true}]
@@ -946,11 +1048,11 @@ Host: http://ROUTER_IP:ROUTER_PORT
2023-07-03T22:11:17,902 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Request joining group due to: consumer pro-actively leaving the group
2023-07-03T22:11:17,902 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.KafkaConsumer - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Unsubscribed all topics or patterns and assigned partitions
2023-07-03T22:11:17,912 INFO [task-runner-0-priority-0] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted rows[0] and (estimated) bytes[0]
- 2023-07-03T22:11:17,916 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Flushed in-memory data with commit metadata [AppenderatorDriverMetadata{segments={}, lastSegmentIds={}, callerMetadata={nextPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}}] for segments:
+ 2023-07-03T22:11:17,916 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Flushed in-memory data with commit metadata [AppenderatorDriverMetadata{segments={}, lastSegmentIds={}, callerMetadata={nextPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}}] for segments:
2023-07-03T22:11:17,917 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted stats: processed rows: [0], persisted rows[0], sinks: [0], total fireHydrants (across sinks): [0], persisted fireHydrants (across sinks): [0]
2023-07-03T22:11:17,919 INFO [task-runner-0-priority-0] org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver - Pushing [0] segments in background
2023-07-03T22:11:17,921 INFO [task-runner-0-priority-0] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted rows[0] and (estimated) bytes[0]
- 2023-07-03T22:11:17,924 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Flushed in-memory data with commit metadata [AppenderatorDriverMetadata{segments={}, lastSegmentIds={}, callerMetadata={nextPartitions=SeekableStreamStartSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}, exclusivePartitions=[]}, publishPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}}] for segments:
+ 2023-07-03T22:11:17,924 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Flushed in-memory data with commit metadata [AppenderatorDriverMetadata{segments={}, lastSegmentIds={}, callerMetadata={nextPartitions=SeekableStreamStartSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}, exclusivePartitions=[]}, publishPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}}] for segments:
2023-07-03T22:11:17,924 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-persist] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Persisted stats: processed rows: [0], persisted rows[0], sinks: [0], total fireHydrants (across sinks): [0], persisted fireHydrants (across sinks): [0]
2023-07-03T22:11:17,925 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-merge] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Preparing to push (stats): processed rows: [0], sinks: [0], fireHydrants (across sinks): [0]
2023-07-03T22:11:17,925 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-appenderator-merge] org.apache.druid.segment.realtime.appenderator.StreamAppenderator - Push complete...
@@ -958,7 +1060,7 @@ Host: http://ROUTER_IP:ROUTER_PORT
2023-07-03T22:11:17,930 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-publish] org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver - Published [0] segments with commit metadata [{nextPartitions=SeekableStreamStartSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}, exclusivePartitions=[]}, publishPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}]
2023-07-03T22:11:17,930 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-publish] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Published 0 segments for sequence [index_kafka_social_media_0e905aa31037879_0] with metadata [AppenderatorDriverMetadata{segments={}, lastSegmentIds={}, callerMetadata={nextPartitions=SeekableStreamStartSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}, exclusivePartitions=[]}, publishPartitions=SeekableStreamEndSequenceNumbers{stream='social_media', partitionSequenceNumberMap={0=230985}}}}].
2023-07-03T22:11:17,931 INFO [[index_kafka_social_media_0e905aa31037879_nommnaeg]-publish] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Saved sequence metadata to disk: []
- 2023-07-03T22:11:17,932 INFO [task-runner-0-priority-0] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Handoff complete for segments:
+ 2023-07-03T22:11:17,932 INFO [task-runner-0-priority-0] org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner - Handoff complete for segments:
2023-07-03T22:11:17,932 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Resetting generation and member id due to: consumer pro-actively leaving the group
2023-07-03T22:11:17,932 INFO [task-runner-0-priority-0] org.apache.kafka.clients.consumer.internals.ConsumerCoordinator - [Consumer clientId=consumer-kafka-supervisor-dcanhmig-1, groupId=kafka-supervisor-dcanhmig] Request joining group due to: consumer pro-actively leaving the group
2023-07-03T22:11:17,933 INFO [task-runner-0-priority-0] org.apache.kafka.common.metrics.Metrics - Metrics scheduler closed
@@ -1011,13 +1113,17 @@ Retrieves a [task completion report](../ingestion/tasks.md#task-reports) for a t
#### Responses
-
+
-
+
-*Successfully retrieved task report*
-
+
+
+*Successfully retrieved task report*
+
+
+
---
@@ -1025,28 +1131,32 @@ Retrieves a [task completion report](../ingestion/tasks.md#task-reports) for a t
The following examples shows how to retrieve the completion report of a task with the specified ID `query-52a8aafe-7265-4427-89fe-dc51275cc470`.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports"
```
-
+
+
+
```HTTP
GET /druid/indexer/v1/task/query-52a8aafe-7265-4427-89fe-dc51275cc470/reports HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
{
"ingestionStatsAndErrors": {
@@ -1095,25 +1205,41 @@ Note that for most batch ingestion use cases, you should use the [SQL-ingestion
#### Responses
-
+
+
+
+
+
+
+
+*Successfully submitted task*
-
+
+
-*Successfully submitted task*
-
+
-*Missing information in query*
+*Missing information in query*
-
+
+
-*Incorrect request body media type*
-
+
-*Unexpected token or characters in request body*
+*Incorrect request body media type*
-
+
+
+
+
+
+
+*Unexpected token or characters in request body*
+
+
+
---
@@ -1121,9 +1247,10 @@ Note that for most batch ingestion use cases, you should use the [SQL-ingestion
The following request is an example of submitting a task to create a datasource named `"wikipedia auto"`.
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/task" \
@@ -1170,7 +1297,9 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/task" \
}'
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/task HTTP/1.1
@@ -1220,7 +1349,8 @@ Content-Length: 952
}
```
-
+
+
#### Sample response
@@ -1245,17 +1375,25 @@ Shuts down a task if it not already complete. Returns a JSON object with the ID
#### Responses
-
+
-
+
-*Successfully shut down task*
-
+
-*Cannot find task with ID or task is no longer running*
+*Successfully shut down task*
-
+
+
+
+
+
+
+*Cannot find task with ID or task is no longer running*
+
+
+
---
@@ -1263,22 +1401,26 @@ Shuts down a task if it not already complete. Returns a JSON object with the ID
The following request shows how to shut down a task with the ID `query-52as 8aafe-7265-4427-89fe-dc51275cc470`.
-
+
+
+
-
```shell
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/task/query-52as 8aafe-7265-4427-89fe-dc51275cc470/shutdown"
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/task/query-52as 8aafe-7265-4427-89fe-dc51275cc470/shutdown HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
@@ -1303,17 +1445,25 @@ Shuts down all tasks for a specified datasource. If successful, it returns a JSO
#### Responses
-
+
+
+
-
-*Successfully shut down tasks*
+
-
+*Successfully shut down tasks*
-*Error or datasource does not have a running task*
+
+
-
+
+
+
+*Error or datasource does not have a running task*
+
+
+
---
@@ -1321,28 +1471,32 @@ Shuts down all tasks for a specified datasource. If successful, it returns a JSO
The following request is an example of shutting down all tasks for datasource `wikipedia_auto`.
-
+
+
+
-
```shell
curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/datasources/wikipedia_auto/shutdownAllTasks"
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/datasources/wikipedia_auto/shutdownAllTasks HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
{
"dataSource": "wikipedia_api"
@@ -1363,17 +1517,25 @@ Retrieves list of task status objects for list of task ID strings in request bod
#### Responses
-
+
-
+
-*Successfully retrieved status objects*
-
+
-*Missing request body or incorrect request body type*
+*Successfully retrieved status objects*
-
+
+
+
+
+
+
+*Missing request body or incorrect request body type*
+
+
+
---
@@ -1381,9 +1543,10 @@ Retrieves list of task status objects for list of task ID strings in request bod
The following request is an example of retrieving status objects for task ID `index_parallel_wikipedia_auto_jndhkpbo_2023-06-26T17:23:05.308Z` and `index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z` .
-
+
+
+
-
```shell
curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/taskStatus" \
@@ -1391,7 +1554,9 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/taskStatus" \
--data '["index_parallel_wikipedia_auto_jndhkpbo_2023-06-26T17:23:05.308Z","index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z"]'
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/taskStatus HTTP/1.1
@@ -1402,14 +1567,15 @@ Content-Length: 134
["index_parallel_wikipedia_auto_jndhkpbo_2023-06-26T17:23:05.308Z", "index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z"]
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
{
"index_parallel_wikipedia_auto_jbgiianh_2023-06-26T23:17:56.769Z": {
@@ -1452,13 +1618,17 @@ which automates this operation to perform periodically.
#### Responses
-
+
+
+
+
-
+
-*Successfully deleted pending segments*
+*Successfully deleted pending segments*
-
+
+
---
@@ -1466,28 +1636,32 @@ which automates this operation to perform periodically.
The following request is an example of cleaning up pending segments for the `wikipedia_api` datasource.
-
+
+
+
-
```shell
curl --request DELETE "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/pendingSegments/wikipedia_api"
```
-
+
+
+
```HTTP
DELETE /druid/indexer/v1/pendingSegments/wikipedia_api HTTP/1.1
Host: http://ROUTER_IP:ROUTER_PORT
```
-
+
+
#### Sample response
Click to show sample response
-
+
```json
{
"numDeleted": 2
diff --git a/docs/assets/tutorial-query-deepstorage-retention-rule.png b/docs/assets/tutorial-query-deepstorage-retention-rule.png
new file mode 100644
index 000000000000..9dee37bdeaad
Binary files /dev/null and b/docs/assets/tutorial-query-deepstorage-retention-rule.png differ
diff --git a/docs/configuration/extensions.md b/docs/configuration/extensions.md
index 3a2844221cce..5fbb20e74efe 100644
--- a/docs/configuration/extensions.md
+++ b/docs/configuration/extensions.md
@@ -67,7 +67,9 @@ Core extensions are maintained by Druid committers.
## Community extensions
-> Community extensions are not maintained by Druid committers, although we accept patches from community members using these extensions. They may not have been as extensively tested as the core extensions.
+:::info
+ Community extensions are not maintained by Druid committers, although we accept patches from community members using these extensions. They may not have been as extensively tested as the core extensions.
+:::
A number of community members have contributed their own extensions to Druid that are not packaged with the default Druid tarball.
If you'd like to take on maintenance for a community extension, please post on [dev@druid.apache.org](https://lists.apache.org/list.html?dev@druid.apache.org) to let us know!
@@ -123,12 +125,16 @@ druid.extensions.loadList=["postgresql-metadata-storage", "druid-hdfs-storage"]
These extensions are located in the `extensions` directory of the distribution.
-> Druid bundles two sets of configurations: one for the [quickstart](../tutorials/index.md) and
-> one for a [clustered configuration](../tutorials/cluster.md). Make sure you are updating the correct
-> `common.runtime.properties` for your setup.
+:::info
+ Druid bundles two sets of configurations: one for the [quickstart](../tutorials/index.md) and
+ one for a [clustered configuration](../tutorials/cluster.md). Make sure you are updating the correct
+ `common.runtime.properties` for your setup.
+:::
-> Because of licensing, the mysql-metadata-storage extension does not include the required MySQL JDBC driver. For instructions
-> on how to install this library, see the [MySQL extension page](../development/extensions-core/mysql.md).
+:::info
+ Because of licensing, the mysql-metadata-storage extension does not include the required MySQL JDBC driver. For instructions
+ on how to install this library, see the [MySQL extension page](../development/extensions-core/mysql.md).
+:::
### Loading community extensions
@@ -151,10 +157,14 @@ java \
You only have to install the extension once. Then, add `"druid-example-extension"` to
`druid.extensions.loadList` in common.runtime.properties to instruct Druid to load the extension.
-> Please make sure all the Extensions related configuration properties listed [here](../configuration/index.md#extensions) are set correctly.
+:::info
+ Please make sure all the Extensions related configuration properties listed [here](../configuration/index.md#extensions) are set correctly.
+:::
-> The Maven `groupId` for almost every [community extension](../configuration/extensions.md#community-extensions) is `org.apache.druid.extensions.contrib`. The `artifactId` is the name
-> of the extension, and the version is the latest Druid stable version.
+:::info
+ The Maven `groupId` for almost every [community extension](../configuration/extensions.md#community-extensions) is `org.apache.druid.extensions.contrib`. The `artifactId` is the name
+ of the extension, and the version is the latest Druid stable version.
+:::
### Loading extensions from the classpath
diff --git a/docs/configuration/index.md b/docs/configuration/index.md
index 4690af390b66..287f6872d3b7 100644
--- a/docs/configuration/index.md
+++ b/docs/configuration/index.md
@@ -395,6 +395,7 @@ Metric monitoring is an essential part of Druid operations. The following monit
|`org.apache.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical processes. Available only on Historical processes.|
|`org.apache.druid.server.metrics.SegmentStatsMonitor` | **EXPERIMENTAL** Reports statistics about segments on Historical processes. Available only on Historical processes. Not to be used when lazy loading is configured. |
|`org.apache.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted.|
+|`org.apache.druid.server.metrics.SubqueryCountStatsMonitor`|Reports how many subqueries have been materialized as rows or bytes and various other statistics related to the subquery execution|
|`org.apache.druid.server.emitter.HttpEmittingMonitor`|Reports internal metrics of `http` or `parametrized` emitter (see below). Must not be used with another emitter type. See the description of the metrics here: https://github.com/apache/druid/pull/4973.|
|`org.apache.druid.server.metrics.TaskCountStatsMonitor`|Reports how many ingestion tasks are currently running/pending/waiting and also the number of successful/failed tasks per emission period.|
|`org.apache.druid.server.metrics.TaskSlotCountStatsMonitor`|Reports metrics about task slot usage per emission period.|
@@ -775,7 +776,9 @@ the following properties.
|--------|-----------|-------|
|`druid.javascript.enabled`|Set to "true" to enable JavaScript functionality. This affects the JavaScript parser, filter, extractionFn, aggregator, post-aggregator, router strategy, and worker selection strategy.|false|
-> JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::info
+ JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::
### Double Column storage
@@ -796,7 +799,7 @@ Prior to version 0.13.0, Druid string columns treated `''` and `null` values as
|Property|Description|Default|
|---|---|---|
-|`druid.generic.useDefaultValueForNull`|When set to `true`, `null` values will be stored as `''` for string columns and `0` for numeric columns. Set to `false` to store and query data in SQL compatible mode.|`true`|
+|`druid.generic.useDefaultValueForNull`|Set to `false` to store and query data in SQL compatible mode. When set to `true` (legacy mode), `null` values will be stored as `''` for string columns and `0` for numeric columns.|`false`|
|`druid.generic.ignoreNullsForStringCardinality`|When set to `true`, `null` values will be ignored for the built-in cardinality aggregator over string columns. Set to `false` to include `null` values while estimating cardinality of only string columns using the built-in cardinality aggregator. This setting takes effect only when `druid.generic.useDefaultValueForNull` is set to `true` and is ignored in SQL compatibility mode. Additionally, empty strings (equivalent to null) are not counted when this is set to `true`. |`false`|
This mode does have a storage size and query performance cost, see [segment documentation](../design/segments.md#handling-null-values) for more details.
@@ -853,9 +856,10 @@ These Coordinator static configurations can be defined in the `coordinator/runti
|`druid.coordinator.load.timeout`|The timeout duration for when the Coordinator assigns a segment to a Historical process.|PT15M|
|`druid.coordinator.kill.pendingSegments.on`|Boolean flag for whether or not the Coordinator clean up old entries in the `pendingSegments` table of metadata store. If set to true, Coordinator will check the created time of most recently complete task. If it doesn't exist, it finds the created time of the earliest running/pending/waiting tasks. Once the created time is found, then for all dataSources not in the `killPendingSegmentsSkipList` (see [Dynamic configuration](#dynamic-configuration)), Coordinator will ask the Overlord to clean up the entries 1 day or more older than the found created time in the `pendingSegments` table. This will be done periodically based on `druid.coordinator.period.indexingPeriod` specified.|true|
|`druid.coordinator.kill.on`|Boolean flag for whether or not the Coordinator should submit kill task for unused segments, that is, permanently delete them from metadata store and deep storage. If set to true, then for all whitelisted dataSources (or optionally all), Coordinator will submit tasks periodically based on `period` specified. A whitelist can be set via dynamic configuration `killDataSourceWhitelist` described later.
When `druid.coordinator.kill.on` is true, segments are eligible for permanent deletion once their data intervals are older than `druid.coordinator.kill.durationToRetain` relative to the current time. If a segment's data interval is older than this threshold at the time it is marked unused, it is eligible for permanent deletion immediately after being marked unused.|false|
-|`druid.coordinator.kill.period`|How often to send kill tasks to the indexing service. Value must be greater than `druid.coordinator.period.indexingPeriod`. Only applies if kill is turned on.|P1D (1 Day)|
+|`druid.coordinator.kill.period`| The frequency of sending kill tasks to the indexing service. The value must be greater than or equal to `druid.coordinator.period.indexingPeriod`. Only applies if kill is turned on.|P1D (1 day)|
|`druid.coordinator.kill.durationToRetain`|Only applies if you set `druid.coordinator.kill.on` to `true`. This value is ignored if `druid.coordinator.kill.ignoreDurationToRetain` is `true`. Valid configurations must be a ISO8601 period. Druid will not kill unused segments whose interval end date is beyond `now - durationToRetain`. `durationToRetain` can be a negative ISO8601 period, which would result in `now - durationToRetain` to be in the future.
Note that the `durationToRetain` parameter applies to the segment interval, not the time that the segment was last marked unused. For example, if `durationToRetain` is set to `P90D`, then a segment for a time chunk 90 days in the past is eligible for permanent deletion immediately after being marked unused.|`P90D`|
|`druid.coordinator.kill.ignoreDurationToRetain`|A way to override `druid.coordinator.kill.durationToRetain` and tell the coordinator that you do not care about the end date of unused segment intervals when it comes to killing them. If true, the coordinator considers all unused segments as eligible to be killed.|false|
+|`druid.coordinator.kill.bufferPeriod`|The amount of time that a segment must be unused before it is able to be permanently removed from metadata and deep storage. This can serve as a buffer period to prevent data loss if data ends up being needed after being marked unused.|`P30D`|
|`druid.coordinator.kill.maxSegments`|The number of unused segments to kill per kill task. This number must be greater than 0. This only applies when `druid.coordinator.kill.on=true`.|100|
|`druid.coordinator.balancer.strategy`|Specify the type of balancing strategy for the coordinator to use to distribute segments among the historicals. `cachingCost` is logically equivalent to `cost` but is more CPU-efficient on large clusters. `diskNormalized` weights the costs according to the servers' disk usage ratios - there are known issues with this strategy distributing segments unevenly across the cluster. `random` distributes segments among services randomly.|`cost`|
|`druid.coordinator.balancer.cachingCost.awaitInitialization`|Whether to wait for segment view initialization before creating the `cachingCost` balancing strategy. This property is enabled only when `druid.coordinator.balancer.strategy` is `cachingCost`. If set to 'true', the Coordinator will not start to assign segments, until the segment view is initialized. If set to 'false', the Coordinator will fallback to use the `cost` balancing strategy only if the segment view is not initialized yet. Notes, it may take much time to wait for the initialization since the `cachingCost` balancing strategy involves much computing to build itself.|false|
@@ -934,11 +938,11 @@ A sample Coordinator dynamic config JSON object is shown below:
"replicantLifetime": 15,
"replicationThrottleLimit": 10,
"killDataSourceWhitelist": ["wikipedia", "testDatasource"],
+ "killTaskSlotRatio": 0.10,
+ "maxKillTaskSlots": 5,
"decommissioningNodes": ["localhost:8182", "localhost:8282"],
- "decommissioningMaxPercentOfMaxSegmentsToMove": 70,
"pauseCoordination": false,
- "replicateAfterLoadTimeout": false,
- "maxNonPrimaryReplicantsToLoad": 2147483647
+ "replicateAfterLoadTimeout": false
}
```
@@ -946,40 +950,41 @@ Issuing a GET request at the same URL will return the spec that is currently in
|Property|Description|Default|
|--------|-----------|-------|
-|`millisToWaitBeforeDeleting`|How long does the Coordinator need to be a leader before it can start marking overshadowed segments as unused in metadata storage.|900000 (15 mins)|
+|`millisToWaitBeforeDeleting`|How long does the Coordinator need to be a leader before it can start marking overshadowed segments as unused in metadata storage.| 900000 (15 mins)|
|`mergeBytesLimit`|The maximum total uncompressed size in bytes of segments to merge.|524288000L|
|`mergeSegmentsLimit`|The maximum number of segments that can be in a single [append task](../ingestion/tasks.md).|100|
|`smartSegmentLoading`|Enables ["smart" segment loading mode](#smart-segment-loading) which dynamically computes the optimal values of several properties that maximize Coordinator performance.|true|
-|`maxSegmentsToMove`|The maximum number of segments that can be moved at any given time.|100|
+|`maxSegmentsToMove`|The maximum number of segments that can be moved in a Historical tier at any given time.|100|
|`replicantLifetime`|The maximum number of Coordinator runs for which a segment can wait in the load queue of a Historical before Druid raises an alert.|15|
|`replicationThrottleLimit`|The maximum number of segment replicas that can be assigned to a historical tier in a single Coordinator run. This property prevents historicals from becoming overwhelmed when loading extra replicas of segments that are already available in the cluster.|500|
-|`balancerComputeThreads`|Thread pool size for computing moving cost of segments during segment balancing. Consider increasing this if you have a lot of segments and moving segments begins to stall.|1|
+|`balancerComputeThreads`|Thread pool size for computing moving cost of segments during segment balancing. Consider increasing this if you have a lot of segments and moving segments begins to stall.|`num_cores` / 2|
|`killDataSourceWhitelist`|List of specific data sources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated data source names or a JSON array.|none|
+|`killTaskSlotRatio`|Ratio of total available task slots, including autoscaling if applicable that will be allowed for kill tasks. This limit only applies for kill tasks that are spawned automatically by the coordinator's auto kill duty, which is enabled when `druid.coordinator.kill.on` is true.| 1 - all task slots can be used|
+|`maxKillTaskSlots`|Maximum number of tasks that will be allowed for kill tasks. This limit only applies for kill tasks that are spawned automatically by the coordinator's auto kill duty, which is enabled when `druid.coordinator.kill.on` is true.|`Integer.MAX_VALUE` - no limit|
|`killPendingSegmentsSkipList`|List of data sources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated data sources or a JSON array.|none|
-|`maxSegmentsInNodeLoadingQueue`|The maximum number of segments allowed in the load queue of any given server. Use this parameter to load segments faster if, for example, the cluster contains slow-loading nodes or if there are too many segments to be replicated to a particular node (when faster loading is preferred to better segments distribution). The optimal value depends on the loading speed of segments, acceptable replication time and number of nodes. |500|
-|`useRoundRobinSegmentAssignment`|Boolean flag for whether segments should be assigned to historicals in a round robin fashion. When disabled, segment assignment is done using the chosen balancer strategy. When enabled, this can speed up segment assignments leaving balancing to move the segments to their optimal locations (based on the balancer strategy) lazily. |true|
-|`decommissioningNodes`| List of historical servers to 'decommission'. Coordinator will not assign new segments to 'decommissioning' servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate specified by `decommissioningMaxPercentOfMaxSegmentsToMove`.|none|
-|`decommissioningMaxPercentOfMaxSegmentsToMove`| Upper limit of segments the Coordinator can move from decommissioning servers to active non-decommissioning servers during a single run. This value is relative to the total maximum number of segments that can be moved at any given time based upon the value of `maxSegmentsToMove`.
If `decommissioningMaxPercentOfMaxSegmentsToMove` is 0, the Coordinator does not move segments to decommissioning servers, effectively putting them in a type of "maintenance" mode. In this case, decommissioning servers do not participate in balancing or assignment by load rules. The Coordinator still considers segments on decommissioning servers as candidates to replicate on active servers.
Decommissioning can stall if there are no available active servers to move the segments to. You can use the maximum percent of decommissioning segment movements to prioritize balancing or to decrease commissioning time to prevent active servers from being overloaded. The value must be between 0 and 100.|70|
-|`pauseCoordination`| Boolean flag for whether or not the coordinator should execute its various duties of coordinating the cluster. Setting this to true essentially pauses all coordination work while allowing the API to remain up. Duties that are paused include all classes that implement the `CoordinatorDuty` Interface. Such duties include: Segment balancing, Segment compaction, Submitting kill tasks for unused segments (if enabled), Logging of used segments in the cluster, Marking of newly unused or overshadowed segments, Matching and execution of load/drop rules for used segments, Unloading segments that are no longer marked as used from Historical servers. An example of when an admin may want to pause coordination would be if they are doing deep storage maintenance on HDFS Name Nodes with downtime and don't want the coordinator to be directing Historical Nodes to hit the Name Node with API requests until maintenance is done and the deep store is declared healthy for use again. |false|
-|`replicateAfterLoadTimeout`| Boolean flag for whether or not additional replication is needed for segments that have failed to load due to the expiry of `druid.coordinator.load.timeout`. If this is set to true, the coordinator will attempt to replicate the failed segment on a different historical server. This helps improve the segment availability if there are a few slow historicals in the cluster. However, the slow historical may still load the segment later and the coordinator may issue drop requests if the segment is over-replicated.|false|
-|`maxNonPrimaryReplicantsToLoad`|The maximum number of replicas that can be assigned across all tiers in a single Coordinator run. This parameter serves the same purpose as `replicationThrottleLimit` except this limit applies at the cluster-level instead of per tier. The default value does not apply a limit to the number of replicas assigned per coordination cycle. If you want to use a non-default value for this property, you may want to start with `~20%` of the number of segments found on the historical server with the most segments. Use the Druid metric, `coordinator/time` with the filter `duty=org.apache.druid.server.coordinator.duty.RunRules` to see how different values of this property impact your Coordinator execution time.|`Integer.MAX_VALUE` (no limit)|
+|`maxSegmentsInNodeLoadingQueue`|The maximum number of segments allowed in the load queue of any given server. Use this parameter to load segments faster if, for example, the cluster contains slow-loading nodes or if there are too many segments to be replicated to a particular node (when faster loading is preferred to better segments distribution). The optimal value depends on the loading speed of segments, acceptable replication time and number of nodes.|500|
+|`useRoundRobinSegmentAssignment`|Boolean flag for whether segments should be assigned to historicals in a round robin fashion. When disabled, segment assignment is done using the chosen balancer strategy. When enabled, this can speed up segment assignments leaving balancing to move the segments to their optimal locations (based on the balancer strategy) lazily.|true|
+|`decommissioningNodes`|List of historical servers to 'decommission'. Coordinator will not assign new segments to 'decommissioning' servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate specified by `maxSegmentsToMove`.|none|
+|`pauseCoordination`|Boolean flag for whether or not the coordinator should execute its various duties of coordinating the cluster. Setting this to true essentially pauses all coordination work while allowing the API to remain up. Duties that are paused include all classes that implement the `CoordinatorDuty` Interface. Such duties include: Segment balancing, Segment compaction, Submitting kill tasks for unused segments (if enabled), Logging of used segments in the cluster, Marking of newly unused or overshadowed segments, Matching and execution of load/drop rules for used segments, Unloading segments that are no longer marked as used from Historical servers. An example of when an admin may want to pause coordination would be if they are doing deep storage maintenance on HDFS Name Nodes with downtime and don't want the coordinator to be directing Historical Nodes to hit the Name Node with API requests until maintenance is done and the deep store is declared healthy for use again.|false|
+|`replicateAfterLoadTimeout`|Boolean flag for whether or not additional replication is needed for segments that have failed to load due to the expiry of `druid.coordinator.load.timeout`. If this is set to true, the coordinator will attempt to replicate the failed segment on a different historical server. This helps improve the segment availability if there are a few slow historicals in the cluster. However, the slow historical may still load the segment later and the coordinator may issue drop requests if the segment is over-replicated.|false|
##### Smart segment loading
The `smartSegmentLoading` mode simplifies Coordinator configuration for segment loading and balancing.
If you enable this mode, do not provide values for the properties in the table below as the Coordinator computes them automatically.
Druid computes the values to optimize Coordinator performance, based on the current state of the cluster.
-> If you enable `smartSegmentLoading` mode, Druid ignores any value you provide for the following properties.
+:::info
+ If you enable `smartSegmentLoading` mode, Druid ignores any value you provide for the following properties.
+:::
|Property|Computed value|Description|
|--------|--------------|-----------|
|`useRoundRobinSegmentAssignment`|true|Speeds up segment assignment.|
|`maxSegmentsInNodeLoadingQueue`|0|Removes the limit on load queue size.|
-|`replicationThrottleLimit`|2% of used segments, minimum value 100|Prevents aggressive replication when a historical disappears only intermittently.|
+|`replicationThrottleLimit`|5% of used segments, minimum value 100|Prevents aggressive replication when a historical disappears only intermittently.|
|`replicantLifetime`|60|Allows segments to wait about an hour (assuming a Coordinator period of 1 minute) in the load queue before an alert is raised. In `smartSegmentLoading` mode, load queues are not limited by size. Segments might therefore assigned to a load queue even if the corresponding server is slow to load them.|
-|`maxNonPrimaryReplicantsToLoad`|`Integer.MAX_VALUE` (no limit)|This throttling is already handled by `replicationThrottleLimit`.|
|`maxSegmentsToMove`|2% of used segments, minimum value 100, maximum value 1000|Ensures that some segments are always moving in the cluster to keep it well balanced. The maximum value keeps the Coordinator run times bounded.|
-|`decommissioningMaxPercentOfMaxSegmentsToMove`|100|Prioritizes the move of segments from decommissioning servers so that they can be terminated quickly.|
+|`balancerComputeThreads`|`num_cores` / 2|Ensures that there are enough threads to perform balancing computations without hogging all Coordinator resources.|
When `smartSegmentLoading` is disabled, Druid uses the configured values of these properties.
Disable `smartSegmentLoading` only if you want to explicitly set the values of any of the above properties.
@@ -1187,7 +1192,7 @@ There are additional configs for autoscaling (if it is enabled):
|`druid.supervisor.idleConfig.enabled`|If `true`, supervisor can become idle if there is no data on input stream/topic for some time.|false|
|`druid.supervisor.idleConfig.inactiveAfterMillis`|Supervisor is marked as idle if all existing data has been read from input topic and no new data has been published for `inactiveAfterMillis` milliseconds.|`600_000`|
-The `druid.supervisor.idleConfig.*` specified in the runtime properties of the overlord defines the default behavior for the entire cluster. See [Idle Configuration in Kafka Supervisor IOConfig](../development/extensions-core/kafka-supervisor-reference.md#kafkasupervisorioconfig) to override it for an individual supervisor.
+The `druid.supervisor.idleConfig.*` specified in the Overlord runtime properties defines the default behavior for the entire cluster. See [Idle Configuration in Kafka Supervisor IOConfig](../development/extensions-core/kafka-supervisor-reference.md#supervisor-io-configuration) to override it for an individual supervisor.
#### Overlord dynamic configuration
@@ -1393,7 +1398,9 @@ Example: a function that sends batch_index_task to workers 10.0.0.1 and 10.0.0.2
}
```
-> JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::info
+ JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::
###### affinityConfig
@@ -1491,7 +1498,7 @@ Processing properties set on the MiddleManager will be passed through to Peons.
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB) for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1 GiB)|
|`druid.processing.buffer.poolCacheMaxCount`|Processing buffer pool caches the buffers for later use. This is the maximum count that the cache will grow to. Note that pool can create more buffers than it can cache if necessary.|`Integer.MAX_VALUE`|
|`druid.processing.formatString`|Realtime and Historical processes use this format string to name their processing threads.|processing-%s|
-|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
+|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
|`druid.processing.fifo`|Enables the processing queue to treat tasks of equal priority in a FIFO manner.|`true`|
|`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`|
@@ -1534,7 +1541,7 @@ Additional peon configs include:
|`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/task`|
|`druid.indexer.task.batchProcessingMode`| Batch ingestion tasks have three operating modes to control construction and tracking for intermediary segments: `OPEN_SEGMENTS`, `CLOSED_SEGMENTS`, and `CLOSED_SEGMENT_SINKS`. `OPEN_SEGMENTS` uses the streaming ingestion code path and performs a `mmap` on intermediary segments to build a timeline to make these segments available to realtime queries. Batch ingestion doesn't require intermediary segments, so the default mode, `CLOSED_SEGMENTS`, eliminates `mmap` of intermediary segments. `CLOSED_SEGMENTS` mode still tracks the entire set of segments in heap. The `CLOSED_SEGMENTS_SINKS` mode is the most aggressive configuration and should have the smallest memory footprint. It eliminates in-memory tracking and `mmap` of intermediary segments produced during segment creation. `CLOSED_SEGMENTS_SINKS` mode isn't as well tested as other modes so is currently considered experimental. You can use `OPEN_SEGMENTS` mode if problems occur with the 2 newer modes. |`CLOSED_SEGMENTS`|
-|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.8.5|
+|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|`org.apache.hadoop:hadoop-client-api:3.3.6`, `org.apache.hadoop:hadoop-client-runtime:3.3.6`|
|`druid.indexer.task.defaultRowFlushBoundary`|Highest row count before persisting to disk. Used for indexing generating tasks.|75000|
|`druid.indexer.task.directoryLockTimeout`|Wait this long for zombie peons to exit before giving up on their replacements.|PT10M|
|`druid.indexer.task.gracefulShutdownTimeout`|Wait this long on middleManager restart for restorable tasks to gracefully exit.|PT5M|
@@ -1605,7 +1612,7 @@ then the value from the configuration below is used:
|`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`|
-|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.8.5|
+|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|`org.apache.hadoop:hadoop-client-api:3.3.6`, `org.apache.hadoop:hadoop-client-runtime:3.3.6`|
|`druid.indexer.task.gracefulShutdownTimeout`|Wait this long on Indexer restart for restorable tasks to gracefully exit.|PT5M|
|`druid.indexer.task.hadoopWorkingPath`|Temporary working directory for Hadoop tasks.|`/tmp/druid-indexing`|
|`druid.indexer.task.restoreTasksOnRestart`|If true, the Indexer will attempt to stop tasks gracefully on shutdown and restore them on restart.|false|
@@ -1640,7 +1647,7 @@ Druid uses Jetty to serve HTTP requests.
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB) for the storage of intermediate results. The computation engine in the Indexer processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1GiB)|
|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
|`druid.processing.formatString`|Indexer processes use this format string to name their processing threads.|processing-%s|
-|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
+|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
|`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`|
|`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`|
@@ -1749,7 +1756,7 @@ Druid uses Jetty to serve HTTP requests.
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB), for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1GiB)|
|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
|`druid.processing.formatString`|Realtime and Historical processes use this format string to name their processing threads.|processing-%s|
-|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
+|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|`druid.processing.numThreads`|The number of processing threads to have available for parallel processing of segments. Our rule of thumb is `num_cores - 1`, which means that even under heavy load there will still be one core available to do background tasks like talking with ZooKeeper and pulling down segments. If only one core is available, this property defaults to the value `1`.|Number of cores - 1 (or 1)|
|`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`|
|`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`|
@@ -1928,7 +1935,7 @@ The broker uses processing configs for nested groupBy queries.
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size (less than 2GiB) for the storage of intermediate results. The computation engine in both the Historical and Realtime processes will use a scratch buffer of this size to do all of their intermediate computations off-heap. Larger values allow for more aggregations in a single pass over the data while smaller values can require more passes depending on the query that is being executed. [Human-readable format](human-readable-byte.md) is supported.|auto (max 1GiB)|
|`druid.processing.buffer.poolCacheInitialCount`|initializes the number of buffers allocated on the intermediate results pool. Note that pool can create more buffers if necessary.|`0`|
|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
-|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy v2) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
+|`druid.processing.numMergeBuffers`|The number of direct memory buffers available for merging query results. The buffers are sized by `druid.processing.buffer.sizeBytes`. This property is effectively a concurrency limit for queries that require merging buffers. If you are using any queries that require merge buffers (currently, just groupBy) then you should have at least two of these.|`max(2, druid.processing.numThreads / 4)`|
|`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`|
|`druid.processing.tmpDir`|Path where temporary files created while processing a query should be stored. If specified, this configuration takes priority over the default `java.io.tmpdir` path.|path represented by `java.io.tmpdir`|
|`druid.processing.merge.useParallelMergePool`|Enable automatic parallel merging for Brokers on a dedicated async ForkJoinPool. If `false`, instead merges will be done serially on the `HTTP` thread pool.|`true`|
@@ -1992,9 +1999,11 @@ The Druid SQL server is configured through the following properties on the Broke
|`druid.sql.planner.maxNumericInFilters`|Max limit for the amount of numeric values that can be compared for a string type dimension when the entire SQL WHERE clause of a query translates to an [OR](../querying/filters.md#or) of [Bound filter](../querying/filters.md#bound-filter). By default, Druid does not restrict the amount of numeric Bound Filters on String columns, although this situation may block other queries from running. Set this property to a smaller value to prevent Druid from running queries that have prohibitively long segment processing times. The optimal limit requires some trial and error; we recommend starting with 100. Users who submit a query that exceeds the limit of `maxNumericInFilters` should instead rewrite their queries to use strings in the `WHERE` clause instead of numbers. For example, `WHERE someString IN (‘123’, ‘456’)`. If this value is disabled, `maxNumericInFilters` set through query context is ignored.|`-1` (disabled)|
|`druid.sql.approxCountDistinct.function`|Implementation to use for the [`APPROX_COUNT_DISTINCT` function](../querying/sql-aggregations.md). Without extensions loaded, the only valid value is `APPROX_COUNT_DISTINCT_BUILTIN` (a HyperLogLog, or HLL, based implementation). If the [DataSketches extension](../development/extensions-core/datasketches-extension.md) is loaded, this can also be `APPROX_COUNT_DISTINCT_DS_HLL` (alternative HLL implementation) or `APPROX_COUNT_DISTINCT_DS_THETA`.
Theta sketches use significantly more memory than HLL sketches, so you should prefer one of the two HLL implementations.|APPROX_COUNT_DISTINCT_BUILTIN|
-> Previous versions of Druid had properties named `druid.sql.planner.maxQueryCount` and `druid.sql.planner.maxSemiJoinRowsInMemory`.
-> These properties are no longer available. Since Druid 0.18.0, you can use `druid.server.http.maxSubqueryRows` to control the maximum
-> number of rows permitted across all subqueries.
+:::info
+ Previous versions of Druid had properties named `druid.sql.planner.maxQueryCount` and `druid.sql.planner.maxSemiJoinRowsInMemory`.
+ These properties are no longer available. Since Druid 0.18.0, you can use `druid.server.http.maxSubqueryRows` to control the maximum
+ number of rows permitted across all subqueries.
+:::
#### Broker Caching
@@ -2013,8 +2022,10 @@ You can optionally only configure caching to be enabled on the Broker by setting
See [cache configuration](#cache-configuration) for how to configure cache settings.
-> Note: Even if cache is enabled, for [groupBy v2](../querying/groupbyquery.md#strategies) queries, segment level cache do not work on Brokers.
-> See [Differences between v1 and v2](../querying/groupbyquery.md#differences-between-v1-and-v2) and [Query caching](../querying/caching.md) for more information.
+:::info
+ Note: Even if cache is enabled, for [groupBy](../querying/groupbyquery.md) queries, segment level cache does not work on Brokers.
+ See [Query caching](../querying/caching.md) for more information.
+:::
#### Segment Discovery
|Property|Possible Values|Description|Default|
@@ -2049,7 +2060,9 @@ for both Broker and Historical processes, when defined in the common properties
#### Local Cache
-> DEPRECATED: Use caffeine (default as of v0.12.0) instead
+:::info
+ DEPRECATED: Use caffeine (default as of v0.12.0) instead
+:::
The local cache is deprecated in favor of the Caffeine cache, and may be removed in a future version of Druid. The Caffeine cache affords significantly better performance and control over eviction behavior compared to `local` cache, and is recommended in any situation where you are using JRE 8u60 or higher.
@@ -2186,8 +2199,6 @@ context). If query does have `maxQueuedBytes` in the context, then that value is
This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on Broker, Historical, and MiddleManager processes. You can set the query context parameters through the [query context](../querying/query-context.md).
-#### Configurations for groupBy v2
-
Supported runtime properties:
|Property|Description|Default|
@@ -2208,29 +2219,11 @@ Supported query contexts:
### Advanced configurations
-#### Common configurations for all groupBy strategies
-
Supported runtime properties:
|Property|Description|Default|
|--------|-----------|-------|
-|`druid.query.groupBy.defaultStrategy`|Default groupBy query strategy.|v2|
|`druid.query.groupBy.singleThreaded`|Merge results using a single thread.|false|
-
-Supported query contexts:
-
-|Key|Description|
-|---|-----------|
-|`groupByStrategy`|Overrides the value of `druid.query.groupBy.defaultStrategy` for this query.|
-|`groupByIsSingleThreaded`|Overrides the value of `druid.query.groupBy.singleThreaded` for this query.|
-
-
-#### GroupBy v2 configurations
-
-Supported runtime properties:
-
-|Property|Description|Default|
-|--------|-----------|-------|
|`druid.query.groupBy.bufferGrouperInitialBuckets`|Initial number of buckets in the off-heap hash table used for grouping results. Set to 0 to use a reasonable default (1024).|0|
|`druid.query.groupBy.bufferGrouperMaxLoadFactor`|Maximum load factor of the off-heap hash table used for grouping results. When the load factor exceeds this size, the table will be grown or spilled to disk. Set to 0 to use a reasonable default (0.7).|0|
|`druid.query.groupBy.forceHashAggregation`|Force to use hash-based aggregation.|false|
@@ -2241,6 +2234,7 @@ Supported query contexts:
|Key|Description|Default|
|---|-----------|-------|
+|`groupByIsSingleThreaded`|Overrides the value of `druid.query.groupBy.singleThreaded` for this query.|
|`bufferGrouperInitialBuckets`|Overrides the value of `druid.query.groupBy.bufferGrouperInitialBuckets` for this query.|None|
|`bufferGrouperMaxLoadFactor`|Overrides the value of `druid.query.groupBy.bufferGrouperMaxLoadFactor` for this query.|None|
|`forceHashAggregation`|Overrides the value of `druid.query.groupBy.forceHashAggregation`|None|
@@ -2249,30 +2243,12 @@ Supported query contexts:
|`sortByDimsFirst`|Sort the results first by dimension values and then by timestamp.|false|
|`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|
-
-#### GroupBy v1 configurations
-
-Supported runtime properties:
-
-|Property|Description|Default|
-|--------|-----------|-------|
-|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows for the per-segment grouping engine. This is a tuning parameter that does not impose a hard limit; rather, it potentially shifts merging work from the per-segment engine to the overall merging index. Queries that exceed this limit will not fail.|50000|
-|`druid.query.groupBy.maxResults`|Maximum number of results. Queries that exceed this limit will fail.|500000|
-
-Supported query contexts:
-
-|Key|Description|Default|
-|---|-----------|-------|
-|`maxIntermediateRows`|Ignored by groupBy v2. Can be used to lower the value of `druid.query.groupBy.maxIntermediateRows` for a groupBy v1 query.|None|
-|`maxResults`|Ignored by groupBy v2. Can be used to lower the value of `druid.query.groupBy.maxResults` for a groupBy v1 query.|None|
-|`useOffheap`|Ignored by groupBy v2, and no longer supported for groupBy v1. Enabling this option with groupBy v1 will result in an error. For off-heap aggregation, switch to groupBy v2, which always operates off-heap.|false|
-
#### Expression processing configurations
|Key|Description|Default|
|---|-----------|-------|
-|`druid.expressions.useStrictBooleans`|Controls the behavior of Druid boolean operators and functions, if set to `true` all boolean values will be either a `1` or `0`. See [expression documentation](../querying/math-expr.md#logical-operator-modes)|false|
-|`druid.expressions.allowNestedArrays`|If enabled, Druid array expressions can create nested arrays.|false|
+|`druid.expressions.useStrictBooleans`|Controls the behavior of Druid boolean operators and functions, if set to `true` all boolean values are either `1` or `0`. See [expression documentation](../querying/math-expr.md#logical-operator-modes) for more information.|true|
+|`druid.expressions.allowNestedArrays`|If enabled, Druid array expressions can create nested arrays.|true|
### Router
#### Router Process Configs
diff --git a/docs/configuration/logging.md b/docs/configuration/logging.md
index 175b25387d3f..0764ffcf862e 100644
--- a/docs/configuration/logging.md
+++ b/docs/configuration/logging.md
@@ -108,12 +108,14 @@ The following example log4j2.xml is based upon the micro quickstart:
Peons always output logs to standard output. Middle Managers redirect task logs from standard output to
[long-term storage](index.md#log-long-term-storage).
-> NOTE:
-> Druid shares the log4j configuration file among all services, including task peon processes.
-> However, you must define a console appender in the logger for your peon processes.
-> If you don't define a console appender, Druid creates and configures a new console appender
-> that retains the log level, such as `info` or `warn`, but does not retain any other appender
-> configuration, including non-console ones.
+:::info
+ NOTE:
+ Druid shares the log4j configuration file among all services, including task peon processes.
+ However, you must define a console appender in the logger for your peon processes.
+ If you don't define a console appender, Druid creates and configures a new console appender
+ that retains the log level, such as `info` or `warn`, but does not retain any other appender
+ configuration, including non-console ones.
+:::
## Log directory
The included log4j2.xml configuration for Druid and ZooKeeper writes logs to the `log` directory at the root of the distribution.
diff --git a/docs/data-management/automatic-compaction.md b/docs/data-management/automatic-compaction.md
index 05795339254e..8d696a86d4ef 100644
--- a/docs/data-management/automatic-compaction.md
+++ b/docs/data-management/automatic-compaction.md
@@ -23,6 +23,9 @@ title: "Automatic compaction"
-->
In Apache Druid, compaction is a special type of ingestion task that reads data from a Druid datasource and writes it back into the same datasource. A common use case for this is to [optimally size segments](../operations/segment-optimization.md) after ingestion to improve query performance. Automatic compaction, or auto-compaction, refers to the system for automatic execution of compaction tasks managed by the [Druid Coordinator](../design/coordinator.md).
+This topic guides you through setting up automatic compaction for your Druid cluster. See the [examples](#examples) for common use cases for automatic compaction.
+
+## How Druid manages automatic compaction
The Coordinator [indexing period](../configuration/index.md#coordinator-operation), `druid.coordinator.period.indexingPeriod`, controls the frequency of compaction tasks.
The default indexing period is 30 minutes, meaning that the Coordinator first checks for segments to compact at most 30 minutes from when auto-compaction is enabled.
@@ -33,9 +36,12 @@ At every invocation of auto-compaction, the Coordinator initiates a [segment sea
When there are eligible segments to compact, the Coordinator issues compaction tasks based on available worker capacity.
If a compaction task takes longer than the indexing period, the Coordinator waits for it to finish before resuming the period for segment search.
+:::info
+ Auto-compaction skips datasources that have a segment granularity of `ALL`.
+:::
+
As a best practice, you should set up auto-compaction for all Druid datasources. You can run compaction tasks manually for cases where you want to allocate more system resources. For example, you may choose to run multiple compaction tasks in parallel to compact an existing datasource for the first time. See [Compaction](compaction.md) for additional details and use cases.
-This topic guides you through setting up automatic compaction for your Druid cluster. See the [examples](#examples) for common use cases for automatic compaction.
## Enable automatic compaction
@@ -59,10 +65,10 @@ To disable auto-compaction for a datasource, click **Delete** from the **Compact
### Compaction configuration API
-Use the [Automatic compaction API](../api-reference/automatic-compaction-api.md#automatic-compaction-status) to configure automatic compaction.
+Use the [Automatic compaction API](../api-reference/automatic-compaction-api.md#manage-automatic-compaction) to configure automatic compaction.
To enable auto-compaction for a datasource, create a JSON object with the desired auto-compaction settings.
See [Configure automatic compaction](#configure-automatic-compaction) for the syntax of an auto-compaction spec.
-Send the JSON object as a payload in a [`POST` request](../api-reference/automatic-compaction-api.md#automatic-compaction-configuration) to `/druid/coordinator/v1/config/compaction`.
+Send the JSON object as a payload in a [`POST` request](../api-reference/automatic-compaction-api.md#create-or-update-automatic-compaction-configuration) to `/druid/coordinator/v1/config/compaction`.
The following example configures auto-compaction for the `wikipedia` datasource:
```sh
@@ -76,7 +82,7 @@ curl --location --request POST 'http://localhost:8081/druid/coordinator/v1/confi
}'
```
-To disable auto-compaction for a datasource, send a [`DELETE` request](../api-reference/automatic-compaction-api.md#automatic-compaction-configuration) to `/druid/coordinator/v1/config/compaction/{dataSource}`. Replace `{dataSource}` with the name of the datasource for which to disable auto-compaction. For example:
+To disable auto-compaction for a datasource, send a [`DELETE` request](../api-reference/automatic-compaction-api.md#remove-automatic-compaction-configuration) to `/druid/coordinator/v1/config/compaction/{dataSource}`. Replace `{dataSource}` with the name of the datasource for which to disable auto-compaction. For example:
```sh
curl --location --request DELETE 'http://localhost:8081/druid/coordinator/v1/config/compaction/wikipedia'
@@ -152,7 +158,7 @@ After the Coordinator has initiated auto-compaction, you can view compaction sta
In the web console, the Datasources view displays auto-compaction statistics. The Tasks view shows the task information for compaction tasks that were triggered by the automatic compaction system.
-To get statistics by API, send a [`GET` request](../api-reference/automatic-compaction-api.md#automatic-compaction-status) to `/druid/coordinator/v1/compaction/status`. To filter the results to a particular datasource, pass the datasource name as a query parameter to the request—for example, `/druid/coordinator/v1/compaction/status?dataSource=wikipedia`.
+To get statistics by API, send a [`GET` request](../api-reference/automatic-compaction-api.md#view-automatic-compaction-status) to `/druid/coordinator/v1/compaction/status`. To filter the results to a particular datasource, pass the datasource name as a query parameter to the request—for example, `/druid/coordinator/v1/compaction/status?dataSource=wikipedia`.
## Examples
@@ -174,8 +180,6 @@ The following auto-compaction configuration compacts existing `HOUR` segments in
}
```
-> Auto-compaction skips datasources containing ALL granularity segments when the target granularity is different.
-
### Update partitioning scheme
For your `wikipedia` datasource, you want to optimize segment access when regularly ingesting data without compromising compute time when querying the data. Your ingestion spec for batch append uses [dynamic partitioning](../ingestion/native-batch.md#dynamic-partitioning) to optimize for write-time operations, while your stream ingestion partitioning is configured by the stream service. You want to implement auto-compaction to reorganize the data with a suitable read-time partitioning using [multi-dimension range partitioning](../ingestion/native-batch.md#multi-dimension-range-partitioning). Based on the dimensions frequently accessed in queries, you wish to partition on the following dimensions: `channel`, `countryName`, `namespace`.
diff --git a/docs/data-management/compaction.md b/docs/data-management/compaction.md
index 96a7c0760929..c166623e887d 100644
--- a/docs/data-management/compaction.md
+++ b/docs/data-management/compaction.md
@@ -61,7 +61,9 @@ See [Setting up a manual compaction task](#setting-up-manual-compaction) for mor
During compaction, Druid overwrites the original set of segments with the compacted set. Druid also locks the segments for the time interval being compacted to ensure data consistency. By default, compaction tasks do not modify the underlying data. You can configure the compaction task to change the query granularity or add or remove dimensions in the compaction task. This means that the only changes to query results should be the result of intentional, not automatic, changes.
You can set `dropExisting` in `ioConfig` to "true" in the compaction task to configure Druid to replace all existing segments fully contained by the interval. See the suggestion for reindexing with finer granularity under [Implementation considerations](../ingestion/native-batch.md#implementation-considerations) for an example.
-> WARNING: `dropExisting` in `ioConfig` is a beta feature.
+:::info
+ WARNING: `dropExisting` in `ioConfig` is a beta feature.
+:::
If an ingestion task needs to write data to a segment for a time interval locked for compaction, by default the ingestion task supersedes the compaction task and the compaction task fails without finishing. For manual compaction tasks, you can adjust the input spec interval to avoid conflicts between ingestion and compaction. For automatic compaction, you can set the `skipOffsetFromLatest` key to adjust the auto-compaction starting point from the current time to reduce the chance of conflicts between ingestion and compaction.
Another option is to set the compaction task to higher priority than the ingestion task.
@@ -79,7 +81,9 @@ For example consider two overlapping segments: segment "A" for the interval 01/0
Unless you modify the query granularity in the [`granularitySpec`](#compaction-granularity-spec), Druid retains the query granularity for the compacted segments. If segments have different query granularities before compaction, Druid chooses the finest level of granularity for the resulting compacted segment. For example if a compaction task combines two segments, one with day query granularity and one with minute query granularity, the resulting segment uses minute query granularity.
-> In Apache Druid 0.21.0 and prior, Druid sets the granularity for compacted segments to the default granularity of `NONE` regardless of the query granularity of the original segments.
+:::info
+ In Apache Druid 0.21.0 and prior, Druid sets the granularity for compacted segments to the default granularity of `NONE` regardless of the query granularity of the original segments.
+:::
If you configure query granularity in compaction to go from a finer granularity like month to a coarser query granularity like year, then Druid overshadows the original segment with coarser granularity. Because the new segments have a coarser granularity, running a kill task to remove the overshadowed segments for those intervals will cause you to permanently lose the finer granularity data.
@@ -130,11 +134,15 @@ To perform a manual compaction, you submit a compaction task. Compaction tasks m
|`granularitySpec`|When set, the compaction task uses the specified `granularitySpec` rather than generating one from existing segments. See [Compaction `granularitySpec`](#compaction-granularity-spec) for details.|No|
|`context`|[Task context](../ingestion/tasks.md#context)|No|
-> Note: Use `granularitySpec` over `segmentGranularity` and only set one of these values. If you specify different values for these in the same compaction spec, the task fails.
+:::info
+ Note: Use `granularitySpec` over `segmentGranularity` and only set one of these values. If you specify different values for these in the same compaction spec, the task fails.
+:::
To control the number of result segments per time chunk, you can set [`maxRowsPerSegment`](../ingestion/native-batch.md#partitionsspec) or [`numShards`](../ingestion/../ingestion/native-batch.md#tuningconfig).
-> You can run multiple compaction tasks in parallel. For example, if you want to compact the data for a year, you are not limited to running a single task for the entire year. You can run 12 compaction tasks with month-long intervals.
+:::info
+ You can run multiple compaction tasks in parallel. For example, if you want to compact the data for a year, you are not limited to running a single task for the entire year. You can run 12 compaction tasks with month-long intervals.
+:::
A compaction task internally generates an `index` or `index_parallel` task spec for performing compaction work with some fixed parameters. For example, its `inputSource` is always the [`druid` input source](../ingestion/input-sources.md), and `dimensionsSpec` and `metricsSpec` include all dimensions and metrics of the input segments by default.
diff --git a/docs/data-management/delete.md b/docs/data-management/delete.md
index 260a66a17498..9e59c751bc2d 100644
--- a/docs/data-management/delete.md
+++ b/docs/data-management/delete.md
@@ -96,15 +96,18 @@ The available grammar is:
"dataSource": ,
"interval" : ,
"context": ,
- "batchSize":
+ "batchSize": ,
+ "limit":
}
```
Some of the parameters used in the task payload are further explained below:
-| Parameter |Default| Explanation |
-|--------------|-------|--------------------------------------------------------------------------------------------------------|
+| Parameter | Default | Explanation |
+|-------------|-----------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
| `batchSize` |100 | Maximum number of segments that are deleted in one kill batch. Some operations on the Overlord may get stuck while a `kill` task is in progress due to concurrency constraints (such as in `TaskLockbox`). Thus, a `kill` task splits the list of unused segments to be deleted into smaller batches to yield the Overlord resources intermittently to other task operations.|
+| `limit` | null - no limit | Maximum number of segments for the kill task to delete.|
+
**WARNING:** The `kill` task permanently removes all information about the affected segments from the metadata store and
deep storage. This operation cannot be undone.
diff --git a/docs/data-management/update.md b/docs/data-management/update.md
index 74508d0acfc9..cb36a9d6da42 100644
--- a/docs/data-management/update.md
+++ b/docs/data-management/update.md
@@ -43,7 +43,9 @@ is ongoing for a particular time range of a datasource, new ingestions for that
other time ranges proceed as normal. Read-only queries also proceed as normal, using the pre-existing version of the
data.
-> Druid does not support single-record updates by primary key.
+:::info
+ Druid does not support single-record updates by primary key.
+:::
## Reindex
diff --git a/docs/design/architecture.md b/docs/design/architecture.md
index 323cfb4fd3c3..df59dcb25eab 100644
--- a/docs/design/architecture.md
+++ b/docs/design/architecture.md
@@ -70,12 +70,20 @@ Druid uses deep storage to store any data that has been ingested into the system
storage accessible by every Druid server. In a clustered deployment, this is typically a distributed object store like S3 or
HDFS, or a network mounted filesystem. In a single-server deployment, this is typically local disk.
-Druid uses deep storage only as a backup of your data and as a way to transfer data in the background between
-Druid processes. Druid stores data in files called _segments_. Historical processes cache data segments on
-local disk and serve queries from that cache as well as from an in-memory cache.
-This means that Druid never needs to access deep storage
-during a query, helping it offer the best query latencies possible. It also means that you must have enough disk space
-both in deep storage and across your Historical servers for the data you plan to load.
+Druid uses deep storage for the following purposes:
+
+- To store all the data you ingest. Segments that get loaded onto Historical processes for low latency queries are also kept in deep storage for backup purposes. Additionally, segments that are only in deep storage can be used for [queries from deep storage](../querying/query-from-deep-storage.md).
+- As a way to transfer data in the background between Druid processes. Druid stores data in files called _segments_.
+
+Historical processes cache data segments on local disk and serve queries from that cache as well as from an in-memory cache.
+Segments on disk for Historical processes provide the low latency querying performance Druid is known for.
+
+You can also query directly from deep storage. When you query segments that exist only in deep storage, you trade some performance for the ability to query more of your data without necessarily having to scale your Historical processes.
+
+When determining sizing for your storage, keep the following in mind:
+
+- Deep storage needs to be able to hold all the data that you ingest into Druid.
+- On disk storage for Historical processes need to be able to accommodate the data you want to load onto them to run queries. The data on Historical processes should be data you access frequently and need to run low latency queries for.
Deep storage is an important part of Druid's elastic, fault-tolerant design. Druid bootstraps from deep storage even
if every single data server is lost and re-provisioned.
@@ -210,8 +218,7 @@ available before they are published, since they are only published when the segm
any additional rows of data.
2. **Deep storage:** Segment data files are pushed to deep storage once a segment is done being constructed. This
happens immediately before publishing metadata to the metadata store.
-3. **Availability for querying:** Segments are available for querying on some Druid data server, like a realtime task
-or a Historical process.
+3. **Availability for querying:** Segments are available for querying on some Druid data server, like a realtime task, directly from deep storage, or a Historical process.
You can inspect the state of currently active segments using the Druid SQL
[`sys.segments` table](../querying/sql-metadata-tables.md#segments-table). It includes the following flags:
diff --git a/docs/design/coordinator.md b/docs/design/coordinator.md
index 8c235cd3d54c..8e4536271ef3 100644
--- a/docs/design/coordinator.md
+++ b/docs/design/coordinator.md
@@ -92,7 +92,7 @@ Once some segments are found, it issues a [compaction task](../ingestion/tasks.m
The maximum number of running compaction tasks is `min(sum of worker capacity * slotRatio, maxSlots)`.
Note that even if `min(sum of worker capacity * slotRatio, maxSlots) = 0`, at least one compaction task is always submitted
if the compaction is enabled for a dataSource.
-See [Automatic compaction configuration API](../api-reference/automatic-compaction-api.md#automatic-compaction-configuration) and [Automatic compaction configuration](../configuration/index.md#automatic-compaction-dynamic-configuration) to enable and configure automatic compaction.
+See [Automatic compaction configuration API](../api-reference/automatic-compaction-api.md#manage-automatic-compaction) and [Automatic compaction configuration](../configuration/index.md#automatic-compaction-dynamic-configuration) to enable and configure automatic compaction.
Compaction tasks might fail due to the following reasons:
@@ -141,9 +141,11 @@ This is to avoid conflicts between compaction tasks and realtime tasks.
Note that realtime tasks have a higher priority than compaction tasks by default. Realtime tasks will revoke the locks of compaction tasks if their intervals overlap, resulting in the termination of the compaction task.
For more information, see [Avoid conflicts with ingestion](../data-management/automatic-compaction.md#avoid-conflicts-with-ingestion).
-> This policy currently cannot handle the situation when there are a lot of small segments which have the same interval,
-> and their total size exceeds [`inputSegmentSizeBytes`](../configuration/index.md#automatic-compaction-dynamic-configuration).
-> If it finds such segments, it simply skips them.
+:::info
+ This policy currently cannot handle the situation when there are a lot of small segments which have the same interval,
+ and their total size exceeds [`inputSegmentSizeBytes`](../configuration/index.md#automatic-compaction-dynamic-configuration).
+ If it finds such segments, it simply skips them.
+:::
### FAQ
diff --git a/docs/design/deep-storage.md b/docs/design/deep-storage.md
index f5adf35c6aaf..0674f324292c 100644
--- a/docs/design/deep-storage.md
+++ b/docs/design/deep-storage.md
@@ -23,9 +23,15 @@ title: "Deep storage"
-->
-Deep storage is where segments are stored. It is a storage mechanism that Apache Druid does not provide. This deep storage infrastructure defines the level of durability of your data, as long as Druid processes can see this storage infrastructure and get at the segments stored on it, you will not lose data no matter how many Druid nodes you lose. If segments disappear from this storage layer, then you will lose whatever data those segments represented.
+Deep storage is where segments are stored. It is a storage mechanism that Apache Druid does not provide. This deep storage infrastructure defines the level of durability of your data. As long as Druid processes can see this storage infrastructure and get at the segments stored on it, you will not lose data no matter how many Druid nodes you lose. If segments disappear from this storage layer, then you will lose whatever data those segments represented.
-## Local
+In addition to being the backing store for segments, you can use [query from deep storage](#querying-from-deep-storage) and run queries against segments stored primarily in deep storage. The [load rules](../operations/rule-configuration.md#load-rules) you configure determine whether segments exist primarily in deep storage or in a combination of deep storage and Historical processes.
+
+## Deep storage options
+
+Druid supports multiple options for deep storage, including blob storage from major cloud providers. Select the one that fits your environment.
+
+### Local
Local storage is intended for use in the following situations:
@@ -55,22 +61,28 @@ druid.storage.storageDirectory=/tmp/druid/localStorage
The `druid.storage.storageDirectory` must be set to a different path than `druid.segmentCache.locations` or
`druid.segmentCache.infoDir`.
-## Amazon S3 or S3-compatible
+### Amazon S3 or S3-compatible
See [`druid-s3-extensions`](../development/extensions-core/s3.md).
-## Google Cloud Storage
+### Google Cloud Storage
See [`druid-google-extensions`](../development/extensions-core/google.md).
-## Azure Blob Storage
+### Azure Blob Storage
See [`druid-azure-extensions`](../development/extensions-core/azure.md).
-## HDFS
+### HDFS
See [druid-hdfs-storage extension documentation](../development/extensions-core/hdfs.md).
-## Additional options
+### Additional options
For additional deep storage options, please see our [extensions list](../configuration/extensions.md).
+
+## Querying from deep storage
+
+Although not as performant as querying segments stored on disk for Historical processes, you can query from deep storage to access segments that you may not need frequently or with the extreme low latency Druid queries traditionally provide. You trade some performance for a total lower storage cost because you can access more of your data without the need to increase the number or capacity of your Historical processes.
+
+For information about how to run queries, see [Query from deep storage](../querying/query-from-deep-storage.md).
\ No newline at end of file
diff --git a/docs/design/indexer.md b/docs/design/indexer.md
index 9cce9c35059d..0cb7fbad9105 100644
--- a/docs/design/indexer.md
+++ b/docs/design/indexer.md
@@ -22,8 +22,10 @@ title: "Indexer Process"
~ under the License.
-->
-> The Indexer is an optional and [experimental](../development/experimental.md) feature.
-> Its memory management system is still under development and will be significantly enhanced in later releases.
+:::info
+ The Indexer is an optional and [experimental](../development/experimental.md) feature.
+ Its memory management system is still under development and will be significantly enhanced in later releases.
+:::
The Apache Druid Indexer process is an alternative to the MiddleManager + Peon task execution system. Instead of forking a separate JVM process per-task, the Indexer runs tasks as separate threads within a single JVM process.
diff --git a/docs/design/metadata-storage.md b/docs/design/metadata-storage.md
index 04f9b94eae60..20439aa13599 100644
--- a/docs/design/metadata-storage.md
+++ b/docs/design/metadata-storage.md
@@ -38,7 +38,9 @@ Derby is the default metadata store for Druid, however, it is not suitable for p
[MySQL](../development/extensions-core/mysql.md) and [PostgreSQL](../development/extensions-core/postgresql.md) are more production suitable metadata stores.
See [Metadata storage configuration](../configuration/index.md#metadata-storage) for the default configuration settings.
-> We also recommend you set up a high availability environment because there is no way to restore lost metadata.
+:::info
+ We also recommend you set up a high availability environment because there is no way to restore lost metadata.
+:::
## Available metadata stores
@@ -46,7 +48,9 @@ Druid supports Derby, MySQL, and PostgreSQL for storing metadata.
### Derby
-> For production clusters, consider using MySQL or PostgreSQL instead of Derby.
+:::info
+ For production clusters, consider using MySQL or PostgreSQL instead of Derby.
+:::
Configure metadata storage with Derby by setting the following properties in your Druid configuration.
@@ -99,7 +103,9 @@ system. The table has two main functional columns, the other columns are for ind
Value 1 in the `used` column means that the segment should be "used" by the cluster (i.e., it should be loaded and
available for requests). Value 0 means that the segment should not be loaded into the cluster. We do this as a means of
unloading segments from the cluster without actually removing their metadata (which allows for simpler rolling back if
-that is ever an issue).
+that is ever an issue). The `used` column has a corresponding `used_status_last_updated` column which denotes the time
+when the `used` status of the segment was last updated. This information can be used by the Coordinator to determine if
+a segment is a candidate for deletion (if automated segment killing is enabled).
The `payload` column stores a JSON blob that has all of the metadata for the segment.
Some of the data in the `payload` column intentionally duplicates data from other columns in the segments table.
diff --git a/docs/design/router.md b/docs/design/router.md
index d938c6a8eac1..4c2b19fb8e63 100644
--- a/docs/design/router.md
+++ b/docs/design/router.md
@@ -134,7 +134,9 @@ Allows defining arbitrary routing rules using a JavaScript function. The functio
}
```
-> JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::info
+ JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::
### Routing of SQL queries using strategies
diff --git a/docs/design/segments.md b/docs/design/segments.md
index 5dbc8ba97b38..194520045aa3 100644
--- a/docs/design/segments.md
+++ b/docs/design/segments.md
@@ -82,13 +82,16 @@ For each row in the list of column data, there is only a single bitmap that has
## Handling null values
-By default, Druid string dimension columns use the values `''` and `null` interchangeably. Numeric and metric columns cannot represent `null` but use nulls to mean `0`. However, Druid provides a SQL compatible null handling mode, which you can enable at the system level through `druid.generic.useDefaultValueForNull`. This setting, when set to `false`, allows Druid to create segments _at ingestion time_ in which the following occurs:
-* String columns can distinguish `''` from `null`,
-* Numeric columns can represent `null` valued rows instead of `0`.
+By default Druid stores segments in a SQL compatible null handling mode. String columns always store the null value as id 0, the first position in the value dictionary and an associated entry in the bitmap value indexes used to filter null values. Numeric columns also store a null value bitmap index to indicate the null valued rows, which is used to null check aggregations and for filter matching null values.
-String dimension columns contain no additional column structures in SQL compatible null handling mode. Instead, they reserve an additional dictionary entry for the `null` value. Numeric columns are stored in the segment with an additional bitmap in which the set bits indicate `null`-valued rows.
+Druid also has a legacy mode which uses default values instead of nulls, which was the default prior to Druid 28.0.0. This legacy mode can be enabled by setting `druid.generic.useDefaultValueForNull=true`.
-In addition to slightly increased segment sizes, SQL compatible null handling can incur a performance cost at query time, due to the need to check the null bitmap. This performance cost only occurs for columns that actually contain null values.
+In legacy mode, Druid segments created _at ingestion time_ have the following characteristics:
+
+* String columns can not distinguish `''` from `null`, they are treated interchangeably as the same value
+* Numeric columns can not represent `null` valued rows, and instead store a `0`.
+
+In legacy mode, numeric columns do not have the null value bitmap, and so can have slightly decreased segment sizes, and queries involving numeric columns can have slightly increased performance in some cases since there is no need to check the null value bitmap.
## Segments with different schemas
diff --git a/docs/design/zookeeper.md b/docs/design/zookeeper.md
index ff2f3896e307..8f1a19dff7ee 100644
--- a/docs/design/zookeeper.md
+++ b/docs/design/zookeeper.md
@@ -29,7 +29,9 @@ Apache Druid uses [Apache ZooKeeper](http://zookeeper.apache.org/) (ZK) for mana
Apache Druid supports ZooKeeper versions 3.5.x and above.
-> Note: Starting with Apache Druid 0.22.0, support for ZooKeeper 3.4.x has been removed
+:::info
+ Note: Starting with Apache Druid 0.22.0, support for ZooKeeper 3.4.x has been removed
+:::
## ZooKeeper Operations
diff --git a/docs/development/experimental-features.md b/docs/development/experimental-features.md
index d33f634a4b62..36c72822b4d3 100644
--- a/docs/development/experimental-features.md
+++ b/docs/development/experimental-features.md
@@ -57,4 +57,3 @@ Note that this document does not track the status of contrib extensions, all of
- [Configuration reference](../configuration/index.md)
- `CLOSED_SEGMENTS_SINKS` mode
- - Expression processing configuration `druid.expressions.allowNestedArrays`
diff --git a/docs/development/extensions-contrib/k8s-jobs.md b/docs/development/extensions-contrib/k8s-jobs.md
index cd925c2ee046..2132b55ea1e3 100644
--- a/docs/development/extensions-contrib/k8s-jobs.md
+++ b/docs/development/extensions-contrib/k8s-jobs.md
@@ -234,6 +234,12 @@ data:
|`druid.indexer.runner.graceTerminationPeriodSeconds`| `Long` | Number of seconds you want to wait after a sigterm for container lifecycle hooks to complete. Keep at a smaller value if you want tasks to hold locks for shorter periods. |`PT30S` (K8s default)|No|
|`druid.indexer.runner.capacity`| `Integer` | Number of concurrent jobs that can be sent to Kubernetes. |`2147483647`|No|
+### Metrics added
+
+|Metric|Description|Dimensions|Normal value|
+|------|-----------|----------|------------|
+| `k8s/peon/startup/time` | Metric indicating the milliseconds for peon pod to startup. | `dataSource`, `taskId`, `taskType`, `groupId`, `taskStatus`, `tags` |Varies|
+
### Gotchas
- All Druid Pods belonging to one Druid cluster must be inside the same Kubernetes namespace.
@@ -266,4 +272,18 @@ roleRef:
kind: Role
name: druid-k8s-task-scheduler
apiGroup: rbac.authorization.k8s.io
-```
\ No newline at end of file
+```
+
+## Migration/Kubernetes and Worker Task Runner
+If you are running a cluster with tasks running on middle managers or indexers and want to do a zero downtime migration to mm-less ingestion, the mm-less ingestion system is capable of running in migration mode by reading tasks from middle managers/indexers and Kubernetes and writing tasks to either middle managers or to Kubernetes.
+
+To do this, set the following property.
+`druid.indexer.runner.type: k8sAndWorker` (instead of `druid.indexer.runner.type: k8s`)
+
+### Additional Configurations
+
+|Property| Possible Values |Description|Default|required|
+|--------|-----------------|-----------|-------|--------|
+|`druid.indexer.runner.k8sAndWorker.workerTaskRunnerType`|`String`|Determines whether the `httpRemote` or the `remote` task runner should be used in addition to the Kubernetes task runner.|`httpRemote`|No|
+|`druid.indexer.runner.k8sAndWorker.sendAllTasksToWorkerTaskRunner`|`boolean`| Whether to send all the tasks to the worker task runner. If this is set to false all tasks will be sent to Kubernetes|`false`|No|
+
diff --git a/docs/development/extensions-contrib/prometheus.md b/docs/development/extensions-contrib/prometheus.md
index 2612921505c6..25e521a265f2 100644
--- a/docs/development/extensions-contrib/prometheus.md
+++ b/docs/development/extensions-contrib/prometheus.md
@@ -45,7 +45,8 @@ All the configuration parameters for the Prometheus emitter are under `druid.emi
| `druid.emitter.prometheus.addHostAsLabel` | Flag to include the hostname as a prometheus label. | no | false |
| `druid.emitter.prometheus.addServiceAsLabel` | Flag to include the druid service name (e.g. `druid/broker`, `druid/coordinator`, etc.) as a prometheus label. | no | false |
| `druid.emitter.prometheus.pushGatewayAddress` | Pushgateway address. Required if using `pushgateway` strategy. | no | none |
-|`druid.emitter.prometheus.flushPeriod`|Emit metrics to Pushgateway every `flushPeriod` seconds. Required if `pushgateway` strategy is used.|no|15|
+| `druid.emitter.prometheus.flushPeriod` | Emit metrics to Pushgateway every `flushPeriod` seconds. Required if `pushgateway` strategy is used. | no | 15 |
+| `druid.emitter.prometheus.extraLabels` | JSON key-value pairs for additional labels on all metrics. Keys (label names) must match the regex `[a-zA-Z_:][a-zA-Z0-9_:]*`. Example: `{"cluster_name": "druid_cluster1", "env": "staging"}`. | no | none |
### Ports for colocated Druid processes
diff --git a/docs/development/extensions-core/approximate-histograms.md b/docs/development/extensions-core/approximate-histograms.md
index 7e24f958d464..ee6455e47d5a 100644
--- a/docs/development/extensions-core/approximate-histograms.md
+++ b/docs/development/extensions-core/approximate-histograms.md
@@ -31,7 +31,9 @@ The `druid-histogram` extension provides an approximate histogram aggregator and
## Approximate Histogram aggregator (Deprecated)
-> The Approximate Histogram aggregator is deprecated. Please use [DataSketches Quantiles](../extensions-core/datasketches-quantiles.md) instead which provides a superior distribution-independent algorithm with formal error guarantees.
+:::info
+ The Approximate Histogram aggregator is deprecated. Please use [DataSketches Quantiles](../extensions-core/datasketches-quantiles.md) instead which provides a superior distribution-independent algorithm with formal error guarantees.
+:::
This aggregator is based on
[http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf)
diff --git a/docs/development/extensions-core/datasketches-hll.md b/docs/development/extensions-core/datasketches-hll.md
index 86c0724665a6..3312dcc340d7 100644
--- a/docs/development/extensions-core/datasketches-hll.md
+++ b/docs/development/extensions-core/datasketches-hll.md
@@ -47,7 +47,9 @@ For additional sketch types supported in Druid, see [DataSketches extension](dat
|`round`|Round off values to whole numbers. Only affects query-time behavior and is ignored at ingestion-time.|no, defaults to `false`|
|`shouldFinalize`|Return the final double type representing the estimate rather than the intermediate sketch type itself. In addition to controlling the finalization of this aggregator, you can control whether all aggregators are finalized with the query context parameters [`finalize`](../../querying/query-context.md) and [`sqlFinalizeOuterSketches`](../../querying/sql-query-context.md).|no, defaults to `true`|
-> The default `lgK` value has proven to be sufficient for most use cases; expect only very negligible improvements in accuracy with `lgK` values over `16` in normal circumstances.
+:::info
+ The default `lgK` value has proven to be sufficient for most use cases; expect only very negligible improvements in accuracy with `lgK` values over `16` in normal circumstances.
+:::
### HLLSketchBuild aggregator
@@ -65,20 +67,22 @@ For additional sketch types supported in Druid, see [DataSketches extension](dat
The `HLLSketchBuild` aggregator builds an HLL sketch object from the specified input column. When used during ingestion, Druid stores pre-generated HLL sketch objects in the datasource instead of the raw data from the input column.
When applied at query time on an existing dimension, you can use the resulting column as an intermediate dimension by the [post-aggregators](#post-aggregators).
-> It is very common to use `HLLSketchBuild` in combination with [rollup](../../ingestion/rollup.md) to create a [metric](../../ingestion/ingestion-spec.md#metricsspec) on high-cardinality columns. In this example, a metric called `userid_hll` is included in the `metricsSpec`. This will perform a HLL sketch on the `userid` field at ingestion time, allowing for highly-performant approximate `COUNT DISTINCT` query operations and improving roll-up ratios when `userid` is then left out of the `dimensionsSpec`.
->
-> ```
-> "metricsSpec": [
-> {
-> "type": "HLLSketchBuild",
-> "name": "userid_hll",
-> "fieldName": "userid",
-> "lgK": 12,
-> "tgtHllType": "HLL_4"
-> }
-> ]
-> ```
->
+:::info
+ It is very common to use `HLLSketchBuild` in combination with [rollup](../../ingestion/rollup.md) to create a [metric](../../ingestion/ingestion-spec.md#metricsspec) on high-cardinality columns. In this example, a metric called `userid_hll` is included in the `metricsSpec`. This will perform a HLL sketch on the `userid` field at ingestion time, allowing for highly-performant approximate `COUNT DISTINCT` query operations and improving roll-up ratios when `userid` is then left out of the `dimensionsSpec`.
+
+ ```
+ "metricsSpec": [
+ {
+ "type": "HLLSketchBuild",
+ "name": "userid_hll",
+ "fieldName": "userid",
+ "lgK": 12,
+ "tgtHllType": "HLL_4"
+ }
+ ]
+ ```
+
+:::
### HLLSketchMerge aggregator
diff --git a/docs/development/extensions-core/druid-lookups.md b/docs/development/extensions-core/druid-lookups.md
index 3699f94345cc..d6219b8c7428 100644
--- a/docs/development/extensions-core/druid-lookups.md
+++ b/docs/development/extensions-core/druid-lookups.md
@@ -30,10 +30,12 @@ This module can be used side to side with other lookup module like the global ca
To use this Apache Druid extension, [include](../../configuration/extensions.md#loading-extensions) `druid-lookups-cached-single` in the extensions load list.
-> If using JDBC, you will need to add your database's client JAR files to the extension's directory.
-> For Postgres, the connector JAR is already included.
-> See the MySQL extension documentation for instructions to obtain [MySQL](./mysql.md#installing-the-mysql-connector-library) or [MariaDB](./mysql.md#alternative-installing-the-mariadb-connector-library) connector libraries.
-> Copy or symlink the downloaded file to `extensions/druid-lookups-cached-single` under the distribution root directory.
+:::info
+ If using JDBC, you will need to add your database's client JAR files to the extension's directory.
+ For Postgres, the connector JAR is already included.
+ See the MySQL extension documentation for instructions to obtain [MySQL](./mysql.md#installing-the-mysql-connector-library) or [MariaDB](./mysql.md#alternative-installing-the-mariadb-connector-library) connector libraries.
+ Copy or symlink the downloaded file to `extensions/druid-lookups-cached-single` under the distribution root directory.
+:::
## Architecture
Generally speaking this module can be divided into two main component, namely, the data fetcher layer and caching layer.
diff --git a/docs/development/extensions-core/druid-ranger-security.md b/docs/development/extensions-core/druid-ranger-security.md
index 8c2b3b36535d..a78e2efd45a8 100644
--- a/docs/development/extensions-core/druid-ranger-security.md
+++ b/docs/development/extensions-core/druid-ranger-security.md
@@ -26,7 +26,9 @@ This Apache Druid extension adds an Authorizer which implements access control f
Make sure to [include](../../configuration/extensions.md#loading-extensions) `druid-ranger-security` in the extensions load list.
-> The latest release of Apache Ranger is at the time of writing version 2.0. This version has a dependency on `log4j 1.2.17` which has a vulnerability if you configure it to use a `SocketServer` (CVE-2019-17571). Next to that, it also includes Kafka 2.0.0 which has 2 known vulnerabilities (CVE-2019-12399, CVE-2018-17196). Kafka can be used by the audit component in Ranger, but is not required.
+:::info
+ The latest release of Apache Ranger is at the time of writing version 2.0. This version has a dependency on `log4j 1.2.17` which has a vulnerability if you configure it to use a `SocketServer` (CVE-2019-17571). Next to that, it also includes Kafka 2.0.0 which has 2 known vulnerabilities (CVE-2019-12399, CVE-2018-17196). Kafka can be used by the audit component in Ranger, but is not required.
+:::
## Configuration
@@ -67,7 +69,9 @@ druid.escalator.internalClientPassword=password2
druid.escalator.authorizerName=ranger
```
-> Contrary to the documentation of `druid-basic-auth` Ranger does not automatically provision a highly privileged system user, you will need to do this yourself. This system user in the case of `druid-basic-auth` is named `druid_system` and for the escalator it is configurable, as shown above. Make sure to take note of these user names and configure `READ` access to `state:STATE` and to `config:security` in your ranger policies, otherwise system services will not work properly.
+:::info
+ Contrary to the documentation of `druid-basic-auth` Ranger does not automatically provision a highly privileged system user, you will need to do this yourself. This system user in the case of `druid-basic-auth` is named `druid_system` and for the escalator it is configurable, as shown above. Make sure to take note of these user names and configure `READ` access to `state:STATE` and to `config:security` in your ranger policies, otherwise system services will not work properly.
+:::
#### Properties to configure the extension in Apache Druid
|Property|Description|Default|required|
@@ -92,7 +96,9 @@ You should get back `json` describing the service definition you just added. You
When installing a new Druid service in Apache Ranger for the first time, Ranger will provision the policies to allow the administrative user `read/write` access to all properties and data sources. You might want to limit this. Do not forget to add the correct policies for the `druid_system` user and the `internalClientUserName` of the escalator.
-> Loading new data sources requires `write` access to the `datasource` prior to the loading itself. So if you want to create a datasource `wikipedia` you are required to have an `allow` policy inside Apache Ranger before trying to load the spec.
+:::info
+ Loading new data sources requires `write` access to the `datasource` prior to the loading itself. So if you want to create a datasource `wikipedia` you are required to have an `allow` policy inside Apache Ranger before trying to load the spec.
+:::
## Usage
diff --git a/docs/development/extensions-core/hdfs.md b/docs/development/extensions-core/hdfs.md
index b5264db0fb68..32ef6133a9d4 100644
--- a/docs/development/extensions-core/hdfs.md
+++ b/docs/development/extensions-core/hdfs.md
@@ -112,7 +112,7 @@ example properties. Please follow the instructions at
[https://github.com/GoogleCloudPlatform/bigdata-interop/blob/master/gcs/INSTALL.md](https://github.com/GoogleCloudPlatform/bigdata-interop/blob/master/gcs/INSTALL.md)
for more details.
For more configurations, [GCS core default](https://github.com/GoogleCloudDataproc/hadoop-connectors/blob/v2.0.0/gcs/conf/gcs-core-default.xml)
-and [GCS core template](https://github.com/GoogleCloudPlatform/bdutil/blob/master/conf/hadoop2/gcs-core-template.xml).
+and [GCS core template](https://github.com/GoogleCloudDataproc/hadoop-connectors/blob/master/gcs/src/test/resources/core-site.xml).
```xml
@@ -147,8 +147,6 @@ and [GCS core template](https://github.com/GoogleCloudPlatform/bdutil/blob/maste
```
-Tested with Druid 0.17.0, Hadoop 2.8.5 and gcs-connector jar 2.0.0-hadoop2.
-
## Reading data from HDFS or Cloud Storage
### Native batch ingestion
diff --git a/docs/development/extensions-core/kafka-ingestion.md b/docs/development/extensions-core/kafka-ingestion.md
index 46426e55f27c..329967747bfa 100644
--- a/docs/development/extensions-core/kafka-ingestion.md
+++ b/docs/development/extensions-core/kafka-ingestion.md
@@ -135,11 +135,12 @@ The following example demonstrates a supervisor spec for Kafka that uses the `JS
If you want to parse the Kafka metadata fields in addition to the Kafka payload value contents, you can use the `kafka` input format.
The `kafka` input format wraps around the payload parsing input format and augments the data it outputs with the Kafka event timestamp,
-the Kafka event headers, and the key field that itself can be parsed using any available InputFormat.
+the Kafka topic name, the Kafka event headers, and the key field that itself can be parsed using any available InputFormat.
For example, consider the following structure for a Kafka message that represents a fictitious wiki edit in a development environment:
- **Kafka timestamp**: `1680795276351`
+- **Kafka topic**: `wiki-edits`
- **Kafka headers**:
- `env=development`
- `zone=z1`
@@ -153,6 +154,7 @@ You would configure it as follows:
- `valueFormat`: Define how to parse the payload value. Set this to the payload parsing input format (`{ "type": "json" }`).
- `timestampColumnName`: Supply a custom name for the Kafka timestamp in the Druid schema to avoid conflicts with columns from the payload. The default is `kafka.timestamp`.
+- `topicColumnName`: Supply a custom name for the Kafka topic in the Druid schema to avoid conflicts with columns from the payload. The default is `kafka.topic`. This field is useful when ingesting data from multiple topics into same datasource.
- `headerFormat`: The default value `string` decodes strings in UTF-8 encoding from the Kafka header.
Other supported encoding formats include the following:
- `ISO-8859-1`: ISO Latin Alphabet No. 1, that is, ISO-LATIN-1.
@@ -174,7 +176,7 @@ You would configure it as follows:
Note that for `tsv`,`csv`, and `regex` formats, you need to provide a `columns` array to make a valid input format. Only the first one is used, and its name will be ignored in favor of `keyColumnName`.
- `keyColumnName`: Supply the name for the Kafka key column to avoid conflicts with columns from the payload. The default is `kafka.key`.
-Putting it together, the following input format (that uses the default values for `timestampColumnName`, `headerColumnPrefix`, and `keyColumnName`)
+Putting it together, the following input format (that uses the default values for `timestampColumnName`, `topicColumnName`, `headerColumnPrefix`, and `keyColumnName`)
```json
{
@@ -203,6 +205,7 @@ would parse the example message as follows:
"delta": 31,
"namespace": "Main",
"kafka.timestamp": 1680795276351,
+ "kafka.topic": "wiki-edits",
"kafka.header.env": "development",
"kafka.header.zone": "z1",
"kafka.key": "wiki-edit"
@@ -213,7 +216,7 @@ For more information on data formats, see [Data formats](../../ingestion/data-fo
Finally, add these Kafka metadata columns to the `dimensionsSpec` or set your `dimensionsSpec` to auto-detect columns.
-The following supervisor spec demonstrates how to ingest the Kafka header, key, and timestamp into Druid dimensions:
+The following supervisor spec demonstrates how to ingest the Kafka header, key, timestamp, and topic into Druid dimensions:
```
{
@@ -270,15 +273,16 @@ After Druid ingests the data, you can query the Kafka metadata columns as follow
SELECT
"kafka.header.env",
"kafka.key",
- "kafka.timestamp"
+ "kafka.timestamp",
+ "kafka.topic"
FROM "wikiticker"
```
This query returns:
-| `kafka.header.env` | `kafka.key` | `kafka.timestamp` |
-|--------------------|-----------|---------------|
-| `development` | `wiki-edit` | `1680795276351` |
+| `kafka.header.env` | `kafka.key` | `kafka.timestamp` | `kafka.topic` |
+|--------------------|-----------|---------------|---------------|
+| `development` | `wiki-edit` | `1680795276351` | `wiki-edits` |
For more information, see [`kafka` data format](../../ingestion/data-formats.md#kafka).
diff --git a/docs/development/extensions-core/kafka-supervisor-operations.md b/docs/development/extensions-core/kafka-supervisor-operations.md
index e1de35eb2fd0..8504ced595b3 100644
--- a/docs/development/extensions-core/kafka-supervisor-operations.md
+++ b/docs/development/extensions-core/kafka-supervisor-operations.md
@@ -131,6 +131,70 @@ to start and in flight tasks will fail. This operation enables you to recover fr
Note that the supervisor must be running for this endpoint to be available.
+## Resetting Offsets for a Supervisor
+
+The supervisor must be running for this endpoint to be available.
+
+The `POST /druid/indexer/v1/supervisor//resetOffsets` operation clears stored
+offsets, causing the supervisor to start reading from the specified offsets. After resetting stored
+offsets, the supervisor kills and recreates any active tasks pertaining to the specified partitions,
+so that tasks begin reading from specified offsets. For partitions that are not specified in this operation, the supervisor
+will resume from the last stored offset.
+
+Use care when using this operation! Resetting offsets for a supervisor may cause Kafka messages to be skipped or read
+twice, resulting in missing or duplicate data.
+
+#### Sample request
+
+The following example shows how to reset offsets for a kafka supervisor with the name `social_media`. Let's say the supervisor is reading
+from two kafka topics `ads_media_foo` and `ads_media_bar` and has the stored offsets: `{"ads_media_foo:0": 0, "ads_media_foo:1": 10, "ads_media_bar:0": 20, "ads_media_bar:1": 40}`.
+
+
+
+
+
+```shell
+curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/supervisor/social_media/resetOffsets"
+--header 'Content-Type: application/json'
+--data-raw '{"type":"kafka","partitions":{"type":"end","stream":"ads_media_foo|ads_media_bar","partitionOffsetMap":{"ads_media_foo:0": 3, "ads_media_bar:1": 12}}}'
+```
+
+
+
+```HTTP
+POST /druid/indexer/v1/supervisor/social_media/resetOffsets HTTP/1.1
+Host: http://ROUTER_IP:ROUTER_PORT
+Content-Type: application/json
+
+{
+ "type": "kafka",
+ "partitions": {
+ "type": "end",
+ "stream": "ads_media_foo|ads_media_bar",
+ "partitionOffsetMap": {
+ "ads_media_foo:0": 3,
+ "ads_media_bar:1": 12
+ }
+ }
+}
+```
+The above operation will reset offsets for `ads_media_foo` partition 0 and `ads_media_bar` partition 1 to offsets 3 and 12 respectively. After a successful reset,
+when the supervisor's tasks restart, they will resume reading from `{"ads_media_foo:0": 3, "ads_media_foo:1": 10, "ads_media_bar:0": 20, "ads_media_bar:1": 12}`.
+
+
+
+#### Sample response
+
+
+ Click to show sample response
+
+ ```json
+{
+ "id": "social_media"
+}
+ ```
+
+
## Terminating Supervisors
The `POST /druid/indexer/v1/supervisor//terminate` operation terminates a supervisor and causes all
@@ -199,11 +263,11 @@ and begin publishing their segments. A new supervisor will then be started which
will start reading from the offsets where the previous now-publishing tasks left off, but using the updated schema.
In this way, configuration changes can be applied without requiring any pause in ingestion.
-## Deployment Notes on Kafka partitions and Druid segments
+## Deployment notes on Kafka partitions and Druid segments
Druid assigns each Kafka indexing task Kafka partitions. A task writes the events it consumes from Kafka into a single segment for the segment granularity interval until it reaches one of the following: `maxRowsPerSegment`, `maxTotalRows` or `intermediateHandoffPeriod` limit. At this point, the task creates a new partition for this segment granularity to contain subsequent events.
-The Kafka Indexing Task also does incremental hand-offs. Therefore segments become available as they are ready and you do not have to wait for all segments until the end of the task duration. When the task reaches one of `maxRowsPerSegment`, `maxTotalRows`, or `intermediateHandoffPeriod`, it hands off all the segments and creates a new new set of segments will be created for further events. This allows the task to run for longer durations without accumulating old segments locally on Middle Manager processes.
+The Kafka Indexing Task also does incremental hand-offs. Therefore segments become available as they are ready and you do not have to wait for all segments until the end of the task duration. When the task reaches one of `maxRowsPerSegment`, `maxTotalRows`, or `intermediateHandoffPeriod`, it hands off all the segments and creates a new new set of segments will be created for further events. This allows the task to run for longer durations without accumulating old segments locally on Middle Manager processes.
The Kafka Indexing Service may still produce some small segments. For example, consider the following scenario:
- Task duration is 4 hours
diff --git a/docs/development/extensions-core/kafka-supervisor-reference.md b/docs/development/extensions-core/kafka-supervisor-reference.md
index af9f80add894..d141b23477ff 100644
--- a/docs/development/extensions-core/kafka-supervisor-reference.md
+++ b/docs/development/extensions-core/kafka-supervisor-reference.md
@@ -23,73 +23,103 @@ description: "Reference topic for Apache Kafka supervisors"
~ specific language governing permissions and limitations
~ under the License.
-->
-This topic contains configuration reference information for the Apache Kafka supervisor for Apache Druid. The following table outlines the high-level configuration options:
-
-|Field|Description|Required|
-|--------|-----------|---------|
-|`type`|Supervisor type. For Kafka streaming, set to `kafka`.|yes|
-|`spec`| Container object for the supervisor configuration. | yes |
-|`dataSchema`|Schema for the Kafka indexing task to use during ingestion.|yes|
-|`ioConfig`|A `KafkaSupervisorIOConfig` object to define the Kafka connection and I/O-related settings for the supervisor and indexing task. See [KafkaSupervisorIOConfig](#kafkasupervisorioconfig).|yes|
-|`tuningConfig`|A KafkaSupervisorTuningConfig object to define performance-related settings for the supervisor and indexing tasks. See [KafkaSupervisorTuningConfig](#kafkasupervisortuningconfig).|no|
-
-## KafkaSupervisorIOConfig
-
-|Field|Type|Description|Required|
-|-----|----|-----------|--------|
-|`topic`|String|The Kafka topic to read from. Must be a specific topic. Topic patterns are not supported.|yes|
-|`inputFormat`|Object|`inputFormat` to define input data parsing. See [Specifying data format](#specifying-data-format) for details about specifying the input format.|yes|
-|`consumerProperties`|Map|A map of properties to pass to the Kafka consumer. See [More on consumer properties](#more-on-consumerproperties).|yes|
-|`pollTimeout`|Long|The length of time to wait for the Kafka consumer to poll records, in milliseconds|no (default == 100)|
-|`replicas`|Integer|The number of replica sets. "1" means a single set of tasks without replication. Druid always assigns replica tasks to different workers to provide resiliency against worker failure.|no (default == 1)|
-|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. The maximum number of reading tasks equals `taskCount * replicas`. Therefore, the total number of tasks, *reading* + *publishing*, is greater than this count. See [Capacity Planning](./kafka-supervisor-operations.md#capacity-planning) for more details. When `taskCount > {numKafkaPartitions}`, the actual number of reading tasks is less than the `taskCount` value.|no (default == 1)|
-|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing segments.|no (default == PT1H)|
-|`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)|
-|`period`|ISO8601 Period|Frequency at which the supervisor executes its management logic. The supervisor also runs in response to certain events. For example, task success, task failure, and tasks reaching their `taskDuration`. The `period` value specifies the maximum time between iterations.|no (default == PT30S)|
-|`useEarliestOffset`|Boolean|If a supervisor manages a `dataSource` for the first time, it obtains a set of starting offsets from Kafka. This flag determines whether it retrieves the earliest or latest offsets in Kafka. Under normal circumstances, subsequent tasks will start from where the previous segments ended. Therefore Druid only uses `useEarliestOffset` on first run.|no (default == false)|
-|`completionTimeout`|ISO8601 Period|The length of time to wait before declaring a publishing task as failed and terminating it. If the value is too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|no (default == PT30M)|
-|`lateMessageRejectionStartDateTime`|ISO8601 DateTime|Configure tasks to reject messages with timestamps earlier than this date time; for example if this is set to `2016-01-01T11:00Z` and the supervisor creates a task at *2016-01-01T12:00Z*, Druid drops messages with timestamps earlier than *2016-01-01T11:00Z*. This can prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments (e.g. a realtime and a nightly batch ingestion pipeline).|no (default == none)|
-|`lateMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps earlier than this period before the task was created; for example if this is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps earlier than *2016-01-01T11:00Z* will be dropped. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments (e.g. a realtime and a nightly batch ingestion pipeline). Please note that only one of `lateMessageRejectionPeriod` or `lateMessageRejectionStartDateTime` can be specified.|no (default == none)|
-|`earlyMessageRejectionPeriod`|ISO8601 Period|Configure tasks to reject messages with timestamps later than this period after the task reached its taskDuration; for example if this is set to `PT1H`, the taskDuration is set to `PT1H` and the supervisor creates a task at *2016-01-01T12:00Z*, messages with timestamps later than *2016-01-01T14:00Z* will be dropped. **Note:** Tasks sometimes run past their task duration, for example, in cases of supervisor failover. Setting earlyMessageRejectionPeriod too low may cause messages to be dropped unexpectedly whenever a task runs past its originally configured task duration.|no (default == none)|
-|`autoScalerConfig`|Object|Defines auto scaling behavior for Kafka ingest tasks. See [Tasks Autoscaler Properties](#task-autoscaler-properties).|no (default == null)|
-|`idleConfig`|Object|Defines how and when Kafka Supervisor can become idle. See [Idle Supervisor Configuration](#idle-supervisor-configuration) for more details.|no (default == null)|
-
-## Task Autoscaler Properties
-
-| Property | Description | Required |
-| ------------- | ------------- | ------------- |
-| `enableTaskAutoScaler` | Enable or disable autoscaling. `false` or blank disables the `autoScaler` even when `autoScalerConfig` is not null| no (default == false) |
-| `taskCountMax` | Maximum number of ingestion tasks. Set `taskCountMax >= taskCountMin`. If `taskCountMax > {numKafkaPartitions}`, Druid only scales reading tasks up to the `{numKafkaPartitions}`. In this case `taskCountMax` is ignored. | yes |
-| `taskCountMin` | Minimum number of ingestion tasks. When you enable autoscaler, Druid ignores the value of taskCount in `IOConfig` and starts with the `taskCountMin` number of tasks.| yes |
-| `minTriggerScaleActionFrequencyMillis` | Minimum time interval between two scale actions. | no (default == 600000) |
-| `autoScalerStrategy` | The algorithm of `autoScaler`. Only supports `lagBased`. See [Lag Based AutoScaler Strategy Related Properties](#lag-based-autoscaler-strategy-related-properties) for details.| no (default == `lagBased`) |
-
-## Lag Based AutoScaler Strategy Related Properties
-| Property | Description | Required |
-| ------------- | ------------- | ------------- |
-| `lagCollectionIntervalMillis` | Period of lag points collection. | no (default == 30000) |
-| `lagCollectionRangeMillis` | The total time window of lag collection. Use with `lagCollectionIntervalMillis`,it means that in the recent `lagCollectionRangeMillis`, collect lag metric points every `lagCollectionIntervalMillis`. | no (default == 600000) |
-| `scaleOutThreshold` | The threshold of scale out action | no (default == 6000000) |
-| `triggerScaleOutFractionThreshold` | If `triggerScaleOutFractionThreshold` percent of lag points are higher than `scaleOutThreshold`, then do scale out action. | no (default == 0.3) |
-| `scaleInThreshold` | The Threshold of scale in action | no (default == 1000000) |
-| `triggerScaleInFractionThreshold` | If `triggerScaleInFractionThreshold` percent of lag points are lower than `scaleOutThreshold`, then do scale in action. | no (default == 0.9) |
-| `scaleActionStartDelayMillis` | Number of milliseconds after supervisor starts when first check scale logic. | no (default == 300000) |
-| `scaleActionPeriodMillis` | The frequency of checking whether to do scale action in millis | no (default == 60000) |
-| `scaleInStep` | How many tasks to reduce at a time | no (default == 1) |
-| `scaleOutStep` | How many tasks to add at a time | no (default == 2) |
-
-## Idle Supervisor Configuration
-
-> Note that Idle state transitioning is currently designated as experimental.
-
-| Property | Description | Required |
-| ------------- | ------------- | ------------- |
-| `enabled` | If `true`, Kafka supervisor will become idle if there is no data on input stream/topic for some time. | no (default == false) |
-| `inactiveAfterMillis` | Supervisor is marked as idle if all existing data has been read from input topic and no new data has been published for `inactiveAfterMillis` milliseconds. | no (default == `600_000`) |
-
-> When the supervisor enters the idle state, no new tasks will be launched subsequent to the completion of the currently executing tasks. This strategy may lead to reduced costs for cluster operators while using topics that get sporadic data.
-
-The following example demonstrates supervisor spec with `lagBased` autoScaler and idle config enabled:
+
+This topic contains configuration reference information for the Apache Kafka supervisor for Apache Druid.
+
+The following table outlines the high-level configuration options:
+
+|Property|Type|Description|Required|
+|--------|----|-----------|--------|
+|`type`|String|The supervisor type. For Kafka streaming, set to `kafka`.|Yes|
+|`spec`|Object|The container object for the supervisor configuration.|Yes|
+|`ioConfig`|Object|The I/O configuration object to define the Kafka connection and I/O-related settings for the supervisor and indexing task. See [Supervisor I/O configuration](#supervisor-io-configuration).|Yes|
+|`dataSchema`|Object|The schema for the Kafka indexing task to use during ingestion.|Yes|
+|`tuningConfig`|Object|The tuning configuration object to define performance-related settings for the supervisor and indexing tasks. See [Supervisor tuning configuration](#supervisor-tuning-configuration).|No|
+
+## Supervisor I/O configuration
+
+The following table outlines the configuration options for `ioConfig`:
+
+|Property|Type|Description|Required|Default|
+|--------|----|-----------|--------|-------|
+|`topic`|String|The Kafka topic to read from. Must be a specific topic. Druid does not support topic patterns.|Yes||
+|`inputFormat`|Object|The input format to define input data parsing. See [Specifying data format](#specifying-data-format) for details about specifying the input format.|Yes||
+|`consumerProperties`|String, Object|A map of properties to pass to the Kafka consumer. See [Consumer properties](#consumer-properties).|Yes||
+|`pollTimeout`|Long|The length of time to wait for the Kafka consumer to poll records, in milliseconds.|No|100|
+|`replicas`|Integer|The number of replica sets, where 1 is a single set of tasks (no replication). Druid always assigns replicate tasks to different workers to provide resiliency against process failure.|No|1|
+|`taskCount`|Integer|The maximum number of reading tasks in a replica set. The maximum number of reading tasks equals `taskCount * replicas`. The total number of tasks, reading and publishing, is greater than this count. See [Capacity planning](./kafka-supervisor-operations.md#capacity-planning) for more details. When `taskCount > {numKafkaPartitions}`, the actual number of reading tasks is less than the `taskCount` value.|No|1|
+|`taskDuration`|ISO 8601 period|The length of time before tasks stop reading and begin publishing segments.|No|PT1H|
+|`startDelay`|ISO 8601 period|The period to wait before the supervisor starts managing tasks.|No|PT5S|
+|`period`|ISO 8601 period|Determines how often the supervisor executes its management logic. Note that the supervisor also runs in response to certain events, such as tasks succeeding, failing, and reaching their task duration. The `period` value specifies the maximum time between iterations.|No|PT30S|
+|`useEarliestOffset`|Boolean|If a supervisor manages a datasource for the first time, it obtains a set of starting offsets from Kafka. This flag determines whether it retrieves the earliest or latest offsets in Kafka. Under normal circumstances, subsequent tasks start from where the previous segments ended. Druid only uses `useEarliestOffset` on the first run.|No|`false`|
+|`completionTimeout`|ISO 8601 period|The length of time to wait before declaring a publishing task as failed and terminating it. If the value is too low, your tasks may never publish. The publishing clock for a task begins roughly after `taskDuration` elapses.|No|PT30M|
+|`lateMessageRejectionStartDateTime`|ISO 8601 date time|Configure tasks to reject messages with timestamps earlier than this date time. For example, if this property is set to `2016-01-01T11:00Z` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps earlier than `2016-01-01T11:00Z`. This can prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments, such as a realtime and a nightly batch ingestion pipeline.|No||
+|`lateMessageRejectionPeriod`|ISO 8601 period|Configure tasks to reject messages with timestamps earlier than this period before the task was created. For example, if this property is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps earlier than `2016-01-01T11:00Z`. This may help prevent concurrency issues if your data stream has late messages and you have multiple pipelines that need to operate on the same segments, such as a realtime and a nightly batch ingestion pipeline. Note that you can specify only one of the late message rejection properties.|No||
+|`earlyMessageRejectionPeriod`|ISO 8601 period|Configure tasks to reject messages with timestamps later than this period after the task reached its task duration. For example, if this property is set to `PT1H`, the task duration is set to `PT1H` and the supervisor creates a task at `2016-01-01T12:00Z`, Druid drops messages with timestamps later than `2016-01-01T14:00Z`. Tasks sometimes run past their task duration, such as in cases of supervisor failover. Setting `earlyMessageRejectionPeriod` too low may cause Druid to drop messages unexpectedly whenever a task runs past its originally configured task duration.|No||
+|`autoScalerConfig`|Object|Defines auto scaling behavior for Kafka ingest tasks. See [Task autoscaler properties](#task-autoscaler-properties).|No|null|
+|`idleConfig`|Object|Defines how and when the Kafka supervisor can become idle. See [Idle supervisor configuration](#idle-supervisor-configuration) for more details.|No|null|
+
+### Task autoscaler properties
+
+The following table outlines the configuration options for `autoScalerConfig`:
+
+|Property|Description|Required|Default|
+|--------|-----------|--------|-------|
+|`enableTaskAutoScaler`|Enable or disable autoscaling. `false` or blank disables the `autoScaler` even when `autoScalerConfig` is not null.|No|`false`|
+|`taskCountMax`|Maximum number of ingestion tasks. Set `taskCountMax >= taskCountMin`. If `taskCountMax > {numKafkaPartitions}`, Druid only scales reading tasks up to the `{numKafkaPartitions}`. In this case, `taskCountMax` is ignored.|Yes||
+|`taskCountMin`|Minimum number of ingestion tasks. When you enable the autoscaler, Druid ignores the value of `taskCount` in `ioConfig` and starts with the `taskCountMin` number of tasks.|Yes||
+|`minTriggerScaleActionFrequencyMillis`|Minimum time interval between two scale actions.|No|600000|
+|`autoScalerStrategy`|The algorithm of `autoScaler`. Only supports `lagBased`. See [Lag based autoscaler strategy related properties](#lag-based-autoscaler-strategy-related-properties) for details.|No|`lagBased`|
+
+### Lag based autoscaler strategy related properties
+
+The following table outlines the configuration options for `autoScalerStrategy`:
+
+|Property|Description|Required|Default|
+|--------|-----------|--------|-------|
+|`lagCollectionIntervalMillis`|The time period during which Druid collects lag metric points.|No|30000|
+|`lagCollectionRangeMillis`|The total time window of lag collection. Use with `lagCollectionIntervalMillis` to specify the intervals at which to collect lag metric points.|No|600000|
+|`scaleOutThreshold`|The threshold of scale out action.|No|6000000|
+|`triggerScaleOutFractionThreshold`|Enables scale out action if `triggerScaleOutFractionThreshold` percent of lag points is higher than `scaleOutThreshold`.|No|0.3|
+|`scaleInThreshold`|The threshold of scale in action.|No|1000000|
+|`triggerScaleInFractionThreshold`|Enables scale in action if `triggerScaleInFractionThreshold` percent of lag points is lower than `scaleOutThreshold`.|No|0.9|
+|`scaleActionStartDelayMillis`|The number of milliseconds to delay after the supervisor starts before the first scale logic check.|No|300000|
+|`scaleActionPeriodMillis`|The frequency in milliseconds to check if a scale action is triggered.|No|60000|
+|`scaleInStep`|The number of tasks to reduce at once when scaling down.|No|1|
+|`scaleOutStep`|The number of tasks to add at once when scaling out.|No|2|
+
+### Ingesting from multiple topics
+
+To ingest data from multiple topics, you have to set `topicPattern` in the supervisor I/O configuration and not set `topic`.
+You can pass multiple topics as a regex pattern as the value for `topicPattern` in the I/O configuration. For example, to
+ingest data from clicks and impressions, set `topicPattern` to `clicks|impressions` in the I/O configuration.
+Similarly, you can use `metrics-.*` as the value for `topicPattern` if you want to ingest from all the topics that
+start with `metrics-`. If new topics are added to the cluster that match the regex, Druid automatically starts
+ingesting from those new topics. A topic name that only matches partially such as `my-metrics-12` will not be
+included for ingestion. If you enable multi-topic ingestion for a datasource, downgrading to a version older than
+28.0.0 will cause the ingestion for that datasource to fail.
+
+When ingesting data from multiple topics, partitions are assigned based on the hashcode of the topic name and the
+id of the partition within that topic. The partition assignment might not be uniform across all the tasks. It's also
+assumed that partitions across individual topics have similar load. It is recommended that you have a higher number of
+partitions for a high load topic and a lower number of partitions for a low load topic. Assuming that you want to
+ingest from both high and low load topic in the same supervisor.
+
+## Idle supervisor configuration
+
+:::info
+ Note that idle state transitioning is currently designated as experimental.
+:::
+
+|Property|Description|Required|
+|--------|-----------|--------|
+|`enabled`|If `true`, the supervisor becomes idle if there is no data on input stream/topic for some time.|No|`false`|
+|`inactiveAfterMillis`|The supervisor becomes idle if all existing data has been read from input topic and no new data has been published for `inactiveAfterMillis` milliseconds.|No|`600_000`|
+
+When the supervisor enters the idle state, no new tasks are launched subsequent to the completion of the currently executing tasks. This strategy may lead to reduced costs for cluster operators while using topics that get sporadic data.
+
+The following example demonstrates supervisor spec with `lagBased` autoscaler and idle configuration enabled:
+
```json
{
"type": "kafka",
@@ -137,12 +167,12 @@ The following example demonstrates supervisor spec with `lagBased` autoScaler an
}
```
-## More on consumerProperties
+## Consumer properties
Consumer properties must contain a property `bootstrap.servers` with a list of Kafka brokers in the form: `:,:,...`.
By default, `isolation.level` is set to `read_committed`. If you use older versions of Kafka servers without transactions support or don't want Druid to consume only committed transactions, set `isolation.level` to `read_uncommitted`.
-In some cases, you may need to fetch consumer properties at runtime. For example, when `bootstrap.servers` is not known upfront, or is not static. To enable SSL connections, you must provide passwords for `keystore`, `truststore` and `key` secretly. You can provide configurations at runtime with a dynamic config provider implementation like the environment variable config provider that comes with Druid. For more information, see [DynamicConfigProvider](../../operations/dynamic-config-provider.md).
+In some cases, you may need to fetch consumer properties at runtime. For example, when `bootstrap.servers` is not known upfront, or is not static. To enable SSL connections, you must provide passwords for `keystore`, `truststore` and `key` secretly. You can provide configurations at runtime with a dynamic config provider implementation like the environment variable config provider that comes with Druid. For more information, see [Dynamic config provider](../../operations/dynamic-config-provider.md).
For example, if you are using SASL and SSL with Kafka, set the following environment variables for the Druid user on the machines running the Overlord and the Peon services:
@@ -165,16 +195,17 @@ export SSL_TRUSTSTORE_PASSWORD=mysecrettruststorepassword
}
}
```
-Verify that you've changed the values for all configurations to match your own environment. You can use the environment variable config provider syntax in the **Consumer properties** field on the **Connect tab** in the **Load Data** UI in the web console. When connecting to Kafka, Druid replaces the environment variables with their corresponding values.
-Note: You can provide SSL connections with [Password Provider](../../operations/password-provider.md) interface to define the `keystore`, `truststore`, and `key`, but this feature is deprecated.
+Verify that you've changed the values for all configurations to match your own environment. You can use the environment variable config provider syntax in the **Consumer properties** field on the **Connect tab** in the **Load Data** UI in the web console. When connecting to Kafka, Druid replaces the environment variables with their corresponding values.
+
+You can provide SSL connections with [Password provider](../../operations/password-provider.md) interface to define the `keystore`, `truststore`, and `key`, but this feature is deprecated.
## Specifying data format
-Kafka indexing service supports both [`inputFormat`](../../ingestion/data-formats.md#input-format) and [`parser`](../../ingestion/data-formats.md#parser) to specify the data format.
+The Kafka indexing service supports both [`inputFormat`](../../ingestion/data-formats.md#input-format) and [`parser`](../../ingestion/data-formats.md#parser) to specify the data format.
Use the `inputFormat` to specify the data format for Kafka indexing service unless you need a format only supported by the legacy `parser`.
-Supported `inputFormat`s include:
+Druid supports the following input formats:
- `csv`
- `tsv`
@@ -186,62 +217,45 @@ Supported `inputFormat`s include:
For more information, see [Data formats](../../ingestion/data-formats.md). You can also read [`thrift`](../extensions-contrib/thrift.md) formats using `parser`.
-
-
-## KafkaSupervisorTuningConfig
-
-The `tuningConfig` is optional and default parameters will be used if no `tuningConfig` is specified.
-
-| Field | Type | Description | Required |
-|-----------------------------------|----------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------|
-| `type` | String | The indexing task type, this should always be `kafka`. | yes |
-| `maxRowsInMemory` | Integer | The number of rows to aggregate before persisting. This number is the post-aggregation rows, so it is not equivalent to the number of input events, but the number of aggregated rows that those events result in. This is used to manage the required JVM heap size. Maximum heap memory usage for indexing scales with `maxRowsInMemory` * (2 + `maxPendingPersists`). Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set. | no (default == 150000) |
-| `maxBytesInMemory` | Long | The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally this is computed internally and user does not need to set it. The maximum heap memory usage for indexing is `maxBytesInMemory` * (2 + `maxPendingPersists`). | no (default == One-sixth of max JVM memory) |
-| `maxRowsPerSegment` | Integer | The number of rows to aggregate into a segment; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier. | no (default == 5000000) |
-| `maxTotalRows` | Long | The number of rows to aggregate across all segments; this number is post-aggregation rows. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier. | no (default == 20000000) |
-| `intermediatePersistPeriod` | ISO8601 Period | The period that determines the rate at which intermediate persists occur. | no (default == PT10M) |
-| `maxPendingPersists` | Integer | Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with `maxRowsInMemory` * (2 + `maxPendingPersists`). | no (default == 0, meaning one persist can be running concurrently with ingestion, and none can be queued up) |
-| `indexSpec` | Object | Tune how data is indexed. See [IndexSpec](#indexspec) for more information. | no |
-| `indexSpecForIntermediatePersists`| | Defines segment storage format options to be used at indexing time for intermediate persisted temporary segments. This can be used to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published, see [IndexSpec](#indexspec) for possible values. | no (default = same as `indexSpec`) |
-| `reportParseExceptions` | Boolean | *DEPRECATED*. If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped. Setting `reportParseExceptions` to true will override existing configurations for `maxParseExceptions` and `maxSavedParseExceptions`, setting `maxParseExceptions` to 0 and limiting `maxSavedParseExceptions` to no more than 1. | no (default == false) |
-| `handoffConditionTimeout` | Long | Number of milliseconds to wait for segment handoff. Set to a value >= 0, where 0 means to wait indefinitely. | no (default == 900000 [15 minutes]) |
-| `resetOffsetAutomatically` | Boolean | Controls behavior when Druid needs to read Kafka messages that are no longer available (i.e. when `OffsetOutOfRangeException` is encountered).
If false, the exception will bubble up, which will cause your tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation; potentially using the [Reset Supervisor API](../../api-reference/supervisor-api.md). This mode is useful for production, since it will make you aware of issues with ingestion.
If true, Druid will automatically reset to the earlier or latest offset available in Kafka, based on the value of the `useEarliestOffset` property (earliest if true, latest if false). Note that this can lead to data being _DROPPED_ (if `useEarliestOffset` is false) or _DUPLICATED_ (if `useEarliestOffset` is true) without your knowledge. Messages will be logged indicating that a reset has occurred, but ingestion will continue. This mode is useful for non-production situations, since it will make Druid attempt to recover from problems automatically, even if they lead to quiet dropping or duplicating of data.
This feature behaves similarly to the Kafka `auto.offset.reset` consumer property. | no (default == false) |
-| `workerThreads` | Integer | The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation. | no (default == min(10, taskCount)) |
-| `chatRetries` | Integer | The number of times HTTP requests to indexing tasks will be retried before considering tasks unresponsive. | no (default == 8) |
-| `httpTimeout` | ISO8601 Period | How long to wait for a HTTP response from an indexing task. | no (default == PT10S) |
-| `shutdownTimeout` | ISO8601 Period | How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting. | no (default == PT80S) |
-| `offsetFetchPeriod` | ISO8601 Period | How often the supervisor queries Kafka and the indexing tasks to fetch current offsets and calculate lag. If the user-specified value is below the minimum value (`PT5S`), the supervisor ignores the value and uses the minimum value instead. | no (default == PT30S, min == PT5S) |
-| `segmentWriteOutMediumFactory` | Object | Segment write-out medium to use when creating segments. See below for more information. | no (not specified by default, the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type` is used) |
-| `intermediateHandoffPeriod` | ISO8601 Period | How often the tasks should hand off segments. Handoff will happen either if `maxRowsPerSegment` or `maxTotalRows` is hit or every `intermediateHandoffPeriod`, whichever happens earlier. | no (default == P2147483647D) |
-| `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. Overridden if `reportParseExceptions` is set. | no, unlimited default |
-| `maxSavedParseExceptions` | Integer | When a parse exception occurs, Druid can keep track of the most recent parse exceptions. `maxSavedParseExceptions` limits how many exception instances will be saved. These saved exceptions will be made available after the task finishes in the [task completion report](../../ingestion/tasks.md#task-reports). Overridden if `reportParseExceptions` is set. | no, default == 0 |
-
-#### IndexSpec
-
-|Field|Type|Description|Required|
-|-----|----|-----------|--------|
-|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object. See [Bitmap types](#bitmap-types) below for options.|no (defaults to Roaring)|
-|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, `ZSTD` or `uncompressed`.|no (default == `LZ4`)|
-|metricCompression|String|Compression format for primitive type metric columns. Choose from `LZ4`, `LZF`, `ZSTD`, `uncompressed` or `none`.|no (default == `LZ4`)|
-|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)|
-
-##### Bitmap types
-
-For Roaring bitmaps:
-
-|Field|Type|Description|Required|
-|-----|----|-----------|--------|
-|`type`|String|Must be `roaring`.|yes|
-
-For Concise bitmaps:
-
-|Field|Type|Description|Required|
-|-----|----|-----------|--------|
-|`type`|String|Must be `concise`.|yes|
-
-#### SegmentWriteOutMediumFactory
-
-|Field|Type|Description|Required|
-|-----|----|-----------|--------|
-|`type`|String|See [Additional Peon Configuration: SegmentWriteOutMediumFactory](../../configuration/index.md#segmentwriteoutmediumfactory) for explanation and available options.|yes|
+## Supervisor tuning configuration
+
+The `tuningConfig` object is optional. If you don't specify the `tuningConfig` object, Druid uses the default configuration settings.
+
+|Property|Type|Description|Required|Default|
+|--------|----|-----------|--------|-------|
+|`type`|String|The indexing task type. This should always be `kafka`.|Yes||
+|`maxRowsInMemory`|Integer|The number of rows to aggregate before persisting. This number represents the post-aggregation rows. It is not equivalent to the number of input events, but the resulting number of aggregated rows. Druid uses `maxRowsInMemory` to manage the required JVM heap size. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`. Normally, you do not need to set this, but depending on the nature of data, if rows are short in terms of bytes, you may not want to store a million rows in memory and this value should be set.|No|150000|
+|`maxBytesInMemory`|Long|The number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. Normally, this is computed internally. The maximum heap memory usage for indexing is `maxBytesInMemory * (2 + maxPendingPersists)`.|No|One-sixth of max JVM memory|
+|`skipBytesInMemoryOverheadCheck`|Boolean|The calculation of `maxBytesInMemory` takes into account overhead objects created during ingestion and each intermediate persist. To exclude the bytes of these overhead objects from the `maxBytesInMemory` check, set `skipBytesInMemoryOverheadCheck` to `true`.|No|`false`|
+|`maxRowsPerSegment`|Integer|The number of rows to store in a segment. This number is post-aggregation rows. Handoff occurs when `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|5000000|
+|`maxTotalRows`|Long|The number of rows to aggregate across all segments; this number is post-aggregation rows. Handoff happens either if `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens earlier.|No|20000000|
+|`intermediateHandoffPeriod`|ISO 8601 period|The period that determines how often tasks hand off segments. Handoff occurs if `maxRowsPerSegment` or `maxTotalRows` is reached or every `intermediateHandoffPeriod`, whichever happens first.|No|P2147483647D|
+|`intermediatePersistPeriod`|ISO 8601 period|The period that determines the rate at which intermediate persists occur.|No|PT10M|
+|`maxPendingPersists`|Integer|Maximum number of persists that can be pending but not started. If a new intermediate persist exceeds this limit, Druid blocks ingestion until the currently running persist finishes. One persist can be running concurrently with ingestion, and none can be queued up. The maximum heap memory usage for indexing scales is `maxRowsInMemory * (2 + maxPendingPersists)`.|No|0|
+|`indexSpec`|Object|Defines how Druid indexes the data. See [IndexSpec](#indexspec) for more information.|No||
+|`indexSpecForIntermediatePersists`|Object|Defines segment storage format options to use at indexing time for intermediate persisted temporary segments. You can use `indexSpecForIntermediatePersists` to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. However, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published. See [IndexSpec](#indexspec) for possible values.|No|Same as `indexSpec`|
+|`reportParseExceptions`|Boolean|DEPRECATED. If `true`, Druid throws exceptions encountered during parsing causing ingestion to halt. If `false`, Druid skips unparseable rows and fields. Setting `reportParseExceptions` to `true` overrides existing configurations for `maxParseExceptions` and `maxSavedParseExceptions`, setting `maxParseExceptions` to 0 and limiting `maxSavedParseExceptions` to not more than 1.|No|`false`|
+|`handoffConditionTimeout`|Long|Number of milliseconds to wait for segment handoff. Set to a value >= 0, where 0 means to wait indefinitely.|No|900000 (15 minutes)|
+|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read Kafka messages that are no longer available, when `offsetOutOfRangeException` is encountered. If `false`, the exception bubbles up causing tasks to fail and ingestion to halt. If this occurs, manual intervention is required to correct the situation, potentially using the [Reset Supervisor API](../../api-reference/supervisor-api.md). This mode is useful for production, since it will make you aware of issues with ingestion. If `true`, Druid will automatically reset to the earlier or latest offset available in Kafka, based on the value of the `useEarliestOffset` property (earliest if `true`, latest if `false`). Note that this can lead to dropping data (if `useEarliestSequenceNumber` is `false`) or duplicating data (if `useEarliestSequenceNumber` is `true`) without your knowledge. Druid logs messages indicating that a reset has occurred without interrupting ingestion. This mode is useful for non-production situations since it enables Druid to recover from problems automatically, even if they lead to quiet dropping or duplicating of data. This feature behaves similarly to the Kafka `auto.offset.reset` consumer property.|No|`false`|
+|`workerThreads`|Integer|The number of threads that the supervisor uses to handle requests/responses for worker tasks, along with any other internal asynchronous operation.|No|`min(10, taskCount)`|
+|`chatAsync`|Boolean|If `true`, use asynchronous communication with indexing tasks, and ignore the `chatThreads` parameter. If `false`, use synchronous communication in a thread pool of size `chatThreads`.|No|`true`|
+|`chatThreads`|Integer|The number of threads to use for communicating with indexing tasks. Ignored if `chatAsync` is `true`.|No|`min(10, taskCount * replicas)`|
+|`chatRetries`|Integer|The number of times HTTP requests to indexing tasks are retried before considering tasks unresponsive.|No|8|
+|`httpTimeout`| ISO 8601 period|The period of time to wait for a HTTP response from an indexing task.|No|PT10S|
+|`shutdownTimeout`|ISO 8601 period|The period of time to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|No|PT80S|
+|`offsetFetchPeriod`|ISO 8601 period|Determines how often the supervisor queries Kafka and the indexing tasks to fetch current offsets and calculate lag. If the user-specified value is below the minimum value of `PT5S`, the supervisor ignores the value and uses the minimum value instead.|No|PT30S|
+|`segmentWriteOutMediumFactory`|Object|The segment write-out medium to use when creating segments. See [Additional Peon configuration: SegmentWriteOutMediumFactory](../../configuration/index.md#segmentwriteoutmediumfactory) for explanation and available options.|No|If not specified, Druid uses the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type`.|
+|`logParseExceptions`|Boolean|If `true`, Druid logs an error message when a parsing exception occurs, containing information about the row where the error occurred.|No|`false`|
+|`maxParseExceptions`|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overridden if `reportParseExceptions` is set.|No|unlimited|
+|`maxSavedParseExceptions`|Integer|When a parse exception occurs, Druid keeps track of the most recent parse exceptions. `maxSavedParseExceptions` limits the number of saved exception instances. These saved exceptions are available after the task finishes in the [task completion report](../../ingestion/tasks.md#task-reports). Overridden if `reportParseExceptions` is set.|No|0|
+
+### IndexSpec
+
+The following table outlines the configuration options for `indexSpec`:
+
+|Property|Type|Description|Required|Default|
+|--------|----|-----------|--------|-------|
+|`bitmap`|Object|Compression format for bitmap indexes. Druid supports roaring and concise bitmap types.|No|Roaring|
+|`dimensionCompression`|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, `ZSTD` or `uncompressed`.|No|`LZ4`|
+|`metricCompression`|String|Compression format for primitive type metric columns. Choose from `LZ4`, `LZF`, `ZSTD`, `uncompressed` or `none`.|No|`LZ4`|
+|`longEncoding`|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|No|`longs`|
\ No newline at end of file
diff --git a/docs/development/extensions-core/kinesis-ingestion.md b/docs/development/extensions-core/kinesis-ingestion.md
index 547333317f17..0c3066d2f79d 100644
--- a/docs/development/extensions-core/kinesis-ingestion.md
+++ b/docs/development/extensions-core/kinesis-ingestion.md
@@ -3,8 +3,12 @@ id: kinesis-ingestion
title: "Amazon Kinesis ingestion"
sidebar_label: "Amazon Kinesis"
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
+
+
+
+
-
```shell
curl -X POST "http://SERVICE_IP:SERVICE_PORT/druid/indexer/v1/supervisor" \
-H "Content-Type: application/json" \
@@ -135,7 +140,9 @@ curl -X POST "http://SERVICE_IP:SERVICE_PORT/druid/indexer/v1/supervisor" \
}
}'
```
-
+
+
+
```HTTP
POST /druid/indexer/v1/supervisor
HTTP/1.1
@@ -213,7 +220,8 @@ Content-Type: application/json
}
}
```
-
+
+
## Supervisor I/O configuration
@@ -428,14 +436,26 @@ This section describes how to use the [Supervisor API](../../api-reference/super
### AWS authentication
-To authenticate with AWS, you must provide your AWS access key and AWS secret key using `runtime.properties`, for example:
+Druid uses AWS access and secret keys to authenticate Kinesis API requests. There are a few ways to provide this information to Druid:
-```text
+1. Using roles or short-term credentials:
+
+ Druid looks for credentials set in [environment variables](https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-envvars.html),
+via [Web Identity Token](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles_providers_oidc.html), in the
+default [profile configuration file](https://docs.aws.amazon.com/cli/latest/userguide/cli-configure-files.html), and from the
+EC2 instance profile provider (in this order).
+
+2. Using long-term security credentials:
+
+ You can directly provide your AWS access key and AWS secret key in the `common.runtime.properties` file as shown in the example below:
+
+```properties
druid.kinesis.accessKey=AKIAWxxxxxxxxxx4NCKS
druid.kinesis.secretKey=Jbytxxxxxxxxxxx2+555
```
-Druid uses the AWS access key and AWS secret key to authenticate Kinesis API requests. If not provided, the service looks for credentials set in environment variables, via [Web Identity Token](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles_providers_oidc.html), in the default profile configuration file, and from the EC2 instance profile provider (in this order).
+> Note: AWS does not recommend providing long-term security credentials in configuration files since it might pose a security risk.
+If you use this approach, it takes precedence over all other methods of providing credentials.
To ingest data from Kinesis, ensure that the policy attached to your IAM role contains the necessary permissions.
The required permissions depend on the value of `useListShards`.
@@ -482,7 +502,7 @@ The following is an example policy:
},
{
"Effect": "Allow",
- "Action": ["kinesis:DescribeStreams"],
+ "Action": ["kinesis:DescribeStream"],
"Resource": ["*"]
},
{
@@ -573,6 +593,16 @@ for the generated segments to be accepted. If the messages at the expected start
no longer available in Kinesis (typically because the message retention period has elapsed or the topic was
removed and re-created) the supervisor will refuse to start and in-flight tasks will fail. This endpoint enables you to recover from this condition.
+### Resetting Offsets for a supervisor
+
+To reset partition offsets for a supervisor, send a `POST` request to the `/druid/indexer/v1/supervisor/:supervisorId/resetOffsets` endpoint. This endpoint clears stored
+sequence numbers, prompting the supervisor to start reading from the specified offsets.
+After resetting stored offsets, the supervisor kills and recreates any active tasks pertaining to the specified partitions,
+so that tasks begin reading specified offsets. For partitions that are not specified in this operation, the supervisor will resume from the last
+stored offset.
+
+Use this endpoint with caution as it may result in skipped messages, leading to data loss or duplicate data.
+
### Terminate a supervisor
To terminate a supervisor and its associated indexing tasks, send a `POST` request to the `/druid/indexer/v1/supervisor/:supervisorId/terminate` endpoint.
diff --git a/docs/development/extensions-core/lookups-cached-global.md b/docs/development/extensions-core/lookups-cached-global.md
index 7e9d80d7ec2f..ebeca5a741e0 100644
--- a/docs/development/extensions-core/lookups-cached-global.md
+++ b/docs/development/extensions-core/lookups-cached-global.md
@@ -25,8 +25,10 @@ title: "Globally Cached Lookups"
To use this Apache Druid extension, [include](../../configuration/extensions.md#loading-extensions) `druid-lookups-cached-global` in the extensions load list.
## Configuration
-> Static configuration is no longer supported. Lookups can be configured through
-> [dynamic configuration](../../querying/lookups.md#configuration).
+:::info
+ Static configuration is no longer supported. Lookups can be configured through
+ [dynamic configuration](../../querying/lookups.md#configuration).
+:::
Globally cached lookups are appropriate for lookups which are not possible to pass at query time due to their size,
or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers,
@@ -369,11 +371,13 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol
}
```
-> If using JDBC, you will need to add your database's client JAR files to the extension's directory.
-> For Postgres, the connector JAR is already included.
-> See the MySQL extension documentation for instructions to obtain [MySQL](./mysql.md#installing-the-mysql-connector-library) or [MariaDB](./mysql.md#alternative-installing-the-mariadb-connector-library) connector libraries.
-> The connector JAR should reside in the classpath of Druid's main class loader.
-> To add the connector JAR to the classpath, you can copy the downloaded file to `lib/` under the distribution root directory. Alternatively, create a symbolic link to the connector in the `lib` directory.
+:::info
+ If using JDBC, you will need to add your database's client JAR files to the extension's directory.
+ For Postgres, the connector JAR is already included.
+ See the MySQL extension documentation for instructions to obtain [MySQL](./mysql.md#installing-the-mysql-connector-library) or [MariaDB](./mysql.md#alternative-installing-the-mariadb-connector-library) connector libraries.
+ The connector JAR should reside in the classpath of Druid's main class loader.
+ To add the connector JAR to the classpath, you can copy the downloaded file to `lib/` under the distribution root directory. Alternatively, create a symbolic link to the connector in the `lib` directory.
+:::
## Introspection
diff --git a/docs/development/extensions-core/mysql.md b/docs/development/extensions-core/mysql.md
index 5e08c7f5f3c9..9b8687f6b1c9 100644
--- a/docs/development/extensions-core/mysql.md
+++ b/docs/development/extensions-core/mysql.md
@@ -25,8 +25,10 @@ title: "MySQL Metadata Store"
To use this Apache Druid extension, [include](../../configuration/extensions.md#loading-extensions) `mysql-metadata-storage` in the extensions load list.
-> The MySQL extension requires the MySQL Connector/J library or MariaDB Connector/J library, neither of which are included in the Druid distribution.
-> Refer to the following section for instructions on how to install this library.
+:::info
+ The MySQL extension requires the MySQL Connector/J library or MariaDB Connector/J library, neither of which are included in the Druid distribution.
+ Refer to the following section for instructions on how to install this library.
+:::
## Installing the MySQL connector library
@@ -76,7 +78,7 @@ This extension also supports using MariaDB server, https://mariadb.org/download/
Connect to MySQL from the machine where it is installed.
```bash
- > mysql -u root
+ mysql -u root
```
Paste the following snippet into the mysql prompt:
diff --git a/docs/development/extensions-core/stats.md b/docs/development/extensions-core/stats.md
index 917d3dcdd9fe..be4121369a13 100644
--- a/docs/development/extensions-core/stats.md
+++ b/docs/development/extensions-core/stats.md
@@ -47,12 +47,14 @@ This algorithm was proven to be numerically stable by J.L. Barlow in
"Error analysis of a pairwise summation algorithm to compute sample variance"
Numer. Math, 58 (1991) pp. 583--590
-> As with all [aggregators](../../querying/sql-aggregations.md), the order of operations across segments is
-> non-deterministic. This means that if this aggregator operates with an input type of "float" or "double", the result
-> of the aggregation may not be precisely the same across multiple runs of the query.
->
-> To produce consistent results, round the variance to a fixed number of decimal places so that the results are
-> precisely the same across query runs.
+:::info
+ As with all [aggregators](../../querying/sql-aggregations.md), the order of operations across segments is
+ non-deterministic. This means that if this aggregator operates with an input type of "float" or "double", the result
+ of the aggregation may not be precisely the same across multiple runs of the query.
+
+ To produce consistent results, round the variance to a fixed number of decimal places so that the results are
+ precisely the same across query runs.
+:::
### Pre-aggregating variance at ingestion time
diff --git a/docs/development/modules.md b/docs/development/modules.md
index a0d2335194de..75f4bbbe5461 100644
--- a/docs/development/modules.md
+++ b/docs/development/modules.md
@@ -351,12 +351,12 @@ This config file adds the configs below to enable a custom coordinator duty.
```
druid.coordinator.dutyGroups=["cleanupMetadata"]
druid.coordinator.cleanupMetadata.duties=["killSupervisors"]
-druid.coordinator.cleanupMetadata.duty.killSupervisors.retainDuration=PT0M
+druid.coordinator.cleanupMetadata.duty.killSupervisors.durationToRetain=PT0M
druid.coordinator.cleanupMetadata.period=PT10S
```
These configurations create a custom coordinator duty group called `cleanupMetadata` which runs a custom coordinator duty called `killSupervisors` every 10 seconds.
-The custom coordinator duty `killSupervisors` also has a config called `retainDuration` which is set to 0 minute.
+The custom coordinator duty `killSupervisors` also has a config called `durationToRetain` which is set to 0 minute.
### Routing data through a HTTP proxy for your extension
diff --git a/docs/ingestion/data-formats.md b/docs/ingestion/data-formats.md
index f3ac4d90bd32..7dd1b10c7fa9 100644
--- a/docs/ingestion/data-formats.md
+++ b/docs/ingestion/data-formats.md
@@ -77,7 +77,9 @@ parsing data is less efficient than writing a native Java parser or using an ext
You can use the `inputFormat` field to specify the data format for your input data.
-> `inputFormat` doesn't support all data formats or ingestion methods supported by Druid.
+:::info
+ `inputFormat` doesn't support all data formats or ingestion methods supported by Druid.
+:::
Especially if you want to use the Hadoop ingestion, you still need to use the [Parser](#parser).
If your data is formatted in some format not listed in this section, please consider using the Parser instead.
@@ -167,7 +169,9 @@ For example:
### ORC
To use the ORC input format, load the Druid Orc extension ( [`druid-orc-extensions`](../development/extensions-core/orc.md)).
-> To upgrade from versions earlier than 0.15.0 to 0.15.0 or new, read [Migration from 'contrib' extension](../development/extensions-core/orc.md#migration-from-contrib-extension).
+:::info
+ To upgrade from versions earlier than 0.15.0 to 0.15.0 or new, read [Migration from 'contrib' extension](../development/extensions-core/orc.md#migration-from-contrib-extension).
+:::
Configure the ORC `inputFormat` to load ORC data as follows:
@@ -289,9 +293,11 @@ If `type` is not included, the avroBytesDecoder defaults to `schema_repo`.
###### Inline Schema Based Avro Bytes Decoder
-> The "schema_inline" decoder reads Avro records using a fixed schema and does not support schema migration. If you
-> may need to migrate schemas in the future, consider one of the other decoders, all of which use a message header that
-> allows the parser to identify the proper Avro schema for reading records.
+:::info
+ The "schema_inline" decoder reads Avro records using a fixed schema and does not support schema migration. If you
+ may need to migrate schemas in the future, consider one of the other decoders, all of which use a message header that
+ allows the parser to identify the proper Avro schema for reading records.
+:::
This decoder can be used if all the input events can be read using the same schema. In this case, specify the schema in the input task JSON itself, as described below.
@@ -503,7 +509,9 @@ For example:
### Protobuf
-> You need to include the [`druid-protobuf-extensions`](../development/extensions-core/protobuf.md) as an extension to use the Protobuf input format.
+:::info
+ You need to include the [`druid-protobuf-extensions`](../development/extensions-core/protobuf.md) as an extension to use the Protobuf input format.
+:::
Configure the Protobuf `inputFormat` to load Protobuf data as follows:
@@ -552,17 +560,19 @@ Configure the Kafka `inputFormat` as follows:
| `type` | String | Set value to `kafka`. | yes |
| `valueFormat` | [InputFormat](#input-format) | Any [InputFormat](#input-format) to parse the Kafka value payload. For details about specifying the input format, see [Specifying data format](../development/extensions-core/kafka-supervisor-reference.md#specifying-data-format). | yes |
| `timestampColumnName` | String | Name of the column for the kafka record's timestamp.| no (default = "kafka.timestamp") |
+| `topicColumnName` | String |Name of the column for the kafka record's topic. It is useful when ingesting data from multiple topics.| no (default = "kafka.timestamp") |
| `headerColumnPrefix` | String | Custom prefix for all the header columns. | no (default = "kafka.header.") |
| `headerFormat` | Object | `headerFormat` specifies how to parse the Kafka headers. Supports String types. Because Kafka header values are bytes, the parser decodes them as UTF-8 encoded strings. To change this behavior, implement your own parser based on the encoding style. Change the 'encoding' type in `KafkaStringHeaderFormat` to match your custom implementation. | no |
| `keyFormat` | [InputFormat](#input-format) | Any [input format](#input-format) to parse the Kafka key. It only processes the first entry of the `inputFormat` field. For details, see [Specifying data format](../development/extensions-core/kafka-supervisor-reference.md#specifying-data-format). | no |
| `keyColumnName` | String | Name of the column for the kafka record's key.| no (default = "kafka.key") |
+
The Kafka input format augments the payload with information from the Kafka timestamp, headers, and key.
If there are conflicts between column names in the payload and those created from the metadata, the payload takes precedence.
This ensures that upgrading a Kafka ingestion to use the Kafka input format (by taking its existing input format and setting it as the `valueFormat`) can be done without losing any of the payload data.
-Here is a minimal example that only augments the parsed payload with the Kafka timestamp column:
+Here is a minimal example that only augments the parsed payload with the Kafka timestamp column and kafka topic column:
```
"ioConfig": {
@@ -586,6 +596,7 @@ Here is a complete example:
"type": "json"
}
"timestampColumnName": "kafka.timestamp",
+ "topicColumnName": "kafka.topic",
"headerFormat": {
"type": "string",
"encoding": "UTF-8"
@@ -686,9 +697,11 @@ Each entry in the `fields` list can have the following components:
## Parser
-> The Parser is deprecated for [native batch tasks](./native-batch.md), [Kafka indexing service](../development/extensions-core/kafka-ingestion.md),
+:::info
+ The Parser is deprecated for [native batch tasks](./native-batch.md), [Kafka indexing service](../development/extensions-core/kafka-ingestion.md),
and [Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md).
Consider using the [input format](#input-format) instead for these types of ingestion.
+:::
This section lists all default and core extension parsers.
For community extension parsers, please see our [community extensions list](../configuration/extensions.md#community-extensions).
@@ -705,9 +718,13 @@ Each line can be further parsed using [`parseSpec`](#parsespec).
### Avro Hadoop Parser
-> You need to include the [`druid-avro-extensions`](../development/extensions-core/avro.md) as an extension to use the Avro Hadoop Parser.
+:::info
+ You need to include the [`druid-avro-extensions`](../development/extensions-core/avro.md) as an extension to use the Avro Hadoop Parser.
+:::
-> See the [Avro Types](../development/extensions-core/avro.md#avro-types) section for how Avro types are handled in Druid
+:::info
+ See the [Avro Types](../development/extensions-core/avro.md#avro-types) section for how Avro types are handled in Druid
+:::
This parser is for [Hadoop batch ingestion](./hadoop.md).
The `inputFormat` of `inputSpec` in `ioConfig` must be set to `"org.apache.druid.data.input.avro.AvroValueInputFormat"`.
@@ -764,10 +781,14 @@ For example, using Avro Hadoop parser with custom reader's schema file:
### ORC Hadoop Parser
-> You need to include the [`druid-orc-extensions`](../development/extensions-core/orc.md) as an extension to use the ORC Hadoop Parser.
+:::info
+ You need to include the [`druid-orc-extensions`](../development/extensions-core/orc.md) as an extension to use the ORC Hadoop Parser.
+:::
-> If you are considering upgrading from earlier than 0.15.0 to 0.15.0 or a higher version,
-> please read [Migration from 'contrib' extension](../development/extensions-core/orc.md#migration-from-contrib-extension) carefully.
+:::info
+ If you are considering upgrading from earlier than 0.15.0 to 0.15.0 or a higher version,
+ please read [Migration from 'contrib' extension](../development/extensions-core/orc.md#migration-from-contrib-extension) carefully.
+:::
This parser is for [Hadoop batch ingestion](./hadoop.md).
The `inputFormat` of `inputSpec` in `ioConfig` must be set to `"org.apache.orc.mapreduce.OrcInputFormat"`.
@@ -1005,7 +1026,9 @@ setting `"mapreduce.job.user.classpath.first": "true"`, then this will not be an
### Parquet Hadoop Parser
-> You need to include the [`druid-parquet-extensions`](../development/extensions-core/parquet.md) as an extension to use the Parquet Hadoop Parser.
+:::info
+ You need to include the [`druid-parquet-extensions`](../development/extensions-core/parquet.md) as an extension to use the Parquet Hadoop Parser.
+:::
The Parquet Hadoop parser is for [Hadoop batch ingestion](./hadoop.md) and parses Parquet files directly.
The `inputFormat` of `inputSpec` in `ioConfig` must be set to `org.apache.druid.data.input.parquet.DruidParquetInputFormat`.
@@ -1147,12 +1170,16 @@ However, the Parquet Avro Hadoop Parser was the original basis for supporting th
### Parquet Avro Hadoop Parser
-> Consider using the [Parquet Hadoop Parser](#parquet-hadoop-parser) over this parser to ingest
+:::info
+ Consider using the [Parquet Hadoop Parser](#parquet-hadoop-parser) over this parser to ingest
Parquet files. See [Parquet Hadoop Parser vs Parquet Avro Hadoop Parser](#parquet-hadoop-parser-vs-parquet-avro-hadoop-parser)
for the differences between those parsers.
+:::
-> You need to include both the [`druid-parquet-extensions`](../development/extensions-core/parquet.md)
+:::info
+ You need to include both the [`druid-parquet-extensions`](../development/extensions-core/parquet.md)
[`druid-avro-extensions`] as extensions to use the Parquet Avro Hadoop Parser.
+:::
The Parquet Avro Hadoop Parser is for [Hadoop batch ingestion](./hadoop.md).
This parser first converts the Parquet data into Avro records, and then parses them to ingest into Druid.
@@ -1234,9 +1261,13 @@ an explicitly defined [format](http://www.joda.org/joda-time/apidocs/org/joda/ti
### Avro Stream Parser
-> You need to include the [`druid-avro-extensions`](../development/extensions-core/avro.md) as an extension to use the Avro Stream Parser.
+:::info
+ You need to include the [`druid-avro-extensions`](../development/extensions-core/avro.md) as an extension to use the Avro Stream Parser.
+:::
-> See the [Avro Types](../development/extensions-core/avro.md#avro-types) section for how Avro types are handled in Druid
+:::info
+ See the [Avro Types](../development/extensions-core/avro.md#avro-types) section for how Avro types are handled in Druid
+:::
This parser is for [stream ingestion](./index.md#streaming) and reads Avro data from a stream directly.
@@ -1276,7 +1307,9 @@ For example, using Avro stream parser with schema repo Avro bytes decoder:
### Protobuf Parser
-> You need to include the [`druid-protobuf-extensions`](../development/extensions-core/protobuf.md) as an extension to use the Protobuf Parser.
+:::info
+ You need to include the [`druid-protobuf-extensions`](../development/extensions-core/protobuf.md) as an extension to use the Protobuf Parser.
+:::
This parser is for [stream ingestion](./index.md#streaming) and reads Protocol buffer data from a stream directly.
@@ -1430,9 +1463,11 @@ Multiple Instances:
## ParseSpec
-> The Parser is deprecated for [native batch tasks](./native-batch.md), [Kafka indexing service](../development/extensions-core/kafka-ingestion.md),
+:::info
+ The Parser is deprecated for [native batch tasks](./native-batch.md), [Kafka indexing service](../development/extensions-core/kafka-ingestion.md),
and [Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md).
Consider using the [input format](#input-format) instead for these types of ingestion.
+:::
ParseSpecs serve two purposes:
@@ -1468,7 +1503,9 @@ Sample spec:
### JSON Lowercase ParseSpec
-> The _jsonLowercase_ parser is deprecated and may be removed in a future version of Druid.
+:::info
+ The _jsonLowercase_ parser is deprecated and may be removed in a future version of Druid.
+:::
This is a special variation of the JSON ParseSpec that lower cases all the column names in the incoming JSON data. This parseSpec is required if you are updating to Druid 0.7.x from Druid 0.6.x, are directly ingesting JSON with mixed case column names, do not have any ETL in place to lower case those column names, and would like to make queries that include the data you created using 0.6.x and 0.7.x.
@@ -1608,7 +1645,9 @@ columns names ("column_1", "column2", ... "column_n") will be assigned. Ensure t
Note with the JavaScript parser that data must be fully parsed and returned as a `{key:value}` format in the JS logic.
This means any flattening or parsing multi-dimensional values must be done here.
-> JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::info
+ JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::
### TimeAndDims ParseSpec
diff --git a/docs/ingestion/hadoop.md b/docs/ingestion/hadoop.md
index cb7f03083318..c34fdb921172 100644
--- a/docs/ingestion/hadoop.md
+++ b/docs/ingestion/hadoop.md
@@ -180,7 +180,7 @@ Once you install the GCS Connector jar in all MiddleManager and Indexer processe
your Google Cloud Storage paths in the inputSpec with the below job properties.
For more configurations, see the [instructions to configure Hadoop](https://github.com/GoogleCloudPlatform/bigdata-interop/blob/master/gcs/INSTALL.md#configure-hadoop),
[GCS core default](https://github.com/GoogleCloudDataproc/hadoop-connectors/blob/v2.0.0/gcs/conf/gcs-core-default.xml)
-and [GCS core template](https://github.com/GoogleCloudPlatform/bdutil/blob/master/conf/hadoop2/gcs-core-template.xml).
+and [GCS core template](https://github.com/GoogleCloudDataproc/hadoop-connectors/blob/master/gcs/src/test/resources/core-site.xml).
```
"paths" : "gs://billy-bucket/the/data/is/here/data.gz,gs://billy-bucket/the/data/is/here/moredata.gz,gs://billy-bucket/the/data/is/here/evenmoredata.gz"
diff --git a/docs/ingestion/ingestion-spec.md b/docs/ingestion/ingestion-spec.md
index 43baf601bdbf..bc02faf20061 100644
--- a/docs/ingestion/ingestion-spec.md
+++ b/docs/ingestion/ingestion-spec.md
@@ -102,8 +102,10 @@ available in Druid's [web console](../operations/web-console.md). Druid's visual
## `dataSchema`
-> The `dataSchema` spec has been changed in 0.17.0. The new spec is supported by all ingestion methods
+:::info
+ The `dataSchema` spec has been changed in 0.17.0. The new spec is supported by all ingestion methods
except for _Hadoop_ ingestion. See the [Legacy `dataSchema` spec](#legacy-dataschema-spec) for the old spec.
+:::
The `dataSchema` is a holder for the following components:
@@ -166,10 +168,12 @@ configuring the [primary timestamp](./schema-model.md#primary-timestamp). An exa
}
```
-> Conceptually, after input data records are read, Druid applies ingestion spec components in a particular order:
-> first [`flattenSpec`](data-formats.md#flattenspec) (if any), then [`timestampSpec`](#timestampspec), then [`transformSpec`](#transformspec),
-> and finally [`dimensionsSpec`](#dimensionsspec) and [`metricsSpec`](#metricsspec). Keep this in mind when writing
-> your ingestion spec.
+:::info
+ Conceptually, after input data records are read, Druid applies ingestion spec components in a particular order:
+ first [`flattenSpec`](data-formats.md#flattenspec) (if any), then [`timestampSpec`](#timestampspec), then [`transformSpec`](#transformspec),
+ and finally [`dimensionsSpec`](#dimensionsspec) and [`metricsSpec`](#metricsspec). Keep this in mind when writing
+ your ingestion spec.
+:::
A `timestampSpec` can have the following components:
@@ -186,7 +190,7 @@ Treat `__time` as a millisecond timestamp: the number of milliseconds since Jan
### `dimensionsSpec`
The `dimensionsSpec` is located in `dataSchema` → `dimensionsSpec` and is responsible for
-configuring [dimensions](./schema-model.md#dimensions). An example `dimensionsSpec` is:
+configuring [dimensions](./schema-model.md#dimensions).
You can either manually specify the dimensions or take advantage of schema auto-discovery where you allow Druid to infer all or some of the schema for your data. This means that you don't have to explicitly specify your dimensions and their type.
@@ -212,10 +216,12 @@ The following `dimensionsSpec` example uses schema auto-discovery (`"useSchemaDi
```
-> Conceptually, after input data records are read, Druid applies ingestion spec components in a particular order:
-> first [`flattenSpec`](data-formats.md#flattenspec) (if any), then [`timestampSpec`](#timestampspec), then [`transformSpec`](#transformspec),
-> and finally [`dimensionsSpec`](#dimensionsspec) and [`metricsSpec`](#metricsspec). Keep this in mind when writing
-> your ingestion spec.
+:::info
+ Conceptually, after input data records are read, Druid applies ingestion spec components in a particular order:
+ first [`flattenSpec`](data-formats.md#flattenspec) (if any), then [`timestampSpec`](#timestampspec), then [`transformSpec`](#transformspec),
+ and finally [`dimensionsSpec`](#dimensionsspec) and [`metricsSpec`](#metricsspec). Keep this in mind when writing
+ your ingestion spec.
+:::
A `dimensionsSpec` can have the following components:
@@ -248,7 +254,9 @@ Druid will interpret a `dimensionsSpec` in two possible ways: _normal_ or _schem
Normal interpretation occurs when either `dimensions` or `spatialDimensions` is non-empty. In this case, the combination of the two lists will be taken as the set of dimensions to be ingested, and the list of `dimensionExclusions` will be ignored.
-> The following description of schemaless refers to string-based schemaless where Druid treats dimensions it discovers as strings. We recommend you use schema auto-discovery instead where Druid infers the type for the dimension. For more information, see [`dimensionsSpec`](#dimensionsspec).
+:::info
+ The following description of schemaless refers to string-based schemaless where Druid treats dimensions it discovers as strings. We recommend you use schema auto-discovery instead where Druid infers the type for the dimension. For more information, see [`dimensionsSpec`](#dimensionsspec).
+:::
Schemaless interpretation occurs when both `dimensions` and `spatialDimensions` are empty or null. In this case, the set of dimensions is determined in the following way:
@@ -262,8 +270,10 @@ Schemaless interpretation occurs when both `dimensions` and `spatialDimensions`
Additionally, if you have empty columns that you want to include in the string-based schemaless ingestion, you'll need to include the context parameter `storeEmptyColumns` and set it to `true`.
-> Note: Fields generated by a [`transformSpec`](#transformspec) are not currently considered candidates for
-> schemaless dimension interpretation.
+:::info
+ Note: Fields generated by a [`transformSpec`](#transformspec) are not currently considered candidates for
+ schemaless dimension interpretation.
+:::
### `metricsSpec`
@@ -281,11 +291,13 @@ An example `metricsSpec` is:
]
```
-> Generally, when [rollup](./rollup.md) is disabled, you should have an empty `metricsSpec` (because without rollup,
-> Druid does not do any ingestion-time aggregation, so there is little reason to include an ingestion-time aggregator). However,
-> in some cases, it can still make sense to define metrics: for example, if you want to create a complex column as a way of
-> pre-computing part of an [approximate aggregation](../querying/aggregations.md#approximate-aggregations), this can only
-> be done by defining a metric in a `metricsSpec`.
+:::info
+ Generally, when [rollup](./rollup.md) is disabled, you should have an empty `metricsSpec` (because without rollup,
+ Druid does not do any ingestion-time aggregation, so there is little reason to include an ingestion-time aggregator). However,
+ in some cases, it can still make sense to define metrics: for example, if you want to create a complex column as a way of
+ pre-computing part of an [approximate aggregation](../querying/aggregations.md#approximate-aggregations), this can only
+ be done by defining a metric in a `metricsSpec`.
+:::
### `granularitySpec`
@@ -340,10 +352,12 @@ records during ingestion time. It is optional. An example `transformSpec` is:
}
```
-> Conceptually, after input data records are read, Druid applies ingestion spec components in a particular order:
-> first [`flattenSpec`](data-formats.md#flattenspec) (if any), then [`timestampSpec`](#timestampspec), then [`transformSpec`](#transformspec),
-> and finally [`dimensionsSpec`](#dimensionsspec) and [`metricsSpec`](#metricsspec). Keep this in mind when writing
-> your ingestion spec.
+:::info
+ Conceptually, after input data records are read, Druid applies ingestion spec components in a particular order:
+ first [`flattenSpec`](data-formats.md#flattenspec) (if any), then [`timestampSpec`](#timestampspec), then [`transformSpec`](#transformspec),
+ and finally [`dimensionsSpec`](#dimensionsspec) and [`metricsSpec`](#metricsspec). Keep this in mind when writing
+ your ingestion spec.
+:::
#### Transforms
@@ -369,10 +383,12 @@ Druid currently includes one kind of built-in transform, the expression transfor
The `expression` is a [Druid query expression](../querying/math-expr.md).
-> Conceptually, after input data records are read, Druid applies ingestion spec components in a particular order:
-> first [`flattenSpec`](data-formats.md#flattenspec) (if any), then [`timestampSpec`](#timestampspec), then [`transformSpec`](#transformspec),
-> and finally [`dimensionsSpec`](#dimensionsspec) and [`metricsSpec`](#metricsspec). Keep this in mind when writing
-> your ingestion spec.
+:::info
+ Conceptually, after input data records are read, Druid applies ingestion spec components in a particular order:
+ first [`flattenSpec`](data-formats.md#flattenspec) (if any), then [`timestampSpec`](#timestampspec), then [`transformSpec`](#transformspec),
+ and finally [`dimensionsSpec`](#dimensionsspec) and [`metricsSpec`](#metricsspec). Keep this in mind when writing
+ your ingestion spec.
+:::
#### Filter
@@ -382,8 +398,10 @@ ingested. Any of Druid's standard [query filters](../querying/filters.md) can be
### Legacy `dataSchema` spec
-> The `dataSchema` spec has been changed in 0.17.0. The new spec is supported by all ingestion methods
+:::info
+ The `dataSchema` spec has been changed in 0.17.0. The new spec is supported by all ingestion methods
except for _Hadoop_ ingestion. See [`dataSchema`](#dataschema) for the new spec.
+:::
The legacy `dataSchema` spec has below two more components in addition to the ones listed in the [`dataSchema`](#dataschema) section above.
@@ -485,7 +503,7 @@ is:
|skipBytesInMemoryOverheadCheck|The calculation of maxBytesInMemory takes into account overhead objects created during ingestion and each intermediate persist. Setting this to true can exclude the bytes of these overhead objects from maxBytesInMemory check.|false|
|indexSpec|Defines segment storage format options to use at indexing time.|See [`indexSpec`](#indexspec) for more information.|
|indexSpecForIntermediatePersists|Defines segment storage format options to use at indexing time for intermediate persisted temporary segments.|See [`indexSpec`](#indexspec) for more information.|
-|Other properties|Each ingestion method has its own list of additional tuning properties. See the documentation for each method for a full list: [Kafka indexing service](../development/extensions-core/kafka-supervisor-reference.md#tuningconfig), [Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md#supervisor-tuning-configuration), [Native batch](native-batch.md#tuningconfig), and [Hadoop-based](hadoop.md#tuningconfig).||
+|Other properties|Each ingestion method has its own list of additional tuning properties. See the documentation for each method for a full list: [Kafka indexing service](../development/extensions-core/kafka-supervisor-reference.md#supervisor-tuning-configuration), [Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md#supervisor-tuning-configuration), [Native batch](native-batch.md#tuningconfig), and [Hadoop-based](hadoop.md#tuningconfig).||
### `indexSpec`
@@ -506,7 +524,9 @@ Front coding is an experimental feature starting in version 25.0. Front coding i
You can enable front coding with all types of ingestion. For information on defining an `indexSpec` in a query context, see [SQL-based ingestion reference](../multi-stage-query/reference.md#context-parameters).
-> Front coding was originally introduced in Druid 25.0, and an improved 'version 1' was introduced in Druid 26.0, with typically faster read speed and smaller storage size. The current recommendation is to enable it in a staging environment and fully test your use case before using in production. By default, segments created with front coding enabled in Druid 26.0 are backwards compatible with Druid 25.0, but those created with Druid 26.0 or 25.0 are not compatible with Druid versions older than 25.0. If using front coding in Druid 25.0 and upgrading to Druid 26.0, the `formatVersion` defaults to `0` to keep writing out the older format to enable seamless downgrades to Druid 25.0, and then later is recommended to be changed to `1` once determined that rollback is not necessary.
+:::info
+ Front coding was originally introduced in Druid 25.0, and an improved 'version 1' was introduced in Druid 26.0, with typically faster read speed and smaller storage size. The current recommendation is to enable it in a staging environment and fully test your use case before using in production. By default, segments created with front coding enabled in Druid 26.0 are backwards compatible with Druid 25.0, but those created with Druid 26.0 or 25.0 are not compatible with Druid versions older than 25.0. If using front coding in Druid 25.0 and upgrading to Druid 26.0, the `formatVersion` defaults to `0` to keep writing out the older format to enable seamless downgrades to Druid 25.0, and then later is recommended to be changed to `1` once determined that rollback is not necessary.
+:::
Beyond these properties, each ingestion method has its own specific tuning properties. See the documentation for each
[ingestion method](./index.md#ingestion-methods) for details.
diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md
index f5844c1d3706..df42e69102ed 100644
--- a/docs/ingestion/input-sources.md
+++ b/docs/ingestion/input-sources.md
@@ -29,7 +29,9 @@ For general information on native batch indexing and parallel task indexing, see
## 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.
+:::info
+ 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
@@ -206,11 +208,15 @@ Properties Object:
|assumeRoleArn|AWS ARN of the role to assume [see](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_credentials_temp_request.html). **assumeRoleArn** can be used either with the ingestion spec AWS credentials or with the default S3 credentials|None|no|
|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|
-> **Note:** If `accessKeyId` and `secretAccessKey` are not given, the default [S3 credentials provider chain](../development/extensions-core/s3.md#s3-authentication-methods) is used.
+:::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.
+:::
## 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.
+:::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.
+:::
The Google Cloud Storage input source is to support reading objects directly
from Google Cloud Storage. Objects can be specified as list of Google
@@ -294,7 +300,9 @@ Google Cloud Storage object:
## 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.
+:::info
+ 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 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.
@@ -375,7 +383,9 @@ The `objects` property is:
## 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.
+:::info
+ 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
from HDFS storage. File paths can be specified as an HDFS URI string or a list
@@ -462,9 +472,11 @@ 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.
-> **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.
+:::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.
+:::
For more information about security best practices, see [Security overview](../operations/security-overview.md#best-practices).
@@ -690,10 +702,12 @@ rolled-up datasource `wikipedia_rollup` by grouping on hour, "countryName", and
}
```
-> 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.
+:::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.
+:::
## SQL input source
@@ -796,7 +810,9 @@ The following is an example of a Combining input source spec:
## Iceberg input source
-> To use the Iceberg input source, add the `druid-iceberg-extensions` extension.
+:::info
+ To use the Iceberg input source, add the `druid-iceberg-extensions` extension.
+:::
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.
diff --git a/docs/ingestion/native-batch-firehose.md b/docs/ingestion/native-batch-firehose.md
index d1773b75037c..db2b3e8779e6 100644
--- a/docs/ingestion/native-batch-firehose.md
+++ b/docs/ingestion/native-batch-firehose.md
@@ -23,7 +23,9 @@ sidebar_label: "Firehose (deprecated)"
~ under the License.
-->
-> Firehose ingestion is deprecated. See [Migrate from firehose to input source ingestion](../operations/migrate-from-firehose-ingestion.md) for instructions on migrating from firehose ingestion to using native batch ingestion input sources.
+:::info
+ Firehose ingestion is deprecated. See [Migrate from firehose to input source ingestion](../operations/migrate-from-firehose-ingestion.md) for instructions on migrating from firehose ingestion to using native batch ingestion input sources.
+:::
There are several firehoses readily available in Druid, some are meant for examples, others can be used directly in a production environment.
diff --git a/docs/ingestion/native-batch-simple-task.md b/docs/ingestion/native-batch-simple-task.md
index 105fdb65cbe7..ca6c28c7d009 100644
--- a/docs/ingestion/native-batch-simple-task.md
+++ b/docs/ingestion/native-batch-simple-task.md
@@ -23,8 +23,10 @@ sidebar_label: "JSON-based batch (simple)"
~ under the License.
-->
-> This page describes native batch ingestion using [ingestion specs](ingestion-spec.md). Refer to the [ingestion
-> methods](../ingestion/index.md#batch) table to determine which ingestion method is right for you.
+:::info
+ This page describes native batch ingestion using [ingestion specs](ingestion-spec.md). Refer to the [ingestion
+ methods](../ingestion/index.md#batch) table to determine which ingestion method is right for you.
+:::
The simple task ([task type](tasks.md) `index`) executes single-threaded as a single task within the indexing service. For parallel, scalable options consider using [`index_parallel` tasks](./native-batch.md) or [SQL-based batch ingestion](../multi-stage-query/index.md).
diff --git a/docs/ingestion/native-batch.md b/docs/ingestion/native-batch.md
index 6741921f4faa..8af3f6c1ca9b 100644
--- a/docs/ingestion/native-batch.md
+++ b/docs/ingestion/native-batch.md
@@ -23,7 +23,9 @@ sidebar_label: JSON-based batch
~ under the License.
-->
-> This page describes JSON-based batch ingestion using [ingestion specs](ingestion-spec.md). For SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md) extension, see [SQL-based ingestion](../multi-stage-query/index.md). Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which ingestion method is right for you.
+:::info
+ This page describes JSON-based batch ingestion using [ingestion specs](ingestion-spec.md). For SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md) extension, see [SQL-based ingestion](../multi-stage-query/index.md). Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which ingestion method is right for you.
+:::
Apache Druid supports the following types of native batch indexing tasks:
- Parallel task indexing (`index_parallel`) that can run multiple indexing tasks concurrently. Parallel task works well for production ingestion tasks.
@@ -344,7 +346,9 @@ In hash partitioning, the partition function is used to compute hash of partitio
#### Single-dimension range partitioning
-> Single dimension range partitioning is not supported in the sequential mode of the `index_parallel` task type.
+:::info
+ Single dimension range partitioning is not supported in the sequential mode of the `index_parallel` task type.
+:::
Range partitioning has [several benefits](#benefits-of-range-partitioning) related to storage footprint and query
performance.
@@ -388,13 +392,17 @@ the time chunk and the value of `partitionDimension`; each worker task reads the
falling in the same partition of the same range from multiple MiddleManager/Indexer processes and merges
them to create the final segments. Finally, they push the final segments to the deep storage.
-> Because the task with single-dimension range partitioning makes two passes over the input
-> in `partial dimension distribution` and `partial segment generation` phases,
-> the task may fail if the input changes in between the two passes.
+:::info
+ Because the task with single-dimension range partitioning makes two passes over the input
+ in `partial dimension distribution` and `partial segment generation` phases,
+ the task may fail if the input changes in between the two passes.
+:::
#### Multi-dimension range partitioning
-> Multi-dimension range partitioning is not supported in the sequential mode of the `index_parallel` task type.
+:::info
+ Multi-dimension range partitioning is not supported in the sequential mode of the `index_parallel` task type.
+:::
Range partitioning has [several benefits](#benefits-of-range-partitioning) related to storage footprint and query
performance. Multi-dimension range partitioning improves over single-dimension range partitioning by allowing
diff --git a/docs/ingestion/schema-design.md b/docs/ingestion/schema-design.md
index 6d385c7b60e7..0749d20b6951 100644
--- a/docs/ingestion/schema-design.md
+++ b/docs/ingestion/schema-design.md
@@ -244,20 +244,22 @@ You should query for the number of ingested rows with:
Druid can infer the schema for your data in one of two ways:
-- [Type-aware schema discovery (experimental)](#type-aware-schema-discovery) where Druid infers the schema and type for your data. Type-aware schema discovery is an experimental feature currently available for native batch and streaming ingestion.
+- [Type-aware schema discovery](#type-aware-schema-discovery) where Druid infers the schema and type for your data. Type-aware schema discovery is available for native batch and streaming ingestion.
- [String-based schema discovery](#string-based-schema-discovery) where all the discovered columns are typed as either native string or multi-value string columns.
#### Type-aware schema discovery
-> Note that using type-aware schema discovery can impact downstream BI tools depending on how they handle ARRAY typed columns.
+:::info
+ Note that using type-aware schema discovery can impact downstream BI tools depending on how they handle ARRAY typed columns.
+:::
You can have Druid infer the schema and types for your data partially or fully by setting `dimensionsSpec.useSchemaDiscovery` to `true` and defining some or no dimensions in the dimensions list.
When performing type-aware schema discovery, Druid can discover all of the columns of your input data (that aren't in
the exclusion list). Druid automatically chooses the most appropriate native Druid type among `STRING`, `LONG`,
`DOUBLE`, `ARRAY`, `ARRAY`, `ARRAY`, or `COMPLEX` for nested data. For input formats with
-native boolean types, Druid ingests these values as strings if `druid.expressions.useStrictBooleans` is set to `false`
-(the default), or longs if set to `true` (for more SQL compatible behavior). Array typed columns can be queried using
+native boolean types, Druid ingests these values as longs if `druid.expressions.useStrictBooleans` is set to `true`
+(the default) or strings if set to `false`. Array typed columns can be queried using
the [array functions](../querying/sql-array-functions.md) or [UNNEST](../querying/sql-functions.md#unnest). Nested
columns can be queried with the [JSON functions](../querying/sql-json-functions.md).
diff --git a/docs/ingestion/tasks.md b/docs/ingestion/tasks.md
index eb32d04e7c2b..4b3081cef911 100644
--- a/docs/ingestion/tasks.md
+++ b/docs/ingestion/tasks.md
@@ -304,7 +304,9 @@ For example, a Kafka indexing task and a compaction task can always write segmen
The reason for this is because a Kafka indexing task always appends new segments, while a compaction task always overwrites existing segments.
The segments created with the segment locking have the _same_ major version and a _higher_ minor version.
-> The segment locking is still experimental. It could have unknown bugs which potentially lead to incorrect query results.
+:::info
+ The segment locking is still experimental. It could have unknown bugs which potentially lead to incorrect query results.
+:::
To enable segment locking, you may need to set `forceTimeChunkLock` to `false` in the [task context](#context).
Once `forceTimeChunkLock` is unset, the task will choose a proper lock type to use automatically.
@@ -415,11 +417,15 @@ If you don't see the log file in long-term storage, it means either:
You can check the middleManager / indexer logs locally to see if there was a push failure. If there was not, check the Overlord's own process logs to see why the task failed before it started.
-> If you are running the indexing service in remote mode, the task logs must be stored in S3, Azure Blob Store, Google Cloud Storage or HDFS.
+:::info
+ If you are running the indexing service in remote mode, the task logs must be stored in S3, Azure Blob Store, Google Cloud Storage or HDFS.
+:::
You can configure retention periods for logs in milliseconds by setting `druid.indexer.logs.kill` properties in [configuration](../configuration/index.md#task-logging). The Overlord will then automatically manage task logs in log directories along with entries in task-related metadata storage tables.
-> Automatic log file deletion typically works based on the log file's 'modified' timestamp in the back-end store. Large clock skews between Druid processes and the long-term store might result in unintended behavior.
+:::info
+ Automatic log file deletion typically works based on the log file's 'modified' timestamp in the back-end store. Large clock skews between Druid processes and the long-term store might result in unintended behavior.
+:::
## Configuring task storage sizes
diff --git a/docs/multi-stage-query/concepts.md b/docs/multi-stage-query/concepts.md
index 2d0316102765..2969ba9722a9 100644
--- a/docs/multi-stage-query/concepts.md
+++ b/docs/multi-stage-query/concepts.md
@@ -23,9 +23,11 @@ sidebar_label: "Key concepts"
~ under the License.
-->
-> This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
-> extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
-> ingestion method is right for you.
+:::info
+ This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
+ extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
+ ingestion method is right for you.
+:::
## Multi-stage query task engine
@@ -203,8 +205,8 @@ If you see the error "Encountered multi-value dimension `x` that cannot be proce
groupByEnableMultiValueUnnesting set to false", then wrap that column in `MV_TO_ARRAY(x) AS x`.
The following [aggregation functions](../querying/sql-aggregations.md) are supported for rollup at ingestion time:
-`COUNT` (but switch to `SUM` at query time), `SUM`, `MIN`, `MAX`, `EARLIEST` ([string only](known-issues.md#select-statement)),
-`LATEST` ([string only](known-issues.md#select-statement)), `APPROX_COUNT_DISTINCT`, `APPROX_COUNT_DISTINCT_BUILTIN`,
+`COUNT` (but switch to `SUM` at query time), `SUM`, `MIN`, `MAX`, `EARLIEST` and `EARLIEST_BY` ([string only](known-issues.md#select-statement)),
+`LATEST` and `LATEST_BY` ([string only](known-issues.md#select-statement)), `APPROX_COUNT_DISTINCT`, `APPROX_COUNT_DISTINCT_BUILTIN`,
`APPROX_COUNT_DISTINCT_DS_HLL`, `APPROX_COUNT_DISTINCT_DS_THETA`, and `DS_QUANTILES_SKETCH` (but switch to
`APPROX_QUANTILE_DS` at query time). Do not use `AVG`; instead, use `SUM` and `COUNT` at ingest time and compute the
quotient at query time.
diff --git a/docs/multi-stage-query/examples.md b/docs/multi-stage-query/examples.md
index ba05c4e36293..51a645448daf 100644
--- a/docs/multi-stage-query/examples.md
+++ b/docs/multi-stage-query/examples.md
@@ -23,9 +23,11 @@ sidebar_label: Examples
~ under the License.
-->
-> This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
-> extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
-> ingestion method is right for you.
+:::info
+ This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
+ extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
+ ingestion method is right for you.
+:::
These example queries show you some of the things you can do when modifying queries for your use case. Copy the example queries into the **Query** view of the web console and run them to see what they do.
diff --git a/docs/multi-stage-query/index.md b/docs/multi-stage-query/index.md
index 291211650bfc..d69aca4aada5 100644
--- a/docs/multi-stage-query/index.md
+++ b/docs/multi-stage-query/index.md
@@ -24,9 +24,11 @@ description: Introduces multi-stage query architecture and its task engine
~ under the License.
-->
-> This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
-> extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
-> ingestion method is right for you.
+:::info
+ This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
+ extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
+ ingestion method is right for you.
+:::
Apache Druid supports SQL-based ingestion using the bundled [`druid-multi-stage-query` extension](#load-the-extension).
This extension adds a [multi-stage query task engine for SQL](concepts.md#multi-stage-query-task-engine) that allows running SQL
diff --git a/docs/multi-stage-query/known-issues.md b/docs/multi-stage-query/known-issues.md
index 4b54f55a6fbf..bccb9779a835 100644
--- a/docs/multi-stage-query/known-issues.md
+++ b/docs/multi-stage-query/known-issues.md
@@ -23,9 +23,11 @@ sidebar_label: Known issues
~ under the License.
-->
-> This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
-> extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
-> ingestion method is right for you.
+:::info
+ This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
+ extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
+ ingestion method is right for you.
+:::
## Multi-stage query task runtime
diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md
index 3fa179e1de2b..6236b6545258 100644
--- a/docs/multi-stage-query/reference.md
+++ b/docs/multi-stage-query/reference.md
@@ -23,9 +23,11 @@ sidebar_label: Reference
~ under the License.
-->
-> This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
-> extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
-> ingestion method is right for you.
+:::info
+ This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
+ extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
+ ingestion method is right for you.
+:::
## SQL reference
@@ -343,61 +345,41 @@ CLUSTERED BY user
The context parameter that sets `sqlJoinAlgorithm` to `sortMerge` is not shown in the above example.
-## Durable Storage
+## Durable storage
-Using durable storage with your SQL-based ingestion can improve their reliability by writing intermediate files to a storage location temporarily.
+SQL-based ingestion supports using durable storage to store intermediate files temporarily. Enabling it can improve reliability. For more information, see [Durable storage](../operations/durable-storage.md).
-To prevent durable storage from getting filled up with temporary files in case the tasks fail to clean them up, a periodic
-cleaner can be scheduled to clean the directories corresponding to which there isn't a controller task running. It utilizes
-the storage connector to work upon the durable storage. The durable storage location should only be utilized to store the output
-for cluster's MSQ tasks. If the location contains other files or directories, then they will get cleaned up as well.
+### Durable storage configurations
-Enabling durable storage also enables the use of local disk to store temporary files, such as the intermediate files produced
-by the super sorter. Tasks will use whatever has been configured for their temporary usage as described in [Configuring task storage sizes](../ingestion/tasks.md#configuring-task-storage-sizes)
-If the configured limit is too low, `NotEnoughTemporaryStorageFault` may be thrown.
+Durable storage is supported on Amazon S3 storage and Microsoft's Azure storage. There are a few common configurations that controls the behavior for both the services as documented below. Apart from the common configurations,
+there are a few properties specific to each storage that must be set.
-### Enable durable storage
+Common properties to configure the behavior of durable storage
-To enable durable storage, you need to set the following common service properties:
-
-```
-druid.msq.intermediate.storage.enable=true
-druid.msq.intermediate.storage.type=s3
-druid.msq.intermediate.storage.bucket=YOUR_BUCKET
-druid.msq.intermediate.storage.prefix=YOUR_PREFIX
-druid.msq.intermediate.storage.tempDir=/path/to/your/temp/dir
-```
-
-For detailed information about the settings related to durable storage, see [Durable storage configurations](#durable-storage-configurations).
-
-
-### Use durable storage for queries
-
-When you run a query, include the context parameter `durableShuffleStorage` and set it to `true`.
-
-For queries where you want to use fault tolerance for workers, set `faultTolerance` to `true`, which automatically sets `durableShuffleStorage` to `true`.
-
-Set `selectDestination`:`durableStorage` for select queries that want to write the final results to durable storage instead of the task reports. Saving the results in the durable
-storage allows users to fetch large result sets. The location where the workers write the intermediate results is different than the location where final results get stored. Therefore, `durableShuffleStorage`:`false` and
-`selectDestination`:`durableStorage` is a valid configuration to use in the query context, that instructs the controller to persist only the final result in the durable storage, and not the
-intermediate results.
+|Parameter |Default | Description |
+|-------------------|----------------------------------------|----------------------|
+|`druid.msq.intermediate.storage.enable` | false | Whether to enable durable storage for the cluster. Set it to true to enable durable storage. For more information about enabling durable storage, see [Durable storage](../operations/durable-storage.md).|
+|`druid.msq.intermediate.storage.type` | n/a | Required. The type of storage to use. Set it to `s3` for S3 and `azure` for Azure |
+|`druid.msq.intermediate.storage.tempDir`| n/a | Required. Directory path on the local disk to store temporary files required while uploading and downloading the data |
+|`druid.msq.intermediate.storage.maxRetry` | 10 | Optional. Defines the max number times to attempt S3 API calls to avoid failures due to transient errors. |
+|`druid.msq.intermediate.storage.chunkSize` | 100MiB | Optional. Defines the size of each chunk to temporarily store in `druid.msq.intermediate.storage.tempDir`. The chunk size must be between 5 MiB and 5 GiB. A large chunk size reduces the API calls made to the durable storage, however it requires more disk space to store the temporary chunks. Druid uses a default of 100MiB if the value is not provided.|
+Following properties need to be set in addition to the common properties to enable durable storage on S3
-## Durable storage configurations
+|Parameter |Default | Description |
+|-------------------|----------------------------------------|----------------------|
+|`druid.msq.intermediate.storage.bucket` | n/a | Required. The S3 bucket where the files are uploaded to and download from |
+|`druid.msq.intermediate.storage.prefix` | n/a | Required. Path prepended to all the paths uploaded to the bucket to namespace the connector's files. Provide a unique value for the prefix and do not share the same prefix between different clusters. If the location includes other files or directories, then they might get cleaned up as well. |
-The following common service properties control how durable storage behaves:
+Following properties must be set in addition to the common properties to enable durable storage on Azure.
|Parameter |Default | Description |
|-------------------|----------------------------------------|----------------------|
-|`druid.msq.intermediate.storage.bucket` | n/a | The bucket in S3 where you want to store intermediate files. |
-|`druid.msq.intermediate.storage.chunkSize` | 100MiB | Optional. Defines the size of each chunk to temporarily store in `druid.msq.intermediate.storage.tempDir`. The chunk size must be between 5 MiB and 5 GiB. A large chunk size reduces the API calls made to the durable storage, however it requires more disk space to store the temporary chunks. Druid uses a default of 100MiB if the value is not provided.|
-|`druid.msq.intermediate.storage.enable` | true | Required. Whether to enable durable storage for the cluster.|
-|`druid.msq.intermediate.storage.maxRetry` | 10 | Optional. Defines the max number times to attempt S3 API calls to avoid failures due to transient errors. |
-|`druid.msq.intermediate.storage.prefix` | n/a | S3 prefix to store intermediate stage results. Provide a unique value for the prefix. Don't share the same prefix between clusters. If the location includes other files or directories, then they will get cleaned up as well. |
-|`druid.msq.intermediate.storage.tempDir`| n/a | Required. Directory path on the local disk to temporarily store intermediate stage results. |
-|`druid.msq.intermediate.storage.type` | `s3` if your deep storage is S3 | Required. The type of storage to use. You can either set this to `local` or `s3`. |
+|`druid.msq.intermediate.storage.container` | n/a | Required. The Azure container where the files are uploaded to and downloaded from. |
+|`druid.msq.intermediate.storage.prefix` | n/a | Required. Path prepended to all the paths uploaded to the container to namespace the connector's files. Provide a unique value for the prefix and do not share the same prefix between different clusters. If the location includes other files or directories, then they might get cleaned up as well. |
-In addition to the common service properties, there are certain properties that you configure on the Overlord specifically to clean up intermediate files:
+Durable storage creates files on the remote storage and is cleaned up once the job no longer requires those files. However, due to failures causing abrupt exit of the tasks, these files might not get cleaned up.
+Therefore, there are certain properties that you configure on the Overlord specifically to clean up intermediate files for the tasks that have completed and would no longer require these files:
|Parameter |Default | Description |
|-------------------|----------------------------------------|----------------------|
diff --git a/docs/multi-stage-query/security.md b/docs/multi-stage-query/security.md
index 9a9e02b0ea19..2d412f40654f 100644
--- a/docs/multi-stage-query/security.md
+++ b/docs/multi-stage-query/security.md
@@ -23,9 +23,11 @@ sidebar_label: Security
~ under the License.
-->
-> This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
-> extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
-> ingestion method is right for you.
+:::info
+ This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
+ extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
+ ingestion method is right for you.
+:::
All authenticated users can use the multi-stage query task engine (MSQ task engine) through the UI and API if the
extension is loaded. However, without additional permissions, users are not able to issue queries that read or write
@@ -46,7 +48,9 @@ users with access to the Overlord API can perform some actions even if they didn
retrieving status or canceling a query. For more information about the Overlord API and the task API, see [APIs for
SQL-based ingestion](../api-reference/sql-ingestion-api.md).
-> Keep in mind that any user with access to Overlord APIs can submit `query_controller` tasks with only the WRITE DATASOURCE permission.
+:::info
+ Keep in mind that any user with access to Overlord APIs can submit `query_controller` tasks with only the WRITE DATASOURCE permission.
+:::
Depending on what a user is trying to do, they might also need the following permissions:
diff --git a/docs/operations/auth-ldap.md b/docs/operations/auth-ldap.md
index ccb82ba58f9a..93320049668e 100644
--- a/docs/operations/auth-ldap.md
+++ b/docs/operations/auth-ldap.md
@@ -63,7 +63,9 @@ memberOf: cn=mygroup,ou=groups,dc=example,dc=com
You use this information to map the LDAP group to Druid roles in a later step.
-> Druid uses the `memberOf` attribute to determine a group's membership using LDAP. If your LDAP server implementation doesn't include this attribute, you must complete some additional steps when you [map LDAP groups to Druid roles](#map-ldap-groups-to-druid-roles).
+:::info
+ Druid uses the `memberOf` attribute to determine a group's membership using LDAP. If your LDAP server implementation doesn't include this attribute, you must complete some additional steps when you [map LDAP groups to Druid roles](#map-ldap-groups-to-druid-roles).
+:::
## Configure Druid for LDAP authentication
@@ -105,7 +107,9 @@ In the example below, the LDAP user is `internal@example.com`.
- `userAttribute`: The user search attribute.
- `internal@example.com` is the LDAP user you created in step 1. In the example it serves as both the internal client user and the initial admin user.
- > In the above example, the [Druid escalator](../development/extensions-core/druid-basic-security.md#escalator) and LDAP initial admin user are set to the same user - `internal@example.com`. If the escalator is set to a different user, you must follow steps 4 and 5 to create the group mapping and allocate initial roles before the rest of the cluster can function.
+:::info
+ In the above example, the [Druid escalator](../development/extensions-core/druid-basic-security.md#escalator) and LDAP initial admin user are set to the same user - `internal@example.com`. If the escalator is set to a different user, you must follow steps 4 and 5 to create the group mapping and allocate initial roles before the rest of the cluster can function.
+:::
4. Save your group mapping to a JSON file. An example file `groupmap.json` looks like this:
diff --git a/docs/operations/basic-cluster-tuning.md b/docs/operations/basic-cluster-tuning.md
index be875665b5eb..538ae33d75f2 100644
--- a/docs/operations/basic-cluster-tuning.md
+++ b/docs/operations/basic-cluster-tuning.md
@@ -326,13 +326,13 @@ The TopN and GroupBy queries use these buffers to store intermediate computed re
### GroupBy merging buffers
-If you plan to issue GroupBy V2 queries, `druid.processing.numMergeBuffers` is an important configuration property.
+If you plan to issue GroupBy queries, `druid.processing.numMergeBuffers` is an important configuration property.
-GroupBy V2 queries use an additional pool of off-heap buffers for merging query results. These buffers have the same size as the processing buffers described above, set by the `druid.processing.buffer.sizeBytes` property.
+GroupBy queries use an additional pool of off-heap buffers for merging query results. These buffers have the same size as the processing buffers described above, set by the `druid.processing.buffer.sizeBytes` property.
-Non-nested GroupBy V2 queries require 1 merge buffer per query, while a nested GroupBy V2 query requires 2 merge buffers (regardless of the depth of nesting).
+Non-nested GroupBy queries require 1 merge buffer per query, while a nested GroupBy query requires 2 merge buffers (regardless of the depth of nesting).
-The number of merge buffers determines the number of GroupBy V2 queries that can be processed concurrently.
+The number of merge buffers determines the number of GroupBy queries that can be processed concurrently.
@@ -421,8 +421,10 @@ Enabling process termination on out-of-memory errors is useful as well, since th
-XX:HeapDumpPath=/var/logs/druid/historical.hprof
-XX:MaxDirectMemorySize=1g
```
-> Please note that the flag settings above represent sample, general guidelines only. Be careful to use values appropriate
+:::info
+ Please note that the flag settings above represent sample, general guidelines only. Be careful to use values appropriate
for your specific scenario and be sure to test any changes in staging environments.
+:::
`ExitOnOutOfMemoryError` flag is only supported starting JDK 8u92 . For older versions, `-XX:OnOutOfMemoryError='kill -9 %p'` can be used.
diff --git a/docs/operations/clean-metadata-store.md b/docs/operations/clean-metadata-store.md
index 8aa3c7dc32bb..202b27805edf 100644
--- a/docs/operations/clean-metadata-store.md
+++ b/docs/operations/clean-metadata-store.md
@@ -71,7 +71,9 @@ If you want to skip the details, check out the [example](#example) for configuri
### Segment records and segments in deep storage (kill task)
-> The kill task is the only configuration in this topic that affects actual data in deep storage and not simply metadata or logs.
+:::info
+ The kill task is the only configuration in this topic that affects actual data in deep storage and not simply metadata or logs.
+:::
Segment records and segments in deep storage become eligible for deletion when both of the following conditions hold:
@@ -118,8 +120,10 @@ Rule cleanup uses the following configuration:
Druid retains all compaction configuration records by default, which should be suitable for most use cases.
If you create and delete short-lived datasources with high frequency, and you set auto compaction configuration on those datasources, then consider turning on automated cleanup of compaction configuration records.
-> With automated cleanup of compaction configuration records, if you create a compaction configuration for some datasource before the datasource exists, for example if initial ingestion is still ongoing, Druid may remove the compaction configuration.
+:::info
+ With automated cleanup of compaction configuration records, if you create a compaction configuration for some datasource before the datasource exists, for example if initial ingestion is still ongoing, Druid may remove the compaction configuration.
To prevent the configuration from being prematurely removed, wait for the datasource to be created before applying the compaction configuration to the datasource.
+:::
Unlike other metadata records, compaction configuration records do not have a retention period set by `durationToRetain`. Druid deletes compaction configuration records at every cleanup cycle for inactive datasources, which do not have segments either used or unused.
@@ -130,7 +134,9 @@ Compaction configuration cleanup uses the following configuration:
- `druid.coordinator.kill.compaction.period`: Defines the frequency in [ISO 8601 format](https://en.wikipedia.org/wiki/ISO_8601#Durations) for the cleanup job to check for and delete eligible compaction configuration records. Defaults to `P1D`.
->If you already have an extremely large compaction configuration, you may not be able to delete compaction configuration due to size limits with the audit log. In this case you can set `druid.audit.manager.maxPayloadSizeBytes` and `druid.audit.manager.skipNullField` to avoid the auditing issue. See [Audit logging](../configuration/index.md#audit-logging).
+:::info
+If you already have an extremely large compaction configuration, you may not be able to delete compaction configuration due to size limits with the audit log. In this case you can set `druid.audit.manager.maxPayloadSizeBytes` and `druid.audit.manager.skipNullField` to avoid the auditing issue. See [Audit logging](../configuration/index.md#audit-logging).
+:::
### Datasource records created by supervisors
diff --git a/docs/operations/durable-storage.md b/docs/operations/durable-storage.md
new file mode 100644
index 000000000000..80545f9a9b28
--- /dev/null
+++ b/docs/operations/durable-storage.md
@@ -0,0 +1,86 @@
+---
+id: durable-storage
+title: "Durable storage for the multi-stage query engine"
+sidebar_label: "Durable storage"
+---
+
+
+
+You can use durable storage to improve querying from deep storage and SQL-based ingestion.
+
+> Note that only S3 is supported as a durable storage location.
+
+Durable storage for queries from deep storage provides a location where you can write the results of deep storage queries to. Durable storage for SQL-based ingestion is used to temporarily house intermediate files, which can improve reliability.
+
+Enabling durable storage also enables the use of local disk to store temporary files, such as the intermediate files produced
+while sorting the data. Tasks will use whatever has been configured for their temporary usage as described in [Configuring task storage sizes](../ingestion/tasks.md#configuring-task-storage-sizes).
+If the configured limit is too low, Druid may throw the error, `NotEnoughTemporaryStorageFault`.
+
+## Enable durable storage
+
+To enable durable storage, you need to set the following common service properties:
+
+```
+druid.msq.intermediate.storage.enable=true
+druid.msq.intermediate.storage.type=s3
+druid.msq.intermediate.storage.bucket=YOUR_BUCKET
+druid.msq.intermediate.storage.prefix=YOUR_PREFIX
+druid.msq.intermediate.storage.tempDir=/path/to/your/temp/dir
+```
+
+For detailed information about the settings related to durable storage, see [Durable storage configurations](../multi-stage-query/reference.md#durable-storage-configurations).
+
+
+## Use durable storage for SQL-based ingestion queries
+
+When you run a query, include the context parameter `durableShuffleStorage` and set it to `true`.
+
+For queries where you want to use fault tolerance for workers, set `faultTolerance` to `true`, which automatically sets `durableShuffleStorage` to `true`.
+
+## Use durable storage for queries from deep storage
+
+Depending on the size of the results you're expecting, saving the final results for queries from deep storage to durable storage might be needed.
+
+By default, Druid saves the final results for queries from deep storage to task reports. Generally, this is acceptable for smaller result sets but may lead to timeouts for larger result sets.
+
+When you run a query, include the context parameter `selectDestination` and set it to `DURABLESTORAGE`:
+
+```json
+ "context":{
+ ...
+ "selectDestination": "DURABLESTORAGE"
+ }
+```
+
+You can also write intermediate results to durable storage (`durableShuffleStorage`) for better reliability. The location where workers write intermediate results is different than the location where final results get stored. This means that durable storage for results can be enabled even if you don't write intermediate results to durable storage.
+
+If you write the results for queries from deep storage to durable storage, the results are cleaned up when the task is removed from the metadata store.
+
+## Durable storage clean up
+
+To prevent durable storage from getting filled up with temporary files in case the tasks fail to clean them up, a periodic
+cleaner can be scheduled to clean the directories corresponding to which there isn't a controller task running. It utilizes
+the storage connector to work upon the durable storage. The durable storage location should only be utilized to store the output
+for the cluster's MSQ tasks. If the location contains other files or directories, then they will get cleaned up as well.
+
+Use `druid.msq.intermediate.storage.cleaner.enabled` and `druid.msq.intermediate.storage.cleaner.delaySEconds` to configure the cleaner. For more information, see [Durable storage configurations](../multi-stage-query/reference.md#durable-storage-configurations).
+
+Note that if you choose to write query results to durable storage,the results are cleaned up when the task is removed from the metadata store.
+
diff --git a/docs/operations/getting-started.md b/docs/operations/getting-started.md
deleted file mode 100644
index 8509d6baa1e1..000000000000
--- a/docs/operations/getting-started.md
+++ /dev/null
@@ -1,48 +0,0 @@
----
-id: getting-started
-title: "Getting started with Apache Druid"
----
-
-
-
-
-## Overview
-
-If you are new to Druid, we recommend reading the [Design Overview](../design/index.md) and the [Ingestion Overview](../ingestion/index.md) first for a basic understanding of Druid.
-
-## Single-server Quickstart and Tutorials
-
-To get started with running Druid, the simplest and quickest way is to try the [single-server quickstart and tutorials](../tutorials/index.md).
-
-## Deploying a Druid cluster
-
-If you wish to jump straight to deploying Druid as a cluster, or if you have an existing single-server deployment that you wish to migrate to a clustered deployment, please see the [Clustered Deployment Guide](../tutorials/cluster.md).
-
-## Operating Druid
-
-The [configuration reference](../configuration/index.md) describes all of Druid's configuration properties.
-
-The [API reference](../api-reference/api-reference.md) describes the APIs available on each Druid process.
-
-The [basic cluster tuning guide](../operations/basic-cluster-tuning.md) is an introductory guide for tuning your Druid cluster.
-
-## Need help with Druid?
-
-If you have questions about using Druid, please reach out to the [Druid user mailing list or other community channels](https://druid.apache.org/community/)!
diff --git a/docs/operations/metadata-migration.md b/docs/operations/metadata-migration.md
index b38972caf41d..ea3596784ad2 100644
--- a/docs/operations/metadata-migration.md
+++ b/docs/operations/metadata-migration.md
@@ -57,6 +57,8 @@ Update your Druid runtime properties with the new metadata configuration.
### Create Druid tables
+**If you have set `druid.metadata.storage.connector.createTables` to `true` (which is the default), and your metadata connect user has DDL privileges, you can disregard this section as Druid will create metadata tables automatically on start up.**
+
Druid provides a `metadata-init` tool for creating Druid's metadata tables. After initializing the Druid database, you can run the commands shown below from the root of the Druid package to initialize the tables.
In the example commands below:
@@ -82,6 +84,10 @@ cd ${DRUID_ROOT}
java -classpath "lib/*" -Dlog4j.configurationFile=conf/druid/cluster/_common/log4j2.xml -Ddruid.extensions.directory="extensions" -Ddruid.extensions.loadList="[\"postgresql-metadata-storage\"]" -Ddruid.metadata.storage.type=postgresql -Ddruid.node.type=metadata-init org.apache.druid.cli.Main tools metadata-init --connectURI="" --user --password --base druid
```
+### Update Druid tables to latest compatible schema
+
+The same command as above can be used to update Druid metadata tables to the latest version. If any table already exists, it is not created again but any ALTER statements that may be required are still executed.
+
### Import metadata
After initializing the tables, please refer to the [import commands](../operations/export-metadata.md#importing-metadata) for your target database.
diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md
index 95099406f282..c2a77a4f3d67 100644
--- a/docs/operations/metrics.md
+++ b/docs/operations/metrics.md
@@ -35,11 +35,14 @@ All Druid metrics share a common set of fields:
Metrics may have additional dimensions beyond those listed above.
-> Most metric values reset each emission period, as specified in `druid.monitoring.emissionPeriod`.
+:::info
+Most metric values reset each emission period, as specified in `druid.monitoring.emissionPeriod`.
+:::
## Query metrics
### Router
+
|Metric|Description|Dimensions|Normal value|
|------|-----------|----------|------------|
|`query/time`|Milliseconds taken to complete a query.|Native Query: `dataSource`, `type`, `interval`, `hasFilters`, `duration`, `context`, `remoteAddress`, `id`.|< 1s|
@@ -70,6 +73,13 @@ Metrics may have additional dimensions beyond those listed above.
|`metadatacache/refresh/time`|Time taken to refresh segments in broker segment metadata cache.|`dataSource`|
|`serverview/sync/healthy`|Sync status of the Broker with a segment-loading server such as a Historical or Peon. Emitted only when [HTTP-based server view](../configuration/index.md#segment-management) is enabled. This metric can be used in conjunction with `serverview/sync/unstableTime` to debug slow startup of Brokers.|`server`, `tier`|1 for fully synced servers, 0 otherwise|
|`serverview/sync/unstableTime`|Time in milliseconds for which the Broker has been failing to sync with a segment-loading server. Emitted only when [HTTP-based server view](../configuration/index.md#segment-management) is enabled.|`server`, `tier`|Not emitted for synced servers.|
+|`subquery/rowLimit/count`|Number of subqueries whose results are materialized as rows (Java objects on heap).|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
+|`subquery/byteLimit/count`|Number of subqueries whose results are materialized as frames (Druid's internal byte representation of rows).|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
+|`subquery/fallback/count`|Number of subqueries which cannot be materialized as frames|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
+|`subquery/fallback/insufficientType/count`|Number of subqueries which cannot be materialized as frames due to insufficient type information in the row signature.|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
+|`subquery/fallback/unknownReason/count`|Number of subqueries which cannot be materialized as frames due other reasons.|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
+|`query/rowLimit/exceeded/count`|Number of queries whose inlined subquery results exceeded the given row limit|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
+|`query/byteLimit/exceeded/count`|Number of queries whose inlined subquery results exceeded the given byte limit|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
### Historical
@@ -154,7 +164,7 @@ If SQL is enabled, the Broker will emit the following metrics for SQL.
## Ingestion metrics
-## General native ingestion metrics
+### General native ingestion metrics
|Metric|Description|Dimensions|Normal value|
|------|-----------|----------|------------|
@@ -162,8 +172,9 @@ If SQL is enabled, the Broker will emit the following metrics for SQL.
|`ingest/segments/count`|Count of final segments created by job (includes tombstones). | `dataSource`, `taskId`, `taskType`, `groupId`, `taskIngestionMode`, `tags` |At least `1`.|
|`ingest/tombstones/count`|Count of tombstones created by job. | `dataSource`, `taskId`, `taskType`, `groupId`, `taskIngestionMode`, `tags` |Zero or more for replace. Always zero for non-replace tasks (always zero for legacy replace, see below).|
-The `taskIngestionMode` dimension includes the following modes:
-* `APPEND`: a native ingestion job appending to existing segments
+The `taskIngestionMode` dimension includes the following modes:
+
+* `APPEND`: a native ingestion job appending to existing segments
* `REPLACE_LEGACY`: the original replace before tombstones
* `REPLACE`: a native ingestion job replacing existing segments using tombstones
@@ -179,7 +190,7 @@ task's `IOConfig` as follows:
|`false`|`true`|`REPLACE`|
The `tags` dimension is reported only for metrics emitted from ingestion tasks whose ingest spec specifies the `tags`
-field in the `context` field of the ingestion spec. `tags` is expected to be a map of string to object.
+field in the `context` field of the ingestion spec. `tags` is expected to be a map of string to object.
### Ingestion metrics for Kafka
@@ -203,6 +214,14 @@ These metrics apply to the [Kinesis indexing service](../development/extensions-
|`ingest/kinesis/avgLag/time`|Average lag time in milliseconds between the current message sequence number consumed by the Kinesis indexing tasks and latest sequence number in Kinesis across all shards. Minimum emission period for this metric is a minute.|`dataSource`, `stream`, `tags`|Greater than 0, up to max Kinesis retention period in milliseconds. |
|`ingest/kinesis/partitionLag/time`|Partition-wise lag time in milliseconds between the current message sequence number consumed by the Kinesis indexing tasks and latest sequence number in Kinesis. Minimum emission period for this metric is a minute.|`dataSource`, `stream`, `partition`, `tags`|Greater than 0, up to max Kinesis retention period in milliseconds. |
+### Compaction metrics
+
+[Compaction tasks](../data-management/compaction.md) emit the following metrics.
+
+|Metric|Description|Dimensions|Normal value|
+|------|-----------|----------|------------|
+|`compact/segmentAnalyzer/fetchAndProcessMillis`|Time taken to fetch and process segments to infer the schema for the compaction task to run.|`dataSource`, `taskId`, `taskType`, `groupId`,`tags`| Varies. A high value indicates compaction tasks will speed up from explicitly setting the data schema. |
+
### Other ingestion metrics
Streaming ingestion tasks and certain types of
@@ -271,8 +290,8 @@ If the JVM does not support CPU time measurement for the current thread, `ingest
## Shuffle metrics (Native parallel task)
-The shuffle metrics can be enabled by adding `org.apache.druid.indexing.worker.shuffle.ShuffleMonitor` in `druid.monitoring.monitors`
-See [Enabling Metrics](../configuration/index.md#enabling-metrics) for more details.
+The shuffle metrics can be enabled by adding `org.apache.druid.indexing.worker.shuffle.ShuffleMonitor` in `druid.monitoring.monitors`.
+See [Enabling metrics](../configuration/index.md#enabling-metrics) for more details.
|Metric|Description|Dimensions|Normal value|
|------|-----------|----------|------------|
@@ -312,6 +331,10 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina
|`compact/task/count`|Number of tasks issued in the auto compaction run.| |Varies|
|`compactTask/maxSlot/count`|Maximum number of task slots available for auto compaction tasks in the auto compaction run.| |Varies|
|`compactTask/availableSlot/count`|Number of available task slots that can be used for auto compaction tasks in the auto compaction run. This is the max number of task slots minus any currently running compaction tasks.| |Varies|
+|`killTask/availableSlot/count`| Number of available task slots that can be used for auto kill tasks in the auto kill run. This is the max number of task slots minus any currently running auto kill tasks. | |Varies|
+|`killTask/maxSlot/count`| Maximum number of task slots available for auto kill tasks in the auto kill run. | |Varies|
+|`kill/task/count`| Number of tasks issued in the auto kill run. | |Varies|
+|`kill/pendingSegments/count`|Number of stale pending segments deleted from the metadata store.|`dataSource`|Varies|
|`segment/waitCompact/bytes`|Total bytes of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction).|`dataSource`|Varies|
|`segment/waitCompact/count`|Total number of segments of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction).|`dataSource`|Varies|
|`interval/waitCompact/count`|Total number of intervals of this datasource waiting to be compacted by the auto compaction (only consider intervals/segments that are eligible for auto compaction).|`dataSource`|Varies|
@@ -338,7 +361,7 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina
|Metric|Description|Dimensions|Normal value|
|------|-----------|----------|------------|
-| `service/heartbeat` | Metric indicating the service is up. `ServiceStatusMonitor` must be enabled. |`leader` on the Overlord and Coordinator.|1|
+| `service/heartbeat` | Metric indicating the service is up. `ServiceStatusMonitor` must be enabled. | `leader` on the Overlord and Coordinator. `workerVersion`, `category`, `status` on the Middle Manager. `taskId`, `groupId`, `taskType`, `dataSource` on the Peon |1|
### Historical
@@ -391,7 +414,10 @@ The following metric is only available if the `EventReceiverFirehoseMonitor` mod
|`ingest/events/buffered`|Number of events queued in the `EventReceiverFirehose` buffer.|`serviceName`, `dataSource`, `taskId`, `taskType`, `bufferCapacity`|Equal to the current number of events in the buffer queue.|
|`ingest/bytes/received`|Number of bytes received by the `EventReceiverFirehose`.|`serviceName`, `dataSource`, `taskId`, `taskType`|Varies|
-## Sys
+## Sys [Deprecated]
+
+> SysMonitor is now deprecated and will be removed in future releases.
+> Instead, use the new OSHI monitor called [OshiSysMonitor](#oshisysmonitor). The new monitor has a wider support for different machine architectures including ARM instances.
These metrics are only available if the `SysMonitor` module is included.
@@ -414,6 +440,51 @@ These metrics are only available if the `SysMonitor` module is included.
|`sys/storage/used`|Disk space used|`fsDirName`|Varies|
|`sys/cpu`|CPU used|`cpuName`, `cpuTime`|Varies|
+## OshiSysMonitor
+
+These metrics are only available if the `OshiSysMonitor` module is included.
+
+|Metric|Description|Dimensions|Normal Value|
+|------|-----------|----------|------------|
+|`sys/swap/free`|Free swap||Varies|
+|`sys/swap/max`|Max swap||Varies|
+|`sys/swap/pageIn`|Paged in swap||Varies|
+|`sys/swap/pageOut`|Paged out swap||Varies|
+|`sys/disk/write/count`|Writes to disk|`diskName`|Varies|
+|`sys/disk/read/count`|Reads from disk|`diskName`|Varies|
+|`sys/disk/write/size`|Bytes written to disk. One indicator of the amount of paging occurring for segments.|`diskName`|Varies|
+|`sys/disk/read/size`|Bytes read from disk. One indicator of the amount of paging occurring for segments.|`diskName`|Varies|
+|`sys/disk/queue`|Disk queue length. Measures number of requests waiting to be processed by disk|`diskName`|Generally 0|
+|`sys/disk/transferTime`|Transfer time to read from or write to disk|`diskName`|Depends on hardware|
+|`sys/net/write/size`|Bytes written to the network|`netName`, `netAddress`, `netHwaddr`|Varies|
+|`sys/net/read/size`|Bytes read from the network|`netName`, `netAddress`, `netHwaddr`|Varies|
+|`sys/net/read/packets`|Total packets read from the network|`netName`, `netAddress`, `netHwaddr`|Varies|
+|`sys/net/write/packets`|Total packets written to the network|`netName`, `netAddress`, `netHwaddr`|Varies|
+|`sys/net/read/errors`|Total network read errors|`netName`, `netAddress`, `netHwaddr`|Generally 0|
+|`sys/net/write/errors`|Total network write errors|`netName`, `netAddress`, `netHwaddr`|Generally 0|
+|`sys/net/read/dropped`|Total packets dropped coming from network|`netName`, `netAddress`, `netHwaddr`|Generally 0|
+|`sys/net/write/collisions`|Total network write collisions|`netName`, `netAddress`, `netHwaddr`|Generally 0|
+|`sys/fs/used`|Filesystem bytes used |`fsDevName`, `fsDirName`|< max|
+|`sys/fs/max`|Filesystem bytes max |`fsDevName`, `fsDirName`|Varies|
+|`sys/fs/files/count`|Filesystem total IO nodes |`fsDevName`, `fsDirName`|< max|
+|`sys/fs/files/free`|Filesystem free IO nodes|`fsDevName`, `fsDirName`| Varies |
+|`sys/mem/used`|Memory used||< max|
+|`sys/mem/max`|Memory max||Varies|
+|`sys/mem/free`|Memory free||Varies|
+|`sys/storage/used`|Disk space used|`fsDirName`|Varies|
+|`sys/cpu`|CPU used|`cpuName`, `cpuTime`|Varies|
+|`sys/uptime`|Total system uptime||Varies|
+|`sys/la/{i}`|System CPU load averages over past `i` minutes, where `i={1,5,15}`||Varies|
+|`sys/tcpv4/activeOpens`|Total TCP active open connections||Varies|
+|`sys/tcpv4/passiveOpens`|Total TCP passive open connections||Varies|
+|`sys/tcpv4/attemptFails`|Total TCP active connection failures||Generally 0|
+|`sys/tcpv4/estabResets`|Total TCP connection resets||Generally 0|
+|`sys/tcpv4/in/segs`|Total segments received in connection||Varies|
+|`sys/tcpv4/in/errs`|Errors while reading segments||Generally 0|
+|`sys/tcpv4/out/segs`|Total segments sent||Varies|
+|`sys/tcpv4/out/rsts`|Total "out reset" packets sent to reset the connection||Generally 0|
+|`sys/tcpv4/retrans/segs`|Total segments re-transmitted||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`.
diff --git a/docs/operations/mixed-workloads.md b/docs/operations/mixed-workloads.md
index c70926c12838..19a4b204d83f 100644
--- a/docs/operations/mixed-workloads.md
+++ b/docs/operations/mixed-workloads.md
@@ -60,7 +60,9 @@ Set the following query laning properties in the `broker/runtime.properties` fil
* `druid.query.scheduler.laning.strategy` – The strategy used to assign queries to lanes.
You can use the built-in [“high/low” laning strategy](../configuration/index.md#highlow-laning-strategy), or [define your own laning strategy manually](../configuration/index.md#manual-laning-strategy).
* `druid.query.scheduler.numThreads` – The total number of queries that can be served per Broker. We recommend setting this value to 1-2 less than `druid.server.http.numThreads`.
- > The query scheduler by default does not limit the number of queries that a Broker can serve. Setting this property to a bounded number limits the thread count. If the allocated threads are all occupied, any incoming query, including interactive queries, will be rejected with an HTTP 429 status code.
+:::info
+ The query scheduler by default does not limit the number of queries that a Broker can serve. Setting this property to a bounded number limits the thread count. If the allocated threads are all occupied, any incoming query, including interactive queries, will be rejected with an HTTP 429 status code.
+:::
### Lane-specific properties
diff --git a/docs/operations/other-hadoop.md b/docs/operations/other-hadoop.md
index 14a141a1958e..f5e5839a907c 100644
--- a/docs/operations/other-hadoop.md
+++ b/docs/operations/other-hadoop.md
@@ -89,7 +89,7 @@ classloader.
2. Batch ingestion uses jars from `hadoop-dependencies/` to submit Map/Reduce jobs (location customizable via the
`druid.extensions.hadoopDependenciesDir` runtime property; see [Configuration](../configuration/index.md#extensions)).
-`hadoop-client:2.8.5` is the default version of the Hadoop client bundled with Druid for both purposes. This works with
+The default version of the Hadoop client bundled with Druid is `3.3.6`. This works with
many Hadoop distributions (the version does not necessarily need to match), but if you run into issues, you can instead
have Druid load libraries that exactly match your distribution. To do this, either copy the jars from your Hadoop
cluster, or use the `pull-deps` tool to download the jars from a Maven repository.
diff --git a/docs/operations/pull-deps.md b/docs/operations/pull-deps.md
index 2e375f925c2a..b78d7c532887 100644
--- a/docs/operations/pull-deps.md
+++ b/docs/operations/pull-deps.md
@@ -134,6 +134,8 @@ Note that if you specify `--defaultVersion`, you don't have to put version infor
java -classpath "/my/druid/lib/*" org.apache.druid.cli.Main tools pull-deps --defaultVersion {{DRUIDVERSION}} --clean -c org.apache.druid.extensions:mysql-metadata-storage -h org.apache.hadoop:hadoop-client:2.3.0 -h org.apache.hadoop:hadoop-client:2.4.0
```
-> Please note to use the pull-deps tool you must know the Maven groupId, artifactId, and version of your extension.
->
-> For Druid community extensions listed [here](../configuration/extensions.md), the groupId is "org.apache.druid.extensions.contrib" and the artifactId is the name of the extension.
+:::info
+ Please note to use the pull-deps tool you must know the Maven groupId, artifactId, and version of your extension.
+
+ For Druid community extensions listed [here](../configuration/extensions.md), the groupId is "org.apache.druid.extensions.contrib" and the artifactId is the name of the extension.
+:::
diff --git a/docs/operations/rule-configuration.md b/docs/operations/rule-configuration.md
index 8d12beac96ca..4667817f536e 100644
--- a/docs/operations/rule-configuration.md
+++ b/docs/operations/rule-configuration.md
@@ -95,7 +95,9 @@ curl --location --request GET 'http://localhost:8888/druid/coordinator/v1/rules'
The rules API accepts an array of rules as JSON objects. The JSON object you send in the API request for each rule is specific to the rules types outlined below.
-> You must pass the entire array of rules, in your desired order, with each API request. Each POST request to the rules API overwrites the existing rules for the specified datasource.
+:::info
+ You must pass the entire array of rules, in your desired order, with each API request. Each POST request to the rules API overwrites the existing rules for the specified datasource.
+:::
The order of rules is very important. The Coordinator reads rules in the order in which they appear in the rules list. For example, in the following screenshot the Coordinator evaluates data against rule 1, then rule 2, then rule 3:
@@ -107,7 +109,7 @@ In the web console you can use the up and down arrows on the right side of the i
## Load rules
-Load rules define how Druid assigns segments to [historical process tiers](./mixed-workloads.md#historical-tiering), and how many replicas of a segment exist in each tier.
+Load rules define how Druid assigns segments to [Historical process tiers](./mixed-workloads.md#historical-tiering), and how many replicas of a segment exist in each tier.
If you have a single tier, Druid automatically names the tier `_default`. If you define an additional tier, you must define a load rule to specify which segments to load on that tier. Until you define a load rule, your new tier remains empty.
@@ -120,6 +122,8 @@ All load rules can have these properties:
Specific types of load rules discussed below may have other properties too.
+Load rules are also how you take advantage of the resource savings that [query the data from deep storage](../querying/query-from-deep-storage.md) provides. One way to configure data so that certain segments are not loaded onto Historical tiers but are available to query from deep storage is to set `tieredReplicants` to an empty array and `useDefaultTierForNull` to `false` for those segments, either by interval or by period.
+
### Forever load rule
The forever load rule assigns all datasource segments to specified tiers. It is the default rule Druid applies to datasources. Forever load rules have type `loadForever`.
@@ -167,7 +171,7 @@ Set the following properties:
- the segment interval starts any time after the rule interval starts.
You can use this property to load segments with future start and end dates, where "future" is relative to the time when the Coordinator evaluates data against the rule. Defaults to `true`.
-- `tieredReplicants`: a map of tier names to the number of segment replicas for that tier.
+- `tieredReplicants`: a map of tier names to the number of segment replicas for that tier.
- `useDefaultTierForNull`: This parameter determines the default value of `tieredReplicants` and only has an effect if the field is not present. The default value of `useDefaultTierForNull` is true.
### Interval load rule
@@ -190,7 +194,7 @@ Interval load rules have type `loadByInterval`. The following example places one
Set the following properties:
- `interval`: the load interval specified as an [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) range encoded as a string.
-- `tieredReplicants`: a map of tier names to the number of segment replicas for that tier.
+- `tieredReplicants`: a map of tier names to the number of segment replicas for that tier.
- `useDefaultTierForNull`: This parameter determines the default value of `tieredReplicants` and only has an effect if the field is not present. The default value of `useDefaultTierForNull` is true.
## Drop rules
@@ -256,7 +260,7 @@ Set the following property:
### Interval drop rule
-You can use a drop interval rule to prevent Druid from loading a specified range of data onto any tier. The range is typically your oldest data. The dropped data resides in cold storage, but is not queryable. If you need to query the data, update or remove the interval drop rule so that Druid reloads the data.
+You can use a drop interval rule to prevent Druid from loading a specified range of data onto any tier. The range is typically your oldest data. The dropped data resides in deep storage and can still be [queried from deep storage](../querying/query-from-deep-storage.md).
Interval drop rules have type `dropByInterval` and the following JSON structure:
diff --git a/docs/operations/security-overview.md b/docs/operations/security-overview.md
index 2fa4b45f3889..fa01e1b7e948 100644
--- a/docs/operations/security-overview.md
+++ b/docs/operations/security-overview.md
@@ -32,8 +32,9 @@ By default, security features in Druid are disabled, which simplifies the initia
The following recommendations apply to the Druid cluster setup:
* Run Druid as an unprivileged Unix user. Do not run Druid as the root user.
- > **WARNING!** \
- Druid administrators have the same OS permissions as the Unix user account running Druid. See [Authentication and authorization model](security-user-auth.md#authentication-and-authorization-model). If the Druid process is running under the OS root user account, then Druid administrators can read or write all files that the root account has access to, including sensitive files such as `/etc/passwd`.
+:::caution
+Druid administrators have the same OS permissions as the Unix user account running Druid. See [Authentication and authorization model](security-user-auth.md#authentication-and-authorization-model). If the Druid process is running under the OS root user account, then Druid administrators can read or write all files that the root account has access to, including sensitive files such as `/etc/passwd`.
+:::
* Enable authentication to the Druid cluster for production environments and other environments that can be accessed by untrusted networks.
* Enable authorization and do not expose the web console without authorization enabled. If authorization is not enabled, any user that has access to the web console has the same privileges as the operating system user that runs the web console process.
* Grant users the minimum permissions necessary to perform their functions. For instance, do not allow users who only need to query data to write to data sources or view state.
@@ -82,7 +83,9 @@ keytool -import -file public.cert -alias druid -keystore truststore.jks
Druid uses Jetty as its embedded web server. See [Configuring SSL/TLS KeyStores
](https://www.eclipse.org/jetty/documentation/jetty-11/operations-guide/index.html#og-keystore) from the Jetty documentation.
- > WARNING: Do not use self-signed certificates for production environments. Instead, rely on your current public key infrastructure to generate and distribute trusted keys.
+:::caution
+Do not use self-signed certificates for production environments. Instead, rely on your current public key infrastructure to generate and distribute trusted keys.
+:::
### Update Druid TLS configurations
Edit `common.runtime.properties` for all Druid services on all nodes. Add or update the following TLS options. Restart the cluster when you are finished.
@@ -194,15 +197,19 @@ The following diagram depicts the authorization model, and the relationship betw
The following steps walk through a sample setup procedure:
-> The default Coordinator API port is 8081 for non-TLS connections and 8281 for secured connections.
+:::info
+ The default Coordinator API port is 8081 for non-TLS connections and 8281 for secured connections.
+:::
1. Create a user by issuing a POST request to `druid-ext/basic-security/authentication/db/MyBasicMetadataAuthenticator/users/`.
Replace `` with the *new* username you are trying to create. For example:
```bash
curl -u admin:password1 -XPOST https://my-coordinator-ip:8281/druid-ext/basic-security/authentication/db/MyBasicMetadataAuthenticator/users/myname
```
- > If you have TLS enabled, be sure to adjust the curl command accordingly. For example, if your Druid servers use self-signed certificates,
- you may choose to include the `insecure` curl option to forgo certificate checking for the curl command.
+:::info
+ If you have TLS enabled, be sure to adjust the curl command accordingly. For example, if your Druid servers use self-signed certificates,
+you may choose to include the `insecure` curl option to forgo certificate checking for the curl command.
+:::
2. Add a credential for the user by issuing a POST request to `druid-ext/basic-security/authentication/db/MyBasicMetadataAuthenticator/users//credentials`. For example:
```bash
@@ -244,7 +251,9 @@ The following steps walk through a sample setup procedure:
}
]
```
- > Note: Druid treats the resource name as a regular expression (regex). You can use a specific datasource name or regex to grant permissions for multiple datasources at a time.
+:::info
+ Note: Druid treats the resource name as a regular expression (regex). You can use a specific datasource name or regex to grant permissions for multiple datasources at a time.
+:::
## Configuring an LDAP authenticator
@@ -263,7 +272,9 @@ From the innermost layer:
1. Druid processes have the same access to the local files granted to the specified system user running the process.
2. The Druid ingestion system can create new processes to execute tasks. Those tasks inherit the user of their parent process. This means that any user authorized to submit an ingestion task can use the ingestion task permissions to read or write any local files or external resources that the Druid process has access to.
-> Note: Only grant the `DATASOURCE WRITE` to trusted users because they can act as the Druid process.
+:::info
+ Note: Only grant the `DATASOURCE WRITE` to trusted users because they can act as the Druid process.
+:::
Within the cluster:
1. Druid assumes it operates on an isolated, protected network where no reachable IP within the network is under adversary control. When you implement Druid, take care to setup firewalls and other security measures to secure both inbound and outbound connections.
diff --git a/docs/operations/segment-optimization.md b/docs/operations/segment-optimization.md
index e685671e45ff..73af2444122f 100644
--- a/docs/operations/segment-optimization.md
+++ b/docs/operations/segment-optimization.md
@@ -51,12 +51,14 @@ You may need to consider the followings to optimize your segments.
doesn't match with the "number of rows per segment", please consider optimizing
number of rows per segment rather than this value.
-> The above recommendation works in general, but the optimal setting can
-> vary based on your workload. For example, if most of your queries
-> are heavy and take a long time to process each row, you may want to make
-> segments smaller so that the query processing can be more parallelized.
-> If you still see some performance issue after optimizing segment size,
-> you may need to find the optimal settings for your workload.
+:::info
+ The above recommendation works in general, but the optimal setting can
+ vary based on your workload. For example, if most of your queries
+ are heavy and take a long time to process each row, you may want to make
+ segments smaller so that the query processing can be more parallelized.
+ If you still see some performance issue after optimizing segment size,
+ you may need to find the optimal settings for your workload.
+:::
There might be several ways to check if the compaction is necessary. One way
is using the [System Schema](../querying/sql-metadata-tables.md#system-schema). The
diff --git a/docs/operations/web-console.md b/docs/operations/web-console.md
index 4553658acba6..ae4c30142a91 100644
--- a/docs/operations/web-console.md
+++ b/docs/operations/web-console.md
@@ -35,10 +35,12 @@ Access the web console at the following address:
http://:
```
-> **Security note:** Without [Druid user permissions](../operations/security-overview.md) configured, any user of the
+:::info
+ **Security note:** Without [Druid user permissions](../operations/security-overview.md) configured, any user of the
API or web console has effectively the same level of access to local files and network services as the user under which
Druid runs. It is a best practice to avoid running Druid as the root user, and to use Druid permissions or network
firewalls to restrict which users have access to potentially sensitive resources.
+:::
This topic presents the high-level features and functionality of the web console.
diff --git a/docs/querying/aggregations.md b/docs/querying/aggregations.md
index fb43edf43d68..596d66bffd20 100644
--- a/docs/querying/aggregations.md
+++ b/docs/querying/aggregations.md
@@ -22,10 +22,12 @@ title: "Aggregations"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes the native
-> language. For information about aggregators available in SQL, refer to the
-> [SQL documentation](sql-aggregations.md).
+:::info
+Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+This document describes the native
+language. For information about aggregators available in SQL, refer to the
+[SQL documentation](sql-aggregations.md).
+:::
You can use aggregations:
- in the ingestion spec during ingestion to summarize data before it enters Apache Druid.
@@ -39,8 +41,14 @@ The following sections list the available aggregate functions. Unless otherwise
`count` computes the count of Druid rows that match the filters.
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "count". | Yes |
+| `name` | Output name of the aggregator | Yes |
+
+Example:
```json
-{ "type" : "count", "name" : }
+{ "type" : "count", "name" : "count" }
```
The `count` aggregator counts the number of Druid rows, which does not always reflect the number of raw events ingested.
@@ -50,94 +58,121 @@ query time.
### Sum aggregators
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "longSum", "doubleSum", or "floatSum". | Yes |
+| `name` | Output name for the summed value. | Yes |
+| `fieldName` | Name of the input column to sum over. | No. You must specify `fieldName` or `expression`. |
+| `expression` | You can specify an inline [expression](./math-expr.md) as an alternative to `fieldName`. | No. You must specify `fieldName` or `expression`. |
+
#### `longSum` aggregator
Computes the sum of values as a 64-bit, signed integer.
+Example:
```json
-{ "type" : "longSum", "name" : , "fieldName" : }
+{ "type" : "longSum", "name" : "sumLong", "fieldName" : "aLong" }
```
-The `longSum` aggregator takes the following properties:
-* `name`: Output name for the summed value
-* `fieldName`: Name of the metric column to sum over
-
#### `doubleSum` aggregator
Computes and stores the sum of values as a 64-bit floating point value. Similar to `longSum`.
+Example:
```json
-{ "type" : "doubleSum", "name" : , "fieldName" : }
+{ "type" : "doubleSum", "name" : "sumDouble", "fieldName" : "aDouble" }
```
#### `floatSum` aggregator
Computes and stores the sum of values as a 32-bit floating point value. Similar to `longSum` and `doubleSum`.
+Example:
```json
-{ "type" : "floatSum", "name" : , "fieldName" : }
+{ "type" : "floatSum", "name" : "sumFloat", "fieldName" : "aFloat" }
```
### Min and max aggregators
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "doubleMin", "doubleMax", "floatMin", "floatMax", "longMin", or "longMax". | Yes |
+| `name` | Output name for the min or max value. | Yes |
+| `fieldName` | Name of the input column to compute the minimum or maximum value over. | No. You must specify `fieldName` or `expression`. |
+| `expression` | You can specify an inline [expression](./math-expr.md) as an alternative to `fieldName`. | No. You must specify `fieldName` or `expression`. |
+
#### `doubleMin` aggregator
-`doubleMin` computes the minimum of all metric values and Double.POSITIVE_INFINITY.
+`doubleMin` computes the minimum of all input values and null if `druid.generic.useDefaultValueForNull` is false or Double.POSITIVE_INFINITY if true.
+Example:
```json
-{ "type" : "doubleMin", "name" : , "fieldName" : }
+{ "type" : "doubleMin", "name" : "maxDouble", "fieldName" : "aDouble" }
```
#### `doubleMax` aggregator
-`doubleMax` computes the maximum of all metric values and Double.NEGATIVE_INFINITY.
+`doubleMax` computes the maximum of all input values and null if `druid.generic.useDefaultValueForNull` is false or Double.NEGATIVE_INFINITY if true.
+Example:
```json
-{ "type" : "doubleMax", "name" : , "fieldName" : }
+{ "type" : "doubleMax", "name" : "minDouble", "fieldName" : "aDouble" }
```
#### `floatMin` aggregator
-`floatMin` computes the minimum of all metric values and Float.POSITIVE_INFINITY.
+`floatMin` computes the minimum of all input values and null if `druid.generic.useDefaultValueForNull` is false or Float.POSITIVE_INFINITY if true.
+Example:
```json
-{ "type" : "floatMin", "name" : , "fieldName" : }
+{ "type" : "floatMin", "name" : "minFloat", "fieldName" : "aFloat" }
```
#### `floatMax` aggregator
-`floatMax` computes the maximum of all metric values and Float.NEGATIVE_INFINITY.
+`floatMax` computes the maximum of all input values and null if `druid.generic.useDefaultValueForNull` is false or Float.NEGATIVE_INFINITY if true.
+Example:
```json
-{ "type" : "floatMax", "name" : , "fieldName" : }
+{ "type" : "floatMax", "name" : "maxFloat", "fieldName" : "aFloat" }
```
#### `longMin` aggregator
-`longMin` computes the minimum of all metric values and Long.MAX_VALUE.
+`longMin` computes the minimum of all input values and null if `druid.generic.useDefaultValueForNull` is false or Long.MAX_VALUE if true.
+Example:
```json
-{ "type" : "longMin", "name" : , "fieldName" : }
+{ "type" : "longMin", "name" : "minLong", "fieldName" : "aLong" }
```
#### `longMax` aggregator
-`longMax` computes the maximum of all metric values and Long.MIN_VALUE.
+`longMax` computes the maximum of all metric values and null if `druid.generic.useDefaultValueForNull` is false or Long.MIN_VALUE if true.
+Example:
```json
-{ "type" : "longMax", "name" : , "fieldName" : }
+{ "type" : "longMax", "name" : "maxLong", "fieldName" : "aLong" }
```
### `doubleMean` aggregator
-Computes and returns the arithmetic mean of a column's values as a 64-bit floating point value. `doubleMean` is a query time aggregator only. It is not available for indexing.
+Computes and returns the arithmetic mean of a column's values as a 64-bit floating point value.
-To accomplish mean aggregation on ingestion, refer to the [Quantiles aggregator](../development/extensions-core/datasketches-quantiles.md#aggregator) from the DataSketches extension.
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "doubleMean". | Yes |
+| `name` | Output name for the mean value. | Yes |
+| `fieldName` | Name of the input column to compute the arithmetic mean value over. | Yes |
+Example:
```json
-{ "type" : "doubleMean", "name" : , "fieldName" : }
+{ "type" : "doubleMean", "name" : "aMean", "fieldName" : "aDouble" }
```
+`doubleMean` is a query time aggregator only. It is not available for indexing. To accomplish mean aggregation on ingestion, refer to the [Quantiles aggregator](../development/extensions-core/datasketches-quantiles.md#aggregator) from the DataSketches extension.
+
+
### First and last aggregators
The first and last aggregators determine the metric values that respectively correspond to the earliest and latest values of a time column.
@@ -147,111 +182,131 @@ The string-typed aggregators, `stringFirst` and `stringLast`, are supported for
Queries with first or last aggregators on a segment created with rollup return the rolled up value, not the first or last value from the raw ingested data.
-#### `doubleFirst` aggregator
+#### Numeric first and last aggregators
-`doubleFirst` computes the metric value with the minimum value for time column or 0 in default mode, or `null` in SQL-compatible mode if no row exists.
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "doubleFirst", "doubleLast", "floatFirst", "floatLast", "longFirst", "longLast". | Yes |
+| `name` | Output name for the first or last value. | Yes |
+| `fieldName` | Name of the input column to compute the first or last value over. | Yes |
+| `timeColumn` | Name of the input column to use for time values. Must be a LONG typed column. | No. Defaults to `__time`. |
+##### `doubleFirst` aggregator
+
+`doubleFirst` computes the input value with the minimum value for time column or 0 in default mode, or `null` in SQL-compatible mode if no row exists.
+
+Example:
```json
{
"type" : "doubleFirst",
- "name" : ,
- "fieldName" : ,
- "timeColumn" : # (optional, defaults to __time)
+ "name" : "firstDouble",
+ "fieldName" : "aDouble"
}
```
-#### `doubleLast` aggregator
+##### `doubleLast` aggregator
-`doubleLast` computes the metric value with the maximum value for time column or 0 in default mode, or `null` in SQL-compatible mode if no row exists.
+`doubleLast` computes the input value with the maximum value for time column or 0 in default mode, or `null` in SQL-compatible mode if no row exists.
+Example:
```json
{
"type" : "doubleLast",
- "name" : ,
- "fieldName" : ,
- "timeColumn" : # (optional, defaults to __time)
+ "name" : "lastDouble",
+ "fieldName" : "aDouble",
+ "timeColumn" : "longTime"
}
```
-#### `floatFirst` aggregator
+##### `floatFirst` aggregator
-`floatFirst` computes the metric value with the minimum value for time column or 0 in default mode, or `null` in SQL-compatible mode if no row exists.
+`floatFirst` computes the input value with the minimum value for time column or 0 in default mode, or `null` in SQL-compatible mode if no row exists.
+Example:
```json
{
"type" : "floatFirst",
- "name" : ,
- "fieldName" : ,
- "timeColumn" : # (optional, defaults to __time)
+ "name" : "firstFloat",
+ "fieldName" : "aFloat"
}
```
-#### `floatLast` aggregator
+##### `floatLast` aggregator
`floatLast` computes the metric value with the maximum value for time column or 0 in default mode, or `null` in SQL-compatible mode if no row exists.
+Example:
```json
{
"type" : "floatLast",
- "name" : ,
- "fieldName" : ,
- "timeColumn" : # (optional, defaults to __time)
+ "name" : "lastFloat",
+ "fieldName" : "aFloat"
}
```
-#### `longFirst` aggregator
+##### `longFirst` aggregator
`longFirst` computes the metric value with the minimum value for time column or 0 in default mode, or `null` in SQL-compatible mode if no row exists.
+Example:
```json
{
"type" : "longFirst",
- "name" : ,
- "fieldName" : ,
- "timeColumn" : # (optional, defaults to __time)
+ "name" : "firstLong",
+ "fieldName" : "aLong"
}
```
-#### `longLast` aggregator
+##### `longLast` aggregator
`longLast` computes the metric value with the maximum value for time column or 0 in default mode, or `null` in SQL-compatible mode if no row exists.
+Example:
```json
{
"type" : "longLast",
- "name" : ,
- "fieldName" : ,
- "timeColumn" : # (optional, defaults to __time)
+ "name" : "lastLong",
+ "fieldName" : "aLong",
+ "timeColumn" : "longTime"
}
```
+#### String first and last aggregators
+
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "stringFirst", "stringLast". | Yes |
+| `name` | Output name for the first or last value. | Yes |
+| `fieldName` | Name of the input column to compute the first or last value over. | Yes |
+| `timeColumn` | Name of the input column to use for time values. Must be a LONG typed column. | No. Defaults to `__time`. |
+| `maxStringBytes` | Maximum size of string values to accumulate when computing the first or last value per group. Values longer than this will be truncated. | No. Defaults to 1024. |
+
+
#### `stringFirst` aggregator
`stringFirst` computes the metric value with the minimum value for time column or `null` if no row exists.
+Example:
```json
{
"type" : "stringFirst",
- "name" : ,
- "fieldName" : ,
- "maxStringBytes" : # (optional, defaults to 1024)
- "timeColumn" : # (optional, defaults to __time)
+ "name" : "firstString",
+ "fieldName" : "aString",
+ "maxStringBytes" : 2048,
+ "timeColumn" : "longTime"
}
```
-
-
#### `stringLast` aggregator
`stringLast` computes the metric value with the maximum value for time column or `null` if no row exists.
+Example:
```json
{
"type" : "stringLast",
- "name" : ,
- "fieldName" : ,
- "maxStringBytes" : # (optional, defaults to 1024)
- "timeColumn" : # (optional, defaults to __time)
+ "name" : "lastString",
+ "fieldName" : "aString"
}
```
@@ -261,88 +316,73 @@ Queries with first or last aggregators on a segment created with rollup return t
Returns any value including null. This aggregator can simplify and optimize the performance by returning the first encountered value (including null)
-#### `doubleAny` aggregator
+#### Numeric any aggregators
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "doubleAny", "floatAny", or "longAny". | Yes |
+| `name` | Output name for the value. | Yes |
+| `fieldName` | Name of the input column to compute the value over. | Yes |
+
+##### `doubleAny` aggregator
`doubleAny` returns any double metric value.
+Example:
```json
{
"type" : "doubleAny",
- "name" : ,
- "fieldName" :
+ "name" : "anyDouble",
+ "fieldName" : "aDouble"
}
```
-#### `floatAny` aggregator
+##### `floatAny` aggregator
`floatAny` returns any float metric value.
+Example:
```json
{
"type" : "floatAny",
- "name" : ,
- "fieldName" :
+ "name" : "anyFloat",
+ "fieldName" : "aFloat"
}
```
-#### `longAny` aggregator
+##### `longAny` aggregator
`longAny` returns any long metric value.
+Example:
```json
{
"type" : "longAny",
- "name" : ,
- "fieldName" : ,
+ "name" : "anyLong",
+ "fieldName" : "aLong"
}
```
#### `stringAny` aggregator
-`stringAny` returns any string metric value.
-
-```json
-{
- "type" : "stringAny",
- "name" : ,
- "fieldName" : ,
- "maxStringBytes" : # (optional, defaults to 1024),
-}
-```
-
-### JavaScript aggregator
-
-Computes an arbitrary JavaScript function over a set of columns (both metrics and dimensions are allowed). Your
-JavaScript functions are expected to return floating-point values.
-
-```json
-{ "type": "javascript",
- "name": "",
- "fieldNames" : [ , , ... ],
- "fnAggregate" : "function(current, column1, column2, ...) {
-
- return
- }",
- "fnCombine" : "function(partialA, partialB) { return ; }",
- "fnReset" : "function() { return ; }"
-}
-```
+`stringAny` returns any string value present in the input.
-**Example**
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "stringAny". | Yes |
+| `name` | Output name for the value. | Yes |
+| `fieldName` | Name of the input column to compute the value over. | Yes |
+| `maxStringBytes` | Maximum size of string values to accumulate when computing the first or last value per group. Values longer than this will be truncated. | No. Defaults to 1024. |
+Example:
```json
{
- "type": "javascript",
- "name": "sum(log(x)*y) + 10",
- "fieldNames": ["x", "y"],
- "fnAggregate" : "function(current, a, b) { return current + (Math.log(a) * b); }",
- "fnCombine" : "function(partialA, partialB) { return partialA + partialB; }",
- "fnReset" : "function() { return 10; }"
+ "type" : "stringAny",
+ "name" : "anyString",
+ "fieldName" : "aString",
+ "maxStringBytes" : 2048
}
```
-> JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
-
## Approximate aggregations
@@ -361,8 +401,10 @@ Compared to the Theta sketch, the HLL sketch does not support set operations and
#### Cardinality, hyperUnique
-> For new use cases, we recommend evaluating [DataSketches Theta Sketch](../development/extensions-core/datasketches-theta.md) or [DataSketches HLL Sketch](../development/extensions-core/datasketches-hll.md) instead.
-> The DataSketches aggregators are generally able to offer more flexibility and better accuracy than the classic Druid `cardinality` and `hyperUnique` aggregators.
+:::info
+For new use cases, we recommend evaluating [DataSketches Theta Sketch](../development/extensions-core/datasketches-theta.md) or [DataSketches HLL Sketch](../development/extensions-core/datasketches-hll.md) instead.
+The DataSketches aggregators are generally able to offer more flexibility and better accuracy than the classic Druid `cardinality` and `hyperUnique` aggregators.
+:::
The [Cardinality and HyperUnique](../querying/hll-old.md) aggregators are older aggregator implementations available by default in Druid that also provide distinct count estimates using the HyperLogLog algorithm. The newer DataSketches Theta and HLL extension-provided aggregators described above have superior accuracy and performance and are recommended instead.
@@ -404,9 +446,11 @@ We do not recommend the fixed buckets histogram for general use, as its usefulne
#### Approximate Histogram (deprecated)
-> The Approximate Histogram aggregator is deprecated.
-> There are a number of other quantile estimation algorithms that offer better performance, accuracy, and memory footprint.
-> We recommend using [DataSketches Quantiles](../development/extensions-core/datasketches-quantiles.md) instead.
+:::info
+The Approximate Histogram aggregator is deprecated.
+There are a number of other quantile estimation algorithms that offer better performance, accuracy, and memory footprint.
+We recommend using [DataSketches Quantiles](../development/extensions-core/datasketches-quantiles.md) instead.
+:::
The [Approximate Histogram](../development/extensions-core/approximate-histograms.md) extension-provided aggregator also provides quantile estimates and histogram approximations, based on [http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf](http://jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf).
@@ -422,6 +466,119 @@ It is not possible to determine a priori how well this aggregator will behave fo
For these reasons, we have deprecated this aggregator and recommend using the DataSketches Quantiles aggregator instead for new and existing use cases, although we will continue to support Approximate Histogram for backwards compatibility.
+
+## Expression aggregations
+
+### Expression aggregator
+
+Aggregator applicable only at query time. Aggregates results using [Druid expressions](./math-expr.md) functions to facilitate building custom functions.
+
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "expression". | Yes |
+| `name` | The aggregator output name. | Yes |
+| `fields` | The list of aggregator input columns. | Yes |
+| `accumulatorIdentifier` | The variable which identifies the accumulator value in the `fold` and `combine` expressions. | No. Default `__acc`.|
+| `fold` | The expression to accumulate values from `fields`. The result of the expression is stored in `accumulatorIdentifier` and available to the next computation. | Yes |
+| `combine` | The expression to combine the results of various `fold` expressions of each segment when merging results. The input is available to the expression as a variable identified by the `name`. | No. Default to `fold` expression if the expression has a single input in `fields`.|
+| `compare` | The comparator expression which can only refer to two input variables, `o1` and `o2`, where `o1` and `o2` are the output of `fold` or `combine` expressions, and must adhere to the Java comparator contract. If not set, the aggregator will try to fall back to an output type appropriate comparator. | No |
+| `finalize` | The finalize expression which can only refer to a single input variable, `o`. This expression is used to perform any final transformation of the output of the `fold` or `combine` expressions. If not set, then the value is not transformed. | No |
+| `initialValue` | The initial value of the accumulator for the `fold` (and `combine`, if `InitialCombineValue` is null) expression. | Yes |
+| `initialCombineValue` | The initial value of the accumulator for the `combine` expression. | No. Default `initialValue`. |
+| `isNullUnlessAggregated` | Indicates that the default output value should be `null` if the aggregator does not process any rows. If true, the value is `null`, if false, the result of running the expressions with initial values is used instead. | No. Defaults to the value of `druid.generic.useDefaultValueForNull`. |
+| `shouldAggregateNullInputs` | Indicates if the `fold` expression should operate on any `null` input values. | No. Defaults to `true`. |
+| `shouldCombineAggregateNullInputs` | Indicates if the `combine` expression should operate on any `null` input values. | No. Defaults to the value of `shouldAggregateNullInputs`. |
+| `maxSizeBytes` | Maximum size in bytes that variably sized aggregator output types such as strings and arrays are allowed to grow to before the aggregation fails. | No. Default is 8192 bytes. |
+
+#### Example: a "count" aggregator
+The initial value is `0`. `fold` adds `1` for each row processed.
+
+```json
+{
+ "type": "expression",
+ "name": "expression_count",
+ "fields": [],
+ "initialValue": "0",
+ "fold": "__acc + 1",
+ "combine": "__acc + expression_count"
+}
+```
+
+#### Example: a "sum" aggregator
+The initial value is `0`. `fold` adds the numeric value `column_a` for each row processed.
+
+```json
+{
+ "type": "expression",
+ "name": "expression_sum",
+ "fields": ["column_a"],
+ "initialValue": "0",
+ "fold": "__acc + column_a"
+}
+```
+
+#### Example: a "distinct array element" aggregator, sorted by array_length
+The initial value is an empty array. `fold` adds the elements of `column_a` to the accumulator using set semantics, `combine` merges the sets, and `compare` orders the values by `array_length`.
+
+```json
+{
+ "type": "expression",
+ "name": "expression_array_agg_distinct",
+ "fields": ["column_a"],
+ "initialValue": "[]",
+ "fold": "array_set_add(__acc, column_a)",
+ "combine": "array_set_add_all(__acc, expression_array_agg_distinct)",
+ "compare": "if(array_length(o1) > array_length(o2), 1, if (array_length(o1) == array_length(o2), 0, -1))"
+}
+```
+
+#### Example: an "approximate count" aggregator using the built-in hyper-unique
+Similar to the cardinality aggregator, the default value is an empty hyper-unique sketch, `fold` adds the value of `column_a` to the sketch, `combine` merges the sketches, and `finalize` gets the estimated count from the accumulated sketch.
+
+```json
+{
+ "type": "expression",
+ "name": "expression_cardinality",
+ "fields": ["column_a"],
+ "initialValue": "hyper_unique()",
+ "fold": "hyper_unique_add(column_a, __acc)",
+ "combine": "hyper_unique_add(expression_cardinality, __acc)",
+ "finalize": "hyper_unique_estimate(o)"
+}
+```
+
+### JavaScript aggregator
+
+Computes an arbitrary JavaScript function over a set of columns (both metrics and dimensions are allowed). Your
+JavaScript functions are expected to return floating-point values.
+
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "javascript". | Yes |
+| `name` | The aggregator output name. | Yes |
+| `fieldNames` | The list of aggregator input columns. | Yes |
+| `fnAggregate` | JavaScript function that updates partial aggregate based on the current row values, and returns the updated partial aggregate. | Yes |
+| `fnCombine` | JavaScript function to combine partial aggregates and return the combined result. | Yes |
+| `fnReset` | JavaScript function that returns the 'initial' value. | Yes |
+
+#### Example
+
+```json
+{
+ "type": "javascript",
+ "name": "sum(log(x)*y) + 10",
+ "fieldNames": ["x", "y"],
+ "fnAggregate" : "function(current, a, b) { return current + (Math.log(a) * b); }",
+ "fnCombine" : "function(partialA, partialB) { return partialA + partialB; }",
+ "fnReset" : "function() { return 10; }"
+}
+```
+
+:::info
+JavaScript-based functionality is disabled by default. Refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::
+
+
## Miscellaneous aggregations
### Filtered aggregator
@@ -430,17 +587,30 @@ A filtered aggregator wraps any given aggregator, but only aggregates the values
This makes it possible to compute the results of a filtered and an unfiltered aggregation simultaneously, without having to issue multiple queries, and use both results as part of post-aggregations.
-*Note:* If only the filtered results are required, consider putting the filter on the query itself, which will be much faster since it does not require scanning all the data.
+If only the filtered results are required, consider putting the filter on the query itself. This will be much faster since it does not require scanning all the data.
+
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "filtered". | Yes |
+| `name` | The aggregator output name. | No |
+| `aggregator` | Inline aggregator specification. | Yes |
+| `filter` | Inline [filter](./filters.md) specification. | Yes |
+Example:
```json
{
- "type" : "filtered",
- "filter" : {
+ "type": "filtered",
+ "name": "filteredSumLong",
+ "filter": {
"type" : "selector",
- "dimension" : ,
- "value" :
+ "dimension" : "someColumn",
+ "value" : "abcdef"
},
- "aggregator" :
+ "aggregator": {
+ "type": "longSum",
+ "name": "sumLong",
+ "fieldName": "aLong"
+ }
}
```
@@ -450,7 +620,20 @@ A grouping aggregator can only be used as part of GroupBy queries which have a s
each output row that lets you infer whether a particular dimension is included in the sub-grouping used for that row. You can pass
a *non-empty* list of dimensions to this aggregator which *must* be a subset of dimensions that you are grouping on.
-For example, if the aggregator has `["dim1", "dim2"]` as input dimensions and `[["dim1", "dim2"], ["dim1"], ["dim2"], []]` as subtotals, the
+| Property | Description | Required |
+| --- | --- | --- |
+| `type` | Must be "grouping". | Yes |
+| `name` | The aggregator output name. | Yes |
+| `groupings` | The list of columns to use in the grouping set. | Yes |
+
+
+For example, the following aggregator has `["dim1", "dim2"]` as input dimensions:
+
+```json
+{ "type" : "grouping", "name" : "someGrouping", "groupings" : ["dim1", "dim2"] }
+```
+
+and used in a grouping query with `[["dim1", "dim2"], ["dim1"], ["dim2"], []]` as subtotals, the
possible output of the aggregator is:
| subtotal used in query | Output | (bits representation) |
@@ -462,7 +645,3 @@ possible output of the aggregator is:
As the example illustrates, you can think of the output number as an unsigned _n_ bit number where _n_ is the number of dimensions passed to the aggregator.
Druid sets the bit at position X for the number to 0 if the sub-grouping includes a dimension at position X in the aggregator input. Otherwise, Druid sets this bit to 1.
-
-```json
-{ "type" : "grouping", "name" : , "groupings" : [] }
-```
diff --git a/docs/querying/caching.md b/docs/querying/caching.md
index 26fe063e68fa..a84e3d25eee4 100644
--- a/docs/querying/caching.md
+++ b/docs/querying/caching.md
@@ -45,9 +45,11 @@ Druid supports two types of query caching:
Druid invalidates any cache the moment any underlying data change to avoid returning stale results. This is especially important for `table` datasources that have highly-variable underlying data segments, including real-time data segments.
-> **Druid can store cache data on the local JVM heap or in an external distributed key/value store (e.g. memcached)**
->
-> The default is a local cache based upon [Caffeine](https://github.com/ben-manes/caffeine). The default maximum cache storage size is the minimum of 1 GiB / ten percent of maximum runtime memory for the JVM, with no cache expiration. See [Cache configuration](../configuration/index.md#cache-configuration) for information on how to configure cache storage. When using caffeine, the cache is inside the JVM heap and is directly measurable. Heap usage will grow up to the maximum configured size, and then the least recently used segment results will be evicted and replaced with newer results.
+:::info
+ **Druid can store cache data on the local JVM heap or in an external distributed key/value store (e.g. memcached)**
+
+ The default is a local cache based upon [Caffeine](https://github.com/ben-manes/caffeine). The default maximum cache storage size is the minimum of 1 GiB / ten percent of maximum runtime memory for the JVM, with no cache expiration. See [Cache configuration](../configuration/index.md#cache-configuration) for information on how to configure cache storage. When using caffeine, the cache is inside the JVM heap and is directly measurable. Heap usage will grow up to the maximum configured size, and then the least recently used segment results will be evicted and replaced with newer results.
+:::
### Per-segment caching
@@ -99,12 +101,11 @@ Caching does not solve all types of query performance issues. For each cache typ
**Per-segment caching** doesn't work for the following:
- queries containing a sub-query in them. However the output of sub-queries may be cached. See [Query execution](./query-execution.md) for more details on sub-queries execution.
- queries with joins do not support any caching on the broker.
-- GroupBy v2 queries do not support any caching on broker.
+- GroupBy queries do not support segment level caching on broker.
- queries with `bySegment` set in the query context are not cached on the broker.
**Whole-query caching** doesn't work for the following:
- queries that involve an inline datasource or a lookup datasource.
-- GroupBy v2 queries.
- queries with joins.
- queries with a union datasource.
diff --git a/docs/querying/datasource.md b/docs/querying/datasource.md
index e348bc81c660..ae87431587a1 100644
--- a/docs/querying/datasource.md
+++ b/docs/querying/datasource.md
@@ -3,6 +3,10 @@ id: datasource
title: "Datasources"
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
+
+
-
+
+
+
```sql
SELECT column1, column2 FROM "druid"."dataSourceName"
```
-
+
+
+
```json
{
"queryType": "scan",
@@ -48,7 +55,8 @@ SELECT column1, column2 FROM "druid"."dataSourceName"
"intervals": ["0000/3000"]
}
```
-
+
+
The table datasource is the most common type. This is the kind of datasource you get when you perform
[data ingestion](../ingestion/index.md). They are split up into segments, distributed around the cluster,
@@ -72,12 +80,15 @@ To see a list of all table datasources, use the SQL query
### `lookup`
-
-
+
+
+
```sql
SELECT k, v FROM lookup.countries
```
-
+
+
+
```json
{
"queryType": "scan",
@@ -89,7 +100,8 @@ SELECT k, v FROM lookup.countries
"intervals": ["0000/3000"]
}
```
-
+
+
Lookup datasources correspond to Druid's key-value [lookup](lookups.md) objects. In [Druid SQL](sql.md#from),
they reside in the `lookup` schema. They are preloaded in memory on all servers, so they can be accessed rapidly.
@@ -101,19 +113,22 @@ both are always strings.
To see a list of all lookup datasources, use the SQL query
`SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = 'lookup'`.
-> Performance tip: Lookups can be joined with a base table either using an explicit [join](#join), or by using the
-> SQL [`LOOKUP` function](sql-scalar.md#string-functions).
-> However, the join operator must evaluate the condition on each row, whereas the
-> `LOOKUP` function can defer evaluation until after an aggregation phase. This means that the `LOOKUP` function is
-> usually faster than joining to a lookup datasource.
+:::info
+ Performance tip: Lookups can be joined with a base table either using an explicit [join](#join), or by using the
+ SQL [`LOOKUP` function](sql-scalar.md#string-functions).
+ However, the join operator must evaluate the condition on each row, whereas the
+ `LOOKUP` function can defer evaluation until after an aggregation phase. This means that the `LOOKUP` function is
+ usually faster than joining to a lookup datasource.
+:::
Refer to the [Query execution](query-execution.md#table) page for more details on how queries are executed when you
use table datasources.
### `union`
-
-
+
+
+
```sql
SELECT column1, column2
FROM (
@@ -124,7 +139,9 @@ FROM (
SELECT column1, column2 FROM table3
)
```
-
+
+
+
```json
{
"queryType": "scan",
@@ -136,7 +153,8 @@ FROM (
"intervals": ["0000/3000"]
}
```
-
+
+
Unions allow you to treat two or more tables as a single datasource. In SQL, this is done with the UNION ALL operator
applied directly to tables, called a ["table-level union"](sql.md#table-level). In native queries, this is done with a
@@ -158,8 +176,9 @@ use union datasources.
### `inline`
-
-
+
+
+
```json
{
"queryType": "scan",
@@ -175,7 +194,8 @@ use union datasources.
"intervals": ["0000/3000"]
}
```
-
+
+
Inline datasources allow you to query a small amount of data that is embedded in the query itself. They are useful when
you want to write a query on a small amount of data without loading it first. They are also useful as inputs into a
@@ -193,8 +213,9 @@ use inline datasources.
### `query`
-
-
+
+
+
```sql
-- Uses a subquery to count hits per page, then takes the average.
SELECT
@@ -202,7 +223,9 @@ SELECT
FROM
(SELECT page, COUNT(*) AS hits FROM site_traffic GROUP BY page)
```
-
+
+
+
```json
{
"queryType": "timeseries",
@@ -230,7 +253,8 @@ FROM
]
}
```
-
+
+
Query datasources allow you to issue subqueries. In native queries, they can appear anywhere that accepts a
`dataSource` (except underneath a `union`). In SQL, they can appear in the following places, always surrounded by parentheses:
@@ -239,15 +263,18 @@ Query datasources allow you to issue subqueries. In native queries, they can app
- As inputs to a JOIN: ` t1 INNER JOIN t2 ON t1. = t2.`.
- In the WHERE clause: `WHERE { IN | NOT IN } ()`. These are translated to joins by the SQL planner.
-> Performance tip: In most cases, subquery results are fully buffered in memory on the Broker and then further
-> processing occurs on the Broker itself. This means that subqueries with large result sets can cause performance
-> bottlenecks or run into memory usage limits on the Broker. See the [Query execution](query-execution.md#query)
-> page for more details on how subqueries are executed and what limits will apply.
+:::info
+ Performance tip: In most cases, subquery results are fully buffered in memory on the Broker and then further
+ processing occurs on the Broker itself. This means that subqueries with large result sets can cause performance
+ bottlenecks or run into memory usage limits on the Broker. See the [Query execution](query-execution.md#query)
+ page for more details on how subqueries are executed and what limits will apply.
+:::
### `join`
-
-
+
+
+
```sql
-- Joins "sales" with "countries" (using "store" as the join key) to get sales by country.
SELECT
@@ -259,7 +286,9 @@ FROM
GROUP BY
countries.v
```
-
+
+
+
```json
{
"queryType": "groupBy",
@@ -284,7 +313,8 @@ GROUP BY
]
}
```
-
+
+
Join datasources allow you to do a SQL-style join of two datasources. Stacking joins on top of each other allows
you to join arbitrarily many datasources.
@@ -352,9 +382,9 @@ perform best if `d.field` is a string.
4. As of Druid {{DRUIDVERSION}}, the join operator must evaluate the condition for each row. In the future, we expect
to implement both early and deferred condition evaluation, which we expect to improve performance considerably for
common use cases.
-5. Currently, Druid does not support pushing down predicates (condition and filter) past a Join (i.e. into
-Join's children). Druid only supports pushing predicates into the join if they originated from
-above the join. Hence, the location of predicates and filters in your Druid SQL is very important.
+5. Currently, Druid does not support pushing down predicates (condition and filter) past a Join (i.e. into
+Join's children). Druid only supports pushing predicates into the join if they originated from
+above the join. Hence, the location of predicates and filters in your Druid SQL is very important.
Also, as a result of this, comma joins should be avoided.
#### Future work for joins
@@ -371,21 +401,23 @@ future versions:
### `unnest`
-> The unnest datasource is [experimental](../development/experimental.md). Its API and behavior are subject
-> to change in future releases. It is not recommended to use this feature in production at this time.
+:::info
+ The unnest datasource is [experimental](../development/experimental.md). Its API and behavior are subject
+ to change in future releases. It is not recommended to use this feature in production at this time.
+:::
Use the `unnest` datasource to unnest a column with multiple values in an array.
For example, you have a source column that looks like this:
-| Nested |
-| -- |
+| Nested |
+| -- |
| [a, b] |
| [c, d] |
| [e, [f,g]] |
When you use the `unnest` datasource, the unnested column looks like this:
-| Unnested |
+| Unnested |
| -- |
| a |
| b |
diff --git a/docs/querying/datasourcemetadataquery.md b/docs/querying/datasourcemetadataquery.md
index bdc7128ac898..1f1cc0d49d5a 100644
--- a/docs/querying/datasourcemetadataquery.md
+++ b/docs/querying/datasourcemetadataquery.md
@@ -23,9 +23,11 @@ sidebar_label: "DatasourceMetadata"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes a query
-> type that is only available in the native language.
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes a query
+ type that is only available in the native language.
+:::
Data Source Metadata queries return metadata information for a dataSource. These queries return information about:
diff --git a/docs/querying/dimensionspecs.md b/docs/querying/dimensionspecs.md
index 150ac7419f9b..d336d746d450 100644
--- a/docs/querying/dimensionspecs.md
+++ b/docs/querying/dimensionspecs.md
@@ -23,10 +23,12 @@ sidebar_label: "Dimensions"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes the native
-> language. For information about functions available in SQL, refer to the
-> [SQL documentation](sql-scalar.md).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes the native
+ language. For information about functions available in SQL, refer to the
+ [SQL documentation](sql-scalar.md).
+:::
The following JSON fields can be used in a query to operate on dimension values.
@@ -344,7 +346,9 @@ Example for the `__time` dimension:
}
```
-> JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::info
+ JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::
### Registered lookup extraction function
diff --git a/docs/querying/filters.md b/docs/querying/filters.md
index 82fdb8116886..431a29b556bc 100644
--- a/docs/querying/filters.md
+++ b/docs/querying/filters.md
@@ -23,10 +23,12 @@ sidebar_label: "Filters"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes the native
-> language. For information about aggregators available in SQL, refer to the
-> [SQL documentation](sql-scalar.md).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes the native
+ language. For information about aggregators available in SQL, refer to the
+ [SQL documentation](sql-scalar.md).
+:::
A filter is a JSON object indicating which rows of data should be included in the computation for a query. It’s essentially the equivalent of the WHERE clause in SQL.
Filters are commonly applied on dimensions, but can be applied on aggregated metrics, for example, see [Filtered aggregator](./aggregations.md#filtered-aggregator) and [Having filters](./having.md).
@@ -35,263 +37,221 @@ Apache Druid supports the following types of filters.
## Selector filter
-The simplest filter is a selector filter. The selector filter will match a specific dimension with a specific value. Selector filters can be used as the base filters for more complex Boolean expressions of filters.
+The simplest filter is a selector filter. The selector filter matches a specific dimension with a specific value. Selector filters can be used as the base filters for more complex Boolean expressions of filters.
-The grammar for a SELECTOR filter is as follows:
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "selector".| Yes |
+| `dimension` | Input column or virtual column name to filter. | Yes |
+| `value` | String value to match. | No. If not specified the filter matches NULL values. |
+| `extractionFn` | [Extraction function](./dimensionspecs.md#extraction-functions) to apply to `dimension` prior to value matching. See [filtering with extraction functions](#filtering-with-extraction-functions) for details. | No |
-``` json
-"filter": { "type": "selector", "dimension": , "value": }
-```
-
-This is the equivalent of `WHERE = ''` or `WHERE IS NULL`
-(if the `value` is `null`).
+The selector filter can only match against `STRING` (single and multi-valued), `LONG`, `FLOAT`, `DOUBLE` types. Use the newer null and equality filters to match against `ARRAY` or `COMPLEX` types.
-The selector filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
+When the selector filter matches against numeric inputs, the string `value` will be best-effort coerced into a numeric value.
-## Column comparison filter
-
-The column comparison filter is similar to the selector filter, but instead compares dimensions to each other. For example:
+### Example: equivalent of `WHERE someColumn = 'hello'`
``` json
-"filter": { "type": "columnComparison", "dimensions": [, ] }
+{ "type": "selector", "dimension": "someColumn", "value": "hello" }
```
-This is the equivalent of `WHERE = `.
-`dimensions` is list of [DimensionSpecs](./dimensionspecs.md), making it possible to apply an extraction function if needed.
-
-## Regular expression filter
-
-The regular expression filter is similar to the selector filter, but using regular expressions. It matches the specified dimension with the given pattern. The pattern can be any standard [Java regular expression](http://docs.oracle.com/javase/6/docs/api/java/util/regex/Pattern.html).
+### Example: equivalent of `WHERE someColumn IS NULL`
``` json
-"filter": { "type": "regex", "dimension": , "pattern": }
+{ "type": "selector", "dimension": "someColumn", "value": null }
```
-The regex filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
-
-
-## Logical expression filters
-
-### AND
-The grammar for an AND filter is as follows:
+## Equality Filter
-``` json
-"filter": { "type": "and", "fields": [, , ...] }
-```
+The equality filter is a replacement for the selector filter with the ability to match against any type of column. The equality filter is designed to have more SQL compatible behavior than the selector filter and so can not match null values. To match null values use the null filter.
-The filters in fields can be any other filter defined on this page.
+Druid's SQL planner uses the equality filter by default instead of selector filter whenever `druid.generic.useDefaultValueForNull=false`, or if `sqlUseBoundAndSelectors` is set to false on the [SQL query context](./sql-query-context.md).
-### OR
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "equality".| Yes |
+| `column` | Input column or virtual column name to filter. | Yes |
+| `matchValueType` | String specifying the type of value to match. For example `STRING`, `LONG`, `DOUBLE`, `FLOAT`, `ARRAY`, `ARRAY`, or any other Druid type. The `matchValueType` determines how Druid interprets the `matchValue` to assist in converting to the type of the matched `column`. | Yes |
+| `matchValue` | Value to match, must not be null. | Yes |
-The grammar for an OR filter is as follows:
+### Example: equivalent of `WHERE someColumn = 'hello'`
-``` json
-"filter": { "type": "or", "fields": [, , ...] }
+```json
+{ "type": "equals", "column": "someColumn", "matchValueType": "STRING", "matchValue": "hello" }
```
-The filters in fields can be any other filter defined on this page.
+### Example: equivalent of `WHERE someNumericColumn = 1.23`
-### NOT
+```json
+{ "type": "equals", "column": "someNumericColumn", "matchValueType": "DOUBLE", "matchValue": 1.23 }
+```
-The grammar for a NOT filter is as follows:
+### Example: equivalent of `WHERE someArrayColumn = ARRAY[1, 2, 3]`
```json
-"filter": { "type": "not", "field": }
+{ "type": "equals", "column": "someArrayColumn", "matchValueType": "ARRAY", "matchValue": [1, 2, 3] }
```
-The filter specified at field can be any other filter defined on this page.
-## JavaScript filter
+## Null Filter
-The JavaScript filter matches a dimension against the specified JavaScript function predicate. The filter matches values for which the function returns true.
+The null filter is a partial replacement for the selector filter. It is dedicated to matching NULL values.
-The function takes a single argument, the dimension value, and returns either true or false.
+Druid's SQL planner uses the null filter by default instead of selector filter whenever `druid.generic.useDefaultValueForNull=false`, or if `sqlUseBoundAndSelectors` is set to false on the [SQL query context](./sql-query-context.md).
-```json
-{
- "type" : "javascript",
- "dimension" : ,
- "function" : "function(value) { <...> }"
-}
-```
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "null".| Yes |
+| `column` | Input column or virtual column name to filter. | Yes |
-**Example**
-The following matches any dimension values for the dimension `name` between `'bar'` and `'foo'`
+### Example: equivalent of `WHERE someColumn IS NULL`
```json
-{
- "type" : "javascript",
- "dimension" : "name",
- "function" : "function(x) { return(x >= 'bar' && x <= 'foo') }"
-}
+{ "type": "null", "column": "someColumn" }
```
-The JavaScript filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
-
-> JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
-## Extraction filter
-
-> The extraction filter is now deprecated. The selector filter with an extraction function specified
-> provides identical functionality and should be used instead.
+## Column comparison filter
-Extraction filter matches a dimension using some specific [Extraction function](./dimensionspecs.md#extraction-functions).
-The following filter matches the values for which the extraction function has transformation entry `input_key=output_value` where
-`output_value` is equal to the filter `value` and `input_key` is present as dimension.
+The column comparison filter is similar to the selector filter, but compares dimensions to each other. For example:
-**Example**
-The following matches dimension values in `[product_1, product_3, product_5]` for the column `product`
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "selector".| Yes |
+| `dimensions` | List of [`DimensionSpec`](./dimensionspecs.md) to compare. | Yes |
-```json
-{
- "filter": {
- "type": "extraction",
- "dimension": "product",
- "value": "bar_1",
- "extractionFn": {
- "type": "lookup",
- "lookup": {
- "type": "map",
- "map": {
- "product_1": "bar_1",
- "product_5": "bar_1",
- "product_3": "bar_1"
- }
- }
- }
- }
-}
-```
+`dimensions` is list of [DimensionSpecs](./dimensionspecs.md), making it possible to apply an extraction function if needed.
-## Search filter
+Note that the column comparison filter converts all values to strings prior to comparison. This allows differently-typed input columns to match without a cast operation.
-Search filters can be used to filter on partial string matches.
+### Example: equivalent of `WHERE someColumn = someLongColumn`
-```json
+``` json
{
- "filter": {
- "type": "search",
- "dimension": "product",
- "query": {
- "type": "insensitive_contains",
- "value": "foo"
- }
+ "type": "columnComparison",
+ "dimensions": [
+ "someColumn",
+ {
+ "type" : "default",
+ "dimension" : someLongColumn,
+ "outputType": "LONG"
}
+ ]
}
```
-|property|description|required?|
-|--------|-----------|---------|
-|type|This String should always be "search".|yes|
-|dimension|The dimension to perform the search over.|yes|
-|query|A JSON object for the type of search. See [search query spec](#search-query-spec) for more information.|yes|
-|extractionFn|[Extraction function](#filtering-with-extraction-functions) to apply to the dimension|no|
-The search filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
+## Logical expression filters
-### Search query spec
+### AND
-#### Contains
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "and".| Yes |
+| `fields` | List of filter JSON objects, such as any other filter defined on this page or provided by extensions. | Yes |
-|property|description|required?|
-|--------|-----------|---------|
-|type|This String should always be "contains".|yes|
-|value|A String value to run the search over.|yes|
-|caseSensitive|Whether two string should be compared as case sensitive or not|no (default == false)|
-#### Insensitive Contains
+#### Example: equivalent of `WHERE someColumn = 'a' AND otherColumn = 1234 AND anotherColumn IS NULL`
-|property|description|required?|
-|--------|-----------|---------|
-|type|This String should always be "insensitive_contains".|yes|
-|value|A String value to run the search over.|yes|
+``` json
+{
+ "type": "and",
+ "fields": [
+ { "type": "equals", "column": "someColumn", "matchValue": "a", "matchValueType": "STRING" },
+ { "type": "equals", "column": "otherColumn", "matchValue": 1234, "matchValueType": "LONG" },
+ { "type": "null", "column": "anotherColumn" }
+ ]
+}
+```
-Note that an "insensitive_contains" search is equivalent to a "contains" search with "caseSensitive": false (or not
-provided).
+### OR
-#### Fragment
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "or".| Yes |
+| `fields` | List of filter JSON objects, such as any other filter defined on this page or provided by extensions. | Yes |
-|property|description|required?|
-|--------|-----------|---------|
-|type|This String should always be "fragment".|yes|
-|values|A JSON array of String values to run the search over.|yes|
-|caseSensitive|Whether strings should be compared as case sensitive or not. Default: false(insensitive)|no|
+#### Example: equivalent of `WHERE someColumn = 'a' OR otherColumn = 1234 OR anotherColumn IS NULL`
-## In filter
+``` json
+{
+ "type": "or",
+ "fields": [
+ { "type": "equals", "column": "someColumn", "matchValue": "a", "matchValueType": "STRING" },
+ { "type": "equals", "column": "otherColumn", "matchValue": 1234, "matchValueType": "LONG" },
+ { "type": "null", "column": "anotherColumn" }
+ ]
+}
+```
-In filter can be used to express the following SQL query:
+### NOT
-```sql
- SELECT COUNT(*) AS 'Count' FROM `table` WHERE `outlaw` IN ('Good', 'Bad', 'Ugly')
-```
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "not".| Yes |
+| `field` | Filter JSON objects, such as any other filter defined on this page or provided by extensions. | Yes |
-The grammar for a "in" filter is as follows:
+#### Example: equivalent of `WHERE someColumn IS NOT NULL`
```json
-{
- "type": "in",
- "dimension": "outlaw",
- "values": ["Good", "Bad", "Ugly"]
-}
+{ "type": "not", "field": { "type": "null", "column": "someColumn" }}
```
-The "in" filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
-
-If an empty `values` array is passed to the "in" filter, it will simply return an empty result.
-If the `dimension` is a multi-valued dimension, the "in" filter will return true if one of the dimension values is
-in the `values` array.
-
-If the `values` array contains `null`, the "in" filter matches null values. This differs from the SQL IN filter, which
-does not match NULL values.
+## In filter
+The in filter can match input rows against a set of values, where a match occurs if the value is contained in the set.
-## Like filter
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "in".| Yes |
+| `dimension` | Input column or virtual column name to filter. | Yes |
+| `values` | List of string value to match. | Yes |
+| `extractionFn` | [Extraction function](./dimensionspecs.md#extraction-functions) to apply to `dimension` prior to value matching. See [filtering with extraction functions](#filtering-with-extraction-functions) for details. | No |
-Like filters can be used for basic wildcard searches. They are equivalent to the SQL LIKE operator. Special characters
-supported are "%" (matches any number of characters) and "\_" (matches any one character).
-|property|type|description|required?|
-|--------|-----------|---------|---------|
-|type|String|This should always be "like".|yes|
-|dimension|String|The dimension to filter on|yes|
-|pattern|String|LIKE pattern, such as "foo%" or "___bar".|yes|
-|escape|String|An escape character that can be used to escape special characters.|no|
-|extractionFn|[Extraction function](#filtering-with-extraction-functions)| Extraction function to apply to the dimension|no|
+If an empty `values` array is passed to the "in" filter, it will simply return an empty result.
-Like filters support the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
+If the `values` array contains `null`, the "in" filter matches null values. This differs from the SQL IN filter, which
+does not match NULL values.
-This Like filter expresses the condition `last_name LIKE "D%"` (i.e. last_name starts with "D").
+### Example: equivalent of `WHERE `outlaw` IN ('Good', 'Bad', 'Ugly')`
```json
{
- "type": "like",
- "dimension": "last_name",
- "pattern": "D%"
+ "type": "in",
+ "dimension": "outlaw",
+ "values": ["Good", "Bad", "Ugly"]
}
```
+
## Bound filter
Bound filters can be used to filter on ranges of dimension values. It can be used for comparison filtering like
greater than, less than, greater than or equal to, less than or equal to, and "between" (if both "lower" and
"upper" are set).
-|property|type|description|required?|
-|--------|-----------|---------|---------|
-|type|String|This should always be "bound".|yes|
-|dimension|String|The dimension to filter on|yes|
-|lower|String|The lower bound for the filter|no|
-|upper|String|The upper bound for the filter|no|
-|lowerStrict|Boolean|Perform strict comparison on the lower bound (">" instead of ">=")|no, default: false|
-|upperStrict|Boolean|Perform strict comparison on the upper bound ("<" instead of "<=")|no, default: false|
-|ordering|String|Specifies the sorting order to use when comparing values against the bound. Can be one of the following values: "lexicographic", "alphanumeric", "numeric", "strlen", "version". See [Sorting Orders](./sorting-orders.md) for more details.|no, default: "lexicographic"|
-|extractionFn|[Extraction function](#filtering-with-extraction-functions)| Extraction function to apply to the dimension|no|
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "bound". | Yes |
+| `dimension` | Input column or virtual column name to filter. | Yes |
+| `lower` | The lower bound string match value for the filter. | No |
+| `upper`| The upper bound string match value for the filter. | No |
+| `lowerStrict` | Boolean indicating whether to perform strict comparison on the `lower` bound (">" instead of ">="). | No, default: `false` |
+| `upperStrict` | Boolean indicating whether to perform strict comparison on the upper bound ("<" instead of "<="). | No, default: `false`|
+| `ordering` | String that specifies the sorting order to use when comparing values against the bound. Can be one of the following values: `"lexicographic"`, `"alphanumeric"`, `"numeric"`, `"strlen"`, `"version"`. See [Sorting Orders](./sorting-orders.md) for more details. | No, default: `"lexicographic"`|
+| `extractionFn` | [Extraction function](./dimensionspecs.md#extraction-functions) to apply to `dimension` prior to value matching. See [filtering with extraction functions](#filtering-with-extraction-functions) for details. | No |
+
+When the bound filter matches against numeric inputs, the string `lower` and `upper` bound values are best-effort coerced into a numeric value when using the `"numeric"` mode of ordering.
-Bound filters support the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
+The bound filter can only match against `STRING` (single and multi-valued), `LONG`, `FLOAT`, `DOUBLE` types. Use the newer range to match against `ARRAY` or `COMPLEX` types.
-The following bound filter expresses the condition `21 <= age <= 31`:
+Note that the bound filter matches null values if you don't specify a lower bound. Use the range filter if SQL-compatible behavior.
+
+### Example: equivalent to `WHERE 21 <= age <= 31`
```json
{
@@ -303,7 +263,7 @@ The following bound filter expresses the condition `21 <= age <= 31`:
}
```
-This filter expresses the condition `foo <= name <= hoo`, using the default lexicographic sorting order.
+### Example: equivalent to `WHERE 'foo' <= name <= 'hoo'`, using the default lexicographic sorting order
```json
{
@@ -314,7 +274,7 @@ This filter expresses the condition `foo <= name <= hoo`, using the default lexi
}
```
-Using strict bounds, this filter expresses the condition `21 < age < 31`
+### Example: equivalent to `WHERE 21 < age < 31`
```json
{
@@ -328,7 +288,7 @@ Using strict bounds, this filter expresses the condition `21 < age < 31`
}
```
-The user can also specify a one-sided bound by omitting "upper" or "lower". This filter expresses `age < 31`.
+### Example: equivalent to `WHERE age < 31`
```json
{
@@ -340,7 +300,7 @@ The user can also specify a one-sided bound by omitting "upper" or "lower". This
}
```
-Likewise, this filter expresses `age >= 18`
+### Example: equivalent to `WHERE age >= 18`
```json
{
@@ -352,18 +312,154 @@ Likewise, this filter expresses `age >= 18`
```
+## Range filter
+
+The range filter is a replacement for the bound filter. It compares against any type of column and is designed to have has more SQL compliant behavior than the bound filter. It won't match null values, even if you don't specify a lower bound.
+
+Druid's SQL planner uses the range filter by default instead of bound filter whenever `druid.generic.useDefaultValueForNull=false`, or if `sqlUseBoundAndSelectors` is set to false on the [SQL query context](./sql-query-context.md).
+
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "range".| Yes |
+| `column` | Input column or virtual column name to filter. | Yes |
+| `matchValueType` | String specifying the type of bounds to match. For example `STRING`, `LONG`, `DOUBLE`, `FLOAT`, `ARRAY`, `ARRAY`, or any other Druid type. The `matchValueType` determines how Druid interprets the `matchValue` to assist in converting to the type of the matched `column` and also defines the type of comparison used when matching values. | Yes |
+| `lower` | Lower bound value to match. | No. At least one of `lower` or `upper` must not be null. |
+| `upper` | Upper bound value to match. | No. At least one of `lower` or `upper` must not be null. |
+| `lowerOpen` | Boolean indicating if lower bound is open in the interval of values defined by the range (">" instead of ">="). | No |
+| `upperOpen` | Boolean indicating if upper bound is open on the interval of values defined by range ("<" instead of "<="). | No |
+
+### Example: equivalent to `WHERE 21 <= age <= 31`
+
+```json
+{
+ "type": "range",
+ "column": "age",
+ "matchValueType": "LONG",
+ "lower": 21,
+ "upper": 31
+}
+```
+
+### Example: equivalent to `WHERE 'foo' <= name <= 'hoo'`, using STRING comparison
+
+```json
+{
+ "type": "range",
+ "column": "name",
+ "matchValueType": "STRING",
+ "lower": "foo",
+ "upper": "hoo"
+}
+```
+
+### Example: equivalent to `WHERE 21 < age < 31`
+
+```json
+{
+ "type": "range",
+ "column": "age",
+ "matchValueType": "LONG",
+ "lower": "21",
+ "lowerOpen": true,
+ "upper": "31" ,
+ "upperOpen": true
+}
+```
+
+### Example: equivalent to `WHERE age < 31`
+
+```json
+{
+ "type": "range",
+ "column": "age",
+ "matchValueType": "LONG",
+ "upper": "31" ,
+ "upperOpen": true
+}
+```
+
+### Example: equivalent to `WHERE age >= 18`
+
+```json
+{
+ "type": "range",
+ "column": "age",
+ "matchValueType": "LONG",
+ "lower": 18
+}
+```
+
+### Example: equivalent to `WHERE ARRAY['a','b','c'] < arrayColumn < ARRAY['d','e','f']`, using ARRAY comparison
+
+```json
+{
+ "type": "range",
+ "column": "name",
+ "matchValueType": "ARRAY",
+ "lower": ["a","b","c"],
+ "lowerOpen": true,
+ "upper": ["d","e","f"],
+ "upperOpen": true
+}
+```
+
+
+## Like filter
+
+Like filters can be used for basic wildcard searches. They are equivalent to the SQL LIKE operator. Special characters
+supported are "%" (matches any number of characters) and "\_" (matches any one character).
+
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "like".| Yes |
+| `dimension` | Input column or virtual column name to filter. | Yes |
+| `pattern` | String LIKE pattern, such as "foo%" or "___bar".| Yes |
+| `escape`| A string escape character that can be used to escape special characters. | No |
+| `extractionFn` | [Extraction function](./dimensionspecs.md#extraction-functions) to apply to `dimension` prior to value matching. See [filtering with extraction functions](#filtering-with-extraction-functions) for details. | No |
+
+Like filters support the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
+
+### Example: equivalent of `WHERE last_name LIKE "D%"` (last_name starts with "D")
+
+```json
+{
+ "type": "like",
+ "dimension": "last_name",
+ "pattern": "D%"
+}
+```
+
+## Regular expression filter
+
+The regular expression filter is similar to the selector filter, but using regular expressions. It matches the specified dimension with the given pattern.
+
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "regex".| Yes |
+| `dimension` | Input column or virtual column name to filter. | Yes |
+| `pattern` | String pattern to match - any standard [Java regular expression](http://docs.oracle.com/javase/6/docs/api/java/util/regex/Pattern.html). | Yes |
+| `extractionFn` | [Extraction function](./dimensionspecs.md#extraction-functions) to apply to `dimension` prior to value matching. See [filtering with extraction functions](#filtering-with-extraction-functions) for details. | No |
+
+Note that it is often more optimal to use a like filter instead of a regex for simple matching of prefixes.
+
+### Example: matches values that start with "50."
+
+``` json
+{ "type": "regex", "dimension": "someColumn", "pattern": ^50.* }
+```
+
## Interval filter
The Interval filter enables range filtering on columns that contain long millisecond values, with the boundaries specified as ISO 8601 time intervals. It is suitable for the `__time` column, long metric columns, and dimensions with values that can be parsed as long milliseconds.
This filter converts the ISO 8601 intervals to long millisecond start/end ranges and translates to an OR of Bound filters on those millisecond ranges, with numeric comparison. The Bound filters will have left-closed and right-open matching (i.e., start <= time < end).
-|property|type|description|required?|
-|--------|-----------|---------|---------|
-|type|String|This should always be "interval".|yes|
-|dimension|String|The dimension to filter on|yes|
-|intervals|Array|A JSON array containing ISO-8601 interval strings. This defines the time ranges to filter on.|yes|
-|extractionFn|[Extraction function](#filtering-with-extraction-functions)| Extraction function to apply to the dimension|no|
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "interval". | Yes |
+| `dimension` | Input column or virtual column name to filter. | Yes |
+| `intervals` | A JSON array containing ISO-8601 interval strings that defines the time ranges to filter on. | Yes |
+| `extractionFn` | [Extraction function](./dimensionspecs.md#extraction-functions) to apply to `dimension` prior to value matching. See [filtering with extraction functions](#filtering-with-extraction-functions) for details. | No |
The interval filter supports the use of extraction functions, see [Filtering with Extraction Functions](#filtering-with-extraction-functions) for details.
@@ -410,6 +506,161 @@ The filter above is equivalent to the following OR of Bound filters:
}
```
+
+## True filter
+A filter which matches all values. You can use it to temporarily disable other filters without removing them.
+
+```json
+{ "type" : "true" }
+```
+
+## False filter
+A filter matches no values. You can use it to force a query to match no values.
+
+```json
+{"type": "false" }
+```
+
+
+## Search filter
+
+You can use search filters to filter on partial string matches.
+
+```json
+{
+ "filter": {
+ "type": "search",
+ "dimension": "product",
+ "query": {
+ "type": "insensitive_contains",
+ "value": "foo"
+ }
+ }
+}
+```
+
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "search". | Yes |
+| `dimension` | Input column or virtual column name to filter. | Yes |
+| `query`| A JSON object for the type of search. See [search query spec](#search-query-spec) for more information. | Yes |
+| `extractionFn` | [Extraction function](./dimensionspecs.md#extraction-functions) to apply to `dimension` prior to value matching. See [filtering with extraction functions](#filtering-with-extraction-functions) for details. | No |
+
+### Search query spec
+
+#### Contains
+
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "contains". | Yes |
+| `value` | A String value to search. | Yes |
+| `caseSensitive` | Whether the string comparison is case-sensitive or not. | No, default is false (insensitive) |
+
+#### Insensitive contains
+
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "insensitive_contains". | Yes |
+| `value` | A String value to search. | Yes |
+
+Note that an "insensitive_contains" search is equivalent to a "contains" search with "caseSensitive": false (or not
+provided).
+
+#### Fragment
+
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "fragment". | Yes |
+| `values` | A JSON array of string values to search. | Yes |
+| `caseSensitive` | Whether the string comparison is case-sensitive or not. | No, default is false (insensitive) |
+
+
+
+## Expression filter
+
+The expression filter allows for the implementation of arbitrary conditions, leveraging the Druid expression system. This filter allows for complete flexibility, but it might be less performant than a combination of the other filters on this page because it can't always use the same optimizations available to other filters.
+
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "expression" | Yes |
+| `expression` | Expression string to evaluate into true or false. See the [Druid expression system](math-expr.md) for more details. | Yes |
+
+### Example: expression based matching
+
+```json
+{
+ "type" : "expression" ,
+ "expression" : "((product_type == 42) && (!is_deleted))"
+}
+```
+
+
+## JavaScript filter
+
+The JavaScript filter matches a dimension against the specified JavaScript function predicate. The filter matches values for which the function returns true.
+
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "javascript" | Yes |
+| `dimension` | Input column or virtual column name to filter. | Yes |
+| `function` | JavaScript function which accepts the dimension value as a single argument, and returns either true or false. | Yes |
+| `extractionFn` | [Extraction function](./dimensionspecs.md#extraction-functions) to apply to `dimension` prior to value matching. See [filtering with extraction functions](#filtering-with-extraction-functions) for details. | No |
+
+### Example: matching any dimension values for the dimension `name` between `'bar'` and `'foo'`
+
+```json
+{
+ "type" : "javascript",
+ "dimension" : "name",
+ "function" : "function(x) { return(x >= 'bar' && x <= 'foo') }"
+}
+```
+
+:::info
+ JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::
+
+## Extraction filter
+
+:::info
+ The extraction filter is now deprecated. The selector filter with an extraction function specified
+ provides identical functionality and should be used instead.
+:::
+
+Extraction filter matches a dimension using a specific [extraction function](./dimensionspecs.md#extraction-functions).
+The following filter matches the values for which the extraction function has a transformation entry `input_key=output_value` where
+`output_value` is equal to the filter `value` and `input_key` is present as a dimension.
+
+| Property | Description | Required |
+| -------- | ----------- | -------- |
+| `type` | Must be "extraction" | Yes |
+| `dimension` | Input column or virtual column name to filter. | Yes |
+| `value` | String value to match. | No. If not specified the filter will match NULL values. |
+| `extractionFn` | [Extraction function](./dimensionspecs.md#extraction-functions) to apply to `dimension` prior to value matching. See [filtering with extraction functions](#filtering-with-extraction-functions) for details. | No |
+
+### Example: matching dimension values in `[product_1, product_3, product_5]` for the column `product`
+
+```json
+{
+ "filter": {
+ "type": "extraction",
+ "dimension": "product",
+ "value": "bar_1",
+ "extractionFn": {
+ "type": "lookup",
+ "lookup": {
+ "type": "map",
+ "map": {
+ "product_1": "bar_1",
+ "product_5": "bar_1",
+ "product_3": "bar_1"
+ }
+ }
+ }
+ }
+}
+```
+
## Filtering with extraction functions
All filters except the "spatial" filter support extraction functions.
@@ -420,9 +671,7 @@ If specified, the extraction function will be used to transform input values bef
The example below shows a selector filter combined with an extraction function. This filter will transform input values
according to the values defined in the lookup map; transformed values will then be matched with the string "bar_1".
-
-**Example**
-The following matches dimension values in `[product_1, product_3, product_5]` for the column `product`
+### Example: matches dimension values in `[product_1, product_3, product_5]` for the column `product`
```json
{
@@ -449,29 +698,97 @@ The following matches dimension values in `[product_1, product_3, product_5]` fo
Druid supports filtering on timestamp, string, long, and float columns.
-Note that only string columns have bitmap indexes. Therefore, queries that filter on other column types will need to
+Note that only string columns and columns produced with the ['auto' ingestion spec](../ingestion/ingestion-spec.md#dimension-objects) also used by [type aware schema discovery](../ingestion/schema-design.md#type-aware-schema-discovery) have bitmap indexes. Queries that filter on other column types must
scan those columns.
+### Filtering on multi-value string columns
+
+All filters return true if any one of the dimension values is satisfies the filter.
+
+#### Example: multi-value match behavior
+Given a multi-value STRING row with values `['a', 'b', 'c']`, a filter such as
+
+```json
+{ "type": "equals", "column": "someMultiValueColumn", "matchValueType": "STRING", "matchValue": "b" }
+```
+will successfully match the entire row. This can produce sometimes unintuitive behavior when coupled with the implicit UNNEST functionality of Druid [GroupBy](./groupbyquery.md) and [TopN](./topnquery.md) queries.
+
+Additionally, contradictory filters may be defined and perfectly legal in native queries which will not work in SQL.
+
+#### Example: SQL "contradiction"
+This query is impossible to express as is in SQL since it is a contradiction that the SQL planner will optimize to false and match nothing.
+
+Given a multi-value STRING row with values `['a', 'b', 'c']`, and filter such as
+```json
+{
+ "type": "and",
+ "fields": [
+ {
+ "type": "equals",
+ "column": "someMultiValueColumn",
+ "matchValueType": "STRING",
+ "matchValue": "a"
+ },
+ {
+ "type": "equals",
+ "column": "someMultiValueColumn",
+ "matchValueType": "STRING",
+ "matchValue": "b"
+ }
+ ]
+}
+```
+will successfully match the entire row, but not match a row with value `['a', 'c']`.
+
+To express this filter in SQL, use [SQL multi-value string functions](./sql-multivalue-string-functions.md) such as `MV_CONTAINS`, which can be optimized by the planner to the same native filters.
+
### Filtering on numeric columns
-When filtering on numeric columns, you can write filters as if they were strings. In most cases, your filter will be
+Some filters, such as equality and range filters allow accepting numeric match values directly since they include a secondary `matchValueType` parameter.
+
+When filtering on numeric columns using string based filters such as the selector, in, and bounds filters, you can write filter match values as if they were strings. In most cases, your filter will be
converted into a numeric predicate and will be applied to the numeric column values directly. In some cases (such as
the "regex" filter) the numeric column values will be converted to strings during the scan.
-For example, filtering on a specific value, `myFloatColumn = 10.1`:
+#### Example: filtering on a specific value, `myFloatColumn = 10.1`
```json
-"filter": {
+{
+ "type": "equals",
+ "dimension": "myFloatColumn",
+ "matchValueType": "FLOAT",
+ "value": 10.1
+}
+```
+
+or with a selector filter:
+
+```json
+{
"type": "selector",
"dimension": "myFloatColumn",
"value": "10.1"
}
```
-Filtering on a range of values, `10 <= myFloatColumn < 20`:
+#### Example: filtering on a range of values, `10 <= myFloatColumn < 20`
```json
-"filter": {
+{
+ "type": "range",
+ "column": "myFloatColumn",
+ "matchvalueType": "FLOAT",
+ "lower": 10.1,
+ "lowerOpen": false,
+ "upper": 20.9,
+ "upperOpen": true
+}
+```
+
+or with a bound filter:
+
+```json
+{
"type": "bound",
"dimension": "myFloatColumn",
"ordering": "numeric",
@@ -488,22 +805,33 @@ Query filters can also be applied to the timestamp column. The timestamp column
to the timestamp column, use the string `__time` as the dimension name. Like numeric dimensions, timestamp filters
should be specified as if the timestamp values were strings.
-If the user wishes to interpret the timestamp with a specific format, timezone, or locale, the [Time Format Extraction Function](./dimensionspecs.md#time-format-extraction-function) is useful.
+If you want to interpret the timestamp with a specific format, timezone, or locale, the [Time Format Extraction Function](./dimensionspecs.md#time-format-extraction-function) is useful.
+
+#### Example: filtering on a long timestamp value
+
+```json
+{
+ "type": "equals",
+ "dimension": "__time",
+ "matchValueType": "LONG",
+ "value": 124457387532
+}
+```
-For example, filtering on a long timestamp value:
+or with a selector filter:
```json
-"filter": {
+{
"type": "selector",
"dimension": "__time",
"value": "124457387532"
}
```
-Filtering on day of week:
+#### Example: filtering on day of week using an extraction function
```json
-"filter": {
+{
"type": "selector",
"dimension": "__time",
"value": "Friday",
@@ -516,7 +844,7 @@ Filtering on day of week:
}
```
-Filtering on a set of ISO 8601 intervals:
+#### Example: filtering on a set of ISO 8601 intervals
```json
{
@@ -529,25 +857,3 @@ Filtering on a set of ISO 8601 intervals:
}
```
-### True filter
-The true filter is a filter which matches all values. It can be used to temporarily disable other filters without removing the filter.
-
-```json
-
-{ "type" : "true" }
-```
-
-### Expression filter
-The expression filter allows for the implementation of arbitrary conditions, leveraging the Druid expression system.
-
-This filter allows for more flexibility, but it might be less performant than a combination of the other filters on this page due to the fact that not all filter optimizations are in place yet.
-
-```json
-
-{
- "type" : "expression" ,
- "expression" : "((product_type == 42) && (!is_deleted))"
-}
-```
-
-See the [Druid expression system](math-expr.md) for more details.
diff --git a/docs/querying/geo.md b/docs/querying/geo.md
index d8d6002520f0..065f13eb81ed 100644
--- a/docs/querying/geo.md
+++ b/docs/querying/geo.md
@@ -22,8 +22,10 @@ title: "Spatial filters"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](../querying/sql.md) and [native queries](../querying/querying.md).
-> This document describes a feature that is only available in the native language.
+:::info
+ Apache Druid supports two query languages: [Druid SQL](../querying/sql.md) and [native queries](../querying/querying.md).
+ This document describes a feature that is only available in the native language.
+:::
Apache Druid supports filtering spatially indexed columns based on an origin and a bound.
diff --git a/docs/querying/granularities.md b/docs/querying/granularities.md
index 327bb239f1d4..c9895fa1ee09 100644
--- a/docs/querying/granularities.md
+++ b/docs/querying/granularities.md
@@ -23,10 +23,12 @@ sidebar_label: "Granularities"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes the native
-> language. For information about time functions available in SQL, refer to the
-> [SQL documentation](sql-scalar.md#date-and-time-functions).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes the native
+ language. For information about time functions available in SQL, refer to the
+ [SQL documentation](sql-scalar.md#date-and-time-functions).
+:::
Granularity determines how to bucket data across the time dimension, or how to aggregate data by hour, day, minute, etc.
@@ -59,7 +61,9 @@ Druid supports the following granularity strings:
The minimum and maximum granularities are `none` and `all`, described as follows:
* `all` buckets everything into a single bucket.
* `none` does not mean zero bucketing. It buckets data to millisecond granularity—the granularity of the internal index. You can think of `none` as equivalent to `millisecond`.
- > Do not use `none` in a [timeseries query](../querying/timeseriesquery.md); Druid fills empty interior time buckets with zeroes, meaning the output will contain results for every single millisecond in the requested interval.
+:::info
+ Do not use `none` in a [timeseries query](../querying/timeseriesquery.md); Druid fills empty interior time buckets with zeroes, meaning the output will contain results for every single millisecond in the requested interval.
+:::
*Avoid using the `week` granularity for partitioning at ingestion time, because weeks don't align neatly with months and years, making it difficult to partition by coarser granularities later.
diff --git a/docs/querying/groupbyquery.md b/docs/querying/groupbyquery.md
index 9c6d6c0acff4..d6980476c0bd 100644
--- a/docs/querying/groupbyquery.md
+++ b/docs/querying/groupbyquery.md
@@ -23,17 +23,21 @@ sidebar_label: "GroupBy"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes a query
-> type in the native language. For information about when Druid SQL will use this query type, refer to the
-> [SQL documentation](sql-translation.md#query-types).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes a query
+ type in the native language. For information about when Druid SQL will use this query type, refer to the
+ [SQL documentation](sql-translation.md#query-types).
+:::
These types of Apache Druid queries take a groupBy query object and return an array of JSON objects where each object represents a
grouping asked for by the query.
-> Note: If you are doing aggregations with time as your only grouping, or an ordered groupBy over a single dimension,
-> consider [Timeseries](timeseriesquery.md) and [TopN](topnquery.md) queries as well as
-> groupBy. Their performance may be better in some cases. See [Alternatives](#alternatives) below for more details.
+:::info
+ Note: If you are doing aggregations with time as your only grouping, or an ordered groupBy over a single dimension,
+ consider [Timeseries](timeseriesquery.md) and [TopN](topnquery.md) queries as well as
+ groupBy. Their performance may be better in some cases. See [Alternatives](#alternatives) below for more details.
+:::
An example groupBy query object is shown below:
@@ -227,56 +231,18 @@ The response for the query above would look something like:
]
```
-> Notice that dimensions that are not included in an individual subtotalsSpec grouping are returned with a `null` value. This response format represents a behavior change as of Apache Druid 0.18.0.
-> In release 0.17.0 and earlier, such dimensions were entirely excluded from the result. If you were relying on this old behavior to determine whether a particular dimension was not part of
-> a subtotal grouping, you can now use [Grouping aggregator](aggregations.md#grouping-aggregator) instead.
+:::info
+ Notice that dimensions that are not included in an individual subtotalsSpec grouping are returned with a `null` value. This response format represents a behavior change as of Apache Druid 0.18.0.
+ In release 0.17.0 and earlier, such dimensions were entirely excluded from the result. If you were relying on this old behavior to determine whether a particular dimension was not part of
+ a subtotal grouping, you can now use [Grouping aggregator](aggregations.md#grouping-aggregator) instead.
+:::
## Implementation details
-### Strategies
-
-GroupBy queries can be executed using two different strategies. The default strategy for a cluster is determined by the
-"druid.query.groupBy.defaultStrategy" runtime property on the Broker. This can be overridden using "groupByStrategy" in
-the query context. If neither the context field nor the property is set, the "v2" strategy will be used.
-
-- "v2", the default, is designed to offer better performance and memory management. This strategy generates
-per-segment results using a fully off-heap map. Data processes merge the per-segment results using a fully off-heap
-concurrent facts map combined with an on-heap string dictionary. This may optionally involve spilling to disk. Data
-processes return sorted results to the Broker, which merges result streams using an N-way merge. The broker materializes
-the results if necessary (e.g. if the query sorts on columns other than its dimensions). Otherwise, it streams results
-back as they are merged.
-
-- "v1", a legacy engine, generates per-segment results on data processes (Historical, realtime, MiddleManager) using a map which
-is partially on-heap (dimension keys and the map itself) and partially off-heap (the aggregated values). Data processes then
-merge the per-segment results using Druid's indexing mechanism. This merging is multi-threaded by default, but can
-optionally be single-threaded. The Broker merges the final result set using Druid's indexing mechanism again. The broker
-merging is always single-threaded. Because the Broker merges results using the indexing mechanism, it must materialize
-the full result set before returning any results. On both the data processes and the Broker, the merging index is fully
-on-heap by default, but it can optionally store aggregated values off-heap.
-
-### Differences between v1 and v2
-
-Query API and results are compatible between the two engines; however, there are some differences from a cluster
-configuration perspective:
-
-- groupBy v1 controls resource usage using a row-based limit (maxResults) whereas groupBy v2 uses bytes-based limits.
-In addition, groupBy v1 merges results on-heap, whereas groupBy v2 merges results off-heap. These factors mean that
-memory tuning and resource limits behave differently between v1 and v2. In particular, due to this, some queries
-that can complete successfully in one engine may exceed resource limits and fail with the other engine. See the
-"Memory tuning and resource limits" section for more details.
-- groupBy v1 imposes no limit on the number of concurrently running queries, whereas groupBy v2 controls memory usage
-by using a finite-sized merge buffer pool. By default, the number of merge buffers is 1/4 the number of processing
-threads. You can adjust this as necessary to balance concurrency and memory usage.
-- groupBy v1 supports caching on either the Broker or Historical processes, whereas groupBy v2 only supports caching on
-Historical processes.
-- groupBy v2 supports both array-based aggregation and hash-based aggregation. The array-based aggregation is used only
-when the grouping key is a single indexed string column. In array-based aggregation, the dictionary-encoded value is used
-as the index, so the aggregated values in the array can be accessed directly without finding buckets based on hashing.
-
### Memory tuning and resource limits
-When using groupBy v2, four parameters control resource usage and limits:
+When using groupBy, four parameters control resource usage and limits:
- `druid.processing.buffer.sizeBytes`: size of the off-heap hash table used for aggregation, per query, in bytes. At
most `druid.processing.numMergeBuffers` of these will be created at once, which also serves as an upper limit on the
@@ -300,7 +266,7 @@ sorted and flushed to disk. Then, both in-memory structures will be cleared out
then go on to exceed `maxOnDiskStorage` will fail with a "Resource limit exceeded" error indicating that they ran out of
disk space.
-With groupBy v2, cluster operators should make sure that the off-heap hash tables and on-heap merging dictionaries
+With groupBy, cluster operators should make sure that the off-heap hash tables and on-heap merging dictionaries
will not exceed available memory for the maximum possible concurrent query load (given by
`druid.processing.numMergeBuffers`). See the [basic cluster tuning guide](../operations/basic-cluster-tuning.md)
for more details about direct memory usage, organized by Druid process type.
@@ -309,24 +275,18 @@ Brokers do not need merge buffers for basic groupBy queries. Queries with subque
Historicals and ingestion tasks need one merge buffer for each groupBy query, unless [parallel combination](groupbyquery.md#parallel-combine) is enabled, in which case they need two merge buffers per query.
-When using groupBy v1, all aggregation is done on-heap, and resource limits are done through the parameter
-`druid.query.groupBy.maxResults`. This is a cap on the maximum number of results in a result set. Queries that exceed
-this limit will fail with a "Resource limit exceeded" error indicating they exceeded their row limit. Cluster
-operators should make sure that the on-heap aggregations will not exceed available JVM heap space for the expected
-concurrent query load.
-
-### Performance tuning for groupBy v2
+### Performance tuning for groupBy
#### Limit pushdown optimization
-Druid pushes down the `limit` spec in groupBy queries to the segments on Historicals wherever possible to early prune unnecessary intermediate results and minimize the amount of data transferred to Brokers. By default, this technique is applied only when all fields in the `orderBy` spec is a subset of the grouping keys. This is because the `limitPushDown` doesn't guarantee the exact results if the `orderBy` spec includes any fields that are not in the grouping keys. However, you can enable this technique even in such cases if you can sacrifice some accuracy for fast query processing like in topN queries. See `forceLimitPushDown` in [advanced groupBy v2 configurations](#groupby-v2-configurations).
+Druid pushes down the `limit` spec in groupBy queries to the segments on Historicals wherever possible to early prune unnecessary intermediate results and minimize the amount of data transferred to Brokers. By default, this technique is applied only when all fields in the `orderBy` spec is a subset of the grouping keys. This is because the `limitPushDown` doesn't guarantee the exact results if the `orderBy` spec includes any fields that are not in the grouping keys. However, you can enable this technique even in such cases if you can sacrifice some accuracy for fast query processing like in topN queries. See `forceLimitPushDown` in [advanced configurations](#advanced-configurations).
#### Optimizing hash table
-The groupBy v2 engine uses an open addressing hash table for aggregation. The hash table is initialized with a given initial bucket number and gradually grows on buffer full. On hash collisions, the linear probing technique is used.
+The groupBy engine uses an open addressing hash table for aggregation. The hash table is initialized with a given initial bucket number and gradually grows on buffer full. On hash collisions, the linear probing technique is used.
-The default number of initial buckets is 1024 and the default max load factor of the hash table is 0.7. If you can see too many collisions in the hash table, you can adjust these numbers. See `bufferGrouperInitialBuckets` and `bufferGrouperMaxLoadFactor` in [Advanced groupBy v2 configurations](#groupby-v2-configurations).
+The default number of initial buckets is 1024 and the default max load factor of the hash table is 0.7. If you can see too many collisions in the hash table, you can adjust these numbers. See `bufferGrouperInitialBuckets` and `bufferGrouperMaxLoadFactor` in [advanced configurations](#advanced-configurations).
#### Parallel combine
@@ -344,16 +304,16 @@ longer time than timeseries or topN queries, they should release processing thre
However, you might care about the performance of some really heavy groupBy queries. Usually, the performance bottleneck
of heavy groupBy queries is merging sorted aggregates. In such cases, you can use processing threads for it as well.
This is called _parallel combine_. To enable parallel combine, see `numParallelCombineThreads` in
-[Advanced groupBy v2 configurations](#groupby-v2-configurations). Note that parallel combine can be enabled only when
+[advanced configurations](#advanced-configurations). Note that parallel combine can be enabled only when
data is actually spilled (see [Memory tuning and resource limits](#memory-tuning-and-resource-limits)).
-Once parallel combine is enabled, the groupBy v2 engine can create a combining tree for merging sorted aggregates. Each
+Once parallel combine is enabled, the groupBy engine can create a combining tree for merging sorted aggregates. Each
intermediate node of the tree is a thread merging aggregates from the child nodes. The leaf node threads read and merge
aggregates from hash tables including spilled ones. Usually, leaf processes are slower than intermediate nodes because they
need to read data from disk. As a result, less threads are used for intermediate nodes by default. You can change the
-degree of intermediate nodes. See `intermediateCombineDegree` in [Advanced groupBy v2 configurations](#groupby-v2-configurations).
+degree of intermediate nodes. See `intermediateCombineDegree` in [advanced configurations](#advanced-configurations).
-Please note that each Historical needs two merge buffers to process a groupBy v2 query with parallel combine: one for
+Please note that each Historical needs two merge buffers to process a groupBy query with parallel combine: one for
computing intermediate aggregates from each segment and another for combining intermediate aggregates in parallel.
@@ -371,18 +331,14 @@ results acceptable.
### Nested groupBys
-Nested groupBys (dataSource of type "query") are performed differently for "v1" and "v2". The Broker first runs the
-inner groupBy query in the usual way. "v1" strategy then materializes the inner query's results on-heap with Druid's
-indexing mechanism, and runs the outer query on these materialized results. "v2" strategy runs the outer query on the
-inner query's results stream with off-heap fact map and on-heap string dictionary that can spill to disk. Both
-strategy perform the outer query on the Broker in a single-threaded fashion.
+Nested groupBys (dataSource of type "query") are performed with the Broker first running the inner groupBy query in the
+usual way. Next, the outer query is run on the inner query's results stream with off-heap fact map and on-heap string
+dictionary that can spill to disk. The outer query is run on the Broker in a single-threaded fashion.
### Configurations
This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on Broker, Historical, and MiddleManager processes. You can set the query context parameters through the [query context](query-context.md).
-#### Configurations for groupBy v2
-
Supported runtime properties:
|Property|Description|Default|
@@ -399,30 +355,12 @@ Supported query contexts:
### Advanced configurations
-#### Common configurations for all groupBy strategies
-
Supported runtime properties:
|Property|Description|Default|
|--------|-----------|-------|
-|`druid.query.groupBy.defaultStrategy`|Default groupBy query strategy.|v2|
|`druid.query.groupBy.singleThreaded`|Merge results using a single thread.|false|
|`druid.query.groupBy.intermediateResultAsMapCompat`|Whether Brokers are able to understand map-based result rows. Setting this to `true` adds some overhead to all groupBy queries. It is required for compatibility with data servers running versions older than 0.16.0, which introduced [array-based result rows](#array-based-result-rows).|false|
-
-Supported query contexts:
-
-|Key|Description|
-|---|-----------|
-|`groupByStrategy`|Overrides the value of `druid.query.groupBy.defaultStrategy` for this query.|
-|`groupByIsSingleThreaded`|Overrides the value of `druid.query.groupBy.singleThreaded` for this query.|
-
-
-#### GroupBy v2 configurations
-
-Supported runtime properties:
-
-|Property|Description|Default|
-|--------|-----------|-------|
|`druid.query.groupBy.bufferGrouperInitialBuckets`|Initial number of buckets in the off-heap hash table used for grouping results. Set to 0 to use a reasonable default (1024).|0|
|`druid.query.groupBy.bufferGrouperMaxLoadFactor`|Maximum load factor of the off-heap hash table used for grouping results. When the load factor exceeds this size, the table will be grown or spilled to disk. Set to 0 to use a reasonable default (0.7).|0|
|`druid.query.groupBy.forceHashAggregation`|Force to use hash-based aggregation.|false|
@@ -434,6 +372,7 @@ Supported query contexts:
|Key|Description|Default|
|---|-----------|-------|
+|`groupByIsSingleThreaded`|Overrides the value of `druid.query.groupBy.singleThreaded` for this query.|None|
|`bufferGrouperInitialBuckets`|Overrides the value of `druid.query.groupBy.bufferGrouperInitialBuckets` for this query.|None|
|`bufferGrouperMaxLoadFactor`|Overrides the value of `druid.query.groupBy.bufferGrouperMaxLoadFactor` for this query.|None|
|`forceHashAggregation`|Overrides the value of `druid.query.groupBy.forceHashAggregation`|None|
@@ -446,23 +385,6 @@ Supported query contexts:
|`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|
-#### GroupBy v1 configurations
-
-Supported runtime properties:
-
-|Property|Description|Default|
-|--------|-----------|-------|
-|`druid.query.groupBy.maxIntermediateRows`|Maximum number of intermediate rows for the per-segment grouping engine. This is a tuning parameter that does not impose a hard limit; rather, it potentially shifts merging work from the per-segment engine to the overall merging index. Queries that exceed this limit will not fail.|50000|
-|`druid.query.groupBy.maxResults`|Maximum number of results. Queries that exceed this limit will fail.|500000|
-
-Supported query contexts:
-
-|Key|Description|Default|
-|---|-----------|-------|
-|`maxIntermediateRows`|Ignored by groupBy v2. Can be used to lower the value of `druid.query.groupBy.maxIntermediateRows` for a groupBy v1 query.|None|
-|`maxResults`|Ignored by groupBy v2. Can be used to lower the value of `druid.query.groupBy.maxResults` for a groupBy v1 query.|None|
-|`useOffheap`|Ignored by groupBy v2, and no longer supported for groupBy v1. Enabling this option with groupBy v1 will result in an error. For off-heap aggregation, switch to groupBy v2, which always operates off-heap.|false|
-
#### Array based result rows
Internally Druid always uses an array based representation of groupBy result rows, but by default this is translated
diff --git a/docs/querying/having.md b/docs/querying/having.md
index 8f83ce0a2442..f13020c4b377 100644
--- a/docs/querying/having.md
+++ b/docs/querying/having.md
@@ -22,10 +22,12 @@ title: "Having filters (groupBy)"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes the native
-> language. For information about functions available in SQL, refer to the
-> [SQL documentation](sql-scalar.md).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes the native
+ language. For information about functions available in SQL, refer to the
+ [SQL documentation](sql-scalar.md).
+:::
A having clause is a JSON object identifying which rows from a groupBy query should be returned, by specifying conditions on aggregated values.
diff --git a/docs/querying/limitspec.md b/docs/querying/limitspec.md
index 086ac3bda9f1..a734860e6320 100644
--- a/docs/querying/limitspec.md
+++ b/docs/querying/limitspec.md
@@ -22,9 +22,11 @@ title: "Sorting and limiting (groupBy)"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes the native
-> language. For information about sorting in SQL, refer to the [SQL documentation](sql.md#order-by).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes the native
+ language. For information about sorting in SQL, refer to the [SQL documentation](sql.md#order-by).
+:::
The limitSpec field provides the functionality to sort and limit the set of results from a groupBy query. If you group by a single dimension and are ordering by a single metric, we highly recommend using [TopN Queries](../querying/topnquery.md) instead. The performance will be substantially better. Available options are:
diff --git a/docs/querying/lookups.md b/docs/querying/lookups.md
index ec82dc93933f..4a592c158b5f 100644
--- a/docs/querying/lookups.md
+++ b/docs/querying/lookups.md
@@ -109,9 +109,11 @@ But this one is not, since both "2" and "3" map to the same value:
To tell Druid that your lookup is injective, you must specify `"injective" : true` in the lookup configuration. Druid
will not detect this automatically.
-> Currently, the injective lookup optimization is not triggered when lookups are inputs to a
-> [join datasource](datasource.md#join). It is only used when lookup functions are used directly, without the join
-> operator.
+:::info
+ Currently, the injective lookup optimization is not triggered when lookups are inputs to a
+ [join datasource](datasource.md#join). It is only used when lookup functions are used directly, without the join
+ operator.
+:::
Dynamic Configuration
---------------------
diff --git a/docs/querying/math-expr.md b/docs/querying/math-expr.md
index 8d558f4ceb9f..dcb4b7ce7787 100644
--- a/docs/querying/math-expr.md
+++ b/docs/querying/math-expr.md
@@ -22,9 +22,11 @@ title: "Expressions"
~ under the License.
-->
-> Apache Druid supports two query languages: [native queries](../querying/querying.md) and [Druid SQL](../querying/sql.md).
-> This document describes the native language. For information about functions available in SQL, refer to the
-> [SQL documentation](../querying/sql-scalar.md).
+:::info
+ Apache Druid supports two query languages: [native queries](../querying/querying.md) and [Druid SQL](../querying/sql.md).
+ This document describes the native language. For information about functions available in SQL, refer to the
+ [SQL documentation](../querying/sql-scalar.md).
+:::
Expressions are used in various places in the native query language, including
[virtual columns](../querying/virtual-columns.md) and [join conditions](../querying/datasource.md#join). They are
@@ -159,7 +161,7 @@ See javadoc of java.lang.Math for detailed explanation for each function.
|remainder|remainder(x, y) returns the remainder operation on two arguments as prescribed by the IEEE 754 standard|
|rint|rint(x) returns value that is closest in value to x and is equal to a mathematical integer|
|round|round(x, y) returns the value of the x rounded to the y decimal places. While x can be an integer or floating-point number, y must be an integer. The type of the return value is specified by that of x. y defaults to 0 if omitted. When y is negative, x is rounded on the left side of the y decimal points. If x is `NaN`, x returns 0. If x is infinity, x will be converted to the nearest finite double. |
-|safe_divide|safe_divide(x,y) returns the division of x by y if y is not equal to 0. In case y is 0 it returns 0 or `null` if `druid.generic.useDefaultValueForNull=false` |
+|safe_divide|safe_divide(x,y) returns the division of x by y if y is not equal to 0. In case y is 0 it returns `null` or 0 if `druid.generic.useDefaultValueForNull=true` (legacy mode) |
|scalb|scalb(d, sf) returns d * 2^sf rounded as if performed by a single correctly rounded floating-point multiply to a member of the double value set|
|signum|signum(x) returns the signum function of the argument x|
|sin|sin(x) returns the trigonometric sine of an angle x|
@@ -181,8 +183,8 @@ See javadoc of java.lang.Math for detailed explanation for each function.
| array_ordinal(arr,long) | returns the array element at the 1 based index supplied, or null for an out of range index |
| array_contains(arr,expr) | returns 1 if the array contains the element specified by expr, or contains all elements specified by expr if expr is an array, else 0 |
| array_overlap(arr1,arr2) | returns 1 if arr1 and arr2 have any elements in common, else 0 |
-| array_offset_of(arr,expr) | returns the 0 based index of the first occurrence of expr in the array, or `-1` or `null` if `druid.generic.useDefaultValueForNull=false`if no matching elements exist in the array. |
-| array_ordinal_of(arr,expr) | returns the 1 based index of the first occurrence of expr in the array, or `-1` or `null` if `druid.generic.useDefaultValueForNull=false` if no matching elements exist in the array. |
+| array_offset_of(arr,expr) | returns the 0 based index of the first occurrence of expr in the array, or `null` or `-1` if `druid.generic.useDefaultValueForNull=true` (legacy mode) if no matching elements exist in the array. |
+| array_ordinal_of(arr,expr) | returns the 1 based index of the first occurrence of expr in the array, or `null` or `-1` if `druid.generic.useDefaultValueForNull=true` (legacy mode) if no matching elements exist in the array. |
| array_prepend(expr,arr) | adds expr to arr at the beginning, the resulting array type determined by the type of the array |
| array_append(arr,expr) | appends expr to arr, the resulting array type determined by the type of the first array |
| array_concat(arr1,arr2) | concatenates 2 arrays, the resulting array type determined by the type of the first array |
@@ -297,7 +299,7 @@ Supported features:
* constants and identifiers are supported for any column type
* `cast` is supported for numeric and string types
* math operators: `+`,`-`,`*`,`/`,`%`,`^` are supported for numeric types
-* logical operators: `!`, `&&`, `||`, are supported for string and numeric types (if `druid.expressions.useStrictBooleans=true`)
+* logical operators: `!`, `&&`, `||`, are supported for string and numeric types
* comparison operators: `=`, `!=`, `>`, `>=`, `<`, `<=` are supported for string and numeric types
* math functions: `abs`, `acos`, `asin`, `atan`, `cbrt`, `ceil`, `cos`, `cosh`, `cot`, `exp`, `expm1`, `floor`, `getExponent`, `log`, `log10`, `log1p`, `nextUp`, `rint`, `signum`, `sin`, `sinh`, `sqrt`, `tan`, `tanh`, `toDegrees`, `toRadians`, `ulp`, `atan2`, `copySign`, `div`, `hypot`, `max`, `min`, `nextAfter`, `pow`, `remainder`, `scalb` are supported for numeric types
* time functions: `timestamp_floor` (with constant granularity argument) is supported for numeric types
@@ -307,9 +309,7 @@ Supported features:
* other: `parse_long` is supported for numeric and string types
## Logical operator modes
-Prior to the 0.23 release of Apache Druid, boolean function expressions have inconsistent handling of true and false values, and the logical 'and' and 'or' operators behave in a manner that is incompatible with SQL, even if SQL compatible null handling mode (`druid.generic.useDefaultValueForNull=false`) is enabled. Logical operators also pass through their input values similar to many scripting languages, and treat `null` as false, which can result in some rather strange behavior. Other boolean operations, such as comparisons and equality, retain their input types (e.g. `DOUBLE` comparison would produce `1.0` for true and `0.0` for false), while many other boolean functions strictly produce `LONG` typed values of `1` for true and `0` for false.
-
-After 0.23, while the inconsistent legacy behavior is still the default, it can be optionally be changed by setting `druid.expressions.useStrictBooleans=true`, so that these operations will allow correctly treating `null` values as "unknown" for SQL compatible behavior, and _all boolean output functions_ will output 'homogeneous' `LONG` typed boolean values of `1` for `true` and `0` for `false`. Additionally,
+In Druid 28.0 and later, `druid.expressions.useStrictBooleans=true` is set by default. Logical operations treat `null` values as "unknown" for SQL compatible behavior. _All boolean output functions_ will output 'homogeneous' `LONG` typed boolean values of `1` for `true` and `0` for `false`.
For the "or" operator:
* `true || null`, `null || true`, -> `1`
@@ -320,15 +320,8 @@ For the "and" operator:
* `false && null`, `null && false` -> `0`
Druid currently still retains implicit conversion of `LONG`, `DOUBLE`, and `STRING` types into boolean values in both modes:
-* `LONG` or `DOUBLE` - any value greater than 0 is considered `true`, else `false`
-* `STRING` - the value `'true'` (case insensitive) is considered `true`, everything else is `false`.
-
-Legacy behavior:
-* `100 && 11` -> `11`
-* `0.7 || 0.3` -> `0.3`
-* `100 && 0` -> `0`
-* `'troo' && 'true'` -> `'troo'`
-* `'troo' || 'true'` -> `'true'`
+* `LONG` or `DOUBLE`: any value greater than 0 is considered `true`, else `false`.
+* `STRING`: the value `'true'` (case insensitive) is considered `true`, everything else is `false`.
SQL compatible behavior:
* `100 && 11` -> `1`
@@ -337,4 +330,11 @@ SQL compatible behavior:
* `'troo' && 'true'` -> `0`
* `'troo' || 'true'` -> `1`
+Prior to Druid 28.0.0, `druid.expressions.useStrictBooleans=false` was the default. In this mode, boolean function expressions have inconsistent handling of true and false values. The logical 'and' and 'or' operators behave in a manner that is incompatible with SQL, even if SQL compatible null handling mode (`druid.generic.useDefaultValueForNull=false`) is enabled. Logical operators also pass through their input values, similar to many scripting languages, and treat `null` as false, which results in some rather strange behavior. Other boolean operations, such as comparisons and equality, retain their input types (e.g. `DOUBLE` comparison produces `1.0` for true and `0.0` for false), while many other boolean functions strictly produce `LONG` typed values of `1` for true and `0` for false. This legacy mode can still be enabled by setting `druid.expressions.useStrictBooleans=false`.
+Legacy behavior:
+* `100 && 11` -> `11`
+* `0.7 || 0.3` -> `0.3`
+* `100 && 0` -> `0`
+* `'troo' && 'true'` -> `'troo'`
+* `'troo' || 'true'` -> `'true'`
diff --git a/docs/querying/multi-value-dimensions.md b/docs/querying/multi-value-dimensions.md
index 215c7e422117..f1081d3f4323 100644
--- a/docs/querying/multi-value-dimensions.md
+++ b/docs/querying/multi-value-dimensions.md
@@ -72,7 +72,9 @@ The following sections describe filtering and grouping behavior based on the fol
{"timestamp": "2011-01-14T00:00:00.000Z", "tags": ["t5","t6","t7"]} #row3
{"timestamp": "2011-01-14T00:00:00.000Z", "tags": []} #row4
```
-> Be sure to remove the comments before trying out the sample data.
+:::info
+ Be sure to remove the comments before trying out the sample data.
+:::
### Filtering
diff --git a/docs/querying/nested-columns.md b/docs/querying/nested-columns.md
index 8f13372fdb43..01a86e49a78a 100644
--- a/docs/querying/nested-columns.md
+++ b/docs/querying/nested-columns.md
@@ -3,6 +3,10 @@ id: nested-columns
title: "Nested columns"
sidebar_label: Nested columns
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
+
+
-Apache Druid supports directly storing nested data structures in `COMPLEX` columns. `COMPLEX` columns store a copy of the structured data in JSON format and specialized internal columns and indexes for nested literal values—STRING, LONG, and DOUBLE types. An optimized [virtual column](./virtual-columns.md#nested-field-virtual-column) allows Druid to read and filter these values at speeds consistent with standard Druid LONG, DOUBLE, and STRING columns.
+Apache Druid supports directly storing nested data structures in `COMPLEX` columns. `COMPLEX` columns store a copy of the structured data in JSON format and specialized internal columns and indexes for nested literal values—STRING, LONG, and DOUBLE types, as well as ARRAY of STRING, LONG, and DOUBLE values. An optimized [virtual column](./virtual-columns.md#nested-field-virtual-column) allows Druid to read and filter these values at speeds consistent with standard Druid LONG, DOUBLE, and STRING columns.
Druid [SQL JSON functions](./sql-json-functions.md) allow you to extract, transform, and create `COMPLEX` values in SQL queries, using the specialized virtual columns where appropriate. You can use the [JSON nested columns functions](math-expr.md#json-functions) in [native queries](./querying.md) using [expression virtual columns](./virtual-columns.md#expression-virtual-column), and in native ingestion with a [`transformSpec`](../ingestion/ingestion-spec.md#transformspec).
You can use the JSON functions in INSERT and REPLACE statements in SQL-based ingestion, or in a `transformSpec` in native ingestion as an alternative to using a [`flattenSpec`](../ingestion/data-formats.md#flattenspec) object to "flatten" nested data for ingestion.
+Columns ingested as `COMPLEX` are automatically optimized to store the most appropriate physical column based on the data processed. For example, if only LONG values are processed, Druid stores a LONG column, ARRAY columns if the data consists of arrays, or `COMPLEX` in the general case if the data is actually nested. This is the same functionality that powers ['type aware' schema discovery](../ingestion/schema-design.md#type-aware-schema-discovery).
+
Druid supports directly ingesting nested data with the following formats: JSON, Parquet, Avro, ORC, Protobuf.
## Example nested data
-The examples in this topic use the JSON data in [`nested_example_data.json`](https://static.imply.io/data/nested_example_data.json). The file contains a simple facsimile of an order tracking and shipping table.
+The examples in this topic use the JSON data in [`nested_example_data.json`](https://static.imply.io/data/nested_example_data.json). The file contains a simple facsimile of an order tracking and shipping table.
When pretty-printed, a sample row in `nested_example_data` looks like this:
@@ -124,7 +130,7 @@ For example, the following ingestion spec instructs Druid to ingest `shipTo` and
### Transform data during batch ingestion
-You can use the [SQL JSON functions](./sql-json-functions.md) to transform nested data and reference the transformed data in your ingestion spec.
+You can use the [SQL JSON functions](./sql-json-functions.md) to transform nested data and reference the transformed data in your ingestion spec.
To do this, define the output name and expression in the `transforms` list in the `transformSpec` object of your ingestion spec.
@@ -341,8 +347,9 @@ For example, consider the following deserialized row of the sample data set:
The following examples demonstrate how to ingest the `shipTo` and `details` columns both as string type and as `COMPLEX` in the `shipTo_parsed` and `details_parsed` columns.
-
-
+
+
+
```
REPLACE INTO deserialized_example OVERWRITE ALL
WITH source AS (SELECT * FROM TABLE(
@@ -358,12 +365,14 @@ SELECT
"department",
"shipTo",
"details",
- PARSE_JSON("shipTo") as "shipTo_parsed",
+ PARSE_JSON("shipTo") as "shipTo_parsed",
PARSE_JSON("details") as "details_parsed"
FROM source
PARTITIONED BY DAY
```
-
+
+
+
```
{
"type": "index_parallel",
@@ -423,7 +432,8 @@ PARTITIONED BY DAY
}
}
```
-
+
+
## Querying nested columns
@@ -475,13 +485,15 @@ Example query results:
### Extracting nested data elements
-The `JSON_VALUE` function is specially optimized to provide native Druid level performance when processing nested literal values, as if they were flattened, traditional, Druid column types. It does this by reading from the specialized nested columns and indexes that are built and stored in JSON objects when Druid creates segments.
+The `JSON_VALUE` function is specially optimized to provide native Druid level performance when processing nested literal values, as if they were flattened, traditional, Druid column types. It does this by reading from the specialized nested columns and indexes that are built and stored in JSON objects when Druid creates segments.
Some operations using `JSON_VALUE` run faster than those using native Druid columns. For example, filtering numeric types uses the indexes built for nested numeric columns, which are not available for Druid DOUBLE, FLOAT, or LONG columns.
`JSON_VALUE` only returns literal types. Any paths that reference JSON objects or array types return null.
-> To achieve the best possible performance, use the `JSON_VALUE` function whenever you query JSON objects.
+:::info
+ To achieve the best possible performance, use the `JSON_VALUE` function whenever you query JSON objects.
+:::
#### Example query: Extract nested data elements
@@ -561,7 +573,7 @@ Example query results:
### Transforming JSON object data
-In addition to `JSON_VALUE`, Druid offers a number of operators that focus on transforming JSON object data:
+In addition to `JSON_VALUE`, Druid offers a number of operators that focus on transforming JSON object data:
- `JSON_QUERY`
- `JSON_OBJECT`
diff --git a/docs/querying/post-aggregations.md b/docs/querying/post-aggregations.md
index e42b1d333ff8..74c23065e748 100644
--- a/docs/querying/post-aggregations.md
+++ b/docs/querying/post-aggregations.md
@@ -22,10 +22,12 @@ title: "Post-aggregations"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes the native
-> language. For information about functions available in SQL, refer to the
-> [SQL documentation](sql-aggregations.md).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes the native
+ language. For information about functions available in SQL, refer to the
+ [SQL documentation](sql-aggregations.md).
+:::
Post-aggregations are specifications of processing that should happen on aggregated values as they come out of Apache Druid. If you include a post aggregation as part of a query, make sure to include all aggregators the post-aggregator requires.
@@ -147,7 +149,9 @@ Example JavaScript aggregator:
}
```
-> JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::info
+ JavaScript-based functionality is disabled by default. Please refer to the Druid [JavaScript programming guide](../development/javascript.md) for guidelines about using Druid's JavaScript functionality, including instructions on how to enable it.
+:::
### HyperUnique Cardinality post-aggregator
diff --git a/docs/querying/query-execution.md b/docs/querying/query-execution.md
index b51ee96bb0d4..cbfdbd731e39 100644
--- a/docs/querying/query-execution.md
+++ b/docs/querying/query-execution.md
@@ -22,10 +22,12 @@ title: "Query execution"
~ under the License.
-->
-> This document describes how Druid executes [native queries](querying.md), but since [Druid SQL](sql.md) queries
-> are translated to native queries, this document applies to the SQL runtime as well. Refer to the SQL
-> [Query translation](sql-translation.md) page for information about how SQL queries are translated to native
-> queries.
+:::info
+ This document describes how Druid executes [native queries](querying.md), but since [Druid SQL](sql.md) queries
+ are translated to native queries, this document applies to the SQL runtime as well. Refer to the SQL
+ [Query translation](sql-translation.md) page for information about how SQL queries are translated to native
+ queries.
+:::
Druid's approach to query execution varies depending on the kind of [datasource](datasource.md) you are querying.
diff --git a/docs/querying/query-from-deep-storage.md b/docs/querying/query-from-deep-storage.md
new file mode 100644
index 000000000000..5f076ca47cdf
--- /dev/null
+++ b/docs/querying/query-from-deep-storage.md
@@ -0,0 +1,195 @@
+---
+id: query-deep-storage
+title: "Query from deep storage"
+---
+
+
+
+> Query from deep storage is an [experimental feature](../development/experimental.md).
+
+Druid can query segments that are only stored in deep storage. Running a query from deep storage is slower than running queries from segments that are loaded on Historical processes, but it's a great tool for data that you either access infrequently or where the low latency results that typical Druid queries provide is not necessary. Queries from deep storage can increase the surface area of data available to query without requiring you to scale your Historical processes to accommodate more segments.
+
+## Keep segments in deep storage only
+
+Any data you ingest into Druid is already stored in deep storage, so you don't need to perform any additional configuration from that perspective. However, to take advantage of the cost savings that querying from deep storage provides, make sure not all your segments get loaded onto Historical processes.
+
+To do this, configure [load rules](../operations/rule-configuration.md#load-rules) to manage the which segments are only in deep storage and which get loaded onto Historical processes.
+
+The easiest way to do this is to explicitly configure the segments that don't get loaded onto Historical processes. Set `tieredReplicants` to an empty array and `useDefaultTierForNull` to `false`. For example, if you configure the following rule for a datasource:
+
+```json
+[
+ {
+ "interval": "2016-06-27T00:00:00.000Z/2016-06-27T02:59:00.000Z",
+ "tieredReplicants": {},
+ "useDefaultTierForNull": false,
+ "type": "loadByInterval"
+ }
+]
+```
+
+Any segment that falls within the specified interval exists only in deep storage. For segments that aren't in this interval, they'll use the default cluster load rules or any other load rules you configure.
+
+To configure the load rules through the Druid console, go to **Datasources > ... in the Actions column > Edit retention rules**. Then, paste the provided JSON into the JSON tab:
+
+![](../assets/tutorial-query-deepstorage-retention-rule.png)
+
+
+You can verify that a segment is not loaded on any Historical tiers by querying the Druid metadata table:
+
+```sql
+SELECT "segment_id", "replication_factor" FROM sys."segments" WHERE "replication_factor" = 0 AND "datasource" = YOUR_DATASOURCE
+```
+
+Segments with a `replication_factor` of `0` are not assigned to any Historical tiers. Queries against these segments are run directly against the segment in deep storage.
+
+You can also confirm this through the Druid console. On the **Segments** page, see the **Replication factor** column.
+
+Keep the following in mind when working with load rules to control what exists only in deep storage:
+
+- At least one of the segments in a datasource must be loaded onto a Historical process so that Druid can plan the query. The segment on the Historical process can be any segment from the datasource. It does not need to be a specific segment. One way to verify that a datasource has at least one segment on a Historical process is if it's visible in the Druid console.
+- The actual number of replicas may differ from the replication factor temporarily as Druid processes your load rules.
+
+## Run a query from deep storage
+
+### Submit a query
+
+You can query data from deep storage by submitting a query to the API using `POST /sql/statements` or the Druid console. Druid uses the multi-stage query (MSQ) task engine to perform the query.
+
+To run a query from deep storage, send your query to the Router using the POST method:
+
+```
+POST https://ROUTER:8888/druid/v2/sql/statements
+```
+
+Submitting a query from deep storage uses the same syntax as any other Druid SQL query where the query is contained in the "query" field in the JSON object within the request payload. For example:
+
+```json
+{"query" : "SELECT COUNT(*) FROM data_source WHERE foo = 'bar'"}
+```
+
+Generally, the request body fields are the same between the `sql` and `sql/statements` endpoints.
+
+There are additional context parameters for `sql/statements` specifically:
+
+ - `executionMode` (required) determines how query results are fetched. Set this to `ASYNC`.
+ - `selectDestination` (optional) set to `durableStorage` instructs Druid to write the results from SELECT queries to durable storage. Note that this requires you to have [durable storage for MSQ enabled](../operations/durable-storage.md).
+
+The following sample query includes the two additional context parameters that querying from deep storage supports:
+
+```
+curl --location 'http://localhost:8888/druid/v2/sql/statements' \
+--header 'Content-Type: application/json' \
+--data '{
+ "query":"SELECT * FROM \"YOUR_DATASOURCE\" where \"__time\" >TIMESTAMP'\''2017-09-01'\'' and \"__time\" <= TIMESTAMP'\''2017-09-02'\''",
+ "context":{
+ "executionMode":"ASYNC",
+ "selectDestination": "durableStorage"
+
+ }
+}'
+```
+
+The response for submitting a query includes the query ID along with basic information, such as when you submitted the query and the schema of the results:
+
+```json
+{
+ "queryId": "query-ALPHANUMBERIC-STRING",
+ "state": "ACCEPTED",
+ "createdAt": CREATION_TIMESTAMP,
+"schema": [
+ {
+ "name": COLUMN_NAME,
+ "type": COLUMN_TYPE,
+ "nativeType": COLUMN_TYPE
+ },
+ ...
+],
+"durationMs": DURATION_IN_MS,
+}
+```
+
+
+### Get query status
+
+You can check the status of a query with the following API call:
+
+```
+GET https://ROUTER:8888/druid/v2/sql/statements/QUERYID
+```
+
+The query returns the status of the query, such as `ACCEPTED` or `RUNNING`. Before you attempt to get results, make sure the state is `SUCCESS`.
+
+When you check the status on a successful query, it includes useful information about your query results including a sample record and information about how the results are organized by `pages`. The information for each page includes the following:
+
+- `numRows`: the number of rows in that page of results
+- `sizeInBytes`: the size of the page
+- `id`: the indexed page number that you can use to reference a specific page when you get query results
+
+You can use `page` as a parameter to refine the results you retrieve.
+
+The following snippet shows the structure of the `result` object:
+
+```json
+{
+ ...
+ "result": {
+ "numTotalRows": INTEGER,
+ "totalSizeInBytes": INTEGER,
+ "dataSource": "__query_select",
+ "sampleRecords": [
+ [
+ RECORD_1,
+ RECORD_2,
+ ...
+ ]
+ ],
+ "pages": [
+ {
+ "numRows": INTEGER,
+ "sizeInBytes": INTEGER,
+ "id": INTEGER_PAGE_NUMBER
+ }
+ ...
+ ]
+}
+}
+```
+
+### Get query results
+
+Only the user who submitted a query can retrieve the results for the query.
+
+Use the following endpoint to retrieve results:
+
+```
+GET https://ROUTER:8888/druid/v2/sql/statements/QUERYID/results?page=PAGENUMBER&size=RESULT_SIZE&timeout=TIMEOUT_MS
+```
+
+Results are returned in JSON format.
+
+You can use the optional `page`, `size`, and `timeout` parameters to refine your results. You can retrieve the `page` information for your results by fetching the status of the completed query.
+
+When you try to get results for a query from deep storage, you may receive an error that states the query is still running. Wait until the query completes before you try again.
+
+## Further reading
+
+* [Query from deep storage tutorial](../tutorials/tutorial-query-deep-storage.md)
+* [Query from deep storage API reference](../api-reference/sql-api.md#query-from-deep-storage)
diff --git a/docs/querying/querying.md b/docs/querying/querying.md
index fe15c9a2bb20..894920b72cbb 100644
--- a/docs/querying/querying.md
+++ b/docs/querying/querying.md
@@ -23,10 +23,12 @@ title: "Native queries"
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes the
-> native query language. For information about how Druid SQL chooses which native query types to use when
-> it runs a SQL query, refer to the [SQL documentation](sql-translation.md#query-types).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes the
+ native query language. For information about how Druid SQL chooses which native query types to use when
+ it runs a SQL query, refer to the [SQL documentation](sql-translation.md#query-types).
+:::
Native queries in Druid are JSON objects and are typically issued to the Broker or Router processes. Queries can be
posted like this:
@@ -35,7 +37,9 @@ posted like this:
curl -X POST ':/druid/v2/?pretty' -H 'Content-Type:application/json' -H 'Accept:application/json' -d @
```
-> Replace `:` with the appropriate address and port for your system. For example, if running the quickstart configuration, replace `:` with localhost:8888.
+:::info
+ Replace `:` with the appropriate address and port for your system. For example, if running the quickstart configuration, replace `:` with localhost:8888.
+:::
You can also enter them directly in the web console's Query view. Simply pasting a native query into the console switches the editor into JSON mode.
@@ -50,7 +54,9 @@ The Content-Type/Accept Headers can also take 'application/x-jackson-smile'.
curl -X POST ':/druid/v2/?pretty' -H 'Content-Type:application/json' -H 'Accept:application/x-jackson-smile' -d @
```
-> If the Accept header is not provided, it defaults to the value of 'Content-Type' header.
+:::info
+ If the Accept header is not provided, it defaults to the value of 'Content-Type' header.
+:::
Druid's native query is relatively low level, mapping closely to how computations are performed internally. Druid queries
are designed to be lightweight and complete very quickly. This means that for more complex analysis, or to build
diff --git a/docs/querying/scan-query.md b/docs/querying/scan-query.md
index f83c917ce760..d758450715ec 100644
--- a/docs/querying/scan-query.md
+++ b/docs/querying/scan-query.md
@@ -23,10 +23,12 @@ sidebar_label: "Scan"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes a query
-> type in the native language. For information about when Druid SQL will use this query type, refer to the
-> [SQL documentation](sql-translation.md#query-types).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes a query
+ type in the native language. For information about when Druid SQL will use this query type, refer to the
+ [SQL documentation](sql-translation.md#query-types).
+:::
The Scan query returns raw Apache Druid rows in streaming mode.
diff --git a/docs/querying/searchquery.md b/docs/querying/searchquery.md
index 113e1fba9458..27da025f1450 100644
--- a/docs/querying/searchquery.md
+++ b/docs/querying/searchquery.md
@@ -23,9 +23,11 @@ sidebar_label: "Search"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes a query
-> type that is only available in the native language.
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes a query
+ type that is only available in the native language.
+:::
A search query returns dimension values that match the search specification.
diff --git a/docs/querying/segmentmetadataquery.md b/docs/querying/segmentmetadataquery.md
index 3e1b4a5a245d..beae6ad8a4c7 100644
--- a/docs/querying/segmentmetadataquery.md
+++ b/docs/querying/segmentmetadataquery.md
@@ -23,10 +23,12 @@ sidebar_label: "SegmentMetadata"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes a query
-> type that is only available in the native language. However, Druid SQL contains similar functionality in
-> its [metadata tables](sql-metadata-tables.md).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes a query
+ type that is only available in the native language. However, Druid SQL contains similar functionality in
+ its [metadata tables](sql-metadata-tables.md).
+:::
Segment metadata queries return per-segment information about:
diff --git a/docs/querying/sorting-orders.md b/docs/querying/sorting-orders.md
index 2c420b173693..4860cdee52f1 100644
--- a/docs/querying/sorting-orders.md
+++ b/docs/querying/sorting-orders.md
@@ -22,10 +22,12 @@ title: "String comparators"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes the native
-> language. For information about functions available in SQL, refer to the
-> [SQL documentation](sql-scalar.md).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes the native
+ language. For information about functions available in SQL, refer to the
+ [SQL documentation](sql-scalar.md).
+:::
These sorting orders are used by the [TopNMetricSpec](./topnmetricspec.md), [SearchQuery](./searchquery.md), GroupByQuery's [LimitSpec](./limitspec.md), and [BoundFilter](./filters.md#bound-filter).
diff --git a/docs/querying/sql-aggregations.md b/docs/querying/sql-aggregations.md
index ebb8d78cb18c..b6a6748e624f 100644
--- a/docs/querying/sql-aggregations.md
+++ b/docs/querying/sql-aggregations.md
@@ -30,8 +30,10 @@ sidebar_label: "Aggregation functions"
patterns in this markdown file and parse it to TypeScript file for web console
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
+ This document describes the SQL language.
+:::
You can use aggregation functions in the SELECT clause of any [Druid SQL](./sql.md) query.
@@ -56,52 +58,49 @@ always return 0 as the initial value.
In the aggregation functions supported by Druid, only `COUNT`, `ARRAY_AGG`, and `STRING_AGG` accept the DISTINCT keyword.
-> The order of aggregation operations across segments is not deterministic. This means that non-commutative aggregation
-> functions can produce inconsistent results across the same query.
->
-> Functions that operate on an input type of "float" or "double" may also see these differences in aggregation
-> results across multiple query runs because of this. If precisely the same value is desired across multiple query runs,
-> consider using the `ROUND` function to smooth out the inconsistencies between queries.
+:::info
+ The order of aggregation operations across segments is not deterministic. This means that non-commutative aggregation
+ functions can produce inconsistent results across the same query.
+
+ Functions that operate on an input type of "float" or "double" may also see these differences in aggregation
+ results across multiple query runs because of this. If precisely the same value is desired across multiple query runs,
+ consider using the `ROUND` function to smooth out the inconsistencies between queries.
+:::
|Function|Notes|Default|
|--------|-----|-------|
|`COUNT(*)`|Counts the number of rows.|`0`|
|`COUNT(DISTINCT expr)`|Counts distinct values of `expr`.
When `useApproximateCountDistinct` is set to "true" (the default), this is an alias for `APPROX_COUNT_DISTINCT`. The specific algorithm depends on the value of [`druid.sql.approxCountDistinct.function`](../configuration/index.md#sql). In this mode, you can use strings, numbers, or prebuilt sketches. If counting prebuilt sketches, the prebuilt sketch type must match the selected algorithm.
When `useApproximateCountDistinct` is set to "false", the computation will be exact. In this case, `expr` must be string or numeric, since exact counts are not possible using prebuilt sketches. In exact mode, only one distinct count per query is permitted unless `useGroupingSetForExactDistinct` is enabled.
Counts each distinct value in a [`multi-value`](../querying/multi-value-dimensions.md)-row separately.|`0`|
-|`SUM(expr)`|Sums numbers.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`MIN(expr)`|Takes the minimum of numbers.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `9223372036854775807` (maximum LONG value)|
-|`MAX(expr)`|Takes the maximum of numbers.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `-9223372036854775808` (minimum LONG value)|
-|`AVG(expr)`|Averages numbers.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`SUM(expr)`|Sums numbers.|`null` or `0` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`MIN(expr)`|Takes the minimum of numbers.|`null` or `9223372036854775807` (maximum LONG value) if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`MAX(expr)`|Takes the maximum of numbers.|`null` or `-9223372036854775808` (minimum LONG value) if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`AVG(expr)`|Averages numbers.|`null` or `0` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
|`APPROX_COUNT_DISTINCT(expr)`|Counts distinct values of `expr` using an approximate algorithm. The `expr` can be a regular column or a prebuilt sketch column.
The specific algorithm depends on the value of [`druid.sql.approxCountDistinct.function`](../configuration/index.md#sql). By default, this is `APPROX_COUNT_DISTINCT_BUILTIN`. If the [DataSketches extension](../development/extensions-core/datasketches-extension.md) is loaded, you can set it to `APPROX_COUNT_DISTINCT_DS_HLL` or `APPROX_COUNT_DISTINCT_DS_THETA`.
When run on prebuilt sketch columns, the sketch column type must match the implementation of this function. For example: when `druid.sql.approxCountDistinct.function` is set to `APPROX_COUNT_DISTINCT_BUILTIN`, this function runs on prebuilt hyperUnique columns, but not on prebuilt HLLSketchBuild columns.|
|`APPROX_COUNT_DISTINCT_BUILTIN(expr)`|_Usage note:_ consider using `APPROX_COUNT_DISTINCT_DS_HLL` instead, which offers better accuracy in many cases.
Counts distinct values of `expr` using Druid's built-in "cardinality" or "hyperUnique" aggregators, which implement a variant of [HyperLogLog](http://algo.inria.fr/flajolet/Publications/FlFuGaMe07.pdf). The `expr` can be a string, a number, or a prebuilt hyperUnique column. Results are always approximate, regardless of the value of `useApproximateCountDistinct`.|
|`APPROX_QUANTILE(expr, probability, [resolution])`|_Deprecated._ Use `APPROX_QUANTILE_DS` instead, which provides a superior distribution-independent algorithm with formal error guarantees.
Computes approximate quantiles on numeric or [approxHistogram](../development/extensions-core/approximate-histograms.md#approximate-histogram-aggregator) expressions. `probability` should be between 0 and 1, exclusive. `resolution` is the number of centroids to use for the computation. Higher resolutions will give more precise results but also have higher overhead. If not provided, the default resolution is 50. Load the [approximate histogram extension](../development/extensions-core/approximate-histograms.md) to use this function.|`NaN`|
|`APPROX_QUANTILE_FIXED_BUCKETS(expr, probability, numBuckets, lowerLimit, upperLimit, [outlierHandlingMode])`|Computes approximate quantiles on numeric or [fixed buckets histogram](../development/extensions-core/approximate-histograms.md#fixed-buckets-histogram) expressions. `probability` should be between 0 and 1, exclusive. The `numBuckets`, `lowerLimit`, `upperLimit`, and `outlierHandlingMode` parameters are described in the fixed buckets histogram documentation. Load the [approximate histogram extension](../development/extensions-core/approximate-histograms.md) to use this function.|`0.0`|
|`BLOOM_FILTER(expr, numEntries)`|Computes a bloom filter from values produced by `expr`, with `numEntries` maximum number of distinct values before false positive rate increases. See [bloom filter extension](../development/extensions-core/bloom-filter.md) documentation for additional details.|Empty base64 encoded bloom filter STRING|
-|`VAR_POP(expr)`|Computes variance population of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`VAR_SAMP(expr)`|Computes variance sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`VARIANCE(expr)`|Computes variance sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`STDDEV_POP(expr)`|Computes standard deviation population of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`STDDEV_SAMP(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`STDDEV(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`EARLIEST(expr)`|Returns the earliest value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like `__time` in a Druid datasource), the "earliest" is taken from the row with the overall earliest non-null value of the timestamp column. If the earliest non-null value of the timestamp column appears in multiple rows, the `expr` may be taken from any of those rows. If `expr` does not come from a relation with a timestamp, then it is simply the first value encountered.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit are truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `''`|
-|`EARLIEST_BY(expr, timestampExpr)`|Returns the earliest value of `expr`, which must be numeric. The earliest value of `expr` is taken from the row with the overall earliest non-null value of `timestampExpr`. If the earliest non-null value of `timestampExpr` appears in multiple rows, the `expr` may be taken from any of those rows.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`EARLIEST_BY(expr, timestampExpr, maxBytesPerString)`| Like `EARLIEST_BY(expr, timestampExpr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit are truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `''`|
-|`LATEST(expr)`|Returns the latest value of `expr`, which must be numeric. The `expr` must come from a relation with a timestamp column (like `__time` in a Druid datasource) and the "latest" is taken from the row with the overall latest non-null value of the timestamp column. If the latest non-null value of the timestamp column appears in multiple rows, the `expr` may be taken from any of those rows. |`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit are truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `''`|
-|`LATEST_BY(expr, timestampExpr)`|Returns the latest value of `expr`, which must be numeric. The latest value of `expr` is taken from the row with the overall latest non-null value of `timestampExpr`. If the overall latest non-null value of `timestampExpr` appears in multiple rows, the `expr` may be taken from any of those rows.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`LATEST_BY(expr, timestampExpr, maxBytesPerString)`|Like `LATEST_BY(expr, timestampExpr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit are truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `''`|
-|`ANY_VALUE(expr)`|Returns any value of `expr` including null. `expr` must be numeric. This aggregator can simplify and optimize the performance by returning the first encountered value (including null)|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`ANY_VALUE(expr, maxBytesPerString)`|Like `ANY_VALUE(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit are truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `''`|
+|`VAR_POP(expr)`|Computes variance population of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` or `0` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`VAR_SAMP(expr)`|Computes variance sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` or `0` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`VARIANCE(expr)`|Computes variance sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` or `0` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`STDDEV_POP(expr)`|Computes standard deviation population of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` or `0` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`STDDEV_SAMP(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` or `0` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`STDDEV(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.md) documentation for additional details.|`null` or `0` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`EARLIEST(expr, [maxBytesPerValue])`|Returns the earliest value of `expr`. If `expr` comes from a relation with a timestamp column (like `__time` in a Druid datasource), the "earliest" is taken from the row with the overall earliest non-null value of the timestamp column. If the earliest non-null value of the timestamp column appears in multiple rows, the `expr` may be taken from any of those rows. If `expr` does not come from a relation with a timestamp, then it is simply the first value encountered.
If `expr` is a string or complex type `maxBytesPerValue` amount of space is allocated for the aggregation. Strings longer than this limit are truncated. The `maxBytesPerValue` parameter should be set as low as possible, since high values will lead to wasted memory. If `maxBytesPerValue`is omitted; it defaults to `1024`. |`null` or `0`/`''` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`EARLIEST_BY(expr, timestampExpr, [maxBytesPerValue])`|Returns the earliest value of `expr`. The earliest value of `expr` is taken from the row with the overall earliest non-null value of `timestampExpr`. If the earliest non-null value of `timestampExpr` appears in multiple rows, the `expr` may be taken from any of those rows.
If `expr` is a string or complex type `maxBytesPerValue` amount of space is allocated for the aggregation. Strings longer than this limit are truncated. The `maxBytesPerValue` parameter should be set as low as possible, since high values will lead to wasted memory. If `maxBytesPerValue`is omitted; it defaults to `1024`. |`null` or `0`/`''` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`LATEST(expr, [maxBytesPerValue])`|Returns the latest value of `expr` The `expr` must come from a relation with a timestamp column (like `__time` in a Druid datasource) and the "latest" is taken from the row with the overall latest non-null value of the timestamp column. If the latest non-null value of the timestamp column appears in multiple rows, the `expr` may be taken from any of those rows.
If `expr` is a string or complex type `maxBytesPerValue` amount of space is allocated for the aggregation. Strings longer than this limit are truncated. The `maxBytesPerValue` parameter should be set as low as possible, since high values will lead to wasted memory. If `maxBytesPerValue`is omitted; it defaults to `1024`. |`null` or `0`/`''` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`LATEST_BY(expr, timestampExpr, [maxBytesPerValue])`|Returns the latest value of `expr`. The latest value of `expr` is taken from the row with the overall latest non-null value of `timestampExpr`. If the overall latest non-null value of `timestampExpr` appears in multiple rows, the `expr` may be taken from any of those rows.
If `expr` is a string or complex type `maxBytesPerValue` amount of space is allocated for the aggregation. Strings longer than this limit are truncated. The `maxBytesPerValue` parameter should be set as low as possible, since high values will lead to wasted memory. If `maxBytesPerValue`is omitted; it defaults to `1024`. |`null` or `0`/`''` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`ANY_VALUE(expr, [maxBytesPerValue])`|Returns any value of `expr` including null. This aggregator can simplify and optimize the performance by returning the first encountered value (including `null`).
If `expr` is a string or complex type `maxBytesPerValue` amount of space is allocated for the aggregation. Strings longer than this limit are truncated. The `maxBytesPerValue` parameter should be set as low as possible, since high values will lead to wasted memory. If `maxBytesPerValue`is omitted; it defaults to `1024`. |`null` or `0`/`''` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
|`GROUPING(expr, expr...)`|Returns a number to indicate which groupBy dimension is included in a row, when using `GROUPING SETS`. Refer to [additional documentation](aggregations.md#grouping-aggregator) on how to infer this number.|N/A|
|`ARRAY_AGG(expr, [size])`|Collects all values of `expr` into an ARRAY, including null values, with `size` in bytes limit on aggregation size (default of 1024 bytes). If the aggregated array grows larger than the maximum size in bytes, the query will fail. Use of `ORDER BY` within the `ARRAY_AGG` expression is not currently supported, and the ordering of results within the output array may vary depending on processing order.|`null`|
|`ARRAY_AGG(DISTINCT expr, [size])`|Collects all distinct values of `expr` into an ARRAY, including null values, with `size` in bytes limit on aggregation size (default of 1024 bytes) per aggregate. If the aggregated array grows larger than the maximum size in bytes, the query will fail. Use of `ORDER BY` within the `ARRAY_AGG` expression is not currently supported, and the ordering of results will be based on the default for the element type.|`null`|
|`ARRAY_CONCAT_AGG(expr, [size])`|Concatenates all array `expr` into a single ARRAY, with `size` in bytes limit on aggregation size (default of 1024 bytes). Input `expr` _must_ be an array. Null `expr` will be ignored, but any null values within an `expr` _will_ be included in the resulting array. If the aggregated array grows larger than the maximum size in bytes, the query will fail. Use of `ORDER BY` within the `ARRAY_CONCAT_AGG` expression is not currently supported, and the ordering of results within the output array may vary depending on processing order.|`null`|
|`ARRAY_CONCAT_AGG(DISTINCT expr, [size])`|Concatenates all distinct values of all array `expr` into a single ARRAY, with `size` in bytes limit on aggregation size (default of 1024 bytes) per aggregate. Input `expr` _must_ be an array. Null `expr` will be ignored, but any null values within an `expr` _will_ be included in the resulting array. If the aggregated array grows larger than the maximum size in bytes, the query will fail. Use of `ORDER BY` within the `ARRAY_CONCAT_AGG` expression is not currently supported, and the ordering of results will be based on the default for the element type.|`null`|
-|`STRING_AGG(expr, separator, [size])`|Collects all values of `expr` into a single STRING, ignoring null values. Each value is joined by the `separator` which must be a literal STRING. An optional `size` in bytes can be supplied to limit aggregation size (default of 1024 bytes). If the aggregated string grows larger than the maximum size in bytes, the query will fail. Use of `ORDER BY` within the `STRING_AGG` expression is not currently supported, and the ordering of results within the output string may vary depending on processing order.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `''`|
-|`STRING_AGG(DISTINCT expr, separator, [size])`|Collects all distinct values of `expr` into a single STRING, ignoring null values. Each value is joined by the `separator` which must be a literal STRING. An optional `size` in bytes can be supplied to limit aggregation size (default of 1024 bytes). If the aggregated string grows larger than the maximum size in bytes, the query will fail. Use of `ORDER BY` within the `STRING_AGG` expression is not currently supported, and the ordering of results will be based on the default `STRING` ordering.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `''`|
-|`BIT_AND(expr)`|Performs a bitwise AND operation on all input values.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`BIT_OR(expr)`|Performs a bitwise OR operation on all input values.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
-|`BIT_XOR(expr)`|Performs a bitwise XOR operation on all input values.|`null` if `druid.generic.useDefaultValueForNull=false`, otherwise `0`|
+|`STRING_AGG([DISTINCT] expr, [separator, [size]])`|Collects all values (or all distinct values) of `expr` into a single STRING, ignoring null values. Each value is joined by an optional `separator`, which must be a literal STRING. If the `separator` is not provided, strings are concatenated without a separator.
An optional `size` in bytes can be supplied to limit aggregation size (default of 1024 bytes). If the aggregated string grows larger than the maximum size in bytes, the query will fail. Use of `ORDER BY` within the `STRING_AGG` expression is not currently supported, and the ordering of results within the output string may vary depending on processing order.|`null` or `''` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`LISTAGG([DISTINCT] expr, [separator, [size]])`|Synonym for `STRING_AGG`.|`null` or `''` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`BIT_AND(expr)`|Performs a bitwise AND operation on all input values.|`null` or `0` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`BIT_OR(expr)`|Performs a bitwise OR operation on all input values.|`null` or `0` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
+|`BIT_XOR(expr)`|Performs a bitwise XOR operation on all input values.|`null` or `0` if `druid.generic.useDefaultValueForNull=true` (legacy mode)|
## Sketch functions
diff --git a/docs/querying/sql-array-functions.md b/docs/querying/sql-array-functions.md
index 46d3acc50630..89607e9f0852 100644
--- a/docs/querying/sql-array-functions.md
+++ b/docs/querying/sql-array-functions.md
@@ -31,8 +31,10 @@ sidebar_label: "Array functions"
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
+ This document describes the SQL language.
+:::
This page describes the operations you can perform on arrays using [Druid SQL](./sql.md). See [`ARRAY` data type documentation](./sql-data-types.md#arrays) for additional details.
@@ -52,10 +54,10 @@ The following table describes array functions. To learn more about array aggrega
|`ARRAY_ORDINAL(arr, long)`|Returns the array element at the 1-based index supplied, or null for an out of range index.|
|`ARRAY_CONTAINS(arr, expr)`|If `expr` is a scalar type, returns 1 if `arr` contains `expr`. If `expr` is an array, returns 1 if `arr` contains all elements of `expr`. Otherwise returns 0.|
|`ARRAY_OVERLAP(arr1, arr2)`|Returns 1 if `arr1` and `arr2` have any elements in common, else 0.|
-|`ARRAY_OFFSET_OF(arr, expr)`|Returns the 0-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `-1` or `null` if `druid.generic.useDefaultValueForNull=false`.|
-|`ARRAY_ORDINAL_OF(arr, expr)`|Returns the 1-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `-1` or `null` if `druid.generic.useDefaultValueForNull=false`.|
-|`ARRAY_PREPEND(expr, arr)`|Prepends `expr` to `arr` at the beginning, the resulting array type determined by the type of `arr`.|
-|`ARRAY_APPEND(arr1, expr)`|Appends `expr` to `arr`, the resulting array type determined by the type of `arr1`.|
+|`ARRAY_OFFSET_OF(arr, expr)`|Returns the 0-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `null` or `-1` if `druid.generic.useDefaultValueForNull=true` (legacy mode).|
+|`ARRAY_ORDINAL_OF(arr, expr)`|Returns the 1-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `null` or `-1` if `druid.generic.useDefaultValueForNull=true` (legacy mode).|
+|`ARRAY_PREPEND(expr, arr)`|Adds `expr` to the beginning of `arr`, the resulting array type determined by the type of `arr`.|
+|`ARRAY_APPEND(arr, expr)`|Appends `expr` to `arr`, the resulting array type determined by the type of `arr`.|
|`ARRAY_CONCAT(arr1, arr2)`|Concatenates `arr2` to `arr1`. The resulting array type is determined by the type of `arr1`.|
|`ARRAY_SLICE(arr, start, end)`|Returns the subarray of `arr` from the 0-based index `start` (inclusive) to `end` (exclusive). Returns `null`, if `start` is less than 0, greater than length of `arr`, or greater than `end`.|
|`ARRAY_TO_STRING(arr, str)`|Joins all elements of `arr` by the delimiter specified by `str`.|
diff --git a/docs/querying/sql-data-types.md b/docs/querying/sql-data-types.md
index 8427a8dd7372..6f01369e8449 100644
--- a/docs/querying/sql-data-types.md
+++ b/docs/querying/sql-data-types.md
@@ -23,8 +23,10 @@ sidebar_label: "SQL data types"
~ under the License.
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
+ This document describes the SQL language.
+:::
Druid associates each column with a specific data type. This topic describes supported data types in [Druid SQL](./sql.md).
@@ -64,15 +66,14 @@ The following table describes how Druid maps SQL types onto native types when ru
|ARRAY|ARRAY|`NULL`|Druid native array types work as SQL arrays, and multi-value strings can be converted to arrays. See [Arrays](#arrays) for more information.|
|OTHER|COMPLEX|none|May represent various Druid column types such as hyperUnique, approxHistogram, etc.|
-* Default value applies if `druid.generic.useDefaultValueForNull = true` (the default mode). Otherwise, the default value is `NULL` for all types.
+* The default value is `NULL` for all types, except in legacy mode (`druid.generic.useDefaultValueForNull = true`) which initialize a default value.
Casts between two SQL types with the same Druid runtime type have no effect other than the exceptions noted in the table.
Casts between two SQL types that have different Druid runtime types generate a runtime cast in Druid.
-If a value cannot be cast to the target type, as in `CAST('foo' AS BIGINT)`, Druid either substitutes a default
-value (when `druid.generic.useDefaultValueForNull = true`, the default mode), or substitutes [NULL](#null-values) (when
-`druid.generic.useDefaultValueForNull = false`). NULL values cast to non-nullable types are also substituted with a default value. For example, if `druid.generic.useDefaultValueForNull = true`, a null VARCHAR cast to BIGINT is converted to a zero.
+If a value cannot be cast to the target type, as in `CAST('foo' AS BIGINT)`, Druid a substitutes [NULL](#null-values).
+When `druid.generic.useDefaultValueForNull = true` (legacy mode), Druid instead substitutes a default value, including when NULL values cast to non-nullable types. For example, if `druid.generic.useDefaultValueForNull = true`, a null VARCHAR cast to BIGINT is converted to a zero.
## Multi-value strings
@@ -84,10 +85,12 @@ You can treat multi-value string dimensions as arrays using special
Grouping by multi-value dimensions observes the native Druid multi-value aggregation behavior, which is similar to an implicit SQL UNNEST. See [Grouping](multi-value-dimensions.md#grouping) for more information.
-> Because the SQL planner treats multi-value dimensions as VARCHAR, there are some inconsistencies between how they are handled in Druid SQL and in native queries. For instance, expressions involving multi-value dimensions may be incorrectly optimized by the Druid SQL planner. For example, `multi_val_dim = 'a' AND multi_val_dim = 'b'` is optimized to
+:::info
+ Because the SQL planner treats multi-value dimensions as VARCHAR, there are some inconsistencies between how they are handled in Druid SQL and in native queries. For instance, expressions involving multi-value dimensions may be incorrectly optimized by the Druid SQL planner. For example, `multi_val_dim = 'a' AND multi_val_dim = 'b'` is optimized to
`false`, even though it is possible for a single row to have both `'a'` and `'b'` as values for `multi_val_dim`.
->
-> The SQL behavior of multi-value dimensions may change in a future release to more closely align with their behavior in native queries, but the [multi-value string functions](./sql-multivalue-string-functions.md) should be able to provide nearly all possible native functionality.
+
+ The SQL behavior of multi-value dimensions may change in a future release to more closely align with their behavior in native queries, but the [multi-value string functions](./sql-multivalue-string-functions.md) should be able to provide nearly all possible native functionality.
+:::
## Arrays
@@ -113,9 +116,11 @@ distinguish between empty and null rows. An empty row will never appear natively
but any multi-value function which manipulates the array form of the value may produce an empty array, which is handled
separately while processing.
-> Do not mix the usage of multi-value functions and normal scalar functions within the same expression, as the planner will be unable
-> to determine how to properly process the value given its ambiguous usage. A multi-value string must be treated consistently within
-> an expression.
+:::info
+ Do not mix the usage of multi-value functions and normal scalar functions within the same expression, as the planner will be unable
+ to determine how to properly process the value given its ambiguous usage. A multi-value string must be treated consistently within
+ an expression.
+:::
When converted to ARRAY or used with [array functions](./sql-array-functions.md), multi-value strings behave as standard SQL arrays and can no longer
be manipulated with non-array functions.
@@ -129,33 +134,33 @@ VARCHAR. ARRAY typed results will be serialized into stringified JSON arrays if
## NULL values
The [`druid.generic.useDefaultValueForNull`](../configuration/index.md#sql-compatible-null-handling)
-runtime property controls Druid's NULL handling mode. For the most SQL compliant behavior, set this to `false`.
+runtime property controls Druid's NULL handling mode. For the most SQL compliant behavior, set this to `false` (the default).
-When `druid.generic.useDefaultValueForNull = true` (the default mode), Druid treats NULLs and empty strings
+When `druid.generic.useDefaultValueForNull = false` (the default), NULLs are treated more closely to the SQL standard. In this mode,
+numeric NULL is permitted, and NULLs and empty strings are no longer treated as interchangeable. This property
+affects both storage and querying, and must be set on all Druid service types to be available at both ingestion time
+and query time. There is some overhead associated with the ability to handle NULLs; see
+the [segment internals](../design/segments.md#handling-null-values) documentation for more details.
+
+When `druid.generic.useDefaultValueForNull = true` (legacy mode), Druid treats NULLs and empty strings
interchangeably, rather than according to the SQL standard. In this mode Druid SQL only has partial support for NULLs.
For example, the expressions `col IS NULL` and `col = ''` are equivalent, and both evaluate to true if `col`
contains an empty string. Similarly, the expression `COALESCE(col1, col2)` returns `col2` if `col1` is an empty
string. While the `COUNT(*)` aggregator counts all rows, the `COUNT(expr)` aggregator counts the number of rows
where `expr` is neither null nor the empty string. Numeric columns in this mode are not nullable; any null or missing
-values are treated as zeroes.
-
-When `druid.generic.useDefaultValueForNull = false`, NULLs are treated more closely to the SQL standard. In this mode,
-numeric NULL is permitted, and NULLs and empty strings are no longer treated as interchangeable. This property
-affects both storage and querying, and must be set on all Druid service types to be available at both ingestion time
-and query time. There is some overhead associated with the ability to handle NULLs; see
-the [segment internals](../design/segments.md#handling-null-values) documentation for more details.
+values are treated as zeroes. This was the default prior to Druid 28.0.0.
## Boolean logic
The [`druid.expressions.useStrictBooleans`](../configuration/index.md#expression-processing-configurations)
-runtime property controls Druid's boolean logic mode. For the most SQL compliant behavior, set this to `true`.
-
-When `druid.expressions.useStrictBooleans = false` (the default mode), Druid uses two-valued logic.
+runtime property controls Druid's boolean logic mode. For the most SQL compliant behavior, set this to `true` (the default).
When `druid.expressions.useStrictBooleans = true`, Druid uses three-valued logic for
[expressions](math-expr.md) evaluation, such as `expression` virtual columns or `expression` filters.
However, even in this mode, Druid uses two-valued logic for filter types other than `expression`.
+When `druid.expressions.useStrictBooleans = false` (legacy mode), Druid uses two-valued logic.
+
## Nested columns
Druid supports storing nested data structures in segments using the native `COMPLEX` type. See [Nested columns](./nested-columns.md) for more information.
diff --git a/docs/querying/sql-functions.md b/docs/querying/sql-functions.md
index 3e4cf711dcdc..3df24ea607f6 100644
--- a/docs/querying/sql-functions.md
+++ b/docs/querying/sql-functions.md
@@ -23,8 +23,10 @@ sidebar_label: "All functions"
~ under the License.
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
+ This document describes the SQL language.
+:::
This page provides a reference of all Druid SQL functions in alphabetical order.
@@ -48,11 +50,7 @@ Calculates the arc cosine of a numeric expression.
## ANY_VALUE
-`ANY_VALUE()`
-
-`ANY_VALUE()`
-
-`ANY_VALUE(, )`
+`ANY_VALUE(expr, [maxBytesPerValue])`
**Function type:** [Aggregation](sql-aggregations.md)
@@ -183,7 +181,7 @@ Returns the array element at the 0-based index supplied, or null for an out of r
**Function type:** [Array](./sql-array-functions.md)
-Returns the 0-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `-1` or `null` if `druid.generic.useDefaultValueForNull=false`.
+Returns the 0-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `null` or `-1` if `druid.generic.useDefaultValueForNull=true` (legacy mode)..
## ARRAY_ORDINAL
@@ -198,7 +196,7 @@ Returns the array element at the 1-based index supplied, or null for an out of r
**Function type:** [Array](./sql-array-functions.md)
-Returns the 1-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `-1` or `null` if `druid.generic.useDefaultValueForNull=false`.|
+Returns the 1-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `null` or `-1` if `druid.generic.useDefaultValueForNull=true` (legacy mode)..|
## ARRAY_OVERLAP
@@ -639,9 +637,7 @@ Returns a union of Tuple sketches which each contain an array of double values a
## EARLIEST
-`EARLIEST(expr)`
-
-`EARLIEST(expr, maxBytesPerString)`
+`EARLIEST(expr, [maxBytesPerValue])`
**Function type:** [Aggregation](sql-aggregations.md)
@@ -649,9 +645,7 @@ Returns the value of a numeric or string expression corresponding to the earlies
## EARLIEST_BY
-`EARLIEST_BY(expr, timestampExpr)`
-
-`EARLIEST_BY(expr, timestampExpr, maxBytesPerString)`
+`EARLIEST_BY(expr, timestampExpr, [maxBytesPerValue])`
**Function type:** [Aggregation](sql-aggregations.md)
@@ -835,9 +829,7 @@ Extracts a literal value from `expr` at the specified `path`. If you specify `RE
## LATEST
-`LATEST(expr)`
-
-`LATEST(expr, maxBytesPerString)`
+`LATEST(expr, [maxBytesPerValue])`
**Function type:** [Aggregation](sql-aggregations.md)
@@ -845,9 +837,7 @@ Returns the value of a numeric or string expression corresponding to the latest
## LATEST_BY
-`LATEST_BY(expr, timestampExpr)`
-
-`LATEST_BY(expr, timestampExpr, maxBytesPerString)`
+`LATEST_BY(expr, timestampExpr, [maxBytesPerValue])`
**Function type:** [Aggregation](sql-aggregations.md)
diff --git a/docs/querying/sql-metadata-tables.md b/docs/querying/sql-metadata-tables.md
index 9b0a42b6c1df..8e9bce9fad95 100644
--- a/docs/querying/sql-metadata-tables.md
+++ b/docs/querying/sql-metadata-tables.md
@@ -23,8 +23,10 @@ sidebar_label: "SQL metadata tables"
~ under the License.
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
+ This document describes the SQL language.
+:::
Druid Brokers infer table and column metadata for each datasource from segments loaded in the cluster, and use this to
@@ -47,8 +49,10 @@ FROM INFORMATION_SCHEMA.COLUMNS
WHERE "TABLE_SCHEMA" = 'druid' AND "TABLE_NAME" = 'foo'
```
-> Note: INFORMATION_SCHEMA tables do not currently support Druid-specific functions like `TIME_PARSE` and
-> `APPROX_QUANTILE_DS`. Only standard SQL functions can be used.
+:::info
+ Note: INFORMATION_SCHEMA tables do not currently support Druid-specific functions like `TIME_PARSE` and
+ `APPROX_QUANTILE_DS`. Only standard SQL functions can be used.
+:::
### SCHEMATA table
`INFORMATION_SCHEMA.SCHEMATA` provides a list of all known schemas, which include `druid` for standard [Druid Table datasources](datasource.md#table), `lookup` for [Lookups](datasource.md#lookup), `sys` for the virtual [System metadata tables](#system-schema), and `INFORMATION_SCHEMA` for these virtual tables. Tables are allowed to have the same name across different schemas, so the schema may be included in an SQL statement to distinguish them, e.g. `lookup.table` vs `druid.table`.
@@ -130,8 +134,10 @@ WHERE "IS_AGGREGATOR" = 'YES'
The "sys" schema provides visibility into Druid segments, servers and tasks.
-> Note: "sys" tables do not currently support Druid-specific functions like `TIME_PARSE` and
-> `APPROX_QUANTILE_DS`. Only standard SQL functions can be used.
+:::info
+ Note: "sys" tables do not currently support Druid-specific functions like `TIME_PARSE` and
+ `APPROX_QUANTILE_DS`. Only standard SQL functions can be used.
+:::
### SEGMENTS table
@@ -228,7 +234,7 @@ Servers table lists all discovered servers in the cluster.
|tier|VARCHAR|Distribution tier see [druid.server.tier](../configuration/index.md#historical-general-configuration). Only valid for HISTORICAL type, for other types it's null|
|current_size|BIGINT|Current size of segments in bytes on this server. Only valid for HISTORICAL type, for other types it's 0|
|max_size|BIGINT|Max size in bytes this server recommends to assign to segments see [druid.server.maxSize](../configuration/index.md#historical-general-configuration). Only valid for HISTORICAL type, for other types it's 0|
-|is_leader|BIGINT|1 if the server is currently the 'leader' (for services which have the concept of leadership), otherwise 0 if the server is not the leader, or the default long value (0 or null depending on `druid.generic.useDefaultValueForNull`) if the server type does not have the concept of leadership|
+|is_leader|BIGINT|1 if the server is currently the 'leader' (for services which have the concept of leadership), otherwise 0 if the server is not the leader, or the default long value (null or zero depending on `druid.generic.useDefaultValueForNull`) if the server type does not have the concept of leadership|
|start_time|STRING|Timestamp in ISO8601 format when the server was announced in the cluster|
To retrieve information about all servers, use the query:
diff --git a/docs/querying/sql-multivalue-string-functions.md b/docs/querying/sql-multivalue-string-functions.md
index e2b2a6a5c24d..8b4a17c7b5ec 100644
--- a/docs/querying/sql-multivalue-string-functions.md
+++ b/docs/querying/sql-multivalue-string-functions.md
@@ -31,8 +31,10 @@ sidebar_label: "Multi-value string functions"
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
+ This document describes the SQL language.
+:::
Druid supports string dimensions containing multiple values.
This page describes the operations you can perform on multi-value string dimensions using [Druid SQL](./sql.md).
@@ -53,10 +55,10 @@ All array references in the multi-value string function documentation can refer
|`MV_ORDINAL(arr, long)`|Returns the array element at the 1-based index supplied, or null for an out of range index.|
|`MV_CONTAINS(arr, expr)`|If `expr` is a scalar type, returns 1 if `arr` contains `expr`. If `expr` is an array, returns 1 if `arr` contains all elements of `expr`. Otherwise returns 0.|
|`MV_OVERLAP(arr1, arr2)`|Returns 1 if `arr1` and `arr2` have any elements in common, else 0.|
-|`MV_OFFSET_OF(arr, expr)`|Returns the 0-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `-1` or `null` if `druid.generic.useDefaultValueForNull=false`.|
-|`MV_ORDINAL_OF(arr, expr)`|Returns the 1-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `-1` or `null` if `druid.generic.useDefaultValueForNull=false`.|
-|`MV_PREPEND(expr, arr)`|Adds `expr` to `arr` at the beginning, the resulting array type determined by the type of the array.|
-|`MV_APPEND(arr1, expr)`|Appends `expr` to `arr`, the resulting array type determined by the type of the first array.|
+|`MV_OFFSET_OF(arr, expr)`|Returns the 0-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `null` or -1 if `druid.generic.useDefaultValueForNull=true` (legacy mode).|
+|`MV_ORDINAL_OF(arr, expr)`|Returns the 1-based index of the first occurrence of `expr` in the array. If no matching elements exist in the array, returns `null` or `-1` if `druid.generic.useDefaultValueForNull=true` (legacy mode).|
+|`MV_PREPEND(expr, arr)`|Adds `expr` to the beginning of `arr`, the resulting array type determined by the type `arr`.|
+|`MV_APPEND(arr, expr)`|Appends `expr` to `arr`, the resulting array type determined by the type of `arr`.|
|`MV_CONCAT(arr1, arr2)`|Concatenates `arr2` to `arr1`. The resulting array type is determined by the type of `arr1`.|
|`MV_SLICE(arr, start, end)`|Returns the subarray of `arr` from the 0-based index start(inclusive) to end(exclusive), or `null`, if start is less than 0, greater than length of arr or greater than end.|
|`MV_TO_STRING(arr, str)`|Joins all elements of `arr` by the delimiter specified by `str`.|
diff --git a/docs/querying/sql-operators.md b/docs/querying/sql-operators.md
index 22e216289485..da295821ecf9 100644
--- a/docs/querying/sql-operators.md
+++ b/docs/querying/sql-operators.md
@@ -31,8 +31,10 @@ sidebar_label: "Operators"
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
+ This document describes the SQL language.
+:::
Operators in [Druid SQL](./sql.md) typically operate on one or two values and return a result based on the values. Types of operators in Druid SQL include arithmetic, comparison, logical, and more, as described here.
@@ -77,7 +79,9 @@ Also see the [CONCAT function](sql-scalar.md#string-functions).
|Operator|Description|
|--------|-----------|
|`x = y` |Equal to|
+|`x IS NOT DISTINCT FROM y`|Equal to, considering `NULL` as a value. Never returns `NULL`.|
|`x <> y`|Not equal to|
+|`x IS DISTINCT FROM y`|Not equal to, considering `NULL` as a value. Never returns `NULL`.|
|`x > y` |Greater than|
|`x >= y`|Greater than or equal to|
|`x < y` |Less than|
diff --git a/docs/querying/sql-query-context.md b/docs/querying/sql-query-context.md
index e469fa390a7e..dc192db17183 100644
--- a/docs/querying/sql-query-context.md
+++ b/docs/querying/sql-query-context.md
@@ -23,8 +23,10 @@ sidebar_label: "SQL query context"
~ under the License.
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
+ This document describes the SQL language.
+:::
Druid supports query context parameters which affect [SQL query](./sql.md) planning.
See [Query context](query-context.md) for general query context parameters for all query types.
@@ -44,6 +46,7 @@ Configure Druid SQL query planning using the parameters in the table below.
|`enableTimeBoundaryPlanning`|If true, SQL queries will get converted to TimeBoundary queries wherever possible. TimeBoundary queries are very efficient for min-max calculation on `__time` column in a datasource |`druid.query.default.context.enableTimeBoundaryPlanning` on the Broker (default: false)|
|`useNativeQueryExplain`|If true, `EXPLAIN PLAN FOR` will return the explain plan as a JSON representation of equivalent native query(s), else it will return the original version of explain plan generated by Calcite.
This property is provided for backwards compatibility. It is not recommended to use this parameter unless you were depending on the older behavior.|`druid.sql.planner.useNativeQueryExplain` on the Broker (default: true)|
|`sqlFinalizeOuterSketches`|If false (default behavior in Druid 25.0.0 and later), `DS_HLL`, `DS_THETA`, and `DS_QUANTILES_SKETCH` return sketches in query results, as documented. If true (default behavior in Druid 24.0.1 and earlier), sketches from these functions are finalized when they appear in query results.
This property is provided for backwards compatibility with behavior in Druid 24.0.1 and earlier. It is not recommended to use this parameter unless you were depending on the older behavior. Instead, use a function that does not return a sketch, such as `APPROX_COUNT_DISTINCT_DS_HLL`, `APPROX_COUNT_DISTINCT_DS_THETA`, `APPROX_QUANTILE_DS`, `DS_THETA_ESTIMATE`, or `DS_GET_QUANTILE`.|`druid.query.default.context.sqlFinalizeOuterSketches` on the Broker (default: false)|
+|`sqlUseBoundAndSelectors`|If false (default behavior if `druid.generic.useDefaultValueForNull=false` in Druid 27.0.0 and later), the SQL planner will use [equality](./filters.md#equality-filter), [null](./filters.md#null-filter), and [range](./filters.md#range-filter) filters instead of [selector](./filters.md#selector-filter) and [bounds](./filters.md#bound-filter). This value must be set to `false` for correct behavior for filtering `ARRAY` typed values. | Defaults to same value as `druid.generic.useDefaultValueForNull`, which is `false`|
## Setting the query context
The query context parameters can be specified as a "context" object in the [JSON API](../api-reference/sql-api.md) or as a [JDBC connection properties object](../api-reference/sql-jdbc.md).
diff --git a/docs/querying/sql-scalar.md b/docs/querying/sql-scalar.md
index a741c1ff8bab..e60a8fe43967 100644
--- a/docs/querying/sql-scalar.md
+++ b/docs/querying/sql-scalar.md
@@ -31,8 +31,10 @@ sidebar_label: "Scalar functions"
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
+ This document describes the SQL language.
+:::
[Druid SQL](./sql.md) includes scalar functions that include numeric and string functions, IP address functions, Sketch functions, and more, as described on this page.
diff --git a/docs/querying/sql-translation.md b/docs/querying/sql-translation.md
index 5528db95323f..29edc9325318 100644
--- a/docs/querying/sql-translation.md
+++ b/docs/querying/sql-translation.md
@@ -23,8 +23,10 @@ sidebar_label: "SQL query translation"
~ under the License.
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the Druid SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
+ This document describes the Druid SQL language.
+:::
Druid uses [Apache Calcite](https://calcite.apache.org/) to parse and plan SQL queries.
Druid translates SQL statements into its [native JSON-based query language](querying.md).
@@ -782,7 +784,9 @@ Refer to the [Query execution](query-execution.md#join) page for information abo
Subqueries in SQL are generally translated to native query datasources. Refer to the
[Query execution](query-execution.md#query) page for information about how subqueries are executed.
-> Note: Subqueries in the WHERE clause, like `WHERE col1 IN (SELECT foo FROM ...)` are translated to inner joins.
+:::info
+ Note: Subqueries in the WHERE clause, like `WHERE col1 IN (SELECT foo FROM ...)` are translated to inner joins.
+:::
## Approximations
diff --git a/docs/querying/sql.md b/docs/querying/sql.md
index c68ce28c845b..378bf302872b 100644
--- a/docs/querying/sql.md
+++ b/docs/querying/sql.md
@@ -23,8 +23,10 @@ sidebar_label: "Overview and syntax"
~ under the License.
-->
-> Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
-> This document describes the SQL language.
+:::info
+ Apache Druid supports two query languages: Druid SQL and [native queries](querying.md).
+ This document describes the SQL language.
+:::
You can query data in Druid datasources using Druid SQL. Druid translates SQL queries into its [native query language](querying.md). To learn about translation and how to get the best performance from Druid SQL, see [SQL query translation](sql-translation.md).
@@ -85,8 +87,10 @@ documentation.
## UNNEST
-> The UNNEST SQL function is [experimental](../development/experimental.md). Its API and behavior are subject
-> to change in future releases. It is not recommended to use this feature in production at this time.
+:::info
+ The UNNEST SQL function is [experimental](../development/experimental.md). Its API and behavior are subject
+ to change in future releases. It is not recommended to use this feature in production at this time.
+:::
The UNNEST clause unnests array values. It's the SQL equivalent to the [unnest datasource](./datasource.md#unnest). The source for UNNEST can be an array or an input that's been transformed into an array, such as with helper functions like MV_TO_ARRAY or ARRAY.
@@ -220,7 +224,9 @@ UNION ALL
SELECT COUNT(*) FROM tbl WHERE my_column = 'value2'
```
-> With top-level queries, you can't apply GROUP BY, ORDER BY, or any other operator to the results of a UNION ALL.
+:::info
+ With top-level queries, you can't apply GROUP BY, ORDER BY, or any other operator to the results of a UNION ALL.
+:::
### Table-level
@@ -250,8 +256,10 @@ Add "EXPLAIN PLAN FOR" to the beginning of any query to get information about ho
the query will not actually be executed. Refer to the [Query translation](sql-translation.md#interpreting-explain-plan-output)
documentation for more information on the output of EXPLAIN PLAN.
-> For the legacy plan, be careful when interpreting EXPLAIN PLAN output, and use [request logging](../configuration/index.md#request-logging) if in doubt.
+:::info
+ For the legacy plan, be careful when interpreting EXPLAIN PLAN output, and use [request logging](../configuration/index.md#request-logging) if in doubt.
Request logs show the exact native query that will be run. Alternatively, to see the native query plan, set `useNativeQueryExplain` to true in the query context.
+:::
## Identifiers and literals
diff --git a/docs/querying/timeboundaryquery.md b/docs/querying/timeboundaryquery.md
index e2f3cf0de46e..81fece0e60e0 100644
--- a/docs/querying/timeboundaryquery.md
+++ b/docs/querying/timeboundaryquery.md
@@ -23,9 +23,11 @@ sidebar_label: "TimeBoundary"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes a query
-> type that is only available in the native language.
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes a query
+ type that is only available in the native language.
+:::
Time boundary queries return the earliest and latest data points of a data set. The grammar is:
diff --git a/docs/querying/timeseriesquery.md b/docs/querying/timeseriesquery.md
index 1de8d901de7e..78821b73910d 100644
--- a/docs/querying/timeseriesquery.md
+++ b/docs/querying/timeseriesquery.md
@@ -23,10 +23,12 @@ sidebar_label: "Timeseries"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes a query
-> type in the native language. For information about when Druid SQL will use this query type, refer to the
-> [SQL documentation](sql-translation.md#query-types).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes a query
+ type in the native language. For information about when Druid SQL will use this query type, refer to the
+ [SQL documentation](sql-translation.md#query-types).
+:::
These types of queries take a timeseries query object and return an array of JSON objects where each object represents a value asked for by the timeseries query.
diff --git a/docs/querying/tips-good-queries.md b/docs/querying/tips-good-queries.md
new file mode 100644
index 000000000000..8b718d9b76ff
--- /dev/null
+++ b/docs/querying/tips-good-queries.md
@@ -0,0 +1,200 @@
+---
+id: tips-good-queries
+title: "Tips for writing good queries in Druid"
+sidebar_label: "Tips for writing good queries"
+---
+
+
+
+This topic includes tips and examples that can help you investigate and improve query performance and accuracy using [Apache Druid SQL](./sql.md). Use this topic as a companion to the Jupyter Notebook tutorial [Learn the basics of Druid SQL](https://github.com/apache/druid/blob/master/examples/quickstart/jupyter-notebooks/notebooks/03-query/00-using-sql-with-druidapi.ipynb).
+
+Your ability to effectively query your data depends in large part on the way you've ingested and stored the data in Apache Druid. This document assumes that you've followed the best practices described in [Schema design tips and best practices](../ingestion/schema-design.md#general-tips-and-best-practices) when modeling your data.
+
+## Investigate query performance
+
+If your queries run slower than anticipated, you can use the following tools to investigate query performance issues.
+
+### Analyze query metrics
+
+You can configure Druid processes to emit metrics that are essential for monitoring query execution. See [Query metrics](../operations/metrics.md#query-metrics) for more information.
+
+### Generate an explain plan
+
+An explain plan shows the full query details and all of the operations Druid performs to execute it. You can use the information in the plan to identify possible areas of query improvement.
+
+See [Explain plan](./sql.md#explain-plan) and [Interpreting explain plan output](./sql-translation.md#interpreting-explain-plan-output) for more information.
+
+You can follow the [Get to know Query view tutorial](../tutorials/tutorial-sql-query-view.md) to create an example explain plan in the Druid console.
+
+## Improve query performance
+
+In most cases, you can improve query performance by adjusting Druid settings and by manually tuning your queries.
+
+### Adjust Druid settings
+
+This section outlines Druid settings that can help to improve query performance.
+
+#### Turn on query caching
+
+You can enable caching in Druid to improve query times for frequently accessed data. Caching enables increased concurrency on the same system, leading to noticeable performance improvements for queries handling throughput for concurrent, mixed workloads.
+
+The largest performance gains from caching tend to apply to TopN and timeseries queries. For GroupBy queries, if the bottleneck is in the merging phase on the Broker, enabling caching results in little noticeable query improvement. See [Performance considerations for caching](./caching.md#performance-considerations-for-caching) for more information.
+
+#### Use approximation
+
+When possible, design your SQL queries in such a way that they match the rules for TopN approximation, so that Druid enables TopN by default. For Druid to automatically optimize for TopN, your SQL query must include the following:
+
+- GROUP BY on one dimension, and
+- ORDER BY on one aggregate.
+
+ See [TopN queries](./topnquery.md) for more information.
+
+Note that TopN queries are approximate in that each data process ranks its top K results and only returns those top K results to the Broker.
+
+You can follow the tutorial [Using TopN approximation in Druid queries](https://github.com/apache/druid/blob/master/examples/quickstart/jupyter-notebooks/notebooks/03-query/02-approxRanking.ipynb) to work through some examples with approximation turned on and off. The tutorial [Get to know Query view](../tutorials/tutorial-sql-query-view.md) demonstrates running aggregate queries in the Druid console.
+
+### Manually tune your queries
+
+This section outlines techniques you can use to improve your query accuracy and performance.
+
+#### Query one table at a time
+
+Query a single table at a time to minimize the load on the Druid processor.
+
+#### Select specific columns
+
+Only select the columns needed for the query instead of retrieving all columns from the table. This reduces the amount of data retrieved from the database, which improves query performance.
+
+#### Use filters
+
+Use filters, for example the WHERE clause, and filter on time. Try to minimize the use of inequality filters, because they're very resource-intensive.
+
+The following example query filters on `__time` and `product`:
+
+```
+SELECT
+ FLOOR(__time to day),
+ product,
+ sum(quantity * price) as revenue
+FROM "orders"
+WHERE
+ __time > '2023-08-20' and product = 'product 1'
+GROUP BY 1, 2
+```
+
+The following example uses a wildcard filter on the `diffUrl` column:
+
+```
+SELECT * from Wikipedia
+WHERE diffUrl LIKE 'https://en.wikipedia%'
+AND TIME_IN_INTERVAL(__time, '2016-06-27T01:00:00/2016-06-27T02:00:00')
+```
+
+#### Shorten your queries
+
+Make your queries shorter where possible—Druid processes shorter queries faster. You might also be able to divide a single query into multiple queries.
+
+For example, the following query aggregates over multiple datasources using UNION ALL:
+
+```
+SELECT id, SUM(revenue) FROM
+ (SELECT id, revenue from datasource_1
+UNION ALL
+ SELECT id, revenue FROM datasource_2)
+...
+UNION ALL
+ SELECT id, revenue FROM datasource_n)
+GROUP BY id
+```
+
+To simplify this query, you could split it into several queries, for example:
+
+```
+SELECT id, SUM(revenue) FROM datasource_1
+
+SELECT id, SUM(revenue) FROM datasource_2
+...
+SELECT id, SUM(revenue) FROM datasource_n
+```
+
+You could then manually aggregate the results of the individual queries.
+
+#### Minimize or remove subqueries
+
+Consider whether you can pre-compute a subquery task and store it as a join or make it a part of the datasource. See [Datasources: join](./datasource.md#join) and [SQL query translation: Joins](./sql-translation.md#joins) for more information and examples.
+
+#### Consider alternatives to GroupBy
+
+Consider using Timeseries and TopN as alternatives to GroupBy. See [GroupBy queries: alternatives](./groupbyquery.md#alternatives) for more information.
+
+Avoid grouping on high cardinality columns, for example user ID. Investigate whether you can apply a filter first, to reduce the number of results for grouping.
+
+#### Query over smaller intervals
+
+Consider whether you can query a smaller time interval to return a smaller results set.
+
+For example, the following query doesn't limit on time and could be resource-intensive:
+
+```
+SELECT cust_id, sum(revenue) FROM myDatasource
+GROUP BY cust_id
+```
+
+This query could be split into multiple queries over smaller time spans, with the results combined client-side. For example:
+
+```
+SELECT cust_id, sum(revenue) FROM myDatasource
+GROUP BY cust_id
+WHERE __time BETWEEN '2023-07-01' AND '2023-07-31'
+
+SELECT cust_id, sum(revenue) FROM myDatasource
+GROUP BY cust_id
+WHERE __time BETWEEN '2023-08-01' AND '2023-08-31'
+```
+
+#### Reduce the computation in your queries
+
+Examine your query to see if it uses a lot of transformations, functions, and expressions. Consider whether you could rewrite the query to reduce the level of computation.
+
+## Druid SQL query example
+
+The following example query demonstrates many of the tips outlined in this topic.
+The query:
+
+- selects specific dimensions and metrics
+- uses approximation
+- selects from a single table
+- groups by low cardinality columns
+- filters on both dimensions and time
+- orders by a dimension and a measure
+- includes a limit
+
+```
+SELECT
+ FLOOR() AS month,
+ country,
+ SUM(price),
+ APPROX_COUNT_DISTINCT_DS_HLL(userid)
+FROM sales
+GROUP BY month, country
+WHERE artist = 'Madonna' AND TIME_IN_INTERVAL(__time, '2023-08-01/P1M')
+ORDER BY country, SUM(price) DESC
+LIMIT 100
+```
diff --git a/docs/querying/topnmetricspec.md b/docs/querying/topnmetricspec.md
index 5c2253667538..844f0cf30662 100644
--- a/docs/querying/topnmetricspec.md
+++ b/docs/querying/topnmetricspec.md
@@ -22,9 +22,11 @@ title: "Sorting (topN)"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes the native
-> language. For information about sorting in SQL, refer to the [SQL documentation](sql.md#order-by).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes the native
+ language. For information about sorting in SQL, refer to the [SQL documentation](sql.md#order-by).
+:::
In Apache Druid, the topN metric spec specifies how topN values should be sorted.
diff --git a/docs/querying/topnquery.md b/docs/querying/topnquery.md
index 775d34af3a8a..663ae2ce7db5 100644
--- a/docs/querying/topnquery.md
+++ b/docs/querying/topnquery.md
@@ -23,10 +23,12 @@ sidebar_label: "TopN"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes a query
-> type in the native language. For information about when Druid SQL will use this query type, refer to the
-> [SQL documentation](sql-translation.md#query-types).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes a query
+ type in the native language. For information about when Druid SQL will use this query type, refer to the
+ [SQL documentation](sql-translation.md#query-types).
+:::
Apache Druid TopN queries return a sorted set of results for the values in a given dimension according to some criteria. Conceptually, they can be thought of as an approximate [GroupByQuery](../querying/groupbyquery.md) over a single dimension with an [Ordering](../querying/limitspec.md) spec. TopNs are much faster and resource efficient than GroupBys for this use case. These types of queries take a topN query object and return an array of JSON objects where each object represents a value asked for by the topN query.
diff --git a/docs/querying/troubleshooting.md b/docs/querying/troubleshooting.md
index 042cfb65dfbc..4b9a83d8a291 100644
--- a/docs/querying/troubleshooting.md
+++ b/docs/querying/troubleshooting.md
@@ -63,6 +63,6 @@ To mitigate query failure due to web server timeout:
Set the max idle time in the `druid.server.http.maxIdleTime` property in the `historical/runtime.properties` file.
You must restart the Druid cluster for this change to take effect.
See [Configuration reference](../configuration/index.md) for more information on configuring the server.
-* If the timeout occurs because the data servers have not pushed any results to the Broker, consider optimizing data server performance. Significant slowdown in the data servers may be a result of spilling too much data to disk in [groupBy v2 queries](groupbyquery.md#performance-tuning-for-groupby-v2), large [`IN` filters](filters.md#in-filter) in the query, or an under scaled cluster. Analyze your [Druid query metrics](../operations/metrics.md#query-metrics) to determine the bottleneck.
+* If the timeout occurs because the data servers have not pushed any results to the Broker, consider optimizing data server performance. Significant slowdown in the data servers may be a result of spilling too much data to disk in [groupBy queries](groupbyquery.md#performance-tuning-for-groupby), large [`IN` filters](filters.md#in-filter) in the query, or an under scaled cluster. Analyze your [Druid query metrics](../operations/metrics.md#query-metrics) to determine the bottleneck.
* If the timeout is caused by Broker backpressure, consider optimizing Broker performance. Check whether the connection is fast enough between the Broker and deep storage.
diff --git a/docs/querying/virtual-columns.md b/docs/querying/virtual-columns.md
index 6a7e8604c4a7..fbfae24ed545 100644
--- a/docs/querying/virtual-columns.md
+++ b/docs/querying/virtual-columns.md
@@ -22,10 +22,12 @@ title: "Virtual columns"
~ under the License.
-->
-> Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
-> This document describes the native
-> language. For information about functions available in SQL, refer to the
-> [SQL documentation](sql-scalar.md).
+:::info
+ Apache Druid supports two query languages: [Druid SQL](sql.md) and [native queries](querying.md).
+ This document describes the native
+ language. For information about functions available in SQL, refer to the
+ [SQL documentation](sql-scalar.md).
+:::
Virtual columns are queryable column "views" created from a set of columns during a query.
diff --git a/docs/tutorials/cluster.md b/docs/tutorials/cluster.md
index 85acf063f7e9..f1bea59ff957 100644
--- a/docs/tutorials/cluster.md
+++ b/docs/tutorials/cluster.md
@@ -136,8 +136,10 @@ We recommend running your favorite Linux distribution. You will also need
* [Java 8u92+, 11, or 17](../operations/java.md)
* Python 2 or Python 3
-> If needed, you can specify where to find Java using the environment variables
-> `DRUID_JAVA_HOME` or `JAVA_HOME`. For more details run the `bin/verify-java` script.
+:::info
+ If needed, you can specify where to find Java using the environment variables
+ `DRUID_JAVA_HOME` or `JAVA_HOME`. For more details run the `bin/verify-java` script.
+:::
For information about installing Java, see the documentation for your OS package manager. If your Ubuntu-based OS does not have a recent enough version of Java, WebUpd8 offers [packages for those
OSes](http://www.webupd8.org/2012/09/install-oracle-java-8-in-ubuntu-via-ppa.html).
@@ -409,8 +411,10 @@ inbound connections on the following:
- 8082 (Broker)
- 8088 (Router, if used)
-> In production, we recommend deploying ZooKeeper and your metadata store on their own dedicated hardware,
-> rather than on the Master server.
+:::info
+ In production, we recommend deploying ZooKeeper and your metadata store on their own dedicated hardware,
+ rather than on the Master server.
+:::
## Start Master Server
@@ -439,7 +443,9 @@ can start the Master server processes together with ZK using:
bin/start-cluster-master-with-zk-server
```
-> In production, we also recommend running a ZooKeeper cluster on its own dedicated hardware.
+:::info
+ In production, we also recommend running a ZooKeeper cluster on its own dedicated hardware.
+:::
## Start Data Server
@@ -453,8 +459,10 @@ bin/start-cluster-data-server
You can add more Data servers as needed.
-> For clusters with complex resource allocation needs, you can break apart Historicals and MiddleManagers and scale the components individually.
-> This also allows you take advantage of Druid's built-in MiddleManager autoscaling facility.
+:::info
+ For clusters with complex resource allocation needs, you can break apart Historicals and MiddleManagers and scale the components individually.
+ This also allows you take advantage of Druid's built-in MiddleManager autoscaling facility.
+:::
## Start Query Server
diff --git a/docs/tutorials/tutorial-batch-hadoop.md b/docs/tutorials/tutorial-batch-hadoop.md
index 065dc76c1e0b..ec3c2014dd9f 100644
--- a/docs/tutorials/tutorial-batch-hadoop.md
+++ b/docs/tutorials/tutorial-batch-hadoop.md
@@ -38,18 +38,18 @@ Once the Docker install is complete, please proceed to the next steps in the tut
## Build the Hadoop docker image
-For this tutorial, we've provided a Dockerfile for a Hadoop 2.8.5 cluster, which we'll use to run the batch indexing task.
+For this tutorial, we've provided a Dockerfile for a Hadoop 3.3.6 cluster, which we'll use to run the batch indexing task.
This Dockerfile and related files are located at `quickstart/tutorial/hadoop/docker`.
-From the apache-druid-{{DRUIDVERSION}} package root, run the following commands to build a Docker image named "druid-hadoop-demo" with version tag "2.8.5":
+From the apache-druid-{{DRUIDVERSION}} package root, run the following commands to build a Docker image named "druid-hadoop-demo" with version tag "3.3.6":
```bash
cd quickstart/tutorial/hadoop/docker
-docker build -t druid-hadoop-demo:2.8.5 .
+docker build -t druid-hadoop-demo:3.3.6 .
```
-This will start building the Hadoop image. Once the image build is done, you should see the message `Successfully tagged druid-hadoop-demo:2.8.5` printed to the console.
+This will start building the Hadoop image. Once the image build is done, you should see the message `Successfully tagged druid-hadoop-demo:3.3.6` printed to the console.
## Setup the Hadoop docker cluster
@@ -77,7 +77,7 @@ On the host machine, add the following entry to `/etc/hosts`:
Once the `/tmp/shared` folder has been created and the `etc/hosts` entry has been added, run the following command to start the Hadoop container.
```bash
-docker run -it -h druid-hadoop-demo --name druid-hadoop-demo -p 2049:2049 -p 2122:2122 -p 8020:8020 -p 8021:8021 -p 8030:8030 -p 8031:8031 -p 8032:8032 -p 8033:8033 -p 8040:8040 -p 8042:8042 -p 8088:8088 -p 8443:8443 -p 9000:9000 -p 10020:10020 -p 19888:19888 -p 34455:34455 -p 49707:49707 -p 50010:50010 -p 50020:50020 -p 50030:50030 -p 50060:50060 -p 50070:50070 -p 50075:50075 -p 50090:50090 -p 51111:51111 -v /tmp/shared:/shared druid-hadoop-demo:2.8.5 /etc/bootstrap.sh -bash
+docker run -it -h druid-hadoop-demo --name druid-hadoop-demo -p 2049:2049 -p 2122:2122 -p 8020:8020 -p 8021:8021 -p 8030:8030 -p 8031:8031 -p 8032:8032 -p 8033:8033 -p 8040:8040 -p 8042:8042 -p 8088:8088 -p 8443:8443 -p 9000:9000 -p 10020:10020 -p 19888:19888 -p 34455:34455 -p 49707:49707 -p 50010:50010 -p 50020:50020 -p 50030:50030 -p 50060:50060 -p 50070:50070 -p 50075:50075 -p 50090:50090 -p 51111:51111 -v /tmp/shared:/shared druid-hadoop-demo:3.3.6 /etc/bootstrap.sh -bash
```
Once the container is started, your terminal will attach to a bash shell running inside the container:
@@ -125,6 +125,7 @@ cd /usr/local/hadoop/bin
./hdfs dfs -mkdir /druid
./hdfs dfs -mkdir /druid/segments
./hdfs dfs -mkdir /quickstart
+./hdfs dfs -mkdir /user
./hdfs dfs -chmod 777 /druid
./hdfs dfs -chmod 777 /druid/segments
./hdfs dfs -chmod 777 /quickstart
@@ -205,10 +206,10 @@ We've included a sample of Wikipedia edits from September 12, 2015 to get you st
To load this data into Druid, you can submit an *ingestion task* pointing to the file. We've included
a task that loads the `wikiticker-2015-09-12-sampled.json.gz` file included in the archive.
-Let's submit the `wikipedia-index-hadoop.json` task:
+Let's submit the `wikipedia-index-hadoop3.json` task:
```bash
-bin/post-index-task --file quickstart/tutorial/wikipedia-index-hadoop.json --url http://localhost:8081
+bin/post-index-task --file quickstart/tutorial/wikipedia-index-hadoop3.json --url http://localhost:8081
```
## Querying your data
diff --git a/docs/tutorials/tutorial-batch-native.md b/docs/tutorials/tutorial-batch-native.md
index 2f1bd7c47cba..e264b27b5763 100644
--- a/docs/tutorials/tutorial-batch-native.md
+++ b/docs/tutorials/tutorial-batch-native.md
@@ -150,7 +150,10 @@ You can now see the data as a datasource in the console and try out a query, as
![Datasource view](../assets/tutorial-batch-data-loader-10.png "Datasource view")
- > Notice the other actions you can perform for a datasource, including configuring retention rules, compaction, and more.
+:::info
+ Notice the other actions you can perform for a datasource, including configuring retention rules, compaction, and more.
+:::
+
3. Run the prepopulated query, `SELECT * FROM "wikipedia"` to see the results.
![Query view](../assets/tutorial-batch-data-loader-11.png "Query view")
diff --git a/docs/tutorials/tutorial-compaction.md b/docs/tutorials/tutorial-compaction.md
index 51d67c331fc1..c4a918897ab0 100644
--- a/docs/tutorials/tutorial-compaction.md
+++ b/docs/tutorials/tutorial-compaction.md
@@ -51,7 +51,9 @@ Submit the spec as follows to create a datasource called `compaction-tutorial`:
bin/post-index-task --file quickstart/tutorial/compaction-init-index.json --url http://localhost:8081
```
-> `maxRowsPerSegment` in the tutorial ingestion spec is set to 1000 to generate multiple segments per hour for demonstration purposes. Do not use this spec in production.
+:::info
+ `maxRowsPerSegment` in the tutorial ingestion spec is set to 1000 to generate multiple segments per hour for demonstration purposes. Do not use this spec in production.
+:::
After the ingestion completes, navigate to [http://localhost:8888/unified-console.html#datasources](http://localhost:8888/unified-console.html#datasources) in a browser to see the new datasource in the web console.
diff --git a/docs/tutorials/tutorial-jupyter-docker.md b/docs/tutorials/tutorial-jupyter-docker.md
index 0cb055d2fe2f..a1091f0ab7a4 100644
--- a/docs/tutorials/tutorial-jupyter-docker.md
+++ b/docs/tutorials/tutorial-jupyter-docker.md
@@ -111,7 +111,7 @@ If Druid is running local to the same machine as Jupyter, open the tutorial and
host = "host.docker.internal"
```
-To enable Druid to ingest data from Kafka within the Docker Compose environment, update the `bootstrap.servers` property in the Kafka ingestion spec to `localhost:9094` before ingesting. For reference, see [more on consumer properties](../development/extensions-core/kafka-supervisor-reference.md#more-on-consumerproperties).
+To enable Druid to ingest data from Kafka within the Docker Compose environment, update the `bootstrap.servers` property in the Kafka ingestion spec to `localhost:9094` before ingesting. For reference, see [Consumer properties](../development/extensions-core/kafka-supervisor-reference.md#consumer-properties).
### Update image from Docker Hub
@@ -140,7 +140,9 @@ At startup, Docker creates and mounts a volume to persist data from the containe
This way you can save your work completed within the Docker container.
1. Navigate to the notebooks at http://localhost:8889.
- > If you set `JUPYTER_PORT` to another port number, replace `8889` with the value of the Jupyter port.
+:::info
+ If you set `JUPYTER_PORT` to another port number, replace `8889` with the value of the Jupyter port.
+:::
2. Select a tutorial. If you don't plan to save your changes, you can use the notebook directly as is. Otherwise, continue to the next step.
diff --git a/docs/tutorials/tutorial-jupyter-index.md b/docs/tutorials/tutorial-jupyter-index.md
index 99279b646114..a0e14a5885fa 100644
--- a/docs/tutorials/tutorial-jupyter-index.md
+++ b/docs/tutorials/tutorial-jupyter-index.md
@@ -56,7 +56,7 @@ topics and use a simple set of Python wrappers around the underlying REST API. T
wrappers reside in the `druidapi` package within the notebooks directory. While the package
can be used in any Python program, the key purpose, at present, is to support these
notebooks. See
-[Introduction to the Druid Python API](https://github.com/apache/druid/blob/master/examples/quickstart/jupyter-notebooks/Python_API_Tutorial.ipynb)
+[Introduction to the Druid Python API](https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb)
for an overview of the Python API.
The `druidapi` package is already installed in the custom Jupyter Docker container for Druid tutorials.
diff --git a/docs/tutorials/tutorial-kafka.md b/docs/tutorials/tutorial-kafka.md
index 3340f42c6ece..9e74f467c4f3 100644
--- a/docs/tutorials/tutorial-kafka.md
+++ b/docs/tutorials/tutorial-kafka.md
@@ -45,9 +45,11 @@ Before you follow the steps in this tutorial, download Druid as described in the
```
2. If you're already running Kafka on the machine you're using for this tutorial, delete or rename the `kafka-logs` directory in `/tmp`.
- > Druid and Kafka both rely on [Apache ZooKeeper](https://zookeeper.apache.org/) to coordinate and manage services. Because Druid is already running, Kafka attaches to the Druid ZooKeeper instance when it starts up.
- >
- > In a production environment where you're running Druid and Kafka on different machines, [start the Kafka ZooKeeper](https://kafka.apache.org/quickstart) before you start the Kafka broker.
+:::info
+ Druid and Kafka both rely on [Apache ZooKeeper](https://zookeeper.apache.org/) to coordinate and manage services. Because Druid is already running, Kafka attaches to the Druid ZooKeeper instance when it starts up.
+
+ In a production environment where you're running Druid and Kafka on different machines, [start the Kafka ZooKeeper](https://kafka.apache.org/quickstart) before you start the Kafka broker.
+:::
3. In the Kafka root directory, run this command to start a Kafka broker:
@@ -76,8 +78,7 @@ In this section, you download sample data to the tutorial's directory and send t
2. Download the sample data to your new directory and extract it:
```bash
- cd sample-data
- curl -O https://static.imply.io/example-data/kttm-nested-v2/kttm-nested-v2-2019-08-25.json.gz
+ (cd sample-data && curl -O https://static.imply.io/example-data/kttm-nested-v2/kttm-nested-v2-2019-08-25.json.gz)
```
3. In your Kafka root directory, run the following commands to post sample events to the `kttm` Kafka topic:
@@ -171,7 +172,9 @@ To use the console data loader:
When the `kttm-kafka` datasource appears here, you can query it. See [Query your data](#query-your-data) for details.
- > If the datasource doesn't appear after a minute you might not have set the supervisor to read data from the start of the stream—the `Use earliest offset` setting in the **Tune** step. Go to the **Ingestion** page and terminate the supervisor using the **Actions(...)** menu. [Load the sample data](#load-data-with-the-console-data-loader) again and apply the correct setting when you get to the **Tune** step.
+:::info
+ If the datasource doesn't appear after a minute you might not have set the supervisor to read data from the start of the stream—the `Use earliest offset` setting in the **Tune** step. Go to the **Ingestion** page and terminate the supervisor using the **Actions(...)** menu. [Load the sample data](#load-data-with-the-console-data-loader) again and apply the correct setting when you get to the **Tune** step.
+:::
### Submit a supervisor spec
@@ -265,13 +268,13 @@ You can also use the Druid API to submit a supervisor spec.
1. Run the following command to download the sample spec:
```bash
- curl -O https://druid.apache.org/docs/latest/assets/files/kttm-kafka-supervisor.json
+ curl -o kttm-kafka-supervisor.json https://raw.githubusercontent.com/apache/druid/master/docs/assets/files/kttm-kafka-supervisor.json
```
2. Run the following command to submit the spec in the `kttm-kafka-supervisor.json` file:
```bash
- curl -XPOST -H 'Content-Type: application/json' kttm-kafka-supervisor.json http://localhost:8081/druid/indexer/v1/supervisor
+ curl -X POST -H 'Content-Type: application/json' -d @kttm-kafka-supervisor.json http://localhost:8081/druid/indexer/v1/supervisor
```
After Druid successfully creates the supervisor, you get a response containing the supervisor ID: `{"id":"kttm-kafka-supervisor-api"}`.
diff --git a/docs/tutorials/tutorial-msq-convert-spec.md b/docs/tutorials/tutorial-msq-convert-spec.md
index e98913574022..bbb66eec4688 100644
--- a/docs/tutorials/tutorial-msq-convert-spec.md
+++ b/docs/tutorials/tutorial-msq-convert-spec.md
@@ -24,9 +24,11 @@ description: How to convert an ingestion spec to a query for SQL-based ingestion
~ under the License.
-->
-> This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
-> extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
-> ingestion method is right for you.
+:::info
+ This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
+ extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
+ ingestion method is right for you.
+:::
If you're already ingesting data with [native batch ingestion](../ingestion/native-batch.md), you can use the [web console](../operations/web-console.md) to convert the ingestion spec to a SQL query that the multi-stage query task engine can use to ingest data.
diff --git a/docs/tutorials/tutorial-msq-extern.md b/docs/tutorials/tutorial-msq-extern.md
index 931c28d622c3..afd454e79a49 100644
--- a/docs/tutorials/tutorial-msq-extern.md
+++ b/docs/tutorials/tutorial-msq-extern.md
@@ -24,9 +24,11 @@ description: How to generate a query that references externally hosted data
~ under the License.
-->
-> This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
-> extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
-> ingestion method is right for you.
+:::info
+ This page describes SQL-based batch ingestion using the [`druid-multi-stage-query`](../multi-stage-query/index.md)
+ extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
+ ingestion method is right for you.
+:::
This tutorial demonstrates how to generate a query that references externally hosted data using the **Connect external data** wizard.
diff --git a/docs/tutorials/tutorial-query-deep-storage.md b/docs/tutorials/tutorial-query-deep-storage.md
new file mode 100644
index 000000000000..5502ad94228d
--- /dev/null
+++ b/docs/tutorials/tutorial-query-deep-storage.md
@@ -0,0 +1,293 @@
+---
+id: tutorial-query-deep-storage
+title: "Tutorial: Query from deep storage"
+sidebar_label: "Query from deep storage"
+---
+
+
+
+
+> Query from deep storage is an [experimental feature](../development/experimental.md).
+
+Query from deep storage allows you to query segments that are stored only in deep storage, which provides lower costs than if you were to load everything onto Historical processes. The tradeoff is that queries from deep storage may take longer to complete.
+
+This tutorial walks you through loading example data, configuring load rules so that not all the segments get loaded onto Historical processes, and querying data from deep storage.
+
+To run the queries in this tutorial, replace `ROUTER:PORT` with the location of the Router process and its port number. For example, use `localhost:8888` for the quickstart deployment.
+
+For more general information, see [Query from deep storage](../querying/query-from-deep-storage.md).
+
+## Load example data
+
+Use the **Load data** wizard or the following SQL query to ingest the `wikipedia` sample datasource bundled with Druid. If you use the wizard, make sure you change the partitioning to be by hour.
+
+Partitioning by hour provides more segment granularity, so you can selectively load segments onto Historicals or keep them in deep storage.
+
+Show the query
+
+```sql
+REPLACE INTO "wikipedia" OVERWRITE ALL
+WITH "ext" AS (SELECT *
+FROM TABLE(
+ EXTERN(
+ '{"type":"http","uris":["https://druid.apache.org/data/wikipedia.json.gz"]}',
+ '{"type":"json"}'
+ )
+) EXTEND ("isRobot" VARCHAR, "channel" VARCHAR, "timestamp" VARCHAR, "flags" VARCHAR, "isUnpatrolled" VARCHAR, "page" VARCHAR, "diffUrl" VARCHAR, "added" BIGINT, "comment" VARCHAR, "commentLength" BIGINT, "isNew" VARCHAR, "isMinor" VARCHAR, "delta" BIGINT, "isAnonymous" VARCHAR, "user" VARCHAR, "deltaBucket" BIGINT, "deleted" BIGINT, "namespace" VARCHAR, "cityName" VARCHAR, "countryName" VARCHAR, "regionIsoCode" VARCHAR, "metroCode" BIGINT, "countryIsoCode" VARCHAR, "regionName" VARCHAR))
+SELECT
+ TIME_PARSE("timestamp") AS "__time",
+ "isRobot",
+ "channel",
+ "flags",
+ "isUnpatrolled",
+ "page",
+ "diffUrl",
+ "added",
+ "comment",
+ "commentLength",
+ "isNew",
+ "isMinor",
+ "delta",
+ "isAnonymous",
+ "user",
+ "deltaBucket",
+ "deleted",
+ "namespace",
+ "cityName",
+ "countryName",
+ "regionIsoCode",
+ "metroCode",
+ "countryIsoCode",
+ "regionName"
+FROM "ext"
+PARTITIONED BY HOUR
+```
+
+
+
+## Configure a load rule
+
+The load rule configures Druid to keep any segments that fall within the following interval only in deep storage:
+
+```
+2016-06-27T00:00:00.000Z/2016-06-27T02:59:00.000Z
+```
+
+The JSON form of the rule is as follows:
+
+```json
+[
+ {
+ "interval": "2016-06-27T00:00:00.000Z/2016-06-27T02:59:00.000Z",
+ "tieredReplicants": {},
+ "useDefaultTierForNull": false,
+ "type": "loadByInterval"
+ }
+]
+```
+
+The rest of the segments use the default load rules for the cluster. For the quickstart, that means all the other segments get loaded onto Historical processes.
+
+You can configure the load rules through the API or the Druid console. To configure the load rules through the Druid console, go to **Datasources > ... in the Actions column > Edit retention rules**. Then, paste the provided JSON into the JSON tab:
+
+![](../assets/tutorial-query-deepstorage-retention-rule.png)
+
+
+### Verify the replication factor
+
+Segments that are only available from deep storage have a `replication_factor` of 0 in the Druid system table. You can verify that your load rule worked as intended using the following query:
+
+```sql
+SELECT "segment_id", "replication_factor", "num_replicas" FROM sys."segments" WHERE datasource = 'wikipedia'
+```
+
+You can also verify it through the Druid console by checking the **Replication factor** column in the **Segments** view.
+
+Note that the number of replicas and replication factor may differ temporarily as Druid processes your retention rules.
+
+## Query from deep storage
+
+Now that there are segments that are only available from deep storage, run the following query:
+
+```sql
+SELECT page FROM wikipedia WHERE __time < TIMESTAMP'2016-06-27 00:10:00' LIMIT 10
+```
+
+With the context parameter:
+
+```json
+"executionMode": "ASYNC"
+```
+
+For example, run the following curl command:
+
+```
+curl --location 'http://localhost:8888/druid/v2/sql/statements' \
+--header 'Content-Type: application/json' \
+--data '{
+ "query":"SELECT page FROM wikipedia WHERE __time < TIMESTAMP'\''2016-06-27 00:10:00'\'' LIMIT 10",
+ "context":{
+ "executionMode":"ASYNC"
+ }
+}'
+```
+
+This query looks for records with timestamps that precede `00:10:00`. Based on the load rule you configured earlier, this data is only available from deep storage.
+
+When you submit the query from deep storage through the API, you get the following response:
+
+Show the response
+
+```json
+{
+ "queryId": "query-6888b6f6-e597-456c-9004-222b05b97051",
+ "state": "ACCEPTED",
+ "createdAt": "2023-07-28T21:59:02.334Z",
+ "schema": [
+ {
+ "name": "page",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ }
+ ],
+ "durationMs": -1
+}
+```
+
+Make sure you note the `queryID`. You'll need it to interact with the query.
+
+
+
+Compare this to if you were to submit the query to Druid SQL's regular endpoint, `POST /sql`:
+
+```
+curl --location 'http://localhost:8888/druid/v2/sql/' \
+--header 'Content-Type: application/json' \
+--data '{
+ "query":"SELECT page FROM wikipedia WHERE __time < TIMESTAMP'\''2016-06-27 00:10:00'\'' LIMIT 10",
+ "context":{
+ "executionMode":"ASYNC"
+ }
+}'
+```
+
+The response you get back is an empty response cause there are no records on the Historicals that match the query.
+
+## Get query status
+
+Replace `:queryId` with the ID for your query and run the following curl command to get your query status:
+
+```
+curl --location --request GET 'http://localhost:8888/druid/v2/sql/statements/:queryId' \
+--header 'Content-Type: application/json' \
+```
+
+
+### Response for a running query
+
+The response for a running query is the same as the response from when you submitted the query except the `state` is `RUNNING` instead of `ACCEPTED`.
+
+### Response for a completed query
+
+A successful query also returns a `pages` object that includes the page numbers (`id`), rows per page (`numRows`), and the size of the page (`sizeInBytes`). You can pass the page number as a parameter when you get results to refine the results you get.
+
+Note that `sampleRecords` has been truncated for brevity.
+
+Show the response
+
+```json
+{
+ "queryId": "query-6888b6f6-e597-456c-9004-222b05b97051",
+ "state": "SUCCESS",
+ "createdAt": "2023-07-28T21:59:02.334Z",
+ "schema": [
+ {
+ "name": "page",
+ "type": "VARCHAR",
+ "nativeType": "STRING"
+ }
+ ],
+ "durationMs": 87351,
+ "result": {
+ "numTotalRows": 152,
+ "totalSizeInBytes": 9036,
+ "dataSource": "__query_select",
+ "sampleRecords": [
+ [
+ "Salo Toraut"
+ ],
+ [
+ "利用者:ワーナー成増/放送ウーマン賞"
+ ],
+ [
+ "Bailando 2015"
+ ],
+ ...
+ ...
+ ...
+ ],
+ "pages": [
+ {
+ "id": 0,
+ "numRows": 152,
+ "sizeInBytes": 9036
+ }
+ ]
+ }
+}
+```
+
+
+
+## Get query results
+
+Replace `:queryId` with the ID for your query and run the following curl command to get your query results:
+
+```
+curl --location 'http://ROUTER:PORT/druid/v2/sql/statements/:queryId'
+```
+
+Note that the response has been truncated for brevity.
+
+Show the response
+
+```json
+[
+ {
+ "page": "Salo Toraut"
+ },
+ {
+ "page": "利用者:ワーナー成増/放送ウーマン賞"
+ },
+ {
+ "page": "Bailando 2015"
+ },
+ ...
+ ...
+ ...
+]
+```
+
+
+
+## Further reading
+
+* [Query from deep storage](../querying/query-from-deep-storage.md)
+* [Query from deep storage API reference](../api-reference/sql-api.md#query-from-deep-storage)
\ No newline at end of file
diff --git a/docs/tutorials/tutorial-query.md b/docs/tutorials/tutorial-query.md
index 4769968b23fe..1ad7e8e28bf9 100644
--- a/docs/tutorials/tutorial-query.md
+++ b/docs/tutorials/tutorial-query.md
@@ -124,21 +124,23 @@ performance issues. For more information, see [Native queries](../querying/query
![Explain query](../assets/tutorial-query-05.png "Explain query")
- > Another way to view the explain plan is by adding EXPLAIN PLAN FOR to the front of your query, as follows:
- >
- >```sql
- >EXPLAIN PLAN FOR
- >SELECT
- > "page",
- > "countryName",
- > COUNT(*) AS "Edits"
- >FROM "wikipedia"
- >WHERE "countryName" IS NOT NULL
- >GROUP BY 1, 2
- >ORDER BY "Edits" DESC
- >```
- >This is particularly useful when running queries
- from the command line or over HTTP.
+:::info
+ Another way to view the explain plan is by adding EXPLAIN PLAN FOR to the front of your query, as follows:
+
+```sql
+EXPLAIN PLAN FOR
+SELECT
+ "page",
+ "countryName",
+ COUNT(*) AS "Edits"
+FROM "wikipedia"
+WHERE "countryName" IS NOT NULL
+GROUP BY 1, 2
+ORDER BY "Edits" DESC
+```
+This is particularly useful when running queries
+from the command line or over HTTP.
+:::
9. Finally, click `...` and **Edit context** to see how you can add additional parameters controlling the execution of the query execution. In the field, enter query context options as JSON key-value pairs, as described in [Context flags](../querying/query-context.md).
diff --git a/docs/tutorials/tutorial-sketches-theta.md b/docs/tutorials/tutorial-sketches-theta.md
index 5b3f0fd132a2..a02ca9e23689 100644
--- a/docs/tutorials/tutorial-sketches-theta.md
+++ b/docs/tutorials/tutorial-sketches-theta.md
@@ -243,7 +243,9 @@ That is, `APPROX_COUNT_DISTINCT_DS_THETA` applies the following:
### Filtered metrics
Druid has the capability to use [filtered metrics](../querying/sql-aggregations.md). This means you can include a WHERE clause in the SELECT part of the query.
-> In the case of Theta sketches, the filter clause has to be inserted between the aggregator and the estimator.
+:::info
+ In the case of Theta sketches, the filter clause has to be inserted between the aggregator and the estimator.
+:::
As an example, query the total unique users that watched _Bridgerton:_
diff --git a/docs/tutorials/tutorial-unnest-arrays.md b/docs/tutorials/tutorial-unnest-arrays.md
index 954142f4fa69..1f8c530f8d01 100644
--- a/docs/tutorials/tutorial-unnest-arrays.md
+++ b/docs/tutorials/tutorial-unnest-arrays.md
@@ -4,6 +4,9 @@ sidebar_label: "Unnesting arrays"
title: "Unnest arrays within a column"
---
+import Tabs from '@theme/Tabs';
+import TabItem from '@theme/TabItem';
+
-> If you're looking for information about how to unnest `COMPLEX` columns, see [Nested columns](../querying/nested-columns.md).
+:::info
+ If you're looking for information about how to unnest `COMPLEX` columns, see [Nested columns](../querying/nested-columns.md).
+:::
-> The unnest datasource and UNNEST SQL function are [experimental](../development/experimental.md). Their API and behavior are subject
-> to change in future releases. It is not recommended to use this feature in production at this time.
+:::info
+ The unnest datasource and UNNEST SQL function are [experimental](../development/experimental.md). Their API and behavior are subject
+ to change in future releases. It is not recommended to use this feature in production at this time.
+:::
This tutorial demonstrates how to use the unnest datasource to unnest a column that has data stored in arrays. For example, if you have a column named `dim3` with values like `[a,b]` or `[c,d,f]`, the unnest datasource can output the data to a new column with individual rows that contain single values like `a` and `b`. When doing this, be mindful of the following:
-- Unnesting data can dramatically increase the total number of rows.
-- You cannot unnest an array within an array.
+- Unnesting data can dramatically increase the total number of rows.
+- You cannot unnest an array within an array.
-You can use the Druid console or API to unnest data. To start though, you may want to use the Druid console so that viewing the nested and unnested data is easier.
+You can use the Druid console or API to unnest data. To start though, you may want to use the Druid console so that viewing the nested and unnested data is easier.
-## Prerequisites
+## Prerequisites
You need a Druid cluster, such as the [quickstart](./index.md). The cluster does not need any existing datasources. You'll load a basic one as part of this tutorial.
@@ -51,9 +58,10 @@ The focus of this tutorial is on the nested array of values in `dim3`.
You can load this data by running a query for SQL-based ingestion or submitting a JSON-based ingestion spec. The example loads data into a table named `nested_data`:
-
+
+
+
-
```sql
REPLACE INTO nested_data OVERWRITE ALL
@@ -73,10 +81,12 @@ FROM TABLE(
'[{"name":"t","type":"string"},{"name":"dim1","type":"string"},{"name":"dim2","type":"string"},{"name":"dim3","type":"string"},{"name":"dim4","type":"string"},{"name":"dim5","type":"string"},{"name":"m1","type":"float"},{"name":"m2","type":"double"}]'
)
)
-PARTITIONED BY YEAR
+PARTITIONED BY YEAR
```
-
+
+
+
```json
{
@@ -135,7 +145,8 @@ PARTITIONED BY YEAR
}
}
```
-
+
+
## View the data
@@ -168,10 +179,10 @@ For more information about the syntax, see [UNNEST](../querying/sql.md#unnest).
The following query returns a column called `d3` from the table `nested_data`. `d3` contains the unnested values from the source column `dim3`:
```sql
-SELECT d3 FROM "nested_data", UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3)
+SELECT d3 FROM "nested_data", UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3)
```
-Notice the MV_TO_ARRAY helper function, which converts the multi-value records in `dim3` to arrays. It is required since `dim3` is a multi-value string dimension.
+Notice the MV_TO_ARRAY helper function, which converts the multi-value records in `dim3` to arrays. It is required since `dim3` is a multi-value string dimension.
If the column you are unnesting is not a string dimension, then you do not need to use the MV_TO_ARRAY helper function.
@@ -191,7 +202,7 @@ Another way to unnest a virtual column is to concatenate them with ARRAY_CONCAT:
SELECT dim4,dim5,d45 FROM nested_data, UNNEST(ARRAY_CONCAT(dim4,dim5)) AS example_table(d45)
```
-Decide which method to use based on what your goals are.
+Decide which method to use based on what your goals are.
### Unnest multiple source expressions
@@ -227,7 +238,7 @@ SELECT d3 FROM (SELECT dim1, dim2, dim3 FROM "nested_data"), UNNEST(MV_TO_ARRAY(
You can specify which rows to unnest by including a filter in your query. The following query:
* Filters the source expression based on `dim2`
-* Unnests the records in `dim3` into `d3`
+* Unnests the records in `dim3` into `d3`
* Returns the records for the unnested `d3` that have a `dim2` record that matches the filter
```sql
@@ -240,7 +251,7 @@ You can also filter the results of an UNNEST clause. The following example unnes
SELECT * FROM UNNEST(ARRAY[1,2,3]) AS example_table(d1) WHERE d1 IN ('1','2')
```
-This means that you can run a query like the following where Druid only return rows that meet the following conditions:
+This means that you can run a query like the following where Druid only return rows that meet the following conditions:
- The unnested values of `dim3` (aliased to `d3`) matches `IN ('b', 'd')`
- The value of `m1` is less than 2.
@@ -256,7 +267,7 @@ The query only returns a single row since only one row meets the conditions. You
The following query unnests `dim3` and then performs a GROUP BY on the output `d3`.
```sql
-SELECT d3 FROM nested_data, UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3) GROUP BY d3
+SELECT d3 FROM nested_data, UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3) GROUP BY d3
```
You can further transform your results by including clauses like `ORDER BY d3 DESC` or LIMIT.
@@ -267,7 +278,7 @@ The following section shows examples of how you can use the unnest datasource in
You can use a single unnest datasource to unnest multiple columns. Be careful when doing this though because it can lead to a very large number of new rows.
-### Scan query
+### Scan query
The following native Scan query returns the rows of the datasource and unnests the values in the `dim3` column by using the `unnest` datasource type:
diff --git a/examples/conf/druid/auto/_common/common.runtime.properties b/examples/conf/druid/auto/_common/common.runtime.properties
index b0adb0695cd7..4f4884d52644 100644
--- a/examples/conf/druid/auto/_common/common.runtime.properties
+++ b/examples/conf/druid/auto/_common/common.runtime.properties
@@ -119,7 +119,7 @@ druid.selectors.coordinator.serviceName=druid/coordinator
# Monitoring
#
-druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor"]
+druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor", "org.apache.druid.server.metrics.ServiceStatusMonitor"]
druid.emitter=noop
druid.emitter.logging.logLevel=info
diff --git a/examples/conf/druid/cluster/_common/common.runtime.properties b/examples/conf/druid/cluster/_common/common.runtime.properties
index b0adb0695cd7..4f4884d52644 100644
--- a/examples/conf/druid/cluster/_common/common.runtime.properties
+++ b/examples/conf/druid/cluster/_common/common.runtime.properties
@@ -119,7 +119,7 @@ druid.selectors.coordinator.serviceName=druid/coordinator
# Monitoring
#
-druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor"]
+druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor", "org.apache.druid.server.metrics.ServiceStatusMonitor"]
druid.emitter=noop
druid.emitter.logging.logLevel=info
diff --git a/examples/conf/druid/single-server/large/_common/common.runtime.properties b/examples/conf/druid/single-server/large/_common/common.runtime.properties
index b0adb0695cd7..4f4884d52644 100644
--- a/examples/conf/druid/single-server/large/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/large/_common/common.runtime.properties
@@ -119,7 +119,7 @@ druid.selectors.coordinator.serviceName=druid/coordinator
# Monitoring
#
-druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor"]
+druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor", "org.apache.druid.server.metrics.ServiceStatusMonitor"]
druid.emitter=noop
druid.emitter.logging.logLevel=info
diff --git a/examples/conf/druid/single-server/medium/_common/common.runtime.properties b/examples/conf/druid/single-server/medium/_common/common.runtime.properties
index b0adb0695cd7..4f4884d52644 100644
--- a/examples/conf/druid/single-server/medium/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/medium/_common/common.runtime.properties
@@ -119,7 +119,7 @@ druid.selectors.coordinator.serviceName=druid/coordinator
# Monitoring
#
-druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor"]
+druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor", "org.apache.druid.server.metrics.ServiceStatusMonitor"]
druid.emitter=noop
druid.emitter.logging.logLevel=info
diff --git a/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties b/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties
index b0adb0695cd7..4f4884d52644 100644
--- a/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/micro-quickstart/_common/common.runtime.properties
@@ -119,7 +119,7 @@ druid.selectors.coordinator.serviceName=druid/coordinator
# Monitoring
#
-druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor"]
+druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor", "org.apache.druid.server.metrics.ServiceStatusMonitor"]
druid.emitter=noop
druid.emitter.logging.logLevel=info
diff --git a/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties b/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties
index b0adb0695cd7..4f4884d52644 100644
--- a/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/nano-quickstart/_common/common.runtime.properties
@@ -119,7 +119,7 @@ druid.selectors.coordinator.serviceName=druid/coordinator
# Monitoring
#
-druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor"]
+druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor", "org.apache.druid.server.metrics.ServiceStatusMonitor"]
druid.emitter=noop
druid.emitter.logging.logLevel=info
diff --git a/examples/conf/druid/single-server/small/_common/common.runtime.properties b/examples/conf/druid/single-server/small/_common/common.runtime.properties
index b0adb0695cd7..4f4884d52644 100644
--- a/examples/conf/druid/single-server/small/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/small/_common/common.runtime.properties
@@ -119,7 +119,7 @@ druid.selectors.coordinator.serviceName=druid/coordinator
# Monitoring
#
-druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor"]
+druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor", "org.apache.druid.server.metrics.ServiceStatusMonitor"]
druid.emitter=noop
druid.emitter.logging.logLevel=info
diff --git a/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties b/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties
index b0adb0695cd7..4f4884d52644 100644
--- a/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties
+++ b/examples/conf/druid/single-server/xlarge/_common/common.runtime.properties
@@ -119,7 +119,7 @@ druid.selectors.coordinator.serviceName=druid/coordinator
# Monitoring
#
-druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor"]
+druid.monitoring.monitors=["org.apache.druid.java.util.metrics.JvmMonitor", "org.apache.druid.server.metrics.ServiceStatusMonitor"]
druid.emitter=noop
druid.emitter.logging.logLevel=info
diff --git a/examples/quickstart/jupyter-notebooks/Dockerfile b/examples/quickstart/jupyter-notebooks/Dockerfile
index 52e10175c21b..57970e2cc087 100644
--- a/examples/quickstart/jupyter-notebooks/Dockerfile
+++ b/examples/quickstart/jupyter-notebooks/Dockerfile
@@ -37,7 +37,8 @@ RUN pip install requests \
pip install seaborn \
pip install bokeh \
pip install kafka-python \
- pip install sortedcontainers
+ pip install sortedcontainers \
+ pip install tqdm
# Install druidapi client from apache/druid
# Local install requires sudo privileges
@@ -46,12 +47,6 @@ ADD druidapi /home/jovyan/druidapi
WORKDIR /home/jovyan/druidapi
RUN pip install .
-
-
-# WIP -- install DruidDataDriver as a package
-# Import data generator and configuration file
-# Change permissions to allow import (requires sudo privileges)
-
# The Jupyter notebooks themselves are mounted into the image's /home/jovyan/notebooks
# path when running this image.
RUN mkdir -p /home/jovyan/notebooks
@@ -59,8 +54,3 @@ RUN mkdir -p /home/jovyan/notebooks
WORKDIR /home/jovyan/notebooks
USER jovyan
-
-
-# Add location of the data generator to PYTHONPATH
-ENV PYTHONPATH "${PYTHONPATH}:/home/jovyan/notebooks/02-ingestion"
-
diff --git a/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose-local.yaml b/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose-local.yaml
index 2525fc485cd3..aecb24bdaf52 100644
--- a/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose-local.yaml
+++ b/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose-local.yaml
@@ -27,6 +27,7 @@ volumes:
coordinator_var: {}
router_var: {}
druid_shared: {}
+ datagen_data: {}
services:
@@ -175,3 +176,12 @@ services:
- "${JUPYTER_PORT:-8889}:8888"
volumes:
- ../notebooks:/home/jovyan/notebooks
+
+ datagen:
+ image: imply/datagen:latest
+ container_name: datagen
+ profiles: ["jupyter", "kafka-jupyter", "druid-jupyter", "all-services"]
+ ports:
+ - "${DATAGEN_PORT:-9999}:9999"
+ volumes:
+ - datagen_data:/files
diff --git a/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose.yaml b/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose.yaml
index b0b2d206e5dd..910646bf241c 100644
--- a/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose.yaml
+++ b/examples/quickstart/jupyter-notebooks/docker-jupyter/docker-compose.yaml
@@ -27,6 +27,7 @@ volumes:
coordinator_var: {}
router_var: {}
druid_shared: {}
+ datagen_data: {}
services:
@@ -173,3 +174,12 @@ services:
- "${JUPYTER_PORT:-8889}:8888"
volumes:
- ../notebooks:/home/jovyan/notebooks
+
+ datagen:
+ image: imply/datagen:latest
+ container_name: datagen
+ profiles: ["jupyter", "kafka-jupyter", "druid-jupyter", "all-services"]
+ ports:
+ - "${DATAGEN_PORT:-9999}:9999"
+ volumes:
+ - datagen_data:/files
diff --git a/examples/quickstart/jupyter-notebooks/docker-jupyter/environment b/examples/quickstart/jupyter-notebooks/docker-jupyter/environment
index c63a5c0e8868..4b548f8d4b3c 100644
--- a/examples/quickstart/jupyter-notebooks/docker-jupyter/environment
+++ b/examples/quickstart/jupyter-notebooks/docker-jupyter/environment
@@ -39,8 +39,8 @@ druid_metadata_storage_connector_password=FoolishPassword
druid_coordinator_balancer_strategy=cachingCost
-druid_indexer_runner_javaOptsArray=["-server", "-Xmx1g", "-Xms1g", "-XX:MaxDirectMemorySize=3g", "-Duser.timezone=UTC", "-Dfile.encoding=UTF-8", "-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager"]
-druid_indexer_fork_property_druid_processing_buffer_sizeBytes=256MiB
+druid_indexer_runner_javaOptsArray=["-server", "-Xmx256m", "-Xms256m", "-XX:MaxDirectMemorySize=324m", "-Duser.timezone=UTC", "-Dfile.encoding=UTF-8", "-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager"]
+druid_indexer_fork_property_druid_processing_buffer_sizeBytes=64MiB
diff --git a/examples/quickstart/jupyter-notebooks/druidapi/druidapi/display.py b/examples/quickstart/jupyter-notebooks/druidapi/druidapi/display.py
index 5b4368325ce4..e51bff70ce45 100644
--- a/examples/quickstart/jupyter-notebooks/druidapi/druidapi/display.py
+++ b/examples/quickstart/jupyter-notebooks/druidapi/druidapi/display.py
@@ -14,6 +14,7 @@
# limitations under the License.
from druidapi import consts
+import time
class DisplayClient:
'''
@@ -144,3 +145,36 @@ def schemas(self):
def tables(self, schema=consts.DRUID_SCHEMA):
self._druid.sql._tables_query(schema).show(display=self)
+
+ def run_task(self, query):
+ '''
+ Run an MSQ task while displaying progress in the cell output.
+ :param query: INSERT/REPLACE statement to run
+ :return: None
+ '''
+ from tqdm import tqdm
+
+ task = self._druid.sql.task(query)
+ with tqdm(total=100.0) as pbar:
+ previous_progress = 0.0
+ while True:
+ reports=task.reports_no_wait()
+ # check if progress metric is available and display it
+ if 'multiStageQuery' in reports.keys():
+ if 'payload' in reports['multiStageQuery'].keys():
+ if 'counters' in reports['multiStageQuery']['payload'].keys():
+ if ('0' in reports['multiStageQuery']['payload']['counters'].keys() ) and \
+ ('0' in reports['multiStageQuery']['payload']['counters']['0'].keys()):
+ if 'progressDigest' in reports['multiStageQuery']['payload']['counters']['0']['0']['sortProgress'].keys():
+ current_progress = reports['multiStageQuery']['payload']['counters']['0']['0']['sortProgress']['progressDigest']*100.0
+ pbar.update( current_progress - previous_progress ) # update requires a relative value
+ previous_progress = current_progress
+ # present status if available
+ if 'status' in reports['multiStageQuery']['payload'].keys():
+ pbar.set_description(f"Loading data, status:[{reports['multiStageQuery']['payload']['status']['status']}]")
+ # stop when job is done
+ if reports['multiStageQuery']['payload']['status']['status'] in ['SUCCESS', 'FAILED']:
+ break;
+ else:
+ pbar.set_description('Initializing...')
+ time.sleep(1)
diff --git a/examples/quickstart/jupyter-notebooks/druidapi/druidapi/sql.py b/examples/quickstart/jupyter-notebooks/druidapi/druidapi/sql.py
index f38d0a994afd..46bad764dae9 100644
--- a/examples/quickstart/jupyter-notebooks/druidapi/druidapi/sql.py
+++ b/examples/quickstart/jupyter-notebooks/druidapi/druidapi/sql.py
@@ -585,6 +585,9 @@ def reports(self) -> dict:
self._reports = self._tasks().task_reports(self._id)
return self._reports
+ def reports_no_wait(self) -> dict:
+ return self._tasks().task_reports(self._id, require_ok=False)
+
@property
def results(self):
if not self._results:
@@ -844,7 +847,7 @@ def function_parameters(self, table_name):
'''
return self._function_args_query(table_name).rows
- def wait_until_ready(self, table_name):
+ def wait_until_ready(self, table_name, verify_load_status=True):
'''
Waits for a datasource to be loaded in the cluster, and to become available to SQL.
@@ -852,8 +855,12 @@ def wait_until_ready(self, table_name):
----------
table_name str
The name of a datasource in the 'druid' schema.
+ verify_load_status
+ If true, checks whether all published segments are loaded before testing query.
+ If false, tries the test query before checking whether all published segments are loaded.
'''
- self.druid_client.datasources.wait_until_ready(table_name)
+ if verify_load_status:
+ self.druid_client.datasources.wait_until_ready(table_name)
while True:
try:
self.sql('SELECT 1 FROM "{}" LIMIT 1'.format(table_name));
diff --git a/examples/quickstart/jupyter-notebooks/druidapi/druidapi/tasks.py b/examples/quickstart/jupyter-notebooks/druidapi/druidapi/tasks.py
index 0c428eda126f..b5652ba6aba1 100644
--- a/examples/quickstart/jupyter-notebooks/druidapi/druidapi/tasks.py
+++ b/examples/quickstart/jupyter-notebooks/druidapi/druidapi/tasks.py
@@ -14,6 +14,7 @@
# limitations under the License.
from druidapi.consts import OVERLORD_BASE
+import requests
REQ_TASKS = OVERLORD_BASE + '/tasks'
REQ_POST_TASK = OVERLORD_BASE + '/task'
@@ -112,7 +113,7 @@ def task_status(self, task_id) -> dict:
'''
return self.client.get_json(REQ_TASK_STATUS, args=[task_id])
- def task_reports(self, task_id) -> dict:
+ def task_reports(self, task_id, require_ok = True) -> dict:
'''
Retrieves the completion report for a completed task.
@@ -129,7 +130,19 @@ def task_reports(self, task_id) -> dict:
---------
`GET /druid/indexer/v1/task/{taskId}/reports`
'''
- return self.client.get_json(REQ_TASK_REPORTS, args=[task_id])
+ if require_ok:
+ return self.client.get_json(REQ_TASK_REPORTS, args=[task_id])
+ else:
+ resp = self.client.get(REQ_TASK_REPORTS, args=[task_id], require_ok=require_ok)
+ if resp.status_code == requests.codes.ok:
+ try:
+ result = resp.json()
+ except Exception as ex:
+ result = {"message":"Payload could not be converted to json.", "payload":f"{resp.content}", "exception":f"{ex}"}
+ return result
+ else:
+ return {"message":f"Request return code:{resp.status_code}"}
+
def submit_task(self, payload):
'''
diff --git a/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/00-START-HERE.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/00-START-HERE.ipynb
index 0f89633c22fd..d813dacae273 100644
--- a/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/00-START-HERE.ipynb
+++ b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/00-START-HERE.ipynb
@@ -91,7 +91,8 @@
" basics related to the Druid REST API and several endpoints.\n",
"- [Introduction to the Druid Python API](01-druidapi-package-intro.ipynb) walks you through some of the\n",
" basics related to the Druid API using the Python wrapper API.\n",
- "- [Learn the basics of Druid SQL](../03-query/00-using-sql-with-druidapi.ipynb) introduces you to the unique aspects of Druid SQL with the primary focus on the SELECT statement. \n",
+ "- [Learn the basics of Druid SQL](../03-query/00-using-sql-with-druidapi.ipynb) introduces you to the unique aspects of Druid SQL with the primary focus on the SELECT statement.\n",
+ "- [Learn to use the Data Generator](./02-datagen-intro.ipynb) gets you started with streaming and batch file data generation for testing of any data schema.\n",
"- [Ingest and query data from Apache Kafka](../02-ingestion/01-streaming-from-kafka.ipynb) walks you through ingesting an event stream from Kafka."
]
},
diff --git a/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb
index 6c943c4286be..88b79fd8d927 100644
--- a/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb
+++ b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/01-druidapi-package-intro.ipynb
@@ -445,7 +445,7 @@
"metadata": {},
"outputs": [],
"source": [
- "sql_client.run_task(sql)"
+ "display.run_task(sql)"
]
},
{
@@ -473,7 +473,7 @@
"id": "11d9c95a",
"metadata": {},
"source": [
- "`describe_table()` lists the columns in a table."
+ "`display.table()` lists the columns in a table."
]
},
{
diff --git a/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/02-datagen-intro.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/02-datagen-intro.ipynb
new file mode 100644
index 000000000000..e3b3df2994ea
--- /dev/null
+++ b/examples/quickstart/jupyter-notebooks/notebooks/01-introduction/02-datagen-intro.ipynb
@@ -0,0 +1,642 @@
+{
+ "cells": [
+ {
+ "cell_type": "markdown",
+ "id": "9e07b3f5-d919-4179-91a1-0f6b66c42757",
+ "metadata": {},
+ "source": [
+ "# Data Generator Server\n",
+ "\n",
+ "The default Docker Compose deployment includes a data generation service created from the published Docker image at `imply/datagen:latest`. \n",
+ "This image is built by the project https://github.com/implydata/druid-datagenerator. \n",
+ "\n",
+ "This notebook shows you how to use the data generation service included in the Docker Compose deployment. It explains how to use predefined data generator configurations as well as how to build a custom data generator. You will also learn how to create sample data files for batch ingestion and how to generate live streaming data for streaming ingestion.\n",
+ "\n",
+ "## Table of contents\n",
+ "\n",
+ "* [Initialization](#Initialization)\n",
+ "* [List available configurations](#List-available-configurations)\n",
+ "* [Generate a data file for backfilling history](#Generate-a-data-file-for-backfilling-history)\n",
+ "* [Batch ingestion of generated files](#Batch-ingestion-of-generated-files)\n",
+ "* [Generate custom data](#Generate-custom-data)\n",
+ "* [Stream generated data](#Stream-generated-data)\n",
+ "* [Ingest data from a stream](#Ingest-data-from-a-stream)\n",
+ "* [Cleanup](#Cleanup)\n",
+ "\n",
+ "\n",
+ "## Initialization\n",
+ "\n",
+ "To interact with the data generation service, use the REST client provided in the [`druidapi` Python package](https://druid.apache.org/docs/latest/tutorials/tutorial-jupyter-index.html#python-api-for-druid)."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "f84766c7-c6a5-4496-91a3-abdb8ddd2375",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "import druidapi\n",
+ "import os\n",
+ "import time\n",
+ "\n",
+ "# Datagen client \n",
+ "datagen = druidapi.rest.DruidRestClient(\"http://datagen:9999\")\n",
+ "\n",
+ "if (os.environ['DRUID_HOST'] == None):\n",
+ " druid_host=f\"http://router:8888\"\n",
+ "else:\n",
+ " druid_host=f\"http://{os.environ['DRUID_HOST']}:8888\"\n",
+ "\n",
+ "# Druid client\n",
+ "druid = druidapi.jupyter_client(druid_host)\n",
+ "\n",
+ "\n",
+ "\n",
+ "# these imports and constants are used by multiple cells\n",
+ "from datetime import datetime, timedelta\n",
+ "import json\n",
+ "\n",
+ "headers = {\n",
+ " 'Content-Type': 'application/json'\n",
+ "}"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "c54af617-0998-4010-90c3-9b5a38a09a5f",
+ "metadata": {},
+ "source": [
+ "### List available configurations\n",
+ "Use the `/list` API endpoint to get the data generator's available configuration values with predefined data generator schemas."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "1ba6a80a-c49b-4abf-943b-9dad82f2ae13",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "display(datagen.get(f\"/list\", require_ok=False).json())"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "ae88a3b7-60da-405d-bcf4-fb4affcfe973",
+ "metadata": {},
+ "source": [
+ "### Generate a data file for backfilling history\n",
+ "When generating a file for backfill purposes, you can select the start time and the duration of the simulation.\n",
+ "\n",
+ "Configure the data generator request as follows:\n",
+ "* `name`: an arbitrary name you assign to the job. Refer to the job name to get the job status or to stop the job.\n",
+ "* `target.type`: \"file\" to generate a data file\n",
+ "* `target.path`: identifies the name of the file to generate. The data generator ignores any path specified and creates the file in the current working directory.\n",
+ "* `time_type`,`time`: The data generator simulates the time range you specify with a start timestamp in the `time_type` property and a duration in the `time` property. To specify `time`, use the `h` suffix for hours, `m` for minutes, and `s` for seconds.\n",
+ "- `concurrency` indicates the maximum number of entities used concurrently to generate events. Each entity is a separate state machine that simulates things like user sessions, IoT devices, or other concurrent sources of event data.\n",
+ "\n",
+ "The following example uses the `clickstream.json` predefined configuration to generate data into a file called `clicks.json`. The data generator starts the sample data at one hour prior to the current time and simulates events for a duration of one hour. Since it is simulated, it does this in just a few seconds."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "811ff58f-75af-4092-a08d-5e07a51592ff",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "# Configure the start time to one hour prior to the current time. \n",
+ "startDateTime = (datetime.now() - timedelta(hours = 1)).strftime('%Y-%m-%dT%H:%M:%S.001')\n",
+ "print(f\"Starting to generate history at {startDateTime}.\")\n",
+ "\n",
+ "# Give the datagen job a name for use in subsequent API calls\n",
+ "job_name=\"gen_clickstream1\"\n",
+ "\n",
+ "# Generate a data file on the datagen server\n",
+ "datagen_request = {\n",
+ " \"name\": job_name,\n",
+ " \"target\": { \"type\": \"file\", \"path\":\"clicks.json\"},\n",
+ " \"config_file\": \"clickstream/clickstream.json\", \n",
+ " \"time_type\": startDateTime,\n",
+ " \"time\": \"1h\",\n",
+ " \"concurrency\":100\n",
+ "}\n",
+ "response = datagen.post(\"/start\", json.dumps(datagen_request), headers=headers, require_ok=False)\n",
+ "response.json()"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "d407d1d9-3f01-4128-a014-6a5f371c25a5",
+ "metadata": {},
+ "source": [
+ "#### Display jobs\n",
+ "Use the `/jobs` API endpoint to get the current jobs and job statuses."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "3de698c5-bcf4-40c7-b295-728fb54d1f0a",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "display(datagen.get(f\"/jobs\").json())"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "972ebed0-34a1-4ad2-909d-69b8b27c3046",
+ "metadata": {},
+ "source": [
+ "#### Get status of a job\n",
+ "Use the `/status/JOB_NAME` API endpoint to get the current jobs and their status."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "debce4f8-9c16-476c-9593-21ec984985d2",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "display(datagen.get(f\"/status/{job_name}\", require_ok=False).json())"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "ef818d78-6aa6-4d38-8a43-83416aede96f",
+ "metadata": {},
+ "source": [
+ "#### Stop a job\n",
+ "Use the `/stop/JOB_NAME` API endpoint to stop a job."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "7631b8b8-d3d6-4803-9162-587f440d2ef2",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "display(datagen.post(f\"/stop/{job_name}\", '').json())"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "0a8dc7d3-64e5-41e3-8c28-c5f19c0536f5",
+ "metadata": {},
+ "source": [
+ "#### List files created on datagen server\n",
+ "Use the `/files` API endpoint to list files available on the server."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "06ee36bd-2d2b-4904-9987-10636cf52aac",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "display(datagen.get(f\"/files\", '').json())"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "83ef9edb-98e2-45b4-88e8-578703faedc1",
+ "metadata": {},
+ "source": [
+ "### Batch ingestion of generated files\n",
+ "Use a [Druid HTTP input source](https://druid.apache.org/docs/latest/ingestion/native-batch-input-sources.html#http-input-source) in the [EXTERN function](https://druid.apache.org/docs/latest/multi-stage-query/reference.html#extern-function) of a [SQL-based ingestion](https://druid.apache.org/docs/latest/multi-stage-query/index.html) to load generated files.\n",
+ "You can access files by name from within Druid using the URI `http://datagen:9999/file/FILE_NAME`. Alternatively, if you run Druid outside of Docker but on the same machine, access the file with `http://localhost:9999/file/FILE_NAME`.\n",
+ "The following example assumes that both Druid and the data generator server are running in Docker Compose."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "0d72b015-f8ec-4713-b6f2-fe7a15afff59",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "REPLACE INTO \"clicks\" OVERWRITE ALL\n",
+ "WITH \"ext\" AS (SELECT *\n",
+ "FROM TABLE(\n",
+ " EXTERN(\n",
+ " '{\"type\":\"http\",\"uris\":[\"http://datagen:9999/file/clicks.json\"]}',\n",
+ " '{\"type\":\"json\"}'\n",
+ " )\n",
+ ") EXTEND (\"time\" VARCHAR, \"user_id\" VARCHAR, \"event_type\" VARCHAR, \"client_ip\" VARCHAR, \"client_device\" VARCHAR, \"client_lang\" VARCHAR, \"client_country\" VARCHAR, \"referrer\" VARCHAR, \"keyword\" VARCHAR, \"product\" VARCHAR))\n",
+ "SELECT\n",
+ " TIME_PARSE(\"time\") AS \"__time\",\n",
+ " \"user_id\",\n",
+ " \"event_type\",\n",
+ " \"client_ip\",\n",
+ " \"client_device\",\n",
+ " \"client_lang\",\n",
+ " \"client_country\",\n",
+ " \"referrer\",\n",
+ " \"keyword\",\n",
+ " \"product\"\n",
+ "FROM \"ext\"\n",
+ "PARTITIONED BY DAY\n",
+ "''' \n",
+ "\n",
+ "druid.display.run_task(sql)\n",
+ "print(\"Waiting for segment avaialbility ...\")\n",
+ "druid.sql.wait_until_ready('clicks')\n",
+ "print(\"Data is available for query.\")"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "b0997b38-02c2-483e-bd15-439c4bf0097a",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT \"event_type\", \"user_id\", count( DISTINCT \"client_ip\") ip_count\n",
+ "FROM \"clicks\"\n",
+ "GROUP BY 1,2\n",
+ "ORDER BY 3 DESC\n",
+ "LIMIT 10\n",
+ "'''\n",
+ "druid.display.sql(sql)"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "66ec013f-28e4-4d5a-94a6-06e0ed537b4e",
+ "metadata": {},
+ "source": [
+ "## Generate custom data\n",
+ "\n",
+ "You can find the full set of configuration options for the data generator in the [README](https://github.com/implydata/druid-datagenerator#data-generator-configuration).\n",
+ "\n",
+ "This section demonstrates a simple custom configuration as an example. Notice that the emitter defined the schema as a list of dimensions, each dimension specifies how its values are generated: "
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "d6451310-b7dd-4b39-a23b-7b735b152d6c",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "gen_config = {\n",
+ " \"emitters\": [\n",
+ " {\n",
+ " \"name\": \"simple_record\",\n",
+ " \"dimensions\": [\n",
+ " {\n",
+ " \"type\": \"string\",\n",
+ " \"name\": \"random_string_column\",\n",
+ " \"length_distribution\": {\n",
+ " \"type\": \"constant\",\n",
+ " \"value\": 13\n",
+ " },\n",
+ " \"cardinality\": 0,\n",
+ " \"chars\": \"#.abcdefghijklmnopqrstuvwxyz\"\n",
+ " },\n",
+ " {\n",
+ " \"type\": \"int\",\n",
+ " \"name\": \"distributed_number\",\n",
+ " \"distribution\": {\n",
+ " \"type\": \"uniform\",\n",
+ " \"min\": 0,\n",
+ " \"max\": 1000\n",
+ " },\n",
+ " \"cardinality\": 10,\n",
+ " \"cardinality_distribution\": {\n",
+ " \"type\": \"exponential\",\n",
+ " \"mean\": 5\n",
+ " }\n",
+ " }\n",
+ " ]\n",
+ " }\n",
+ " ],\n",
+ " \"interarrival\": {\n",
+ " \"type\": \"constant\",\n",
+ " \"value\": 1\n",
+ " },\n",
+ " \"states\": [\n",
+ " {\n",
+ " \"name\": \"state_1\",\n",
+ " \"emitter\": \"simple_record\",\n",
+ " \"delay\": {\n",
+ " \"type\": \"constant\",\n",
+ " \"value\": 1\n",
+ " },\n",
+ " \"transitions\": [\n",
+ " {\n",
+ " \"next\": \"state_1\",\n",
+ " \"probability\": 1.0\n",
+ " }\n",
+ " ]\n",
+ " }\n",
+ " ]\n",
+ "}\n",
+ "\n",
+ "target = { \"type\":\"file\", \"path\":\"sample_data.json\"}"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "89a22645-aea5-4c15-b81a-959b27df731f",
+ "metadata": {},
+ "source": [
+ "This example uses the `config` attribute of the request to configure a new custom data generator instead of using a predefined `config_file`."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "e5e5c535-3474-42b4-9772-14279e712f3d",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "# generate 1 hour of simulated time using custom configuration\n",
+ "datagen_request = {\n",
+ " \"name\": \"sample_custom\",\n",
+ " \"target\": target,\n",
+ " \"config\": gen_config, \n",
+ " \"time\": \"1h\",\n",
+ " \"concurrency\":10,\n",
+ " \"time_type\": \"SIM\"\n",
+ "}\n",
+ "response = datagen.post(\"/start\", json.dumps(datagen_request), headers=headers, require_ok=False)\n",
+ "response.json()"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "952386f7-8181-4325-972b-5f30dc12cf21",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "display(datagen.get(f\"/jobs\", require_ok=False).json())"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "470b3a2a-4fd9-45a2-9221-497d906f62a9",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "# display the first 1k characters of the generated data file\n",
+ "display( datagen.get(f\"/file/sample_data.json\").content[:1024])"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "350faea6-55b0-4386-830c-5160ae495012",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "datagen.post(f\"/stop/sample_custom\",'')"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "77bff054-0f16-4fd5-8ade-2d44b30d0cf2",
+ "metadata": {},
+ "source": [
+ "## Stream generated data\n",
+ "\n",
+ "The data generator works exactly the same whether it is writing data to a file or publishing messages into a stream. You only need to change the target configuration.\n",
+ "\n",
+ "To use the Kafka container running on Docker Compose, use the host name `kafka:9092`. This tutorial uses the KAFKA_HOST environment variable from Docker Compose to specify the Kafka host. "
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "9959b7c3-6223-479d-b0c2-115a1c555090",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "if (os.environ['KAFKA_HOST'] == None):\n",
+ " kafka_host=f\"kafka:9092\"\n",
+ "else:\n",
+ " kafka_host=f\"{os.environ['KAFKA_HOST']}:9092\""
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "497abc18-6538-4536-a17f-fe10c4367611",
+ "metadata": {},
+ "source": [
+ "The simplest `target` object for Kafka and, similarly, Confluent is:"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "686a74ab-e2dd-458e-9e93-10291064e9db",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "target = {\n",
+ " \"type\":\"kafka\",\n",
+ " \"endpoint\": kafka_host,\n",
+ " \"topic\": \"custom_data\"\n",
+ "}\n",
+ "\n",
+ "# Generate 1 hour of real time using custom configuration, this means that this stream will run for an hour if not stopped\n",
+ "datagen_request = {\n",
+ " \"name\": \"sample_custom\",\n",
+ " \"target\": target,\n",
+ " \"config\": gen_config, \n",
+ " \"time\": \"1h\",\n",
+ " \"concurrency\":10,\n",
+ " \"time_type\": \"REAL\"\n",
+ "}\n",
+ "response = datagen.post(\"/start\", json.dumps(datagen_request), headers=headers, require_ok=False)\n",
+ "response.json()"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "ec17d0c7-a3ab-4f37-bbf0-cc02bff44cf1",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "time.sleep(1) # avoid race condition of async job start\n",
+ "display(datagen.get(f\"/jobs\", require_ok=False).json())"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "84d7b706-9040-4a69-a956-1b1bbb037c32",
+ "metadata": {},
+ "source": [
+ "### Ingest data from a stream \n",
+ "This example shows how to start a streaming ingestion supervisor in Apache Druid to consume your custom data:"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "51912409-e4e7-48d1-b3a5-b269622b4e56",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "ingestion_spec ={\n",
+ " \"type\": \"kafka\",\n",
+ " \"spec\": {\n",
+ " \"ioConfig\": {\n",
+ " \"type\": \"kafka\",\n",
+ " \"consumerProperties\": {\n",
+ " \"bootstrap.servers\": \"kafka:9092\"\n",
+ " },\n",
+ " \"topic\": \"custom_data\",\n",
+ " \"inputFormat\": {\n",
+ " \"type\": \"json\"\n",
+ " },\n",
+ " \"useEarliestOffset\": True\n",
+ " },\n",
+ " \"tuningConfig\": {\n",
+ " \"type\": \"kafka\",\n",
+ " \"maxRowsInMemory\": 100000,\n",
+ " \"resetOffsetAutomatically\": False\n",
+ " },\n",
+ " \"dataSchema\": {\n",
+ " \"dataSource\": \"custom_data\",\n",
+ " \"timestampSpec\": {\n",
+ " \"column\": \"time\",\n",
+ " \"format\": \"iso\"\n",
+ " },\n",
+ " \"dimensionsSpec\": {\n",
+ " \"dimensions\": [\n",
+ " \"random_string_column\",\n",
+ " {\n",
+ " \"type\": \"long\",\n",
+ " \"name\": \"distributed_number\"\n",
+ " }\n",
+ " ]\n",
+ " },\n",
+ " \"granularitySpec\": {\n",
+ " \"queryGranularity\": \"none\",\n",
+ " \"rollup\": False,\n",
+ " \"segmentGranularity\": \"hour\"\n",
+ " }\n",
+ " }\n",
+ " }\n",
+ "}\n",
+ "\n",
+ "headers = {\n",
+ " 'Content-Type': 'application/json'\n",
+ "}\n",
+ "\n",
+ "druid.rest.post(\"/druid/indexer/v1/supervisor\", json.dumps(ingestion_spec), headers=headers)"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "dddfb1cc-f863-4bf4-8c5a-b261b0b9c2f0",
+ "metadata": {},
+ "source": [
+ "Query the data on the stream, but first wait for its availability. It takes a bit of time for the streaming tasks to start, but once they are consuming you can see data very close to real time: Run the following cell multiple times to see how the data is changing:"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "7e1284ed-5c49-4f37-81f7-c3b720473158",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "druid.sql.wait_until_ready('custom_data', verify_load_status=False)\n",
+ "druid.display.sql('''\n",
+ "SELECT SUM(distributed_number) sum_randoms, count(*) total_count\n",
+ "FROM custom_data\n",
+ "''')"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "4486e430-0776-46ad-8a8b-4f0354f17bfb",
+ "metadata": {},
+ "source": [
+ "### Cleanup\n",
+ "\n",
+ "Stop the streaming ingestion and the streaming producer:"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "38943a92-dc23-41cf-91a4-1b68d2178033",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "print(f\"Stop streaming generator: [{datagen.post('/stop/sample_custom','',require_ok=False)}]\")\n",
+ "print(f'Reset offsets for streaming ingestion: [{druid.rest.post(\"/druid/indexer/v1/supervisor/custom_data/reset\",\"\", require_ok=False)}]')\n",
+ "print(f'Stop streaming ingestion: [{druid.rest.post(\"/druid/indexer/v1/supervisor/custom_data/terminate\",\"\", require_ok=False)}]')"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "0cf53bdc-de7f-425d-84b1-68d0cef420d8",
+ "metadata": {},
+ "source": [
+ "Wait for streaming ingestion to complete and then remove the custom data table:"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "87341e7c-f7ab-488c-9913-091f712534cb",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "print(f\"Drop datasource: [{druid.datasources.drop('custom_data')}]\")"
+ ]
+ }
+ ],
+ "metadata": {
+ "kernelspec": {
+ "display_name": "Python 3 (ipykernel)",
+ "language": "python",
+ "name": "python3"
+ },
+ "language_info": {
+ "codemirror_mode": {
+ "name": "ipython",
+ "version": 3
+ },
+ "file_extension": ".py",
+ "mimetype": "text/x-python",
+ "name": "python",
+ "nbconvert_exporter": "python",
+ "pygments_lexer": "ipython3",
+ "version": "3.11.4"
+ }
+ },
+ "nbformat": 4,
+ "nbformat_minor": 5
+}
diff --git a/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/01-streaming-from-kafka.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/01-streaming-from-kafka.ipynb
index 6a62dbbd1971..fc36b4b19ad2 100644
--- a/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/01-streaming-from-kafka.ipynb
+++ b/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/01-streaming-from-kafka.ipynb
@@ -4,7 +4,7 @@
"cell_type": "markdown",
"metadata": {},
"source": [
- "Ingest and query data from Apache Kafka\n",
+ "# Ingest and query data from Apache Kafka\n",
"\n",
"\n",
+ "\n",
+ "This tutorial demonstrates how to work with [nested columns](https://druid.apache.org/docs/latest/querying/nested-columns.html) in Apache Druid.\n",
+ "\n",
+ "Druid stores nested data structures in `COMPLEX` columns. In this tutorial you perform the following tasks:\n",
+ "\n",
+ "- Ingest nested JSON data using SQL-based ingestion.\n",
+ "- Transform nested data during ingestion using SQL JSON functions.\n",
+ "- Perform queries to display, filter, and aggregate nested data.\n",
+ "- Use helper operators to examine nested data and plan your queries.\n",
+ "\n",
+ "Druid supports directly ingesting nested data with the following formats: JSON, Parquet, Avro, ORC, Protobuf."
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "## Table of contents\n",
+ "\n",
+ "- [Prerequisites](#Prerequisites)\n",
+ "- [Initialization](#Initialization)\n",
+ "- [Ingest nested data](#Ingest-nested-data)\n",
+ "- [Transform nested data](#Transform-nested-data)\n",
+ "- [Query nested data](#Query-nested-data)\n",
+ "- [Group, filter, and aggregate nested data](#Group-filter-and-aggregate-nested-data)\n",
+ "- [Use helper operators](#Use-helper-operators)\n",
+ "- [Learn more](#Learn-more)"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "## Prerequisites\n",
+ "\n",
+ "This tutorial works with Druid 25.0.0 or later.\n",
+ "\n",
+ "### Run with Docker\n",
+ "\n",
+ "Launch this tutorial and all prerequisites using the `druid-jupyter` profile of the Docker Compose file for Jupyter-based Druid tutorials. For more information, see [Docker for Jupyter Notebook tutorials](https://druid.apache.org/docs/latest/tutorials/tutorial-jupyter-docker.html).\n",
+ "\n",
+ "### Run without Docker\n",
+ "\n",
+ "If you do not use the Docker Compose environment, you need the following:\n",
+ "\n",
+ "* A running Apache Druid instance, with a `DRUID_HOST` local environment variable containing the server name of your Druid router.\n",
+ "* [druidapi](https://github.com/apache/druid/blob/master/examples/quickstart/jupyter-notebooks/druidapi/README.md), a Python client for Apache Druid. Follow the instructions in the Install section of the README file."
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "## Initialization"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "Run the next cell to set up the Druid Python client's connection to Apache Druid.\n",
+ "\n",
+ "If successful, the Druid version number will be shown in the output."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "import druidapi\n",
+ "import os\n",
+ "\n",
+ "if 'DRUID_HOST' not in os.environ.keys():\n",
+ " druid_host=f\"http://localhost:8888\"\n",
+ "else:\n",
+ " druid_host=f\"http://{os.environ['DRUID_HOST']}:8888\"\n",
+ " \n",
+ "print(f\"Opening a connection to {druid_host}.\")\n",
+ "druid = druidapi.jupyter_client(druid_host)\n",
+ "\n",
+ "display = druid.display\n",
+ "sql_client = druid.sql\n",
+ "status_client = druid.status\n",
+ "\n",
+ "status_client.version"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "## Ingest nested data"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "Run the following cell to ingest sample clickstream data from the [Koalas to the Max](https://www.koalastothemax.com/) game."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "INSERT INTO example_koalas_nesteddata\n",
+ " WITH \"source\" AS\n",
+ " (SELECT * FROM TABLE(EXTERN('{\"type\":\"http\",\"uris\":[\"https://static.imply.io/example-data/kttm-nested-v2/kttm-nested-v2-2019-08-25.json.gz\"]}',\n",
+ " '{\"type\":\"json\"}','[{\"name\":\"timestamp\",\"type\":\"string\"},{\"name\":\"client_ip\",\"type\":\"string\"},\n",
+ " {\"name\":\"session\",\"type\":\"string\"},{\"name\":\"session_length\",\"type\":\"string\"},{\"name\":\"event\",\"type\":\"COMPLEX\"},\n",
+ " {\"name\":\"agent\",\"type\":\"COMPLEX\"},{\"name\":\"geo_ip\",\"type\":\"COMPLEX\"}]')))\n",
+ " SELECT TIME_PARSE(\"timestamp\") AS \"__time\",\n",
+ " \"client_ip\", \n",
+ " \"session\", \n",
+ " \"session_length\", \n",
+ " \"event\", \n",
+ " \"agent\", \n",
+ " \"geo_ip\"\n",
+ " FROM \"source\"\n",
+ " PARTITIONED BY DAY\n",
+ "'''\n",
+ "\n",
+ "sql_client.run_task(sql)\n",
+ "sql_client.wait_until_ready(\"example_koalas_nesteddata\")\n",
+ "display.table(\"example_koalas_nesteddata\")"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "Druid reports task completion as soon as ingestion is done. However, it takes a while for Druid to load the resulting segments.\n",
+ "\n",
+ "Wait for the table detail to display, then run the following cell to query the data and return selected columns from 3 rows. Note the nested structure of the `event`, `agent`, and `geo_ip` columns."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT session, event, agent, geo_ip \n",
+ "FROM example_koalas_nesteddata LIMIT 3\n",
+ "'''\n",
+ "resp = sql_client.sql_query(sql)\n",
+ "resp.show()"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "## Transform nested data"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "You can use Druid's [SQL JSON functions](https://druid.apache.org/docs/latest/querying/sql-json-functions.html) to transform nested data in your ingestion query.\n",
+ "\n",
+ "Run the following cell to insert sample data into a new datasource named `example_koalas_nesteddata_transform`. The SELECT query extracts the `country` and `city` elements from the nested `geo_ip` column and creates a composite object `sessionDetails` containing `session` and `session_length`."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "INSERT INTO example_koalas_nesteddata_transform\n",
+ " WITH \"source\" AS\n",
+ " (SELECT * FROM TABLE(EXTERN('{\"type\":\"http\",\"uris\":[\"https://static.imply.io/example-data/kttm-nested-v2/kttm-nested-v2-2019-08-25.json.gz\"]}',\n",
+ " '{\"type\":\"json\"}','[{\"name\":\"timestamp\",\"type\":\"string\"},{\"name\":\"session\",\"type\":\"string\"},{\"name\":\"session_length\",\"type\":\"string\"},\n",
+ " {\"name\":\"event\",\"type\":\"COMPLEX\"},{\"name\":\"agent\",\"type\":\"COMPLEX\"},{\"name\":\"geo_ip\",\"type\":\"COMPLEX\"}]')))\n",
+ " SELECT TIME_PARSE(\"timestamp\") AS \"__time\",\n",
+ " JSON_QUERY(geo_ip, '$.country') as country,\n",
+ " JSON_QUERY(geo_ip, '$.city') as city,\n",
+ " JSON_OBJECT('session':session, 'session_length':session_length) as sessionDetails\n",
+ " FROM \"source\"\n",
+ " PARTITIONED BY DAY\n",
+ "'''\n",
+ "\n",
+ "sql_client.run_task(sql)\n",
+ "sql_client.wait_until_ready(\"example_koalas_nesteddata_transform\")\n",
+ "display.table(\"example_koalas_nesteddata_transform\")"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "When the table detail displays, run the following cell to query the data and return `country`, `city`, and `sessionDetails` from 3 rows:"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT country, city, sessionDetails \n",
+ "FROM example_koalas_nesteddata_transform \n",
+ "LIMIT 3\n",
+ "'''\n",
+ "resp = sql_client.sql_query(sql)\n",
+ "resp.show()"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "## Query nested data"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "Run the following cell to display the data types for columns in the `example_koalas_nesteddata` datasource. Note that nested columns display as `COMPLEX`."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT TABLE_NAME, COLUMN_NAME, DATA_TYPE\n",
+ "FROM INFORMATION_SCHEMA.COLUMNS\n",
+ "WHERE TABLE_NAME = 'example_koalas_nesteddata'\n",
+ "'''\n",
+ "resp = sql_client.sql_query(sql)\n",
+ "resp.show()"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "You can use [`JSON_VALUE`](https://druid.apache.org/docs/latest/querying/sql-json-functions.html) to extract specific elements from a `COMPLEX` object.\n",
+ " \n",
+ "Run the following cell to extract `continent` from `geo_ip` and `category` from `agent` for 3 rows:"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT JSON_VALUE(geo_ip, '$.continent') as continent,\n",
+ "JSON_VALUE(agent, '$.category') as category\n",
+ "FROM example_koalas_nesteddata LIMIT 3\n",
+ "'''\n",
+ "resp = sql_client.sql_query(sql)\n",
+ "resp.show()"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "### Group, filter, and aggregate nested data"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "Run the following cell to see how you can use the SELECT COUNT(DISTINCT) operator with `JSON_VALUE`."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT COUNT(DISTINCT(JSON_VALUE(geo_ip, '$.city'))) as \"Number of cities\"\n",
+ "FROM example_koalas_nesteddata\n",
+ "'''\n",
+ "resp = sql_client.sql_query(sql)\n",
+ "resp.show()"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "Run the following cell to filter and group a query using `JSON_VALUE`. The query selects the `browser` element from the `agent` column and the `country` and `city` elements from the `geo_ip` column, for all rows with city `Helsinki`. "
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT JSON_VALUE(agent, '$.browser') as browser,\n",
+ "JSON_VALUE(geo_ip, '$.country') as country,\n",
+ "JSON_VALUE(geo_ip, '$.city') as city\n",
+ "FROM example_koalas_nesteddata\n",
+ "WHERE JSON_VALUE(geo_ip, '$.city') in ('Helsinki')\n",
+ "GROUP BY 1,2,3\n",
+ "ORDER BY 1\n",
+ "'''\n",
+ "resp = sql_client.sql_query(sql)\n",
+ "resp.show()"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "### Use helper operators"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "You can use SQL helper operators such as [`JSON_KEYS`](https://druid.apache.org/docs/latest/querying/sql-json-functions.html) and [`JSON_PATHS`](https://druid.apache.org/docs/latest/querying/sql-json-functions.html) to examine nested data and plan your queries. Run the following cell to return an array of field names and an array of paths for the `geo_ip` nested column."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT ARRAY_CONCAT_AGG(DISTINCT JSON_KEYS(geo_ip, '$.')) as \"geo_ip keys\",\n",
+ "ARRAY_CONCAT_AGG(DISTINCT JSON_PATHS(geo_ip)) as \"geo_ip paths\"\n",
+ "FROM example_koalas_nesteddata\n",
+ "'''\n",
+ "resp = sql_client.sql_query(sql)\n",
+ "resp.show()"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "## Learn more"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "metadata": {},
+ "source": [
+ "This tutorial covers the basics of working with nested data. To learn more about nested data in Druid and related Druid features, see the following topics:\n",
+ "\n",
+ "- [Nested columns](https://druid.apache.org/docs/latest/querying/nested-columns.html) for information about the nested columns feature, with ingestion and query examples. \n",
+ "- [SQL JSON functions](https://druid.apache.org/docs/latest/querying/sql-json-functions.html) for details on all of the functions you used in this tutorial.\n",
+ "- [SQL-based ingestion](https://druid.apache.org/docs/latest/multi-stage-query/index.html) for information on how to use Druid SQL-based ingestion."
+ ]
+ }
+ ],
+ "metadata": {
+ "kernelspec": {
+ "display_name": "Python 3 (ipykernel)",
+ "language": "python",
+ "name": "python3"
+ },
+ "language_info": {
+ "codemirror_mode": {
+ "name": "ipython",
+ "version": 3
+ },
+ "file_extension": ".py",
+ "mimetype": "text/x-python",
+ "name": "python",
+ "nbconvert_exporter": "python",
+ "pygments_lexer": "ipython3",
+ "version": "3.10.3"
+ },
+ "vscode": {
+ "interpreter": {
+ "hash": "a4289e5b8bae5973a6609d90f7bc464162478362b9a770893a3c5c597b0b36e7"
+ }
+ }
+ },
+ "nbformat": 4,
+ "nbformat_minor": 4
+}
diff --git a/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/DruidDataDriver.py b/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/DruidDataDriver.py
deleted file mode 100644
index 5acd25210be6..000000000000
--- a/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/DruidDataDriver.py
+++ /dev/null
@@ -1,1133 +0,0 @@
-# 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.
-#
-# DruidDataDriver - generates JSON records as a workload for Apache Druid.
-#
-
-import argparse
-import dateutil.parser
-from datetime import datetime, timedelta
-import json
-import numpy as np
-import random
-import re
-from sortedcontainers import SortedList
-import string
-import sys
-import threading
-import time
-
-############################################################################
-#
-# DruidDataDriver simulates Druid workloads by producing JSON records.
-# Use a JSON config file to describe the characteristics of the workload
-# you want to simulate.
-#
-# Run the program as follows:
-# python DruidDataDriver.py
-# Options include:
-# -n
-# -t
-#
-# See the associated documentation for the format of the config file.
-#
-############################################################################
-
-
-class FutureEvent:
- def __init__(self, t):
- self.t = t
- self.name = threading.current_thread().name
- self.event = threading.Event()
- def get_time(self):
- return self.t
- def get_name(self):
- return self.name
- def __lt__(self, other):
- return self.t < other.t
- def __eq__(self, other):
- return self.t == other.t
- def __le__(self, other):
- return self.t <= other.t
- def __gt__(self, other):
- return self.t > other.t
- def __ge__(self, other):
- return self.t >= other.t
-
- def __str__(self):
- return 'FutureEvent('+self.name+', '+str(self.t)+')'
- def pause(self):
- #print(self.name+" pausing")
- self.event.clear()
- self.event.wait()
- def resume(self):
- #print(self.name+" resuming")
- self.event.set()
-
-class Clock:
- future_events = SortedList()
- active_threads = 0
- lock = threading.Lock()
- sleep_lock = threading.Lock()
-
- def __init__(self, time_type, start_time = datetime.now()):
- self.sim_time = start_time
- self.time_type = time_type
-
- def __str__(self):
- s = 'Clock(time='+str(self.sim_time)
- for e in self.future_events:
- s += ', '+str(e)
- s += ')'
- return s
-
- def activate_thread(self):
- if self.time_type == 'SIM':
- self.lock.acquire()
- self.active_threads += 1
- self.lock.release()
-
- def deactivate_thread(self):
- if self.time_type == 'SIM':
- self.lock.acquire()
- self.active_threads -= 1
- self.lock.release()
-
- def end_thread(self):
- if self.time_type == 'SIM':
- self.lock.acquire()
- self.active_threads -= 1
- if len(self.future_events) > 0:
- self.remove_event().resume()
- self.lock.release()
-
- def release_all(self):
- if self.time_type == 'SIM':
- self.lock.acquire()
- #print('release_all - active_threads = '+str(self.active_threads))
- for e in self.future_events:
- e.resume()
- self.lock.release()
-
- def add_event(self, future_t):
- this_event = FutureEvent(future_t)
- self.future_events.add(this_event)
- #print('add_event (after) '+threading.current_thread().name+' - '+str(self))
- return this_event
-
- def remove_event(self):
- #print('remove_event (before) '+threading.current_thread().name+' - '+str(self))
- next_event = self.future_events[0]
- self.future_events.remove(next_event)
- return next_event
-
- def pause(self, event):
- self.active_threads -= 1
- self.lock.release()
- event.pause()
- self.lock.acquire()
- self.active_threads += 1
-
- def resume(self, event):
- event.resume()
-
- def now(self):
- if self.time_type == 'SIM':
- t = self.sim_time
- else:
- t = datetime.now()
- return t
-
- def sleep(self, delta):
- if self.time_type == 'SIM': # Simulated time
- self.lock.acquire()
- #print(threading.current_thread().name+" begin sleep "+str(self.sim_time)+" + "+str(delta))
- this_event = self.add_event(self.sim_time + timedelta(seconds=delta))
- #print(threading.current_thread().name+" active threads "+str(self.active_threads))
- if self.active_threads == 1:
- next_event = self.remove_event()
- if str(this_event) != str(next_event):
- self.resume(next_event)
- #print(threading.current_thread().name+" start pause if")
- self.pause(this_event)
- #print(threading.current_thread().name+" end pause if")
- else:
- #print(threading.current_thread().name+" start pause else")
- self.pause(this_event)
- #print(threading.current_thread().name+" end pause else")
- self.sim_time = this_event.get_time()
- #print(threading.current_thread().name+" end sleep "+str(self.sim_time))
- self.lock.release()
-
- else: # Real time
- time.sleep(delta)
-
-
-#
-# Set up the target
-#
-
-class PrintStdout:
- lock = threading.Lock()
- def print(self, record):
- with self.lock:
- print(str(record))
- sys.stdout.flush()
- def __str__(self):
- return '#printStdout()'
-
-class PrintFile:
- f = None
- def __init__(self, file_name):
- self.file_name = file_name
- self.f = open(file_name, 'w')
- def __del__(self):
- if self.f != None:
- self.f.close()
- def __str__(self):
- return 'PrintFile(file_name='+self.file_name+')'
- def print(self, record):
- self.f.write(str(record)+'\n')
- self.f.flush()
-
-class PrintKafka:
- producer = None
- topic = None
- def __init__(self, endpoint, topic, security_protocol, compression_type):
- from kafka import KafkaProducer
-
- #print('PrintKafka('+str(endpoint)+', '+str(topic)+', '+str(security_protocol)+', '+str(compression_type)+')')
- self.endpoint = endpoint
- self.producer = KafkaProducer(bootstrap_servers=endpoint, security_protocol=security_protocol, compression_type=compression_type, value_serializer=lambda v: json.dumps(v).encode('utf-8'))
- self.topic = topic
- def __str__(self):
- return 'PrintKafka(endpoint='+self.endpoint+', topic='+self.topic+')'
- def print(self, record):
- self.producer.send(self.topic, json.loads(str(record)))
-
-class PrintConfluent:
- producer = None
- topic = None
- username = None
- password = None
- def __init__(self, servers, topic, username, password):
- from confluent_kafka import Producer
-
- #print('PrintKafka('+str(endpoint)+', '+str(topic)+', '+str(security_protocol)+', '+str(compression_type)+')')
- self.servers = servers
- self.producer = Producer({
- 'bootstrap.servers': servers,
- 'sasl.mechanisms': 'PLAIN',
- 'security.protocol': 'SASL_SSL',
- 'sasl.username': username,
- 'sasl.password': password
- })
- self.topic = topic
- self.username = username
- self.password = password
- def __str__(self):
- return 'PrintConfluent(servers='+self.servers+', topic='+self.topic+', username='+self.username+', password='+self.password+')'
- def print(self, record):
- print('producing '+str(record))
- self.producer.produce(topic=self.topic, value=str(record))
- self.producer.flush()
-
-
-#
-# Handle distributions
-#
-
-class DistConstant:
- def __init__(self, value):
- self.value = value
- def __str__(self):
- return 'DistConstant(value='+str(self.value)+')'
- def get_sample(self):
- return self.value
-
-class DistUniform:
- def __init__(self, min_value, max_value):
- self.min_value = min_value
- self.max_value = max_value
- def __str__(self):
- return 'DistUniform(min_value='+str(self.min_value)+', max_value='+str(self.max_value)+')'
- def get_sample(self):
- return np.random.uniform(self.min_value, self.max_value+1)
-
-class DistExponential:
- def __init__(self, mean):
- self.mean = mean
- def __str__(self):
- return 'DistExponential(mean='+str(self.mean)+')'
- def get_sample(self):
- return np.random.exponential(scale = self.mean)
-
-class DistNormal:
- def __init__(self, mean, stddev):
- self.mean = mean
- self.stddev = stddev
- def __str__(self):
- return 'DistNormal(mean='+str(self.mean )+', stddev='+str(self.stddev)+')'
- def get_sample(self):
- return np.random.normal(self.mean, self.stddev)
-
-def parse_distribution(desc):
- dist_type = desc['type'].lower()
- dist_gen = None
- if dist_type == 'constant':
- value = desc['value']
- dist_gen = DistConstant(value)
- elif dist_type == 'uniform':
- min_value = desc['min']
- max_value = desc['max']
- dist_gen = DistUniform(min_value, max_value)
- elif dist_type == 'exponential':
- mean = desc['mean']
- dist_gen = DistExponential(mean)
- elif dist_type == 'normal':
- mean = desc['mean']
- stddev = desc['stddev']
- dist_gen = DistNormal(mean, stddev)
- else:
- print('Error: Unknown distribution "'+dist_type+'"')
- exit()
- return dist_gen
-
-def parse_timestamp_distribution(desc):
- dist_type = desc['type'].lower()
- dist_gen = None
- if dist_type == 'constant':
- value = dateutil.parser.isoparse(desc['value']).timestamp()
- dist_gen = DistConstant(value)
- elif dist_type == 'uniform':
- min_value = dateutil.parser.isoparse(desc['min']).timestamp()
- max_value = dateutil.parser.isoparse(desc['max']).timestamp()
- dist_gen = DistUniform(min_value, max_value)
- elif dist_type == 'exponential':
- mean = dateutil.parser.isoparse(desc['mean']).timestamp()
- dist_gen = DistExponential(mean)
- elif dist_type == 'normal':
- mean = desc[dateutil.parser.isoparse(desc['mean']).timestamp()]
- stddev = desc['stddev']
- dist_gen = DistNormal(mean, stddev)
- else:
- print('Error: Unknown distribution "'+dist_type+'"')
- exit()
- return dist_gen
-
-
-#
-# Set up the dimensions for the emitters (see below)
-# There is one element class for each dimension type. This code creates a list of
-# elements and then runs through the list to create a single record.
-# Notice that the get_json_field_string() method produces the JSON dimension
-# field object based on the dimension configuration.
-# The get_stochastic_value() method is like a private method used to get a random
-# idividual value.
-#
-
-class ElementNow: # The __time dimension
- def __init__(self, global_clock):
- self.global_clock = global_clock
- def __str__(self):
- return 'ElementNow()'
- def get_json_field_string(self):
- now = self.global_clock.now().isoformat()[:-3]
- return '"__time":"'+now+'"'
-
-class ElementCounter: # The __time dimension
- def __init__(self, desc):
- self.name = desc['name']
- if 'percent_nulls' in desc.keys():
- self.percent_nulls = desc['percent_nulls'] / 100.0
- else:
- self.percent_nulls = 0.0
- if 'percent_missing' in desc.keys():
- self.percent_missing = desc['percent_missing'] / 100.0
- else:
- self.percent_missing = 0.0
- if 'start' in desc.keys():
- self.start = desc['start']
- else:
- self.start = 0
- if 'increment' in desc.keys():
- self.increment = desc['increment']
- else:
- self.increment = 1
- self.value = self.start
- def __str__(self):
- s = 'ElementCounter(name='+self.name
- if self.start != 0:
- s += ', '+str(self.start)
- if self.increment != 1:
- s += ', '+str(self.increment)
- s += ')'
- return s
-
- def get_stochastic_value(self):
- v = self.value
- self.value += self.increment
- return v
-
- def get_json_field_string(self):
- if random.random() < self.percent_nulls:
- s = '"'+self.name+'": null'
- else:
- s = '"'+self.name+'":"'+str(self.get_stochastic_value())+'"'
- return s
-
- def is_missing(self):
- return random.random() < self.percent_missing
-
-
-class ElementEnum: # enumeration dimensions
- def __init__(self, desc):
- self.name = desc['name']
- if 'percent_nulls' in desc.keys():
- self.percent_nulls = desc['percent_nulls'] / 100.0
- else:
- self.percent_nulls = 0.0
- if 'percent_missing' in desc.keys():
- self.percent_missing = desc['percent_missing'] / 100.0
- else:
- self.percent_missing = 0.0
- self.cardinality = desc['values']
- if 'cardinality_distribution' not in desc.keys():
- print('Element '+self.name+' specifies a cardinality without a cardinality distribution')
- exit()
- self.cardinality_distribution = parse_distribution(desc['cardinality_distribution'])
-
- def __str__(self):
- return 'ElementEnum(name='+self.name+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+')'
-
- def get_stochastic_value(self):
- index = int(self.cardinality_distribution.get_sample())
- if index < 0:
- index = 0
- if index >= len(self.cardinality):
- index = len(self.cardinality)-1
- return self.cardinality[index]
-
- def get_json_field_string(self):
- if random.random() < self.percent_nulls:
- s = '"'+self.name+'": null'
- else:
- s = '"'+self.name+'":"'+str(self.get_stochastic_value())+'"'
- return s
-
- def is_missing(self):
- return random.random() < self.percent_missing
-
-class ElementVariable: # Variable dimensions
- def __init__(self, desc):
- self.name = desc['name']
- self.variable_name = desc['variable']
-
- def __str__(self):
- return 'ElementVariable(name='+self.name+', value='+self.variable_name+')'
-
- def get_json_field_string(self, variables): # NOTE: because of timing, this method has a different signature than the other elements
- value = variables[self.variable_name]
- return '"'+self.name+'":"'+str(value)+'"'
-
-# TODO: Refactor ElementBase and subclasses, and those element classes that don't inherit from ElementBase
-class ElementBase: # Base class for the remainder of the dimensions
- def __init__(self, desc):
- self.name = desc['name']
- if 'percent_nulls' in desc.keys():
- self.percent_nulls = desc['percent_nulls'] / 100.0
- else:
- self.percent_nulls = 0.0
- if 'percent_missing' in desc.keys():
- self.percent_missing = desc['percent_missing'] / 100.0
- else:
- self.percent_missing = 0.0
-
- self.cardinality_setting = desc['cardinality']
- self.cardinality_distribution = None
-
- if self.cardinality_setting == 0:
- self.cardinality = None
-
- else:
- self.cardinality = []
- if 'cardinality_distribution' not in desc.keys():
- print('Element '+self.name+' specifies a cardinality without a cardinality distribution')
- exit()
- self.cardinality_distribution = parse_distribution(desc['cardinality_distribution'])
- self.init_cardinality()
-
- def init_cardinality(self):
- for i in range(self.cardinality_setting):
- value = None
- while True:
- value = self.get_stochastic_value()
- if value not in self.cardinality:
- break
- self.cardinality.append(value)
-
-
- def get_stochastic_value(self):
- pass
-
- def get_json_field_string(self):
- if random.random() < self.percent_nulls:
- s = '"'+self.name+'": null'
- else:
- if self.cardinality is None:
- value = self.get_stochastic_value()
- else:
- index = int(self.cardinality_distribution.get_sample())
- if index < 0:
- index = 0
- if index >= len(self.cardinality):
- index = len(self.cardinality)-1
- value = self.cardinality[index]
- s = '"'+self.name+'":'+str(value)
- return s
-
- def is_missing(self):
- return random.random() < self.percent_missing
-
-
-class ElementString(ElementBase):
-
- def __init__(self, desc):
- self.length_distribution = parse_distribution(desc['length_distribution'])
- if 'chars' in desc:
- self.chars = desc['chars']
- else:
- self.chars = string.printable
- super().__init__(desc)
-
- def __str__(self):
- return 'ElementString(name='+self.name+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+', chars='+self.chars+')'
-
- def get_stochastic_value(self):
- length = int(self.length_distribution.get_sample())
- return ''.join(random.choices(list(self.chars), k=length))
-
- def get_json_field_string(self):
- if random.random() < self.percent_nulls:
- s = '"'+self.name+'": null'
- else:
- if self.cardinality is None:
- value = self.get_stochastic_value()
- else:
- index = int(self.cardinality_distribution.get_sample())
- if index < 0:
- index = 0
- if index >= len(self.cardinality):
- index = len(self.cardinality)-1
- value = self.cardinality[index]
- s = '"'+self.name+'":"'+str(value)+'"'
- return s
-
-class ElementInt(ElementBase):
- def __init__(self, desc):
- self.value_distribution = parse_distribution(desc['distribution'])
- super().__init__(desc)
-
- def __str__(self):
- return 'ElementInt(name='+self.name+', value_distribution='+str(self.value_distribution)+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+')'
-
- def get_stochastic_value(self):
- return int(self.value_distribution.get_sample())
-
-class ElementFloat(ElementBase):
- def __init__(self, desc):
- self.value_distribution = parse_distribution(desc['distribution'])
- if 'precision' in desc:
- self.precision = desc['precision']
- else:
- self.precision = None
- super().__init__(desc)
-
- def __str__(self):
- return 'ElementFloat(name='+self.name+', value_distribution='+str(self.value_distribution)+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+')'
-
- def get_stochastic_value(self):
- return float(self.value_distribution.get_sample())
-
- def get_json_field_string(self):
- if random.random() < self.percent_nulls:
- s = '"'+self.name+'": null'
- else:
- if self.cardinality is None:
- value = self.get_stochastic_value()
- else:
- index = int(self.cardinality_distribution.get_sample())
- if index < 0:
- index = 0
- if index >= len(self.cardinality):
- index = len(self.cardinality)-1
- value = self.cardinality[index]
- if self.precision is None:
- s = '"'+self.name+'":'+str(value)
- else:
- format = '%.'+str(self.precision)+'f'
- s = '"'+self.name+'":'+str(format%value)
- return s
-
-class ElementTimestamp(ElementBase):
- def __init__(self, desc):
- super().__init__(desc)
-
- def __str__(self):
- return 'ElementTimestamp(name='+self.name+', value_distribution='+str(self.value_distribution)+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+')'
-
- def get_stochastic_value(self):
- return datetime.fromtimestamp(self.value_distribution.get_sample()).isoformat()[:-3]
-
- def get_json_field_string(self):
- if random.random() < self.percent_nulls:
- s = '"'+self.name+'": null'
- else:
- if self.cardinality is None:
- value = self.get_stochastic_value()
- else:
- index = int(self.cardinality_distribution.get_sample())
- if index < 0:
- index = 0
- if index >= len(self.cardinality):
- index = len(self.cardinality)-1
- value = self.cardinality[index]
- s = '"'+self.name+'":"'+str(value)+'"'
- return s
-
- def is_missing(self):
- return random.random() < self.percent_missing
-
-class ElementIPAddress(ElementBase):
- def __init__(self, desc):
- self.value_distribution = parse_distribution(desc['distribution'])
- super().__init__(desc)
-
- def __str__(self):
- return 'ElementIPAddress(name='+self.name+', value_distribution='+str(self.value_distribution)+', cardinality='+str(self.cardinality)+', cardinality_distribution='+str(self.cardinality_distribution)+')'
-
- def get_stochastic_value(self):
- value = int(self.value_distribution.get_sample())
- return str((value & 0xFF000000) >> 24)+'.'+str((value & 0x00FF0000) >> 16)+'.'+str((value & 0x0000FF00) >> 8)+'.'+str(value & 0x000000FF)
-
- def get_json_field_string(self):
- if random.random() < self.percent_nulls:
- s = '"'+self.name+'": null'
- else:
- if self.cardinality is None:
- value = self.get_stochastic_value()
- else:
- index = int(self.cardinality_distribution.get_sample())
- if index < 0:
- index = 0
- if index >= len(self.cardinality):
- index = len(self.cardinality)-1
- value = self.cardinality[index]
- s = '"'+self.name+'":"'+str(value)+'"'
- return s
-
-class ElementObject():
- def __init__(self, desc):
- self.name = desc['name']
- self.dimensions = get_variables(desc['dimensions'])
- if 'percent_nulls' in desc.keys():
- self.percent_nulls = desc['percent_nulls'] / 100.0
- else:
- self.percent_nulls = 0.0
- if 'percent_missing' in desc.keys():
- self.percent_missing = desc['percent_missing'] / 100.0
- else:
- self.percent_missing = 0.0
- cardinality = desc['cardinality']
- if cardinality == 0:
- self.cardinality = None
- self.cardinality_distribution = None
- else:
- self.cardinality = []
- if 'cardinality_distribution' not in desc.keys():
- print('Element '+self.name+' specifies a cardinality without a cardinality distribution')
- exit()
- self.cardinality_distribution = parse_distribution(desc['cardinality_distribution'])
- for i in range(cardinality):
- value = None
- while True:
- value = self.get_instance()
- if value not in self.cardinality:
- break
- self.cardinality.append(value)
-
- def __str__(self):
- s = 'ElementObject(name='+self.name+', dimensions=['
- for e in self.dimensions:
- s += ',' + str(e)
- s += '])'
- return s
-
- def get_instance(self):
- s = '"'+self.name+'": {'
- for e in self.dimensions:
- s += e.get_json_field_string() + ','
- s = s[:-1] + '}'
- return s
-
-
- def get_json_field_string(self):
- if random.random() < self.percent_nulls:
- s = '"'+self.name+'": null'
- else:
- if self.cardinality is None:
- s = self.get_instance()
- else:
- index = int(self.cardinality_distribution.get_sample())
- if index < 0:
- index = 0
- if index >= len(self.cardinality):
- index = len(self.cardinality)-1
- s = self.cardinality[index]
- return s
-
- def is_missing(self):
- return random.random() < self.percent_missing
-
-class ElementList():
- def __init__(self, desc):
- self.name = desc['name']
- self.elements = get_variables(desc['elements'])
- self.length_distribution = parse_distribution(desc['length_distribution'])
- self.selection_distribution = parse_distribution(desc['selection_distribution'])
- if 'percent_nulls' in desc.keys():
- self.percent_nulls = desc['percent_nulls'] / 100.0
- else:
- self.percent_nulls = 0.0
- if 'percent_missing' in desc.keys():
- self.percent_missing = desc['percent_missing'] / 100.0
- else:
- self.percent_missing = 0.0
- cardinality = desc['cardinality']
- if cardinality == 0:
- self.cardinality = None
- self.cardinality_distribution = None
- else:
- self.cardinality = []
- if 'cardinality_distribution' not in desc.keys():
- print('Element '+self.name+' specifies a cardinality without a cardinality distribution')
- exit()
- self.cardinality_distribution = parse_distribution(desc['cardinality_distribution'])
- for i in range(cardinality):
- value = None
- while True:
- value = self.get_instance()
- if value not in self.cardinality:
- break
- self.cardinality.append(value)
-
- def __str__(self):
- s = 'ElementObject(name='+self.name
- s += ', length_distribution='+str(self.length_distribution)
- s += ', selection_distribution='+str(self.selection_distribution)
- s += ', elements=['
- for e in self.elements:
- s += ',' + str(e)
- s += '])'
- return s
-
- def get_instance(self):
- s = '"'+self.name+'": ['
- length = int(self.length_distribution.get_sample())
- for i in range(length):
- index = int(self.selection_distribution.get_sample())
- if index < 0:
- index = 0
- if index >= length:
- index = length-1
- s += re.sub('^.*?:', '', self.elements[index].get_json_field_string(), count=1) + ','
- s = s[:-1] + ']'
- return s
-
-
- def get_json_field_string(self):
- if random.random() < self.percent_nulls:
- s = '"'+self.name+'": null'
- else:
- if self.cardinality is None:
- s = self.get_instance()
- else:
- index = int(self.cardinality_distribution.get_sample())
- if index < 0:
- index = 0
- if index >= len(self.cardinality):
- index = len(self.cardinality)-1
- s = self.cardinality[index]
- return s
-
- def is_missing(self):
- return random.random() < self.percent_missing
-
-
-def parse_element(desc):
- if desc['type'].lower() == 'counter':
- el = ElementCounter(desc)
- elif desc['type'].lower() == 'enum':
- el = ElementEnum(desc)
- elif desc['type'].lower() == 'string':
- el = ElementString(desc)
- elif desc['type'].lower() == 'int':
- el = ElementInt(desc)
- elif desc['type'].lower() == 'float':
- el = ElementFloat(desc)
- elif desc['type'].lower() == 'timestamp':
- el = ElementTimestamp(desc)
- elif desc['type'].lower() == 'ipaddress':
- el = ElementIPAddress(desc)
- elif desc['type'].lower() == 'variable':
- el = ElementVariable(desc)
- elif desc['type'].lower() == 'object':
- el = ElementObject(desc)
- elif desc['type'].lower() == 'list':
- el = ElementList(desc)
- else:
- print('Error: Unknown dimension type "'+desc['type']+'"')
- exit()
- return el
-
-
-def get_variables(desc):
- elements = []
- for element in desc:
- el = parse_element(element)
- elements.append(el)
- return elements
-
-def get_dimensions(desc, global_clock):
- elements = get_variables(desc)
- elements.insert(0, ElementNow(global_clock))
- return elements
-
-
-#
-# Set up the state machine
-#
-
-class Transition:
- def __init__(self, next_state, probability):
- self.next_state = next_state
- self.probability = probability
-
- def __str__(self):
- return 'Transition(next_state='+str(self.next_state)+', probability='+str(self.probability)+')'
-
-def parse_transitions(desc):
- transitions = []
- for trans in desc:
- next_state = trans['next']
- probability = float(trans['probability'])
- transitions.append(Transition(next_state, probability))
- return transitions
-
-class State:
- def __init__(self, name, dimensions, delay, transitions, variables):
- self.name = name
- self.dimensions = dimensions
- self.delay = delay
- self.transistion_states = [t.next_state for t in transitions]
- self.transistion_probabilities = [t.probability for t in transitions]
- self.variables = variables
-
- def __str__(self):
- return 'State(name='+self.name+', dimensions='+str([str(d) for d in self.dimensions])+', delay='+str(self.delay)+', transistion_states='+str(self.transistion_states)+', transistion_probabilities='+str(self.transistion_probabilities)+'variables='+str([str(v) for v in self.variables])+')'
-
- def get_next_state_name(self):
- return random.choices(self.transistion_states, weights=self.transistion_probabilities, k=1)[0]
-
-class SimEnd:
- lock = threading.Lock()
- thread_end_event = threading.Event()
- def __init__(self, total_recs, runtime, global_clock):
- self.total_recs = total_recs
- self.record_count = 0
- self.global_clock = global_clock
- if runtime is None:
- self.t = None
- else:
- if runtime[-1].lower() == 's':
- self.t = int(runtime[:-1])
- elif runtime[-1].lower() == 'm':
- self.t = int(runtime[:-1]) * 60
- elif runtime[-1].lower() == 'h':
- self.t = int(runtime[:-1]) * 60 * 60
- else:
- print('Error: Unknown runtime value"'+runtime+'"')
- exit()
-
- def inc_rec_count(self):
- self.lock.acquire()
- self.record_count += 1
- self.lock.release()
- if (self.total_recs is not None) and (self.record_count >= self.total_recs):
- self.thread_end_event.set()
-
- def is_done(self):
- return ((self.total_recs is not None) and (self.record_count >= self.total_recs)) or ((self.t is not None) and self.thread_end_event.is_set())
-
- def wait_for_end(self):
- if self.t is not None:
- self.global_clock.activate_thread()
- self.global_clock.sleep(self.t)
- self.thread_end_event.set()
- self.global_clock.deactivate_thread()
- elif self.total_recs is not None:
- self.thread_end_event.wait()
- self.global_clock.release_all()
- else:
- while True:
- time.sleep(60)
-
-
-#
-# Run the driver
-#
-
-def create_record(dimensions, variables):
- json_string = '{'
- for element in dimensions:
- if isinstance(element, ElementVariable):
- json_string += element.get_json_field_string(variables) + ','
- else:
- if isinstance(element, ElementNow) or not element.is_missing():
- json_string += element.get_json_field_string() + ','
- json_string = json_string[:-1] + '}'
- return json_string
-
-def set_variable_values(variables, dimensions):
- for d in dimensions:
- variables[d.name] = d.get_stochastic_value()
-
-def worker_thread(target_printer, states, initial_state, sim_end, global_clock):
- # Process the state machine using worker threads
- #print('Thread '+threading.current_thread().name+' starting...')
- global_clock.activate_thread()
- current_state = initial_state
- variables = {}
- while True:
- set_variable_values(variables, current_state.variables)
- record = create_record(current_state.dimensions, variables)
- target_printer.print(record)
- sim_end.inc_rec_count()
- if sim_end.is_done():
- break
- delta = float(current_state.delay.get_sample())
- global_clock.sleep(delta)
- if sim_end.is_done():
- break
- next_state_name = current_state.get_next_state_name()
- if next_state_name.lower() == 'stop':
- break
- current_state = states[next_state_name]
-
- #print('Thread '+threading.current_thread().name+' done!')
- global_clock.end_thread()
-
-def spawning_thread(target_printer, rate_delay, states, initial_state, sim_end, global_clock):
- #print('Thread '+threading.current_thread().name+' starting...')
- global_clock.activate_thread()
- # Spawn the workers in a separate thread so we can stop the whole thing in the middle of spawning if necessary
- count = 0
- while not sim_end.is_done():
- thread_name = 'W'+str(count)
-
- count += 1
- t = threading.Thread(target=worker_thread, args=(target_printer, states, initial_state, sim_end, global_clock, ), name=thread_name, daemon=True)
- t.start()
- global_clock.sleep(float(rate_delay.get_sample()))
- global_clock.end_thread()
- #print('Thread '+threading.current_thread().name+' done!')
-
-
-def simulate(config_file_name, runtime, total_recs, time_type, start_time):
-
- if config_file_name:
- with open(config_file_name, 'r') as f:
- config = json.load(f)
- else:
- config = json.load(sys.stdin)
-
- #
- # Set up the gloabl clock
- #
-
- global_clock = Clock(time_type, start_time)
- sim_end = SimEnd(total_recs, runtime, global_clock)
-
-
- #
- # Set up the output target
- #
-
- target = config['target']
-
- if target['type'].lower() == 'stdout':
- target_printer = PrintStdout()
- elif target['type'].lower() == 'file':
- path = target['path']
- if path is None:
- print('Error: File target requires a path item')
- exit()
- target_printer = PrintFile(path)
- elif target['type'].lower() == 'kafka':
- if 'endpoint' in target.keys():
- endpoint = target['endpoint']
- else:
- print('Error: Kafka target requires an endpoint item')
- exit()
- if 'topic' in target.keys():
- topic = target['topic']
- else:
- print('Error: Kafka target requires a topic item')
- exit()
- if 'security_protocol' in target.keys():
- security_protocol = target['security_protocol']
- else:
- security_protocol = 'PLAINTEXT'
- if 'compression_type' in target.keys():
- compression_type = target['compression_type']
- else:
- compression_type = None
- target_printer = PrintKafka(endpoint, topic, security_protocol, compression_type)
- elif target['type'].lower() == 'confluent':
- if 'servers' in target.keys():
- servers = target['servers']
- else:
- print('Error: Conlfuent target requires a servers item')
- exit()
- if 'topic' in target.keys():
- topic = target['topic']
- else:
- print('Error: Confluent target requires a topic item')
- exit()
- if 'username' in target.keys():
- username = target['username']
- else:
- print('Error: Confluent target requires a username')
- exit()
- if 'password' in target.keys():
- password = target['password']
- else:
- print('Error: Confluent target requires a password')
- exit()
- target_printer = PrintConfluent(servers, topic, username, password)
- else:
- print('Error: Unknown target type "'+target['type']+'"')
- exit()
-
- #sys.stderr.write('target='+str(target_printer)+'\n')
-
-
- #
- # Set up the interarrival rate
- #
-
- rate = config['interarrival']
- rate_delay = parse_distribution(rate)
-
- #sys.stderr.write('rate_delay='+str(rate_delay)+'\n')
-
-
- #
- # Set up emitters list
- #
-
- emitters = {}
- for emitter in config['emitters']:
- name = emitter['name']
- dimensions = get_dimensions(emitter['dimensions'], global_clock)
- emitters[name] = dimensions
-
- #sys.stderr.write('emitters='+str(['(name='+str(key)+', dimensions='+str([str(e) for e in emitters[key]])+')' for key in emitters])+'\n')
-
-
- #
- # Set up the state machine
- #
-
- state_desc = config['states']
- initial_state = None
- states = {}
- for state in state_desc:
- name = state['name']
- emitter_name = state['emitter']
- if 'variables' not in state.keys():
- variables = []
- else:
- variables = get_variables(state['variables'])
- dimensions = emitters[emitter_name]
- delay = parse_distribution(state['delay'])
- transitions = parse_transitions(state['transitions'])
- this_state = State(name, dimensions, delay, transitions, variables)
- states[name] = this_state
- if initial_state is None:
- initial_state = this_state
-
- #sys.stderr.write('states='+str(['('+str(key)+':'+str(states[key])+')' for key in states])+'\n')
-
- #
- # Finally, start the simulation
- #
-
- thrd = threading.Thread(target=spawning_thread, args=(target_printer, rate_delay, states, initial_state, sim_end, global_clock, ), name='Spawning', daemon=True)
- thrd.start()
- sim_end.wait_for_end()
-
-def main():
-
- #
- # Parse the command line
- #
-
- parser = argparse.ArgumentParser(description='Generates JSON records as a workload for Apache Druid.')
- #parser.add_argument('config_file', metavar='', help='the workload config file name')
- parser.add_argument('-f', dest='config_file', nargs='?', help='the workload config file name')
- parser.add_argument('-t', dest='time', nargs='?', help='the script runtime (may not be used with -n)')
- parser.add_argument('-n', dest='n_recs', nargs='?', help='the number of records to generate (may not be used with -t)')
- parser.add_argument('-s', dest='time_type', nargs='?', const='SIM', default='REAL', help='simulate time (default is real, not simulated)')
-
- args = parser.parse_args()
-
- config_file_name = args.config_file
- runtime = args.time
- total_recs = None
- if args.n_recs is not None:
- total_recs = int(args.n_recs)
- time_type = args.time_type
- if time_type == 'SIM':
- start_time = datetime.now()
- elif time_type == 'REAL':
- start_time = datetime.now()
- else:
- start_time = dateutil.parser.isoparse(time_type)
- time_type = 'SIM'
-
-
- if (runtime is not None) and (total_recs is not None):
- print("Use either -t or -n, but not both")
- parser.print_help()
- exit()
-
-
- simulate(config_file_name, runtime, total_recs, time_type, start_time)
-
-
-
-if __name__ == "__main__":
- main()
diff --git a/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/kafka_docker_config.json b/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/kafka_docker_config.json
deleted file mode 100644
index 2add8f3fa143..000000000000
--- a/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/kafka_docker_config.json
+++ /dev/null
@@ -1,90 +0,0 @@
-{
- "target": {
- "type": "kafka",
- "endpoint": "kafka:9092",
- "topic": "social_media"
- },
- "emitters": [
- {
- "name": "example_record_1",
- "dimensions": [
- {
- "type": "enum",
- "name": "username",
- "values": ["willow", "mia", "leon", "milton", "miette", "gus", "jojo", "rocket"],
- "cardinality_distribution": {
- "type": "uniform",
- "min": 0,
- "max": 7
- }
- },
- {
- "type": "string",
- "name": "post_title",
- "length_distribution": {"type": "uniform", "min": 1, "max": 140},
- "cardinality": 0,
- "chars": "abcdefghijklmnopqrstuvwxyz0123456789_ABCDEFGHIJKLMNOPQRSTUVWXYZ!';:,."
- },
- {
- "type": "int",
- "name": "views",
- "distribution": {
- "type": "exponential",
- "mean": 10000
- },
- "cardinality": 0
- },
- {
- "type": "int",
- "name": "upvotes",
- "distribution": {
- "type": "normal",
- "mean": 70,
- "stddev": 20
- },
- "cardinality": 0
- },
- {
- "type": "int",
- "name": "comments",
- "distribution": {
- "type": "normal",
- "mean": 10,
- "stddev": 5
- },
- "cardinality": 0
- },
- {
- "type": "enum",
- "name": "edited",
- "values": ["True","False"],
- "cardinality_distribution": {
- "type": "uniform",
- "min": 0,
- "max": 1
- }
- }
- ]
- }
- ],
- "interarrival": {
- "type": "constant",
- "value": 1
- },
- "states": [
- {
- "name": "state_1",
- "emitter": "example_record_1",
- "delay": {
- "type": "constant",
- "value": 1
- },
- "transitions": [
- {
- "next": "state_1",
- "probability": 1.0
- }
- ]
- }
- ]
-}
diff --git a/examples/quickstart/jupyter-notebooks/notebooks/03-query/02-approxRanking.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/03-query/02-approxRanking.ipynb
new file mode 100644
index 000000000000..e49ac92e84a0
--- /dev/null
+++ b/examples/quickstart/jupyter-notebooks/notebooks/03-query/02-approxRanking.ipynb
@@ -0,0 +1,596 @@
+{
+ "cells": [
+ {
+ "cell_type": "markdown",
+ "id": "557e06e8-9b35-4b34-8322-8a8ede6de709",
+ "metadata": {},
+ "source": [
+ "# Using TopN approximation in Druid queries\n",
+ "\n",
+ "\n",
+ "\n",
+ "Imagine you’re building a dynamic filter in your app: you want to populate it with, say, the top most popular (COUNT) dimension values in descending order (ORDER BY). Druid speeds up this type of query using TopN approximation by default. In this tutorial, work through some examples and see the effect of turning approximation off."
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "c94ff5c9-ada9-4f1d-8541-649e70cfc9a3",
+ "metadata": {},
+ "source": [
+ "## Prerequisites\n",
+ "\n",
+ "This tutorial works with Druid 26.0.0 or later.\n",
+ "\n",
+ "#### Run using Docker\n",
+ "\n",
+ "Launch this tutorial and all prerequisites using the `druid-jupyter` profile of the Docker Compose file for Jupyter-based Druid tutorials. For more information, see [Docker for Jupyter Notebook tutorials](https://druid.apache.org/docs/latest/tutorials/tutorial-jupyter-docker.html).\n",
+ " \n",
+ "#### Run without using Docker\n",
+ "\n",
+ "If you do not use the Docker Compose environment, you need the following:\n",
+ "\n",
+ "* A running Apache Druid instance, with a `DRUID_HOST` local environment variable containing the server name of your Druid router.\n",
+ "* [druidapi](https://github.com/apache/druid/blob/master/examples/quickstart/jupyter-notebooks/druidapi/README.md), a Python client for Apache Druid. Follow the instructions in the Install section of the README file.\n",
+ "* [matplotlib](https://matplotlib.org/), a library for creating visualizations in Python.\n",
+ "* [pandas](https://pandas.pydata.org/), a data analysis and manipulation tool."
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "e6b56cfc-9951-4a4e-a3f4-828e2dd5b3b5",
+ "metadata": {},
+ "source": [
+ "### Initialize Python\n",
+ "\n",
+ "Run the next cell to set up the Druid Python client's connection to Apache Druid.\n",
+ "\n",
+ "If successful, the Druid version number will be shown in the output."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "685b872e-0d59-4100-a636-39ec93c627fb",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "import druidapi\n",
+ "import os\n",
+ "\n",
+ "if 'DRUID_HOST' not in os.environ.keys():\n",
+ " druid_host=f\"http://localhost:8888\"\n",
+ "else:\n",
+ " druid_host=f\"http://{os.environ['DRUID_HOST']}:8888\"\n",
+ " \n",
+ "print(f\"Opening a connection to {druid_host}.\")\n",
+ "druid = druidapi.jupyter_client(druid_host)\n",
+ "\n",
+ "display = druid.display\n",
+ "sql_client = druid.sql\n",
+ "status_client = druid.status\n",
+ "\n",
+ "status_client.version"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "d6fe9c99-ee0d-4205-9ca5-a8810c977335",
+ "metadata": {},
+ "source": [
+ "### Load example data\n",
+ "\n",
+ "Once your Druid environment is up and running, ingest the sample data for this tutorial.\n",
+ "\n",
+ "Run the following cell to create a table called `example-flights-topn`. When completed, you'll see a description of the final table.\n",
+ "\n",
+ "Monitor the ingestion task process in the Druid console."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "e89a3000-a65e-4c4a-a917-3c37cbe975b3",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql='''\n",
+ "REPLACE INTO \"example-flights-topn\" OVERWRITE ALL\n",
+ "WITH \"ext\" AS (SELECT *\n",
+ "FROM TABLE(\n",
+ " EXTERN(\n",
+ " '{\"type\":\"http\",\"uris\":[\"https://static.imply.io/example-data/flight_on_time/flights/On_Time_Reporting_Carrier_On_Time_Performance_(1987_present)_2005_11.csv.zip\"]}',\n",
+ " '{\"type\":\"csv\",\"findColumnsFromHeader\":true}'\n",
+ " )\n",
+ ") EXTEND (\"depaturetime\" VARCHAR, \"arrivalime\" VARCHAR, \"Year\" BIGINT, \"Quarter\" BIGINT, \"Month\" BIGINT, \"DayofMonth\" BIGINT, \"DayOfWeek\" BIGINT, \"FlightDate\" VARCHAR, \"Reporting_Airline\" VARCHAR, \"DOT_ID_Reporting_Airline\" BIGINT, \"IATA_CODE_Reporting_Airline\" VARCHAR, \"Tail_Number\" VARCHAR, \"Flight_Number_Reporting_Airline\" BIGINT, \"OriginAirportID\" BIGINT, \"OriginAirportSeqID\" BIGINT, \"OriginCityMarketID\" BIGINT, \"Origin\" VARCHAR, \"OriginCityName\" VARCHAR, \"OriginState\" VARCHAR, \"OriginStateFips\" BIGINT, \"OriginStateName\" VARCHAR, \"OriginWac\" BIGINT, \"DestAirportID\" BIGINT, \"DestAirportSeqID\" BIGINT, \"DestCityMarketID\" BIGINT, \"Dest\" VARCHAR, \"DestCityName\" VARCHAR, \"DestState\" VARCHAR, \"DestStateFips\" BIGINT, \"DestStateName\" VARCHAR, \"DestWac\" BIGINT, \"CRSDepTime\" BIGINT, \"DepTime\" BIGINT, \"DepDelay\" BIGINT, \"DepDelayMinutes\" BIGINT, \"DepDel15\" BIGINT, \"DepartureDelayGroups\" BIGINT, \"DepTimeBlk\" VARCHAR, \"TaxiOut\" BIGINT, \"WheelsOff\" BIGINT, \"WheelsOn\" BIGINT, \"TaxiIn\" BIGINT, \"CRSArrTime\" BIGINT, \"ArrTime\" BIGINT, \"ArrDelay\" BIGINT, \"ArrDelayMinutes\" BIGINT, \"ArrDel15\" BIGINT, \"ArrivalDelayGroups\" BIGINT, \"ArrTimeBlk\" VARCHAR, \"Cancelled\" BIGINT, \"CancellationCode\" VARCHAR, \"Diverted\" BIGINT, \"CRSElapsedTime\" BIGINT, \"ActualElapsedTime\" BIGINT, \"AirTime\" BIGINT, \"Flights\" BIGINT, \"Distance\" BIGINT, \"DistanceGroup\" BIGINT, \"CarrierDelay\" BIGINT, \"WeatherDelay\" BIGINT, \"NASDelay\" BIGINT, \"SecurityDelay\" BIGINT, \"LateAircraftDelay\" BIGINT, \"FirstDepTime\" VARCHAR, \"TotalAddGTime\" VARCHAR, \"LongestAddGTime\" VARCHAR, \"DivAirportLandings\" VARCHAR, \"DivReachedDest\" VARCHAR, \"DivActualElapsedTime\" VARCHAR, \"DivArrDelay\" VARCHAR, \"DivDistance\" VARCHAR, \"Div1Airport\" VARCHAR, \"Div1AirportID\" VARCHAR, \"Div1AirportSeqID\" VARCHAR, \"Div1WheelsOn\" VARCHAR, \"Div1TotalGTime\" VARCHAR, \"Div1LongestGTime\" VARCHAR, \"Div1WheelsOff\" VARCHAR, \"Div1TailNum\" VARCHAR, \"Div2Airport\" VARCHAR, \"Div2AirportID\" VARCHAR, \"Div2AirportSeqID\" VARCHAR, \"Div2WheelsOn\" VARCHAR, \"Div2TotalGTime\" VARCHAR, \"Div2LongestGTime\" VARCHAR, \"Div2WheelsOff\" VARCHAR, \"Div2TailNum\" VARCHAR, \"Div3Airport\" VARCHAR, \"Div3AirportID\" VARCHAR, \"Div3AirportSeqID\" VARCHAR, \"Div3WheelsOn\" VARCHAR, \"Div3TotalGTime\" VARCHAR, \"Div3LongestGTime\" VARCHAR, \"Div3WheelsOff\" VARCHAR, \"Div3TailNum\" VARCHAR, \"Div4Airport\" VARCHAR, \"Div4AirportID\" VARCHAR, \"Div4AirportSeqID\" VARCHAR, \"Div4WheelsOn\" VARCHAR, \"Div4TotalGTime\" VARCHAR, \"Div4LongestGTime\" VARCHAR, \"Div4WheelsOff\" VARCHAR, \"Div4TailNum\" VARCHAR, \"Div5Airport\" VARCHAR, \"Div5AirportID\" VARCHAR, \"Div5AirportSeqID\" VARCHAR, \"Div5WheelsOn\" VARCHAR, \"Div5TotalGTime\" VARCHAR, \"Div5LongestGTime\" VARCHAR, \"Div5WheelsOff\" VARCHAR, \"Div5TailNum\" VARCHAR, \"Unnamed: 109\" VARCHAR))\n",
+ "SELECT\n",
+ " TIME_PARSE(\"depaturetime\") AS \"__time\",\n",
+ " \"Reporting_Airline\",\n",
+ " \"Tail_Number\",\n",
+ " \"Distance\",\n",
+ " \"Origin\",\n",
+ " \"Flight_Number_Reporting_Airline\"\n",
+ "FROM \"ext\"\n",
+ "PARTITIONED BY DAY\n",
+ "'''\n",
+ "\n",
+ "sql_client.run_task(sql)\n",
+ "sql_client.wait_until_ready('example-flights-topn')\n",
+ "display.table('example-flights-topn')"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "00141575-29b4-440e-b23f-f7c6b237ef28",
+ "metadata": {},
+ "source": [
+ "When this is completed, run the following cell to load some Python libraries we need to explore what TopN does."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "f2a19226-6abc-436d-ac3c-9c04d6026707",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "import json\n",
+ "import matplotlib\n",
+ "import matplotlib.pyplot as plt\n",
+ "import pandas as pd"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "f388633f-195b-4381-98cc-7a2f80f48690",
+ "metadata": {},
+ "source": [
+ "## Example TopN style queries\n",
+ "\n",
+ "Druid looks for patterns in `SELECT` statements to determine if they would benefit from using approximation. A ranking query, like the following, matches the rules for TopN approximation, so Druid enables it by default.\n",
+ "\n",
+ "For Druid to automatically optimize for TopN, you need an SQL statement that has:\n",
+ "* A GROUP BY on one dimension, and\n",
+ "* an ORDER BY on one aggregate.\n",
+ "\n",
+ "Run this query to see what the results are like:"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "b76e5184-9fe4-4f21-a471-4e15d16515c8",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT\n",
+ " \"Reporting_Airline\",\n",
+ " COUNT(*) AS Flights,\n",
+ " SUM(\"Distance\") AS SumDistance\n",
+ "FROM\n",
+ " \"example-flights-topn\"\n",
+ "GROUP BY 1\n",
+ "ORDER BY 2 DESC\n",
+ "LIMIT 10\n",
+ "'''\n",
+ "display.sql(sql)"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "5600fc48-c999-406f-800b-3f0f6a973aa0",
+ "metadata": {},
+ "source": [
+ "Run the following cell, which uses the `explain_sql` method to show the [`EXPLAIN PLAN`](https://druid.apache.org/docs/latest/querying/sql-translation#interpreting-explain-plan-output) for this query."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "7595eec0-a709-4cd6-985e-eec8a6e37b61",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "print(json.dumps(json.loads(sql_client.explain_sql(sql)['PLAN']), indent=2))"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "8658b26e-2f09-4a97-96e8-589168130559",
+ "metadata": {},
+ "source": [
+ "The plan `queryType` is `topN`, showing that TopN approximation was used.\n",
+ "\n",
+ "Druid applied a `LIMIT` operation on the results calculated by each data service involved in the query, improving processing efficiency by minimizing the amount of data transferred to the Broker.\n",
+ "\n",
+ "This [pushed-down](https://druid.apache.org/docs/latest/querying/groupbyquery#limit-pushdown-optimization) `LIMIT` is the `max` of the `threshold` in the plan (which came from the `LIMIT` in the SQL) and the [`minTopNThreshold`](https://druid.apache.org/docs/latest/querying/topnquery.html#aliasing) setting in your cluster - the default being 1,000.\n",
+ "\n",
+ "To see the implication of this `LIMIT` in action, the cardinality of the `GROUP BY` dimension therefore needs to exceed this cap.\n",
+ "\n",
+ "Run the following query to discover the cardinality of the `GROUP BY` on `Reporting_Airline`."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "65a968e5-d51e-47e9-af04-88181f3b865b",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT COUNT (DISTINCT \"Reporting_Airline\") AS UniqueReportingAirlines\n",
+ "FROM \"example-flights-topn\"\n",
+ "'''\n",
+ "display.sql(sql)"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "13c8a101-dcba-49be-8d05-0a5dbd9731ca",
+ "metadata": {},
+ "source": [
+ "The number of unique values is below the `LIMIT` cap, meaning, there is no trimming and the results are not approximate; all the data servers will return all their results, without trimming, to be merged and passed back to us.\n",
+ "\n",
+ "What is the cardinality for the `Tail_Number` dimension?"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "71c1816b-f090-4a3d-b476-8d40eb9c2dec",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT\n",
+ " COUNT (DISTINCT \"Tail_Number\") AS UniqueTailNumbers\n",
+ "FROM \"example-flights-topn\"\n",
+ "WHERE \"Tail_Number\" <> ''\n",
+ "'''\n",
+ "display.sql(sql)"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "72a1a858-bda8-464e-988b-c4ed80b63f43",
+ "metadata": {},
+ "source": [
+ "With this many distinct values to `GROUP BY`, the services involved in the query will trim their results when using the\n",
+ "`topN` engine.\n",
+ "\n",
+ "Run the next query to visualize the distribution of unique `Tail_Number`s in the example dataset."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "208f1463-34dd-4b0e-aa78-e582e2133a8f",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT\n",
+ " \"Tail_Number\",\n",
+ " COUNT(*) AS RecordCount\n",
+ "FROM \"example-flights-topn\"\n",
+ "WHERE \"Tail_Number\" <> ''\n",
+ "GROUP BY 1\n",
+ "ORDER BY 2 DESC\n",
+ "LIMIT 500\n",
+ "'''\n",
+ "\n",
+ "df4 = pd.DataFrame(sql_client.sql(sql))\n",
+ "\n",
+ "df4.plot(x='Tail_Number', y='RecordCount', marker='o')\n",
+ "plt.xticks(rotation=45, ha='right')\n",
+ "plt.gca().get_legend().remove()\n",
+ "plt.show()"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "b16d9a4d-c9e7-447b-8cdb-7ad1c0f88d73",
+ "metadata": {},
+ "source": [
+ "The plot shows that we have a long tail distribution, meaning there is a high likelihood the same `Tail_Number` will be in rank position one across the data set, and therefore across all segments. The flatter the distribution, the less reliable this assertion is.\n",
+ "\n",
+ "Take a look at the following cell to see a query that counts the number of records and sums total distance for each `Tail_Number`.\n",
+ "\n",
+ "Run the cell to execute this query in both TopN and non-TopN modes. The first run puts the results into a Dataframe `df1` running `sql_client.sql(sql)` directly. The second uses a crafted `req` object that adds the `useApproximateTopN` query context parameter to turn off approximation, storing the results in `df2`.\n",
+ "\n",
+ "It then runs a `compare` of `df2` against `df1` using `df3` and prints the results."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "71db4746-4e8a-447e-aa58-f4c4ce3d7ffc",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT\n",
+ " \"Tail_Number\",\n",
+ " COUNT(*) AS \"count\",\n",
+ " SUM(Distance) AS \"distance\"\n",
+ "FROM \"example-flights-topn\"\n",
+ "WHERE \"Tail_Number\" IS NOT NULL\n",
+ "GROUP BY 1\n",
+ "ORDER BY 3 DESC\n",
+ "LIMIT 500\n",
+ "'''\n",
+ "\n",
+ "# Load the results into a pandas DataFrame\n",
+ "\n",
+ "df1 = pd.DataFrame(sql_client.sql(sql))\n",
+ "\n",
+ "# Set up a sql_request to turn off TopN approximation\n",
+ "\n",
+ "req = sql_client.sql_request(sql)\n",
+ "req.add_context(\"useApproximateTopN\", \"false\")\n",
+ "resp = sql_client.sql_query(req)\n",
+ "\n",
+ "# Load the non-TopN results into a second pandas DataFrame using that request\n",
+ "\n",
+ "df2 = pd.DataFrame(sql_client.sql_query(req).rows)\n",
+ "\n",
+ "# Load the compare of df1 to df2 into a new dataframe and print\n",
+ "\n",
+ "df3 = df1.compare(df2, keep_equal=True)\n",
+ "df3"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "78836242-acc8-4403-9e96-2177b96110ed",
+ "metadata": {},
+ "source": [
+ "You can see:\n",
+ "\n",
+ "* The `self` (df1) and `other` (df2) rank position of each `Tail_Number` in each position\n",
+ "* The self / other values for the calculated `count` and `distance`\n",
+ "\n",
+ "You may notice some `Tail_Number`s are in different positions depending on what the calculated `distance` is: certain data servers returned different sets of results, depending entirely on local data distribution. And some `Tail_Number`s may not appear in the list at all as they drop below the fold the cut-off applied to that specific process.\n",
+ "\n",
+ "Let's try this with a different dimension, `Flight_Number_Reporting_Airline`. The example dataset has more unique values, but the distribution is much flatter than `Tail_Number`. Run the following cell to see the count and a distribution plot."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "a96f924c-9fc1-4000-9a54-7a951db5d2bb",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT COUNT(DISTINCT \"Flight_Number_Reporting_Airline\") AS UniqueReportingAirlines\n",
+ "FROM \"example-flights-topn\"\n",
+ "WHERE \"Flight_Number_Reporting_Airline\" <> ''\n",
+ "'''\n",
+ "\n",
+ "display.sql(sql)\n",
+ "\n",
+ "sql = '''\n",
+ "SELECT \"Flight_Number_Reporting_Airline\", COUNT(*) AS Flights\n",
+ "FROM \"example-flights-topn\"\n",
+ "WHERE \"Flight_Number_Reporting_Airline\" <> ''\n",
+ "GROUP BY 1\n",
+ "ORDER BY 2 DESC\n",
+ "LIMIT 500\n",
+ "'''\n",
+ "\n",
+ "# Load the results into a pandas DataFrame\n",
+ "\n",
+ "df5 = pd.DataFrame(sql_client.sql(sql))\n",
+ "\n",
+ "# Display a plot\n",
+ "\n",
+ "df5.plot(x='Flight_Number_Reporting_Airline', y='Flights', kind=\"bar\", xticks=[])\n",
+ "plt.gca().get_legend().remove()\n",
+ "plt.show()"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "4866091d-e689-4209-8f6e-4edd526646e9",
+ "metadata": {},
+ "source": [
+ "This dimension, unlike `Tail_Number`, has a flatter distribution. Each data process is likely to have a flatter distribution of data, too, meaning the top ranking results are less prominent. The \"voting\" across the servers as to what is in the top is less clear.\n",
+ "\n",
+ "Run the following cell to repeat the same test we did before, creating two sets of results, and comparing them."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "405f7a9b-ac13-4c13-8e30-42058df4cbce",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT\n",
+ " \"Flight_Number_Reporting_Airline\",\n",
+ " AVG(\"Distance\") AS AverageDistance\n",
+ "FROM \"example-flights-topn\"\n",
+ "WHERE \"Flight_Number_Reporting_Airline\" IS NOT NULL\n",
+ "GROUP BY 1\n",
+ "ORDER BY 2 DESC\n",
+ "LIMIT 10\n",
+ "'''\n",
+ "\n",
+ "# Set up a sql_request to turn off TopN approximation\n",
+ "\n",
+ "req = sql_client.sql_request(sql)\n",
+ "req.add_context(\"useApproximateTopN\", \"false\")\n",
+ "resp = sql_client.sql_query(req)\n",
+ "\n",
+ "# Load two pandas DataFrames - one with the TopN and one with the non-TopN results\n",
+ "\n",
+ "df1 = pd.DataFrame(sql_client.sql(sql))\n",
+ "df2 = pd.DataFrame(sql_client.sql_query(req).rows)\n",
+ "\n",
+ "# Load the compare of df1 to df2 into a new dataframe and print\n",
+ "\n",
+ "df3 = df1.compare(df2, keep_equal=True)\n",
+ "df3"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "989a3e1c-cc8a-47c9-ad15-0b95fa00c7a6",
+ "metadata": {},
+ "source": [
+ "Here, the flatter distribution exaggerates ranking and calculation error. Further issues are caused by the calculation being non-additive.\n",
+ "\n",
+ "The following cell contains a query that is a good example of TopN being applied: it creates a list of `Tail_Number`s within a particular period of time. Imagine that you might use this list to provide an interactive filter on `Tail_Number` to the end user when they're looking at this specific time period.\n",
+ "\n",
+ "Run the following cell to show the cardinality of `Tail_Number`s in that period, and then to plot the distribution:"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "d039c393-96f4-4847-ac60-4414477ebc3b",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT COUNT (DISTINCT \"Tail_Number\") AS UniqueTailNumbers\n",
+ "FROM \"example-flights-topn\"\n",
+ "WHERE \"Tail_Number\" <> ''\n",
+ "AND (TIMESTAMP '2005-11-01' <= \"__time\" AND \"__time\" <= TIMESTAMP '2005-11-14')\n",
+ "'''\n",
+ "display.sql(sql)\n",
+ "\n",
+ "sql = '''\n",
+ "SELECT\n",
+ " \"Tail_Number\",\n",
+ " COUNT(*) AS \"Flights\"\n",
+ "FROM \"example-flights-topn\"\n",
+ "WHERE \"Tail_Number\" <> ''\n",
+ "AND (TIMESTAMP '2005-11-01' <= \"__time\" AND \"__time\" <= TIMESTAMP '2005-11-14')\n",
+ "GROUP BY 1\n",
+ "ORDER BY 2 DESC\n",
+ "LIMIT 500\n",
+ "'''\n",
+ "\n",
+ "df4 = pd.DataFrame(sql_client.sql(sql))\n",
+ "\n",
+ "df4.plot(x='Tail_Number', y='Flights', marker='o')\n",
+ "plt.xticks(rotation=45, ha='right')\n",
+ "plt.gca().get_legend().remove()\n",
+ "plt.show()"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "eeed8fa8-d1ce-41b2-955b-88fb0834ab36",
+ "metadata": {},
+ "source": [
+ "This distribution pattern is good for TopN - the highest ranking values are very prominent.\n",
+ "\n",
+ "Run the following cell to compare the two styles of execution:"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "d47d2017-1d89-4622-a42c-d86f29a774e1",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "SELECT \"Tail_Number\", COUNT(*) AS \"count\", SUM(Distance) AS \"distance\"\n",
+ " FROM \"example-flights-topn\"\n",
+ " WHERE \"Tail_Number\" IS NOT NULL\n",
+ " AND (TIMESTAMP '2005-11-01' <= \"__time\" AND \"__time\" <= TIMESTAMP '2005-11-14')\n",
+ " GROUP BY 1\n",
+ " ORDER BY 3 DESC\n",
+ " LIMIT 500\n",
+ "'''\n",
+ "\n",
+ "req = sql_client.sql_request(sql)\n",
+ "req.add_context(\"useApproximateTopN\", \"false\")\n",
+ "resp = sql_client.sql_query(req)\n",
+ "\n",
+ "df1 = pd.DataFrame(sql_client.sql(sql))\n",
+ "df2 = pd.DataFrame(sql_client.sql_query(req).rows)\n",
+ "\n",
+ "df3 = df1.compare(df2, keep_equal=True)\n",
+ "df3"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "f58a1846-5072-4495-b840-a620de3c0442",
+ "metadata": {},
+ "source": [
+ "The distribution, together with our filters, means that these results are useful for this kind of interactive UI element."
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "b43cd060-429e-4e84-b559-ad63624e7c14",
+ "metadata": {},
+ "source": [
+ "## Summary\n",
+ "\n",
+ "The speed boost we receive through TopN, at the expense of some accuracy, makes it useful for interactive elements like filters or initial lists of results that people will then deep dive into.\n",
+ "\n",
+ "* TopN is the default execution model for `GROUP BY` queries with one dimension, an `ORDER BY` and a `LIMIT` clause\n",
+ "* You can turn TopN off with a query context parameter\n",
+ "* Accuracy is highly dependent on distribution of the data, after filters etc., across the database"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "d08b8804-1051-4d38-88e7-2be1776934eb",
+ "metadata": {},
+ "source": [
+ "## Learn more\n",
+ "\n",
+ "Read the following documentation for more information:\n",
+ "\n",
+ "* [TopN queries](https://druid.apache.org/docs/latest/querying/topnquery)"
+ ]
+ }
+ ],
+ "metadata": {
+ "kernelspec": {
+ "display_name": "Python 3 (ipykernel)",
+ "language": "python",
+ "name": "python3"
+ },
+ "language_info": {
+ "codemirror_mode": {
+ "name": "ipython",
+ "version": 3
+ },
+ "file_extension": ".py",
+ "mimetype": "text/x-python",
+ "name": "python",
+ "nbconvert_exporter": "python",
+ "pygments_lexer": "ipython3",
+ "version": "3.11.4"
+ }
+ },
+ "nbformat": 4,
+ "nbformat_minor": 5
+}
diff --git a/examples/quickstart/jupyter-notebooks/notebooks/03-query/04-UnionOperations.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/03-query/04-UnionOperations.ipynb
new file mode 100644
index 000000000000..69fe16eafb6f
--- /dev/null
+++ b/examples/quickstart/jupyter-notebooks/notebooks/03-query/04-UnionOperations.ipynb
@@ -0,0 +1,509 @@
+{
+ "cells": [
+ {
+ "cell_type": "markdown",
+ "id": "557e06e8-9b35-4b34-8322-8a8ede6de709",
+ "metadata": {},
+ "source": [
+ "# Using `UNION ALL` to address multiple `TABLE`s in the same query\n",
+ "\n",
+ "\n",
+ " \n",
+ "While working with Druid, you may need to bring together two different tables of results together into a single result list, or to treat multiple tables as a single input to a query. This notebook introduces the `UNION ALL` operator, walking through two ways in which this operator can be used to achieve this result: top-level and table-level `UNION ALL`."
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "cf4554ae-6516-4e76-b202-d6e2fdf31603",
+ "metadata": {},
+ "source": [
+ "## Prerequisites\n",
+ "\n",
+ "This tutorial works with Druid 26.0.0 or later.\n",
+ "\n",
+ "#### Run using Docker\n",
+ "\n",
+ "Launch this tutorial and all prerequisites using the `druid-jupyter` profile of the Docker Compose file for Jupyter-based Druid tutorials. For more information, see [Docker for Jupyter Notebook tutorials](https://druid.apache.org/docs/latest/tutorials/tutorial-jupyter-docker.html).\n",
+ " \n",
+ "#### Run Druid without Docker\n",
+ "\n",
+ "If you do not use the Docker Compose environment, you need the following:\n",
+ "\n",
+ "* A running Druid instance, with a `DRUID_HOST` local environment variable containing the servername of your Druid router\n",
+ "* [druidapi](https://github.com/apache/druid/blob/master/examples/quickstart/jupyter-notebooks/druidapi/README.md), a Python client for Apache Druid. Follow the instructions in the Install section of the README file."
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "ee0c3171-def8-4ad9-9c56-d3a67f309631",
+ "metadata": {},
+ "source": [
+ "### Initialization\n",
+ "\n",
+ "Run the next cell to attempt a connection to Druid services. If successful, the output shows the Druid version number."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "9fa4abfe-f878-4031-88f2-94c13e922279",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "import druidapi\n",
+ "import os\n",
+ "\n",
+ "if 'DRUID_HOST' not in os.environ.keys():\n",
+ " druid_host=f\"http://localhost:8888\"\n",
+ "else:\n",
+ " druid_host=f\"http://{os.environ['DRUID_HOST']}:8888\"\n",
+ " \n",
+ "print(f\"Opening a connection to {druid_host}.\")\n",
+ "druid = druidapi.jupyter_client(druid_host)\n",
+ "\n",
+ "display = druid.display\n",
+ "sql_client = druid.sql\n",
+ "status_client = druid.status\n",
+ "\n",
+ "status_client.version"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "fc3001a0-27e5-4f41-876a-ce6eab2acd6a",
+ "metadata": {},
+ "source": [
+ "Finally, run the following cell to import the Python JSON module."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "6b058d8b-2bae-4929-ab0c-5a6df1850387",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "import json"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "f388633f-195b-4381-98cc-7a2f80f48690",
+ "metadata": {},
+ "source": [
+ "## Using Top-level `UNION ALL` to concatenate result sets\n",
+ "\n",
+ "Run the following cell to ingest the wikipedia data example. Once completed, you will see a description of the new table.\n",
+ "\n",
+ "You can optionally monitor the ingestion in the Druid console while it runs."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "a399196b-12db-42ff-ae24-c7232f150aba",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql='''\n",
+ "REPLACE INTO \"example-wikipedia-unionall\" OVERWRITE ALL\n",
+ "WITH \"ext\" AS (SELECT *\n",
+ "FROM TABLE(\n",
+ " EXTERN(\n",
+ " '{\"type\":\"http\",\"uris\":[\"https://druid.apache.org/data/wikipedia.json.gz\"]}',\n",
+ " '{\"type\":\"json\"}'\n",
+ " )\n",
+ ") EXTEND (\"isRobot\" VARCHAR, \"channel\" VARCHAR, \"timestamp\" VARCHAR, \"flags\" VARCHAR, \"isUnpatrolled\" VARCHAR, \"page\" VARCHAR, \"diffUrl\" VARCHAR, \"added\" BIGINT, \"comment\" VARCHAR, \"commentLength\" BIGINT, \"isNew\" VARCHAR, \"isMinor\" VARCHAR, \"delta\" BIGINT, \"isAnonymous\" VARCHAR, \"user\" VARCHAR, \"deltaBucket\" BIGINT, \"deleted\" BIGINT, \"namespace\" VARCHAR, \"cityName\" VARCHAR, \"countryName\" VARCHAR, \"regionIsoCode\" VARCHAR, \"metroCode\" BIGINT, \"countryIsoCode\" VARCHAR, \"regionName\" VARCHAR))\n",
+ "SELECT\n",
+ " TIME_PARSE(\"timestamp\") AS \"__time\",\n",
+ " \"isRobot\",\n",
+ " \"channel\",\n",
+ " \"page\",\n",
+ " \"commentLength\",\n",
+ " \"countryName\",\n",
+ " \"user\"\n",
+ "FROM \"ext\"\n",
+ "PARTITIONED BY DAY\n",
+ "'''\n",
+ "\n",
+ "sql_client.run_task(sql)\n",
+ "sql_client.wait_until_ready('example-wikipedia-unionall')\n",
+ "display.table('example-wikipedia-unionall')"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "24b47cc3-68f5-4a73-b374-94bbfa32d91d",
+ "metadata": {},
+ "source": [
+ "You can use `UNION ALL` to append the results of one query with another.\n",
+ "\n",
+ "The first query in the cell below, `set1`, returns the ten first edits to any \"fr\"-like `channel` between midday and 1pm on the 27th June 2016. The second query repeats this but for any \"en\"-like `channel`."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "b76e5184-9fe4-4f21-a471-4e15d16515c8",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "WITH\n",
+ "set1 AS (\n",
+ " SELECT\n",
+ " __time,\n",
+ " \"channel\",\n",
+ " \"page\",\n",
+ " \"isRobot\"\n",
+ " FROM \"example-wikipedia-unionall\"\n",
+ " WHERE DATE_TRUNC('HOUR', __time) = TIMESTAMP '2016-06-27 12:00:00'\n",
+ " AND channel LIKE '#fr%'\n",
+ " ORDER BY __time\n",
+ " LIMIT 10\n",
+ " ),\n",
+ "set2 AS (\n",
+ " SELECT\n",
+ " __time,\n",
+ " \"channel\",\n",
+ " \"page\",\n",
+ " \"isRobot\"\n",
+ " FROM \"example-wikipedia-unionall\"\n",
+ " WHERE DATE_TRUNC('HOUR', __time) = TIMESTAMP '2016-06-27 12:00:00'\n",
+ " AND channel LIKE '#en%'\n",
+ " ORDER BY __time\n",
+ " LIMIT 10\n",
+ " )\n",
+ " \n",
+ "SELECT * from set1\n",
+ "UNION ALL\n",
+ "SELECT * from set2\n",
+ "'''\n",
+ "\n",
+ "display.sql(sql)"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "f5e77fa9-a60c-4955-b763-58d970d7326d",
+ "metadata": {},
+ "source": [
+ "This is a [top-level](https://druid.apache.org/docs/latest/querying/sql.html#top-level) `UNION` operation. First, Druid calculated `set1` and appended subsequent results sets.\n",
+ "\n",
+ "Notice that these results are not in order by time – even though the individual sets did `ORDER BY` time. Druid simply concatenated the two result sets together.\n",
+ "\n",
+ "Optionally, run the next cell to show the precise [`EXPLAIN PLAN`](https://druid.apache.org/docs/latest/querying/sql-translation#interpreting-explain-plan-output) for the query. You can see there are two `query` execution plans, one for each subquery. Also, Druid's planning process optimized execution of the outer query."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "97934da2-17d1-4c91-8ae3-926cc89185c1",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "print(json.dumps(json.loads(sql_client.explain_sql(sql)['PLAN']), indent=2))"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "800add1a-d459-4796-b974-b2f094db417f",
+ "metadata": {},
+ "source": [
+ "Run next cell to perform another top-level UNION ALL, this time where the sets use `GROUP BY`.\n",
+ "\n",
+ "Notice that the aggregates have `AS` to set specific field names."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "8e687466-74bb-4cc0-ba17-913d1807fc60",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql='''\n",
+ "WITH\n",
+ "set1 AS (\n",
+ " SELECT\n",
+ " TIME_FLOOR(__time, 'PT1H') AS \"Period\",\n",
+ " countryName,\n",
+ " AVG(commentLength) AS \"Average Comment Size\",\n",
+ " COUNT(DISTINCT \"page\") AS \"Pages\"\n",
+ " FROM \"example-wikipedia-unionall\"\n",
+ " WHERE countryName='China'\n",
+ " GROUP BY 1, 2\n",
+ " LIMIT 10\n",
+ " ),\n",
+ "set2 AS (\n",
+ " SELECT\n",
+ " TIME_FLOOR(__time, 'PT1H') AS \"Episode\",\n",
+ " countryName,\n",
+ " COUNT(DISTINCT \"page\") AS \"Pages\",\n",
+ " AVG(commentLength) AS \"Average Comment Length\"\n",
+ " FROM \"example-wikipedia-unionall\"\n",
+ " WHERE countryName='Austria'\n",
+ " GROUP BY 1, 2\n",
+ " LIMIT 10\n",
+ " )\n",
+ "\n",
+ "SELECT * from set1\n",
+ "UNION ALL\n",
+ "SELECT * from set2\n",
+ "'''\n",
+ "\n",
+ "display.sql(sql)"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "f2c95ffc-b260-4671-bacc-c8cc3137e9c2",
+ "metadata": {},
+ "source": [
+ "Look carefully at these results - Druid has simply appended the results from `set2` to `set1` without introducing redundant columns.\n",
+ "\n",
+ "* Column name in `set2` (`Period` versus `Episode` and `Average Comment Size` versus `Average Comment Length`) did not result in new columns\n",
+ "* Columns with the same name (`Pages`) did not result in that aggregate being put into same column - Austria's values are simply appended `Average Comment Size`\n",
+ "\n",
+ "Run the next cell, which uses explicit column names at the top-level, rather than `*`, to ensure the calculations appear in the right columns in the final result. It also aliases the columns for the results by using `AS`."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "39f9be92-7b2e-417c-b16a-5060b8cd2c30",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql='''\n",
+ "WITH\n",
+ "set1 AS (\n",
+ " SELECT\n",
+ " TIME_FLOOR(__time, 'PT1H') AS \"Period\",\n",
+ " countryName,\n",
+ " AVG(commentLength) AS \"Average Comment Size\",\n",
+ " COUNT(DISTINCT \"page\") AS \"Pages\"\n",
+ " FROM \"example-wikipedia-unionall\"\n",
+ " WHERE countryName='China'\n",
+ " GROUP BY 1, 2\n",
+ " LIMIT 10\n",
+ " ),\n",
+ "set2 AS (\n",
+ " SELECT\n",
+ " TIME_FLOOR(__time, 'PT1H') AS \"Episode\",\n",
+ " countryName,\n",
+ " COUNT(DISTINCT \"page\") AS \"Pages\",\n",
+ " AVG(commentLength) AS \"Average Comment Length\"\n",
+ " FROM \"example-wikipedia-unionall\"\n",
+ " WHERE countryName='Austria'\n",
+ " GROUP BY 1, 2\n",
+ " LIMIT 10\n",
+ " )\n",
+ "\n",
+ "SELECT \"Period\", \"countryName\" AS \"Country\", \"Average Comment Size\" AS \"Edit Size\", \"Pages\" AS \"Unique Pages\" from set1\n",
+ "UNION ALL\n",
+ "SELECT \"Episode\", \"countryName\", \"Average Comment Length\", \"Pages\" from set2\n",
+ "'''\n",
+ "\n",
+ "display.sql(sql)"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "25001794-e1a7-4325-adb3-2b8f26036261",
+ "metadata": {},
+ "source": [
+ "## Using Table-level `UNION ALL` to work with multiple tables\n",
+ "\n",
+ "From one source of data, data engineers may create multiple `TABLE` datasources in order to:\n",
+ "\n",
+ "* Separate data with different levels of `__time` granularity (ie. the level of summarisation),\n",
+ "* Apply different security to different parts, for example, per tenant,\n",
+ "* Break up the data using filtering at ingestion time, for example, different tables for different HTTP error codes,\n",
+ "* Separate upstream data by the source device or system, for example, different types of IOT device,\n",
+ "* Isolate different periods of time, perhaps with different retention periods.\n",
+ "\n",
+ "You can use `UNION ALL` to access _all_ the source data, referencing all the `TABLE` datasources through a sub-query or a `FROM` clause.\n",
+ "\n",
+ "The next two cells create two new tables, `example-wikipedia-unionall-en` and `example-wikipedia-unionall-fr`. `example-wikipedia-unionall-en` contains only data for English language channel edits, while `example-wikipedia-unionall-fr` contains only French channels.\n",
+ "\n",
+ "Run the next two cells, monitoring the ingestion in the Druid Console as they run."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "269c6aef-c3a5-46ad-8332-30b7bf30ddfb",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql='''\n",
+ "REPLACE INTO \"example-wikipedia-unionall-en\" OVERWRITE ALL\n",
+ "WITH \"ext\" AS (SELECT *\n",
+ "FROM TABLE(\n",
+ " EXTERN(\n",
+ " '{\"type\":\"http\",\"uris\":[\"https://druid.apache.org/data/wikipedia.json.gz\"]}',\n",
+ " '{\"type\":\"json\"}'\n",
+ " )\n",
+ ") EXTEND (\"isRobot\" VARCHAR, \"channel\" VARCHAR, \"timestamp\" VARCHAR, \"flags\" VARCHAR, \"isUnpatrolled\" VARCHAR, \"page\" VARCHAR, \"diffUrl\" VARCHAR, \"added\" BIGINT, \"comment\" VARCHAR, \"commentLength\" BIGINT, \"isNew\" VARCHAR, \"isMinor\" VARCHAR, \"delta\" BIGINT, \"isAnonymous\" VARCHAR, \"user\" VARCHAR, \"deltaBucket\" BIGINT, \"deleted\" BIGINT, \"namespace\" VARCHAR, \"cityName\" VARCHAR, \"countryName\" VARCHAR, \"regionIsoCode\" VARCHAR, \"metroCode\" BIGINT, \"countryIsoCode\" VARCHAR, \"regionName\" VARCHAR))\n",
+ "SELECT\n",
+ " TIME_PARSE(\"timestamp\") AS \"__time\",\n",
+ " \"isRobot\",\n",
+ " \"channel\",\n",
+ " \"page\",\n",
+ " \"commentLength\",\n",
+ " \"countryName\",\n",
+ " \"user\"\n",
+ "FROM \"ext\"\n",
+ "WHERE \"channel\" LIKE '#en%'\n",
+ "PARTITIONED BY DAY\n",
+ "'''\n",
+ "\n",
+ "sql_client.run_task(sql)\n",
+ "sql_client.wait_until_ready('example-wikipedia-unionall-en')\n",
+ "display.table('example-wikipedia-unionall-en')"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "61740d61-28fc-48e9-b026-d472bd04f390",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql='''\n",
+ "REPLACE INTO \"example-wikipedia-unionall-fr\" OVERWRITE ALL\n",
+ "WITH \"ext\" AS (SELECT *\n",
+ "FROM TABLE(\n",
+ " EXTERN(\n",
+ " '{\"type\":\"http\",\"uris\":[\"https://druid.apache.org/data/wikipedia.json.gz\"]}',\n",
+ " '{\"type\":\"json\"}'\n",
+ " )\n",
+ ") EXTEND (\"isRobot\" VARCHAR, \"channel\" VARCHAR, \"timestamp\" VARCHAR, \"flags\" VARCHAR, \"isUnpatrolled\" VARCHAR, \"page\" VARCHAR, \"diffUrl\" VARCHAR, \"added\" BIGINT, \"comment\" VARCHAR, \"commentLength\" BIGINT, \"isNew\" VARCHAR, \"isMinor\" VARCHAR, \"delta\" BIGINT, \"isAnonymous\" VARCHAR, \"user\" VARCHAR, \"deltaBucket\" BIGINT, \"deleted\" BIGINT, \"namespace\" VARCHAR, \"cityName\" VARCHAR, \"countryName\" VARCHAR, \"regionIsoCode\" VARCHAR, \"metroCode\" BIGINT, \"countryIsoCode\" VARCHAR, \"regionName\" VARCHAR))\n",
+ "SELECT\n",
+ " TIME_PARSE(\"timestamp\") AS \"__time\",\n",
+ " \"isRobot\",\n",
+ " \"channel\",\n",
+ " \"page\",\n",
+ " \"commentLength\",\n",
+ " \"countryName\",\n",
+ " \"user\"\n",
+ "FROM \"ext\"\n",
+ "WHERE \"channel\" LIKE '#fr%'\n",
+ "PARTITIONED BY DAY\n",
+ "'''\n",
+ "\n",
+ "sql_client.run_task(sql)\n",
+ "sql_client.wait_until_ready('example-wikipedia-unionall-fr')\n",
+ "display.table('example-wikipedia-unionall-fr')"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "f8bbf2c6-681a-46f5-82f2-201cbbe8058d",
+ "metadata": {},
+ "source": [
+ "The next cell uses `UNION ALL` in a `WITH` statement that creates `unifiedSource`. This will be a unified source of data for both tables that can then be used in a `SELECT` query.\n",
+ "\n",
+ "Druid executes these \"[top level](https://druid.apache.org/docs/26.0.0/querying/sql.html#top-level)\" `UNION ALL` queries differently to \"[table level](https://druid.apache.org/docs/26.0.0/querying/sql.html#table-level)\" queries you have used so far. Table level `UNION ALL` makes use of `union` datasources, and it's important that you read the [documentation](https://druid.apache.org/docs/26.0.0/querying/datasource.html#union) to understand the functionality available to you. Operations such as filtering, for example, can only be done in the outer `SELECT` statement on `unifiedSource` in the sample query below. \n",
+ "\n",
+ "Run the following cell to count the number of robot and non-robot edits by channel across both sets."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "783fe77d-2e7b-476a-9748-67ea90c8bb91",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "sql = '''\n",
+ "WITH unifiedSource AS (\n",
+ " SELECT\n",
+ " \"__time\",\n",
+ " \"isRobot\",\n",
+ " \"channel\",\n",
+ " \"user\",\n",
+ " \"countryName\"\n",
+ " FROM \"example-wikipedia-unionall-en\"\n",
+ " UNION ALL\n",
+ " SELECT\n",
+ " \"__time\",\n",
+ " \"isRobot\",\n",
+ " \"channel\",\n",
+ " \"user\",\n",
+ " \"countryName\"\n",
+ " FROM \"example-wikipedia-unionall-fr\"\n",
+ " )\n",
+ "\n",
+ "SELECT\n",
+ " \"channel\",\n",
+ " COUNT(*) FILTER (WHERE isRobot=true) AS \"Robot Edits\",\n",
+ " COUNT (DISTINCT user) FILTER (WHERE isRobot=true) AS \"Robot Editors\",\n",
+ " COUNT(*) FILTER (WHERE isRobot=false) AS \"Human Edits\",\n",
+ " COUNT (DISTINCT user) FILTER (WHERE isRobot=false) AS \"Human Editors\"\n",
+ "FROM unifiedSource\n",
+ "GROUP BY 1\n",
+ "'''\n",
+ "\n",
+ "display.sql(sql)"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "f58a1846-5072-4495-b840-a620de3c0442",
+ "metadata": {},
+ "source": [
+ "## Conclusion\n",
+ "\n",
+ "* There are two modes for `UNION ALL` in Druid - top level and table level\n",
+ "* Top level is a simple concatenation, and operations must be done on the source `TABLE`s\n",
+ "* Table level uses a `union` data source, and operations must be done on the outer `SELECT`\n",
+ "\n",
+ "## Learn more\n",
+ "\n",
+ "* Watch [Plan your Druid table datasources](https://youtu.be/OpYDX4RYLV0?list=PLDZysOZKycN7MZvNxQk_6RbwSJqjSrsNR) by Peter Marshall\n",
+ "* Read about [union](https://druid.apache.org/docs/26.0.0/querying/datasource.html#union) datasources in the documentation\n",
+ "* Read the latest [documentation](https://druid.apache.org/docs/26.0.0/querying/sql.html#union-all) on the `UNION ALL` operator"
+ ]
+ }
+ ],
+ "metadata": {
+ "kernelspec": {
+ "display_name": "Python 3 (ipykernel)",
+ "language": "python",
+ "name": "python3"
+ },
+ "language_info": {
+ "codemirror_mode": {
+ "name": "ipython",
+ "version": 3
+ },
+ "file_extension": ".py",
+ "mimetype": "text/x-python",
+ "name": "python",
+ "nbconvert_exporter": "python",
+ "pygments_lexer": "ipython3",
+ "version": "3.11.4"
+ }
+ },
+ "nbformat": 4,
+ "nbformat_minor": 5
+}
diff --git a/examples/quickstart/jupyter-notebooks/notebooks/99-contributing/notebook-template.ipynb b/examples/quickstart/jupyter-notebooks/notebooks/99-contributing/notebook-template.ipynb
index 8f815693b3da..5fafa2a4a696 100644
--- a/examples/quickstart/jupyter-notebooks/notebooks/99-contributing/notebook-template.ipynb
+++ b/examples/quickstart/jupyter-notebooks/notebooks/99-contributing/notebook-template.ipynb
@@ -5,7 +5,7 @@
"id": "0cb3b009-ebde-4d56-9d59-a028d66d8309",
"metadata": {},
"source": [
- "# Title\n",
+ "# (Result) by (action) using (feature)\n",
"\n",
- "Introduction to Notebook\n",
- "Lorem Ipsum"
+ "\n",
+ "Introductory paragraph - for example:\n",
+ "\n",
+ "This tutorial demonstrates how to work with [feature](link to feature doc). In this tutorial you perform the following tasks:\n",
+ "\n",
+ "- Task 1\n",
+ "- Task 2\n",
+ "- Task 3\n",
+ "- etc\n",
+ "\n"
]
},
{
"cell_type": "markdown",
- "id": "bbdbf6ad-ca7b-40f5-8ca3-1070f4a3ee42",
+ "id": "b74aa63d-3d21-472d-8ade-8573ef3c50cf",
"metadata": {},
"source": [
- "## Prerequisites\n",
- "\n",
- "This tutorial works with Druid XX.0.0 or later.\n",
+ "## Table of contents\n",
"\n",
- "Launch this tutorial and all prerequisites using the `all-services` profile of the Docker Compose file for Jupyter-based Druid tutorials. For more information, see [Docker for Jupyter Notebook tutorials](https://druid.apache.org/docs/latest/tutorials/tutorial-jupyter-docker.html).\n"
+ "- [Prerequisites](#Prerequisites)\n",
+ "- [Initalization](#Initalization)\n",
+ "- [Next section](#Nextsection)\n",
+ "- etc"
]
},
{
"cell_type": "markdown",
- "id": "7ee6aef8-a11d-48d5-bcdc-e6231ba594b7",
+ "id": "bbdbf6ad-ca7b-40f5-8ca3-1070f4a3ee42",
"metadata": {},
"source": [
- " \n",
- "Run without Docker Compose \n",
- "\n",
+ "## Prerequisites\n",
+ "\n",
+ "This tutorial works with Druid XX.0.0 or later.\n",
+ "\n",
+ "#### Run with Docker\n",
+ "\n",
+ "\n",
+ "\n",
+ "Launch this tutorial and all prerequisites using the ....... profile of the Docker Compose file for Jupyter-based Druid tutorials. For more information, see [Docker for Jupyter Notebook tutorials](https://druid.apache.org/docs/latest/tutorials/tutorial-jupyter-docker.html).\n",
+ " \n",
+ "#### Run without Docker\n",
"\n",
- "In order to run this notebook you will need:\n",
+ "If you do not use the Docker Compose environment, you need the following:\n",
"\n",
- "Required Services\n",
- "* \n",
+ "* A running Apache Druid instance, with a `DRUID_HOST` local environment variable containing the server name of your Druid router\n",
+ "* [druidapi](https://github.com/apache/druid/blob/master/examples/quickstart/jupyter-notebooks/druidapi/README.md), a Python client for Apache Druid. Follow the instructions in the Install section of the README file.\n",
"\n",
- "Python packages\n",
- "* druidapi, a [Python client for Apache Druid](https://github.com/apache/druid/blob/master/examples/quickstart/jupyter-notebooks/druidapi/README.md)\n",
- "* \n",
- ""
+ " \n",
+ "* A running Apache Kafka instance, with a `KAFKA_HOST` local environment variable containing the broker server name.\n",
+ "* [matplotlib](https://matplotlib.org/), a library for creating visualizations in Python.\n",
+ "* [pandas](https://pandas.pydata.org/), a data analysis and manipulation tool."
]
},
{
@@ -65,7 +85,11 @@
"id": "5007a243-b81a-4601-8f57-5b14940abbff",
"metadata": {},
"source": [
- "### Initialization"
+ "### Initialization\n",
+ "\n",
+ "Run the next cell to set up the Druid Python client's connection to Apache Druid.\n",
+ "\n",
+ "If successful, the Druid version number will be shown in the output."
]
},
{
@@ -84,7 +108,23 @@
" druid_host=f\"http://{os.environ['DRUID_HOST']}:8888\"\n",
" \n",
"print(f\"Opening a connection to {druid_host}.\")\n",
- "druid = druidapi.jupyter_client(druid_host)"
+ "druid = druidapi.jupyter_client(druid_host)\n",
+ "\n",
+ "display = druid.display\n",
+ "sql_client = druid.sql\n",
+ "status_client = druid.status\n",
+ "\n",
+ "status_client.version"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "2efdbee0-62da-4fd3-84e1-f66b8c0150b3",
+ "metadata": {},
+ "source": [
+ "\n",
+ "\n",
+ "Run the next cell to set up the connection to Apache Kafka."
]
},
{
@@ -94,15 +134,165 @@
"metadata": {},
"outputs": [],
"source": [
- "# INCLUDE THIS CELL IF YOUR NOTEBOOK USES KAFKA \n",
- "# Use kafka_host variable when connecting to kafka \n",
- "import os\n",
- "\n",
"if 'KAFKA_HOST' not in os.environ.keys():\n",
" kafka_host=f\"http://localhost:9092\"\n",
"else:\n",
" kafka_host=f\"{os.environ['KAFKA_HOST']}:9092\""
]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "472589e4-1026-4b3b-bb79-eedabb2b44c4",
+ "metadata": {},
+ "source": [
+ "\n",
+ "\n",
+ "### Load example data\n",
+ "\n",
+ "Once your Druid environment is up and running, ingest the sample data for this tutorial.\n",
+ "\n",
+ "Run the following cell to create a table called `example-dataset-notebook`. Notice {the use of X as a timestamp | only required columns are ingested | WHERE / expressions / GROUP BY are front-loaded | partitions on X period and clusters by Y}.\n",
+ "\n",
+ "When completed, you'll see a description of the final table.\n",
+ "\n",
+ "\n",
+ "\n",
+ "Monitor the ingestion task process in the Druid console."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "f52a94fb-d2e4-403f-ab10-84d3af7bf2c8",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "# Replace `example-dataset-notebook` with your table name here.\n",
+ "# Remember to apply good data modelling practice to your INSERT / REPLACE.\n",
+ "\n",
+ "sql='''\n",
+ "'''\n",
+ "\n",
+ "sql_client.run_task(sql)\n",
+ "sql_client.wait_until_ready('example-dataset-notebook')\n",
+ "display.table('example-dataset-notebook')"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "9c3d6b39-6551-4b2a-bdfb-9606aa92c853",
+ "metadata": {},
+ "source": [
+ "\n",
+ "\n",
+ "Finally, run the following cell to import additional Python modules that you will use to X, Y, Z."
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "dc4c2524-0eba-4bc6-84ed-da3a25aa5fbe",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "# Add your modules here, remembering to align this with the prerequisites section\n",
+ "\n",
+ "import json\n",
+ "import matplotlib\n",
+ "import matplotlib.pyplot as plt\n",
+ "import pandas as pd"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "1b6c9b88-837d-4c80-a28d-36184ba63355",
+ "metadata": {},
+ "source": [
+ "## Awesome!\n",
+ "\n",
+ "The main body of your notebook goes here!\n",
+ "\n",
+ "### This is a step\n",
+ "\n",
+ "Here things get done\n",
+ "\n",
+ "### And so is this!\n",
+ "\n",
+ "Wow! Awesome!"
+ ]
+ },
+ {
+ "cell_type": "markdown",
+ "id": "54b8d5fe-ba85-4b5b-9669-0dd47dfbccd1",
+ "metadata": {},
+ "source": [
+ "## Summary\n",
+ "\n",
+ "* You learned this\n",
+ "* Remember this\n",
+ "\n",
+ "## Go further\n",
+ "\n",
+ "* Try this out on your own data\n",
+ "* Solve for problem X that is't covered here\n",
+ "\n",
+ "## Learn more\n",
+ "\n",
+ "* Read docs pages\n",
+ "* Watch or read something cool from the community\n",
+ "* Do some exploratory stuff on your own"
+ ]
+ },
+ {
+ "cell_type": "code",
+ "execution_count": null,
+ "id": "ca4d3362-b1a4-47a4-a782-9773c216b3ba",
+ "metadata": {},
+ "outputs": [],
+ "source": [
+ "# STANDARD CODE BLOCKS\n",
+ "\n",
+ "# When just wanting to display some SQL results\n",
+ "display.sql(sql)\n",
+ "\n",
+ "# When ingesting data:\n",
+ "sql_client.run_task(sql)\n",
+ "sql_client.wait_until_ready('wikipedia-en')\n",
+ "display.table('wikipedia-en')\n",
+ "\n",
+ "# When you want to make an EXPLAIN look pretty\n",
+ "print(json.dumps(json.loads(sql_client.explain_sql(sql)['PLAN']), indent=2))\n",
+ "\n",
+ "# When you want a simple plot\n",
+ "df = pd.DataFrame(sql_client.sql(sql))\n",
+ "df.plot(x='Tail_Number', y='Flights', marker='o')\n",
+ "plt.xticks(rotation=45, ha='right')\n",
+ "plt.gca().get_legend().remove()\n",
+ "plt.show()\n",
+ "\n",
+ "# When you want to add some query context parameters\n",
+ "req = sql_client.sql_request(sql)\n",
+ "req.add_context(\"useApproximateTopN\", \"false\")\n",
+ "resp = sql_client.sql_query(req)\n",
+ "\n",
+ "# When you want to compare two different sets of results\n",
+ "df3 = df1.compare(df2, keep_equal=True)\n",
+ "df3"
+ ]
}
],
"metadata": {
@@ -121,7 +311,7 @@
"name": "python",
"nbconvert_exporter": "python",
"pygments_lexer": "ipython3",
- "version": "3.11.4"
+ "version": "3.10.3"
}
},
"nbformat": 4,
diff --git a/examples/quickstart/tutorial/hadoop/docker/Dockerfile b/examples/quickstart/tutorial/hadoop/docker/Dockerfile
index fd71f5942fad..7739e2955456 100644
--- a/examples/quickstart/tutorial/hadoop/docker/Dockerfile
+++ b/examples/quickstart/tutorial/hadoop/docker/Dockerfile
@@ -14,7 +14,7 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-# Creates pseudo distributed hadoop 2.8.5 with java 8
+# Creates pseudo distributed hadoop 3.3.6 with java 8
FROM centos:7
USER root
@@ -56,32 +56,40 @@ ENV PATH $PATH:$JAVA_HOME/bin
# hadoop
ARG APACHE_ARCHIVE_MIRROR_HOST=https://archive.apache.org
-RUN curl -s ${APACHE_ARCHIVE_MIRROR_HOST}/dist/hadoop/core/hadoop-2.8.5/hadoop-2.8.5.tar.gz | tar -xz -C /usr/local/
-RUN cd /usr/local && ln -s ./hadoop-2.8.5 hadoop
+RUN curl -s ${APACHE_ARCHIVE_MIRROR_HOST}/dist/hadoop/core/hadoop-3.3.6/hadoop-3.3.6.tar.gz | tar -xz -C /usr/local/
+RUN cd /usr/local && ln -s ./hadoop-3.3.6 hadoop
-ENV HADOOP_PREFIX /usr/local/hadoop
+ENV HADOOP_HOME /usr/local/hadoop
ENV HADOOP_COMMON_HOME /usr/local/hadoop
ENV HADOOP_HDFS_HOME /usr/local/hadoop
ENV HADOOP_MAPRED_HOME /usr/local/hadoop
ENV HADOOP_YARN_HOME /usr/local/hadoop
ENV HADOOP_CONF_DIR /usr/local/hadoop/etc/hadoop
-ENV YARN_CONF_DIR $HADOOP_PREFIX/etc/hadoop
+ENV YARN_CONF_DIR $HADOOP_HOME/etc/hadoop
-RUN sed -i '/^export JAVA_HOME/ s:.*:export JAVA_HOME=/usr/lib/jvm/zulu8\nexport HADOOP_PREFIX=/usr/local/hadoop\nexport HADOOP_HOME=/usr/local/hadoop\n:' $HADOOP_PREFIX/etc/hadoop/hadoop-env.sh
-RUN sed -i '/^export HADOOP_CONF_DIR/ s:.*:export HADOOP_CONF_DIR=/usr/local/hadoop/etc/hadoop/:' $HADOOP_PREFIX/etc/hadoop/hadoop-env.sh
+# in hadoop 3 the example file is nearly empty so we can just append stuff
+RUN sed -i '$ a export JAVA_HOME=/usr/lib/jvm/zulu8' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
+RUN sed -i '$ a export HADOOP_HOME=/usr/local/hadoop' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
+RUN sed -i '$ a export HADOOP_CONF_DIR=/usr/local/hadoop/etc/hadoop/' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
+RUN sed -i '$ a export HDFS_NAMENODE_USER=root' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
+RUN sed -i '$ a export HDFS_DATANODE_USER=root' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
+RUN sed -i '$ a export HDFS_SECONDARYNAMENODE_USER=root' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
+RUN sed -i '$ a export YARN_RESOURCEMANAGER_USER=root' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
+RUN sed -i '$ a export YARN_NODEMANAGER_USER=root' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
-RUN mkdir $HADOOP_PREFIX/input
-RUN cp $HADOOP_PREFIX/etc/hadoop/*.xml $HADOOP_PREFIX/input
+RUN cat $HADOOP_HOME/etc/hadoop/hadoop-env.sh
+
+RUN mkdir $HADOOP_HOME/input
+RUN cp $HADOOP_HOME/etc/hadoop/*.xml $HADOOP_HOME/input
# pseudo distributed
-ADD core-site.xml.template $HADOOP_PREFIX/etc/hadoop/core-site.xml.template
+ADD core-site.xml.template $HADOOP_HOME/etc/hadoop/core-site.xml.template
RUN sed s/HOSTNAME/localhost/ /usr/local/hadoop/etc/hadoop/core-site.xml.template > /usr/local/hadoop/etc/hadoop/core-site.xml
-ADD hdfs-site.xml $HADOOP_PREFIX/etc/hadoop/hdfs-site.xml
-
-ADD mapred-site.xml $HADOOP_PREFIX/etc/hadoop/mapred-site.xml
-ADD yarn-site.xml $HADOOP_PREFIX/etc/hadoop/yarn-site.xml
+ADD hdfs-site.xml $HADOOP_HOME/etc/hadoop/hdfs-site.xml
+ADD mapred-site.xml $HADOOP_HOME/etc/hadoop/mapred-site.xml
+ADD yarn-site.xml $HADOOP_HOME/etc/hadoop/yarn-site.xml
-RUN $HADOOP_PREFIX/bin/hdfs namenode -format
+RUN $HADOOP_HOME/bin/hdfs namenode -format
ADD ssh_config /root/.ssh/config
RUN chmod 600 /root/.ssh/config
@@ -120,16 +128,16 @@ RUN echo -e \
/usr/local/bin/start_sshd && \
chmod a+x /usr/local/bin/start_sshd
-RUN start_sshd && $HADOOP_PREFIX/etc/hadoop/hadoop-env.sh && $HADOOP_PREFIX/sbin/start-dfs.sh && $HADOOP_PREFIX/bin/hdfs dfs -mkdir -p /user/root
-RUN start_sshd && $HADOOP_PREFIX/etc/hadoop/hadoop-env.sh && $HADOOP_PREFIX/sbin/start-dfs.sh && $HADOOP_PREFIX/bin/hdfs dfs -put $HADOOP_PREFIX/etc/hadoop/ input
+RUN start_sshd && $HADOOP_HOME/etc/hadoop/hadoop-env.sh && $HADOOP_HOME/sbin/start-dfs.sh
+RUN start_sshd && $HADOOP_HOME/etc/hadoop/hadoop-env.sh && $HADOOP_HOME/sbin/start-dfs.sh
CMD ["/etc/bootstrap.sh", "-d"]
# Hdfs ports
-EXPOSE 50010 50020 50070 50075 50090 8020 9000
+EXPOSE 8020 9000 9820 9864 9865 9866 9867 9868 9869 9870 9871 50010 50020 50070 50075 50090
# Mapred ports
EXPOSE 10020 19888
#Yarn ports
EXPOSE 8030 8031 8032 8033 8040 8042 8088
#Other ports
-EXPOSE 49707 2122
+EXPOSE 2122 49707
\ No newline at end of file
diff --git a/examples/quickstart/tutorial/hadoop/docker/bootstrap.sh b/examples/quickstart/tutorial/hadoop/docker/bootstrap.sh
old mode 100755
new mode 100644
index 053662b95245..d1fa493d4ea6
--- a/examples/quickstart/tutorial/hadoop/docker/bootstrap.sh
+++ b/examples/quickstart/tutorial/hadoop/docker/bootstrap.sh
@@ -15,23 +15,22 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-: ${HADOOP_PREFIX:=/usr/local/hadoop}
+: ${HADOOP_HOME:=/usr/local/hadoop}
-$HADOOP_PREFIX/etc/hadoop/hadoop-env.sh
+$HADOOP_HOME/etc/hadoop/hadoop-env.sh
rm /tmp/*.pid
-
# installing libraries if any - (resource urls added comma separated to the ACP system variable)
-cd $HADOOP_PREFIX/share/hadoop/common ; for cp in ${ACP//,/ }; do echo == $cp; curl -LO $cp ; done; cd -
+cd $HADOOP_HOME/share/hadoop/common ; for cp in ${ACP//,/ }; do echo == $cp; curl -LO $cp ; done; cd -
# altering the core-site configuration
sed s/HOSTNAME/$HOSTNAME/ /usr/local/hadoop/etc/hadoop/core-site.xml.template > /usr/local/hadoop/etc/hadoop/core-site.xml
start_sshd
-$HADOOP_PREFIX/sbin/start-dfs.sh
-$HADOOP_PREFIX/sbin/start-yarn.sh
-$HADOOP_PREFIX/sbin/mr-jobhistory-daemon.sh start historyserver
+$HADOOP_HOME/sbin/start-dfs.sh
+$HADOOP_HOME/sbin/start-yarn.sh
+$HADOOP_HOME/sbin/mr-jobhistory-daemon.sh start historyserver
if [[ $1 == "-d" ]]; then
while true; do sleep 1000; done
diff --git a/examples/quickstart/tutorial/hadoop3/docker/Dockerfile b/examples/quickstart/tutorial/hadoop3/docker/Dockerfile
deleted file mode 100644
index c0475651ee7f..000000000000
--- a/examples/quickstart/tutorial/hadoop3/docker/Dockerfile
+++ /dev/null
@@ -1,143 +0,0 @@
-# Based on the SequenceIQ hadoop-docker project hosted at
-# https://github.com/sequenceiq/hadoop-docker, and modified at
-# the Apache Software Foundation (ASF).
-#
-# Licensed 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.
-
-# Creates pseudo distributed hadoop 3.3.1 with java 8
-FROM centos:7
-
-USER root
-
-# install dev tools
-RUN yum clean all \
- && rpm --rebuilddb \
- && yum install -y curl which tar sudo openssh-server openssh-clients rsync yum-plugin-ovl\
- && yum clean all \
- && yum update -y libselinux \
- && yum update -y nss \
- && yum clean all
-# update libselinux. see https://github.com/sequenceiq/hadoop-docker/issues/14
-# update nss. see https://unix.stackexchange.com/questions/280548/curl-doesnt-connect-to-https-while-wget-does-nss-error-12286
-
-# passwordless ssh
-RUN ssh-keygen -q -N "" -t dsa -f /etc/ssh/ssh_host_dsa_key
-RUN ssh-keygen -q -N "" -t rsa -f /etc/ssh/ssh_host_rsa_key
-RUN ssh-keygen -q -N "" -t rsa -f /root/.ssh/id_rsa
-RUN cp /root/.ssh/id_rsa.pub /root/.ssh/authorized_keys
-
-#
-# Pull Zulu OpenJDK binaries from official repository:
-#
-
-ARG ZULU_REPO_VER=1.0.0-1
-
-RUN rpm --import http://repos.azulsystems.com/RPM-GPG-KEY-azulsystems && \
- curl -sLO https://cdn.azul.com/zulu/bin/zulu-repo-${ZULU_REPO_VER}.noarch.rpm && \
- rpm -ivh zulu-repo-${ZULU_REPO_VER}.noarch.rpm && \
- yum -q -y update && \
- yum -q -y upgrade && \
- yum -q -y install zulu8-jdk && \
- yum clean all && \
- rm -rf /var/cache/yum zulu-repo_${ZULU_REPO_VER}.noarch.rpm
-
-ENV JAVA_HOME=/usr/lib/jvm/zulu8
-ENV PATH $PATH:$JAVA_HOME/bin
-
-# hadoop
-ARG APACHE_ARCHIVE_MIRROR_HOST=https://archive.apache.org
-RUN curl -s ${APACHE_ARCHIVE_MIRROR_HOST}/dist/hadoop/core/hadoop-3.3.1/hadoop-3.3.1.tar.gz | tar -xz -C /usr/local/
-RUN cd /usr/local && ln -s ./hadoop-3.3.1 hadoop
-
-ENV HADOOP_HOME /usr/local/hadoop
-ENV HADOOP_COMMON_HOME /usr/local/hadoop
-ENV HADOOP_HDFS_HOME /usr/local/hadoop
-ENV HADOOP_MAPRED_HOME /usr/local/hadoop
-ENV HADOOP_YARN_HOME /usr/local/hadoop
-ENV HADOOP_CONF_DIR /usr/local/hadoop/etc/hadoop
-ENV YARN_CONF_DIR $HADOOP_HOME/etc/hadoop
-
-# in hadoop 3 the example file is nearly empty so we can just append stuff
-RUN sed -i '$ a export JAVA_HOME=/usr/lib/jvm/zulu8' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
-RUN sed -i '$ a export HADOOP_HOME=/usr/local/hadoop' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
-RUN sed -i '$ a export HADOOP_CONF_DIR=/usr/local/hadoop/etc/hadoop/' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
-RUN sed -i '$ a export HDFS_NAMENODE_USER=root' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
-RUN sed -i '$ a export HDFS_DATANODE_USER=root' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
-RUN sed -i '$ a export HDFS_SECONDARYNAMENODE_USER=root' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
-RUN sed -i '$ a export YARN_RESOURCEMANAGER_USER=root' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
-RUN sed -i '$ a export YARN_NODEMANAGER_USER=root' $HADOOP_HOME/etc/hadoop/hadoop-env.sh
-
-RUN cat $HADOOP_HOME/etc/hadoop/hadoop-env.sh
-
-RUN mkdir $HADOOP_HOME/input
-RUN cp $HADOOP_HOME/etc/hadoop/*.xml $HADOOP_HOME/input
-
-# pseudo distributed
-ADD core-site.xml.template $HADOOP_HOME/etc/hadoop/core-site.xml.template
-RUN sed s/HOSTNAME/localhost/ /usr/local/hadoop/etc/hadoop/core-site.xml.template > /usr/local/hadoop/etc/hadoop/core-site.xml
-ADD hdfs-site.xml $HADOOP_HOME/etc/hadoop/hdfs-site.xml
-ADD mapred-site.xml $HADOOP_HOME/etc/hadoop/mapred-site.xml
-ADD yarn-site.xml $HADOOP_HOME/etc/hadoop/yarn-site.xml
-
-RUN $HADOOP_HOME/bin/hdfs namenode -format
-
-ADD ssh_config /root/.ssh/config
-RUN chmod 600 /root/.ssh/config
-RUN chown root:root /root/.ssh/config
-
-# # installing supervisord
-# RUN yum install -y python-setuptools
-# RUN easy_install pip
-# RUN curl https://bitbucket.org/pypa/setuptools/raw/bootstrap/ez_setup.py -o - | python
-# RUN pip install supervisor
-#
-# ADD supervisord.conf /etc/supervisord.conf
-
-ADD bootstrap.sh /etc/bootstrap.sh
-RUN chown root:root /etc/bootstrap.sh
-RUN chmod 700 /etc/bootstrap.sh
-
-ENV BOOTSTRAP /etc/bootstrap.sh
-
-# workingaround docker.io build error
-RUN ls -la /usr/local/hadoop/etc/hadoop/*-env.sh
-RUN chmod +x /usr/local/hadoop/etc/hadoop/*-env.sh
-RUN ls -la /usr/local/hadoop/etc/hadoop/*-env.sh
-
-# Copy additional .jars to classpath
-RUN cp /usr/local/hadoop/share/hadoop/tools/lib/*.jar /usr/local/hadoop/share/hadoop/common/lib/
-
-# fix the 254 error code
-RUN sed -i "/^[^#]*UsePAM/ s/.*/#&/" /etc/ssh/sshd_config
-RUN echo "UsePAM no" >> /etc/ssh/sshd_config
-RUN echo "Port 2122" >> /etc/ssh/sshd_config
-
-# script for plain sshd start
-RUN echo -e \
- '#!/bin/bash\n/usr/sbin/sshd\ntimeout 10 bash -c "until printf \"\" 2>>/dev/null >>/dev/tcp/127.0.0.1/2122; do sleep 0.5; done"' > \
- /usr/local/bin/start_sshd && \
- chmod a+x /usr/local/bin/start_sshd
-
-RUN start_sshd && $HADOOP_HOME/etc/hadoop/hadoop-env.sh && $HADOOP_HOME/sbin/start-dfs.sh
-RUN start_sshd && $HADOOP_HOME/etc/hadoop/hadoop-env.sh && $HADOOP_HOME/sbin/start-dfs.sh
-
-CMD ["/etc/bootstrap.sh", "-d"]
-
-# Hdfs ports
-EXPOSE 8020 9000 9820 9864 9865 9866 9867 9868 9869 9870 9871 50010 50020 50070 50075 50090
-# Mapred ports
-EXPOSE 10020 19888
-#Yarn ports
-EXPOSE 8030 8031 8032 8033 8040 8042 8088
-#Other ports
-EXPOSE 2122 49707
\ No newline at end of file
diff --git a/examples/quickstart/tutorial/hadoop3/docker/bootstrap.sh b/examples/quickstart/tutorial/hadoop3/docker/bootstrap.sh
deleted file mode 100644
index d1fa493d4ea6..000000000000
--- a/examples/quickstart/tutorial/hadoop3/docker/bootstrap.sh
+++ /dev/null
@@ -1,41 +0,0 @@
-#!/bin/bash
-
-# 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.
-
-: ${HADOOP_HOME:=/usr/local/hadoop}
-
-$HADOOP_HOME/etc/hadoop/hadoop-env.sh
-
-rm /tmp/*.pid
-# installing libraries if any - (resource urls added comma separated to the ACP system variable)
-cd $HADOOP_HOME/share/hadoop/common ; for cp in ${ACP//,/ }; do echo == $cp; curl -LO $cp ; done; cd -
-
-# altering the core-site configuration
-sed s/HOSTNAME/$HOSTNAME/ /usr/local/hadoop/etc/hadoop/core-site.xml.template > /usr/local/hadoop/etc/hadoop/core-site.xml
-
-
-start_sshd
-$HADOOP_HOME/sbin/start-dfs.sh
-$HADOOP_HOME/sbin/start-yarn.sh
-$HADOOP_HOME/sbin/mr-jobhistory-daemon.sh start historyserver
-
-if [[ $1 == "-d" ]]; then
- while true; do sleep 1000; done
-fi
-
-if [[ $1 == "-bash" ]]; then
- /bin/bash
-fi
diff --git a/examples/quickstart/tutorial/hadoop3/docker/core-site.xml.template b/examples/quickstart/tutorial/hadoop3/docker/core-site.xml.template
deleted file mode 100644
index 256de4f0988f..000000000000
--- a/examples/quickstart/tutorial/hadoop3/docker/core-site.xml.template
+++ /dev/null
@@ -1,24 +0,0 @@
-
-
-
-
- fs.defaultFS
- hdfs://HOSTNAME:9000
-
-
diff --git a/examples/quickstart/tutorial/hadoop3/docker/hdfs-site.xml b/examples/quickstart/tutorial/hadoop3/docker/hdfs-site.xml
deleted file mode 100644
index 18758b97ff3c..000000000000
--- a/examples/quickstart/tutorial/hadoop3/docker/hdfs-site.xml
+++ /dev/null
@@ -1,32 +0,0 @@
-
-
-
-
- dfs.replication
- 1
-
-
- dfs.client.use.datanode.hostname
- true
-
-
- dfs.datanode.use.datanode.hostname
- true
-
-
diff --git a/examples/quickstart/tutorial/hadoop3/docker/mapred-site.xml b/examples/quickstart/tutorial/hadoop3/docker/mapred-site.xml
deleted file mode 100644
index 96aa1fa9ec8d..000000000000
--- a/examples/quickstart/tutorial/hadoop3/docker/mapred-site.xml
+++ /dev/null
@@ -1,24 +0,0 @@
-
-
-
-
- mapreduce.framework.name
- yarn
-
-
diff --git a/examples/quickstart/tutorial/hadoop3/docker/ssh_config b/examples/quickstart/tutorial/hadoop3/docker/ssh_config
deleted file mode 100644
index e9d0b917b533..000000000000
--- a/examples/quickstart/tutorial/hadoop3/docker/ssh_config
+++ /dev/null
@@ -1,20 +0,0 @@
-# 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.
-
-Host *
- UserKnownHostsFile /dev/null
- StrictHostKeyChecking no
- LogLevel quiet
- Port 2122
diff --git a/examples/quickstart/tutorial/hadoop3/docker/yarn-site.xml b/examples/quickstart/tutorial/hadoop3/docker/yarn-site.xml
deleted file mode 100644
index 68a0ecfa60c8..000000000000
--- a/examples/quickstart/tutorial/hadoop3/docker/yarn-site.xml
+++ /dev/null
@@ -1,65 +0,0 @@
-
-
-
-
- yarn.nodemanager.aux-services
- mapreduce_shuffle
-
-
-
- yarn.application.classpath
- /usr/local/hadoop/etc/hadoop, /usr/local/hadoop/share/hadoop/common/*, /usr/local/hadoop/share/hadoop/common/lib/*, /usr/local/hadoop/share/hadoop/hdfs/*, /usr/local/hadoop/share/hadoop/hdfs/lib/*, /usr/local/hadoop/share/hadoop/mapreduce/*, /usr/local/hadoop/share/hadoop/mapreduce/lib/*, /usr/local/hadoop/share/hadoop/yarn/*, /usr/local/hadoop/share/hadoop/yarn/lib/*
-
-
-
-
- Number of seconds after an application finishes before the nodemanager's
- DeletionService will delete the application's localized file directory
- and log directory.
-
- To diagnose Yarn application problems, set this property's value large
- enough (for example, to 600 = 10 minutes) to permit examination of these
- directories. After changing the property's value, you must restart the
- nodemanager in order for it to have an effect.
-
- The roots of Yarn applications' work directories is configurable with
- the yarn.nodemanager.local-dirs property (see below), and the roots
- of the Yarn applications' log directories is configurable with the
- yarn.nodemanager.log-dirs property (see also below).
-
- yarn.nodemanager.delete.debug-delay-sec
- 600
-
-
-
- yarn.log-aggregation-enable
- true
-
-
-
- yarn.log-aggregation.retain-seconds
- 900000
-
-
-
- yarn.nodemanager.vmem-check-enabled
- false
-
-
-
diff --git a/examples/quickstart/tutorial/wikipedia-index-hadoop.json b/examples/quickstart/tutorial/wikipedia-index-hadoop.json
deleted file mode 100644
index c727fcfcc38b..000000000000
--- a/examples/quickstart/tutorial/wikipedia-index-hadoop.json
+++ /dev/null
@@ -1,79 +0,0 @@
-{
- "type" : "index_hadoop",
- "spec" : {
- "dataSchema" : {
- "dataSource" : "wikipedia",
- "parser" : {
- "type" : "hadoopyString",
- "parseSpec" : {
- "format" : "json",
- "dimensionsSpec" : {
- "dimensions" : [
- "channel",
- "cityName",
- "comment",
- "countryIsoCode",
- "countryName",
- "isAnonymous",
- "isMinor",
- "isNew",
- "isRobot",
- "isUnpatrolled",
- "metroCode",
- "namespace",
- "page",
- "regionIsoCode",
- "regionName",
- "user",
- { "name": "added", "type": "long" },
- { "name": "deleted", "type": "long" },
- { "name": "delta", "type": "long" }
- ]
- },
- "timestampSpec" : {
- "format" : "auto",
- "column" : "time"
- }
- }
- },
- "metricsSpec" : [],
- "granularitySpec" : {
- "type" : "uniform",
- "segmentGranularity" : "day",
- "queryGranularity" : "none",
- "intervals" : ["2015-09-12/2015-09-13"],
- "rollup" : false
- }
- },
- "ioConfig" : {
- "type" : "hadoop",
- "inputSpec" : {
- "type" : "static",
- "paths" : "/quickstart/wikiticker-2015-09-12-sampled.json.gz"
- }
- },
- "tuningConfig" : {
- "type" : "hadoop",
- "partitionsSpec" : {
- "type" : "hashed",
- "targetPartitionSize" : 5000000
- },
- "forceExtendableShardSpecs" : true,
- "jobProperties" : {
- "fs.default.name" : "hdfs://druid-hadoop-demo:9000",
- "fs.defaultFS" : "hdfs://druid-hadoop-demo:9000",
- "dfs.datanode.address" : "druid-hadoop-demo",
- "dfs.client.use.datanode.hostname" : "true",
- "dfs.datanode.use.datanode.hostname" : "true",
- "yarn.resourcemanager.hostname" : "druid-hadoop-demo",
- "yarn.nodemanager.vmem-check-enabled" : "false",
- "mapreduce.map.java.opts" : "-Duser.timezone=UTC -Dfile.encoding=UTF-8",
- "mapreduce.job.user.classpath.first" : "true",
- "mapreduce.reduce.java.opts" : "-Duser.timezone=UTC -Dfile.encoding=UTF-8",
- "mapreduce.map.memory.mb" : 1024,
- "mapreduce.reduce.memory.mb" : 1024
- }
- }
- },
- "hadoopDependencyCoordinates": ["org.apache.hadoop:hadoop-client:2.8.5"]
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/pom.xml b/extensions-contrib/aliyun-oss-extensions/pom.xml
index d21409d641cd..43f7d558adb7 100644
--- a/extensions-contrib/aliyun-oss-extensions/pom.xml
+++ b/extensions-contrib/aliyun-oss-extensions/pom.xml
@@ -91,8 +91,8 @@
provided
- javax.validation
- validation-api
+ jakarta.validation
+ jakarta.validation-apiprovided
@@ -168,6 +168,13 @@
+
+ org.owasp
+ dependency-check-maven
+
+ true
+
+
diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml
index 8f8620586693..61400d13487b 100644
--- a/extensions-contrib/ambari-metrics-emitter/pom.xml
+++ b/extensions-contrib/ambari-metrics-emitter/pom.xml
@@ -145,6 +145,13 @@
+
+ org.owasp
+ dependency-check-maven
+
+ true
+
+
diff --git a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java b/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java
index 094f9f28b84d..eded587fb2ed 100644
--- a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java
+++ b/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java
@@ -24,10 +24,8 @@
import org.apache.commons.io.IOUtils;
import org.apache.druid.annotations.UsedByJUnitParamsRunner;
import org.apache.druid.jackson.DefaultObjectMapper;
-import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
-import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -37,14 +35,12 @@
import java.io.IOException;
import java.io.OutputStream;
-
@RunWith(JUnitParamsRunner.class)
public class WhiteListBasedDruidToTimelineEventConverterTest
{
private final String prefix = "druid";
private final WhiteListBasedDruidToTimelineEventConverter defaultWhiteListBasedDruidToTimelineEventConverter =
new WhiteListBasedDruidToTimelineEventConverter(prefix, "druid", null, new DefaultObjectMapper());
- private final DateTime createdTime = DateTimes.nowUtc();
private final String hostname = "testHost:8080";
private final String serviceName = "historical";
private final String defaultNamespace = prefix + "." + serviceName;
@@ -77,7 +73,7 @@ public void testDefaultIsInWhiteList(String key, boolean expectedValue)
ServiceMetricEvent event = ServiceMetricEvent
.builder()
.setFeed("metrics")
- .build(createdTime, key, 10)
+ .setMetric(key, 10)
.build(serviceName, hostname);
boolean isIn = defaultWhiteListBasedDruidToTimelineEventConverter.druidEventToTimelineMetric(event) != null;
@@ -118,7 +114,7 @@ public void testWhiteListedStringArrayDimension() throws IOException
ServiceMetricEvent event = new ServiceMetricEvent.Builder()
.setDimension("gcName", new String[] {"g1"})
- .build(createdTime, "jvm/gc/cpu", 10)
+ .setMetric("jvm/gc/cpu", 10)
.build(serviceName, hostname);
TimelineMetric metric = converter.druidEventToTimelineMetric(event);
@@ -136,14 +132,14 @@ private Object[] parametersForTestGetName()
.setDimension("status", "some_status")
.setDimension("numDimensions", "1")
.setDimension("segment", "dummy_segment")
- .build(createdTime, "query/segment/time/balabla/more", 10)
+ .setMetric("query/segment/time/balabla/more", 10)
.build(serviceName, hostname),
defaultNamespace + ".query/segment/time/balabla/more"
},
new Object[]{
new ServiceMetricEvent.Builder().setDimension("dataSource", "some_data_source")
.setDimension("tier", "_default_tier")
- .build(createdTime, "segment/max", 10)
+ .setMetric("segment/max", 10)
.build(serviceName, hostname),
null
},
@@ -158,7 +154,7 @@ private Object[] parametersForTestGetName()
.setDimension("remoteAddress", "194.0.90.2")
.setDimension("id", "ID")
.setDimension("context", "{context}")
- .build(createdTime, "query/time", 10)
+ .setMetric("query/time", 10)
.build(serviceName, hostname),
defaultNamespace + ".data-source.groupBy.query/time"
},
@@ -166,7 +162,7 @@ private Object[] parametersForTestGetName()
new ServiceMetricEvent.Builder().setDimension("dataSource", "data-source")
.setDimension("type", "groupBy")
.setDimension("some_random_dim1", "random_dim_value1")
- .build(createdTime, "ingest/persists/count", 10)
+ .setMetric("ingest/persists/count", 10)
.build(serviceName, hostname),
defaultNamespace + ".data-source.ingest/persists/count"
},
@@ -174,7 +170,7 @@ private Object[] parametersForTestGetName()
new ServiceMetricEvent.Builder().setDimension("bufferpoolName", "BufferPool")
.setDimension("type", "groupBy")
.setDimension("some_random_dim1", "random_dim_value1")
- .build(createdTime, "jvm/bufferpool/capacity", 10)
+ .setMetric("jvm/bufferpool/capacity", 10)
.build(serviceName, hostname),
null
}
diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml
index 1fb31cb20a60..458bb761376c 100644
--- a/extensions-contrib/cassandra-storage/pom.xml
+++ b/extensions-contrib/cassandra-storage/pom.xml
@@ -150,8 +150,8 @@
provided
- javax.validation
- validation-api
+ jakarta.validation
+ jakarta.validation-apiprovided
@@ -173,4 +173,15 @@
+
+
+
+ org.owasp
+ dependency-check-maven
+
+ true
+
+
+
+
diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml
index f24cab9357f8..de4466cca361 100644
--- a/extensions-contrib/cloudfiles-extensions/pom.xml
+++ b/extensions-contrib/cloudfiles-extensions/pom.xml
@@ -35,10 +35,7 @@
UTF-8
- 1.9.1
-
- 3.0
+ 2.5.0
@@ -91,8 +88,8 @@
guava
- javax.validation
- validation-api
+ jakarta.validation
+ jakarta.validation-apiprovided
@@ -151,8 +148,8 @@
test
- org.easymock
- easymock
+ org.mockito
+ mockito-coretest
@@ -163,4 +160,15 @@
+
+
+
+ org.owasp
+ dependency-check-maven
+
+ true
+
+
+
+
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesStorageDruidModule.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesStorageDruidModule.java
index c54342fd32de..224be81125f2 100644
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesStorageDruidModule.java
+++ b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesStorageDruidModule.java
@@ -32,7 +32,7 @@
import org.apache.druid.java.util.common.logger.Logger;
import org.jclouds.ContextBuilder;
import org.jclouds.logging.slf4j.config.SLF4JLoggingModule;
-import org.jclouds.openstack.v2_0.config.InternalUrlModule;
+import org.jclouds.openstack.keystone.catalog.config.InternalUrlModule;
import org.jclouds.osgi.ProviderRegistry;
import org.jclouds.rackspace.cloudfiles.uk.CloudFilesUKProviderMetadata;
import org.jclouds.rackspace.cloudfiles.us.CloudFilesUSProviderMetadata;
diff --git a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesByteSourceTest.java b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesByteSourceTest.java
index 6a07063c722e..89084affc05f 100644
--- a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesByteSourceTest.java
+++ b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesByteSourceTest.java
@@ -19,8 +19,6 @@
package org.apache.druid.storage.cloudfiles;
-import org.easymock.EasyMock;
-import org.easymock.EasyMockSupport;
import org.jclouds.io.Payload;
import org.junit.Assert;
import org.junit.Test;
@@ -28,30 +26,35 @@
import java.io.IOException;
import java.io.InputStream;
-public class CloudFilesByteSourceTest extends EasyMockSupport
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class CloudFilesByteSourceTest
{
@Test
public void openStreamTest() throws IOException
{
final String path = "path";
- CloudFilesObjectApiProxy objectApi = createMock(CloudFilesObjectApiProxy.class);
- CloudFilesObject cloudFilesObject = createMock(CloudFilesObject.class);
- Payload payload = createMock(Payload.class);
- InputStream stream = createMock(InputStream.class);
+ CloudFilesObjectApiProxy objectApi = mock(CloudFilesObjectApiProxy.class);
+ CloudFilesObject cloudFilesObject = mock(CloudFilesObject.class);
+ Payload payload = mock(Payload.class);
+ InputStream stream = mock(InputStream.class);
- EasyMock.expect(objectApi.get(path, 0)).andReturn(cloudFilesObject);
- EasyMock.expect(cloudFilesObject.getPayload()).andReturn(payload);
- EasyMock.expect(payload.openStream()).andReturn(stream);
+ when(objectApi.get(path, 0)).thenReturn(cloudFilesObject);
+ when(cloudFilesObject.getPayload()).thenReturn(payload);
+ when(payload.openStream()).thenReturn(stream);
payload.close();
- replayAll();
-
CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path);
Assert.assertEquals(stream, byteSource.openStream());
byteSource.closeStream();
- verifyAll();
+ verify(objectApi).get(path, 0);
+ verify(cloudFilesObject).getPayload();
+ verify(payload).openStream();
}
@Test()
@@ -59,18 +62,17 @@ public void openStreamWithRecoverableErrorTest() throws IOException
{
final String path = "path";
- CloudFilesObjectApiProxy objectApi = createMock(CloudFilesObjectApiProxy.class);
- CloudFilesObject cloudFilesObject = createMock(CloudFilesObject.class);
- Payload payload = createMock(Payload.class);
- InputStream stream = createMock(InputStream.class);
+ CloudFilesObjectApiProxy objectApi = mock(CloudFilesObjectApiProxy.class);
+ CloudFilesObject cloudFilesObject = mock(CloudFilesObject.class);
+ Payload payload = mock(Payload.class);
+ InputStream stream = mock(InputStream.class);
- EasyMock.expect(objectApi.get(path, 0)).andReturn(cloudFilesObject);
- EasyMock.expect(cloudFilesObject.getPayload()).andReturn(payload);
- EasyMock.expect(payload.openStream()).andThrow(new IOException()).andReturn(stream);
+ when(objectApi.get(path, 0)).thenReturn(cloudFilesObject);
+ when(cloudFilesObject.getPayload()).thenReturn(payload);
+ when(payload.openStream()).thenThrow(new IOException())
+ .thenReturn(stream);
payload.close();
- replayAll();
-
CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path);
try {
byteSource.openStream();
@@ -82,6 +84,8 @@ public void openStreamWithRecoverableErrorTest() throws IOException
Assert.assertEquals(stream, byteSource.openStream());
byteSource.closeStream();
- verifyAll();
+ verify(objectApi).get(path, 0);
+ verify(cloudFilesObject).getPayload();
+ verify(payload, times(2)).openStream();
}
}
diff --git a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusherTest.java b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusherTest.java
index 8a2e3f697eb3..6c08382548a1 100644
--- a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusherTest.java
+++ b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusherTest.java
@@ -24,7 +24,6 @@
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NoneShardSpec;
-import org.easymock.EasyMock;
import org.jclouds.openstack.swift.v1.features.ObjectApi;
import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
import org.junit.Assert;
@@ -36,6 +35,12 @@
import java.util.ArrayList;
import java.util.HashMap;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.atLeastOnce;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
/**
*/
public class CloudFilesDataSegmentPusherTest
@@ -46,16 +51,12 @@ public class CloudFilesDataSegmentPusherTest
@Test
public void testPush() throws Exception
{
- ObjectApi objectApi = EasyMock.createStrictMock(ObjectApi.class);
- EasyMock.expect(objectApi.put(EasyMock.anyString(), EasyMock.anyObject())).andReturn(null).atLeastOnce();
- EasyMock.replay(objectApi);
-
- CloudFilesApi api = EasyMock.createStrictMock(CloudFilesApi.class);
- EasyMock.expect(api.getObjectApi(EasyMock.anyString(), EasyMock.anyString()))
- .andReturn(objectApi)
- .atLeastOnce();
- EasyMock.replay(api);
+ ObjectApi objectApi = mock(ObjectApi.class);
+ when(objectApi.put(any(), any())).thenReturn(null);
+ CloudFilesApi api = mock(CloudFilesApi.class);
+ when(api.getObjectApi(any(), any()))
+ .thenReturn(objectApi);
CloudFilesDataSegmentPusherConfig config = new CloudFilesDataSegmentPusherConfig();
config.setRegion("region");
@@ -87,6 +88,7 @@ public void testPush() throws Exception
Assert.assertEquals(segmentToPush.getSize(), segment.getSize());
- EasyMock.verify(api);
+ verify(objectApi, atLeastOnce()).put(any(), any());
+ verify(api, atLeastOnce()).getObjectApi(any(), any());
}
}
diff --git a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesObjectApiProxyTest.java b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesObjectApiProxyTest.java
index eb3b61c21494..cc07e77a1210 100644
--- a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesObjectApiProxyTest.java
+++ b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesObjectApiProxyTest.java
@@ -19,8 +19,6 @@
package org.apache.druid.storage.cloudfiles;
-import org.easymock.EasyMock;
-import org.easymock.EasyMockSupport;
import org.jclouds.io.Payload;
import org.jclouds.openstack.swift.v1.domain.SwiftObject;
import org.jclouds.openstack.swift.v1.features.ObjectApi;
@@ -28,7 +26,11 @@
import org.junit.Assert;
import org.junit.Test;
-public class CloudFilesObjectApiProxyTest extends EasyMockSupport
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class CloudFilesObjectApiProxyTest
{
@Test
public void getTest()
@@ -37,16 +39,14 @@ public void getTest()
final String region = "region";
final String container = "container";
- CloudFilesApi cloudFilesApi = createMock(CloudFilesApi.class);
- ObjectApi objectApi = createMock(ObjectApi.class);
- SwiftObject swiftObject = createMock(SwiftObject.class);
- Payload payload = createMock(Payload.class);
-
- EasyMock.expect(cloudFilesApi.getObjectApi(region, container)).andReturn(objectApi);
- EasyMock.expect(objectApi.get(path)).andReturn(swiftObject);
- EasyMock.expect(swiftObject.getPayload()).andReturn(payload);
+ CloudFilesApi cloudFilesApi = mock(CloudFilesApi.class);
+ ObjectApi objectApi = mock(ObjectApi.class);
+ SwiftObject swiftObject = mock(SwiftObject.class);
+ Payload payload = mock(Payload.class);
- replayAll();
+ when(cloudFilesApi.getObjectApi(region, container)).thenReturn(objectApi);
+ when(objectApi.get(path)).thenReturn(swiftObject);
+ when(swiftObject.getPayload()).thenReturn(payload);
CloudFilesObjectApiProxy cfoApiProxy = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
CloudFilesObject cloudFilesObject = cfoApiProxy.get(path, 0);
@@ -56,6 +56,8 @@ public void getTest()
Assert.assertEquals(cloudFilesObject.getContainer(), container);
Assert.assertEquals(cloudFilesObject.getPath(), path);
- verifyAll();
+ verify(cloudFilesApi).getObjectApi(region, container);
+ verify(objectApi).get(path);
+ verify(swiftObject).getPayload();
}
}
diff --git a/extensions-contrib/compressed-bigdecimal/pom.xml b/extensions-contrib/compressed-bigdecimal/pom.xml
index 29a177e86850..5a7b278f8b3a 100644
--- a/extensions-contrib/compressed-bigdecimal/pom.xml
+++ b/extensions-contrib/compressed-bigdecimal/pom.xml
@@ -126,24 +126,32 @@
joda-timejoda-time
- 2.10.5providedcom.google.guavaguava
- 16.0.1providedcom.fasterxml.jackson.corejackson-core
- 2.10.2com.fasterxml.jackson.corejackson-annotations
- 2.10.2
+
+
+
+
+ org.owasp
+ dependency-check-maven
+
+ true
+
+
+
+
diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryBase.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryBase.java
index 201ceee405e3..da1dd19cfaa1 100644
--- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryBase.java
+++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryBase.java
@@ -128,9 +128,6 @@ public Comparator getComparator()
@Override
public abstract AggregateCombiner makeAggregateCombiner();
- @Override
- public abstract List getRequiredColumns();
-
@Override
public abstract String toString();
diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorFactory.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorFactory.java
index 47fbda0ec633..bce2bf257084 100644
--- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorFactory.java
+++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorFactory.java
@@ -28,9 +28,6 @@
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
-import java.util.Collections;
-import java.util.List;
-
public class CompressedBigDecimalMaxAggregatorFactory extends CompressedBigDecimalAggregatorFactoryBase
{
private static final byte CACHE_TYPE_ID = 0x37;
@@ -113,18 +110,6 @@ public AggregateCombiner makeAggregateCombiner()
return new CompressedBigDecimalMaxAggregateCombiner();
}
- @Override
- public List getRequiredColumns()
- {
- return Collections.singletonList(new CompressedBigDecimalMaxAggregatorFactory(
- fieldName,
- fieldName,
- size,
- scale,
- strictNumberParsing
- ));
- }
-
@Override
public String toString()
{
diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorFactory.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorFactory.java
index 8d3590a7329b..091af3453532 100644
--- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorFactory.java
+++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorFactory.java
@@ -29,8 +29,6 @@
import org.apache.druid.segment.ColumnValueSelector;
import javax.annotation.Nonnull;
-import java.util.Collections;
-import java.util.List;
/**
@@ -118,18 +116,6 @@ public AggregateCombiner makeAggregateCombiner()
return new CompressedBigDecimalMinAggregateCombiner();
}
- @Override
- public List getRequiredColumns()
- {
- return Collections.singletonList(new CompressedBigDecimalMinAggregatorFactory(
- name,
- fieldName,
- size,
- scale,
- strictNumberParsing
- ));
- }
-
@Override
public String toString()
{
diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorFactory.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorFactory.java
index bbe5d360813f..d7e08a2daf42 100644
--- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorFactory.java
+++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorFactory.java
@@ -30,8 +30,6 @@
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
-import java.util.Collections;
-import java.util.List;
public class CompressedBigDecimalSumAggregatorFactory extends CompressedBigDecimalAggregatorFactoryBase
{
@@ -118,18 +116,6 @@ public AggregateCombiner makeAggregateCombiner()
return new CompressedBigDecimalSumAggregateCombiner();
}
- @Override
- public List getRequiredColumns()
- {
- return Collections.singletonList(new CompressedBigDecimalSumAggregatorFactory(
- name,
- fieldName,
- size,
- scale,
- strictNumberParsing
- ));
- }
-
@Override
public String toString()
{
diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxFactoryTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxFactoryTest.java
index 0f1d548cec58..9eef55d4ee7e 100644
--- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxFactoryTest.java
+++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxFactoryTest.java
@@ -26,7 +26,6 @@
import java.io.IOException;
import java.math.BigDecimal;
-import java.util.Arrays;
public class CompressedBigDecimalMaxFactoryTest extends CompressedBigDecimalFactoryTestBase
@@ -50,10 +49,6 @@ public void testCompressedBigDecimalMaxAggregatorFactory()
Assert.assertEquals("5", aggregatorFactory.deserialize(new BigDecimal(5)).toString());
Assert.assertEquals("5.0", aggregatorFactory.deserialize(5d).toString());
Assert.assertEquals("5", aggregatorFactory.deserialize("5").toString());
- Assert.assertEquals(
- "[CompressedBigDecimalMaxAggregatorFactory{name='fieldName', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}]",
- Arrays.toString(aggregatorFactory.getRequiredColumns().toArray())
- );
Assert.assertNull(aggregatorFactory.combine(null, null));
Assert.assertEquals("4", aggregatorFactory.combine(new BigDecimal(4), null).toString());
Assert.assertEquals("4", aggregatorFactory.combine(null, new BigDecimal(4)).toString());
diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinFactoryTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinFactoryTest.java
index 1c445011ff6b..7a9aa0f82b09 100644
--- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinFactoryTest.java
+++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinFactoryTest.java
@@ -26,7 +26,6 @@
import java.io.IOException;
import java.math.BigDecimal;
-import java.util.Arrays;
public class CompressedBigDecimalMinFactoryTest extends CompressedBigDecimalFactoryTestBase
@@ -50,10 +49,7 @@ public void testCompressedBigDecimalMinAggregatorFactory()
Assert.assertEquals("5", aggregatorFactory.deserialize(new BigDecimal(5)).toString());
Assert.assertEquals("5.0", aggregatorFactory.deserialize(5d).toString());
Assert.assertEquals("5", aggregatorFactory.deserialize("5").toString());
- Assert.assertEquals(
- "[CompressedBigDecimalMinAggregatorFactory{name='name', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}]",
- Arrays.toString(aggregatorFactory.getRequiredColumns().toArray())
- );
+
// default is to initialize to
Assert.assertNull(aggregatorFactory.combine(null, null));
Assert.assertEquals("4", aggregatorFactory.combine(new BigDecimal(4), null).toString());
diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumFactoryTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumFactoryTest.java
index c26a2568f7f1..572772507b09 100644
--- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumFactoryTest.java
+++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumFactoryTest.java
@@ -26,7 +26,6 @@
import java.io.IOException;
import java.math.BigDecimal;
-import java.util.Arrays;
/**
* test CompressedBigDecimalSumFactory and various aggregators and combiner produced
@@ -52,10 +51,7 @@ public void testCompressedBigDecimalAggregatorFactory()
Assert.assertEquals("5", aggregatorFactory.deserialize(new BigDecimal(5)).toString());
Assert.assertEquals("5.0", aggregatorFactory.deserialize(5d).toString());
Assert.assertEquals("5", aggregatorFactory.deserialize("5").toString());
- Assert.assertEquals(
- "[CompressedBigDecimalSumAggregatorFactory{name='name', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}]",
- Arrays.toString(aggregatorFactory.getRequiredColumns().toArray())
- );
+
Assert.assertEquals("0", aggregatorFactory.combine(null, null).toString());
Assert.assertEquals("4", aggregatorFactory.combine(new BigDecimal(4), null).toString());
Assert.assertEquals("4", aggregatorFactory.combine(null, new BigDecimal(4)).toString());
diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml
index 6a66b6bc6c07..a0c1fb46cbb6 100644
--- a/extensions-contrib/distinctcount/pom.xml
+++ b/extensions-contrib/distinctcount/pom.xml
@@ -101,4 +101,15 @@
+
+
+
+ org.owasp
+ dependency-check-maven
+
+ true
+
+
+
+
diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java
index ce6f2f30bc9c..5d4b09ef4928 100644
--- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java
+++ b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java
@@ -141,14 +141,6 @@ public AggregatorFactory getCombiningFactory()
return new LongSumAggregatorFactory(name, name);
}
- @Override
- public List getRequiredColumns()
- {
- return Collections.singletonList(
- new DistinctCountAggregatorFactory(fieldName, fieldName, bitMapFactory)
- );
- }
-
@Override
public Object deserialize(Object object)
{
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
index 325d5701dedc..c7946255b07e 100644
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
+++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
@@ -25,6 +25,8 @@
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.FluentQueryRunner;
+import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
@@ -63,7 +65,6 @@ public class DistinctCountGroupByQueryTest extends InitializedNullHandlingTest
public void setup()
{
final GroupByQueryConfig config = new GroupByQueryConfig();
- config.setMaxIntermediateRows(10000);
this.resourceCloser = Closer.create();
this.factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(
config,
@@ -134,11 +135,13 @@ public void testGroupByWithDistinctCountAgg() throws Exception
.build();
final Segment incrementalIndexSegment = new IncrementalIndexSegment(index, null);
- Iterable results = GroupByQueryRunnerTestHelper.runQuery(
- factory,
- factory.createRunner(incrementalIndexSegment),
- query
- );
+ Iterable results = FluentQueryRunner
+ .create(factory.createRunner(incrementalIndexSegment), factory.getToolchest())
+ .applyPreMergeDecoration()
+ .mergeResults()
+ .applyPostMergeDecoration()
+ .run(QueryPlus.wrap(query))
+ .toList();
List expectedResults = Arrays.asList(
GroupByQueryRunnerTestHelper.createExpectedRow(
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
index ef1344c72bd0..c61a793ff837 100644
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
+++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
@@ -135,7 +135,7 @@ public void testTopNWithDistinctCountAgg() throws Exception
List> expectedResults = Collections.singletonList(
new Result<>(
time,
- new TopNResultValue(
+ TopNResultValue.create(
Arrays.