mirror of https://github.com/apache/druid.git
Merge remote-tracking branch 'upstream/master' into vectorize_earliest_num
This commit is contained in:
commit
5f65c42a51
31
.asf.yaml
31
.asf.yaml
|
@ -1,3 +1,34 @@
|
|||
# note: documentation can be found here https://cwiki.apache.org/confluence/display/INFRA/Git+-+.asf.yaml+features
|
||||
|
||||
github:
|
||||
description: 'Apache Druid: a high performance real-time analytics database.'
|
||||
homepage: https://druid.apache.org/
|
||||
|
||||
# Attempt to make the auto-generated github emails more easily readable in email clients.
|
||||
# Configuration adapted from https://github.com/apache/plc4x/blob/develop/.asf.yaml.
|
||||
custom_subjects:
|
||||
new_pr: "[PR] {title} ({repository})"
|
||||
close_pr: "Re: [PR] {title} ({repository})"
|
||||
comment_pr: "Re: [PR] {title} ({repository})"
|
||||
diffcomment: "Re: [PR] {title} ({repository})"
|
||||
merge_pr: "Re: [PR] {title} ({repository})"
|
||||
new_issue: "[I] {title} ({repository})"
|
||||
comment_issue: "Re: [I] {title} ({repository})"
|
||||
close_issue: "Re: [I] {title} ({repository})"
|
||||
catchall: "[GH] {title} ({repository})"
|
||||
new_discussion: "[D] {title} ({repository})"
|
||||
edit_discussion: "Re: [D] {title} ({repository})"
|
||||
close_discussion: "Re: [D] {title} ({repository})"
|
||||
close_discussion_with_comment: "Re: [D] {title} ({repository})"
|
||||
reopen_discussion: "Re: [D] {title} ({repository})"
|
||||
new_comment_discussion: "Re: [D] {title} ({repository})"
|
||||
edit_comment_discussion: "Re: [D] {title} ({repository})"
|
||||
delete_comment_discussion: "Re: [D] {title} ({repository})"
|
||||
|
||||
notifications:
|
||||
commits: commits@druid.apache.org
|
||||
issues: commits@druid.apache.org
|
||||
discussions: commits@druid.apache.org
|
||||
pullrequests_status: commits@druid.apache.org
|
||||
pullrequests_comment: commits@druid.apache.org
|
||||
pullrequests_bot_dependabot: commits@druid.apache.org
|
||||
|
|
|
@ -21,7 +21,30 @@ on:
|
|||
workflow_dispatch:
|
||||
|
||||
jobs:
|
||||
changes:
|
||||
runs-on: ubuntu-latest
|
||||
# Required permissions
|
||||
permissions:
|
||||
pull-requests: read
|
||||
# Set job outputs to values from filter step
|
||||
outputs:
|
||||
# run everything if not a PR
|
||||
core: ${{ steps.filter.outputs.core || github.event_name != 'pull_request'}}
|
||||
# the common extension in revised ITs is different from the one in standard ITs
|
||||
common-extensions: ${{ steps.filter.outputs.common-extensions }}
|
||||
steps:
|
||||
- uses: dorny/paths-filter@v2
|
||||
if: github.event_name == 'pull_request'
|
||||
id: filter
|
||||
with:
|
||||
filters: |
|
||||
common-extensions:
|
||||
- 'extension-core/(mysql-metadata-storage|druid-it-tools|druid-lookups-cached-global|druid-histogram|druid-datasketches|druid-parquet-extensions|druid-avro-extensions|druid-protobuf-extensions|druid-orc-extensions|druid-kafka-indexing-service|druid-s3-extensions|druid-multi-stage-query|druid-catalog)/**'
|
||||
core:
|
||||
- '!extension*/**'
|
||||
|
||||
it:
|
||||
needs: changes
|
||||
strategy:
|
||||
fail-fast: false
|
||||
matrix:
|
||||
|
@ -31,6 +54,7 @@ jobs:
|
|||
#indexer: [indexer, middleManager]
|
||||
indexer: [middleManager]
|
||||
uses: ./.github/workflows/reusable-revised-its.yml
|
||||
if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }}
|
||||
with:
|
||||
build_jdk: ${{ matrix.jdk }}
|
||||
runtime_jdk: ${{ matrix.jdk }}
|
||||
|
@ -40,7 +64,9 @@ jobs:
|
|||
mysql_driver: com.mysql.jdbc.Driver
|
||||
|
||||
s3-deep-storage-minio:
|
||||
needs: changes
|
||||
uses: ./.github/workflows/reusable-revised-its.yml
|
||||
if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }}
|
||||
with:
|
||||
build_jdk: 8
|
||||
runtime_jdk: 11
|
||||
|
|
|
@ -21,12 +21,35 @@ on:
|
|||
workflow_dispatch:
|
||||
|
||||
jobs:
|
||||
changes:
|
||||
runs-on: ubuntu-latest
|
||||
# Required permissions
|
||||
permissions:
|
||||
pull-requests: read
|
||||
# Set job outputs to values from filter step
|
||||
outputs:
|
||||
# run everything if not a PR
|
||||
core: ${{ steps.filter.outputs.core || github.event_name != 'pull_request'}}
|
||||
common-extensions: ${{ steps.filter.outputs.common-extensions }}
|
||||
steps:
|
||||
- uses: dorny/paths-filter@v2
|
||||
if: github.event_name == 'pull_request'
|
||||
id: filter
|
||||
with:
|
||||
filters: |
|
||||
common-extensions:
|
||||
- 'extension-core/(mysql-metadata-storage|druid-basic-security|simple-client-sslcontext|druid-testing-tools|druid-lookups-cached-global|druid-histogram|druid-datasketches|druid-parquet-extensions|druid-avro-extensions|druid-protobuf-extensions|druid-orc-extensions|druid-kafka-indexing-service|druid-s3-extensions)/**'
|
||||
core:
|
||||
- '!extension*/**'
|
||||
|
||||
integration-index-tests-middleManager:
|
||||
needs: changes
|
||||
strategy:
|
||||
fail-fast: false
|
||||
matrix:
|
||||
testing_group: [batch-index, input-format, input-source, perfect-rollup-parallel-batch-index, kafka-index, kafka-index-slow, kafka-transactional-index, kafka-transactional-index-slow, kafka-data-format, ldap-security, realtime-index, append-ingestion, compaction]
|
||||
uses: ./.github/workflows/reusable-standard-its.yml
|
||||
if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }}
|
||||
with:
|
||||
build_jdk: 8
|
||||
runtime_jdk: 8
|
||||
|
@ -35,11 +58,13 @@ jobs:
|
|||
group: ${{ matrix.testing_group }}
|
||||
|
||||
integration-index-tests-indexer:
|
||||
needs: changes
|
||||
strategy:
|
||||
fail-fast: false
|
||||
matrix:
|
||||
testing_group: [input-source, perfect-rollup-parallel-batch-index, kafka-index, kafka-transactional-index, kafka-index-slow, kafka-transactional-index-slow, kafka-data-format, append-ingestion, compaction]
|
||||
testing_group: [input-source, perfect-rollup-parallel-batch-index, kafka-index, append-ingestion, compaction]
|
||||
uses: ./.github/workflows/reusable-standard-its.yml
|
||||
if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }}
|
||||
with:
|
||||
build_jdk: 8
|
||||
runtime_jdk: 8
|
||||
|
@ -48,11 +73,13 @@ jobs:
|
|||
group: ${{ matrix.testing_group }}
|
||||
|
||||
integration-query-tests-middleManager:
|
||||
needs: changes
|
||||
strategy:
|
||||
fail-fast: false
|
||||
matrix:
|
||||
testing_group: [query, query-retry, query-error, security, high-availability]
|
||||
uses: ./.github/workflows/reusable-standard-its.yml
|
||||
if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }}
|
||||
with:
|
||||
build_jdk: 8
|
||||
runtime_jdk: 8
|
||||
|
@ -62,11 +89,13 @@ jobs:
|
|||
group: ${{ matrix.testing_group }}
|
||||
|
||||
integration-query-tests-middleManager-mariaDB:
|
||||
needs: changes
|
||||
strategy:
|
||||
fail-fast: false
|
||||
matrix:
|
||||
jdk: [8, 11]
|
||||
uses: ./.github/workflows/reusable-standard-its.yml
|
||||
if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }}
|
||||
with:
|
||||
build_jdk: 8
|
||||
runtime_jdk: ${{ matrix.jdk }}
|
||||
|
@ -77,11 +106,13 @@ jobs:
|
|||
group: query
|
||||
|
||||
integration-shuffle-deep-store-tests:
|
||||
needs: changes
|
||||
strategy:
|
||||
fail-fast: false
|
||||
matrix:
|
||||
indexer: [indexer, middleManager]
|
||||
uses: ./.github/workflows/reusable-standard-its.yml
|
||||
if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }}
|
||||
with:
|
||||
build_jdk: 8
|
||||
runtime_jdk: 8
|
||||
|
@ -91,7 +122,9 @@ jobs:
|
|||
group: shuffle deep store
|
||||
|
||||
integration-custom-coordinator-duties-tests:
|
||||
needs: changes
|
||||
uses: ./.github/workflows/reusable-standard-its.yml
|
||||
if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }}
|
||||
with:
|
||||
build_jdk: 8
|
||||
runtime_jdk: 8
|
||||
|
@ -101,7 +134,9 @@ jobs:
|
|||
group: custom coordinator duties
|
||||
|
||||
integration-k8s-leadership-tests:
|
||||
needs: changes
|
||||
name: (Compile=openjdk8, Run=openjdk8, Cluster Build On K8s) ITNestedQueryPushDownTest integration test
|
||||
if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.common-extensions == 'true' }}
|
||||
runs-on: ubuntu-22.04
|
||||
env:
|
||||
MVN: mvn --no-snapshot-updates
|
||||
|
|
|
@ -122,8 +122,25 @@ jobs:
|
|||
if: ${{ matrix.java == 'jdk8' }}
|
||||
run: ${MVN} spotbugs:check --fail-at-end -pl '!benchmarks'
|
||||
|
||||
intellij-inspections:
|
||||
strategy:
|
||||
fail-fast: false
|
||||
runs-on: ubuntu-latest
|
||||
steps:
|
||||
- name: checkout branch
|
||||
uses: actions/checkout@v3
|
||||
|
||||
- name: setup JDK8
|
||||
run: |
|
||||
echo "JAVA_HOME=$JAVA_HOME_8_X64" >> $GITHUB_ENV
|
||||
|
||||
- name: maven install
|
||||
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: intellij inspections
|
||||
if: ${{ matrix.java == 'jdk8' }}
|
||||
run: |
|
||||
docker run --rm \
|
||||
-v $(pwd):/project \
|
||||
|
|
|
@ -31,14 +31,44 @@ on:
|
|||
description: 'Flag to decide if next tests need to run incase coverage issue failures'
|
||||
value: |
|
||||
${{
|
||||
(jobs.indexing_modules_test.result == 'success' || fromJson(jobs.indexing_modules_test.outputs.coverage_failure)) &&
|
||||
(jobs.processing_modules_test.result == 'success' || fromJson(jobs.processing_modules_test.outputs.coverage_failure)) &&
|
||||
(jobs.server_modules_test.result == 'success' || fromJson(jobs.server_modules_test.outputs.coverage_failure)) &&
|
||||
(jobs.other_modules_test.result == 'success' || fromJson(jobs.other_modules_test.outputs.coverage_failure))
|
||||
(jobs.indexing_modules_test.result == 'success' || jobs.indexing_modules_test.result == 'skipped' ||
|
||||
fromJson(jobs.indexing_modules_test.outputs.coverage_failure)) &&
|
||||
(jobs.processing_modules_test.result == 'success' || jobs.processing_modules_test.result == 'skipped' ||
|
||||
fromJson(jobs.processing_modules_test.outputs.coverage_failure)) &&
|
||||
(jobs.server_modules_test.result == 'success' || jobs.server_modules_test.result == 'skipped' || fromJson
|
||||
(jobs.server_modules_test.outputs.coverage_failure)) &&
|
||||
(jobs.other_modules_test.result == 'success' || jobs.other_modules_test.result == 'skipped' || fromJson(jobs
|
||||
.other_modules_test.outputs.coverage_failure))
|
||||
}}
|
||||
|
||||
jobs:
|
||||
changes:
|
||||
runs-on: ubuntu-latest
|
||||
# Required permissions
|
||||
permissions:
|
||||
pull-requests: read
|
||||
# Set job outputs to values from filter step
|
||||
outputs:
|
||||
kafka: ${{ steps.filter.outputs.kafka }}
|
||||
kinesis: ${{ steps.filter.outputs.kinesis }}
|
||||
# run everything if not a PR
|
||||
core: ${{ steps.filter.outputs.core || github.event_name != 'pull_request'}}
|
||||
steps:
|
||||
- uses: dorny/paths-filter@v2
|
||||
if: github.event_name == 'pull_request'
|
||||
id: filter
|
||||
with:
|
||||
filters: |
|
||||
core:
|
||||
- '!extension*/**'
|
||||
kafka:
|
||||
- 'extensions-core/kafka-indexing-service/**'
|
||||
kinesis:
|
||||
- 'extensions-core/kinesis-indexing-service/**'
|
||||
|
||||
indexing_modules_test:
|
||||
needs: changes
|
||||
if: ${{ needs.changes.outputs.core == 'true' || needs.changes.outputs.kafka == 'true' || needs.changes.outputs.kinesis == 'true'}}
|
||||
uses: ./.github/workflows/reusable-unit-tests.yml
|
||||
with:
|
||||
jdk: ${{ inputs.jdk }}
|
||||
|
@ -47,6 +77,8 @@ jobs:
|
|||
maven_projects: 'indexing-hadoop,indexing-service,extensions-core/kafka-indexing-service,extensions-core/kinesis-indexing-service'
|
||||
|
||||
processing_modules_test:
|
||||
needs: changes
|
||||
if: ${{ needs.changes.outputs.core == 'true' }}
|
||||
uses: ./.github/workflows/reusable-unit-tests.yml
|
||||
with:
|
||||
jdk: ${{ inputs.jdk }}
|
||||
|
@ -55,6 +87,8 @@ jobs:
|
|||
maven_projects: 'processing'
|
||||
|
||||
server_modules_test:
|
||||
needs: changes
|
||||
if: ${{ needs.changes.outputs.core == 'true' }}
|
||||
uses: ./.github/workflows/reusable-unit-tests.yml
|
||||
with:
|
||||
jdk: ${{ inputs.jdk }}
|
||||
|
|
|
@ -35,7 +35,7 @@ integration-tests/gen-scripts/
|
|||
**/.local/
|
||||
**/druidapi.egg-info/
|
||||
examples/quickstart/jupyter-notebooks/docker-jupyter/notebooks
|
||||
|
||||
website/i18n/*
|
||||
# ignore NetBeans IDE specific files
|
||||
nbproject
|
||||
nbactions.xml
|
||||
|
|
6
LICENSE
6
LICENSE
|
@ -279,6 +279,12 @@ SOURCE/JAVA-CORE
|
|||
This product contains lpad and rpad methods adapted from Apache Flink.
|
||||
* processing/src/main/java/org/apache/druid/java/util/common/StringUtils.java
|
||||
|
||||
This product contains SystemInfo methods adapted from oshi
|
||||
* processing/src/main/java/org/apache/druid/java/util/metrics/OshiSysMonitor.java
|
||||
|
||||
This product contains test cases adapted from Test Framework for Apache Drill (https://github.com/apache/drill-test-framework).
|
||||
* sql/src/test/resources/drill/window
|
||||
|
||||
|
||||
MIT License
|
||||
================================
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.druid.segment.data.GenericIndexed;
|
|||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.filter.BoundFilter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -161,10 +161,6 @@ public class BoundFilterBenchmark
|
|||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance();
|
||||
final List<Integer> ints = generateInts();
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints).transform(Object::toString),
|
||||
GenericIndexed.STRING_STRATEGY
|
||||
);
|
||||
final GenericIndexed<ImmutableBitmap> bitmaps = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(
|
||||
|
@ -183,7 +179,7 @@ public class BoundFilterBenchmark
|
|||
);
|
||||
selector = new MockColumnIndexSelector(
|
||||
bitmapFactory,
|
||||
new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null)
|
||||
new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -65,6 +65,7 @@ public class DataSketchesHllBenchmark
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
false
|
||||
);
|
||||
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.druid.segment.column.StringValueSetIndex;
|
|||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -93,11 +93,6 @@ public class DictionaryEncodedStringIndexSupplierBenchmark
|
|||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance();
|
||||
final Iterable<Integer> ints = intGenerator();
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(Object::toString),
|
||||
GenericIndexed.STRING_STRATEGY
|
||||
);
|
||||
final GenericIndexed<ByteBuffer> dictionaryUtf8 = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))),
|
||||
|
@ -115,8 +110,8 @@ public class DictionaryEncodedStringIndexSupplierBenchmark
|
|||
.iterator(),
|
||||
serdeFactory.getObjectStrategy()
|
||||
);
|
||||
DictionaryEncodedStringIndexSupplier indexSupplier =
|
||||
new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null);
|
||||
StringUtf8ColumnIndexSupplier<?> indexSupplier =
|
||||
new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null);
|
||||
stringValueSetIndex = (IndexedUtf8ValueSetIndex<?>) indexSupplier.as(StringValueSetIndex.class);
|
||||
List<Integer> filterValues = new ArrayList<>();
|
||||
List<Integer> nonFilterValues = new ArrayList<>();
|
||||
|
|
|
@ -38,7 +38,7 @@ import org.apache.druid.segment.data.GenericIndexed;
|
|||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.filter.DimensionPredicateFilter;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -122,11 +122,6 @@ public class DimensionPredicateFilterBenchmark
|
|||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance();
|
||||
final List<Integer> ints = generateInts();
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(Object::toString),
|
||||
GenericIndexed.STRING_STRATEGY
|
||||
);
|
||||
final GenericIndexed<ByteBuffer> dictionaryUtf8 = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))),
|
||||
|
@ -145,7 +140,7 @@ public class DimensionPredicateFilterBenchmark
|
|||
);
|
||||
selector = new MockColumnIndexSelector(
|
||||
bitmapFactory,
|
||||
new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null)
|
||||
new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -137,11 +137,6 @@ public class FilterPartitionBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -140,11 +140,6 @@ public class FilteredAggregatorBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -158,11 +158,6 @@ public class GroupByTypeInterfaceBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.druid.segment.data.BitmapSerdeFactory;
|
|||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -85,11 +85,6 @@ public class InFilterBenchmark
|
|||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance();
|
||||
final Iterable<Integer> ints = intGenerator();
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(Object::toString),
|
||||
GenericIndexed.STRING_STRATEGY
|
||||
);
|
||||
final GenericIndexed<ByteBuffer> dictionaryUtf8 = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))),
|
||||
|
@ -109,7 +104,7 @@ public class InFilterBenchmark
|
|||
);
|
||||
selector = new MockColumnIndexSelector(
|
||||
bitmapFactory,
|
||||
new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null)
|
||||
new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null)
|
||||
);
|
||||
inFilter = new InDimFilter(
|
||||
"dummy",
|
||||
|
|
|
@ -95,9 +95,6 @@ public class JoinAndLookupBenchmark
|
|||
@Param({"500000"})
|
||||
int rows;
|
||||
|
||||
@Param({"0", "16384"})
|
||||
int columnCacheSizeBytes;
|
||||
|
||||
private File tmpDir = null;
|
||||
private QueryableIndex index = null;
|
||||
private Segment baseSegment = null;
|
||||
|
@ -123,8 +120,7 @@ public class JoinAndLookupBenchmark
|
|||
public void setup() throws IOException
|
||||
{
|
||||
tmpDir = FileUtils.createTempDir();
|
||||
ColumnConfig columnConfig = () -> columnCacheSizeBytes;
|
||||
index = JoinTestHelper.createFactIndexBuilder(columnConfig, tmpDir, rows).buildMMappedIndex();
|
||||
index = JoinTestHelper.createFactIndexBuilder(ColumnConfig.DEFAULT, tmpDir, rows).buildMMappedIndex();
|
||||
|
||||
final String prefix = "c.";
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.druid.segment.data.BitmapSerdeFactory;
|
|||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.filter.Filters;
|
||||
import org.apache.druid.segment.serde.DictionaryEncodedStringIndexSupplier;
|
||||
import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -122,11 +122,6 @@ public class LikeFilterBenchmark
|
|||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = RoaringBitmapSerdeFactory.getInstance();
|
||||
final List<Integer> ints = generateInts();
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(Object::toString),
|
||||
GenericIndexed.STRING_STRATEGY
|
||||
);
|
||||
final GenericIndexed<ByteBuffer> dictionaryUtf8 = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(i -> ByteBuffer.wrap(StringUtils.toUtf8(String.valueOf(i)))),
|
||||
|
@ -145,7 +140,7 @@ public class LikeFilterBenchmark
|
|||
);
|
||||
selector = new MockColumnIndexSelector(
|
||||
bitmapFactory,
|
||||
new DictionaryEncodedStringIndexSupplier(bitmapFactory, dictionary, dictionaryUtf8, bitmaps, null)
|
||||
new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null)
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -138,11 +138,6 @@ public class TopNTypeInterfaceBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.LongsColumn;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
@ -118,10 +119,7 @@ public class ColumnarLongsEncodeDataFromSegmentBenchmark extends BaseColumnarLon
|
|||
File dataFile = new File(dir, getColumnDataFileName(segmentName, columnName));
|
||||
|
||||
if (!dataFile.exists()) {
|
||||
final IndexIO indexIO = new IndexIO(
|
||||
new DefaultObjectMapper(),
|
||||
() -> 0
|
||||
);
|
||||
final IndexIO indexIO = new IndexIO(new DefaultObjectMapper(), ColumnConfig.DEFAULT);
|
||||
try (final QueryableIndex index = indexIO.loadIndex(new File(segmentPath))) {
|
||||
final Set<String> columnNames = new LinkedHashSet<>();
|
||||
columnNames.add(ColumnHolder.TIME_COLUMN_NAME);
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.druid.segment.IndexIO;
|
|||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.generator.DataGenerator;
|
||||
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
|
||||
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
|
||||
|
@ -105,10 +106,7 @@ public class IndexMergeBenchmark
|
|||
InjectableValues.Std injectableValues = new InjectableValues.Std();
|
||||
injectableValues.addValue(ExprMacroTable.class, ExprMacroTable.nil());
|
||||
JSON_MAPPER.setInjectableValues(injectableValues);
|
||||
INDEX_IO = new IndexIO(
|
||||
JSON_MAPPER,
|
||||
() -> 0
|
||||
);
|
||||
INDEX_IO = new IndexIO(JSON_MAPPER, ColumnConfig.DEFAULT);
|
||||
}
|
||||
|
||||
@Setup
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
|||
import org.apache.druid.segment.IndexIO;
|
||||
import org.apache.druid.segment.IndexMergerV9;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.generator.DataGenerator;
|
||||
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
|
||||
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
|
||||
|
@ -74,10 +75,7 @@ public class IndexPersistBenchmark
|
|||
static {
|
||||
NullHandling.initializeForTests();
|
||||
JSON_MAPPER = new DefaultObjectMapper();
|
||||
INDEX_IO = new IndexIO(
|
||||
JSON_MAPPER,
|
||||
() -> 0
|
||||
);
|
||||
INDEX_IO = new IndexIO(JSON_MAPPER, ColumnConfig.DEFAULT);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
}
|
||||
|
||||
|
|
|
@ -174,11 +174,6 @@ public class GroupByBenchmark
|
|||
),
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -64,6 +64,7 @@ import org.apache.druid.segment.IndexMergerV9;
|
|||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.QueryableIndexSegment;
|
||||
import org.apache.druid.segment.column.ColumnConfig;
|
||||
import org.apache.druid.segment.generator.DataGenerator;
|
||||
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
|
||||
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
|
||||
|
@ -136,10 +137,7 @@ public class ScanBenchmark
|
|||
|
||||
static {
|
||||
JSON_MAPPER = new DefaultObjectMapper();
|
||||
INDEX_IO = new IndexIO(
|
||||
JSON_MAPPER,
|
||||
() -> 0
|
||||
);
|
||||
INDEX_IO = new IndexIO(JSON_MAPPER, ColumnConfig.DEFAULT);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
}
|
||||
|
||||
|
|
|
@ -143,11 +143,6 @@ public class SearchBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -32,8 +32,10 @@ import org.apache.druid.java.util.common.logger.Logger;
|
|||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchApproxCountDistinctSqlAggregator;
|
||||
import org.apache.druid.query.aggregation.datasketches.hll.sql.HllSketchApproxCountDistinctUtf8SqlAggregator;
|
||||
import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchApproxQuantileSqlAggregator;
|
||||
import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchObjectSqlAggregator;
|
||||
import org.apache.druid.query.aggregation.datasketches.theta.sql.ThetaSketchApproxCountDistinctSqlAggregator;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.QueryableIndex;
|
||||
import org.apache.druid.segment.QueryableIndexSegment;
|
||||
|
@ -405,26 +407,35 @@ public class SqlBenchmark
|
|||
"SELECT * FROM foo WHERE dimSequential IN ('1', '2', '3', '4', '5', '10', '11', '20', '21', '23', '40', '50', '64', '70', '100')",
|
||||
"SELECT * FROM foo WHERE dimSequential > '10' AND dimSequential < '8500'",
|
||||
"SELECT dimSequential, dimZipf, SUM(sumLongSequential) FROM foo WHERE dimSequential IN ('1', '2', '3', '4', '5', '10', '11', '20', '21', '23', '40', '50', '64', '70', '100') GROUP BY 1, 2",
|
||||
"SELECT dimSequential, dimZipf, SUM(sumLongSequential) FROM foo WHERE dimSequential > '10' AND dimSequential < '8500' GROUP BY 1, 2"
|
||||
|
||||
"SELECT dimSequential, dimZipf, SUM(sumLongSequential) FROM foo WHERE dimSequential > '10' AND dimSequential < '8500' GROUP BY 1, 2",
|
||||
|
||||
// 28, 29, 30, 31: Approximate count distinct of strings
|
||||
"SELECT APPROX_COUNT_DISTINCT_BUILTIN(dimZipf) FROM foo",
|
||||
"SELECT APPROX_COUNT_DISTINCT_DS_HLL(dimZipf) FROM foo",
|
||||
"SELECT APPROX_COUNT_DISTINCT_DS_HLL_UTF8(dimZipf) FROM foo",
|
||||
"SELECT APPROX_COUNT_DISTINCT_DS_THETA(dimZipf) FROM foo"
|
||||
);
|
||||
|
||||
@Param({"5000000"})
|
||||
private int rowsPerSegment;
|
||||
|
||||
@Param({"false", "force"})
|
||||
// Can be "false", "true", or "force"
|
||||
@Param({"force"})
|
||||
private String vectorize;
|
||||
@Param({"none", "front-coded-4", "front-coded-16"})
|
||||
|
||||
// Can be "none" or "front-coded-N"
|
||||
@Param({"none", "front-coded-4"})
|
||||
private String stringEncoding;
|
||||
|
||||
@Param({"4", "5", "6", "7", "8", "10", "11", "12", "19", "21", "22", "23", "26", "27"})
|
||||
@Param({"28", "29", "30", "31"})
|
||||
private String query;
|
||||
|
||||
@Param({STORAGE_MMAP, STORAGE_FRAME_ROW, STORAGE_FRAME_COLUMNAR})
|
||||
// Can be STORAGE_MMAP, STORAGE_FRAME_ROW, or STORAGE_FRAME_COLUMNAR
|
||||
@Param({STORAGE_MMAP})
|
||||
private String storageType;
|
||||
|
||||
private SqlEngine engine;
|
||||
|
||||
@Nullable
|
||||
private PlannerFactory plannerFactory;
|
||||
private final Closer closer = Closer.create();
|
||||
|
@ -520,13 +531,19 @@ public class SqlBenchmark
|
|||
try {
|
||||
final Set<SqlOperatorConversion> extractionOperators = new HashSet<>();
|
||||
extractionOperators.add(CalciteTests.INJECTOR.getInstance(QueryLookupOperatorConversion.class));
|
||||
final Set<SqlAggregator> aggregators = new HashSet<>();
|
||||
aggregators.add(CalciteTests.INJECTOR.getInstance(DoublesSketchApproxQuantileSqlAggregator.class));
|
||||
aggregators.add(CalciteTests.INJECTOR.getInstance(DoublesSketchObjectSqlAggregator.class));
|
||||
final ApproxCountDistinctSqlAggregator countDistinctSqlAggregator =
|
||||
new ApproxCountDistinctSqlAggregator(new HllSketchApproxCountDistinctSqlAggregator());
|
||||
aggregators.add(new CountSqlAggregator(countDistinctSqlAggregator));
|
||||
aggregators.add(countDistinctSqlAggregator);
|
||||
final Set<SqlAggregator> aggregators = new HashSet<>(
|
||||
ImmutableList.of(
|
||||
new DoublesSketchApproxQuantileSqlAggregator(),
|
||||
new DoublesSketchObjectSqlAggregator(),
|
||||
new HllSketchApproxCountDistinctSqlAggregator(),
|
||||
new HllSketchApproxCountDistinctUtf8SqlAggregator(),
|
||||
new ThetaSketchApproxCountDistinctSqlAggregator(),
|
||||
new CountSqlAggregator(countDistinctSqlAggregator),
|
||||
countDistinctSqlAggregator
|
||||
)
|
||||
);
|
||||
return new DruidOperatorTable(aggregators, extractionOperators);
|
||||
}
|
||||
catch (Exception e) {
|
||||
|
@ -543,7 +560,7 @@ public class SqlBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
public void querySql(Blackhole blackhole) throws Exception
|
||||
public void querySql(Blackhole blackhole)
|
||||
{
|
||||
final Map<String, Object> context = ImmutableMap.of(
|
||||
QueryContexts.VECTORIZE_KEY, vectorize,
|
||||
|
@ -561,7 +578,7 @@ public class SqlBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
public void planSql(Blackhole blackhole) throws Exception
|
||||
public void planSql(Blackhole blackhole)
|
||||
{
|
||||
final Map<String, Object> context = ImmutableMap.of(
|
||||
QueryContexts.VECTORIZE_KEY, vectorize,
|
||||
|
|
|
@ -67,7 +67,6 @@ import org.openjdk.jmh.annotations.Warmup;
|
|||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -359,7 +358,7 @@ public class SqlExpressionBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
public void querySql(Blackhole blackhole) throws Exception
|
||||
public void querySql(Blackhole blackhole)
|
||||
{
|
||||
final Map<String, Object> context = ImmutableMap.of(
|
||||
QueryContexts.VECTORIZE_KEY, vectorize,
|
||||
|
|
|
@ -375,7 +375,7 @@ public class SqlNestedDataBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
public void querySql(Blackhole blackhole) throws Exception
|
||||
public void querySql(Blackhole blackhole)
|
||||
{
|
||||
final Map<String, Object> context = ImmutableMap.of(
|
||||
QueryContexts.VECTORIZE_KEY, vectorize,
|
||||
|
|
|
@ -170,7 +170,7 @@ public class SqlVsNativeBenchmark
|
|||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
public void queryPlanner(Blackhole blackhole) throws Exception
|
||||
public void queryPlanner(Blackhole blackhole)
|
||||
{
|
||||
try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sqlQuery, Collections.emptyMap())) {
|
||||
final PlannerResult plannerResult = planner.plan();
|
||||
|
|
|
@ -136,11 +136,6 @@ public class TimeseriesBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -134,11 +134,6 @@ public class TopNBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -160,11 +160,6 @@ public class TimeCompareBenchmark
|
|||
JSON_MAPPER,
|
||||
new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
);
|
||||
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
|
||||
|
|
|
@ -20,13 +20,14 @@
|
|||
package org.apache.druid.server.coordinator;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import org.apache.druid.client.ImmutableDruidDataSource;
|
||||
import org.apache.druid.client.ImmutableDruidServer;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.server.coordination.DruidServerMetadata;
|
||||
import org.apache.druid.server.coordination.ServerType;
|
||||
import org.apache.druid.server.coordinator.balancer.BalancerSegmentHolder;
|
||||
import org.apache.druid.server.coordinator.balancer.ReservoirSegmentSampler;
|
||||
import org.apache.druid.server.coordinator.loading.LoadQueuePeonTester;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.joda.time.Interval;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
|
@ -63,9 +64,6 @@ public class BalancerStrategyBenchmark
|
|||
private static final Interval TEST_SEGMENT_INTERVAL = Intervals.of("2012-03-15T00:00:00.000/2012-03-16T00:00:00.000");
|
||||
private static final int NUMBER_OF_SERVERS = 20;
|
||||
|
||||
@Param({"default", "50percentOfSegmentsToConsiderPerMove", "useBatchedSegmentSampler"})
|
||||
private String mode;
|
||||
|
||||
@Param({"10000", "100000", "1000000"})
|
||||
private int numberOfSegments;
|
||||
|
||||
|
@ -73,27 +71,12 @@ public class BalancerStrategyBenchmark
|
|||
private int maxSegmentsToMove;
|
||||
|
||||
private final List<ServerHolder> serverHolders = new ArrayList<>();
|
||||
private boolean useBatchedSegmentSampler;
|
||||
private int reservoirSize = 1;
|
||||
private double percentOfSegmentsToConsider = 100;
|
||||
private final BalancerStrategy balancerStrategy = new CostBalancerStrategy(
|
||||
MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "BalancerStrategyBenchmark-%d"))
|
||||
);
|
||||
|
||||
@Setup(Level.Trial)
|
||||
public void setup()
|
||||
{
|
||||
switch (mode) {
|
||||
case "50percentOfSegmentsToConsiderPerMove":
|
||||
percentOfSegmentsToConsider = 50;
|
||||
useBatchedSegmentSampler = false;
|
||||
break;
|
||||
case "useBatchedSegmentSampler":
|
||||
reservoirSize = maxSegmentsToMove;
|
||||
useBatchedSegmentSampler = true;
|
||||
break;
|
||||
default:
|
||||
}
|
||||
reservoirSize = maxSegmentsToMove;
|
||||
|
||||
List<List<DataSegment>> segmentList = new ArrayList<>(NUMBER_OF_SERVERS);
|
||||
IntStream.range(0, NUMBER_OF_SERVERS).forEach(i -> segmentList.add(new ArrayList<>()));
|
||||
|
@ -131,20 +114,9 @@ public class BalancerStrategyBenchmark
|
|||
@Benchmark
|
||||
public void pickSegmentsToMove(Blackhole blackhole)
|
||||
{
|
||||
Iterator<BalancerSegmentHolder> iterator;
|
||||
if (useBatchedSegmentSampler) {
|
||||
iterator = balancerStrategy.pickSegmentsToMove(
|
||||
serverHolders,
|
||||
Collections.emptySet(),
|
||||
reservoirSize
|
||||
);
|
||||
} else {
|
||||
iterator = balancerStrategy.pickSegmentsToMove(
|
||||
serverHolders,
|
||||
Collections.emptySet(),
|
||||
percentOfSegmentsToConsider
|
||||
);
|
||||
}
|
||||
Iterator<BalancerSegmentHolder> iterator = ReservoirSegmentSampler
|
||||
.pickMovableSegmentsFrom(serverHolders, reservoirSize, ServerHolder::getServedSegments, Collections.emptySet())
|
||||
.iterator();
|
||||
|
||||
for (int i = 0; i < maxSegmentsToMove && iterator.hasNext(); i++) {
|
||||
blackhole.consume(iterator.next());
|
||||
|
|
|
@ -21,7 +21,8 @@ package org.apache.druid.server.coordinator;
|
|||
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.server.coordinator.cost.SegmentsCostCache;
|
||||
import org.apache.druid.server.coordinator.balancer.CostBalancerStrategy;
|
||||
import org.apache.druid.server.coordinator.balancer.SegmentsCostCache;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.server.coordinator;
|
||||
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.server.coordinator.balancer.CostBalancerStrategy;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
|
|
@ -45,6 +45,7 @@
|
|||
<Bug pattern="EQ_CHECK_FOR_OPERAND_NOT_COMPATIBLE_WITH_THIS"/>
|
||||
<Or>
|
||||
<Class name="org.apache.druid.jackson.DefaultTrueJsonIncludeFilter"/>
|
||||
<Class name="org.apache.druid.java.util.common.StringEncodingDefaultUTF16LEJsonIncludeFilter"/>
|
||||
<Class name="org.apache.druid.query.scan.ScanQuery$ScanRowsLimitJsonIncludeFilter"/>
|
||||
<Class name="org.apache.druid.query.scan.ScanQuery$ScanTimeOrderJsonIncludeFilter"/>
|
||||
<Class name="org.apache.druid.query.scan.ScanQuery$BatchSizeJsonIncludeFilter"/>
|
||||
|
|
|
@ -292,6 +292,7 @@ def build_compatible_license_names():
|
|||
compatible_licenses['MIT License'] = 'MIT License'
|
||||
compatible_licenses['The MIT License (MIT)'] = 'MIT License'
|
||||
compatible_licenses['Bouncy Castle Licence'] = 'MIT License'
|
||||
compatible_licenses['SPDX-License-Identifier: MIT'] = 'MIT License'
|
||||
|
||||
compatible_licenses['The Go license'] = 'The Go license'
|
||||
|
||||
|
@ -435,7 +436,6 @@ if __name__ == "__main__":
|
|||
|
||||
license_yaml = args.license_yaml
|
||||
dependency_reports_root = args.dependency_reports_root
|
||||
|
||||
check_licenses(license_yaml, dependency_reports_root)
|
||||
|
||||
except KeyboardInterrupt:
|
||||
|
|
|
@ -49,7 +49,7 @@ RUN --mount=type=cache,target=/root/.m2 VERSION=$(mvn -B -q org.apache.maven.plu
|
|||
&& tar -zxf ./distribution/target/apache-druid-${VERSION}-bin.tar.gz -C /opt \
|
||||
&& mv /opt/apache-druid-${VERSION} /opt/druid
|
||||
|
||||
FROM busybox:1.35.0-glibc as busybox
|
||||
FROM busybox:1.34.1-glibc as busybox
|
||||
|
||||
FROM gcr.io/distroless/java$JDK_VERSION-debian11
|
||||
LABEL maintainer="Apache Druid Developers <dev@druid.apache.org>"
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
---
|
||||
id: api-reference
|
||||
title: HTTP API endpoints reference
|
||||
sidebar_label: API endpoints reference
|
||||
title: API reference
|
||||
sidebar_label: Overview
|
||||
---
|
||||
|
||||
<!--
|
||||
|
@ -24,864 +24,21 @@ sidebar_label: API endpoints reference
|
|||
-->
|
||||
|
||||
|
||||
This topic documents all of the API endpoints for each Druid service type.
|
||||
|
||||
## Common
|
||||
|
||||
All processes support the following endpoints.
|
||||
|
||||
### Process information
|
||||
|
||||
`GET /status`
|
||||
|
||||
Returns the Druid version, loaded extensions, memory used, total memory, and other useful information about the process.
|
||||
|
||||
`GET /status/health`
|
||||
|
||||
Always returns a boolean `true` value with a 200 OK response, useful for automated health checks.
|
||||
|
||||
`GET /status/properties`
|
||||
|
||||
Returns the current configuration properties of the process.
|
||||
|
||||
`GET /status/selfDiscovered/status`
|
||||
|
||||
Returns 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. It is recommended to not consider a Druid node "healthy" or "ready" in automated deployment/container
|
||||
management systems until it returns `{"selfDiscovered": true}` from this endpoint. This is because a node may be
|
||||
isolated from the rest of the cluster due to network issues and it doesn't make sense to consider nodes "healthy" in
|
||||
this case. Also, when nodes such as Brokers use ZooKeeper segment discovery for building their view of the Druid cluster
|
||||
(as opposed to HTTP segment discovery), they may be unusable until the ZooKeeper client is fully initialized and starts
|
||||
to receive data from the ZooKeeper cluster. `{"selfDiscovered": true}` is a proxy event indicating that the ZooKeeper
|
||||
client on the node has started to receive data from the ZooKeeper cluster and it's expected that all segments and other
|
||||
nodes will be discovered by this node timely from this point.
|
||||
|
||||
`GET /status/selfDiscovered`
|
||||
|
||||
Similar to `/status/selfDiscovered/status`, but returns 200 OK response with empty body if the node has discovered itself
|
||||
and 503 SERVICE UNAVAILABLE if the node hasn't discovered itself yet. This endpoint might be useful because some
|
||||
monitoring checks such as AWS load balancer health checks are not able to look at the response body.
|
||||
|
||||
## Master server
|
||||
|
||||
This section documents the API endpoints for the processes that reside on Master servers (Coordinators and Overlords)
|
||||
in the suggested [three-server configuration](../design/processes.md#server-types).
|
||||
|
||||
### Coordinator
|
||||
|
||||
#### Leadership
|
||||
|
||||
`GET /druid/coordinator/v1/leader`
|
||||
|
||||
Returns the current leader Coordinator of the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/isLeader`
|
||||
|
||||
Returns a JSON object with `leader` parameter, either true or false, indicating if this server is the current leader
|
||||
Coordinator of the cluster. In addition, returns HTTP 200 if the server is the current leader and HTTP 404 if not.
|
||||
This is suitable for use as a load balancer status check if you only want the active leader to be considered in-service
|
||||
at the load balancer.
|
||||
|
||||
|
||||
<a name="coordinator-segment-loading"></a>
|
||||
|
||||
#### Segment loading
|
||||
|
||||
`GET /druid/coordinator/v1/loadstatus`
|
||||
|
||||
Returns the percentage of segments actually loaded in the cluster versus segments that should be loaded in the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/loadstatus?simple`
|
||||
|
||||
Returns the number of segments left to load until segments that should be loaded in the cluster are available for queries. This does not include segment replication counts.
|
||||
|
||||
`GET /druid/coordinator/v1/loadstatus?full`
|
||||
|
||||
Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
|
||||
|
||||
`GET /druid/coordinator/v1/loadstatus?full&computeUsingClusterView`
|
||||
|
||||
Returns the number of segments not yet loaded for each tier until all segments loading in the cluster are available.
|
||||
The result includes segment replication counts. It also factors in the number of available nodes that are of a service type that can load the segment when computing the number of segments remaining to load.
|
||||
A segment is considered fully loaded when:
|
||||
- Druid has replicated it the number of times configured in the corresponding load rule.
|
||||
- Or the number of replicas for the segment in each tier where it is configured to be replicated equals the available nodes of a service type that are currently allowed to load the segment in the tier.
|
||||
|
||||
`GET /druid/coordinator/v1/loadqueue`
|
||||
|
||||
Returns the ids of segments to load and drop for each Historical process.
|
||||
|
||||
`GET /druid/coordinator/v1/loadqueue?simple`
|
||||
|
||||
Returns the number of segments to load and drop, as well as the total segment load and drop size in bytes for each Historical process.
|
||||
|
||||
`GET /druid/coordinator/v1/loadqueue?full`
|
||||
|
||||
Returns the serialized JSON of segments to load and drop for each Historical process.
|
||||
|
||||
#### Segment loading by datasource
|
||||
|
||||
Note that all _interval_ query parameters are ISO 8601 strings—for example, 2016-06-27/2016-06-28.
|
||||
Also note that these APIs only guarantees that the segments are available at the time of the call.
|
||||
Segments can still become missing because of historical process failures or any other reasons afterward.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?forceMetadataRefresh={boolean}&interval={myInterval}`
|
||||
|
||||
Returns the percentage of segments actually loaded in the cluster versus segments that should be loaded in the cluster for the given
|
||||
datasource over the given interval (or last 2 weeks if interval is not given). `forceMetadataRefresh` is required to be set.
|
||||
* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store
|
||||
(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms
|
||||
of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status)
|
||||
* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh.
|
||||
If no used segments are found for the given inputs, this API returns `204 No Content`
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?simple&forceMetadataRefresh={boolean}&interval={myInterval}`
|
||||
|
||||
Returns the number of segments left to load until segments that should be loaded in the cluster are available for the given datasource
|
||||
over the given interval (or last 2 weeks if interval is not given). This does not include segment replication counts. `forceMetadataRefresh` is required to be set.
|
||||
* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store
|
||||
(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms
|
||||
of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status)
|
||||
* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh.
|
||||
If no used segments are found for the given inputs, this API returns `204 No Content`
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?full&forceMetadataRefresh={boolean}&interval={myInterval}`
|
||||
|
||||
Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available for the given datasource over the given interval (or last 2 weeks if interval is not given). This includes segment replication counts. `forceMetadataRefresh` is required to be set.
|
||||
* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store
|
||||
(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms
|
||||
of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status)
|
||||
* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh.
|
||||
|
||||
You can pass the optional query parameter `computeUsingClusterView` to factor in the available cluster services when calculating
|
||||
the segments left to load. See [Coordinator Segment Loading](#coordinator-segment-loading) for details.
|
||||
If no used segments are found for the given inputs, this API returns `204 No Content`
|
||||
|
||||
#### 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.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/segments`
|
||||
|
||||
Returns a list of all segments for each datasource enabled in the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/segments?datasources={dataSourceName1}&datasources={dataSourceName2}`
|
||||
|
||||
Returns a list of all segments for one or more specific datasources enabled in the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/segments?includeOvershadowedStatus`
|
||||
|
||||
Returns a list of all segments for each datasource with the full segment metadata and an extra field `overshadowed`.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/segments?includeOvershadowedStatus&datasources={dataSourceName1}&datasources={dataSourceName2}`
|
||||
|
||||
Returns a list of all segments for one or more specific datasources with the full segment metadata and an extra field `overshadowed`.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources`
|
||||
|
||||
Returns a list of the names of datasources with at least one used segment in the cluster, retrieved from the metadata database. Users should call this API to get the eventual state that the system will be in.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources?includeUnused`
|
||||
|
||||
Returns a list of the names of datasources, regardless of whether there are used segments belonging to those datasources in the cluster or not.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources?includeDisabled`
|
||||
|
||||
Returns a list of the names of datasources, regardless of whether the datasource is disabled or not.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources?full`
|
||||
|
||||
Returns a list of all datasources with at least one used segment in the cluster. Returns all metadata about those datasources as stored in the metadata store.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}`
|
||||
|
||||
Returns full metadata for a datasource as stored in the metadata store.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments`
|
||||
|
||||
Returns a list of all segments for a datasource as stored in the metadata store.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full`
|
||||
|
||||
Returns a list of all segments for a datasource with the full segment metadata as stored in the metadata store.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments/{segmentId}`
|
||||
|
||||
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.
|
||||
|
||||
##### POST
|
||||
|
||||
`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"]`.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full`
|
||||
|
||||
Returns a list of all segments, overlapping with any of given intervals, for a datasource with the full segment metadata as stored in the metadata store. Request body is array of string ISO 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"]`.
|
||||
|
||||
<a name="coordinator-datasources"></a>
|
||||
|
||||
#### Datasources
|
||||
|
||||
Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/`—for example, `2016-06-27_2016-06-28`.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources`
|
||||
|
||||
Returns a list of datasource names found in the cluster as seen by the coordinator. This view is updated every [`druid.coordinator.period`](../configuration/index.md#coordinator-operation).
|
||||
|
||||
`GET /druid/coordinator/v1/datasources?simple`
|
||||
|
||||
Returns a list of JSON objects containing the name and properties of datasources found in the cluster. Properties include segment count, total segment byte size, replicated total segment byte size, minTime, and maxTime.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources?full`
|
||||
|
||||
Returns a list of datasource names found in the cluster with all metadata about those datasources.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}`
|
||||
|
||||
Returns a JSON object containing the name and properties of a datasource. Properties include segment count, total segment byte size, replicated total segment byte size, minTime, and maxTime.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}?full`
|
||||
|
||||
Returns full metadata for a datasource.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals`
|
||||
|
||||
Returns a set of segment intervals.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals?simple`
|
||||
|
||||
Returns a map of an interval to a JSON object containing the total byte size of segments and number of segments for that interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals?full`
|
||||
|
||||
Returns a map of an interval to a map of segment metadata to a set of server names that contain the segment for that interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}`
|
||||
|
||||
Returns a set of segment ids for an interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}?simple`
|
||||
|
||||
Returns a map of segment intervals contained within the specified interval to a JSON object containing the total byte size of segments and number of segments for an interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}?full`
|
||||
|
||||
Returns a map of segment intervals contained within the specified interval to a map of segment metadata to a set of server names that contain the segment for an interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}/serverview`
|
||||
|
||||
Returns a map of segment intervals contained within the specified interval to information about the servers that contain the segment for an interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments`
|
||||
|
||||
Returns a list of all segments for a datasource in the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments?full`
|
||||
|
||||
Returns a list of all segments for a datasource in the cluster with the full segment metadata.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
|
||||
|
||||
Returns full segment metadata for a specific segment in the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/tiers`
|
||||
|
||||
Return the tiers that a datasource exists in.
|
||||
|
||||
#### Note for Coordinator's POST and DELETE APIs
|
||||
|
||||
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.
|
||||
|
||||
`POST /druid/coordinator/v1/datasources/{dataSourceName}`
|
||||
|
||||
Marks as used all segments belonging to a datasource. Returns a JSON object of the form
|
||||
`{"numChangedSegments": <number>}` with the number of segments in the database whose state has been changed (that is,
|
||||
the segments were marked as used) as the result of this API call.
|
||||
|
||||
`POST /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
|
||||
|
||||
Marks as used a segment of a datasource. Returns a JSON object of the form `{"segmentStateChanged": <boolean>}` with
|
||||
the boolean indicating if the state of the segment has been changed (that is, the segment was marked as used) as the
|
||||
result of this API call.
|
||||
|
||||
`POST /druid/coordinator/v1/datasources/{dataSourceName}/markUsed`
|
||||
|
||||
`POST /druid/coordinator/v1/datasources/{dataSourceName}/markUnused`
|
||||
|
||||
Marks segments (un)used for a datasource by interval or set of segment Ids. When marking used only segments that are not overshadowed will be updated.
|
||||
|
||||
The request payload contains the interval or set of segment IDs to be marked unused.
|
||||
Either interval or segment IDs should be provided, if both or none are provided in the payload, the API would throw an error (400 BAD REQUEST).
|
||||
|
||||
Interval specifies the start and end times as IS0 8601 strings. `interval=(start/end)` where start and end both are inclusive and only the segments completely contained within the specified interval will be disabled, partially overlapping segments will not be affected.
|
||||
|
||||
JSON Request Payload:
|
||||
|
||||
|Key|Description|Example|
|
||||
|----------|-------------|---------|
|
||||
|`interval`|The interval for which to mark segments unused|`"2015-09-12T03:00:00.000Z/2015-09-12T05:00:00.000Z"`|
|
||||
|`segmentIds`|Set of segment IDs to be marked unused|`["segmentId1", "segmentId2"]`|
|
||||
|
||||
`DELETE /druid/coordinator/v1/datasources/{dataSourceName}`
|
||||
|
||||
Marks as unused all segments belonging to a datasource. Returns a JSON object of the form
|
||||
`{"numChangedSegments": <number>}` with the number of segments in the database whose state has been changed (that is,
|
||||
the segments were marked as unused) as the result of this API call.
|
||||
|
||||
`DELETE /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}`
|
||||
`@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myInterval}`
|
||||
|
||||
Runs a [Kill task](../ingestion/tasks.md) for a given interval and datasource.
|
||||
|
||||
`DELETE /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
|
||||
|
||||
Marks as unused a segment of a datasource. Returns a JSON object of the form `{"segmentStateChanged": <boolean>}` with
|
||||
the boolean indicating if the state of the segment has been changed (that is, the segment was marked as unused) as the
|
||||
result of this API call.
|
||||
|
||||
#### 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`.
|
||||
|
||||
`GET /druid/coordinator/v1/rules`
|
||||
|
||||
Returns all rules as JSON objects for all datasources in the cluster including the default datasource.
|
||||
|
||||
`GET /druid/coordinator/v1/rules/{dataSourceName}`
|
||||
|
||||
Returns all rules for a specified datasource.
|
||||
|
||||
`GET /druid/coordinator/v1/rules/{dataSourceName}?full`
|
||||
|
||||
Returns all rules for a specified datasource and includes default datasource.
|
||||
|
||||
`GET /druid/coordinator/v1/rules/history?interval=<interval>`
|
||||
|
||||
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=<n>`
|
||||
|
||||
Returns last `n` entries of audit history of rules for all datasources.
|
||||
|
||||
`GET /druid/coordinator/v1/rules/{dataSourceName}/history?interval=<interval>`
|
||||
|
||||
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=<n>`
|
||||
|
||||
Returns last `n` entries of audit history of rules for a specified datasource.
|
||||
|
||||
`POST /druid/coordinator/v1/rules/{dataSourceName}`
|
||||
|
||||
POST with a list of rules in JSON form to update rules.
|
||||
|
||||
Optional Header Parameters for auditing the config change can also be specified.
|
||||
|
||||
|Header Param Name| Description | Default |
|
||||
|----------|-------------|---------|
|
||||
|`X-Druid-Author`| Author making the config change|`""`|
|
||||
|`X-Druid-Comment`| Comment describing the change being done|`""`|
|
||||
|
||||
#### Intervals
|
||||
|
||||
Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` as in `2016-06-27_2016-06-28`.
|
||||
|
||||
`GET /druid/coordinator/v1/intervals`
|
||||
|
||||
Returns all intervals for all datasources with total size and count.
|
||||
|
||||
`GET /druid/coordinator/v1/intervals/{interval}`
|
||||
|
||||
Returns aggregated total size and count for all intervals that intersect given ISO interval.
|
||||
|
||||
`GET /druid/coordinator/v1/intervals/{interval}?simple`
|
||||
|
||||
Returns total size and count for each interval within given ISO interval.
|
||||
|
||||
`GET /druid/coordinator/v1/intervals/{interval}?full`
|
||||
|
||||
Returns total size and count for each datasource for each interval within given ISO interval.
|
||||
|
||||
#### Dynamic configuration
|
||||
|
||||
See [Coordinator Dynamic Configuration](../configuration/index.md#dynamic-configuration) for details.
|
||||
|
||||
Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/`
|
||||
as in `2016-06-27_2016-06-28`.
|
||||
|
||||
`GET /druid/coordinator/v1/config`
|
||||
|
||||
Retrieves current coordinator dynamic configuration.
|
||||
|
||||
`GET /druid/coordinator/v1/config/history?interval={interval}&count={count}`
|
||||
|
||||
Retrieves history of changes to overlord dynamic configuration. Accepts `interval` and `count` query string parameters
|
||||
to filter by interval and limit the number of results respectively.
|
||||
|
||||
`POST /druid/coordinator/v1/config`
|
||||
|
||||
Update overlord dynamic worker configuration.
|
||||
|
||||
#### Automatic compaction status
|
||||
|
||||
`GET /druid/coordinator/v1/compaction/progress?dataSource={dataSource}`
|
||||
|
||||
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.
|
||||
|
||||
`GET /druid/coordinator/v1/compaction/status`
|
||||
|
||||
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 `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
|
||||
|
||||
`GET /druid/coordinator/v1/compaction/status?dataSource={dataSource}`
|
||||
|
||||
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.
|
||||
|
||||
#### Automatic compaction configuration
|
||||
|
||||
`GET /druid/coordinator/v1/config/compaction`
|
||||
|
||||
Returns all automatic compaction configs.
|
||||
|
||||
`GET /druid/coordinator/v1/config/compaction/{dataSource}`
|
||||
|
||||
Returns an automatic compaction config of a dataSource.
|
||||
|
||||
`GET /druid/coordinator/v1/config/compaction/{dataSource}/history?interval={interval}&count={count}`
|
||||
|
||||
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.
|
||||
|
||||
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`.
|
||||
* `auditTime`: The date and time when the change was made.
|
||||
|
||||
`POST /druid/coordinator/v1/config/compaction/taskslots?ratio={someRatio}&max={someMaxSlots}`
|
||||
|
||||
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.
|
||||
|
||||
`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.
|
||||
|
||||
`DELETE /druid/coordinator/v1/config/compaction/{dataSource}`
|
||||
|
||||
Removes the automatic compaction config for a dataSource.
|
||||
|
||||
#### Server information
|
||||
|
||||
`GET /druid/coordinator/v1/servers`
|
||||
|
||||
Returns a list of servers URLs using the format `{hostname}:{port}`. Note that
|
||||
processes that run with different types will appear multiple times with different
|
||||
ports.
|
||||
|
||||
`GET /druid/coordinator/v1/servers?simple`
|
||||
|
||||
Returns a list of server data objects in which each object has the following keys:
|
||||
* `host`: host URL include (`{hostname}:{port}`)
|
||||
* `type`: process type (`indexer-executor`, `historical`)
|
||||
* `currSize`: storage size currently used
|
||||
* `maxSize`: maximum storage size
|
||||
* `priority`
|
||||
* `tier`
|
||||
|
||||
### Overlord
|
||||
|
||||
#### Leadership
|
||||
|
||||
`GET /druid/indexer/v1/leader`
|
||||
|
||||
Returns the current leader Overlord of the cluster. If you have multiple Overlords, just one is leading at any given time. The others are on standby.
|
||||
|
||||
`GET /druid/indexer/v1/isLeader`
|
||||
|
||||
This returns a JSON object with field `leader`, either true or false. In addition, this call returns HTTP 200 if the
|
||||
server is the current leader and HTTP 404 if not. This is suitable for use as a load balancer status check if you
|
||||
only want the active leader to be considered in-service at the load balancer.
|
||||
|
||||
#### Tasks
|
||||
|
||||
Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/`
|
||||
as in `2016-06-27_2016-06-28`.
|
||||
|
||||
`GET /druid/indexer/v1/tasks`
|
||||
|
||||
Retrieve list of tasks. Accepts query string parameters `state`, `datasource`, `createdTimeInterval`, `max`, and `type`.
|
||||
|
||||
|Query Parameter |Description |
|
||||
|---|---|
|
||||
|`state`|filter list of tasks by task state, valid options are `running`, `complete`, `waiting`, and `pending`.|
|
||||
| `datasource`| return tasks filtered by Druid datasource.|
|
||||
| `createdTimeInterval`| return tasks created within the specified interval. |
|
||||
| `max`| maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.|
|
||||
| `type`| filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.|
|
||||
|
||||
|
||||
`GET /druid/indexer/v1/completeTasks`
|
||||
|
||||
Retrieve list of complete tasks. Equivalent to `/druid/indexer/v1/tasks?state=complete`.
|
||||
|
||||
`GET /druid/indexer/v1/runningTasks`
|
||||
|
||||
Retrieve list of running tasks. Equivalent to `/druid/indexer/v1/tasks?state=running`.
|
||||
|
||||
`GET /druid/indexer/v1/waitingTasks`
|
||||
|
||||
Retrieve list of waiting tasks. Equivalent to `/druid/indexer/v1/tasks?state=waiting`.
|
||||
|
||||
`GET /druid/indexer/v1/pendingTasks`
|
||||
|
||||
Retrieve list of pending tasks. Equivalent to `/druid/indexer/v1/tasks?state=pending`.
|
||||
|
||||
`GET /druid/indexer/v1/task/{taskId}`
|
||||
|
||||
Retrieve the 'payload' of a task.
|
||||
|
||||
`GET /druid/indexer/v1/task/{taskId}/status`
|
||||
|
||||
Retrieve the status of a task.
|
||||
|
||||
`GET /druid/indexer/v1/task/{taskId}/segments`
|
||||
|
||||
> This API is deprecated and will be removed in future releases.
|
||||
|
||||
Retrieve information about the segments of a task.
|
||||
|
||||
`GET /druid/indexer/v1/task/{taskId}/reports`
|
||||
|
||||
Retrieve a [task completion report](../ingestion/tasks.md#task-reports) for a task. Only works for completed tasks.
|
||||
|
||||
`POST /druid/indexer/v1/task`
|
||||
|
||||
Endpoint for submitting tasks and supervisor specs to the Overlord. Returns the taskId of the submitted task.
|
||||
|
||||
`POST /druid/indexer/v1/task/{taskId}/shutdown`
|
||||
|
||||
Shuts down a task.
|
||||
|
||||
`POST /druid/indexer/v1/datasources/{dataSource}/shutdownAllTasks`
|
||||
|
||||
Shuts down all tasks for a dataSource.
|
||||
|
||||
`POST /druid/indexer/v1/taskStatus`
|
||||
|
||||
Retrieve list of task status objects for list of task id strings in request body.
|
||||
|
||||
`DELETE /druid/indexer/v1/pendingSegments/{dataSource}`
|
||||
|
||||
Manually clean up pending segments table in metadata storage for `datasource`. Returns a JSON object response with
|
||||
`numDeleted` and count of rows deleted from the pending segments table. This API is used by the
|
||||
`druid.coordinator.kill.pendingSegments.on` [coordinator setting](../configuration/index.md#coordinator-operation)
|
||||
which automates this operation to perform periodically.
|
||||
|
||||
#### Supervisors
|
||||
|
||||
`GET /druid/indexer/v1/supervisor`
|
||||
|
||||
Returns a list of strings of the currently active supervisor ids.
|
||||
|
||||
`GET /druid/indexer/v1/supervisor?full`
|
||||
|
||||
Returns a list of objects of the currently active supervisors.
|
||||
|
||||
|Field|Type|Description|
|
||||
|---|---|---|
|
||||
|`id`|String|supervisor unique identifier|
|
||||
|`state`|String|basic state of the supervisor. Available states:`UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`. Check [Kafka Docs](../development/extensions-core/kafka-supervisor-operations.md) for details.|
|
||||
|`detailedState`|String|supervisor specific state. See documentation of specific supervisor for details: [Kafka](../development/extensions-core/kafka-ingestion.md) or [Kinesis](../development/extensions-core/kinesis-ingestion.md)|
|
||||
|`healthy`|Boolean|true or false indicator of overall supervisor health|
|
||||
|`spec`|SupervisorSpec|JSON specification of supervisor|
|
||||
|
||||
`GET /druid/indexer/v1/supervisor?state=true`
|
||||
|
||||
Returns a list of objects of the currently active supervisors and their current state.
|
||||
|
||||
|Field|Type|Description|
|
||||
|---|---|---|
|
||||
|`id`|String|supervisor unique identifier|
|
||||
|`state`|String|basic state of the supervisor. Available states: `UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`. Check [Kafka Docs](../development/extensions-core/kafka-supervisor-operations.md) for details.|
|
||||
|`detailedState`|String|supervisor specific state. See documentation of the specific supervisor for details: [Kafka](../development/extensions-core/kafka-ingestion.md) or [Kinesis](../development/extensions-core/kinesis-ingestion.md)|
|
||||
|`healthy`|Boolean|true or false indicator of overall supervisor health|
|
||||
|`suspended`|Boolean|true or false indicator of whether the supervisor is in suspended state|
|
||||
|
||||
`GET /druid/indexer/v1/supervisor/<supervisorId>`
|
||||
|
||||
Returns the current spec for the supervisor with the provided ID.
|
||||
|
||||
`GET /druid/indexer/v1/supervisor/<supervisorId>/status`
|
||||
|
||||
Returns the current status of the supervisor with the provided ID.
|
||||
|
||||
`GET/druid/indexer/v1/supervisor/history`
|
||||
|
||||
Returns an audit history of specs for all supervisors (current and past).
|
||||
|
||||
`GET /druid/indexer/v1/supervisor/<supervisorId>/history`
|
||||
|
||||
Returns an audit history of specs for the supervisor with the provided ID.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor`
|
||||
|
||||
Create a new supervisor or update an existing one.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/<supervisorId>/suspend`
|
||||
|
||||
Suspend the current running supervisor of the provided ID. Responds with updated SupervisorSpec.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/suspendAll`
|
||||
|
||||
Suspend all supervisors at once.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/<supervisorId>/resume`
|
||||
|
||||
Resume indexing tasks for a supervisor. Responds with updated SupervisorSpec.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/resumeAll`
|
||||
|
||||
Resume all supervisors at once.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/<supervisorId>/reset`
|
||||
|
||||
Reset the specified supervisor.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/<supervisorId>/terminate`
|
||||
|
||||
Terminate a supervisor of the provided ID.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/terminateAll`
|
||||
|
||||
Terminate all supervisors at once.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/<supervisorId>/shutdown`
|
||||
|
||||
> This API is deprecated and will be removed in future releases.
|
||||
> Please use the equivalent `terminate` instead.
|
||||
|
||||
Shutdown a supervisor.
|
||||
|
||||
#### Dynamic configuration
|
||||
|
||||
See [Overlord Dynamic Configuration](../configuration/index.md#overlord-dynamic-configuration) for details.
|
||||
|
||||
Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/`
|
||||
as in `2016-06-27_2016-06-28`.
|
||||
|
||||
`GET /druid/indexer/v1/worker`
|
||||
|
||||
Retrieves current overlord dynamic configuration.
|
||||
|
||||
`GET /druid/indexer/v1/worker/history?interval={interval}&count={count}`
|
||||
|
||||
Retrieves history of changes to overlord dynamic configuration. Accepts `interval` and `count` query string parameters
|
||||
to filter by interval and limit the number of results respectively.
|
||||
|
||||
`GET /druid/indexer/v1/workers`
|
||||
|
||||
Retrieves a list of all the worker nodes in the cluster along with its metadata.
|
||||
|
||||
`GET /druid/indexer/v1/scaling`
|
||||
|
||||
Retrieves overlord scaling events if auto-scaling runners are in use.
|
||||
|
||||
`POST /druid/indexer/v1/worker`
|
||||
|
||||
Update overlord dynamic worker configuration.
|
||||
|
||||
## Data server
|
||||
|
||||
This section documents the API endpoints for the processes that reside on Data servers (MiddleManagers/Peons and Historicals)
|
||||
in the suggested [three-server configuration](../design/processes.md#server-types).
|
||||
|
||||
### MiddleManager
|
||||
|
||||
`GET /druid/worker/v1/enabled`
|
||||
|
||||
Check whether a MiddleManager is in an enabled or disabled state. Returns JSON object keyed by the combined `druid.host`
|
||||
and `druid.port` with the boolean state as the value.
|
||||
|
||||
```json
|
||||
{"localhost:8091":true}
|
||||
```
|
||||
|
||||
`GET /druid/worker/v1/tasks`
|
||||
|
||||
Retrieve a list of active tasks being run on MiddleManager. Returns JSON list of taskid strings. Normal usage should
|
||||
prefer to use the `/druid/indexer/v1/tasks` [Overlord API](#overlord) or one of it's task state specific variants instead.
|
||||
|
||||
```json
|
||||
["index_wikiticker_2019-02-11T02:20:15.316Z"]
|
||||
```
|
||||
|
||||
`GET /druid/worker/v1/task/{taskid}/log`
|
||||
|
||||
Retrieve task log output stream by task id. Normal usage should prefer to use the `/druid/indexer/v1/task/{taskId}/log`
|
||||
[Overlord API](#overlord) instead.
|
||||
|
||||
`POST /druid/worker/v1/disable`
|
||||
|
||||
Disable a MiddleManager, causing it to stop accepting new tasks but complete all existing tasks. Returns JSON object
|
||||
keyed by the combined `druid.host` and `druid.port`:
|
||||
|
||||
```json
|
||||
{"localhost:8091":"disabled"}
|
||||
```
|
||||
|
||||
`POST /druid/worker/v1/enable`
|
||||
|
||||
Enable a MiddleManager, allowing it to accept new tasks again if it was previously disabled. Returns JSON object
|
||||
keyed by the combined `druid.host` and `druid.port`:
|
||||
|
||||
```json
|
||||
{"localhost:8091":"enabled"}
|
||||
```
|
||||
|
||||
`POST /druid/worker/v1/task/{taskid}/shutdown`
|
||||
|
||||
Shutdown a running task by `taskid`. Normal usage should prefer to use the `/druid/indexer/v1/task/{taskId}/shutdown`
|
||||
[Overlord API](#overlord) instead. Returns JSON:
|
||||
|
||||
```json
|
||||
{"task":"index_kafka_wikiticker_f7011f8ffba384b_fpeclode"}
|
||||
```
|
||||
|
||||
|
||||
### Peon
|
||||
|
||||
`GET /druid/worker/v1/chat/{taskId}/rowStats`
|
||||
|
||||
Retrieve a live row stats report from a Peon. See [task reports](../ingestion/tasks.md#task-reports) for more details.
|
||||
|
||||
`GET /druid/worker/v1/chat/{taskId}/unparseableEvents`
|
||||
|
||||
Retrieve an unparseable events report from a Peon. See [task reports](../ingestion/tasks.md#task-reports) for more details.
|
||||
|
||||
### Historical
|
||||
|
||||
#### Segment loading
|
||||
|
||||
`GET /druid/historical/v1/loadstatus`
|
||||
|
||||
Returns JSON of the form `{"cacheInitialized":<value>}`, where value is either `true` or `false` indicating if all
|
||||
segments in the local cache have been loaded. This can be used to know when a Historical process is ready
|
||||
to be queried after a restart.
|
||||
|
||||
`GET /druid/historical/v1/readiness`
|
||||
|
||||
Similar to `/druid/historical/v1/loadstatus`, but instead of returning JSON with a flag, responses 200 OK if segments
|
||||
in the local cache have been loaded, and 503 SERVICE UNAVAILABLE, if they haven't.
|
||||
|
||||
|
||||
## Query server
|
||||
|
||||
This section documents the API endpoints for the processes that reside on Query servers (Brokers) in the suggested [three-server configuration](../design/processes.md#server-types).
|
||||
|
||||
### Broker
|
||||
|
||||
#### Datasource information
|
||||
|
||||
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.
|
||||
|
||||
`GET /druid/v2/datasources`
|
||||
|
||||
Returns a list of queryable datasources.
|
||||
|
||||
`GET /druid/v2/datasources/{dataSourceName}`
|
||||
|
||||
Returns the dimensions and metrics of the datasource. Optionally, you can provide request parameter "full" to get list of served intervals with dimensions and metrics being served for those intervals. You can also provide request param "interval" explicitly to refer to a particular interval.
|
||||
|
||||
If no interval is specified, a default interval spanning a configurable period before the current time will be used. The default duration of this interval is specified in ISO 8601 duration format via: `druid.query.segmentMetadata.defaultHistory`
|
||||
|
||||
`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.
|
||||
>
|
||||
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.
|
||||
|
||||
Returns the metrics of the datasource.
|
||||
|
||||
`GET /druid/v2/datasources/{dataSourceName}/candidates?intervals={comma-separated-intervals}&numCandidates={numCandidates}`
|
||||
|
||||
Returns segment information lists including server locations for the given datasource and intervals. If "numCandidates" is not specified, it will return all servers for each interval.
|
||||
|
||||
#### Load Status
|
||||
|
||||
`GET /druid/broker/v1/loadstatus`
|
||||
|
||||
Returns a flag indicating if the Broker knows about all segments in the cluster. This can be used to know when a Broker process is ready to be queried after a restart.
|
||||
|
||||
`GET /druid/broker/v1/readiness`
|
||||
|
||||
Similar to `/druid/broker/v1/loadstatus`, but instead of returning a JSON, responses 200 OK if its ready and otherwise 503 SERVICE UNAVAILABLE.
|
||||
|
||||
#### Queries
|
||||
|
||||
`POST /druid/v2/`
|
||||
|
||||
The endpoint for submitting queries. Accepts an option `?pretty` that pretty prints the results.
|
||||
|
||||
`POST /druid/v2/candidates/`
|
||||
|
||||
Returns segment information lists including server locations for the given query..
|
||||
|
||||
### Router
|
||||
|
||||
> 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`
|
||||
|
||||
Returns a list of queryable datasources.
|
||||
|
||||
`GET /druid/v2/datasources/{dataSourceName}`
|
||||
|
||||
Returns the dimensions and metrics of the datasource.
|
||||
|
||||
`GET /druid/v2/datasources/{dataSourceName}/dimensions`
|
||||
|
||||
Returns the dimensions of the datasource.
|
||||
|
||||
`GET /druid/v2/datasources/{dataSourceName}/metrics`
|
||||
|
||||
Returns the metrics of the datasource.
|
||||
This topic is an index to the Apache Druid API documentation.
|
||||
|
||||
## HTTP APIs
|
||||
* [Druid SQL queries](./sql-api.md) to submit SQL queries using the Druid SQL API.
|
||||
* [SQL-based ingestion](./sql-ingestion-api.md) to submit SQL-based batch ingestion requests.
|
||||
* [JSON querying](./json-querying-api.md) to submit JSON-based native queries.
|
||||
* [Tasks](./tasks-api.md) to manage data ingestion operations.
|
||||
* [Supervisors](./supervisor-api.md) to manage supervisors for data ingestion lifecycle and data processing.
|
||||
* [Retention rules](./retention-rules-api.md) to define and manage data retention rules across datasources.
|
||||
* [Data management](./data-management-api.md) to manage data segments.
|
||||
* [Automatic compaction](./automatic-compaction-api.md) to optimize segment sizes after ingestion.
|
||||
* [Lookups](./lookups-api.md) to manage and modify key-value datasources.
|
||||
* [Service status](./service-status-api.md) to monitor components within the Druid cluster.
|
||||
* [Dynamic configuration](./dynamic-configuration-api.md) to configure the behavior of the Coordinator and Overlord processes.
|
||||
* [Legacy metadata](./legacy-metadata-api.md) to retrieve datasource metadata.
|
||||
|
||||
## Java APIs
|
||||
* [SQL JDBC driver](./sql-jdbc.md) to connect to Druid and make Druid SQL queries using the Avatica JDBC driver.
|
|
@ -0,0 +1,91 @@
|
|||
---
|
||||
id: automatic-compaction-api
|
||||
title: Automatic compaction API
|
||||
sidebar_label: Automatic compaction
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
This document describes status and configuration API endpoints for [automatic compaction](../data-management/automatic-compaction.md) in Apache Druid.
|
||||
|
||||
## Automatic compaction status
|
||||
|
||||
`GET /druid/coordinator/v1/compaction/progress?dataSource={dataSource}`
|
||||
|
||||
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.
|
||||
|
||||
`GET /druid/coordinator/v1/compaction/status`
|
||||
|
||||
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 `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
|
||||
|
||||
`GET /druid/coordinator/v1/compaction/status?dataSource={dataSource}`
|
||||
|
||||
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.
|
||||
|
||||
## Automatic compaction configuration
|
||||
|
||||
`GET /druid/coordinator/v1/config/compaction`
|
||||
|
||||
Returns all automatic compaction configs.
|
||||
|
||||
`GET /druid/coordinator/v1/config/compaction/{dataSource}`
|
||||
|
||||
Returns an automatic compaction config of a dataSource.
|
||||
|
||||
`GET /druid/coordinator/v1/config/compaction/{dataSource}/history?interval={interval}&count={count}`
|
||||
|
||||
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.
|
||||
|
||||
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`.
|
||||
* `auditTime`: The date and time when the change was made.
|
||||
|
||||
`POST /druid/coordinator/v1/config/compaction/taskslots?ratio={someRatio}&max={someMaxSlots}`
|
||||
|
||||
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.
|
||||
|
||||
`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.
|
||||
|
||||
`DELETE /druid/coordinator/v1/config/compaction/{dataSource}`
|
||||
|
||||
Removes the automatic compaction config for a dataSource.
|
|
@ -0,0 +1,79 @@
|
|||
---
|
||||
id: data-management-api
|
||||
title: Data management API
|
||||
sidebar_label: Data management
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
This document describes the data management API endpoints for Apache Druid. This includes information on how to mark segments as `used` or `unused` and delete them from Druid.
|
||||
|
||||
## Note for Coordinator's POST and DELETE APIs
|
||||
|
||||
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.
|
||||
|
||||
`POST /druid/coordinator/v1/datasources/{dataSourceName}`
|
||||
|
||||
Marks as used all segments belonging to a datasource. Returns a JSON object of the form
|
||||
`{"numChangedSegments": <number>}` with the number of segments in the database whose state has been changed (that is,
|
||||
the segments were marked as used) as the result of this API call.
|
||||
|
||||
`POST /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
|
||||
|
||||
Marks as used a segment of a datasource. Returns a JSON object of the form `{"segmentStateChanged": <boolean>}` with
|
||||
the boolean indicating if the state of the segment has been changed (that is, the segment was marked as used) as the
|
||||
result of this API call.
|
||||
|
||||
`POST /druid/coordinator/v1/datasources/{dataSourceName}/markUsed`
|
||||
|
||||
`POST /druid/coordinator/v1/datasources/{dataSourceName}/markUnused`
|
||||
|
||||
Marks segments (un)used for a datasource by interval or set of segment Ids. When marking used only segments that are not overshadowed will be updated.
|
||||
|
||||
The request payload contains the interval or set of segment IDs to be marked unused.
|
||||
Either interval or segment IDs should be provided, if both or none are provided in the payload, the API would throw an error (400 BAD REQUEST).
|
||||
|
||||
Interval specifies the start and end times as IS0 8601 strings. `interval=(start/end)` where start and end both are inclusive and only the segments completely contained within the specified interval will be disabled, partially overlapping segments will not be affected.
|
||||
|
||||
JSON Request Payload:
|
||||
|
||||
|Key|Description|Example|
|
||||
|----------|-------------|---------|
|
||||
|`interval`|The interval for which to mark segments unused|`"2015-09-12T03:00:00.000Z/2015-09-12T05:00:00.000Z"`|
|
||||
|`segmentIds`|Set of segment IDs to be marked unused|`["segmentId1", "segmentId2"]`|
|
||||
|
||||
`DELETE /druid/coordinator/v1/datasources/{dataSourceName}`
|
||||
|
||||
Marks as unused all segments belonging to a datasource. Returns a JSON object of the form
|
||||
`{"numChangedSegments": <number>}` with the number of segments in the database whose state has been changed (that is,
|
||||
the segments were marked as unused) as the result of this API call.
|
||||
|
||||
`DELETE /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}`
|
||||
`@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myInterval}`
|
||||
|
||||
Runs a [Kill task](../ingestion/tasks.md) for a given interval and datasource.
|
||||
|
||||
`DELETE /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
|
||||
|
||||
Marks as unused a segment of a datasource. Returns a JSON object of the form `{"segmentStateChanged": <boolean>}` with
|
||||
the boolean indicating if the state of the segment has been changed (that is, the segment was marked as unused) as the
|
||||
result of this API call.
|
|
@ -0,0 +1,75 @@
|
|||
---
|
||||
id: dynamic-configuration-api
|
||||
title: Dynamic configuration API
|
||||
sidebar_label: Dynamic configuration
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
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.
|
||||
|
||||
## Coordinator dynamic configuration
|
||||
|
||||
See [Coordinator Dynamic Configuration](../configuration/index.md#dynamic-configuration) for details.
|
||||
|
||||
Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/`
|
||||
as in `2016-06-27_2016-06-28`.
|
||||
|
||||
`GET /druid/coordinator/v1/config`
|
||||
|
||||
Retrieves current coordinator dynamic configuration.
|
||||
|
||||
`GET /druid/coordinator/v1/config/history?interval={interval}&count={count}`
|
||||
|
||||
Retrieves history of changes to overlord dynamic configuration. Accepts `interval` and `count` query string parameters
|
||||
to filter by interval and limit the number of results respectively.
|
||||
|
||||
`POST /druid/coordinator/v1/config`
|
||||
|
||||
Update overlord dynamic worker configuration.
|
||||
|
||||
|
||||
## Overlord dynamic configuration
|
||||
|
||||
See [Overlord Dynamic Configuration](../configuration/index.md#overlord-dynamic-configuration) for details.
|
||||
|
||||
Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/`
|
||||
as in `2016-06-27_2016-06-28`.
|
||||
|
||||
`GET /druid/indexer/v1/worker`
|
||||
|
||||
Retrieves current overlord dynamic configuration.
|
||||
|
||||
`GET /druid/indexer/v1/worker/history?interval={interval}&count={count}`
|
||||
|
||||
Retrieves history of changes to overlord dynamic configuration. Accepts `interval` and `count` query string parameters
|
||||
to filter by interval and limit the number of results respectively.
|
||||
|
||||
`GET /druid/indexer/v1/workers`
|
||||
|
||||
Retrieves a list of all the worker nodes in the cluster along with its metadata.
|
||||
|
||||
`GET /druid/indexer/v1/scaling`
|
||||
|
||||
Retrieves overlord scaling events if auto-scaling runners are in use.
|
||||
|
||||
`POST /druid/indexer/v1/worker`
|
||||
|
||||
Update overlord dynamic worker configuration.
|
|
@ -0,0 +1,36 @@
|
|||
---
|
||||
id: json-querying-api
|
||||
title: JSON querying API
|
||||
sidebar_label: JSON querying
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
This document describes the API endpoints to submit JSON-based [native queries](../querying/querying.md) to Apache Druid.
|
||||
|
||||
## Queries
|
||||
|
||||
`POST /druid/v2/`
|
||||
|
||||
The endpoint for submitting queries. Accepts an option `?pretty` that pretty prints the results.
|
||||
|
||||
`POST /druid/v2/candidates/`
|
||||
|
||||
Returns segment information lists including server locations for the given query.
|
|
@ -0,0 +1,315 @@
|
|||
---
|
||||
id: legacy-metadata-api
|
||||
title: Legacy metadata API
|
||||
sidebar_label: Legacy metadata
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
This document describes the legacy API endpoints to retrieve datasource metadata from Apache Druid. Use the [SQL metadata tables](../querying/sql-metadata-tables.md) to retrieve datasource metadata instead.
|
||||
|
||||
## Segment loading
|
||||
|
||||
`GET /druid/coordinator/v1/loadstatus`
|
||||
|
||||
Returns the percentage of segments actually loaded in the cluster versus segments that should be loaded in the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/loadstatus?simple`
|
||||
|
||||
Returns the number of segments left to load until segments that should be loaded in the cluster are available for queries. This does not include segment replication counts.
|
||||
|
||||
`GET /druid/coordinator/v1/loadstatus?full`
|
||||
|
||||
Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts.
|
||||
|
||||
`GET /druid/coordinator/v1/loadstatus?full&computeUsingClusterView`
|
||||
|
||||
Returns the number of segments not yet loaded for each tier until all segments loading in the cluster are available.
|
||||
The result includes segment replication counts. It also factors in the number of available nodes that are of a service type that can load the segment when computing the number of segments remaining to load.
|
||||
A segment is considered fully loaded when:
|
||||
- Druid has replicated it the number of times configured in the corresponding load rule.
|
||||
- Or the number of replicas for the segment in each tier where it is configured to be replicated equals the available nodes of a service type that are currently allowed to load the segment in the tier.
|
||||
|
||||
`GET /druid/coordinator/v1/loadqueue`
|
||||
|
||||
Returns the ids of segments to load and drop for each Historical process.
|
||||
|
||||
`GET /druid/coordinator/v1/loadqueue?simple`
|
||||
|
||||
Returns the number of segments to load and drop, as well as the total segment load and drop size in bytes for each Historical process.
|
||||
|
||||
`GET /druid/coordinator/v1/loadqueue?full`
|
||||
|
||||
Returns the serialized JSON of segments to load and drop for each Historical process.
|
||||
|
||||
## Segment loading by datasource
|
||||
|
||||
Note that all _interval_ query parameters are ISO 8601 strings—for example, 2016-06-27/2016-06-28.
|
||||
Also note that these APIs only guarantees that the segments are available at the time of the call.
|
||||
Segments can still become missing because of historical process failures or any other reasons afterward.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?forceMetadataRefresh={boolean}&interval={myInterval}`
|
||||
|
||||
Returns the percentage of segments actually loaded in the cluster versus segments that should be loaded in the cluster for the given
|
||||
datasource over the given interval (or last 2 weeks if interval is not given). `forceMetadataRefresh` is required to be set.
|
||||
* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store
|
||||
(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms
|
||||
of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status)
|
||||
* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh.
|
||||
If no used segments are found for the given inputs, this API returns `204 No Content`
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?simple&forceMetadataRefresh={boolean}&interval={myInterval}`
|
||||
|
||||
Returns the number of segments left to load until segments that should be loaded in the cluster are available for the given datasource
|
||||
over the given interval (or last 2 weeks if interval is not given). This does not include segment replication counts. `forceMetadataRefresh` is required to be set.
|
||||
* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store
|
||||
(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms
|
||||
of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status)
|
||||
* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh.
|
||||
If no used segments are found for the given inputs, this API returns `204 No Content`
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?full&forceMetadataRefresh={boolean}&interval={myInterval}`
|
||||
|
||||
Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available for the given datasource over the given interval (or last 2 weeks if interval is not given). This includes segment replication counts. `forceMetadataRefresh` is required to be set.
|
||||
* Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store
|
||||
(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms
|
||||
of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status)
|
||||
* Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh.
|
||||
|
||||
You can pass the optional query parameter `computeUsingClusterView` to factor in the available cluster services when calculating
|
||||
the segments left to load. See [Coordinator Segment Loading](#segment-loading) for details.
|
||||
If no used segments are found for the given inputs, this API returns `204 No Content`
|
||||
|
||||
## 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.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/segments`
|
||||
|
||||
Returns a list of all segments for each datasource enabled in the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/segments?datasources={dataSourceName1}&datasources={dataSourceName2}`
|
||||
|
||||
Returns a list of all segments for one or more specific datasources enabled in the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/segments?includeOvershadowedStatus`
|
||||
|
||||
Returns a list of all segments for each datasource with the full segment metadata and an extra field `overshadowed`.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/segments?includeOvershadowedStatus&datasources={dataSourceName1}&datasources={dataSourceName2}`
|
||||
|
||||
Returns a list of all segments for one or more specific datasources with the full segment metadata and an extra field `overshadowed`.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources`
|
||||
|
||||
Returns a list of the names of datasources with at least one used segment in the cluster, retrieved from the metadata database. Users should call this API to get the eventual state that the system will be in.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources?includeUnused`
|
||||
|
||||
Returns a list of the names of datasources, regardless of whether there are used segments belonging to those datasources in the cluster or not.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources?includeDisabled`
|
||||
|
||||
Returns a list of the names of datasources, regardless of whether the datasource is disabled or not.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources?full`
|
||||
|
||||
Returns a list of all datasources with at least one used segment in the cluster. Returns all metadata about those datasources as stored in the metadata store.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}`
|
||||
|
||||
Returns full metadata for a datasource as stored in the metadata store.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments`
|
||||
|
||||
Returns a list of all segments for a datasource as stored in the metadata store.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full`
|
||||
|
||||
Returns a list of all segments for a datasource with the full segment metadata as stored in the metadata store.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments/{segmentId}`
|
||||
|
||||
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`
|
||||
|
||||
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"]`.
|
||||
|
||||
`GET /druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments?full`
|
||||
|
||||
Returns a list of all segments, overlapping with any of given intervals, for a datasource with the full segment metadata as stored in the metadata store. Request body is array of string ISO 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"]`.
|
||||
|
||||
<a name="coordinator-datasources"></a>
|
||||
|
||||
## Datasources
|
||||
|
||||
Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/`—for example, `2016-06-27_2016-06-28`.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources`
|
||||
|
||||
Returns a list of datasource names found in the cluster as seen by the coordinator. This view is updated every [`druid.coordinator.period`](../configuration/index.md#coordinator-operation).
|
||||
|
||||
`GET /druid/coordinator/v1/datasources?simple`
|
||||
|
||||
Returns a list of JSON objects containing the name and properties of datasources found in the cluster. Properties include segment count, total segment byte size, replicated total segment byte size, minTime, and maxTime.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources?full`
|
||||
|
||||
Returns a list of datasource names found in the cluster with all metadata about those datasources.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}`
|
||||
|
||||
Returns a JSON object containing the name and properties of a datasource. Properties include segment count, total segment byte size, replicated total segment byte size, minTime, and maxTime.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}?full`
|
||||
|
||||
Returns full metadata for a datasource.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals`
|
||||
|
||||
Returns a set of segment intervals.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals?simple`
|
||||
|
||||
Returns a map of an interval to a JSON object containing the total byte size of segments and number of segments for that interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals?full`
|
||||
|
||||
Returns a map of an interval to a map of segment metadata to a set of server names that contain the segment for that interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}`
|
||||
|
||||
Returns a set of segment ids for an interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}?simple`
|
||||
|
||||
Returns a map of segment intervals contained within the specified interval to a JSON object containing the total byte size of segments and number of segments for an interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}?full`
|
||||
|
||||
Returns a map of segment intervals contained within the specified interval to a map of segment metadata to a set of server names that contain the segment for an interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}/serverview`
|
||||
|
||||
Returns a map of segment intervals contained within the specified interval to information about the servers that contain the segment for an interval.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments`
|
||||
|
||||
Returns a list of all segments for a datasource in the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments?full`
|
||||
|
||||
Returns a list of all segments for a datasource in the cluster with the full segment metadata.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
|
||||
|
||||
Returns full segment metadata for a specific segment in the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/datasources/{dataSourceName}/tiers`
|
||||
|
||||
Return the tiers that a datasource exists in.
|
||||
|
||||
## Intervals
|
||||
|
||||
Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/` as in `2016-06-27_2016-06-28`.
|
||||
|
||||
`GET /druid/coordinator/v1/intervals`
|
||||
|
||||
Returns all intervals for all datasources with total size and count.
|
||||
|
||||
`GET /druid/coordinator/v1/intervals/{interval}`
|
||||
|
||||
Returns aggregated total size and count for all intervals that intersect given ISO interval.
|
||||
|
||||
`GET /druid/coordinator/v1/intervals/{interval}?simple`
|
||||
|
||||
Returns total size and count for each interval within given ISO interval.
|
||||
|
||||
`GET /druid/coordinator/v1/intervals/{interval}?full`
|
||||
|
||||
Returns total size and count for each datasource for each interval within given ISO interval.
|
||||
|
||||
## Server information
|
||||
|
||||
`GET /druid/coordinator/v1/servers`
|
||||
|
||||
Returns a list of servers URLs using the format `{hostname}:{port}`. Note that
|
||||
processes that run with different types will appear multiple times with different
|
||||
ports.
|
||||
|
||||
`GET /druid/coordinator/v1/servers?simple`
|
||||
|
||||
Returns a list of server data objects in which each object has the following keys:
|
||||
* `host`: host URL include (`{hostname}:{port}`)
|
||||
* `type`: process type (`indexer-executor`, `historical`)
|
||||
* `currSize`: storage size currently used
|
||||
* `maxSize`: maximum storage size
|
||||
* `priority`
|
||||
* `tier`
|
||||
|
||||
|
||||
## Query server
|
||||
|
||||
This section documents the API endpoints for the processes that reside on Query servers (Brokers) in the suggested [three-server configuration](../design/processes.md#server-types).
|
||||
|
||||
### Broker
|
||||
|
||||
#### Datasource information
|
||||
|
||||
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.
|
||||
|
||||
`GET /druid/v2/datasources`
|
||||
|
||||
Returns a list of queryable datasources.
|
||||
|
||||
`GET /druid/v2/datasources/{dataSourceName}`
|
||||
|
||||
Returns the dimensions and metrics of the datasource. Optionally, you can provide request parameter "full" to get list of served intervals with dimensions and metrics being served for those intervals. You can also provide request param "interval" explicitly to refer to a particular interval.
|
||||
|
||||
If no interval is specified, a default interval spanning a configurable period before the current time will be used. The default duration of this interval is specified in ISO 8601 duration format via: `druid.query.segmentMetadata.defaultHistory`
|
||||
|
||||
`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.
|
||||
>
|
||||
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.
|
||||
|
||||
Returns the metrics of the datasource.
|
||||
|
||||
`GET /druid/v2/datasources/{dataSourceName}/candidates?intervals={comma-separated-intervals}&numCandidates={numCandidates}`
|
||||
|
||||
Returns segment information lists including server locations for the given datasource and intervals. If "numCandidates" is not specified, it will return all servers for each interval.
|
|
@ -0,0 +1,278 @@
|
|||
---
|
||||
id: lookups-api
|
||||
title: Lookups API
|
||||
sidebar_label: Lookups
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
This document describes the API endpoints to configure, update, retrieve, and manage lookups for Apache Druid.
|
||||
|
||||
## Configure lookups
|
||||
|
||||
### Bulk update
|
||||
Lookups can be updated in bulk by posting a JSON object to `/druid/coordinator/v1/lookups/config`. The format of the json object is as follows:
|
||||
|
||||
```json
|
||||
{
|
||||
"<tierName>": {
|
||||
"<lookupName>": {
|
||||
"version": "<version>",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "<someExtractorFactoryType>",
|
||||
"<someExtractorField>": "<someExtractorValue>"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
Note that "version" is an arbitrary string assigned by the user, when making updates to existing lookup then user would need to specify a lexicographically higher version.
|
||||
|
||||
For example, a config might look something like:
|
||||
|
||||
```json
|
||||
{
|
||||
"__default": {
|
||||
"country_code": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"77483": "United States"
|
||||
}
|
||||
}
|
||||
},
|
||||
"site_id": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "cachedNamespace",
|
||||
"extractionNamespace": {
|
||||
"type": "jdbc",
|
||||
"connectorConfig": {
|
||||
"createTables": true,
|
||||
"connectURI": "jdbc:mysql:\/\/localhost:3306\/druid",
|
||||
"user": "druid",
|
||||
"password": "diurd"
|
||||
},
|
||||
"table": "lookupTable",
|
||||
"keyColumn": "country_id",
|
||||
"valueColumn": "country_name",
|
||||
"tsColumn": "timeColumn"
|
||||
},
|
||||
"firstCacheTimeout": 120000,
|
||||
"injective": true
|
||||
}
|
||||
},
|
||||
"site_id_customer1": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"847632": "Internal Use Only"
|
||||
}
|
||||
}
|
||||
},
|
||||
"site_id_customer2": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"AHF77": "Home"
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"realtime_customer1": {
|
||||
"country_code": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"77483": "United States"
|
||||
}
|
||||
}
|
||||
},
|
||||
"site_id_customer1": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"847632": "Internal Use Only"
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"realtime_customer2": {
|
||||
"country_code": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"77483": "United States"
|
||||
}
|
||||
}
|
||||
},
|
||||
"site_id_customer2": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"AHF77": "Home"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
All entries in the map will UPDATE existing entries. No entries will be deleted.
|
||||
|
||||
### Update lookup
|
||||
|
||||
A `POST` to a particular lookup extractor factory via `/druid/coordinator/v1/lookups/config/{tier}/{id}` creates or updates that specific extractor factory.
|
||||
|
||||
For example, a post to `/druid/coordinator/v1/lookups/config/realtime_customer1/site_id_customer1` might contain the following:
|
||||
|
||||
```json
|
||||
{
|
||||
"version": "v1",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"847632": "Internal Use Only"
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
This will replace the `site_id_customer1` lookup in the `realtime_customer1` with the definition above.
|
||||
|
||||
Assign a unique version identifier each time you update a lookup extractor factory. Otherwise the call will fail.
|
||||
|
||||
### Get all lookups
|
||||
|
||||
A `GET` to `/druid/coordinator/v1/lookups/config/all` will return all known lookup specs for all tiers.
|
||||
|
||||
### Get lookup
|
||||
|
||||
A `GET` to a particular lookup extractor factory is accomplished via `/druid/coordinator/v1/lookups/config/{tier}/{id}`
|
||||
|
||||
Using the prior example, a `GET` to `/druid/coordinator/v1/lookups/config/realtime_customer2/site_id_customer2` should return
|
||||
|
||||
```json
|
||||
{
|
||||
"version": "v1",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"AHF77": "Home"
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
### Delete lookup
|
||||
|
||||
A `DELETE` to `/druid/coordinator/v1/lookups/config/{tier}/{id}` will remove that lookup from the cluster. If it was last lookup in the tier, then tier is deleted as well.
|
||||
|
||||
### Delete tier
|
||||
|
||||
A `DELETE` to `/druid/coordinator/v1/lookups/config/{tier}` will remove that tier from the cluster.
|
||||
|
||||
### List tier names
|
||||
|
||||
A `GET` to `/druid/coordinator/v1/lookups/config` will return a list of known tier names in the dynamic configuration.
|
||||
To discover a list of tiers currently active in the cluster in addition to ones known in the dynamic configuration, the parameter `discover=true` can be added as per `/druid/coordinator/v1/lookups/config?discover=true`.
|
||||
|
||||
### List lookup names
|
||||
|
||||
A `GET` to `/druid/coordinator/v1/lookups/config/{tier}` will return a list of known lookup names for that tier.
|
||||
|
||||
These end points can be used to get the propagation status of configured lookups to processes using lookups such as Historicals.
|
||||
|
||||
## Lookup status
|
||||
|
||||
### List load status of all lookups
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/status` with optional query parameter `detailed`.
|
||||
|
||||
### List load status of lookups in a tier
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/status/{tier}` with optional query parameter `detailed`.
|
||||
|
||||
### List load status of single lookup
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/status/{tier}/{lookup}` with optional query parameter `detailed`.
|
||||
|
||||
### List lookup state of all processes
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/nodeStatus` with optional query parameter `discover` to discover tiers advertised by other Druid nodes, or by default, returning all configured lookup tiers. The default response will also include the lookups which are loaded, being loaded, or being dropped on each node, for each tier, including the complete lookup spec. Add the optional query parameter `detailed=false` to only include the 'version' of the lookup instead of the complete spec.
|
||||
|
||||
### List lookup state of processes in a tier
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}`
|
||||
|
||||
### List lookup state of single process
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}/{host:port}`
|
||||
|
||||
## Internal API
|
||||
|
||||
The Peon, Router, Broker, and Historical processes all have the ability to consume lookup configuration.
|
||||
There is an internal API these processes use to list/load/drop their lookups starting at `/druid/listen/v1/lookups`.
|
||||
These follow the same convention for return values as the cluster wide dynamic configuration. Following endpoints
|
||||
can be used for debugging purposes but not otherwise.
|
||||
|
||||
### Get lookups
|
||||
|
||||
A `GET` to the process at `/druid/listen/v1/lookups` will return a json map of all the lookups currently active on the process.
|
||||
The return value will be a json map of the lookups to their extractor factories.
|
||||
|
||||
```json
|
||||
{
|
||||
"site_id_customer2": {
|
||||
"version": "v1",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"AHF77": "Home"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
### Get lookup
|
||||
|
||||
A `GET` to the process at `/druid/listen/v1/lookups/some_lookup_name` will return the LookupExtractorFactory for the lookup identified by `some_lookup_name`.
|
||||
The return value will be the json representation of the factory.
|
||||
|
||||
```json
|
||||
{
|
||||
"version": "v1",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"AHF77": "Home"
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
|
@ -0,0 +1,69 @@
|
|||
---
|
||||
id: retention-rules-api
|
||||
title: Retention rules API
|
||||
sidebar_label: Retention rules
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
This document describes the API endpoints for managing retention rules in Apache Druid.
|
||||
|
||||
## 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`.
|
||||
|
||||
`GET /druid/coordinator/v1/rules`
|
||||
|
||||
Returns all rules as JSON objects for all datasources in the cluster including the default datasource.
|
||||
|
||||
`GET /druid/coordinator/v1/rules/{dataSourceName}`
|
||||
|
||||
Returns all rules for a specified datasource.
|
||||
|
||||
`GET /druid/coordinator/v1/rules/{dataSourceName}?full`
|
||||
|
||||
Returns all rules for a specified datasource and includes default datasource.
|
||||
|
||||
`GET /druid/coordinator/v1/rules/history?interval=<interval>`
|
||||
|
||||
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=<n>`
|
||||
|
||||
Returns last `n` entries of audit history of rules for all datasources.
|
||||
|
||||
`GET /druid/coordinator/v1/rules/{dataSourceName}/history?interval=<interval>`
|
||||
|
||||
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=<n>`
|
||||
|
||||
Returns last `n` entries of audit history of rules for a specified datasource.
|
||||
|
||||
`POST /druid/coordinator/v1/rules/{dataSourceName}`
|
||||
|
||||
POST with a list of rules in JSON form to update rules.
|
||||
|
||||
Optional Header Parameters for auditing the config change can also be specified.
|
||||
|
||||
|Header Param Name| Description | Default |
|
||||
|----------|-------------|---------|
|
||||
|`X-Druid-Author`| Author making the config change|`""`|
|
||||
|`X-Druid-Comment`| Comment describing the change being done|`""`|
|
|
@ -0,0 +1,176 @@
|
|||
---
|
||||
id: service-status-api
|
||||
title: Service status API
|
||||
sidebar_label: Service status
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
This document describes the API endpoints to retrieve service (process) status, cluster information for Apache Druid
|
||||
|
||||
## Common
|
||||
|
||||
All processes support the following endpoints.
|
||||
|
||||
### Process information
|
||||
|
||||
`GET /status`
|
||||
|
||||
Returns the Druid version, loaded extensions, memory used, total memory, and other useful information about the process.
|
||||
|
||||
`GET /status/health`
|
||||
|
||||
Always returns a boolean `true` value with a 200 OK response, useful for automated health checks.
|
||||
|
||||
`GET /status/properties`
|
||||
|
||||
Returns the current configuration properties of the process.
|
||||
|
||||
`GET /status/selfDiscovered/status`
|
||||
|
||||
Returns 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. It is recommended to not consider a Druid node "healthy" or "ready" in automated deployment/container
|
||||
management systems until it returns `{"selfDiscovered": true}` from this endpoint. This is because a node may be
|
||||
isolated from the rest of the cluster due to network issues and it doesn't make sense to consider nodes "healthy" in
|
||||
this case. Also, when nodes such as Brokers use ZooKeeper segment discovery for building their view of the Druid cluster
|
||||
(as opposed to HTTP segment discovery), they may be unusable until the ZooKeeper client is fully initialized and starts
|
||||
to receive data from the ZooKeeper cluster. `{"selfDiscovered": true}` is a proxy event indicating that the ZooKeeper
|
||||
client on the node has started to receive data from the ZooKeeper cluster and it's expected that all segments and other
|
||||
nodes will be discovered by this node timely from this point.
|
||||
|
||||
`GET /status/selfDiscovered`
|
||||
|
||||
Similar to `/status/selfDiscovered/status`, but returns 200 OK response with empty body if the node has discovered itself
|
||||
and 503 SERVICE UNAVAILABLE if the node hasn't discovered itself yet. This endpoint might be useful because some
|
||||
monitoring checks such as AWS load balancer health checks are not able to look at the response body.
|
||||
|
||||
## Master server
|
||||
|
||||
### Coordinator
|
||||
|
||||
#### Leadership
|
||||
|
||||
`GET /druid/coordinator/v1/leader`
|
||||
|
||||
Returns the current leader Coordinator of the cluster.
|
||||
|
||||
`GET /druid/coordinator/v1/isLeader`
|
||||
|
||||
Returns a JSON object with `leader` parameter, either true or false, indicating if this server is the current leader
|
||||
Coordinator of the cluster. In addition, returns HTTP 200 if the server is the current leader and HTTP 404 if not.
|
||||
This is suitable for use as a load balancer status check if you only want the active leader to be considered in-service
|
||||
at the load balancer.
|
||||
|
||||
<a name="coordinator-segment-loading"></a>
|
||||
|
||||
### Overlord
|
||||
|
||||
#### Leadership
|
||||
|
||||
`GET /druid/indexer/v1/leader`
|
||||
|
||||
Returns the current leader Overlord of the cluster. If you have multiple Overlords, just one is leading at any given time. The others are on standby.
|
||||
|
||||
`GET /druid/indexer/v1/isLeader`
|
||||
|
||||
This returns a JSON object with field `leader`, either true or false. In addition, this call returns HTTP 200 if the
|
||||
server is the current leader and HTTP 404 if not. This is suitable for use as a load balancer status check if you
|
||||
only want the active leader to be considered in-service at the load balancer.
|
||||
|
||||
## Data server
|
||||
|
||||
### MiddleManager
|
||||
|
||||
`GET /druid/worker/v1/enabled`
|
||||
|
||||
Check whether a MiddleManager is in an enabled or disabled state. Returns JSON object keyed by the combined `druid.host`
|
||||
and `druid.port` with the boolean state as the value.
|
||||
|
||||
```json
|
||||
{"localhost:8091":true}
|
||||
```
|
||||
|
||||
`GET /druid/worker/v1/tasks`
|
||||
|
||||
Retrieve a list of active tasks being run on MiddleManager. Returns JSON list of taskid strings. Normal usage should
|
||||
prefer to use the `/druid/indexer/v1/tasks` [Tasks API](./tasks-api.md) or one of it's task state specific variants instead.
|
||||
|
||||
```json
|
||||
["index_wikiticker_2019-02-11T02:20:15.316Z"]
|
||||
```
|
||||
|
||||
`GET /druid/worker/v1/task/{taskid}/log`
|
||||
|
||||
Retrieve task log output stream by task id. Normal usage should prefer to use the `/druid/indexer/v1/task/{taskId}/log`
|
||||
[Tasks API](./tasks-api.md) instead.
|
||||
|
||||
`POST /druid/worker/v1/disable`
|
||||
|
||||
Disable a MiddleManager, causing it to stop accepting new tasks but complete all existing tasks. Returns JSON object
|
||||
keyed by the combined `druid.host` and `druid.port`:
|
||||
|
||||
```json
|
||||
{"localhost:8091":"disabled"}
|
||||
```
|
||||
|
||||
`POST /druid/worker/v1/enable`
|
||||
|
||||
Enable a MiddleManager, allowing it to accept new tasks again if it was previously disabled. Returns JSON object
|
||||
keyed by the combined `druid.host` and `druid.port`:
|
||||
|
||||
```json
|
||||
{"localhost:8091":"enabled"}
|
||||
```
|
||||
|
||||
`POST /druid/worker/v1/task/{taskid}/shutdown`
|
||||
|
||||
Shutdown a running task by `taskid`. Normal usage should prefer to use the `/druid/indexer/v1/task/{taskId}/shutdown`
|
||||
[Tasks API](./tasks-api.md) instead. Returns JSON:
|
||||
|
||||
```json
|
||||
{"task":"index_kafka_wikiticker_f7011f8ffba384b_fpeclode"}
|
||||
```
|
||||
|
||||
|
||||
## Historical
|
||||
### Segment loading
|
||||
|
||||
`GET /druid/historical/v1/loadstatus`
|
||||
|
||||
Returns JSON of the form `{"cacheInitialized":<value>}`, where value is either `true` or `false` indicating if all
|
||||
segments in the local cache have been loaded. This can be used to know when a Historical process is ready
|
||||
to be queried after a restart.
|
||||
|
||||
`GET /druid/historical/v1/readiness`
|
||||
|
||||
Similar to `/druid/historical/v1/loadstatus`, but instead of returning JSON with a flag, responses 200 OK if segments
|
||||
in the local cache have been loaded, and 503 SERVICE UNAVAILABLE, if they haven't.
|
||||
|
||||
|
||||
## Load Status
|
||||
|
||||
`GET /druid/broker/v1/loadstatus`
|
||||
|
||||
Returns a flag indicating if the Broker knows about all segments in the cluster. This can be used to know when a Broker process is ready to be queried after a restart.
|
||||
|
||||
`GET /druid/broker/v1/readiness`
|
||||
|
||||
Similar to `/druid/broker/v1/loadstatus`, but instead of returning a JSON, responses 200 OK if its ready and otherwise 503 SERVICE UNAVAILABLE.
|
|
@ -186,4 +186,4 @@ 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.
|
||||
Druid returns an HTTP 403 response for authorization failure.
|
|
@ -123,7 +123,7 @@ print(response.text)
|
|||
|
||||
| Field | Description |
|
||||
|---|---|
|
||||
| `taskId` | Controller task ID. You can use Druid's standard [task APIs](api-reference.md#overlord) to interact with this controller task. |
|
||||
| `taskId` | Controller task ID. You can use Druid's standard [Tasks API](./tasks-api.md) to interact with this controller task. |
|
||||
| `state` | Initial state for the query, which is "RUNNING". |
|
||||
|
||||
## Get the status for a query task
|
||||
|
@ -628,4 +628,4 @@ print(response.text)
|
|||
{
|
||||
"task": "query-655efe33-781a-4c50-ae84-c2911b42d63c"
|
||||
}
|
||||
```
|
||||
```
|
|
@ -0,0 +1,111 @@
|
|||
---
|
||||
id: supervisor-api
|
||||
title: Supervisor API
|
||||
sidebar_label: Supervisors
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
This document describes the API endpoints to manage and monitor supervisors for Apache Druid.
|
||||
|
||||
## Supervisors
|
||||
|
||||
`GET /druid/indexer/v1/supervisor`
|
||||
|
||||
Returns a list of strings of the currently active supervisor ids.
|
||||
|
||||
`GET /druid/indexer/v1/supervisor?full`
|
||||
|
||||
Returns a list of objects of the currently active supervisors.
|
||||
|
||||
|Field|Type|Description|
|
||||
|---|---|---|
|
||||
|`id`|String|supervisor unique identifier|
|
||||
|`state`|String|basic state of the supervisor. Available states:`UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`. Check [Kafka Docs](../development/extensions-core/kafka-supervisor-operations.md) for details.|
|
||||
|`detailedState`|String|supervisor specific state. See documentation of specific supervisor for details: [Kafka](../development/extensions-core/kafka-ingestion.md) or [Kinesis](../development/extensions-core/kinesis-ingestion.md)|
|
||||
|`healthy`|Boolean|true or false indicator of overall supervisor health|
|
||||
|`spec`|SupervisorSpec|JSON specification of supervisor|
|
||||
|
||||
`GET /druid/indexer/v1/supervisor?state=true`
|
||||
|
||||
Returns a list of objects of the currently active supervisors and their current state.
|
||||
|
||||
|Field|Type|Description|
|
||||
|---|---|---|
|
||||
|`id`|String|supervisor unique identifier|
|
||||
|`state`|String|basic state of the supervisor. Available states: `UNHEALTHY_SUPERVISOR`, `UNHEALTHY_TASKS`, `PENDING`, `RUNNING`, `SUSPENDED`, `STOPPING`. Check [Kafka Docs](../development/extensions-core/kafka-supervisor-operations.md) for details.|
|
||||
|`detailedState`|String|supervisor specific state. See documentation of the specific supervisor for details: [Kafka](../development/extensions-core/kafka-ingestion.md) or [Kinesis](../development/extensions-core/kinesis-ingestion.md)|
|
||||
|`healthy`|Boolean|true or false indicator of overall supervisor health|
|
||||
|`suspended`|Boolean|true or false indicator of whether the supervisor is in suspended state|
|
||||
|
||||
`GET /druid/indexer/v1/supervisor/<supervisorId>`
|
||||
|
||||
Returns the current spec for the supervisor with the provided ID.
|
||||
|
||||
`GET /druid/indexer/v1/supervisor/<supervisorId>/status`
|
||||
|
||||
Returns the current status of the supervisor with the provided ID.
|
||||
|
||||
`GET/druid/indexer/v1/supervisor/history`
|
||||
|
||||
Returns an audit history of specs for all supervisors (current and past).
|
||||
|
||||
`GET /druid/indexer/v1/supervisor/<supervisorId>/history`
|
||||
|
||||
Returns an audit history of specs for the supervisor with the provided ID.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor`
|
||||
|
||||
Create a new supervisor or update an existing one.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/<supervisorId>/suspend`
|
||||
|
||||
Suspend the current running supervisor of the provided ID. Responds with updated SupervisorSpec.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/suspendAll`
|
||||
|
||||
Suspend all supervisors at once.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/<supervisorId>/resume`
|
||||
|
||||
Resume indexing tasks for a supervisor. Responds with updated SupervisorSpec.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/resumeAll`
|
||||
|
||||
Resume all supervisors at once.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/<supervisorId>/reset`
|
||||
|
||||
Reset the specified supervisor.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/<supervisorId>/terminate`
|
||||
|
||||
Terminate a supervisor of the provided ID.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/terminateAll`
|
||||
|
||||
Terminate all supervisors at once.
|
||||
|
||||
`POST /druid/indexer/v1/supervisor/<supervisorId>/shutdown`
|
||||
|
||||
> This API is deprecated and will be removed in future releases.
|
||||
> Please use the equivalent `terminate` instead.
|
||||
|
||||
Shutdown a supervisor.
|
|
@ -0,0 +1,101 @@
|
|||
---
|
||||
id: tasks-api
|
||||
title: Tasks API
|
||||
sidebar_label: Tasks
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
This document describes the API endpoints for task retrieval, submission, and deletion for Apache Druid.
|
||||
|
||||
## Tasks
|
||||
|
||||
Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` instead of a `/`
|
||||
as in `2016-06-27_2016-06-28`.
|
||||
|
||||
`GET /druid/indexer/v1/tasks`
|
||||
|
||||
Retrieve list of tasks. Accepts query string parameters `state`, `datasource`, `createdTimeInterval`, `max`, and `type`.
|
||||
|
||||
|Query Parameter |Description |
|
||||
|---|---|
|
||||
|`state`|filter list of tasks by task state, valid options are `running`, `complete`, `waiting`, and `pending`.|
|
||||
| `datasource`| return tasks filtered by Druid datasource.|
|
||||
| `createdTimeInterval`| return tasks created within the specified interval. |
|
||||
| `max`| maximum number of `"complete"` tasks to return. Only applies when `state` is set to `"complete"`.|
|
||||
| `type`| filter tasks by task type. See [task documentation](../ingestion/tasks.md) for more details.|
|
||||
|
||||
|
||||
`GET /druid/indexer/v1/completeTasks`
|
||||
|
||||
Retrieve list of complete tasks. Equivalent to `/druid/indexer/v1/tasks?state=complete`.
|
||||
|
||||
`GET /druid/indexer/v1/runningTasks`
|
||||
|
||||
Retrieve list of running tasks. Equivalent to `/druid/indexer/v1/tasks?state=running`.
|
||||
|
||||
`GET /druid/indexer/v1/waitingTasks`
|
||||
|
||||
Retrieve list of waiting tasks. Equivalent to `/druid/indexer/v1/tasks?state=waiting`.
|
||||
|
||||
`GET /druid/indexer/v1/pendingTasks`
|
||||
|
||||
Retrieve list of pending tasks. Equivalent to `/druid/indexer/v1/tasks?state=pending`.
|
||||
|
||||
`GET /druid/indexer/v1/task/{taskId}`
|
||||
|
||||
Retrieve the 'payload' of a task.
|
||||
|
||||
`GET /druid/indexer/v1/task/{taskId}/status`
|
||||
|
||||
Retrieve the status of a task.
|
||||
|
||||
`GET /druid/indexer/v1/task/{taskId}/segments`
|
||||
|
||||
> This API is deprecated and will be removed in future releases.
|
||||
|
||||
Retrieve information about the segments of a task.
|
||||
|
||||
`GET /druid/indexer/v1/task/{taskId}/reports`
|
||||
|
||||
Retrieve a [task completion report](../ingestion/tasks.md#task-reports) for a task. Only works for completed tasks.
|
||||
|
||||
`POST /druid/indexer/v1/task`
|
||||
|
||||
Endpoint for submitting tasks and supervisor specs to the Overlord. Returns the taskId of the submitted task.
|
||||
|
||||
`POST /druid/indexer/v1/task/{taskId}/shutdown`
|
||||
|
||||
Shuts down a task.
|
||||
|
||||
`POST /druid/indexer/v1/datasources/{dataSource}/shutdownAllTasks`
|
||||
|
||||
Shuts down all tasks for a dataSource.
|
||||
|
||||
`POST /druid/indexer/v1/taskStatus`
|
||||
|
||||
Retrieve list of task status objects for list of task id strings in request body.
|
||||
|
||||
`DELETE /druid/indexer/v1/pendingSegments/{dataSource}`
|
||||
|
||||
Manually clean up pending segments table in metadata storage for `datasource`. Returns a JSON object response with
|
||||
`numDeleted` and count of rows deleted from the pending segments table. This API is used by the
|
||||
`druid.coordinator.kill.pendingSegments.on` [coordinator setting](../configuration/index.md#coordinator-operation)
|
||||
which automates this operation to perform periodically.
|
|
@ -399,6 +399,7 @@ Metric monitoring is an essential part of Druid operations. The following monit
|
|||
|`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.|
|
||||
|`org.apache.druid.server.metrics.WorkerTaskCountStatsMonitor`|Reports how many ingestion tasks are currently running/pending/waiting, the number of successful/failed tasks, and metrics about task slot usage for the reporting worker, per emission period. Only supported by middleManager node types.|
|
||||
|`org.apache.druid.server.metrics.ServiceStatusMonitor`|Reports a heartbeat for the service.|
|
||||
|
||||
For example, you might configure monitors on all processes for system and JVM information within `common.runtime.properties` as follows:
|
||||
|
||||
|
@ -427,8 +428,8 @@ The use this emitter module, set `druid.emitter=logging`. The `logging` emitter
|
|||
[Marker](https://logging.apache.org/log4j/2.x/manual/markers.html) as the feed of the event. Users may wish to edit the
|
||||
log4j config to route these logs to different sources based on the feed of the event.
|
||||
|
||||
|Property|Description| Default |
|
||||
|--------|-----------|-----------------------------------------------------|
|
||||
|Property|Description| Default|
|
||||
|--------|-----------|--------|
|
||||
|`druid.emitter.logging.loggerClass`|The class used for logging.|`org.apache.druid.java.util.emitter.core.LoggingEmitter`|
|
||||
|`druid.emitter.logging.logLevel`|Choices: debug, info, warn, error. The log level at which message are logged.|info|
|
||||
|
||||
|
@ -816,8 +817,8 @@ All Druid components can communicate with each other over HTTP.
|
|||
|
||||
This section contains the configuration options for endpoints that are supported by all processes.
|
||||
|
||||
|Property| Description | Default |
|
||||
|--------|----------------------------------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------|
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.server.hiddenProperties`| If property names or substring of property names (case insensitive) is in this list, responses of the `/status/properties` endpoint do not show these properties | `["druid.s3.accessKey","druid.s3.secretKey","druid.metadata.storage.connector.password", "password", "key", "token", "pwd"]` |
|
||||
|
||||
## Master Server
|
||||
|
@ -864,7 +865,7 @@ These Coordinator static configurations can be defined in the `coordinator/runti
|
|||
|
||||
##### Metadata Management
|
||||
|
||||
|Property|Description|Required?|Default|
|
||||
|Property|Description|Required|Default|
|
||||
|--------|-----------|---------|-------|
|
||||
|`druid.coordinator.period.metadataStoreManagementPeriod`|How often to run metadata management tasks in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. |No | `PT1H`|
|
||||
|`druid.coordinator.kill.supervisor.on`| Boolean value for whether to enable automatic deletion of terminated supervisors. If set to true, Coordinator will periodically remove terminated supervisors from the supervisor table in metadata storage.| No | True|
|
||||
|
@ -917,8 +918,8 @@ http://<COORDINATOR_IP>:<PORT>/druid/coordinator/v1/config
|
|||
|
||||
Optional Header Parameters for auditing the config change can also be specified.
|
||||
|
||||
|Header Param Name| Description | Default |
|
||||
|----------|-------------|---------|
|
||||
|Header Param Name|Description|Default|
|
||||
|-----------------|-----------|-------|
|
||||
|`X-Druid-Author`| author making the config change|""|
|
||||
|`X-Druid-Comment`| comment describing the change being done|""|
|
||||
|
||||
|
@ -930,11 +931,8 @@ A sample Coordinator dynamic config JSON object is shown below:
|
|||
"mergeBytesLimit": 100000000,
|
||||
"mergeSegmentsLimit" : 1000,
|
||||
"maxSegmentsToMove": 5,
|
||||
"useBatchedSegmentSampler": false,
|
||||
"percentOfSegmentsToConsiderPerMove": 100,
|
||||
"replicantLifetime": 15,
|
||||
"replicationThrottleLimit": 10,
|
||||
"emitBalancingStats": false,
|
||||
"killDataSourceWhitelist": ["wikipedia", "testDatasource"],
|
||||
"decommissioningNodes": ["localhost:8182", "localhost:8282"],
|
||||
"decommissioningMaxPercentOfMaxSegmentsToMove": 70,
|
||||
|
@ -952,19 +950,16 @@ Issuing a GET request at the same URL will return the spec that is currently in
|
|||
|`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|
|
||||
|`maxSegmentsToMove`|The maximum number of segments that can be moved at any given time.|100|
|
||||
|`useBatchedSegmentSampler`|Deprecated. Boolean flag for whether or not we should use the Reservoir Sampling with a reservoir of size k instead of fixed size 1 to pick segments to move. This option can be enabled to speed up the sampling of segments to be balanced, especially if there is a large number of segments in the cluster or if there are too many segments to move.|true|
|
||||
|`percentOfSegmentsToConsiderPerMove`|Deprecated. This will eventually be phased out by the batched segment sampler. You can enable the batched segment sampler now by setting the dynamic Coordinator config, `useBatchedSegmentSampler`, to `true`. Note that if you choose to enable the batched segment sampler, `percentOfSegmentsToConsiderPerMove` will no longer have any effect on balancing. If `useBatchedSegmentSampler == false`, this config defines the percentage of the total number of segments in the cluster that are considered every time a segment needs to be selected for a move. Druid orders servers by available capacity ascending (the least available capacity first) and then iterates over the servers. For each server, Druid iterates over the segments on the server, considering them for moving. The default config of 100% means that every segment on every server is a candidate to be moved. This should make sense for most small to medium-sized clusters. However, an admin may find it preferable to drop this value lower if they don't think that it is worthwhile to consider every single segment in the cluster each time it is looking for a segment to move.|100|
|
||||
|`replicantLifetime`|The maximum number of Coordinator runs for a segment to be replicated before we start alerting.|15|
|
||||
|`replicationThrottleLimit`|The maximum number of segments that can be in the replication queue of a historical tier at any given time.|500|
|
||||
|`balancerComputeThreads`|Thread pool size for computing moving cost of segments in segment balancing. Consider increasing this if you have a lot of segments and moving segments starts to get stuck.|1|
|
||||
|`emitBalancingStats`|Boolean flag for whether or not we should emit balancing stats. This is an expensive operation.|false|
|
||||
|`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|
|
||||
|`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`.<br /><br />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.<br /><br />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, Emission of metrics controlled by the dynamic coordinator config `emitBalancingStats`, 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|
|
||||
|`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`|This is the maximum number of non-primary segment replicants to load per Coordination run. This number can be set to put a hard upper limit on the number of replicants loaded. It is a tool that can help prevent long delays in new data being available for query after events that require many non-primary replicants to be loaded by the cluster; such as a Historical node disconnecting from the cluster. The default value essentially means there is no limit on the number of replicants loaded per coordination cycle. If you want to use a non-default value for this config, you may want to start with it being `~20%` of the number of segments found on your Historical server with the most segments. You can use the Druid metric, `coordinator/time` with the filter `duty=org.apache.druid.server.coordinator.duty.RunRules` to see how different values of this config impact your Coordinator execution time.|`Integer.MAX_VALUE`|
|
||||
|
||||
|
@ -998,7 +993,7 @@ These configuration options control Coordinator lookup management. See [dynamic
|
|||
##### Automatic compaction dynamic configuration
|
||||
|
||||
You can set or update [automatic compaction](../data-management/automatic-compaction.md) properties dynamically using the
|
||||
[Coordinator API](../api-reference/api-reference.md#automatic-compaction-configuration) without restarting Coordinators.
|
||||
[Automatic compaction API](../api-reference/automatic-compaction-api.md) without restarting Coordinators.
|
||||
|
||||
For details about segment compaction, see [Segment size optimization](../operations/segment-optimization.md).
|
||||
|
||||
|
@ -1185,8 +1180,8 @@ http://<OVERLORD_IP>:<port>/druid/indexer/v1/worker
|
|||
|
||||
Optional header parameters for auditing the config change can also be specified.
|
||||
|
||||
|Header Param Name| Description | Default |
|
||||
|----------|-------------|---------|
|
||||
|Header Param Name|Description|Default|
|
||||
|-----------------|-----------|-------|
|
||||
|`X-Druid-Author`| author making the config change|""|
|
||||
|`X-Druid-Comment`| comment describing the change being done|""|
|
||||
|
||||
|
@ -1230,8 +1225,8 @@ An example Overlord dynamic config is shown below:
|
|||
|
||||
Issuing a GET request to the same URL returns the current Overlord dynamic config.
|
||||
|
||||
|Property| Description | Default |
|
||||
|--------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------------|
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`selectStrategy`| Describes how to assign tasks to MiddleManagers. The type can be `equalDistribution`, `equalDistributionWithCategorySpec`, `fillCapacity`, `fillCapacityWithCategorySpec`, and `javascript`. | `{"type":"equalDistribution"}` |
|
||||
|`autoScaler`| Only used if autoscaling is enabled. See below. | null |
|
||||
|
||||
|
@ -1337,9 +1332,9 @@ leave others empty. The empty ones can be safely terminated.
|
|||
Note that if `druid.indexer.runner.pendingTasksRunnerNumThreads` is set to _N_ > 1, then this strategy will fill _N_
|
||||
MiddleManagers up to capacity simultaneously, rather than a single MiddleManager.
|
||||
|
||||
|Property| Description |Default|
|
||||
|--------|-----------------------------------------|-------|
|
||||
|`type`| `fillCapacity` |required; must be `fillCapacity`|
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`type`| `fillCapacity`|required; must be `fillCapacity`|
|
||||
|`affinityConfig`| [`AffinityConfig`](#affinityconfig) object |null (no affinity)|
|
||||
|
||||
###### `fillCapacityWithCategorySpec`
|
||||
|
@ -1412,14 +1407,14 @@ Amazon's EC2 together with Google's GCE are currently the only supported autosca
|
|||
|
||||
EC2's autoscaler properties are:
|
||||
|
||||
| Property | Description |Default|
|
||||
|------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------|
|
||||
| `type` | `ec2` |0|
|
||||
| `minNumWorkers` | The minimum number of workers that can be in the cluster at any given time. |0|
|
||||
| `maxNumWorkers` | The maximum number of workers that can be in the cluster at any given time. |0|
|
||||
| `envConfig.availabilityZone` | What Amazon availability zone to run in. |none|
|
||||
| `envConfig.nodeData` | A JSON object that describes how to launch new nodes. |none; required|
|
||||
| `envConfig.userData` | A JSON object that describes how to configure new nodes. If you have set druid.indexer.autoscale.workerVersion, this must have a versionReplacementString. Otherwise, a versionReplacementString is not necessary. |none; optional|
|
||||
|Property| Description|Default|
|
||||
|--------|------------|-------|
|
||||
|`type`|`ec2`|0|
|
||||
|`minNumWorkers`| The minimum number of workers that can be in the cluster at any given time.|0|
|
||||
|`maxNumWorkers`| The maximum number of workers that can be in the cluster at any given time.|0|
|
||||
|`envConfig.availabilityZone` | What Amazon availability zone to run in.|none|
|
||||
|`envConfig.nodeData`| A JSON object that describes how to launch new nodes.|none; required|
|
||||
| `envConfig.userData`| A JSON object that describes how to configure new nodes. If you have set `druid.indexer.autoscale.workerVersion`, this must have a `versionReplacementString`. Otherwise, a `versionReplacementString` is not necessary.|none; optional|
|
||||
|
||||
For GCE's properties, please refer to the [gce-extensions](../development/extensions-contrib/gce-extensions.md).
|
||||
|
||||
|
@ -1468,19 +1463,18 @@ Middle managers pass their configurations down to their child peons. The MiddleM
|
|||
|
||||
#### Peon Processing
|
||||
|
||||
Processing properties set on the Middlemanager will be passed through to Peons.
|
||||
Processing properties set on the MiddleManager will be passed through to Peons.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`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 cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
|
||||
|`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.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.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)|
|
||||
|`druid.processing.fifo`|If the processing queue should treat tasks of equal priority in a FIFO manner|`true`|
|
||||
|`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`|
|
||||
|`druid.processing.intermediaryData.storage.type`|Storage type for storing intermediary segments of data shuffle between native parallel index tasks. Current choices are "local" which stores segment files in local storage of Middle Managers (or Indexer) or "deepstore" which uses configured deep storage. Note - With "deepstore" type data is stored in `shuffle-data` directory under the configured deep storage path, auto clean up for this directory is not supported yet. One can setup cloud storage lifecycle rules for auto clean up of data at `shuffle-data` prefix location.|local|
|
||||
|`druid.processing.intermediaryData.storage.type`|Storage type for intermediary segments of data shuffle between native parallel index tasks. <br />Set to `local` to store segment files in the local storage of the MiddleManager or Indexer. <br />Set to `deepstore` to use configured deep storage for better fault tolerance during rolling updates. When the storage type is `deepstore`, Druid stores the data in the `shuffle-data` directory under the configured deep storage path. Druid does not support automated cleanup for the `shuffle-data` directory. You can set up cloud storage lifecycle rules for automated cleanup of data at the `shuffle-data` prefix location.|`local`|
|
||||
|
||||
The amount of direct memory needed by Druid is at least
|
||||
`druid.processing.buffer.sizeBytes * (druid.processing.numMergeBuffers + druid.processing.numThreads + 1)`. You can
|
||||
|
@ -1627,7 +1621,6 @@ Druid uses Jetty to serve HTTP requests.
|
|||
|`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.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.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)|
|
||||
|`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`|
|
||||
|
||||
|
@ -1737,7 +1730,6 @@ Druid uses Jetty to serve HTTP requests.
|
|||
|`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.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.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)|
|
||||
|`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`|
|
||||
|
||||
|
@ -1840,6 +1832,19 @@ This strategy can be enabled by setting `druid.query.scheduler.laning.strategy=h
|
|||
|--------|-----------|-------|
|
||||
|`druid.query.scheduler.laning.maxLowPercent`|Maximum percent of the smaller number of `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, defining the number of HTTP threads that can be used by queries with a priority lower than 0. Value must be an integer in the range 1 to 100, and will be rounded up|No default, must be set if using this mode|
|
||||
|
||||
##### Guardrails for materialization of subqueries
|
||||
Druid stores the subquery rows in temporary tables that live in the Java heap. It is a good practice to avoid large subqueries in Druid.
|
||||
Therefore there are guardrails that are built in Druid to prevent the queries from generating subquery results which can exhaust the heap
|
||||
space. They can be set on a cluster level or modified per query level as desired.
|
||||
Note the following guardrails that can be set by the cluster admin to limit the subquery results:
|
||||
|
||||
1. `druid.server.http.maxSubqueryRows` in broker's config to set a default for the entire cluster or `maxSubqueryRows` in the query context to set an upper limit on the number of rows a subquery can generate
|
||||
2. `druid.server.http.maxSubqueryBytes` in broker's config to set a default for the entire cluster or `maxSubqueryBytes` in the query context to set an upper limit on the number of bytes a subquery can generate
|
||||
|
||||
Note that limiting the subquery by bytes is a newer feature therefore it is experimental as it materializes the results differently.
|
||||
|
||||
If you choose to modify or set any of the above limits, you must also think about the heap size of all Brokers, Historicals, and task Peons that process data for the subqueries to accommodate the subquery results.
|
||||
There is no formula to calculate the correct value. Trial and error is the best approach.
|
||||
|
||||
###### 'Manual' laning strategy
|
||||
This laning strategy is best suited for cases where one or more external applications which query Druid are capable of manually deciding what lane a given query should belong to. Configured with a map of lane names to percent or exact max capacities, queries with a matching `lane` parameter in the [query context](../querying/query-context.md) will be subjected to those limits.
|
||||
|
@ -1862,6 +1867,7 @@ Druid uses Jetty to serve HTTP requests. Each query being processed consumes a s
|
|||
|`druid.server.http.defaultQueryTimeout`|Query timeout in millis, beyond which unfinished queries will be cancelled|300000|
|
||||
|`druid.server.http.maxScatterGatherBytes`|Maximum number of bytes gathered from data processes such as Historicals and realtime processes to execute a query. Queries that exceed this limit will fail. This is an advance configuration that allows to protect in case Broker is under heavy load and not utilizing the data gathered in memory fast enough and leading to OOMs. This limit can be further reduced at query time using `maxScatterGatherBytes` in the context. Note that having large limit is not necessarily bad if broker is never under heavy concurrent load in which case data gathered is processed quickly and freeing up the memory used. Human-readable format is supported, see [here](human-readable-byte.md). |Long.MAX_VALUE|
|
||||
|`druid.server.http.maxSubqueryRows`|Maximum number of rows from all subqueries per query. Druid stores the subquery rows in temporary tables that live in the Java heap. `druid.server.http.maxSubqueryRows` is a guardrail to prevent the system from exhausting available heap. When a subquery exceeds the row limit, Druid throws a resource limit exceeded exception: "Subquery generated results beyond maximum."<br /><br />It is a good practice to avoid large subqueries in Druid. However, if you choose to raise the subquery row limit, you must also increase the heap size of all Brokers, Historicals, and task Peons that process data for the subqueries to accommodate the subquery results.<br /><br />There is no formula to calculate the correct value. Trial and error is the best approach.|100000|
|
||||
|`druid.server.http.maxSubqueryBytes`|Maximum number of bytes from all subqueries per query. Since the results are stored on the Java heap, `druid.server.http.maxSubqueryBytes` is a guardrail like `druid.server.http.maxSubqueryRows` to prevent the heap space from exhausting. When a subquery exceeds the byte limit, Druid throws a resource limit exceeded exception. A negative value for the guardrail indicates that Druid won't guardrail by memory. Check the docs for `druid.server.http.maxSubqueryRows` to see how to set the optimal value for a cluster. This is an experimental feature for now as this materializes the results in a different format.|-1|
|
||||
|`druid.server.http.gracefulShutdownTimeout`|The maximum amount of time Jetty waits after receiving shutdown signal. After this timeout the threads will be forcefully shutdown. This allows any queries that are executing to complete(Only values greater than zero are valid).|`PT30S`|
|
||||
|`druid.server.http.unannouncePropagationDelay`|How long to wait for ZooKeeper unannouncements to propagate before shutting down Jetty. This is a minimum and `druid.server.http.gracefulShutdownTimeout` does not start counting down until after this period elapses.|`PT0S` (do not wait)|
|
||||
|`druid.server.http.maxQueryTimeout`|Maximum allowed value (in milliseconds) for `timeout` parameter. See [query-context](../querying/query-context.md) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE|
|
||||
|
@ -1902,7 +1908,6 @@ The broker uses processing configs for nested groupBy queries.
|
|||
|`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.columnCache.sizeBytes`|Maximum size in bytes for the dimension value lookup cache. Any value greater than `0` enables the cache. It is currently disabled by default. Enabling the lookup cache can significantly improve the performance of aggregators operating on dimension values, such as the JavaScript aggregator, or cardinality aggregator, but can slow things down if the cache hit rate is low (i.e. dimensions with few repeating values). Enabling it may also require additional garbage collection tuning to avoid long GC pauses.|`0` (disabled)|
|
||||
|`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`|
|
||||
|
|
|
@ -40,9 +40,9 @@ This topic guides you through setting up automatic compaction for your Druid clu
|
|||
## Enable automatic compaction
|
||||
|
||||
You can enable automatic compaction for a datasource using the web console or programmatically via an API.
|
||||
This process differs for manual compaction tasks, which can be submitted from the [Tasks view of the web console](../operations/web-console.md) or the [Tasks API](../api-reference/api-reference.md#tasks).
|
||||
This process differs for manual compaction tasks, which can be submitted from the [Tasks view of the web console](../operations/web-console.md) or the [Tasks API](../api-reference/tasks-api.md).
|
||||
|
||||
### web console
|
||||
### Web console
|
||||
|
||||
Use the web console to enable automatic compaction for a datasource as follows.
|
||||
|
||||
|
@ -59,10 +59,10 @@ To disable auto-compaction for a datasource, click **Delete** from the **Compact
|
|||
|
||||
### Compaction configuration API
|
||||
|
||||
Use the [Coordinator API](../api-reference/api-reference.md#automatic-compaction-status) to configure automatic compaction.
|
||||
Use the [Automatic compaction API](../api-reference/automatic-compaction-api.md#automatic-compaction-status) 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/api-reference.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#automatic-compaction-configuration) to `/druid/coordinator/v1/config/compaction`.
|
||||
The following example configures auto-compaction for the `wikipedia` datasource:
|
||||
|
||||
```sh
|
||||
|
@ -76,7 +76,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/api-reference.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#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'
|
||||
|
@ -115,11 +115,11 @@ The following properties are automatically set by the Coordinator:
|
|||
* `id`: Generated using the task type, datasource name, interval, and timestamp. The task ID is prefixed with `coordinator-issued`.
|
||||
* `context`: Set according to the user-provided `taskContext`.
|
||||
|
||||
Compaction tasks fetch all [relevant segments](compaction.md#compaction-io-configuration) prior to launching any subtasks,
|
||||
_unless_ the following items are all set. It is strongly recommended to set all of these items to maximize performance
|
||||
and minimize disk usage of the `compact` tasks launched by auto-compaction:
|
||||
Compaction tasks typically fetch all [relevant segments](compaction.md#compaction-io-configuration) prior to launching any subtasks,
|
||||
_unless_ the following properties are all set to non-null values. It is strongly recommended to set them to non-null values to
|
||||
maximize performance and minimize disk usage of the `compact` tasks launched by auto-compaction:
|
||||
|
||||
- [`granularitySpec`](compaction.md#compaction-granularity-spec). All three values must be set to non-null values: `segmentGranularity`, `queryGranularity`, and `rollup`.
|
||||
- [`granularitySpec`](compaction.md#compaction-granularity-spec), with non-null values for each of `segmentGranularity`, `queryGranularity`, and `rollup`
|
||||
- [`dimensionsSpec`](compaction.md#compaction-dimensions-spec)
|
||||
- `metricsSpec`
|
||||
|
||||
|
@ -152,7 +152,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/api-reference.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#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
|
||||
|
||||
|
|
|
@ -138,9 +138,9 @@ To control the number of result segments per time chunk, you can set [`maxRowsPe
|
|||
|
||||
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.
|
||||
|
||||
Compaction tasks fetch all [relevant segments](#compaction-io-configuration) prior to launching any subtasks, _unless_ the following items are all set. It is strongly recommended to set all of these items to maximize performance and minimize disk usage of the `compact` task:
|
||||
Compaction tasks typically fetch all [relevant segments](#compaction-io-configuration) prior to launching any subtasks, _unless_ the following properties are all set to non-null values. It is strongly recommended to set them to non-null values to maximize performance and minimize disk usage of the `compact` task:
|
||||
|
||||
- [`granularitySpec`](#compaction-granularity-spec). All three values must be set to non-null values: `segmentGranularity`, `queryGranularity`, and `rollup`.
|
||||
- [`granularitySpec`](#compaction-granularity-spec), with non-null values for each of `segmentGranularity`, `queryGranularity`, and `rollup`
|
||||
- [`dimensionsSpec`](#compaction-dimensions-spec)
|
||||
- `metricsSpec`
|
||||
|
||||
|
|
|
@ -38,7 +38,7 @@ Deletion by time range happens in two steps:
|
|||
you have a backup.
|
||||
|
||||
For documentation on disabling segments using the Coordinator API, see the
|
||||
[Coordinator API reference](../api-reference/api-reference.md#coordinator-datasources).
|
||||
[Legacy metadata API reference](../api-reference/legacy-metadata-api.md#datasources).
|
||||
|
||||
A data deletion tutorial is available at [Tutorial: Deleting data](../tutorials/tutorial-delete-data.md).
|
||||
|
||||
|
|
|
@ -31,7 +31,7 @@ For basic tuning guidance for the Broker process, see [Basic cluster tuning](../
|
|||
|
||||
### HTTP endpoints
|
||||
|
||||
For a list of API endpoints supported by the Broker, see [Broker API](../api-reference/api-reference.md#broker).
|
||||
For a list of API endpoints supported by the Broker, see [Broker API](../api-reference/legacy-metadata-api.md#broker).
|
||||
|
||||
### Overview
|
||||
|
||||
|
|
|
@ -31,7 +31,7 @@ For basic tuning guidance for the Coordinator process, see [Basic cluster tuning
|
|||
|
||||
### HTTP endpoints
|
||||
|
||||
For a list of API endpoints supported by the Coordinator, see [Coordinator API](../api-reference/api-reference.md#coordinator).
|
||||
For a list of API endpoints supported by the Coordinator, see [Service status API reference](../api-reference/service-status-api.md#coordinator).
|
||||
|
||||
### Overview
|
||||
|
||||
|
@ -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/api-reference.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#automatic-compaction-configuration) 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:
|
||||
|
||||
|
|
|
@ -31,7 +31,7 @@ For basic tuning guidance for the Historical process, see [Basic cluster tuning]
|
|||
|
||||
### HTTP endpoints
|
||||
|
||||
For a list of API endpoints supported by the Historical, please see the [API reference](../api-reference/api-reference.md#historical).
|
||||
For a list of API endpoints supported by the Historical, please see the [Service status API reference](../api-reference/service-status-api.md#historical).
|
||||
|
||||
### Running
|
||||
|
||||
|
|
|
@ -35,7 +35,7 @@ For Apache Druid Indexer Process Configuration, see [Indexer Configuration](../c
|
|||
|
||||
### HTTP endpoints
|
||||
|
||||
The Indexer process shares the same HTTP endpoints as the [MiddleManager](../api-reference/api-reference.md#middlemanager).
|
||||
The Indexer process shares the same HTTP endpoints as the [MiddleManager](../api-reference/service-status-api.md#middlemanager).
|
||||
|
||||
### Running
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ Indexing [tasks](../ingestion/tasks.md) are responsible for creating and [killin
|
|||
The indexing service is composed of three main components: [Peons](../design/peons.md) that can run a single task, [MiddleManagers](../design/middlemanager.md) that manage Peons, and an [Overlord](../design/overlord.md) that manages task distribution to MiddleManagers.
|
||||
Overlords and MiddleManagers may run on the same process or across multiple processes, while MiddleManagers and Peons always run on the same process.
|
||||
|
||||
Tasks are managed using API endpoints on the Overlord service. Please see [Overlord Task API](../api-reference/api-reference.md#tasks) for more information.
|
||||
Tasks are managed using API endpoints on the Overlord service. Please see [Tasks API](../api-reference/tasks-api.md) for more information.
|
||||
|
||||
![Indexing Service](../assets/indexing_service.png "Indexing Service")
|
||||
|
||||
|
|
|
@ -31,7 +31,7 @@ For basic tuning guidance for the MiddleManager process, see [Basic cluster tuni
|
|||
|
||||
### HTTP endpoints
|
||||
|
||||
For a list of API endpoints supported by the MiddleManager, please see the [API reference](../api-reference/api-reference.md#middlemanager).
|
||||
For a list of API endpoints supported by the MiddleManager, please see the [Service status API reference](../api-reference/service-status-api.md#middlemanager).
|
||||
|
||||
### Overview
|
||||
|
||||
|
|
|
@ -31,7 +31,7 @@ For basic tuning guidance for the Overlord process, see [Basic cluster tuning](.
|
|||
|
||||
### HTTP endpoints
|
||||
|
||||
For a list of API endpoints supported by the Overlord, please see the [API reference](../api-reference/api-reference.md#overlord).
|
||||
For a list of API endpoints supported by the Overlord, please see the [Service status API reference](../api-reference/service-status-api.md#overlord).
|
||||
|
||||
### Overview
|
||||
|
||||
|
|
|
@ -31,8 +31,6 @@ For basic tuning guidance for MiddleManager tasks, see [Basic cluster tuning](..
|
|||
|
||||
### HTTP endpoints
|
||||
|
||||
For a list of API endpoints supported by the Peon, please see the [Peon API reference](../api-reference/api-reference.md#peon).
|
||||
|
||||
Peons run a single task in a single JVM. MiddleManager is responsible for creating Peons for running tasks.
|
||||
Peons should rarely (if ever for testing purposes) be run on their own.
|
||||
|
||||
|
|
|
@ -36,7 +36,7 @@ For basic tuning guidance for the Router process, see [Basic cluster tuning](../
|
|||
|
||||
### HTTP endpoints
|
||||
|
||||
For a list of API endpoints supported by the Router, see [Router API](../api-reference/api-reference.md#router).
|
||||
For a list of API endpoints supported by the Router, see [Legacy metadata API reference](../api-reference/legacy-metadata-api.md#datasource-information).
|
||||
|
||||
### Running
|
||||
|
||||
|
|
|
@ -125,6 +125,7 @@ metadata:
|
|||
rules:
|
||||
- apiGroups:
|
||||
- ""
|
||||
- batch
|
||||
resources:
|
||||
- pods
|
||||
- configmaps
|
||||
|
|
|
@ -25,7 +25,7 @@ description: "Reference topic for running and maintaining Apache Kafka superviso
|
|||
-->
|
||||
This topic contains operations reference information to run and maintain Apache Kafka supervisors for Apache Druid. It includes descriptions of how some supervisor APIs work within Kafka Indexing Service.
|
||||
|
||||
For all supervisor APIs, see [Supervisor APIs](../../api-reference/api-reference.md#supervisors).
|
||||
For all supervisor APIs, see [Supervisor API reference](../../api-reference/supervisor-api.md).
|
||||
|
||||
## Getting Supervisor Status Report
|
||||
|
||||
|
|
|
@ -205,7 +205,7 @@ The `tuningConfig` is optional and default parameters will be used if no `tuning
|
|||
| `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 | Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever. | no (default == 0) |
|
||||
| `resetOffsetAutomatically` | Boolean | Controls behavior when Druid needs to read Kafka messages that are no longer available (i.e. when `OffsetOutOfRangeException` is encountered).<br/><br/>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/api-reference.md#supervisors). This mode is useful for production, since it will make you aware of issues with ingestion.<br/><br/>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.<br/><br/>This feature behaves similarly to the Kafka `auto.offset.reset` consumer property. | no (default == false) |
|
||||
| `resetOffsetAutomatically` | Boolean | Controls behavior when Druid needs to read Kafka messages that are no longer available (i.e. when `OffsetOutOfRangeException` is encountered).<br/><br/>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.<br/><br/>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.<br/><br/>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)) |
|
||||
| `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 (default == true) |
|
||||
| `chatThreads` | Integer | The number of threads that will be used for communicating with indexing tasks. Ignored if `chatAsync` is `true` (the default). | no (default == min(10, taskCount * replicas)) |
|
||||
|
|
|
@ -284,7 +284,7 @@ The `tuningConfig` is optional. If no `tuningConfig` is specified, default param
|
|||
|`indexSpecForIntermediatePersists`|Object|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|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|no (default == false)|
|
||||
|`handoffConditionTimeout`|Long| Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever.| no (default == 0)|
|
||||
|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read Kinesis messages that are no longer available.<br/><br/>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/api-reference.md#supervisors). This mode is useful for production, since it highlights issues with ingestion.<br/><br/>If true, Druid automatically resets to the earliest or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if true, latest if false). Note that this can lead to data being *DROPPED* (if `useEarliestSequenceNumber` is false) or *DUPLICATED* (if `useEarliestSequenceNumber` is true) without your knowledge. Druid will log 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.|no (default == false)|
|
||||
|`resetOffsetAutomatically`|Boolean|Controls behavior when Druid needs to read Kinesis messages that are no longer available.<br/><br/>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 highlights issues with ingestion.<br/><br/>If true, Druid automatically resets to the earliest or latest sequence number available in Kinesis, based on the value of the `useEarliestSequenceNumber` property (earliest if true, latest if false). Note that this can lead to data being *DROPPED* (if `useEarliestSequenceNumber` is false) or *DUPLICATED* (if `useEarliestSequenceNumber` is true) without your knowledge. Druid will log 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.|no (default == false)|
|
||||
|`skipSequenceNumberAvailabilityCheck`|Boolean|Whether to enable checking if the current sequence number is still available in a particular Kinesis shard. If set to false, the indexing task will attempt to reset the current sequence number (or not), depending on the value of `resetOffsetAutomatically`.|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))|
|
||||
|`chatAsync`|Boolean| If true, the supervisor uses asynchronous communication with indexing tasks and ignores the `chatThreads` parameter. If false, the supervisor uses synchronous communication in a thread pool of size `chatThreads`.| no (default == true)|
|
||||
|
@ -338,7 +338,7 @@ For Concise bitmaps:
|
|||
## Operations
|
||||
|
||||
This section describes how some supervisor APIs work in Kinesis Indexing Service.
|
||||
For all supervisor APIs, check [Supervisor APIs](../../api-reference/api-reference.md#supervisors).
|
||||
For all supervisor APIs, check [Supervisor API reference](../../api-reference/supervisor-api.md).
|
||||
|
||||
### AWS Authentication
|
||||
|
||||
|
|
|
@ -57,15 +57,15 @@ Make sure to include the `druid-hdfs-storage` and all the hadoop configuration,
|
|||
|
||||
You can verify if segments created by a recent ingestion task are loaded onto historicals and available for querying using the following workflow.
|
||||
1. Submit your ingestion task.
|
||||
2. Repeatedly poll the [Overlord's tasks API](../api-reference/api-reference.md#tasks) ( `/druid/indexer/v1/task/{taskId}/status`) until your task is shown to be successfully completed.
|
||||
3. Poll the [Segment Loading by Datasource API](../api-reference/api-reference.md#segment-loading-by-datasource) (`/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus`) with
|
||||
2. Repeatedly poll the [Overlord's tasks API](../api-reference/tasks-api.md) ( `/druid/indexer/v1/task/{taskId}/status`) until your task is shown to be successfully completed.
|
||||
3. Poll the [Segment Loading by Datasource API](../api-reference/legacy-metadata-api.md#segment-loading-by-datasource) (`/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus`) with
|
||||
`forceMetadataRefresh=true` and `interval=<INTERVAL_OF_INGESTED_DATA>` once.
|
||||
(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms of the load on the metadata store but is necessary to make sure that we verify all the latest segments' load status)
|
||||
If there are segments not yet loaded, continue to step 4, otherwise you can now query the data.
|
||||
4. Repeatedly poll the [Segment Loading by Datasource API](../api-reference/api-reference.md#segment-loading-by-datasource) (`/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus`) with
|
||||
4. Repeatedly poll the [Segment Loading by Datasource API](../api-reference/legacy-metadata-api.md#segment-loading-by-datasource) (`/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus`) with
|
||||
`forceMetadataRefresh=false` and `interval=<INTERVAL_OF_INGESTED_DATA>`.
|
||||
Continue polling until all segments are loaded. Once all segments are loaded you can now query the data.
|
||||
Note that this workflow only guarantees that the segments are available at the time of the [Segment Loading by Datasource API](../api-reference/api-reference.md#segment-loading-by-datasource) call. Segments can still become missing because of historical process failures or any other reasons afterward.
|
||||
Note that this workflow only guarantees that the segments are available at the time of the [Segment Loading by Datasource API](../api-reference/legacy-metadata-api.md#segment-loading-by-datasource) call. Segments can still become missing because of historical process failures or any other reasons afterward.
|
||||
|
||||
## I don't see my Druid segments on my Historical processes
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@ instance of a Druid [Overlord](../design/overlord.md). Please refer to our [Hado
|
|||
comparisons between Hadoop-based, native batch (simple), and native batch (parallel) ingestion.
|
||||
|
||||
To run a Hadoop-based ingestion task, write an ingestion spec as specified below. Then POST it to the
|
||||
[`/druid/indexer/v1/task`](../api-reference/api-reference.md#tasks) endpoint on the Overlord, or use the
|
||||
[`/druid/indexer/v1/task`](../api-reference/tasks-api.md) endpoint on the Overlord, or use the
|
||||
`bin/post-index-task` script included with Druid.
|
||||
|
||||
## Tutorial
|
||||
|
|
|
@ -69,7 +69,7 @@ runs for the duration of the job.
|
|||
| **Method** | [Native batch](./native-batch.md) | [SQL](../multi-stage-query/index.md) | [Hadoop-based](hadoop.md) |
|
||||
|---|-----|--------------|------------|
|
||||
| **Controller task type** | `index_parallel` | `query_controller` | `index_hadoop` |
|
||||
| **How you submit it** | Send an `index_parallel` spec to the [task API](../api-reference/api-reference.md#tasks). | Send an [INSERT](../multi-stage-query/concepts.md#insert) or [REPLACE](../multi-stage-query/concepts.md#replace) statement to the [SQL task API](../api-reference/sql-ingestion-api.md#submit-a-query). | Send an `index_hadoop` spec to the [task API](../api-reference/api-reference.md#tasks). |
|
||||
| **How you submit it** | Send an `index_parallel` spec to the [Tasks API](../api-reference/tasks-api.md). | Send an [INSERT](../multi-stage-query/concepts.md#insert) or [REPLACE](../multi-stage-query/concepts.md#replace) statement to the [SQL task API](../api-reference/sql-ingestion-api.md#submit-a-query). | Send an `index_hadoop` spec to the [Tasks API](../api-reference/tasks-api.md). |
|
||||
| **Parallelism** | Using subtasks, if [`maxNumConcurrentSubTasks`](native-batch.md#tuningconfig) is greater than 1. | Using `query_worker` subtasks. | Using YARN. |
|
||||
| **Fault tolerance** | Workers automatically relaunched upon failure. Controller task failure leads to job failure. | Controller or worker task failure leads to job failure. | YARN containers automatically relaunched upon failure. Controller task failure leads to job failure. |
|
||||
| **Can append?** | Yes. | Yes (INSERT). | No. |
|
||||
|
|
|
@ -41,7 +41,7 @@ For related information on batch indexing, see:
|
|||
|
||||
To run either kind of native batch indexing task you can:
|
||||
- Use the **Load Data** UI in the web console to define and submit an ingestion spec.
|
||||
- Define an ingestion spec in JSON based upon the [examples](#parallel-indexing-example) and reference topics for batch indexing. Then POST the ingestion spec to the [Indexer API endpoint](../api-reference/api-reference.md#tasks),
|
||||
- Define an ingestion spec in JSON based upon the [examples](#parallel-indexing-example) and reference topics for batch indexing. Then POST the ingestion spec to the [Tasks API endpoint](../api-reference/tasks-api.md),
|
||||
`/druid/indexer/v1/task`, the Overlord service. Alternatively you can use the indexing script included with Druid at `bin/post-index-task`.
|
||||
|
||||
## Parallel task indexing
|
||||
|
|
|
@ -26,7 +26,7 @@ sidebar_label: Task reference
|
|||
Tasks do all [ingestion](index.md)-related work in Druid.
|
||||
|
||||
For batch ingestion, you will generally submit tasks directly to Druid using the
|
||||
[Task APIs](../api-reference/api-reference.md#tasks). For streaming ingestion, tasks are generally submitted for you by a
|
||||
[Tasks APIs](../api-reference/tasks-api.md). For streaming ingestion, tasks are generally submitted for you by a
|
||||
supervisor.
|
||||
|
||||
## Task API
|
||||
|
@ -34,7 +34,7 @@ supervisor.
|
|||
Task APIs are available in two main places:
|
||||
|
||||
- The [Overlord](../design/overlord.md) process offers HTTP APIs to submit tasks, cancel tasks, check their status,
|
||||
review logs and reports, and more. Refer to the [Tasks API reference page](../api-reference/api-reference.md#tasks) for a
|
||||
review logs and reports, and more. Refer to the [Tasks API reference](../api-reference/tasks-api.md) for a
|
||||
full list.
|
||||
- Druid SQL includes a [`sys.tasks`](../querying/sql-metadata-tables.md#tasks-table) table that provides information about currently
|
||||
running tasks. This table is read-only, and has a limited (but useful!) subset of the full information available through
|
||||
|
@ -406,7 +406,7 @@ The task then starts creating logs in a local directory of the middle manager (o
|
|||
|
||||
When the task completes - whether it succeeds or fails - the middle manager (or indexer) will push the task log file into the location specified in [`druid.indexer.logs`](../configuration/index.md#task-logging).
|
||||
|
||||
Task logs on the Druid web console are retrieved via an [API](../api-reference/api-reference.md#overlord) on the Overlord. It automatically detects where the log file is, either in the middleManager / indexer or in long-term storage, and passes it back.
|
||||
Task logs on the Druid web console are retrieved via an [API](../api-reference/service-status-api.md#overlord) on the Overlord. It automatically detects where the log file is, either in the middleManager / indexer or in long-term storage, and passes it back.
|
||||
|
||||
If you don't see the log file in long-term storage, it means either:
|
||||
|
||||
|
|
|
@ -365,7 +365,7 @@ For detailed information about the settings related to durable storage, see [Dur
|
|||
|
||||
### Use durable storage for queries
|
||||
|
||||
When you run a query, include the context parameter `durableShuffleStorage` and set it to `true`.
|
||||
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`.
|
||||
|
||||
|
@ -423,13 +423,12 @@ The following table describes error codes you may encounter in the `multiStageQu
|
|||
| <a name="error_CannotParseExternalData">`CannotParseExternalData`</a> | A worker task could not parse data from an external datasource. | `errorMessage`: More details on why parsing failed. |
|
||||
| <a name="error_ColumnNameRestricted">`ColumnNameRestricted`</a> | The query uses a restricted column name. | `columnName`: The restricted column name. |
|
||||
| <a name="error_ColumnTypeNotSupported">`ColumnTypeNotSupported`</a> | The column type is not supported. This can be because:<br /> <br /><ul><li>Support for writing or reading from a particular column type is not supported.</li><li>The query attempted to use a column type that is not supported by the frame format. This occurs with ARRAY types, which are not yet implemented for frames.</li></ul> | `columnName`: The column name with an unsupported type.<br /> <br />`columnType`: The unknown column type. |
|
||||
| <a name="error_InsertCannotAllocateSegment">`InsertCannotAllocateSegment`</a> | The controller task could not allocate a new segment ID due to conflict with existing segments or pending segments. Common reasons for such conflicts:<br /> <br /><ul><li>Attempting to mix different granularities in the same intervals of the same datasource.</li><li>Prior ingestions that used non-extendable shard specs.</li></ul>| `dataSource`<br /> <br />`interval`: The interval for the attempted new segment allocation. |
|
||||
| <a name="error_InsertCannotAllocateSegment">`InsertCannotAllocateSegment`</a> | The controller task could not allocate a new segment ID due to conflict with existing segments or pending segments. Common reasons for such conflicts:<br /> <br /><ul><li>Attempting to mix different granularities in the same intervals of the same datasource.</li><li>Prior ingestions that used non-extendable shard specs.</li></ul> <br /> <br /> Use REPLACE to overwrite the existing data or if the error contains the `allocatedInterval` then alternatively rerun the INSERT job with the mentioned granularity to append to existing data. Note that it might not always be possible to append to the existing data using INSERT and can only be done if `allocatedInterval` is present. | `dataSource`<br /> <br />`interval`: The interval for the attempted new segment allocation. <br /> <br /> `allocatedInterval`: The incorrect interval allocated by the overlord. It can be null |
|
||||
| <a name="error_InsertCannotBeEmpty">`InsertCannotBeEmpty`</a> | An INSERT or REPLACE query did not generate any output rows in a situation where output rows are required for success. This can happen for INSERT or REPLACE queries with `PARTITIONED BY` set to something other than `ALL` or `ALL TIME`. | `dataSource` |
|
||||
| <a name="error_InsertCannotOrderByDescending">`InsertCannotOrderByDescending`</a> | An INSERT query contained a `CLUSTERED BY` expression in descending order. Druid's segment generation code only supports ascending order. | `columnName` |
|
||||
| <a name="error_InsertLockPreempted">`InsertLockPreempted`</a> | An INSERT or REPLACE query was canceled by a higher-priority ingestion job, such as a real-time ingestion task. | |
|
||||
| <a name="error_InsertTimeNull">`InsertTimeNull`</a> | An INSERT or REPLACE query encountered a null timestamp in the `__time` field.<br /><br />This can happen due to using an expression like `TIME_PARSE(timestamp) AS __time` with a timestamp that cannot be parsed. ([`TIME_PARSE`](../querying/sql-scalar.md#date-and-time-functions) returns null when it cannot parse a timestamp.) In this case, try parsing your timestamps using a different function or pattern. Or, if your timestamps may genuinely be null, consider using [`COALESCE`](../querying/sql-scalar.md#other-scalar-functions) to provide a default value. One option is [`CURRENT_TIMESTAMP`](../querying/sql-scalar.md#date-and-time-functions), which represents the start time of the job.|
|
||||
| <a name="error_InsertTimeOutOfBounds">`InsertTimeOutOfBounds`</a> | A REPLACE query generated a timestamp outside the bounds of the TIMESTAMP parameter for your OVERWRITE WHERE clause.<br /> <br />To avoid this error, verify that the you specified is valid. | `interval`: time chunk interval corresponding to the out-of-bounds timestamp |
|
||||
| <a name="error_InvalidNullByte">`InvalidNullByte`</a> | A string column included a null byte. Null bytes in strings are not permitted. | `column`: The column that included the null byte |
|
||||
| <a name="error_InvalidNullByte">`InvalidNullByte`</a> | A string column included a null byte. Null bytes in strings are not permitted. |`source`: The source that included the null byte <br /></br /> `rowNumber`: The row number (1-indexed) that included the null byte <br /><br /> `column`: The column that included the null byte <br /><br /> `value`: Actual string containing the null byte <br /><br /> `position`: Position (1-indexed) of occurrence of null byte|
|
||||
| <a name="error_QueryNotSupported">`QueryNotSupported`</a> | QueryKit could not translate the provided native query to a multi-stage query.<br /> <br />This can happen if the query uses features that aren't supported, like GROUPING SETS. | |
|
||||
| <a name="error_QueryRuntimeError">`QueryRuntimeError`</a> | MSQ uses the native query engine to run the leaf stages. This error tells MSQ that error is in native query runtime.<br /> <br /> Since this is a generic error, the user needs to look at logs for the error message and stack trace to figure out the next course of action. If the user is stuck, consider raising a `github` issue for assistance. | `baseErrorMessage` error message from the native query runtime. |
|
||||
| <a name="error_RowTooLarge">`RowTooLarge`</a> | The query tried to process a row that was too large to write to a single frame. See the [Limits](#limits) table for specific limits on frame size. Note that the effective maximum row size is smaller than the maximum frame size due to alignment considerations during frame writing. | `maxFrameSize`: The limit on the frame size. |
|
||||
|
@ -449,3 +448,4 @@ The following table describes error codes you may encounter in the `multiStageQu
|
|||
| <a name="error_WorkerFailed">`WorkerFailed`</a> | A worker task failed unexpectedly. | `errorMsg`<br /><br />`workerTaskId`: The ID of the worker task. |
|
||||
| <a name="error_WorkerRpcFailed">`WorkerRpcFailed`</a> | A remote procedure call to a worker task failed and could not recover. | `workerTaskId`: the id of the worker task |
|
||||
| <a name="error_UnknownError">`UnknownError`</a> | All other errors. | `message` |
|
||||
| <a name="error_InsertCannotOrderByDescending">`InsertCannotOrderByDescending`</a> | Deprecated. An INSERT query contained a `CLUSTERED BY` expression in descending order. Druid's segment generation code only supports ascending order. The query returns a `ValidationException` instead of the fault. | `columnName` |
|
||||
|
|
|
@ -66,6 +66,10 @@ Metrics may have additional dimensions beyond those listed above.
|
|||
|`sqlQuery/bytes`|Number of bytes returned in the SQL query response.|`id`, `nativeQueryIds`, `dataSource`, `remoteAddress`, `success`, `engine`| |
|
||||
|`init/serverview/time`|Time taken to initialize the broker server view. Useful to detect if brokers are taking too long to start.||Depends on the number of segments.|
|
||||
|`init/metadatacache/time`|Time taken to initialize the broker segment metadata cache. Useful to detect if brokers are taking too long to start||Depends on the number of segments.|
|
||||
|`segment/metadatacache/refresh/count`|Number of segments to refresh in broker segment metadata cache.|`dataSource`|
|
||||
|`segment/metadatacache/refresh/time`|Time taken to refresh segments in broker segment metadata cache.|`dataSource`|
|
||||
|`segment/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 `segment/serverview/sync/unstableTime` to debug slow startup of Brokers.|`server`, `tier`|1 for fully synced servers, 0 otherwise|
|
||||
|`segment/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.|
|
||||
|
||||
### Historical
|
||||
|
||||
|
@ -154,9 +158,9 @@ If SQL is enabled, the Broker will emit the following metrics for SQL.
|
|||
|
||||
|Metric|Description| Dimensions |Normal Value|
|
||||
|------|-----------|---------------------------------------------------------|------------|
|
||||
|`ingest/count`|Count of `1` every time an ingestion job runs (includes compaction jobs). Aggregate using dimensions. | `dataSource`, `taskId`, `taskType`, `taskIngestionMode`, `tags` |Always `1`.|
|
||||
|`ingest/segments/count`|Count of final segments created by job (includes tombstones). | `dataSource`, `taskId`, `taskType`, `taskIngestionMode`, `tags` |At least `1`.|
|
||||
|`ingest/tombstones/count`|Count of tombstones created by job. | `dataSource`, `taskId`, `taskType`, `taskIngestionMode`, `tags` |Zero or more for replace. Always zero for non-replace tasks (always zero for legacy replace, see below).|
|
||||
|`ingest/count`|Count of `1` every time an ingestion job runs (includes compaction jobs). Aggregate using dimensions. | `dataSource`, `taskId`, `taskType`, `groupId`, `taskIngestionMode`, `tags` |Always `1`.|
|
||||
|`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
|
||||
|
@ -206,26 +210,26 @@ batch ingestion emit the following metrics. These metrics are deltas for each em
|
|||
|
||||
|Metric|Description|Dimensions|Normal Value|
|
||||
|------|-----------|----------|------------|
|
||||
|`ingest/events/thrownAway`|Number of events rejected because they are either null, or filtered by the transform spec, or outside the windowPeriod.|`dataSource`, `taskId`, `taskType`, `tags`|0|
|
||||
|`ingest/events/unparseable`|Number of events rejected because the events are unparseable.|`dataSource`, `taskId`, `taskType`, `tags`|0|
|
||||
|`ingest/events/duplicate`|Number of events rejected because the events are duplicated.|`dataSource`, `taskId`, `taskType`, `tags`|0|
|
||||
|`ingest/events/processed`|Number of events successfully processed per emission period.|`dataSource`, `taskId`, `taskType`, `tags`|Equal to the number of events per emission period.|
|
||||
|`ingest/rows/output`|Number of Druid rows persisted.|`dataSource`, `taskId`, `taskType`|Your number of events with rollup.|
|
||||
|`ingest/persists/count`|Number of times persist occurred.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on configuration.|
|
||||
|`ingest/persists/time`|Milliseconds spent doing intermediate persist.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on configuration. Generally a few minutes at most.|
|
||||
|`ingest/persists/cpu`|Cpu time in Nanoseconds spent on doing intermediate persist.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on configuration. Generally a few minutes at most.|
|
||||
|`ingest/persists/backPressure`|Milliseconds spent creating persist tasks and blocking waiting for them to finish.|`dataSource`, `taskId`, `taskType`, `tags`|0 or very low|
|
||||
|`ingest/persists/failed`|Number of persists that failed.|`dataSource`, `taskId`, `taskType`, `tags`|0|
|
||||
|`ingest/handoff/failed`|Number of handoffs that failed.|`dataSource`, `taskId`, `taskType`, `tags`|0|
|
||||
|`ingest/merge/time`|Milliseconds spent merging intermediate segments.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on configuration. Generally a few minutes at most.|
|
||||
|`ingest/merge/cpu`|Cpu time in Nanoseconds spent on merging intermediate segments.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on configuration. Generally a few minutes at most.|
|
||||
|`ingest/handoff/count`|Number of handoffs that happened.|`dataSource`, `taskId`, `taskType`, `tags`|Varies. Generally greater than 0 once every segment granular period if cluster operating normally.|
|
||||
|`ingest/sink/count`|Number of sinks not handoffed.|`dataSource`, `taskId`, `taskType`, `tags`|1~3|
|
||||
|`ingest/events/messageGap`|Time gap in milliseconds between the latest ingested event timestamp and the current system timestamp of metrics emission. If the value is increasing but lag is low, Druid may not be receiving new data. This metric is reset as new tasks spawn up.|`dataSource`, `taskId`, `taskType`, `tags`|Greater than 0, depends on the time carried in event. |
|
||||
|`ingest/events/thrownAway`|Number of events rejected because they are either null, or filtered by the transform spec, or outside the windowPeriod.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0|
|
||||
|`ingest/events/unparseable`|Number of events rejected because the events are unparseable.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0|
|
||||
|`ingest/events/duplicate`|Number of events rejected because the events are duplicated.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0|
|
||||
|`ingest/events/processed`|Number of events successfully processed per emission period.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Equal to the number of events per emission period.|
|
||||
|`ingest/rows/output`|Number of Druid rows persisted.|`dataSource`, `taskId`, `taskType`, `groupId`|Your number of events with rollup.|
|
||||
|`ingest/persists/count`|Number of times persist occurred.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on configuration.|
|
||||
|`ingest/persists/time`|Milliseconds spent doing intermediate persist.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on configuration. Generally a few minutes at most.|
|
||||
|`ingest/persists/cpu`|Cpu time in Nanoseconds spent on doing intermediate persist.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on configuration. Generally a few minutes at most.|
|
||||
|`ingest/persists/backPressure`|Milliseconds spent creating persist tasks and blocking waiting for them to finish.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0 or very low|
|
||||
|`ingest/persists/failed`|Number of persists that failed.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|0|
|
||||
|`ingest/handoff/failed`|Number of handoffs that failed.|`dataSource`, `taskId`, `taskType`, `groupId`,`tags`|0|
|
||||
|`ingest/merge/time`|Milliseconds spent merging intermediate segments.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on configuration. Generally a few minutes at most.|
|
||||
|`ingest/merge/cpu`|Cpu time in Nanoseconds spent on merging intermediate segments.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on configuration. Generally a few minutes at most.|
|
||||
|`ingest/handoff/count`|Number of handoffs that happened.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Varies. Generally greater than 0 once every segment granular period if cluster operating normally.|
|
||||
|`ingest/sink/count`|Number of sinks not handoffed.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|1~3|
|
||||
|`ingest/events/messageGap`|Time gap in milliseconds between the latest ingested event timestamp and the current system timestamp of metrics emission. If the value is increasing but lag is low, Druid may not be receiving new data. This metric is reset as new tasks spawn up.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Greater than 0, depends on the time carried in event. |
|
||||
|`ingest/notices/queueSize`|Number of pending notices to be processed by the coordinator.|`dataSource`, `tags`|Typically 0 and occasionally in lower single digits. Should not be a very high number. |
|
||||
|`ingest/notices/time`|Milliseconds taken to process a notice by the supervisor.|`dataSource`, `tags`| < 1s |
|
||||
|`ingest/pause/time`|Milliseconds spent by a task in a paused state without ingesting.|`dataSource`, `taskId`, `tags`| < 10 seconds|
|
||||
|`ingest/handoff/time`|Total number of milliseconds taken to handoff a set of segments.|`dataSource`, `taskId`, `taskType`, `tags`|Depends on coordinator cycle time.|
|
||||
|`ingest/handoff/time`|Total number of milliseconds taken to handoff a set of segments.|`dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Depends on coordinator cycle time.|
|
||||
|
||||
Note: If the JVM does not support CPU time measurement for the current thread, `ingest/merge/cpu` and `ingest/persists/cpu` will be 0.
|
||||
|
||||
|
@ -233,20 +237,20 @@ Note: If the JVM does not support CPU time measurement for the current thread, `
|
|||
|
||||
|Metric|Description| Dimensions |Normal Value|
|
||||
|------|-----------|------------------------------------------------------------|------------|
|
||||
|`task/run/time`|Milliseconds taken to run a task.| `dataSource`, `taskId`, `taskType`, `taskStatus`, `tags`|Varies|
|
||||
|`task/pending/time`|Milliseconds taken for a task to wait for running.| `dataSource`, `taskId`, `taskType`, `tags`|Varies|
|
||||
|`task/action/log/time`|Milliseconds taken to log a task action to the audit log.| `dataSource`, `taskId`, `taskType`, `taskActionType`, `tags`|< 1000 (subsecond)|
|
||||
|`task/action/run/time`|Milliseconds taken to execute a task action.| `dataSource`, `taskId`, `taskType`, `taskActionType`, `tags`|Varies from subsecond to a few seconds, based on action type.|
|
||||
|`task/action/success/count`|Number of task actions that were executed successfully during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `taskActionType`, `tags`|Varies|
|
||||
|`task/action/failed/count`|Number of task actions that failed during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `taskActionType`, `tags`|Varies|
|
||||
|`task/run/time`|Milliseconds taken to run a task.| `dataSource`, `taskId`, `taskType`, `groupId`, `taskStatus`, `tags`|Varies|
|
||||
|`task/pending/time`|Milliseconds taken for a task to wait for running.| `dataSource`, `taskId`, `taskType`, `groupId`, `tags`|Varies|
|
||||
|`task/action/log/time`|Milliseconds taken to log a task action to the audit log.| `dataSource`, `taskId`, `taskType`, `groupId`, `taskActionType`, `tags`|< 1000 (subsecond)|
|
||||
|`task/action/run/time`|Milliseconds taken to execute a task action.| `dataSource`, `taskId`, `taskType`, `groupId`, `taskActionType`, `tags`|Varies from subsecond to a few seconds, based on action type.|
|
||||
|`task/action/success/count`|Number of task actions that were executed successfully during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `groupId`, `taskActionType`, `tags`|Varies|
|
||||
|`task/action/failed/count`|Number of task actions that failed during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskId`, `taskType`, `groupId`, `taskActionType`, `tags`|Varies|
|
||||
|`task/action/batch/queueTime`|Milliseconds spent by a batch of task actions in queue. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies based on the `batchAllocationWaitTime` and number of batches in queue.|
|
||||
|`task/action/batch/runTime`|Milliseconds taken to execute a batch of task actions. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies from subsecond to a few seconds, based on action type and batch size.|
|
||||
|`task/action/batch/size`|Number of task actions in a batch that was executed during the emission period. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|Varies based on number of concurrent task actions.|
|
||||
|`task/action/batch/attempts`|Number of execution attempts for a single batch of task actions. Currently only being emitted for [batched `segmentAllocate` actions](../ingestion/tasks.md#batching-segmentallocate-actions).| `dataSource`, `taskActionType`, `interval`|1 if there are no failures or retries.|
|
||||
|`task/segmentAvailability/wait/time`|The amount of milliseconds a batch indexing task waited for newly created segments to become available for querying.| `dataSource`, `taskType`, `taskId`, `segmentAvailabilityConfirmed`, `tags`|Varies|
|
||||
|`segment/added/bytes`|Size in bytes of new segments created.| `dataSource`, `taskId`, `taskType`, `interval`, `tags`|Varies|
|
||||
|`segment/moved/bytes`|Size in bytes of segments moved/archived via the Move Task.| `dataSource`, `taskId`, `taskType`, `interval`, `tags`|Varies|
|
||||
|`segment/nuked/bytes`|Size in bytes of segments deleted via the Kill Task.| `dataSource`, `taskId`, `taskType`, `interval`, `tags`|Varies|
|
||||
|`task/segmentAvailability/wait/time`|The amount of milliseconds a batch indexing task waited for newly created segments to become available for querying.| `dataSource`, `taskType`, `groupId`, `taskId`, `segmentAvailabilityConfirmed`, `tags`|Varies|
|
||||
|`segment/added/bytes`|Size in bytes of new segments created.| `dataSource`, `taskId`, `taskType`, `groupId`, `interval`, `tags`|Varies|
|
||||
|`segment/moved/bytes`|Size in bytes of segments moved/archived via the Move Task.| `dataSource`, `taskId`, `taskType`, `groupId`, `interval`, `tags`|Varies|
|
||||
|`segment/nuked/bytes`|Size in bytes of segments deleted via the Kill Task.| `dataSource`, `taskId`, `taskType`, `groupId`, `interval`, `tags`|Varies|
|
||||
|`task/success/count`|Number of successful tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.| `dataSource`|Varies|
|
||||
|`task/failed/count`|Number of failed tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included.|`dataSource`|Varies|
|
||||
|`task/running/count`|Number of current running tasks. This metric is only available if the `TaskCountStatsMonitor` module is included.|`dataSource`|Varies|
|
||||
|
@ -321,11 +325,17 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina
|
|||
|`metadata/kill/rule/count`|Total number of rules that were automatically deleted from metadata store per each Coordinator kill rule duty run. This metric can help adjust `druid.coordinator.kill.rule.durationToRetain` configuration based on whether more or less rules need to be deleted per cycle. Note that this metric is only emitted when `druid.coordinator.kill.rule.on` is set to true.| |Varies|
|
||||
|`metadata/kill/datasource/count`|Total number of datasource metadata that were automatically deleted from metadata store per each Coordinator kill datasource duty run (Note: datasource metadata only exists for datasource created from supervisor). This metric can help adjust `druid.coordinator.kill.datasource.durationToRetain` configuration based on whether more or less datasource metadata need to be deleted per cycle. Note that this metric is only emitted when `druid.coordinator.kill.datasource.on` is set to true.| |Varies|
|
||||
|`init/serverview/time`|Time taken to initialize the coordinator server view.||Depends on the number of segments|
|
||||
|
||||
If `emitBalancingStats` is set to `true` in the Coordinator [dynamic configuration](../configuration/index.md#dynamic-configuration), then [log entries](../configuration/logging.md) for class `org.apache.druid.server.coordinator.duty.EmitClusterStatsAndMetrics` will have extra information on balancing decisions.
|
||||
|`segment/serverview/sync/healthy`|Sync status of the Coordinator 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 `segment/serverview/sync/unstableTime` to debug slow startup of the Coordinator.|`server`, `tier`|1 for fully synced servers, 0 otherwise|
|
||||
|`segment/serverview/sync/unstableTime`|Time in milliseconds for which the Coordinator 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.|
|
||||
|
||||
## General Health
|
||||
|
||||
### Service Health
|
||||
|
||||
|Metric|Description|Dimensions|Normal Value|
|
||||
|------|-----------|----------|------------|
|
||||
| `service/heartbeat` | Metric indicating the service is up. `ServiceStatusMonitor` must be enabled. |`leader` on the Overlord and Coordinator.|1|
|
||||
|
||||
### Historical
|
||||
|
||||
|Metric|Description|Dimensions|Normal Value|
|
||||
|
|
|
@ -38,7 +38,7 @@ Retention rules are persistent: they remain in effect until you change them. Dru
|
|||
|
||||
## Set retention rules
|
||||
|
||||
You can use the Druid [web console](./web-console.md) or the [Coordinator API](../api-reference/api-reference.md#coordinator) to create and manage retention rules.
|
||||
You can use the Druid [web console](./web-console.md) or the [Service status API reference](../api-reference/service-status-api.md#coordinator) to create and manage retention rules.
|
||||
|
||||
### Use the web console
|
||||
|
||||
|
@ -109,7 +109,16 @@ In the web console you can use the up and down arrows on the right side of the i
|
|||
|
||||
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` and loads all segments onto it. 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.
|
||||
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.
|
||||
|
||||
All load rules can have these properties:
|
||||
|
||||
|Property|Description|Required|Default value|
|
||||
|---------|-----------|---------|-------------|
|
||||
| `tieredReplicants`| Map from tier names to the respective number of segment replicas to be loaded on those tiers. The number of replicas for each tier must be either 0 or a positive integer.| No | When `useDefaultTierForNull` is `true`, the default value is `{"_default_tier": 2}` i.e. 2 replicas to be loaded on the `_default_tier`.<br/><br/>When `useDefaultTierForNull` is `false`, the default value is `{}` i.e. no replicas to be loaded on any tier. |
|
||||
|`useDefaultTierForNull`|Determines the default value of `tieredReplicants` if it is not specified or set to `null`.| No | `true`|
|
||||
|
||||
Specific types of load rules discussed below may have other properties too.
|
||||
|
||||
### Forever load rule
|
||||
|
||||
|
@ -130,6 +139,7 @@ The following example places one replica of each segment on a custom tier named
|
|||
Set the following property:
|
||||
|
||||
- `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.
|
||||
|
||||
### Period load rule
|
||||
|
||||
|
@ -158,6 +168,7 @@ Set the following properties:
|
|||
|
||||
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.
|
||||
- `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
|
||||
|
||||
|
@ -180,6 +191,7 @@ 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.
|
||||
- `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
|
||||
|
||||
|
|
|
@ -146,257 +146,8 @@ The Coordinator periodically checks if any of the processes need to load/drop lo
|
|||
|
||||
Please note that only 2 simultaneous lookup configuration propagation requests can be concurrently handled by a single query serving process. This limit is applied to prevent lookup handling from consuming too many server HTTP connections.
|
||||
|
||||
## API for configuring lookups
|
||||
|
||||
### Bulk update
|
||||
Lookups can be updated in bulk by posting a JSON object to `/druid/coordinator/v1/lookups/config`. The format of the json object is as follows:
|
||||
|
||||
```json
|
||||
{
|
||||
"<tierName>": {
|
||||
"<lookupName>": {
|
||||
"version": "<version>",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "<someExtractorFactoryType>",
|
||||
"<someExtractorField>": "<someExtractorValue>"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
Note that "version" is an arbitrary string assigned by the user, when making updates to existing lookup then user would need to specify a lexicographically higher version.
|
||||
|
||||
For example, a config might look something like:
|
||||
|
||||
```json
|
||||
{
|
||||
"__default": {
|
||||
"country_code": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"77483": "United States"
|
||||
}
|
||||
}
|
||||
},
|
||||
"site_id": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "cachedNamespace",
|
||||
"extractionNamespace": {
|
||||
"type": "jdbc",
|
||||
"connectorConfig": {
|
||||
"createTables": true,
|
||||
"connectURI": "jdbc:mysql:\/\/localhost:3306\/druid",
|
||||
"user": "druid",
|
||||
"password": "diurd"
|
||||
},
|
||||
"table": "lookupTable",
|
||||
"keyColumn": "country_id",
|
||||
"valueColumn": "country_name",
|
||||
"tsColumn": "timeColumn"
|
||||
},
|
||||
"firstCacheTimeout": 120000,
|
||||
"injective": true
|
||||
}
|
||||
},
|
||||
"site_id_customer1": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"847632": "Internal Use Only"
|
||||
}
|
||||
}
|
||||
},
|
||||
"site_id_customer2": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"AHF77": "Home"
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"realtime_customer1": {
|
||||
"country_code": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"77483": "United States"
|
||||
}
|
||||
}
|
||||
},
|
||||
"site_id_customer1": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"847632": "Internal Use Only"
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"realtime_customer2": {
|
||||
"country_code": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"77483": "United States"
|
||||
}
|
||||
}
|
||||
},
|
||||
"site_id_customer2": {
|
||||
"version": "v0",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"AHF77": "Home"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
All entries in the map will UPDATE existing entries. No entries will be deleted.
|
||||
|
||||
### Update lookup
|
||||
|
||||
A `POST` to a particular lookup extractor factory via `/druid/coordinator/v1/lookups/config/{tier}/{id}` creates or updates that specific extractor factory.
|
||||
|
||||
For example, a post to `/druid/coordinator/v1/lookups/config/realtime_customer1/site_id_customer1` might contain the following:
|
||||
|
||||
```json
|
||||
{
|
||||
"version": "v1",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"847632": "Internal Use Only"
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
This will replace the `site_id_customer1` lookup in the `realtime_customer1` with the definition above.
|
||||
|
||||
Assign a unique version identifier each time you update a lookup extractor factory. Otherwise the call will fail.
|
||||
|
||||
### Get all lookups
|
||||
|
||||
A `GET` to `/druid/coordinator/v1/lookups/config/all` will return all known lookup specs for all tiers.
|
||||
|
||||
### Get lookup
|
||||
|
||||
A `GET` to a particular lookup extractor factory is accomplished via `/druid/coordinator/v1/lookups/config/{tier}/{id}`
|
||||
|
||||
Using the prior example, a `GET` to `/druid/coordinator/v1/lookups/config/realtime_customer2/site_id_customer2` should return
|
||||
|
||||
```json
|
||||
{
|
||||
"version": "v1",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"AHF77": "Home"
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
### Delete lookup
|
||||
|
||||
A `DELETE` to `/druid/coordinator/v1/lookups/config/{tier}/{id}` will remove that lookup from the cluster. If it was last lookup in the tier, then tier is deleted as well.
|
||||
|
||||
### Delete tier
|
||||
|
||||
A `DELETE` to `/druid/coordinator/v1/lookups/config/{tier}` will remove that tier from the cluster.
|
||||
|
||||
### List tier names
|
||||
|
||||
A `GET` to `/druid/coordinator/v1/lookups/config` will return a list of known tier names in the dynamic configuration.
|
||||
To discover a list of tiers currently active in the cluster in addition to ones known in the dynamic configuration, the parameter `discover=true` can be added as per `/druid/coordinator/v1/lookups/config?discover=true`.
|
||||
|
||||
### List lookup names
|
||||
|
||||
A `GET` to `/druid/coordinator/v1/lookups/config/{tier}` will return a list of known lookup names for that tier.
|
||||
|
||||
These end points can be used to get the propagation status of configured lookups to processes using lookups such as Historicals.
|
||||
|
||||
## API for lookup status
|
||||
|
||||
### List load status of all lookups
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/status` with optional query parameter `detailed`.
|
||||
|
||||
### List load status of lookups in a tier
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/status/{tier}` with optional query parameter `detailed`.
|
||||
|
||||
### List load status of single lookup
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/status/{tier}/{lookup}` with optional query parameter `detailed`.
|
||||
|
||||
### List lookup state of all processes
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/nodeStatus` with optional query parameter `discover` to discover tiers advertised by other Druid nodes, or by default, returning all configured lookup tiers. The default response will also include the lookups which are loaded, being loaded, or being dropped on each node, for each tier, including the complete lookup spec. Add the optional query parameter `detailed=false` to only include the 'version' of the lookup instead of the complete spec.
|
||||
|
||||
### List lookup state of processes in a tier
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}`
|
||||
|
||||
### List lookup state of single process
|
||||
|
||||
`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}/{host:port}`
|
||||
|
||||
## Internal API
|
||||
|
||||
The Peon, Router, Broker, and Historical processes all have the ability to consume lookup configuration.
|
||||
There is an internal API these processes use to list/load/drop their lookups starting at `/druid/listen/v1/lookups`.
|
||||
These follow the same convention for return values as the cluster wide dynamic configuration. Following endpoints
|
||||
can be used for debugging purposes but not otherwise.
|
||||
|
||||
### Get lookups
|
||||
|
||||
A `GET` to the process at `/druid/listen/v1/lookups` will return a json map of all the lookups currently active on the process.
|
||||
The return value will be a json map of the lookups to their extractor factories.
|
||||
|
||||
```json
|
||||
{
|
||||
"site_id_customer2": {
|
||||
"version": "v1",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"AHF77": "Home"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
### Get lookup
|
||||
|
||||
A `GET` to the process at `/druid/listen/v1/lookups/some_lookup_name` will return the LookupExtractorFactory for the lookup identified by `some_lookup_name`.
|
||||
The return value will be the json representation of the factory.
|
||||
|
||||
```json
|
||||
{
|
||||
"version": "v1",
|
||||
"lookupExtractorFactory": {
|
||||
"type": "map",
|
||||
"map": {
|
||||
"AHF77": "Home"
|
||||
}
|
||||
}
|
||||
}
|
||||
```
|
||||
## API
|
||||
See [Lookups API](../api-reference/lookups-api.md) for reference on configuring lookups and lookup status.
|
||||
|
||||
## Configuration
|
||||
|
||||
|
|
|
@ -84,6 +84,7 @@ The following built-in functions are available.
|
|||
|parse_long|parse_long(string[, radix]) parses a string as a long with the given radix, or 10 (decimal) if a radix is not provided.|
|
||||
|regexp_extract|regexp_extract(expr, pattern[, index]) applies a regular expression pattern and extracts a capture group index, or null if there is no match. If index is unspecified or zero, returns the substring that matched the pattern. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern.|
|
||||
|regexp_like|regexp_like(expr, pattern) returns whether `expr` matches regular expression `pattern`. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern. |
|
||||
|regexp_replace|regexp_replace(expr, pattern, replacement) replaces all instances of a regular expression pattern with a given replacement string. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern.|
|
||||
|contains_string|contains_string(expr, string) returns whether `expr` contains `string` as a substring. This method is case-sensitive.|
|
||||
|icontains_string|contains_string(expr, string) returns whether `expr` contains `string` as a substring. This method is case-insensitive.|
|
||||
|replace|replace(expr, pattern, replacement) replaces pattern with replacement|
|
||||
|
|
|
@ -85,10 +85,15 @@ their "base" (bottom-leftmost) datasource, as described in the [join](#join) sec
|
|||
the results are brought back to the Broker. Then, the Broker continues on with the rest of the query as if the subquery
|
||||
was replaced with an inline datasource.
|
||||
|
||||
In most cases, Druid buffers subquery results in memory on the Broker before the rest of the query proceeds. Therefore, subqueries execute sequentially. The total number of rows buffered across all subqueries of a given query cannot exceed the [`druid.server.http.maxSubqueryRows`](../configuration/index.md) which defaults to 100000 rows. Otherwise Druid throws a resource limit exceeded exception: "Subquery generated results beyond maximum."
|
||||
In most cases, Druid buffers subquery results in memory on the Broker before the rest of the query proceeds.
|
||||
Therefore, subqueries execute sequentially. The total number of rows buffered across all subqueries of a given query
|
||||
cannot exceed the [`druid.server.http.maxSubqueryRows`](../configuration/index.md) which defaults to 100000 rows, or the
|
||||
[`druid.server.http.maxSubqueryBytes`](../configuration/index.md) if set. Otherwise, Druid throws a resource limit exceeded
|
||||
exception.
|
||||
|
||||
There is one exception: if the outer query and all subqueries are the [groupBy](groupbyquery.md) type, then subquery
|
||||
results can be processed in a streaming fashion and the `druid.server.http.maxSubqueryRows` limit does not apply.
|
||||
results can be processed in a streaming fashion and the `druid.server.http.maxSubqueryRows` and `druid.server.http.maxSubqueryBytes`
|
||||
limits do not apply.
|
||||
|
||||
### `join`
|
||||
|
||||
|
|
|
@ -1141,6 +1141,15 @@ Applies a regular expression to the string expression and returns the _n_th matc
|
|||
|
||||
Returns true or false signifying whether the regular expression finds a match in the string expression.
|
||||
|
||||
## REGEXP_REPLACE
|
||||
|
||||
`REGEXP_REPLACE(<CHARACTER>, <CHARACTER>, <CHARACTER>)`
|
||||
|
||||
**Function type:** [Scalar, string](sql-scalar.md#string-functions)
|
||||
|
||||
Replaces all occurrences of a regular expression in a string expression with a replacement string. The replacement
|
||||
string may refer to capture groups using `$1`, `$2`, etc.
|
||||
|
||||
## REPEAT
|
||||
|
||||
`REPEAT(<CHARACTER>, [<INTEGER>])`
|
||||
|
|
|
@ -157,6 +157,7 @@ Segments table provides details on all Druid segments, whether they are publishe
|
|||
|dimensions|VARCHAR|JSON-serialized form of the segment dimensions|
|
||||
|metrics|VARCHAR|JSON-serialized form of the segment metrics|
|
||||
|last_compaction_state|VARCHAR|JSON-serialized form of the compaction task's config (compaction task which created this segment). May be null if segment was not created by compaction task.|
|
||||
|replication_factor|BIGINT|Total number of replicas of the segment that are required to be loaded across all historical tiers, based on the load rule that currently applies to this segment. If this value is 0, the segment is not assigned to any historical and will not be loaded. This value is -1 if load rules for the segment have not been evaluated yet.|
|
||||
|
||||
For example, to retrieve all currently active segments for datasource "wikipedia", use the query:
|
||||
|
||||
|
|
|
@ -103,6 +103,7 @@ String functions accept strings, and return a type appropriate to the function.
|
|||
|`POSITION(needle IN haystack [FROM fromIndex])`|Returns the index of `needle` within `haystack`, with indexes starting from 1. The search will begin at `fromIndex`, or 1 if `fromIndex` is not specified. If `needle` is not found, returns 0.|
|
||||
|`REGEXP_EXTRACT(expr, pattern, [index])`|Apply regular expression `pattern` to `expr` and extract a capture group, or `NULL` if there is no match. If index is unspecified or zero, returns the first substring that matched the pattern. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern. Note: when `druid.generic.useDefaultValueForNull = true`, it is not possible to differentiate an empty-string match from a non-match (both will return `NULL`).|
|
||||
|`REGEXP_LIKE(expr, pattern)`|Returns whether `expr` matches regular expression `pattern`. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern. Similar to [`LIKE`](sql-operators.md#logical-operators), but uses regexps instead of LIKE patterns. Especially useful in WHERE clauses.|
|
||||
|`REGEXP_REPLACE(expr, pattern, replacement)`|Replaces all occurrences of regular expression `pattern` within `expr` with `replacement`. The replacement string may refer to capture groups using `$1`, `$2`, etc. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern.|
|
||||
|`CONTAINS_STRING(expr, str)`|Returns true if the `str` is a substring of `expr`.|
|
||||
|`ICONTAINS_STRING(expr, str)`|Returns true if the `str` is a substring of `expr`. The match is case-insensitive.|
|
||||
|`REPLACE(expr, pattern, replacement)`|Replaces pattern with replacement in `expr`, and returns the result.|
|
||||
|
|
|
@ -76,6 +76,8 @@ EXPLAIN PLAN statements return:
|
|||
|
||||
Example 1: EXPLAIN PLAN for a `SELECT` query on the `wikipedia` datasource:
|
||||
|
||||
<details><summary>Show the query</summary>
|
||||
|
||||
```sql
|
||||
EXPLAIN PLAN FOR
|
||||
SELECT
|
||||
|
@ -85,9 +87,12 @@ FROM wikipedia
|
|||
WHERE channel IN (SELECT page FROM wikipedia GROUP BY page ORDER BY COUNT(*) DESC LIMIT 10)
|
||||
GROUP BY channel
|
||||
```
|
||||
</details>
|
||||
|
||||
The above EXPLAIN PLAN query returns the following result:
|
||||
|
||||
<details><summary>Show the result</summary>
|
||||
|
||||
```json
|
||||
[
|
||||
[
|
||||
|
@ -224,13 +229,15 @@ The above EXPLAIN PLAN query returns the following result:
|
|||
}
|
||||
]
|
||||
```
|
||||
</details>
|
||||
|
||||
Example 2: EXPLAIN PLAN for a `REPLACE` query that replaces all the data in the `wikipedia` datasource:
|
||||
Example 2: EXPLAIN PLAN for an `INSERT` query that inserts data into the `wikipedia` datasource:
|
||||
|
||||
<details><summary>Show the query</summary>
|
||||
|
||||
```sql
|
||||
EXPLAIN PLAN FOR
|
||||
REPLACE INTO wikipedia
|
||||
OVERWRITE ALL
|
||||
INSERT INTO wikipedia2
|
||||
SELECT
|
||||
TIME_PARSE("timestamp") AS __time,
|
||||
namespace,
|
||||
|
@ -247,11 +254,14 @@ FROM TABLE(
|
|||
'[{"name":"timestamp","type":"string"},{"name":"namespace","type":"string"},{"name":"cityName","type":"string"},{"name":"countryName","type":"string"},{"name":"regionIsoCode","type":"string"},{"name":"metroCode","type":"long"},{"name":"countryIsoCode","type":"string"},{"name":"regionName","type":"string"}]'
|
||||
)
|
||||
)
|
||||
PARTITIONED BY HOUR
|
||||
CLUSTERED BY cityName
|
||||
PARTITIONED BY ALL
|
||||
```
|
||||
</details>
|
||||
|
||||
The above EXPLAIN PLAN query returns the following result:
|
||||
|
||||
The above EXPLAIN PLAN returns the following result:
|
||||
|
||||
<details><summary>Show the result</summary>
|
||||
|
||||
```json
|
||||
[
|
||||
|
@ -323,12 +333,229 @@ The above EXPLAIN PLAN query returns the following result:
|
|||
}
|
||||
],
|
||||
"resultFormat": "compactedList",
|
||||
"orderBy": [
|
||||
"columns": [
|
||||
"cityName",
|
||||
"countryIsoCode",
|
||||
"countryName",
|
||||
"metroCode",
|
||||
"namespace",
|
||||
"regionIsoCode",
|
||||
"regionName",
|
||||
"v0"
|
||||
],
|
||||
"legacy": false,
|
||||
"context": {
|
||||
"finalizeAggregations": false,
|
||||
"forceExpressionVirtualColumns": true,
|
||||
"groupByEnableMultiValueUnnesting": false,
|
||||
"maxNumTasks": 5,
|
||||
"multiStageQuery": true,
|
||||
"queryId": "42e3de2b-daaf-40f9-a0e7-2c6184529ea3",
|
||||
"scanSignature": "[{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"v0\",\"type\":\"LONG\"}]",
|
||||
"sqlInsertSegmentGranularity": "{\"type\":\"all\"}",
|
||||
"sqlQueryId": "42e3de2b-daaf-40f9-a0e7-2c6184529ea3",
|
||||
"useNativeQueryExplain": true
|
||||
},
|
||||
"granularity": {
|
||||
"type": "all"
|
||||
}
|
||||
},
|
||||
"signature": [
|
||||
{
|
||||
"name": "v0",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "namespace",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "cityName",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "countryName",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "regionIsoCode",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "metroCode",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "countryIsoCode",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "regionName",
|
||||
"type": "STRING"
|
||||
}
|
||||
],
|
||||
"columnMappings": [
|
||||
{
|
||||
"queryColumn": "v0",
|
||||
"outputColumn": "__time"
|
||||
},
|
||||
{
|
||||
"queryColumn": "namespace",
|
||||
"outputColumn": "namespace"
|
||||
},
|
||||
{
|
||||
"queryColumn": "cityName",
|
||||
"outputColumn": "cityName"
|
||||
},
|
||||
{
|
||||
"queryColumn": "countryName",
|
||||
"outputColumn": "countryName"
|
||||
},
|
||||
{
|
||||
"queryColumn": "regionIsoCode",
|
||||
"outputColumn": "regionIsoCode"
|
||||
},
|
||||
{
|
||||
"queryColumn": "metroCode",
|
||||
"outputColumn": "metroCode"
|
||||
},
|
||||
{
|
||||
"queryColumn": "countryIsoCode",
|
||||
"outputColumn": "countryIsoCode"
|
||||
},
|
||||
{
|
||||
"queryColumn": "regionName",
|
||||
"outputColumn": "regionName"
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
[
|
||||
{
|
||||
"name": "EXTERNAL",
|
||||
"type": "EXTERNAL"
|
||||
},
|
||||
{
|
||||
"name": "wikipedia",
|
||||
"type": "DATASOURCE"
|
||||
}
|
||||
],
|
||||
{
|
||||
"statementType": "INSERT",
|
||||
"targetDataSource": "wikipedia",
|
||||
"partitionedBy": {
|
||||
"type": "all"
|
||||
}
|
||||
}
|
||||
]
|
||||
```
|
||||
</details>
|
||||
|
||||
Example 3: EXPLAIN PLAN for a `REPLACE` query that replaces all the data in the `wikipedia` datasource with a `DAY`
|
||||
time partitioning, and `cityName` and `countryName` as the clustering columns:
|
||||
|
||||
<details><summary>Show the query</summary>
|
||||
|
||||
```sql
|
||||
EXPLAIN PLAN FOR
|
||||
REPLACE INTO wikipedia
|
||||
OVERWRITE ALL
|
||||
SELECT
|
||||
TIME_PARSE("timestamp") AS __time,
|
||||
namespace,
|
||||
cityName,
|
||||
countryName,
|
||||
regionIsoCode,
|
||||
metroCode,
|
||||
countryIsoCode,
|
||||
regionName
|
||||
FROM TABLE(
|
||||
EXTERN(
|
||||
'{"type":"http","uris":["https://druid.apache.org/data/wikipedia.json.gz"]}',
|
||||
'{"type":"json"}',
|
||||
'[{"name":"timestamp","type":"string"},{"name":"namespace","type":"string"},{"name":"cityName","type":"string"},{"name":"countryName","type":"string"},{"name":"regionIsoCode","type":"string"},{"name":"metroCode","type":"long"},{"name":"countryIsoCode","type":"string"},{"name":"regionName","type":"string"}]'
|
||||
)
|
||||
)
|
||||
PARTITIONED BY DAY
|
||||
CLUSTERED BY cityName, countryName
|
||||
```
|
||||
</details>
|
||||
|
||||
|
||||
The above EXPLAIN PLAN query returns the following result:
|
||||
|
||||
<details><summary>Show the result</summary>
|
||||
|
||||
```json
|
||||
[
|
||||
[
|
||||
{
|
||||
"query": {
|
||||
"queryType": "scan",
|
||||
"dataSource": {
|
||||
"type": "external",
|
||||
"inputSource": {
|
||||
"type": "http",
|
||||
"uris": [
|
||||
"https://druid.apache.org/data/wikipedia.json.gz"
|
||||
]
|
||||
},
|
||||
"inputFormat": {
|
||||
"type": "json",
|
||||
"keepNullColumns": false,
|
||||
"assumeNewlineDelimited": false,
|
||||
"useJsonNodeReader": false
|
||||
},
|
||||
"signature": [
|
||||
{
|
||||
"name": "timestamp",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "namespace",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "cityName",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "countryName",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "regionIsoCode",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "metroCode",
|
||||
"type": "LONG"
|
||||
},
|
||||
{
|
||||
"name": "countryIsoCode",
|
||||
"type": "STRING"
|
||||
},
|
||||
{
|
||||
"name": "regionName",
|
||||
"type": "STRING"
|
||||
}
|
||||
]
|
||||
},
|
||||
"intervals": {
|
||||
"type": "intervals",
|
||||
"intervals": [
|
||||
"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z"
|
||||
]
|
||||
},
|
||||
"virtualColumns": [
|
||||
{
|
||||
"columnName": "cityName",
|
||||
"order": "ascending"
|
||||
"type": "expression",
|
||||
"name": "v0",
|
||||
"expression": "timestamp_parse(\"timestamp\",null,'UTC')",
|
||||
"outputType": "LONG"
|
||||
}
|
||||
],
|
||||
"resultFormat": "compactedList",
|
||||
"columns": [
|
||||
"cityName",
|
||||
"countryIsoCode",
|
||||
|
@ -344,10 +571,10 @@ The above EXPLAIN PLAN query returns the following result:
|
|||
"finalizeAggregations": false,
|
||||
"groupByEnableMultiValueUnnesting": false,
|
||||
"maxNumTasks": 5,
|
||||
"queryId": "b474c0d5-a5ce-432d-be94-535ccdb7addc",
|
||||
"queryId": "d88e0823-76d4-40d9-a1a7-695c8577b79f",
|
||||
"scanSignature": "[{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"countryName\",\"type\":\"STRING\"},{\"name\":\"metroCode\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"regionIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"},{\"name\":\"v0\",\"type\":\"LONG\"}]",
|
||||
"sqlInsertSegmentGranularity": "\"HOUR\"",
|
||||
"sqlQueryId": "b474c0d5-a5ce-432d-be94-535ccdb7addc",
|
||||
"sqlInsertSegmentGranularity": "\"DAY\"",
|
||||
"sqlQueryId": "d88e0823-76d4-40d9-a1a7-695c8577b79f",
|
||||
"sqlReplaceTimeChunks": "all"
|
||||
},
|
||||
"granularity": {
|
||||
|
@ -437,13 +664,16 @@ The above EXPLAIN PLAN query returns the following result:
|
|||
{
|
||||
"statementType": "REPLACE",
|
||||
"targetDataSource": "wikipedia",
|
||||
"partitionedBy": "HOUR",
|
||||
"clusteredBy": "`cityName`",
|
||||
"replaceTimeChunks": "'ALL'"
|
||||
"partitionedBy": "DAY",
|
||||
"clusteredBy": ["cityName","countryName"],
|
||||
"replaceTimeChunks": "all"
|
||||
}
|
||||
]
|
||||
```
|
||||
|
||||
</details>
|
||||
|
||||
|
||||
In this case the JOIN operator gets translated to a `join` datasource. See the [Join translation](#joins) section
|
||||
for more details about how this works.
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ sidebar_label: "Docker for tutorials"
|
|||
|
||||
|
||||
Apache Druid provides a custom Jupyter container that contains the prerequisites
|
||||
for all Jupyter-based Druid tutorials, as well as all of the tutorials themselves.
|
||||
for all [Jupyter-based Druid tutorials](tutorial-jupyter-index.md), as well as all of the tutorials themselves.
|
||||
You can run the Jupyter container, as well as containers for Druid and Apache Kafka,
|
||||
using the Docker Compose file provided in the Druid GitHub repository.
|
||||
|
||||
|
|
|
@ -67,10 +67,9 @@ The notebooks are located in the [apache/druid repo](https://github.com/apache/d
|
|||
|
||||
The links that follow are the raw GitHub URLs, so you can use them to download the notebook directly, such as with `wget`, or manually through your web browser. Note that if you save the file from your web browser, make sure to remove the `.txt` extension.
|
||||
|
||||
- [Introduction to the Druid REST API](
|
||||
https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/api-tutorial.ipynb)
|
||||
walks you through some of the basics related to the Druid REST API and several endpoints.
|
||||
- [Introduction to the Druid Python API](
|
||||
https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/Python_API_Tutorial.ipynb)
|
||||
walks you through some of the basics related to the Druid API using the Python wrapper API.
|
||||
- [Introduction to Druid SQL](https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/sql-tutorial.ipynb) covers the basics of Druid SQL.
|
||||
- [Introduction to the Druid REST API](https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/notebooks/04-api/00-getting-started.ipynb) walks you through some of the
|
||||
basics related to the Druid REST API and several endpoints.
|
||||
- [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) walks you through some of the
|
||||
basics related to the Druid API using the Python wrapper API.
|
||||
- [Learn the basics of Druid SQL](https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/notebooks/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.
|
||||
- [Ingest and query data from Apache Kafka](https://raw.githubusercontent.com/apache/druid/master/examples/quickstart/jupyter-notebooks/notebooks/02-ingestion/01-streaming-from-kafka.ipynb) walks you through ingesting an event stream from Kafka.
|
||||
|
|
|
@ -154,7 +154,7 @@ sample usage:
|
|||
'\'common.jvm.config\' & \'common.runtime.properties\' files. \n'
|
||||
'If this argument is not given, config from \n'
|
||||
'conf/druid/auto directory is used.\n'
|
||||
'Note. zookeeper config cannot be overriden.\n')
|
||||
'Note. zookeeper config cannot be overridden.\n')
|
||||
parser.add_argument('--compute', action='store_true',
|
||||
help='Does not start Druid, only displays the memory allocated \n'
|
||||
'to each service if started with the given total memory.\n')
|
||||
|
|
|
@ -82,5 +82,10 @@
|
|||
<Logger name="com.sun.jersey.guice" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
|
||||
<!-- Quieter KafkaSupervisors -->
|
||||
<Logger name="org.apache.kafka.clients.consumer.internals" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
</Loggers>
|
||||
</Configuration>
|
||||
|
|
|
@ -82,5 +82,10 @@
|
|||
<Logger name="com.sun.jersey.guice" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
|
||||
<!-- Quieter KafkaSupervisors -->
|
||||
<Logger name="org.apache.kafka.clients.consumer.internals" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
</Loggers>
|
||||
</Configuration>
|
||||
|
|
|
@ -82,5 +82,10 @@
|
|||
<Logger name="com.sun.jersey.guice" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
|
||||
<!-- Quieter KafkaSupervisors -->
|
||||
<Logger name="org.apache.kafka.clients.consumer.internals" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
</Loggers>
|
||||
</Configuration>
|
||||
|
|
|
@ -82,5 +82,10 @@
|
|||
<Logger name="com.sun.jersey.guice" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
|
||||
<!-- Quieter KafkaSupervisors -->
|
||||
<Logger name="org.apache.kafka.clients.consumer.internals" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
</Loggers>
|
||||
</Configuration>
|
||||
|
|
|
@ -82,5 +82,10 @@
|
|||
<Logger name="com.sun.jersey.guice" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
|
||||
<!-- Quieter KafkaSupervisors -->
|
||||
<Logger name="org.apache.kafka.clients.consumer.internals" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
</Loggers>
|
||||
</Configuration>
|
||||
|
|
|
@ -82,5 +82,10 @@
|
|||
<Logger name="com.sun.jersey.guice" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
|
||||
<!-- Quieter KafkaSupervisors -->
|
||||
<Logger name="org.apache.kafka.clients.consumer.internals" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
</Loggers>
|
||||
</Configuration>
|
||||
|
|
|
@ -82,5 +82,10 @@
|
|||
<Logger name="com.sun.jersey.guice" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
|
||||
<!-- Quieter KafkaSupervisors -->
|
||||
<Logger name="org.apache.kafka.clients.consumer.internals" level="warn" additivity="false">
|
||||
<Appender-ref ref="FileAppender"/>
|
||||
</Logger>
|
||||
</Loggers>
|
||||
</Configuration>
|
||||
|
|
|
@ -31,13 +31,13 @@ FROM jupyter/base-notebook
|
|||
WORKDIR /home/jovyan
|
||||
|
||||
# Install required Python packages
|
||||
RUN pip install requests
|
||||
RUN pip install pandas
|
||||
RUN pip install numpy
|
||||
RUN pip install seaborn
|
||||
RUN pip install bokeh
|
||||
RUN pip install kafka-python
|
||||
RUN pip install sortedcontainers
|
||||
RUN pip install requests \
|
||||
pip install pandas \
|
||||
pip install numpy \
|
||||
pip install seaborn \
|
||||
pip install bokeh \
|
||||
pip install kafka-python \
|
||||
pip install sortedcontainers
|
||||
|
||||
# Install druidapi client from apache/druid
|
||||
# Local install requires sudo privileges
|
||||
|
@ -45,21 +45,22 @@ USER root
|
|||
ADD druidapi /home/jovyan/druidapi
|
||||
WORKDIR /home/jovyan/druidapi
|
||||
RUN pip install .
|
||||
WORKDIR /home/jovyan
|
||||
|
||||
|
||||
|
||||
# WIP -- install DruidDataDriver as a package
|
||||
# Import data generator and configuration file
|
||||
# Change permissions to allow import (requires sudo privileges)
|
||||
# WIP -- change to apache repo
|
||||
ADD https://raw.githubusercontent.com/shallada/druid/data-generator/examples/quickstart/jupyter-notebooks/data-generator/DruidDataDriver.py .
|
||||
ADD docker-jupyter/kafka_docker_config.json .
|
||||
RUN chmod 664 DruidDataDriver.py
|
||||
RUN chmod 664 kafka_docker_config.json
|
||||
|
||||
# The Jupyter notebooks themselves are mounted into the image's /home/jovyan/notebooks
|
||||
# path when running this image.
|
||||
RUN mkdir -p /home/jovyan/notebooks
|
||||
|
||||
WORKDIR /home/jovyan/notebooks
|
||||
USER jovyan
|
||||
|
||||
# Copy the Jupyter notebook tutorials from the
|
||||
# build directory to the image working directory
|
||||
COPY ./*ipynb .
|
||||
|
||||
|
||||
# Add location of the data generator to PYTHONPATH
|
||||
ENV PYTHONPATH "${PYTHONPATH}:/home/jovyan"
|
||||
ENV PYTHONPATH "${PYTHONPATH}:/home/jovyan/notebooks/02-ingestion"
|
||||
|
||||
|
|
|
@ -45,9 +45,7 @@ services:
|
|||
zookeeper:
|
||||
image: zookeeper:latest
|
||||
container_name: zookeeper
|
||||
profiles: ["druid-jupyter", "all-services"]
|
||||
ports:
|
||||
- "2181:2181"
|
||||
profiles: ["druid-jupyter", "kafka-jupyter", "all-services"]
|
||||
environment:
|
||||
- ZOO_MY_ID=1
|
||||
- ALLOW_ANONYMOUS_LOGIN=yes
|
||||
|
@ -55,7 +53,7 @@ services:
|
|||
kafka:
|
||||
image: bitnami/kafka:latest
|
||||
container_name: kafka-broker
|
||||
profiles: ["all-services"]
|
||||
profiles: ["kafka-jupyter", "all-services"]
|
||||
ports:
|
||||
# To learn about configuring Kafka for access across networks see
|
||||
# https://www.confluent.io/blog/kafka-client-cannot-connect-to-broker-on-aws-on-docker-etc/
|
||||
|
@ -163,11 +161,13 @@ services:
|
|||
context: ..
|
||||
dockerfile: Dockerfile
|
||||
container_name: jupyter
|
||||
profiles: ["jupyter", "druid-jupyter", "all-services"]
|
||||
profiles: ["jupyter", "kafka-jupyter", "druid-jupyter", "all-services"]
|
||||
environment:
|
||||
DOCKER_STACKS_JUPYTER_CMD: "notebook"
|
||||
JUPYTER_ENABLE_LAB: "yes"
|
||||
JUPYTER_TOKEN: "docker"
|
||||
DOCKER_STACKS_JUPYTER_CMD: "lab"
|
||||
NOTEBOOK_ARGS: "--NotebookApp.token=''"
|
||||
ports:
|
||||
- "${JUPYTER_PORT:-8889}:8888"
|
||||
volumes:
|
||||
- ./notebooks:/home/jovyan/work
|
||||
- ../notebooks:/home/jovyan/notebooks
|
||||
|
|
|
@ -45,9 +45,7 @@ services:
|
|||
zookeeper:
|
||||
image: zookeeper:latest
|
||||
container_name: zookeeper
|
||||
profiles: ["druid-jupyter", "all-services"]
|
||||
ports:
|
||||
- "2181:2181"
|
||||
profiles: ["druid-jupyter", "kafka-jupyter", "all-services"]
|
||||
environment:
|
||||
- ZOO_MY_ID=1
|
||||
- ALLOW_ANONYMOUS_LOGIN=yes
|
||||
|
@ -55,7 +53,7 @@ services:
|
|||
kafka:
|
||||
image: bitnami/kafka:latest
|
||||
container_name: kafka-broker
|
||||
profiles: ["all-services"]
|
||||
profiles: ["kafka-jupyter", "all-services"]
|
||||
ports:
|
||||
# To learn about configuring Kafka for access across networks see
|
||||
# https://www.confluent.io/blog/kafka-client-cannot-connect-to-broker-on-aws-on-docker-etc/
|
||||
|
@ -161,11 +159,13 @@ services:
|
|||
jupyter:
|
||||
image: imply/druid-notebook:latest
|
||||
container_name: jupyter
|
||||
profiles: ["jupyter", "druid-jupyter", "all-services"]
|
||||
profiles: ["jupyter", "kafka-jupyter", "druid-jupyter", "all-services"]
|
||||
environment:
|
||||
DOCKER_STACKS_JUPYTER_CMD: "notebook"
|
||||
JUPYTER_ENABLE_LAB: "yes"
|
||||
JUPYTER_TOKEN: "docker"
|
||||
DOCKER_STACKS_JUPYTER_CMD: "lab"
|
||||
NOTEBOOK_ARGS: "--NotebookApp.token=''"
|
||||
ports:
|
||||
- "${JUPYTER_PORT:-8889}:8888"
|
||||
volumes:
|
||||
- ./notebooks:/home/jovyan/work
|
||||
- ../notebooks:/home/jovyan/notebooks
|
||||
|
|
File diff suppressed because one or more lines are too long
|
@ -66,7 +66,7 @@
|
|||
"topics and use a simple set of Python wrappers around the underlying REST API. The\n",
|
||||
"wrappers reside in the `druidapi` package within this directory. While the package\n",
|
||||
"can be used in any Python program, the key purpose, at present, is to support these\n",
|
||||
"notebooks. See the [Introduction to the Druid Python API](Python_API_Tutorial.ipynb)\n",
|
||||
"notebooks. See the [Introduction to the Druid Python API](01-druidapi-package-intro.ipynb)\n",
|
||||
"for an overview of the Python API."
|
||||
]
|
||||
},
|
||||
|
@ -87,12 +87,12 @@
|
|||
"notebook directly, such as with `wget`, or manually through your web browser. Note\n",
|
||||
"that if you save the file from your web browser, make sure to remove the `.txt` extension.\n",
|
||||
"\n",
|
||||
"- [Introduction to the Druid REST API](api-tutorial.ipynb) walks you through some of the\n",
|
||||
"- [Introduction to the Druid REST API](../04-api/00-getting-started.ipynb) walks you through some of the\n",
|
||||
" basics related to the Druid REST API and several endpoints.\n",
|
||||
"- [Introduction to the Druid Python API](Python_API_Tutorial.ipynb) walks you through some of the\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](sql-tutorial.ipynb) introduces you to the unique aspects of Druid SQL with the primary focus on the SELECT statement. \n",
|
||||
"- [Ingest and query data from Apache Kafka](kafka-tutorial.ipynb) walks you through ingesting an event stream from Kafka."
|
||||
"- [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",
|
||||
"- [Ingest and query data from Apache Kafka](../02-ingestion/01-streaming-from-kafka.ipynb) walks you through ingesting an event stream from Kafka."
|
||||
]
|
||||
},
|
||||
{
|
||||
|
@ -154,7 +154,7 @@
|
|||
"name": "python",
|
||||
"nbconvert_exporter": "python",
|
||||
"pygments_lexer": "ipython3",
|
||||
"version": "3.8.16"
|
||||
"version": "3.11.4"
|
||||
}
|
||||
},
|
||||
"nbformat": 4,
|
|
@ -1,6 +1,7 @@
|
|||
{
|
||||
"cells": [
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "ce2efaaa",
|
||||
"metadata": {},
|
||||
|
@ -26,7 +27,7 @@
|
|||
" ~ under the License.\n",
|
||||
" -->\n",
|
||||
"\n",
|
||||
"This notebook provides a quick introduction to the Python wrapper around the [Druid REST API](api-tutorial.ipynb). This notebook assumes you are familiar with the basics of the REST API, and the [set of operations which Druid provides](https://druid.apache.org/docs/latest/operations/api-reference.html). This tutorial focuses on using Python to access those APIs rather than explaining the APIs themselves. The APIs themselves are covered in other notebooks that use the Python API.\n",
|
||||
"This notebook provides a quick introduction to the Python wrapper around the [Druid REST API](api-tutorial.ipynb). This notebook assumes you are familiar with the basics of the REST API, and the [set of operations which Druid provides](https://druid.apache.org/docs/latest/api-reference/api-reference.html). This tutorial focuses on using Python to access those APIs rather than explaining the APIs themselves. The APIs themselves are covered in other notebooks that use the Python API.\n",
|
||||
"\n",
|
||||
"This tutorial works with Druid 25.0.0 or later.\n",
|
||||
"\n",
|
||||
|
@ -52,6 +53,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "fb68a838",
|
||||
"metadata": {},
|
||||
|
@ -78,6 +80,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "8b4e774b",
|
||||
"metadata": {},
|
||||
|
@ -98,6 +101,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "be992774",
|
||||
"metadata": {},
|
||||
|
@ -116,6 +120,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "e803c9fe",
|
||||
"metadata": {},
|
||||
|
@ -134,6 +139,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "d78a6c35",
|
||||
"metadata": {},
|
||||
|
@ -152,6 +158,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "012b2e61",
|
||||
"metadata": {},
|
||||
|
@ -174,6 +181,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "d051bc5e",
|
||||
"metadata": {},
|
||||
|
@ -192,6 +200,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "b8261ab0",
|
||||
"metadata": {},
|
||||
|
@ -210,6 +219,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "ff311595",
|
||||
"metadata": {},
|
||||
|
@ -228,6 +238,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "7392e484",
|
||||
"metadata": {},
|
||||
|
@ -251,6 +262,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "c6c4e1d4",
|
||||
"metadata": {},
|
||||
|
@ -259,6 +271,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "f414d145",
|
||||
"metadata": {},
|
||||
|
@ -279,6 +292,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "7b944084",
|
||||
"metadata": {},
|
||||
|
@ -305,6 +319,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "937dc6b1",
|
||||
"metadata": {},
|
||||
|
@ -349,6 +364,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "481af1f2",
|
||||
"metadata": {},
|
||||
|
@ -367,6 +383,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "99f8db7b",
|
||||
"metadata": {},
|
||||
|
@ -375,6 +392,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "9e3be017",
|
||||
"metadata": {},
|
||||
|
@ -431,6 +449,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "ef4512f8",
|
||||
"metadata": {},
|
||||
|
@ -449,6 +468,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "11d9c95a",
|
||||
"metadata": {},
|
||||
|
@ -467,6 +487,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "936f57fb",
|
||||
"metadata": {},
|
||||
|
@ -485,13 +506,14 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "c1152f41",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"## Datasource Client\n",
|
||||
"\n",
|
||||
"The Datasource client lets you perform operations on datasource objects. The SQL layer allows you to get metadata and do queries. The datasource client works with the underlying segments. Explaining the full functionality is the topic of another notebook. For now, you can use the datasource client to clean up the datasource created above. The `True` argument asks for \"if exists\" semantics so you don't get an error if the datasource was alredy deleted."
|
||||
"The Datasource client lets you perform operations on datasource objects. The SQL layer allows you to get metadata and do queries. The datasource client works with the underlying segments. Explaining the full functionality is the topic of another notebook. For now, you can use the datasource client to clean up the datasource created above. The `True` argument asks for \"if exists\" semantics so you don't get an error if the datasource was already deleted."
|
||||
]
|
||||
},
|
||||
{
|
||||
|
@ -506,6 +528,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "c96fdcc6",
|
||||
"metadata": {},
|
||||
|
@ -527,6 +550,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "1deaf95f",
|
||||
"metadata": {},
|
||||
|
@ -548,6 +572,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "dcb8055f",
|
||||
"metadata": {},
|
||||
|
@ -566,6 +591,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "2654e72c",
|
||||
"metadata": {},
|
||||
|
@ -579,15 +605,14 @@
|
|||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"id": "9e42dfbc",
|
||||
"metadata": {
|
||||
"scrolled": true
|
||||
},
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"rest_client.get_json('/status')"
|
||||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "837e08b0",
|
||||
"metadata": {},
|
||||
|
@ -629,6 +654,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "edc4ee39",
|
||||
"metadata": {},
|
||||
|
@ -659,6 +685,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "b661b29f",
|
||||
"metadata": {},
|
||||
|
@ -677,6 +704,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "5e789ca7",
|
||||
"metadata": {},
|
||||
|
@ -697,6 +725,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "7b9dc7e3",
|
||||
"metadata": {},
|
||||
|
@ -715,6 +744,7 @@
|
|||
]
|
||||
},
|
||||
{
|
||||
"attachments": {},
|
||||
"cell_type": "markdown",
|
||||
"id": "ddaf0dc2",
|
||||
"metadata": {},
|
||||
|
@ -741,7 +771,7 @@
|
|||
"name": "python",
|
||||
"nbconvert_exporter": "python",
|
||||
"pygments_lexer": "ipython3",
|
||||
"version": "3.8.16"
|
||||
"version": "3.11.4"
|
||||
}
|
||||
},
|
||||
"nbformat": 4,
|
|
@ -0,0 +1,537 @@
|
|||
{
|
||||
"cells": [
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"Ingest and query data from Apache Kafka\n",
|
||||
"\n",
|
||||
"<!--\n",
|
||||
" ~ Licensed to the Apache Software Foundation (ASF) under one\n",
|
||||
" ~ or more contributor license agreements. See the NOTICE file\n",
|
||||
" ~ distributed with this work for additional information\n",
|
||||
" ~ regarding copyright ownership. The ASF licenses this file\n",
|
||||
" ~ to you under the Apache License, Version 2.0 (the\n",
|
||||
" ~ \"License\"); you may not use this file except in compliance\n",
|
||||
" ~ with the License. You may obtain a copy of the License at\n",
|
||||
" ~\n",
|
||||
" ~ http://www.apache.org/licenses/LICENSE-2.0\n",
|
||||
" ~\n",
|
||||
" ~ Unless required by applicable law or agreed to in writing,\n",
|
||||
" ~ software distributed under the License is distributed on an\n",
|
||||
" ~ \"AS IS\" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY\n",
|
||||
" ~ KIND, either express or implied. See the License for the\n",
|
||||
" ~ specific language governing permissions and limitations\n",
|
||||
" ~ under the License.\n",
|
||||
" -->\n",
|
||||
"\n",
|
||||
"This tutorial introduces you to streaming ingestion in Apache Druid using the Apache Kafka event streaming platform.\n",
|
||||
"Follow along to learn how to create and load data into a Kafka topic, start ingesting data from the topic into Druid, and query results over time. This tutorial assumes you have a basic understanding of Druid ingestion, querying, and API requests."
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"## Table of contents\n",
|
||||
"\n",
|
||||
"* [Prerequisites](#Prerequisites)\n",
|
||||
"* [Load Druid API client](#Load-Druid-API-client)\n",
|
||||
"* [Create Kafka topic](#Create-Kafka-topic)\n",
|
||||
"* [Load data into Kafka topic](#Load-data-into-Kafka-topic)\n",
|
||||
"* [Start Druid ingestion](#Start-Druid-ingestion)\n",
|
||||
"* [Query Druid datasource and visualize query results](#Query-Druid-datasource-and-visualize-query-results)\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",
|
||||
"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",
|
||||
"\n",
|
||||
"If you do not use the Docker Compose environment, you need the following:\n",
|
||||
"* A running Druid instance.\n",
|
||||
" * Update the `druid_host` variable to point to your Router endpoint. For example, `druid_host = \"http://localhost:8888\"`.\n",
|
||||
" * Update the `rest_client` variable to point to your Coordinator endpoint. For example, `\"http://localhost:8081\"`.\n",
|
||||
"* A running Kafka cluster.\n",
|
||||
" * Update the Kafka bootstrap servers to point to your servers. For example, `bootstrap_servers=[\"localhost:9092\"]`.\n",
|
||||
"* The following Python packages:\n",
|
||||
" * `druidapi`, a Python client for Apache Druid\n",
|
||||
" * `DruidDataDriver`, a data generator\n",
|
||||
" * `kafka`, a Python client for Apache Kafka\n",
|
||||
" * `pandas`, `matplotlib`, and `seaborn` for data visualization\n"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"## Load Druid API client"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"To start the tutorial, run the following cell. It imports the required Python packages and defines a variable for the Druid client, and another for the SQL client used to run SQL commands."
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"import druidapi\n",
|
||||
"import json\n",
|
||||
"\n",
|
||||
"# druid_host is the hostname and port for your Druid deployment. \n",
|
||||
"# In the Docker Compose tutorial environment, this is the Router\n",
|
||||
"# service running at \"http://router:8888\".\n",
|
||||
"# If you are not using the Docker Compose environment, edit the `druid_host`.\n",
|
||||
"\n",
|
||||
"druid_host = \"http://router:8888\"\n",
|
||||
"druid_host\n",
|
||||
"\n",
|
||||
"druid = druidapi.jupyter_client(druid_host)\n",
|
||||
"display = druid.display\n",
|
||||
"sql_client = druid.sql\n",
|
||||
"\n",
|
||||
"# Create a rest client for native JSON ingestion for streaming data\n",
|
||||
"rest_client = druidapi.rest.DruidRestClient(\"http://coordinator:8081\")"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"## Create Kafka topic"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"This notebook relies on the Python client for the Apache Kafka. Import the Kafka producer and consumer modules, then create a Kafka client. You use the Kafka producer to create and publish records to a new topic named `social_media`."
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"from kafka import KafkaProducer\n",
|
||||
"from kafka import KafkaConsumer\n",
|
||||
"\n",
|
||||
"# Kafka runs on kafka:9092 in multi-container tutorial application\n",
|
||||
"producer = KafkaProducer(bootstrap_servers='kafka:9092')\n",
|
||||
"topic_name = \"social_media\""
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"Create the `social_media` topic and send a sample event. The `send()` command returns a metadata descriptor for the record."
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"event = {\n",
|
||||
" \"__time\": \"2023-01-03T16:40:21.501\",\n",
|
||||
" \"username\": \"willow\",\n",
|
||||
" \"post_title\": \"This title is required\",\n",
|
||||
" \"views\": 15284,\n",
|
||||
" \"upvotes\": 124,\n",
|
||||
" \"comments\": 21,\n",
|
||||
" \"edited\": \"True\"\n",
|
||||
"}\n",
|
||||
"\n",
|
||||
"producer.send(topic_name, json.dumps(event).encode('utf-8'))"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"To verify that the Kafka topic stored the event, create a consumer client to read records from the Kafka cluster, and get the next (only) message:"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"consumer = KafkaConsumer(topic_name, bootstrap_servers=['kafka:9092'], auto_offset_reset='earliest',\n",
|
||||
" enable_auto_commit=True)\n",
|
||||
"\n",
|
||||
"print(next(consumer).value.decode('utf-8'))"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"## Load data into Kafka topic"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"Instead of manually creating events to send to the Kafka topic, use a data generator to simulate a continuous data stream. This tutorial makes use of Druid Data Driver to simulate a continuous data stream into the `social_media` Kafka topic. To learn more about the Druid Data Driver, see the Druid Summit talk, [Generating Time centric Data for Apache Druid](https://www.youtube.com/watch?v=3zAOeLe3iAo).\n",
|
||||
"\n",
|
||||
"In this notebook, you use a background process to continuously load data into the Kafka topic.\n",
|
||||
"This allows you to keep executing commands in this notebook while data is constantly being streamed into the topic."
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"Run the following cells to load sample data into the `social_media` Kafka topic:"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"import multiprocessing as mp\n",
|
||||
"from datetime import datetime\n",
|
||||
"import DruidDataDriver"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"def run_driver():\n",
|
||||
" DruidDataDriver.simulate(\"kafka_docker_config.json\", None, None, \"REAL\", datetime.now())\n",
|
||||
" \n",
|
||||
"mp.set_start_method('fork')\n",
|
||||
"ps = mp.Process(target=run_driver)\n",
|
||||
"ps.start()"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"## Start Druid ingestion"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"Now that you have a new Kafka topic and data being streamed into the topic, you ingest the data into Druid by submitting a Kafka ingestion spec.\n",
|
||||
"The ingestion spec describes the following:\n",
|
||||
"* where to source the data to ingest (in `spec > ioConfig`),\n",
|
||||
"* the datasource to ingest data into (in `spec > dataSchema > dataSource`), and\n",
|
||||
"* what the data looks like (in `spec > dataSchema > dimensionsSpec`).\n",
|
||||
"\n",
|
||||
"Other properties control how Druid aggregates and stores data. For more information, see the Druid documenation:\n",
|
||||
"* [Apache Kafka ingestion](https://druid.apache.org/docs/latest/development/extensions-core/kafka-ingestion.html)\n",
|
||||
"* [Ingestion spec reference](https://druid.apache.org/docs/latest/ingestion/ingestion-spec.html)\n",
|
||||
"\n",
|
||||
"Run the following cells to define and view the Kafka ingestion spec."
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"kafka_ingestion_spec = \"{\\\"type\\\": \\\"kafka\\\",\\\"spec\\\": {\\\"ioConfig\\\": {\\\"type\\\": \\\"kafka\\\",\\\"consumerProperties\\\": {\\\"bootstrap.servers\\\": \\\"kafka:9092\\\"},\\\"topic\\\": \\\"social_media\\\",\\\"inputFormat\\\": {\\\"type\\\": \\\"json\\\"},\\\"useEarliestOffset\\\": true},\\\"tuningConfig\\\": {\\\"type\\\": \\\"kafka\\\"},\\\"dataSchema\\\": {\\\"dataSource\\\": \\\"social_media\\\",\\\"timestampSpec\\\": {\\\"column\\\": \\\"__time\\\",\\\"format\\\": \\\"iso\\\"},\\\"dimensionsSpec\\\": {\\\"dimensions\\\": [\\\"username\\\",\\\"post_title\\\",{\\\"type\\\": \\\"long\\\",\\\"name\\\": \\\"views\\\"},{\\\"type\\\": \\\"long\\\",\\\"name\\\": \\\"upvotes\\\"},{\\\"type\\\": \\\"long\\\",\\\"name\\\": \\\"comments\\\"},\\\"edited\\\"]},\\\"granularitySpec\\\": {\\\"queryGranularity\\\": \\\"none\\\",\\\"rollup\\\": false,\\\"segmentGranularity\\\": \\\"hour\\\"}}}}\""
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"print(json.dumps(json.loads(kafka_ingestion_spec), indent=4))"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"Send the spec to Druid to start the streaming ingestion from Kafka:"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"headers = {\n",
|
||||
" 'Content-Type': 'application/json'\n",
|
||||
"}\n",
|
||||
"\n",
|
||||
"rest_client.post(\"/druid/indexer/v1/supervisor\", kafka_ingestion_spec, headers=headers)"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"A `200` response indicates that the request was successful. You can view the running ingestion task and the new datasource in the web console at http://localhost:8888/unified-console.html."
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"## Query Druid datasource and visualize query results"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"You can now query the new datasource called `social_media`. In this section, you also visualize query results using the Matplotlib and Seaborn visualization libraries. Run the following cell import these packages."
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"import pandas as pd\n",
|
||||
"import matplotlib\n",
|
||||
"import matplotlib.pyplot as plt\n",
|
||||
"import seaborn as sns"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"Run a simple query to view a subset of rows from the new datasource:"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"sql = '''\n",
|
||||
"SELECT * FROM social_media LIMIT 5\n",
|
||||
"'''\n",
|
||||
"display.sql(sql)"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"In this social media scenario, each incoming event represents a post on social media, for which you collect the timestamp, username, and post metadata. You are interested in analyzing the total number of upvotes for all posts, compared between users. Preview this data with the following query:"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"sql = '''\n",
|
||||
"SELECT\n",
|
||||
" COUNT(post_title) as num_posts,\n",
|
||||
" SUM(upvotes) as total_upvotes,\n",
|
||||
" username\n",
|
||||
"FROM social_media\n",
|
||||
"GROUP BY username\n",
|
||||
"ORDER BY num_posts\n",
|
||||
"'''\n",
|
||||
"\n",
|
||||
"response = sql_client.sql_query(sql)\n",
|
||||
"response.show()"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"Visualize the total number of upvotes per user using a line plot. You sort the results by username before plotting because the order of users may vary as new results arrive."
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"df = pd.DataFrame(response.json)\n",
|
||||
"df = df.sort_values('username')\n",
|
||||
"\n",
|
||||
"df.plot(x='username', y='total_upvotes', marker='o')\n",
|
||||
"plt.xticks(rotation=45, ha='right')\n",
|
||||
"plt.ylabel(\"Total number of upvotes\")\n",
|
||||
"plt.gca().get_legend().remove()\n",
|
||||
"plt.show()"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"The total number of upvotes likely depends on the total number of posts created per user. To better assess the relative impact per user, you compare the total number of upvotes (line plot) with the total number of posts."
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"matplotlib.rc_file_defaults()\n",
|
||||
"ax1 = sns.set_style(style=None, rc=None )\n",
|
||||
"\n",
|
||||
"fig, ax1 = plt.subplots()\n",
|
||||
"plt.xticks(rotation=45, ha='right')\n",
|
||||
"\n",
|
||||
"\n",
|
||||
"sns.lineplot(\n",
|
||||
" data=df, x='username', y='total_upvotes',\n",
|
||||
" marker='o', ax=ax1, label=\"Sum of upvotes\")\n",
|
||||
"ax1.get_legend().remove()\n",
|
||||
"\n",
|
||||
"ax2 = ax1.twinx()\n",
|
||||
"sns.barplot(data=df, x='username', y='num_posts',\n",
|
||||
" order=df['username'], alpha=0.5, ax=ax2, log=True,\n",
|
||||
" color=\"orange\", label=\"Number of posts\")\n",
|
||||
"\n",
|
||||
"\n",
|
||||
"# ask matplotlib for the plotted objects and their labels\n",
|
||||
"lines, labels = ax1.get_legend_handles_labels()\n",
|
||||
"lines2, labels2 = ax2.get_legend_handles_labels()\n",
|
||||
"ax2.legend(lines + lines2, labels + labels2, bbox_to_anchor=(1.55, 1))"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"You should see a correlation between total number of upvotes and total number of posts. In order to track user impact on a more equal footing, normalize the total number of upvotes relative to the total number of posts, and plot the result:"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"df['upvotes_normalized'] = df['total_upvotes']/df['num_posts']\n",
|
||||
"\n",
|
||||
"df.plot(x='username', y='upvotes_normalized', marker='o', color='green')\n",
|
||||
"plt.xticks(rotation=45, ha='right')\n",
|
||||
"plt.ylabel(\"Number of upvotes (normalized)\")\n",
|
||||
"plt.gca().get_legend().remove()\n",
|
||||
"plt.show()"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"You've been working with data taken at a single snapshot in time from when you ran the last query. Run the same query again, and store the output in `response2`, which you will compare with the previous results:"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"response2 = sql_client.sql_query(sql)\n",
|
||||
"response2.show()"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"Normalizing the data also helps you evaluate trends over time more consistently on the same plot axes. Plot the normalized data again, this time alongside the results from the previous snapshot:"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "code",
|
||||
"execution_count": null,
|
||||
"metadata": {},
|
||||
"outputs": [],
|
||||
"source": [
|
||||
"df2 = pd.DataFrame(response2.json)\n",
|
||||
"df2 = df2.sort_values('username')\n",
|
||||
"df2['upvotes_normalized'] = df2['total_upvotes']/df2['num_posts']\n",
|
||||
"\n",
|
||||
"ax = df.plot(x='username', y='upvotes_normalized', marker='o', color='green', label=\"Time 1\")\n",
|
||||
"df2.plot(x='username', y='upvotes_normalized', marker='o', color='purple', ax=ax, label=\"Time 2\")\n",
|
||||
"plt.xticks(rotation=45, ha='right')\n",
|
||||
"plt.ylabel(\"Number of upvotes (normalized)\")\n",
|
||||
"plt.show()"
|
||||
]
|
||||
},
|
||||
{
|
||||
"cell_type": "markdown",
|
||||
"metadata": {},
|
||||
"source": [
|
||||
"This plot shows how some users maintain relatively consistent social media impact between the two query snapshots, whereas other users grow or decline in their influence.\n",
|
||||
"\n",
|
||||
"## Learn more\n",
|
||||
"\n",
|
||||
"This tutorial showed you how to create a Kafka topic using a Python client for Kafka, send a simulated stream of data to Kafka using a data generator, and query and visualize results over time. For more information, see the following resources:\n",
|
||||
"\n",
|
||||
"* [Apache Kafka ingestion](https://druid.apache.org/docs/latest/development/extensions-core/kafka-ingestion.html)\n",
|
||||
"* [Querying data](https://druid.apache.org/docs/latest/tutorials/tutorial-query.html)\n",
|
||||
"* [Tutorial: Run with Docker](https://druid.apache.org/docs/latest/tutorials/docker.html)"
|
||||
]
|
||||
}
|
||||
],
|
||||
"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"
|
||||
},
|
||||
"vscode": {
|
||||
"interpreter": {
|
||||
"hash": "a4289e5b8bae5973a6609d90f7bc464162478362b9a770893a3c5c597b0b36e7"
|
||||
}
|
||||
}
|
||||
},
|
||||
"nbformat": 4,
|
||||
"nbformat_minor": 4
|
||||
}
|
File diff suppressed because it is too large
Load Diff
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue