Merge branch 'master' into 6088-Time-Ordering-On-Scans-N-Way-Merge
|
@ -46,6 +46,7 @@
|
|||
<inspection_tool class="EqualsUsesNonFinalVariable" enabled="true" level="WARNING" enabled_by_default="true" />
|
||||
<inspection_tool class="EqualsWhichDoesntCheckParameterClass" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
<inspection_tool class="EqualsWithItself" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
<inspection_tool class="FieldAccessNotGuarded" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
<inspection_tool class="FieldCanBeLocal" enabled="true" level="WARNING" enabled_by_default="true">
|
||||
<option name="EXCLUDE_ANNOS">
|
||||
<value>
|
||||
|
@ -119,7 +120,7 @@
|
|||
<inspection_tool class="NumberEquality" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
<inspection_tool class="ObjectEquality" enabled="true" level="WARNING" enabled_by_default="true">
|
||||
<option name="m_ignoreEnums" value="true" />
|
||||
<option name="m_ignoreClassObjects" value="false" />
|
||||
<option name="m_ignoreClassObjects" value="true" />
|
||||
<option name="m_ignorePrivateConstructors" value="false" />
|
||||
</inspection_tool>
|
||||
<inspection_tool class="ObjectEqualsNull" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
|
@ -356,6 +357,5 @@
|
|||
<option name="ADD_SERVLET_TO_ENTRIES" value="true" />
|
||||
<option name="ADD_NONJAVA_TO_ENTRIES" value="true" />
|
||||
</inspection_tool>
|
||||
<inspection_tool class="FieldAccessNotGuarded" enabled="true" level="ERROR" enabled_by_default="true" />
|
||||
</profile>
|
||||
</component>
|
|
@ -1,7 +1,6 @@
|
|||
<component name="DependencyValidationManager">
|
||||
<scope name="UnusedInspectionsScope" pattern="src[druid-processing]:*..*" />
|
||||
<scope name="UnusedInspectionsScope" pattern="src[java-util]:*..*" />
|
||||
<scope name="UnusedInspectionsScope" pattern="src[druid-common]:*..*" />
|
||||
<scope name="UnusedInspectionsScope" pattern="src[druid-core]:*..*" />
|
||||
<scope name="UnusedInspectionsScope" pattern="src[extendedset]:*..*" />
|
||||
<scope name="UnusedInspectionsScope" pattern="src[druid-indexing-service]:*..*" />
|
||||
</component>
|
|
@ -16,8 +16,17 @@
|
|||
~ specific language governing permissions and limitations
|
||||
~ under the License.
|
||||
-->
|
||||
|
||||
## Overview
|
||||
TeamCity is a continuous integration and deployment server responsible for
|
||||
static analysis of Druid source code. Each Github PR request for
|
||||
[Druid](https://teamcity.jetbrains.com/project.html?projectId=OpenSourceProjects_Druid)
|
||||
is checked by TeamCity automatically.
|
||||
|
||||
## Becoming a project administrator
|
||||
## Login
|
||||
One can log in to TeamCity either via credentials or as a guest to check static analysis result of any PR.
|
||||
|
||||
## Becoming a Project Administrator
|
||||
Druid committers shall obtain a status of a [Druid project](
|
||||
https://teamcity.jetbrains.com/project.html?projectId=OpenSourceProjects_Druid)
|
||||
administrator. First, the Druid committer needs to log in teamcity.jetbrains.com using his Github account.
|
||||
|
@ -25,10 +34,10 @@ Then, somebody who is already a project administrator needs to do the following:
|
|||
|
||||
1. Follow the "Administration" link in the top-right corner of the page
|
||||
2. Follow the "Users" link in the "User Management" section in the menu on the left
|
||||
3. Type the comitter's Github handle in the "Find users" text input, press "Filter"
|
||||
4. Select the comitter
|
||||
3. Type the committer's Github handle in the "Find users" text input, press "Filter"
|
||||
4. Select the committer
|
||||
5. Press the "Assign roles" button in the bottom of the page
|
||||
6. Select "Role: Project administrator" and "Scope: Open-source project -> Druid" in the inputs, press "Assign"
|
||||
|
||||
## Restarting a build
|
||||
## Restarting a Build
|
||||
A project administrator could restart a build by pressing the "Run" button on the build page.
|
||||
|
|
|
@ -23,9 +23,9 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.locks.AbstractQueuedSynchronizer;
|
||||
|
||||
/**
|
||||
* A synchronization tool for lifecycled objects (see {@link org.apache.druid.java.util.common.lifecycle.Lifecycle}, that need
|
||||
* happens-before between start() and other methods and/or to check that the object was successfully started in other
|
||||
* methods.
|
||||
* A synchronization tool for lifecycled objects (see {@link org.apache.druid.java.util.common.lifecycle.Lifecycle},
|
||||
* that need happens-before between start() and other methods and/or to check that the object was successfully started
|
||||
* in other methods.
|
||||
*
|
||||
* Guarantees in terms of JMM: happens-before between {@link #exitStart()} and {@link #awaitStarted()},
|
||||
* exitStart() and {@link #canStop()}, if it returns {@code true}.
|
||||
|
|
|
@ -0,0 +1,59 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.concurrent;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.LockSupport;
|
||||
|
||||
public final class Threads
|
||||
{
|
||||
|
||||
/**
|
||||
* Equivalent of {@link Thread#sleep(long)} with arguments and semantics of timed wait methods in classes from {@link
|
||||
* java.util.concurrent} (like {@link java.util.concurrent.Semaphore#tryAcquire(long, TimeUnit)},
|
||||
* {@link java.util.concurrent.locks.Lock#tryLock(long, TimeUnit)}, etc.): if the sleepTime argument is negative or
|
||||
* zero, the method returns immediately. {@link Thread#sleep}, on the contrary, throws an IllegalArgumentException if
|
||||
* the argument is negative and attempts to unschedule the thread if the argument is zero.
|
||||
*
|
||||
* @throws InterruptedException if the current thread is interrupted when this method is called or during sleeping.
|
||||
*/
|
||||
public static void sleepFor(long sleepTime, TimeUnit unit) throws InterruptedException
|
||||
{
|
||||
if (Thread.interrupted()) {
|
||||
throw new InterruptedException();
|
||||
}
|
||||
if (sleepTime <= 0) {
|
||||
return;
|
||||
}
|
||||
long sleepTimeLimitNanos = System.nanoTime() + unit.toNanos(sleepTime);
|
||||
while (true) {
|
||||
long sleepTimeoutNanos = sleepTimeLimitNanos - System.nanoTime();
|
||||
if (sleepTimeoutNanos <= 0) {
|
||||
return;
|
||||
}
|
||||
LockSupport.parkNanos(sleepTimeoutNanos);
|
||||
if (Thread.interrupted()) {
|
||||
throw new InterruptedException();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Threads() {}
|
||||
}
|
|
@ -23,19 +23,24 @@ import org.apache.druid.guice.annotations.ExtensionPoint;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* This is an interface that holds onto the stream of incoming data. Realtime data ingestion is built around this
|
||||
* abstraction. In order to add a new type of source for realtime data ingestion, all you need to do is implement
|
||||
* one of these and register it with the Main.
|
||||
* abstraction.
|
||||
*
|
||||
* This object acts a lot like an Iterator, but it doesn't extend the Iterator interface because it extends
|
||||
* Closeable and it is very important that the close() method doesn't get forgotten, which is easy to do if this
|
||||
* gets passed around as an Iterator.
|
||||
* <p>
|
||||
* The implementation of this interface only needs to be minimally thread-safe. The three methods ##hasMore(),
|
||||
* ##nextRow() and ##commit() are all called from the same thread. ##commit(), however, returns a callback
|
||||
* which will be called on another thread, so the operations inside of that callback must be thread-safe.
|
||||
* Closeable and it is very important that the {@link #close()} method doesn't get forgotten, which is easy to do if
|
||||
* this gets passed around as an Iterator. Note that {@link #close()} doesn't cut the stream of rows for Firehose users
|
||||
* immediately, but rather stops the supply of new rows into internal buffers. {@link #hasMore()} and {@link #nextRow()}
|
||||
* are expected to operate for some time after (or concurrently with) {@link #close()} until the buffered events (if
|
||||
* any) run out.
|
||||
*
|
||||
* Concurrency:
|
||||
* The three methods {@link #hasMore()}, {@link #nextRow()} and {@link #commit()} are all called from the same thread.
|
||||
* {@link #commit()}, however, returns a callback which will be called on another thread. {@link #close()} might be
|
||||
* called concurrently from a thread different from the thread calling {@link #hasMore()}, {@link #nextRow()} and {@link
|
||||
* #commit()}.
|
||||
* </p>
|
||||
*/
|
||||
@ExtensionPoint
|
||||
|
@ -43,8 +48,8 @@ public interface Firehose extends Closeable
|
|||
{
|
||||
/**
|
||||
* Returns whether there are more rows to process. This is used to indicate that another item is immediately
|
||||
* available via ##nextRow(). Thus, if the stream is still available but there are no new messages on it, this call
|
||||
* should block until a new message is available.
|
||||
* available via {@link #nextRow()}. Thus, if the stream is still available but there are no new messages on it, this
|
||||
* call should block until a new message is available.
|
||||
*
|
||||
* If something happens such that the stream is no longer available, this should return false.
|
||||
*
|
||||
|
@ -77,8 +82,22 @@ public interface Firehose extends Closeable
|
|||
* A simple implementation of this interface might do nothing when run() is called
|
||||
* (in which case the same do-nothing instance can be returned every time), or
|
||||
* a more complex implementation might clean up temporary resources that are no longer needed
|
||||
* because of InputRows delivered by prior calls to ##nextRow().
|
||||
* because of InputRows delivered by prior calls to {@link #nextRow()}.
|
||||
* </p>
|
||||
*/
|
||||
Runnable commit();
|
||||
|
||||
/**
|
||||
* Closes the "ingestion side" of the Firehose, potentially concurrently with calls to {@link #hasMore()}, {@link
|
||||
* #nextRow()} and {@link #commit()} being made from a different thread. {@link #hasMore()} and {@link #nextRow()}
|
||||
* continue to work after close(), but since the ingestion side is closed rows will eventually run out.
|
||||
*
|
||||
* The effects of calling run() on the {@link Runnable} object returned from {@link #commit()} (in other words,
|
||||
* doing the commit) concurrently or after close() are unspecified: commit may not be performed silently (that is,
|
||||
* run() call completes without an Exception, but the commit is not actually done), or a error may result. Note that
|
||||
* {@link #commit()} method itself can be called concurrently with close(), but it doesn't make much sense, because
|
||||
* run() on the returned Runnable then can't be called.
|
||||
*/
|
||||
@Override
|
||||
void close() throws IOException;
|
||||
}
|
||||
|
|
|
@ -42,9 +42,10 @@ import java.io.Closeable;
|
|||
* Closeable and it is very important that the close() method doesn't get forgotten, which is easy to do if this
|
||||
* gets passed around as an Iterator.
|
||||
*
|
||||
* The implementation of this interface only needs to be minimally thread-safe. The methods ##start(), ##advance(),
|
||||
* ##currRow() and ##makeCommitter() are all called from the same thread. ##makeCommitter(), however, returns a callback
|
||||
* which will be called on another thread, so the operations inside of that callback must be thread-safe.
|
||||
* The implementation of this interface only needs to be minimally thread-safe. The methods {@link #start()}, {@link
|
||||
* #advance()}, {@link #currRow()} and {@link #makeCommitter()} are all called from the same thread. {@link
|
||||
* #makeCommitter()}, however, returns a callback which will be called on another thread, so the operations inside of
|
||||
* that callback must be thread-safe.
|
||||
*/
|
||||
@ExtensionPoint
|
||||
public interface FirehoseV2 extends Closeable
|
||||
|
|
|
@ -21,10 +21,12 @@ package org.apache.druid.data.input;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.guice.annotations.PublicApi;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.joda.time.DateTime;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -43,7 +45,7 @@ public class MapBasedRow implements Row
|
|||
)
|
||||
{
|
||||
this.timestamp = timestamp;
|
||||
this.event = event;
|
||||
this.event = Collections.unmodifiableMap(Preconditions.checkNotNull(event, "event"));
|
||||
}
|
||||
|
||||
public MapBasedRow(
|
||||
|
|
|
@ -92,6 +92,14 @@ public final class Numbers
|
|||
}
|
||||
}
|
||||
|
||||
public static int toIntExact(long value, String error)
|
||||
{
|
||||
if ((int) value != value) {
|
||||
throw new ArithmeticException(error);
|
||||
}
|
||||
return (int) value;
|
||||
}
|
||||
|
||||
private Numbers()
|
||||
{
|
||||
}
|
||||
|
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.utils;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Methods in this class could have belonged to {@link org.apache.druid.java.util.common.io.Closer}, but not editing
|
||||
* that class to keep its source close to Guava source.
|
||||
*/
|
||||
public final class CloseableUtils
|
||||
{
|
||||
/**
|
||||
* Call method instead of code like
|
||||
*
|
||||
* first.close();
|
||||
* second.close();
|
||||
*
|
||||
* to have safety of {@link org.apache.druid.java.util.common.io.Closer}, but without associated boilerplate code
|
||||
* of creating a Closer and registering objects in it.
|
||||
*/
|
||||
public static void closeBoth(Closeable first, Closeable second) throws IOException
|
||||
{
|
||||
//noinspection EmptyTryBlock
|
||||
try (Closeable ignore1 = second; Closeable ignore2 = first) {
|
||||
// piggy-back try-with-resources semantics
|
||||
}
|
||||
}
|
||||
|
||||
private CloseableUtils() {}
|
||||
}
|
|
@ -22,10 +22,18 @@ package org.apache.druid.data.input;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class MapBasedRowTest
|
||||
{
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
|
||||
@Test
|
||||
public void testGetLongMetricFromString()
|
||||
{
|
||||
|
@ -50,4 +58,15 @@ public class MapBasedRowTest
|
|||
Assert.assertEquals(-9223372036854775807L, row.getMetric("k5"));
|
||||
Assert.assertEquals(9223372036854775802L, row.getMetric("k6"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testImmutability()
|
||||
{
|
||||
final Map<String, Object> event = new HashMap<>();
|
||||
event.put("k0", 1);
|
||||
event.put("k1", 2);
|
||||
final MapBasedRow row = new MapBasedRow(DateTimes.nowUtc(), event);
|
||||
expectedException.expect(UnsupportedOperationException.class);
|
||||
row.getEvent().put("k2", 3);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,4 +37,4 @@ fast in Druid, whereas updates of older data is higher latency. This is by desig
|
|||
and does not need to be updated too frequently. Kudu supports arbitrary primary keys with uniqueness constraints, and
|
||||
efficient lookup by ranges of those keys. Kudu chooses not to include the execution engine, but supports sufficient
|
||||
operations so as to allow node-local processing from the execution engines. This means that Kudu can support multiple frameworks on the same data (eg MR, Spark, and SQL).
|
||||
Druid includes its own query layer that allows it to push down aggregations and computations directly to data nodes for faster query processing.
|
||||
Druid includes its own query layer that allows it to push down aggregations and computations directly to data processes for faster query processing.
|
||||
|
|
|
@ -42,7 +42,7 @@ Druid’s write semantics are not as fluid and does not support full joins (we s
|
|||
|
||||
### Data distribution model
|
||||
|
||||
Druid’s data distribution is segment-based and leverages a highly available "deep" storage such as S3 or HDFS. Scaling up (or down) does not require massive copy actions or downtime; in fact, losing any number of Historical nodes does not result in data loss because new Historical nodes can always be brought up by reading data from "deep" storage.
|
||||
Druid’s data distribution is segment-based and leverages a highly available "deep" storage such as S3 or HDFS. Scaling up (or down) does not require massive copy actions or downtime; in fact, losing any number of Historical processes does not result in data loss because new Historical processes can always be brought up by reading data from "deep" storage.
|
||||
|
||||
To contrast, ParAccel’s data distribution model is hash-based. Expanding the cluster requires re-hashing the data across the nodes, making it difficult to perform without taking downtime. Amazon’s Redshift works around this issue with a multi-step process:
|
||||
|
||||
|
@ -52,7 +52,7 @@ To contrast, ParAccel’s data distribution model is hash-based. Expanding the c
|
|||
|
||||
### Replication strategy
|
||||
|
||||
Druid employs segment-level data distribution meaning that more nodes can be added and rebalanced without having to perform a staged swap. The replication strategy also makes all replicas available for querying. Replication is done automatically and without any impact to performance.
|
||||
Druid employs segment-level data distribution meaning that more processes can be added and rebalanced without having to perform a staged swap. The replication strategy also makes all replicas available for querying. Replication is done automatically and without any impact to performance.
|
||||
|
||||
ParAccel’s hash-based distribution generally means that replication is conducted via hot spares. This puts a numerical limit on the number of nodes you can lose without losing data, and this replication strategy often does not allow the hot spare to help share query load.
|
||||
|
||||
|
|
|
@ -51,7 +51,7 @@ This page documents all of the configuration properties for each Druid service t
|
|||
* [Master Server](#master-server)
|
||||
* [Coordinator](#coordinator)
|
||||
* [Static Configuration](#static-configuration)
|
||||
* [Node Config](#coordinator-node-config)
|
||||
* [Process Config](#coordinator-process-config)
|
||||
* [Coordinator Operation](#coordinator-operation)
|
||||
* [Segment Management](#segment-management)
|
||||
* [Metadata Retrieval](#metadata-retrieval)
|
||||
|
@ -60,34 +60,34 @@ This page documents all of the configuration properties for each Druid service t
|
|||
* [Compaction](#compaction-dynamic-configuration)
|
||||
* [Overlord](#overlord)
|
||||
* [Static Configuration](#overlord-static-configuration)
|
||||
* [Node Config](#overlord-node-config)
|
||||
* [Process Config](#overlord-process-config)
|
||||
* [Overlord Operations](#overlord-operations)
|
||||
* [Dynamic Configuration](#overlord-dynamic-configuration)
|
||||
* [Worker Select Strategy](#worker-select-strategy)
|
||||
* [Autoscaler](#autoscaler)
|
||||
* [Data Server](#data-server)
|
||||
* [MiddleManager & Peons](#middlemanager-and-peons)
|
||||
* [Node Config](#middlemanager-node-config)
|
||||
* [Process Config](#middlemanager-process-config)
|
||||
* [MiddleManager Configuration](#middlemanager-configuration)
|
||||
* [Peon Processing](#peon-processing)
|
||||
* [Peon Query Configuration](#peon-query-configuration)
|
||||
* [Caching](#peon-caching)
|
||||
* [Additional Peon Configuration](#additional-peon-configuration)
|
||||
* [Historical](#historical)
|
||||
* [Node Configuration](#historical-node-config)
|
||||
* [Process Configuration](#historical-process-config)
|
||||
* [General Configuration](#historical-general-configuration)
|
||||
* [Query Configs](#historical-query-configs)
|
||||
* [Caching](#historical-caching)
|
||||
* [Query Server](#query-server)
|
||||
* [Broker](#broker)
|
||||
* [Node Config](#broker-node-configs)
|
||||
* [Process Config](#broker-process-configs)
|
||||
* [Query Configuration](#broker-query-configuration)
|
||||
* [SQL](#sql)
|
||||
* [Caching](#broker-caching)
|
||||
* [Segment Discovery](#segment-discovery)
|
||||
* [Caching](#cache-configuration)
|
||||
* [General Query Configuration](#general-query-configuration)
|
||||
* [Realtime nodes (Deprecated)](#realtime-nodes)
|
||||
* [Realtime processes (Deprecated)](#realtime-processes)
|
||||
|
||||
## Recommended Configuration File Organization
|
||||
|
||||
|
@ -159,7 +159,7 @@ Many of Druid's external dependencies can be plugged in as modules. Extensions c
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.modules.excludeList`|A JSON array of canonical class names (e. g. `"org.apache.druid.somepackage.SomeModule"`) of module classes which shouldn't be loaded, even if they are found in extensions specified by `druid.extensions.loadList`, or in the list of core modules specified to be loaded on a particular Druid node type. Useful when some useful extension contains some module, which shouldn't be loaded on some Druid node type because some dependencies of that module couldn't be satisfied.|[]|
|
||||
|`druid.modules.excludeList`|A JSON array of canonical class names (e. g. `"org.apache.druid.somepackage.SomeModule"`) of module classes which shouldn't be loaded, even if they are found in extensions specified by `druid.extensions.loadList`, or in the list of core modules specified to be loaded on a particular Druid process type. Useful when some useful extension contains some module, which shouldn't be loaded on some Druid process type because some dependencies of that module couldn't be satisfied.|[]|
|
||||
|
||||
### Zookeeper
|
||||
We recommend just setting the base ZK path and the ZK service host, but all ZK paths that Druid uses can be overwritten to absolute paths.
|
||||
|
@ -187,11 +187,11 @@ Druid interacts with ZK through a set of standard path configurations. We recomm
|
|||
|--------|-----------|-------|
|
||||
|`druid.zk.paths.base`|Base Zookeeper path.|`/druid`|
|
||||
|`druid.zk.paths.propertiesPath`|Zookeeper properties path.|`${druid.zk.paths.base}/properties`|
|
||||
|`druid.zk.paths.announcementsPath`|Druid node announcement path.|`${druid.zk.paths.base}/announcements`|
|
||||
|`druid.zk.paths.liveSegmentsPath`|Current path for where Druid nodes announce their segments.|`${druid.zk.paths.base}/segments`|
|
||||
|`druid.zk.paths.loadQueuePath`|Entries here cause Historical nodes to load and drop segments.|`${druid.zk.paths.base}/loadQueue`|
|
||||
|`druid.zk.paths.announcementsPath`|Druid process announcement path.|`${druid.zk.paths.base}/announcements`|
|
||||
|`druid.zk.paths.liveSegmentsPath`|Current path for where Druid processes announce their segments.|`${druid.zk.paths.base}/segments`|
|
||||
|`druid.zk.paths.loadQueuePath`|Entries here cause Historical processes to load and drop segments.|`${druid.zk.paths.base}/loadQueue`|
|
||||
|`druid.zk.paths.coordinatorPath`|Used by the Coordinator for leader election.|`${druid.zk.paths.base}/coordinator`|
|
||||
|`druid.zk.paths.servedSegmentsPath`|@Deprecated. Legacy path for where Druid nodes announce their segments.|`${druid.zk.paths.base}/servedSegments`|
|
||||
|`druid.zk.paths.servedSegmentsPath`|@Deprecated. Legacy path for where Druid processes announce their segments.|`${druid.zk.paths.base}/servedSegments`|
|
||||
|
||||
The indexing service also uses its own set of paths. These configs can be included in the common configuration.
|
||||
|
||||
|
@ -238,7 +238,7 @@ Note that `druid.zk.service.host` is used as a backup in case an Exhibitor insta
|
|||
|`druid.enableTlsPort`|Enable/Disable HTTPS connector.|`false`|
|
||||
|
||||
Although not recommended but both HTTP and HTTPS connectors can be enabled at a time and respective ports are configurable using `druid.plaintextPort`
|
||||
and `druid.tlsPort` properties on each node. Please see `Configuration` section of individual nodes to check the valid and default values for these ports.
|
||||
and `druid.tlsPort` properties on each process. Please see `Configuration` section of individual processes to check the valid and default values for these ports.
|
||||
|
||||
#### Jetty Server TLS Configuration
|
||||
|
||||
|
@ -299,7 +299,7 @@ For configuration options for specific auth extensions, please refer to the exte
|
|||
|
||||
### Startup Logging
|
||||
|
||||
All nodes can log debugging information on startup.
|
||||
All processes can log debugging information on startup.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|
@ -310,7 +310,7 @@ Note that some sensitive information may be logged if these settings are enabled
|
|||
|
||||
### Request Logging
|
||||
|
||||
All nodes that can serve queries can also log the query requests they see. Broker nodes can additionally log the SQL requests (both from HTTP and JDBC) they see.
|
||||
All processes that can serve queries can also log the query requests they see. Broker processes can additionally log the SQL requests (both from HTTP and JDBC) they see.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|
@ -324,7 +324,7 @@ Daily request logs are stored on disk.
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.request.logging.dir`|Historical, Realtime and Broker nodes maintain request logs of all of the requests they get (interacton is via POST, so normal request logs don’t generally capture information about the actual query), this specifies the directory to store the request logs in|none|
|
||||
|`druid.request.logging.dir`|Historical, Realtime and Broker processes maintain request logs of all of the requests they get (interacton is via POST, so normal request logs don’t generally capture information about the actual query), this specifies the directory to store the request logs in|none|
|
||||
|`druid.request.logging.filePattern`|[Joda datetime format](http://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html) for each file|"yyyy-MM-dd'.log'"|
|
||||
|
||||
The format of request logs is TSV, one line per requests, with five fields: timestamp, remote\_addr, native\_query, query\_context, sql\_query.
|
||||
|
@ -397,25 +397,25 @@ Switching Request Logger routes native query's request logs to one request logge
|
|||
|
||||
### Enabling Metrics
|
||||
|
||||
Druid nodes periodically emit metrics and different metrics monitors can be included. Each node can overwrite the default list of monitors.
|
||||
Druid processes periodically emit metrics and different metrics monitors can be included. Each process can overwrite the default list of monitors.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.monitoring.emissionPeriod`|How often metrics are emitted.|PT1M|
|
||||
|`druid.monitoring.monitors`|Sets list of Druid monitors used by a node. See below for names and more information. For example, you can specify monitors for a Broker with `druid.monitoring.monitors=["org.apache.druid.java.util.metrics.SysMonitor","org.apache.druid.java.util.metrics.JvmMonitor"]`.|none (no monitors)|
|
||||
|`druid.monitoring.monitors`|Sets list of Druid monitors used by a process. See below for names and more information. For example, you can specify monitors for a Broker with `druid.monitoring.monitors=["org.apache.druid.java.util.metrics.SysMonitor","org.apache.druid.java.util.metrics.JvmMonitor"]`.|none (no monitors)|
|
||||
|
||||
The following monitors are available:
|
||||
|
||||
|Name|Description|
|
||||
|----|-----------|
|
||||
|`org.apache.druid.client.cache.CacheMonitor`|Emits metrics (to logs) about the segment results cache for Historical and Broker nodes. Reports typical cache statistics include hits, misses, rates, and size (bytes and number of entries), as well as timeouts and and errors.|
|
||||
|`org.apache.druid.client.cache.CacheMonitor`|Emits metrics (to logs) about the segment results cache for Historical and Broker processes. Reports typical cache statistics include hits, misses, rates, and size (bytes and number of entries), as well as timeouts and and errors.|
|
||||
|`org.apache.druid.java.util.metrics.SysMonitor`|This uses the [SIGAR library](https://github.com/hyperic/sigar) to report on various system activities and statuses.|
|
||||
|`org.apache.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical nodes.|
|
||||
|`org.apache.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical processes.|
|
||||
|`org.apache.druid.java.util.metrics.JvmMonitor`|Reports various JVM-related statistics.|
|
||||
|`org.apache.druid.java.util.metrics.JvmCpuMonitor`|Reports statistics of CPU consumption by the JVM.|
|
||||
|`org.apache.druid.java.util.metrics.CpuAcctDeltaMonitor`|Reports consumed CPU as per the cpuacct cgroup.|
|
||||
|`org.apache.druid.java.util.metrics.JvmThreadsMonitor`|Reports Thread statistics in the JVM, like numbers of total, daemon, started, died threads.|
|
||||
|`org.apache.druid.segment.realtime.RealtimeMetricsMonitor`|Reports statistics on Realtime nodes.|
|
||||
|`org.apache.druid.segment.realtime.RealtimeMetricsMonitor`|Reports statistics on Realtime processes.|
|
||||
|`org.apache.druid.server.metrics.EventReceiverFirehoseMonitor`|Reports how many events have been queued in the EventReceiverFirehose.|
|
||||
|`org.apache.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted.|
|
||||
|`org.apache.druid.server.emitter.HttpEmittingMonitor`|Reports internal metrics of `http` or `parametrized` emitter (see below). Must not be used with another emitter type. See the description of the metrics here: https://github.com/apache/incubator-druid/pull/4973.|
|
||||
|
@ -452,7 +452,7 @@ The Druid servers [emit various metrics](../operations/metrics.html) and alerts
|
|||
|
||||
#### Http Emitter Module TLS Overrides
|
||||
|
||||
When emitting events to a TLS-enabled receiver, the Http Emitter will by default use an SSLContext obtained via the process described at [Druid's internal communication over TLS](../operations/tls-support.html#druids-internal-communication-over-tls), i.e., the same SSLContext that would be used for internal communications between Druid nodes.
|
||||
When emitting events to a TLS-enabled receiver, the Http Emitter will by default use an SSLContext obtained via the process described at [Druid's internal communication over TLS](../operations/tls-support.html#druids-internal-communication-over-tls), i.e., the same SSLContext that would be used for internal communications between Druid processes.
|
||||
|
||||
In some use cases it may be desirable to have the Http Emitter use its own separate truststore configuration. For example, there may be organizational policies that prevent the TLS-enabled metrics receiver's certificate from being added to the same truststore used by Druid's internal HTTP client.
|
||||
|
||||
|
@ -492,7 +492,7 @@ To use graphite as emitter set `druid.emitter=graphite`. For configuration detai
|
|||
|
||||
### Metadata Storage
|
||||
|
||||
These properties specify the jdbc connection and other configuration around the metadata storage. The only processes that connect to the metadata storage with these properties are the [Coordinator](../design/coordinator.html), [Overlord](../design/overlord.html) and [Realtime Nodes](../design/realtime.html).
|
||||
These properties specify the jdbc connection and other configuration around the metadata storage. The only processes that connect to the metadata storage with these properties are the [Coordinator](../design/coordinator.html), [Overlord](../design/overlord.html) and [Realtime Processes](../design/realtime.html).
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|
@ -576,7 +576,7 @@ If you are running the indexing service in remote mode, the task logs must be st
|
|||
|`druid.indexer.logs.type`|Choices:noop, s3, azure, google, hdfs, file. Where to store task logs|file|
|
||||
|
||||
You can also configure the Overlord to automatically retain the task logs in log directory and entries in task-related metadata storage tables only for last x milliseconds by configuring following additional properties.
|
||||
Caution: Automatic log file deletion typically works based on log file modification timestamp on the backing store, so large clock skews between druid nodes and backing store nodes might result in un-intended behavior.
|
||||
Caution: Automatic log file deletion typically works based on log file modification timestamp on the backing store, so large clock skews between druid processes and backing store nodes might result in un-intended behavior.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|
@ -636,16 +636,16 @@ This config is used to find the [Overlord](../design/overlord.html) using Curato
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.selectors.indexing.serviceName`|The druid.service name of the Overlord node. To start the Overlord with a different name, set it with this property. |druid/overlord|
|
||||
|`druid.selectors.indexing.serviceName`|The druid.service name of the Overlord process. To start the Overlord with a different name, set it with this property. |druid/overlord|
|
||||
|
||||
|
||||
### Coordinator Discovery
|
||||
|
||||
This config is used to find the [Coordinator](../design/coordinator.html) using Curator service discovery. This config is used by the realtime indexing nodes to get information about the segments loaded in the cluster.
|
||||
This config is used to find the [Coordinator](../design/coordinator.html) using Curator service discovery. This config is used by the realtime indexing processes to get information about the segments loaded in the cluster.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.selectors.coordinator.serviceName`|The druid.service name of the Coordinator node. To start the Coordinator with a different name, set it with this property. |druid/coordinator|
|
||||
|`druid.selectors.coordinator.serviceName`|The druid.service name of the Coordinator process. To start the Coordinator with a different name, set it with this property. |druid/coordinator|
|
||||
|
||||
|
||||
### Announcing Segments
|
||||
|
@ -696,18 +696,18 @@ This section contains the configuration options for the processes that reside on
|
|||
|
||||
### Coordinator
|
||||
|
||||
For general Coordinator Node information, see [here](../design/coordinator.html).
|
||||
For general Coordinator Process information, see [here](../design/coordinator.html).
|
||||
|
||||
#### Static Configuration
|
||||
|
||||
These Coordinator static configurations can be defined in the `coordinator/runtime.properties` file.
|
||||
|
||||
##### Coordinator Node Config
|
||||
##### Coordinator Process Config
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.bindOnHost`|Indicating whether the node's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
|
||||
|`druid.host`|The host for the current process. This is used to advertise the current processes location as reachable from another process and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.bindOnHost`|Indicating whether the process's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
|
||||
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8081|
|
||||
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8281|
|
||||
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/coordinator|
|
||||
|
@ -720,17 +720,17 @@ These Coordinator static configurations can be defined in the `coordinator/runti
|
|||
|`druid.coordinator.period.indexingPeriod`|How often to send compact/merge/conversion tasks to the indexing service. It's recommended to be longer than `druid.manager.segments.pollDuration`|PT1800S (30 mins)|
|
||||
|`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data.|PT300S|
|
||||
|`druid.coordinator.merge.on`|Boolean flag for whether or not the Coordinator should try and merge small segments into a more optimal segment size.|false|
|
||||
|`druid.coordinator.load.timeout`|The timeout duration for when the Coordinator assigns a segment to a Historical node.|PT15M|
|
||||
|`druid.coordinator.load.timeout`|The timeout duration for when the Coordinator assigns a segment to a Historical process.|PT15M|
|
||||
|`druid.coordinator.kill.pendingSegments.on`|Boolean flag for whether or not the Coordinator clean up old entries in the `pendingSegments` table of metadata store. If set to true, Coordinator will check the created time of most recently complete task. If it doesn't exist, it finds the created time of the earlist running/pending/waiting tasks. Once the created time is found, then for all dataSources not in the `killPendingSegmentsSkipList` (see [Dynamic configuration](#dynamic-configuration)), Coordinator will ask the Overlord to clean up the entries 1 day or more older than the found created time in the `pendingSegments` table. This will be done periodically based on `druid.coordinator.period` specified.|false|
|
||||
|`druid.coordinator.kill.on`|Boolean flag for whether or not the Coordinator should submit kill task for unused segments, that is, hard delete them from metadata store and deep storage. If set to true, then for all whitelisted dataSources (or optionally all), Coordinator will submit tasks periodically based on `period` specified. These kill tasks will delete all segments except for the last `durationToRetain` period. Whitelist or All can be set via dynamic configuration `killAllDataSources` and `killDataSourceWhitelist` described later.|false|
|
||||
|`druid.coordinator.kill.period`|How often to send kill tasks to the indexing service. Value must be greater than `druid.coordinator.period.indexingPeriod`. Only applies if kill is turned on.|P1D (1 Day)|
|
||||
|`druid.coordinator.kill.durationToRetain`| Do not kill segments in last `durationToRetain`, must be greater or equal to 0. Only applies and MUST be specified if kill is turned on. Note that default value is invalid.|PT-1S (-1 seconds)|
|
||||
|`druid.coordinator.kill.maxSegments`|Kill at most n segments per kill task submission, must be greater than 0. Only applies and MUST be specified if kill is turned on. Note that default value is invalid.|0|
|
||||
|`druid.coordinator.balancer.strategy`|Specify the type of balancing strategy that the coordinator should use to distribute segments among the historicals. `cachingCost` is logically equivalent to `cost` but is more CPU-efficient on large clusters and will replace `cost` in the future versions, users are invited to try it. Use `diskNormalized` to distribute segments among nodes so that the disks fill up uniformly and use `random` to randomly pick nodes to distribute segments.|`cost`|
|
||||
|`druid.coordinator.balancer.strategy`|Specify the type of balancing strategy that the coordinator should use to distribute segments among the historicals. `cachingCost` is logically equivalent to `cost` but is more CPU-efficient on large clusters and will replace `cost` in the future versions, users are invited to try it. Use `diskNormalized` to distribute segments among processes so that the disks fill up uniformly and use `random` to randomly pick processes to distribute segments.|`cost`|
|
||||
|`druid.coordinator.balancer.cachingCost.awaitInitialization`|Whether to wait for segment view initialization before creating the `cachingCost` balancing strategy. This property is enabled only when `druid.coordinator.balancer.strategy` is `cachingCost`. If set to 'true', the Coordinator will not start to assign segments, until the segment view is initialized. If set to 'false', the Coordinator will fallback to use the `cost` balancing strategy only if the segment view is not initialized yet. Notes, it may take much time to wait for the initialization since the `cachingCost` balancing strategy involves much computing to build itself.|false|
|
||||
|`druid.coordinator.loadqueuepeon.repeatDelay`|The start and repeat delay for the loadqueuepeon , which manages the load and drop of segments.|PT0.050S (50 ms)|
|
||||
|`druid.coordinator.asOverlord.enabled`|Boolean value for whether this Coordinator node should act like an Overlord as well. This configuration allows users to simplify a druid cluster by not having to deploy any standalone Overlord nodes. If set to true, then Overlord console is available at `http://coordinator-host:port/console.html` and be sure to set `druid.coordinator.asOverlord.overlordService` also. See next.|false|
|
||||
|`druid.coordinator.asOverlord.overlordService`| Required, if `druid.coordinator.asOverlord.enabled` is `true`. This must be same value as `druid.service` on standalone Overlord nodes and `druid.selectors.indexing.serviceName` on Middle Managers.|NULL|
|
||||
|`druid.coordinator.asOverlord.enabled`|Boolean value for whether this Coordinator process should act like an Overlord as well. This configuration allows users to simplify a druid cluster by not having to deploy any standalone Overlord processes. If set to true, then Overlord console is available at `http://coordinator-host:port/console.html` and be sure to set `druid.coordinator.asOverlord.overlordService` also. See next.|false|
|
||||
|`druid.coordinator.asOverlord.overlordService`| Required, if `druid.coordinator.asOverlord.enabled` is `true`. This must be same value as `druid.service` on standalone Overlord processes and `druid.selectors.indexing.serviceName` on Middle Managers.|NULL|
|
||||
|
||||
##### Segment Management
|
||||
|Property|Possible Values|Description|Default|
|
||||
|
@ -742,7 +742,7 @@ These Coordinator static configurations can be defined in the `coordinator/runti
|
|||
###### Additional config when "http" loadqueuepeon is used
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than `druid.segmentCache.numLoadingThreads` config on Historical node.|1|
|
||||
|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than `druid.segmentCache.numLoadingThreads` config on Historical process.|1|
|
||||
|
||||
##### Metadata Retrieval
|
||||
|
||||
|
@ -803,7 +803,7 @@ Issuing a GET request at the same URL will return the spec that is currently in
|
|||
|`killDataSourceWhitelist`|List of dataSources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none|
|
||||
|`killAllDataSources`|Send kill tasks for ALL dataSources if property `druid.coordinator.kill.on` is true. If this is set to true then `killDataSourceWhitelist` must not be specified or be empty list.|false|
|
||||
|`killPendingSegmentsSkipList`|List of dataSources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none|
|
||||
|`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 0 (loading queue is unbounded) |0|
|
||||
|`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" processes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of processes. Value 1000 could be a start point for a rather big cluster. Default value is 0 (loading queue is unbounded) |0|
|
||||
|`historicalNodesInMaintenance`| List of Historical nodes in maintenance mode. Coordinator doesn't assign new segments on those nodes and moves segments from the nodes according to a specified priority.|none|
|
||||
|`nodesInMaintenancePriority`| Priority of segments from servers in maintenance. Coordinator takes ceil(maxSegmentsToMove * (priority / 10)) from servers in maitenance during balancing phase, i.e.:<br>0 - no segments from servers in maintenance will be processed during balancing<br>5 - 50% segments from servers in maintenance<br>10 - 100% segments from servers in maintenance<br>By leveraging the priority an operator can prevent general nodes from overload or decrease maitenance time instead.|7|
|
||||
|
||||
|
@ -826,17 +826,19 @@ These configuration options control the behavior of the Lookup dynamic configura
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.manager.lookups.hostDeleteTimeout`|How long to wait for a `DELETE` request to a particular node before considering the `DELETE` a failure|PT1S|
|
||||
|`druid.manager.lookups.hostUpdateTimeout`|How long to wait for a `POST` request to a particular node before considering the `POST` a failure|PT10S|
|
||||
|`druid.manager.lookups.hostDeleteTimeout`|How long to wait for a `DELETE` request to a particular process before considering the `DELETE` a failure|PT1S|
|
||||
|`druid.manager.lookups.hostUpdateTimeout`|How long to wait for a `POST` request to a particular process before considering the `POST` a failure|PT10S|
|
||||
|`druid.manager.lookups.deleteAllTimeout`|How long to wait for all `DELETE` requests to finish before considering the delete attempt a failure|PT10S|
|
||||
|`druid.manager.lookups.updateAllTimeout`|How long to wait for all `POST` requests to finish before considering the attempt a failure|PT60S|
|
||||
|`druid.manager.lookups.threadPoolSize`|How many nodes can be managed concurrently (concurrent POST and DELETE requests). Requests this limit will wait in a queue until a slot becomes available.|10|
|
||||
|`druid.manager.lookups.threadPoolSize`|How many processes can be managed concurrently (concurrent POST and DELETE requests). Requests this limit will wait in a queue until a slot becomes available.|10|
|
||||
|`druid.manager.lookups.period`|How many milliseconds between checks for configuration changes|30_000|
|
||||
|
||||
##### Compaction Dynamic Configuration
|
||||
|
||||
Compaction configurations can also be set or updated dynamically without restarting Coordinators. For segment compaction,
|
||||
please see [Compacting Segments](../design/coordinator.html#compacting-segments).
|
||||
Compaction configurations can also be set or updated dynamically using
|
||||
[Coordinator's API](../operations/api-reference.html#compaction-configuration) without restarting Coordinators.
|
||||
|
||||
For details about segment compaction, please check [Segment Size Optimization](../operations/segment-optimization.html).
|
||||
|
||||
A description of the compaction config is:
|
||||
|
||||
|
@ -880,18 +882,18 @@ If you see this problem, it's recommended to set `skipOffsetFromLatest` to some
|
|||
|
||||
### Overlord
|
||||
|
||||
For general Overlord Node information, see [here](../design/overlord.html).
|
||||
For general Overlord Process information, see [here](../design/overlord.html).
|
||||
|
||||
#### Overlord Static Configuration
|
||||
|
||||
These Overlord static configurations can be defined in the `overlord/runtime.properties` file.
|
||||
|
||||
##### Overlord Node Configs
|
||||
##### Overlord Process Configs
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.bindOnHost`|Indicating whether the node's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
|
||||
|`druid.host`|The host for the current process. This is used to advertise the current processes location as reachable from another process and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.bindOnHost`|Indicating whether the process's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
|
||||
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8090|
|
||||
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8290|
|
||||
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/overlord|
|
||||
|
@ -1100,12 +1102,12 @@ This section contains the configuration options for the processes that reside on
|
|||
|
||||
These MiddleManager and Peon configurations can be defined in the `middleManager/runtime.properties` file.
|
||||
|
||||
#### MiddleManager Node Config
|
||||
#### MiddleManager Process Config
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.bindOnHost`|Indicating whether the node's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
|
||||
|`druid.host`|The host for the current process. This is used to advertise the current processes location as reachable from another process and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.bindOnHost`|Indicating whether the process's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
|
||||
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8091|
|
||||
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8291|
|
||||
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/middlemanager|
|
||||
|
@ -1136,9 +1138,9 @@ Processing properties set on the Middlemanager will be passed through to Peons.
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes 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.|auto (max 1GB)|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size 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.|auto (max 1GB)|
|
||||
|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
|
||||
|`druid.processing.formatString`|Realtime and Historical nodes use this format string to name their processing threads.|processing-%s|
|
||||
|`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)|
|
||||
|
@ -1177,7 +1179,7 @@ Additional peon configs include:
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.peon.mode`|Choices are "local" and "remote". Setting this to local means you intend to run the peon as a standalone node (Not recommended).|remote|
|
||||
|`druid.peon.mode`|Choices are "local" and "remote". Setting this to local means you intend to run the peon as a standalone process (Not recommended).|remote|
|
||||
|`druid.indexer.task.baseDir`|Base temporary working directory.|`System.getProperty("java.io.tmpdir")`|
|
||||
|`druid.indexer.task.baseTaskDir`|Base temporary working directory for tasks.|`${druid.indexer.task.baseDir}/persistent/tasks`|
|
||||
|`druid.indexer.task.defaultHadoopCoordinates`|Hadoop version to use with HadoopIndexTasks that do not request a particular version.|org.apache.hadoop:hadoop-client:2.8.3|
|
||||
|
@ -1221,15 +1223,15 @@ then the value from the configuration below is used:
|
|||
|
||||
### Historical
|
||||
|
||||
For general Historical Node information, see [here](../design/historical.html).
|
||||
For general Historical Process information, see [here](../design/historical.html).
|
||||
|
||||
These Historical configurations can be defined in the `historical/runtime.properties` file.
|
||||
|
||||
#### Historical Node Configuration
|
||||
#### Historical Process Configuration
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.bindOnHost`|Indicating whether the node's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
|
||||
|`druid.host`|The host for the current process. This is used to advertise the current processes location as reachable from another process and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.bindOnHost`|Indicating whether the process's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
|
||||
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8083|
|
||||
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8283|
|
||||
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/historical|
|
||||
|
@ -1239,18 +1241,18 @@ These Historical configurations can be defined in the `historical/runtime.proper
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.server.maxSize`|The maximum number of bytes-worth of segments that the node wants assigned to it. This is not a limit that Historical nodes actually enforces, just a value published to the Coordinator node so it can plan accordingly.|0|
|
||||
|`druid.server.tier`| A string to name the distribution tier that the storage node belongs to. Many of the [rules Coordinator nodes use](../operations/rule-configuration.html) to manage segments can be keyed on tiers. | `_default_tier` |
|
||||
|`druid.server.priority`|In a tiered architecture, the priority of the tier, thus allowing control over which nodes are queried. Higher numbers mean higher priority. The default (no priority) works for architecture with no cross replication (tiers that have no data-storage overlap). Data centers typically have equal priority. | 0 |
|
||||
|`druid.server.maxSize`|The maximum number of bytes-worth of segments that the process wants assigned to it. This is not a limit that Historical processes actually enforces, just a value published to the Coordinator process so it can plan accordingly.|0|
|
||||
|`druid.server.tier`| A string to name the distribution tier that the storage process belongs to. Many of the [rules Coordinator processes use](../operations/rule-configuration.html) to manage segments can be keyed on tiers. | `_default_tier` |
|
||||
|`druid.server.priority`|In a tiered architecture, the priority of the tier, thus allowing control over which processes are queried. Higher numbers mean higher priority. The default (no priority) works for architecture with no cross replication (tiers that have no data-storage overlap). Data centers typically have equal priority. | 0 |
|
||||
|
||||
#### Storing Segments
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.segmentCache.locations`|Segments assigned to a Historical node are first stored on the local file system (in a disk cache) and then served by the Historical node. These locations define where that local cache resides. This value cannot be NULL or EMPTY. Here is an example `druid.segmentCache.locations=[{"path": "/mnt/druidSegments", "maxSize": 10000, "freeSpacePercent": 1.0}]`. "freeSpacePercent" is optional, if provided then enforces that much of free disk partition space while storing segments. But, it depends on File.getTotalSpace() and File.getFreeSpace() methods, so enable if only if they work for your File System.| none |
|
||||
|`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a node is no longer serving a segment.|true|
|
||||
|`druid.segmentCache.dropSegmentDelayMillis`|How long a node delays before completely dropping segment.|30000 (30 seconds)|
|
||||
|`druid.segmentCache.infoDir`|Historical nodes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir|
|
||||
|`druid.segmentCache.locations`|Segments assigned to a Historical process are first stored on the local file system (in a disk cache) and then served by the Historical process. These locations define where that local cache resides. This value cannot be NULL or EMPTY. Here is an example `druid.segmentCache.locations=[{"path": "/mnt/druidSegments", "maxSize": 10000, "freeSpacePercent": 1.0}]`. "freeSpacePercent" is optional, if provided then enforces that much of free disk partition space while storing segments. But, it depends on File.getTotalSpace() and File.getFreeSpace() methods, so enable if only if they work for your File System.| none |
|
||||
|`druid.segmentCache.deleteOnRemove`|Delete segment files from cache once a process is no longer serving a segment.|true|
|
||||
|`druid.segmentCache.dropSegmentDelayMillis`|How long a process delays before completely dropping segment.|30000 (30 seconds)|
|
||||
|`druid.segmentCache.infoDir`|Historical processes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir|
|
||||
|`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)|
|
||||
|`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from from deep storage.|10|
|
||||
|`druid.segmentCache.numBootstrapThreads`|How many segments to load concurrently from local storage at startup.|Same as numLoadingThreads|
|
||||
|
@ -1279,9 +1281,9 @@ Druid uses Jetty to serve HTTP requests.
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes 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.|auto (max 1GB)|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size 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.|auto (max 1GB)|
|
||||
|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
|
||||
|`druid.processing.formatString`|Realtime and Historical nodes use this format string to name their processing threads.|processing-%s|
|
||||
|`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)|
|
||||
|
@ -1319,12 +1321,12 @@ For general Broker process information, see [here](../design/broker.html).
|
|||
|
||||
These Broker configurations can be defined in the `broker/runtime.properties` file.
|
||||
|
||||
#### Broker Node Configs
|
||||
#### Broker Process Configs
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.bindOnHost`|Indicating whether the node's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
|
||||
|`druid.host`|The host for the current process. This is used to advertise the current processes location as reachable from another process and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.bindOnHost`|Indicating whether the process's internal jetty server bind on `druid.host`. Default is false, which means binding to all interfaces.|false|
|
||||
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8082|
|
||||
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8282|
|
||||
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/broker|
|
||||
|
@ -1335,7 +1337,7 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
|
|||
|
||||
|Property|Possible Values|Description|Default|
|
||||
|--------|---------------|-----------|-------|
|
||||
|`druid.broker.balancer.type`|`random`, `connectionCount`|Determines how the broker balances connections to Historical nodes. `random` choose randomly, `connectionCount` picks the node with the fewest number of active connections to|`random`|
|
||||
|`druid.broker.balancer.type`|`random`, `connectionCount`|Determines how the broker balances connections to Historical processes. `random` choose randomly, `connectionCount` picks the process with the fewest number of active connections to|`random`|
|
||||
|`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
|
||||
|`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
|
||||
|
||||
|
@ -1350,7 +1352,7 @@ Druid uses Jetty to serve HTTP requests.
|
|||
|`druid.server.http.maxIdleTime`|The Jetty max idle time for a connection.|PT5M|
|
||||
|`druid.server.http.enableRequestLimit`|If enabled, no requests would be queued in jetty queue and "HTTP 429 Too Many Requests" error response would be sent. |false|
|
||||
|`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 nodes 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.|Long.MAX_VALUE|
|
||||
|`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.|Long.MAX_VALUE|
|
||||
|`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.|`PT0S` (do not wait)|
|
||||
|`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.html) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE|
|
||||
|
@ -1363,7 +1365,7 @@ client has the following configuration options.
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to Historical and real-time processes. If there are more queries than this number that all need to speak to the same node, then they will queue up.|20|
|
||||
|`druid.broker.http.numConnections`|Size of connection pool for the Broker to connect to Historical and real-time processes. If there are more queries than this number that all need to speak to the same process, then they will queue up.|20|
|
||||
|`druid.broker.http.compressionCodec`|Compression codec the Broker uses to communicate with Historical and real-time processes. May be "gzip" or "identity".|gzip|
|
||||
|`druid.broker.http.readTimeout`|The timeout for data reads from Historical servers and real-time tasks.|PT15M|
|
||||
|`druid.broker.http.unusedConnectionTimeout`|The timeout for idle connections in connection pool. This timeout should be less than `druid.broker.http.readTimeout`. Set this timeout = ~90% of `druid.broker.http.readTimeout`|`PT4M`|
|
||||
|
@ -1383,9 +1385,9 @@ The broker uses processing configs for nested groupBy queries. And, if you use g
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes 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.|auto (max 1GB)|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size 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.|auto (max 1GB)|
|
||||
|`druid.processing.buffer.poolCacheMaxCount`|processing buffer pool caches the buffers for later use, this is the maximum count cache will grow to. note that pool can create more buffers than it can cache if necessary.|Integer.MAX_VALUE|
|
||||
|`druid.processing.formatString`|Realtime and Historical nodes use this format string to name their processing threads.|processing-%s|
|
||||
|`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)|
|
||||
|
@ -1447,8 +1449,8 @@ See [cache configuration](#cache-configuration) for how to configure cache setti
|
|||
|Property|Possible Values|Description|Default|
|
||||
|--------|---------------|-----------|-------|
|
||||
|`druid.serverview.type`|batch or http|Segment discovery method to use. "http" enables discovering segments using HTTP instead of zookeeper.|batch|
|
||||
|`druid.broker.segment.watchedTiers`|List of strings|Broker watches the segment announcements from nodes serving segments to build cache of which node is serving which segments, this configuration allows to only consider segments being served from a whitelist of tiers. By default, Broker would consider all tiers. This can be used to partition your dataSources in specific Historical tiers and configure brokers in partitions so that they are only queryable for specific dataSources.|none|
|
||||
|`druid.broker.segment.watchedDataSources`|List of strings|Broker watches the segment announcements from nodes serving segments to build cache of which node is serving which segments, this configuration allows to only consider segments being served from a whitelist of dataSources. By default, Broker would consider all datasources. This can be used to configure brokers in partitions so that they are only queryable for specific dataSources.|none|
|
||||
|`druid.broker.segment.watchedTiers`|List of strings|Broker watches the segment announcements from processes serving segments to build cache of which process is serving which segments, this configuration allows to only consider segments being served from a whitelist of tiers. By default, Broker would consider all tiers. This can be used to partition your dataSources in specific Historical tiers and configure brokers in partitions so that they are only queryable for specific dataSources.|none|
|
||||
|`druid.broker.segment.watchedDataSources`|List of strings|Broker watches the segment announcements from processes serving segments to build cache of which process is serving which segments, this configuration allows to only consider segments being served from a whitelist of dataSources. By default, Broker would consider all datasources. This can be used to configure brokers in partitions so that they are only queryable for specific dataSources.|none|
|
||||
|`druid.broker.segment.awaitInitializationOnStart`|Boolean|Whether the the Broker will wait for its view of segments to fully initialize before starting up. If set to 'true', the Broker's HTTP server will not start up, and the Broker will not announce itself as available, until the server view is initialized. See also `druid.sql.planner.awaitInitializationOnStart`, a related setting.|true|
|
||||
|
||||
## Cache Configuration
|
||||
|
@ -1462,7 +1464,7 @@ Druid uses a local in-memory cache by default, unless a diffrent type of cache i
|
|||
Use the `druid.cache.type` configuration to set a different kind of cache.
|
||||
|
||||
Cache settings are set globally, so the same configuration can be re-used
|
||||
for both Broker and Historical nodes, when defined in the common properties file.
|
||||
for both Broker and Historical processes, when defined in the common properties file.
|
||||
|
||||
|
||||
### Cache Type
|
||||
|
@ -1501,7 +1503,7 @@ Below are the configuration options known to this module:
|
|||
|`druid.cache.sizeInBytes`|The maximum size of the cache in bytes on heap.|min(1GB, Runtime.maxMemory / 10)|
|
||||
|`druid.cache.expireAfter`|The time (in ms) after an access for which a cache entry may be expired|None (no time limit)|
|
||||
|`druid.cache.cacheExecutorFactory`|The executor factory to use for Caffeine maintenance. One of `COMMON_FJP`, `SINGLE_THREAD`, or `SAME_THREAD`|ForkJoinPool common pool (`COMMON_FJP`)|
|
||||
|`druid.cache.evictOnClose`|If a close of a namespace (ex: removing a segment from a node) should cause an eager eviction of associated cache values|`false`|
|
||||
|`druid.cache.evictOnClose`|If a close of a namespace (ex: removing a segment from a process) should cause an eager eviction of associated cache values|`false`|
|
||||
|
||||
##### `druid.cache.cacheExecutorFactory`
|
||||
|
||||
|
@ -1523,7 +1525,7 @@ In addition to the normal cache metrics, the caffeine cache implementation also
|
|||
|
||||
##### Memcached
|
||||
|
||||
Uses memcached as cache backend. This allows all nodes to share the same cache.
|
||||
Uses memcached as cache backend. This allows all processes to share the same cache.
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|
@ -1551,12 +1553,12 @@ If there is an L1 miss and L2 hit, it will also populate L1.
|
|||
|`druid.cache.l2.type`|type of cache to use for L2 cache. See `druid.cache.type` configuration for valid types.|`caffeine`|
|
||||
|`druid.cache.l1.*`|Any property valid for the given type of L1 cache can be set using this prefix. For instance, if you are using a `caffeine` L1 cache, specify `druid.cache.l1.sizeInBytes` to set its size.|defaults are the same as for the given cache type.|
|
||||
|`druid.cache.l2.*`|Prefix for L2 cache settings, see description for L1.|defaults are the same as for the given cache type.|
|
||||
|`druid.cache.useL2`|A boolean indicating whether to query L2 cache, if it's a miss in L1. It makes sense to configure this to `false` on Historical nodes, if L2 is a remote cache like `memcached`, and this cache also used on brokers, because in this case if a query reached Historical it means that a broker didn't find corresponding results in the same remote cache, so a query to the remote cache from Historical is guaranteed to be a miss.|`true`|
|
||||
|`druid.cache.useL2`|A boolean indicating whether to query L2 cache, if it's a miss in L1. It makes sense to configure this to `false` on Historical processes, if L2 is a remote cache like `memcached`, and this cache also used on brokers, because in this case if a query reached Historical it means that a broker didn't find corresponding results in the same remote cache, so a query to the remote cache from Historical is guaranteed to be a miss.|`true`|
|
||||
|`druid.cache.populateL2`|A boolean indicating whether to put results into L2 cache.|`true`|
|
||||
|
||||
## General Query Configuration
|
||||
|
||||
This section describes configurations that control behavior of Druid's query types, applicable to Broker, Historical, and MiddleManager nodes.
|
||||
This section describes configurations that control behavior of Druid's query types, applicable to Broker, Historical, and MiddleManager processes.
|
||||
|
||||
### TopN Query config
|
||||
|
||||
|
@ -1580,7 +1582,7 @@ This section describes configurations that control behavior of Druid's query typ
|
|||
|
||||
### GroupBy Query Config
|
||||
|
||||
This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on Broker, Historical, and MiddleManager nodes. You can set the query context parameters through the [query context](../querying/query-context.html).
|
||||
This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on Broker, Historical, and MiddleManager processes. You can set the query context parameters through the [query context](../querying/query-context.html).
|
||||
|
||||
#### Configurations for groupBy v2
|
||||
|
||||
|
@ -1627,7 +1629,7 @@ Supported runtime properties:
|
|||
|`druid.query.groupBy.bufferGrouperInitialBuckets`|Initial number of buckets in the off-heap hash table used for grouping results. Set to 0 to use a reasonable default (1024).|0|
|
||||
|`druid.query.groupBy.bufferGrouperMaxLoadFactor`|Maximum load factor of the off-heap hash table used for grouping results. When the load factor exceeds this size, the table will be grown or spilled to disk. Set to 0 to use a reasonable default (0.7).|0|
|
||||
|`druid.query.groupBy.forceHashAggregation`|Force to use hash-based aggregation.|false|
|
||||
|`druid.query.groupBy.intermediateCombineDegree`|Number of intermediate nodes combined together in the combining tree. Higher degrees will need less threads which might be helpful to improve the query performance by reducing the overhead of too many threads if the server has sufficiently powerful cpu cores.|8|
|
||||
|`druid.query.groupBy.intermediateCombineDegree`|Number of intermediate processes combined together in the combining tree. Higher degrees will need less threads which might be helpful to improve the query performance by reducing the overhead of too many threads if the server has sufficiently powerful cpu cores.|8|
|
||||
|`druid.query.groupBy.numParallelCombineThreads`|Hint for the number of parallel combining threads. This should be larger than 1 to turn on the parallel combining feature. The actual number of threads used for parallel combining is min(`druid.query.groupBy.numParallelCombineThreads`, `druid.processing.numThreads`).|1 (disabled)|
|
||||
|
||||
Supported query contexts:
|
||||
|
@ -1640,7 +1642,7 @@ Supported query contexts:
|
|||
|`intermediateCombineDegree`|Overrides the value of `druid.query.groupBy.intermediateCombineDegree`|None|
|
||||
|`numParallelCombineThreads`|Overrides the value of `druid.query.groupBy.numParallelCombineThreads`|None|
|
||||
|`sortByDimsFirst`|Sort the results first by dimension values and then by timestamp.|false|
|
||||
|`forceLimitPushDown`|When all fields in the orderby are part of the grouping key, the broker will push limit application down to the Historical nodes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.|false|
|
||||
|`forceLimitPushDown`|When all fields in the orderby are part of the grouping key, the broker will push limit application down to the Historical processes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.|false|
|
||||
|
||||
|
||||
#### GroupBy v1 configurations
|
||||
|
@ -1661,6 +1663,6 @@ Supported query contexts:
|
|||
|`useOffheap`|Set to true to store aggregations off-heap when merging results.|false|
|
||||
|
||||
|
||||
## Realtime nodes
|
||||
## Realtime processes
|
||||
|
||||
Configuration for the deprecated realtime node can be found [here](../configuration/realtime.html).
|
||||
Configuration for the deprecated realtime process can be found [here](../configuration/realtime.html).
|
||||
|
|
|
@ -24,7 +24,7 @@ title: "Logging"
|
|||
|
||||
# Logging
|
||||
|
||||
Druid nodes will emit logs that are useful for debugging to the console. Druid nodes also emit periodic metrics about their state. For more about metrics, see [Configuration](../configuration/index.html#enabling-metrics). Metric logs are printed to the console by default, and can be disabled with `-Ddruid.emitter.logging.logLevel=debug`.
|
||||
Druid processes will emit logs that are useful for debugging to the console. Druid processes also emit periodic metrics about their state. For more about metrics, see [Configuration](../configuration/index.html#enabling-metrics). Metric logs are printed to the console by default, and can be disabled with `-Ddruid.emitter.logging.logLevel=debug`.
|
||||
|
||||
Druid uses [log4j2](http://logging.apache.org/log4j/2.x/) for logging. Logging can be configured with a log4j2.xml file. Add the path to the directory containing the log4j2.xml file (e.g. the _common/ dir) to your classpath if you want to override default Druid log configuration. Note that this directory should be earlier in the classpath than the druid jars. The easiest way to do this is to prefix the classpath with the config dir.
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "Realtime Node Configuration"
|
||||
title: "Realtime Process Configuration"
|
||||
---
|
||||
|
||||
<!--
|
||||
|
@ -22,20 +22,20 @@ title: "Realtime Node Configuration"
|
|||
~ under the License.
|
||||
-->
|
||||
|
||||
# Realtime Node Configuration
|
||||
# Realtime Process Configuration
|
||||
|
||||
For general Realtime Node information, see [here](../design/realtime.html).
|
||||
For general Realtime Process information, see [here](../design/realtime.html).
|
||||
|
||||
Runtime Configuration
|
||||
---------------------
|
||||
|
||||
The realtime node uses several of the global configs in [Configuration](../configuration/index.html) and has the following set of configurations as well:
|
||||
The realtime process uses several of the global configs in [Configuration](../configuration/index.html) and has the following set of configurations as well:
|
||||
|
||||
### Node Config
|
||||
### Process Config
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.host`|The host for the current node. This is used to advertise the current processes location as reachable from another node and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.host`|The host for the current process. This is used to advertise the current processes location as reachable from another process and should generally be specified such that `http://${druid.host}/` could actually talk to this process|InetAddress.getLocalHost().getCanonicalHostName()|
|
||||
|`druid.plaintextPort`|This is the port to actually listen on; unless port mapping is used, this will be the same port as is on `druid.host`|8084|
|
||||
|`druid.tlsPort`|TLS port for HTTPS connector, if [druid.enableTlsPort](../operations/tls-support.html) is set then this config will be used. If `druid.host` contains port then that port will be ignored. This should be a non-negative Integer.|8284|
|
||||
|`druid.service`|The name of the service. This is used as a dimension when emitting metrics and alerts to differentiate between the various services|druid/realtime|
|
||||
|
@ -60,8 +60,8 @@ The realtime node uses several of the global configs in [Configuration](../confi
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes 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.|auto (max 1GB)|
|
||||
|`druid.processing.formatString`|Realtime and Historical nodes use this format string to name their processing threads.|processing-%s|
|
||||
|`druid.processing.buffer.sizeBytes`|This specifies a buffer size 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.|auto (max 1GB)|
|
||||
|`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)|
|
||||
|
@ -86,7 +86,7 @@ See [groupBy server configuration](../querying/groupbyquery.html#server-configur
|
|||
|
||||
### Caching
|
||||
|
||||
You can optionally configure caching to be enabled on the realtime node by setting caching configs here.
|
||||
You can optionally configure caching to be enabled on the realtime process by setting caching configs here.
|
||||
|
||||
|Property|Possible Values|Description|Default|
|
||||
|--------|---------------|-----------|-------|
|
||||
|
|
|
@ -29,8 +29,8 @@ title: "Cassandra Deep Storage"
|
|||
Druid can use Cassandra as a deep storage mechanism. Segments and their metadata are stored in Cassandra in two tables:
|
||||
`index_storage` and `descriptor_storage`. Underneath the hood, the Cassandra integration leverages Astyanax. The
|
||||
index storage table is a [Chunked Object](https://github.com/Netflix/astyanax/wiki/Chunked-Object-Store) repository. It contains
|
||||
compressed segments for distribution to Historical nodes. Since segments can be large, the Chunked Object storage allows the integration to multi-thread
|
||||
the write to Cassandra, and spreads the data across all the nodes in a cluster. The descriptor storage table is a normal C* table that
|
||||
compressed segments for distribution to Historical processes. Since segments can be large, the Chunked Object storage allows the integration to multi-thread
|
||||
the write to Cassandra, and spreads the data across all the processes in a cluster. The descriptor storage table is a normal C* table that
|
||||
stores the segment metadatak.
|
||||
|
||||
## Schema
|
||||
|
|
|
@ -24,7 +24,7 @@ title: "Deep Storage"
|
|||
|
||||
# Deep Storage
|
||||
|
||||
Deep storage is where segments are stored. It is a storage mechanism that Druid does not provide. This deep storage infrastructure defines the level of durability of your data, as long as Druid nodes can see this storage infrastructure and get at the segments stored on it, you will not lose data no matter how many Druid nodes you lose. If segments disappear from this storage layer, then you will lose whatever data those segments represented.
|
||||
Deep storage is where segments are stored. It is a storage mechanism that Druid does not provide. This deep storage infrastructure defines the level of durability of your data, as long as Druid processes can see this storage infrastructure and get at the segments stored on it, you will not lose data no matter how many Druid nodes you lose. If segments disappear from this storage layer, then you will lose whatever data those segments represented.
|
||||
|
||||
## Local Mount
|
||||
|
||||
|
|
|
@ -137,8 +137,8 @@ config changes.
|
|||
|
||||
The Metadata Storage is accessed only by:
|
||||
|
||||
1. Indexing Service Nodes (if any)
|
||||
2. Realtime Nodes (if any)
|
||||
3. Coordinator Nodes
|
||||
1. Indexing Service Processes (if any)
|
||||
2. Realtime Processes (if any)
|
||||
3. Coordinator Processes
|
||||
|
||||
Thus you need to give permissions (eg in AWS Security Groups) only for these machines to access the Metadata storage.
|
||||
|
|
|
@ -44,7 +44,7 @@ ${druid.zk.paths.coordinatorPath}/_COORDINATOR
|
|||
|
||||
The `announcementsPath` and `servedSegmentsPath` are used for this.
|
||||
|
||||
All [Historical](../design/historical.html) and [Realtime](../design/realtime.html) nodes publish themselves on the `announcementsPath`, specifically, they will create an ephemeral znode at
|
||||
All [Historical](../design/historical.html) and [Realtime](../design/realtime.html) processes publish themselves on the `announcementsPath`, specifically, they will create an ephemeral znode at
|
||||
|
||||
```
|
||||
${druid.zk.paths.announcementsPath}/${druid.host}
|
||||
|
@ -62,16 +62,16 @@ And as they load up segments, they will attach ephemeral znodes that look like
|
|||
${druid.zk.paths.servedSegmentsPath}/${druid.host}/_segment_identifier_
|
||||
```
|
||||
|
||||
Nodes like the [Coordinator](../design/coordinator.html) and [Broker](../design/broker.html) can then watch these paths to see which nodes are currently serving which segments.
|
||||
Processes like the [Coordinator](../design/coordinator.html) and [Broker](../design/broker.html) can then watch these paths to see which processes are currently serving which segments.
|
||||
|
||||
### Segment load/drop protocol between Coordinator and Historical
|
||||
|
||||
The `loadQueuePath` is used for this.
|
||||
|
||||
When the [Coordinator](../design/coordinator.html) decides that a [Historical](../design/historical.html) node should load or drop a segment, it writes an ephemeral znode to
|
||||
When the [Coordinator](../design/coordinator.html) decides that a [Historical](../design/historical.html) process should load or drop a segment, it writes an ephemeral znode to
|
||||
|
||||
```
|
||||
${druid.zk.paths.loadQueuePath}/_host_of_historical_node/_segment_identifier
|
||||
${druid.zk.paths.loadQueuePath}/_host_of_historical_process/_segment_identifier
|
||||
```
|
||||
|
||||
This node will contain a payload that indicates to the Historical node what it should do with the given segment. When the Historical node is done with the work, it will delete the znode in order to signify to the Coordinator that it is complete.
|
||||
This znode will contain a payload that indicates to the Historical process what it should do with the given segment. When the Historical process is done with the work, it will delete the znode in order to signify to the Coordinator that it is complete.
|
||||
|
|
|
@ -123,7 +123,7 @@ An Authenticator implementation should provide some means through configuration
|
|||
|
||||
## Internal System User
|
||||
|
||||
Internal requests between Druid nodes (non-user initiated communications) need to have authentication credentials attached.
|
||||
Internal requests between Druid processes (non-user initiated communications) need to have authentication credentials attached.
|
||||
|
||||
These requests should be run as an "internal system user", an identity that represents the Druid cluster itself, with full access permissions.
|
||||
|
||||
|
|
|
@ -26,7 +26,7 @@ title: "Broker"
|
|||
|
||||
### Configuration
|
||||
|
||||
For Broker Node Configuration, see [Broker Configuration](../configuration/index.html#broker).
|
||||
For Broker Process Configuration, see [Broker Configuration](../configuration/index.html#broker).
|
||||
|
||||
### HTTP endpoints
|
||||
|
||||
|
@ -34,8 +34,8 @@ For a list of API endpoints supported by the Broker, see [Broker API](../operati
|
|||
|
||||
### Overview
|
||||
|
||||
The Broker is the node to route queries to if you want to run a distributed cluster. It understands the metadata published to ZooKeeper about what segments exist on what nodes and routes queries such that they hit the right nodes. This node also merges the result sets from all of the individual nodes together.
|
||||
On start up, Realtime nodes announce themselves and the segments they are serving in Zookeeper.
|
||||
The Broker is the process to route queries to if you want to run a distributed cluster. It understands the metadata published to ZooKeeper about what segments exist on what processes and routes queries such that they hit the right processes. This process also merges the result sets from all of the individual processes together.
|
||||
On start up, Historical processes announce themselves and the segments they are serving in Zookeeper.
|
||||
|
||||
### Running
|
||||
|
||||
|
@ -45,11 +45,11 @@ org.apache.druid.cli.Main server broker
|
|||
|
||||
### Forwarding Queries
|
||||
|
||||
Most druid queries contain an interval object that indicates a span of time for which data is requested. Likewise, Druid [Segments](../design/segments.html) are partitioned to contain data for some interval of time and segments are distributed across a cluster. Consider a simple datasource with 7 segments where each segment contains data for a given day of the week. Any query issued to the datasource for more than one day of data will hit more than one segment. These segments will likely be distributed across multiple nodes, and hence, the query will likely hit multiple nodes.
|
||||
Most druid queries contain an interval object that indicates a span of time for which data is requested. Likewise, Druid [Segments](../design/segments.html) are partitioned to contain data for some interval of time and segments are distributed across a cluster. Consider a simple datasource with 7 segments where each segment contains data for a given day of the week. Any query issued to the datasource for more than one day of data will hit more than one segment. These segments will likely be distributed across multiple processes, and hence, the query will likely hit multiple processes.
|
||||
|
||||
To determine which nodes to forward queries to, the Broker node first builds a view of the world from information in Zookeeper. Zookeeper maintains information about [Historical](../design/historical.html) and [Realtime](../design/realtime.html) nodes and the segments they are serving. For every datasource in Zookeeper, the Broker node builds a timeline of segments and the nodes that serve them. When queries are received for a specific datasource and interval, the Broker node performs a lookup into the timeline associated with the query datasource for the query interval and retrieves the nodes that contain data for the query. The Broker process then forwards down the query to the selected nodes.
|
||||
To determine which processes to forward queries to, the Broker process first builds a view of the world from information in Zookeeper. Zookeeper maintains information about [Historical](../design/historical.html) and streaming ingestion [Peon](../design/peon.html) processes and the segments they are serving. For every datasource in Zookeeper, the Broker process builds a timeline of segments and the processes that serve them. When queries are received for a specific datasource and interval, the Broker process performs a lookup into the timeline associated with the query datasource for the query interval and retrieves the processes that contain data for the query. The Broker process then forwards down the query to the selected processes.
|
||||
|
||||
### Caching
|
||||
|
||||
Broker nodes employ a cache with a LRU cache invalidation strategy. The Broker cache stores per-segment results. The cache can be local to each Broker process or shared across multiple processes using an external distributed cache such as [memcached](http://memcached.org/). Each time a broker node receives a query, it first maps the query to a set of segments. A subset of these segment results may already exist in the cache and the results can be directly pulled from the cache. For any segment results that do not exist in the cache, the broker node will forward the query to the
|
||||
Historical nodes. Once the Historical processes return their results, the Broker will store those results in the cache. Real-time segments are never cached and hence requests for real-time data will always be forwarded to real-time nodes. Real-time data is perpetually changing and caching the results would be unreliable.
|
||||
Broker processes employ a cache with a LRU cache invalidation strategy. The Broker cache stores per-segment results. The cache can be local to each Broker process or shared across multiple processes using an external distributed cache such as [memcached](http://memcached.org/). Each time a broker process receives a query, it first maps the query to a set of segments. A subset of these segment results may already exist in the cache and the results can be directly pulled from the cache. For any segment results that do not exist in the cache, the broker process will forward the query to the
|
||||
Historical processes. Once the Historical processes return their results, the Broker will store those results in the cache. Real-time segments are never cached and hence requests for real-time data will always be forwarded to real-time processes. Real-time data is perpetually changing and caching the results would be unreliable.
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "Coordinator Node"
|
||||
title: "Coordinator Process"
|
||||
---
|
||||
|
||||
<!--
|
||||
|
@ -22,11 +22,11 @@ title: "Coordinator Node"
|
|||
~ under the License.
|
||||
-->
|
||||
|
||||
# Coordinator Node
|
||||
# Coordinator Process
|
||||
|
||||
### Configuration
|
||||
|
||||
For Coordinator Node Configuration, see [Coordinator Configuration](../configuration/index.html#coordinator).
|
||||
For Coordinator Process Configuration, see [Coordinator Configuration](../configuration/index.html#coordinator).
|
||||
|
||||
### HTTP endpoints
|
||||
|
||||
|
@ -34,11 +34,11 @@ For a list of API endpoints supported by the Coordinator, see [Coordinator API](
|
|||
|
||||
### Overview
|
||||
|
||||
The Druid Coordinator node is primarily responsible for segment management and distribution. More specifically, the Druid Coordinator node communicates to Historical nodes to load or drop segments based on configurations. The Druid Coordinator is responsible for loading new segments, dropping outdated segments, managing segment replication, and balancing segment load.
|
||||
The Druid Coordinator process is primarily responsible for segment management and distribution. More specifically, the Druid Coordinator process communicates to Historical processes to load or drop segments based on configurations. The Druid Coordinator is responsible for loading new segments, dropping outdated segments, managing segment replication, and balancing segment load.
|
||||
|
||||
The Druid Coordinator runs periodically and the time between each run is a configurable parameter. Each time the Druid Coordinator runs, it assesses the current state of the cluster before deciding on the appropriate actions to take. Similar to the Broker and Historical processses, the Druid Coordinator maintains a connection to a Zookeeper cluster for current cluster information. The Coordinator also maintains a connection to a database containing information about available segments and rules. Available segments are stored in a segment table and list all segments that should be loaded in the cluster. Rules are stored in a rule table and indicate how segments should be handled.
|
||||
|
||||
Before any unassigned segments are serviced by Historical nodes, the available Historical nodes for each tier are first sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always assigned to the nodes with least capacity to maintain a level of balance between nodes. The Coordinator does not directly communicate with a historical node when assigning it a new segment; instead the Coordinator creates some temporary information about the new segment under load queue path of the historical node. Once this request is seen, the historical node will load the segment and begin servicing it.
|
||||
Before any unassigned segments are serviced by Historical processes, the available Historical processes for each tier are first sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always assigned to the processes with least capacity to maintain a level of balance between processes. The Coordinator does not directly communicate with a historical process when assigning it a new segment; instead the Coordinator creates some temporary information about the new segment under load queue path of the historical process. Once this request is seen, the historical process will load the segment and begin servicing it.
|
||||
|
||||
### Running
|
||||
|
||||
|
@ -57,16 +57,16 @@ Note that if all segments in database are deleted(or marked unused), then Coordi
|
|||
|
||||
### Segment Availability
|
||||
|
||||
If a Historical node restarts or becomes unavailable for any reason, the Druid Coordinator will notice a node has gone missing and treat all segments served by that node as being dropped. Given a sufficient period of time, the segments may be reassigned to other Historical nodes in the cluster. However, each segment that is dropped is not immediately forgotten. Instead, there is a transitional data structure that stores all dropped segments with an associated lifetime. The lifetime represents a period of time in which the Coordinator will not reassign a dropped segment. Hence, if a historical node becomes unavailable and available again within a short period of time, the historical node will start up and serve segments from its cache without any those segments being reassigned across the cluster.
|
||||
If a Historical process restarts or becomes unavailable for any reason, the Druid Coordinator will notice a process has gone missing and treat all segments served by that process as being dropped. Given a sufficient period of time, the segments may be reassigned to other Historical processes in the cluster. However, each segment that is dropped is not immediately forgotten. Instead, there is a transitional data structure that stores all dropped segments with an associated lifetime. The lifetime represents a period of time in which the Coordinator will not reassign a dropped segment. Hence, if a historical process becomes unavailable and available again within a short period of time, the historical process will start up and serve segments from its cache without any those segments being reassigned across the cluster.
|
||||
|
||||
### Balancing Segment Load
|
||||
|
||||
To ensure an even distribution of segments across Historical nodes in the cluster, the Coordinator node will find the total size of all segments being served by every Historical node each time the Coordinator runs. For every Historical node tier in the cluster, the Coordinator node will determine the Historical node with the highest utilization and the Historical node with the lowest utilization. The percent difference in utilization between the two nodes is computed, and if the result exceeds a certain threshold, a number of segments will be moved from the highest utilized node to the lowest utilized node. There is a configurable limit on the number of segments that can be moved from one node to another each time the Coordinator runs. Segments to be moved are selected at random and only moved if the resulting utilization calculation indicates the percentage difference between the highest and lowest servers has decreased.
|
||||
To ensure an even distribution of segments across Historical processes in the cluster, the Coordinator process will find the total size of all segments being served by every Historical process each time the Coordinator runs. For every Historical process tier in the cluster, the Coordinator process will determine the Historical process with the highest utilization and the Historical process with the lowest utilization. The percent difference in utilization between the two processes is computed, and if the result exceeds a certain threshold, a number of segments will be moved from the highest utilized process to the lowest utilized process. There is a configurable limit on the number of segments that can be moved from one process to another each time the Coordinator runs. Segments to be moved are selected at random and only moved if the resulting utilization calculation indicates the percentage difference between the highest and lowest servers has decreased.
|
||||
|
||||
### Compacting Segments
|
||||
|
||||
Each run, the Druid Coordinator compacts small segments abutting each other. This is useful when you have a lot of small
|
||||
segments which may degrade the query performance as well as increasing the disk space usage.
|
||||
segments which may degrade query performance as well as increase disk space usage. See [Segment Size Optimization](../operations/segment-optimization.html) for details.
|
||||
|
||||
The Coordinator first finds the segments to compact together based on the [segment search policy](#segment-search-policy).
|
||||
Once some segments are found, it launches a [compaction task](../ingestion/tasks.html#compaction-task) to compact those segments.
|
||||
|
@ -113,28 +113,20 @@ If it finds such segments, it simply skips them.
|
|||
|
||||
### The Coordinator Console
|
||||
|
||||
The Druid Coordinator exposes a web GUI for displaying cluster information and rule configuration. After the Coordinator starts, the console can be accessed at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<COORDINATOR_PORT>
|
||||
```
|
||||
|
||||
There exists a full cluster view (which shows only the realtime and Historical nodes), as well as views for individual Historical nodes, datasources and segments themselves. Segment information can be displayed in raw JSON form or as part of a sortable and filterable table.
|
||||
|
||||
The Coordinator console also exposes an interface to creating and editing rules. All valid datasources configured in the segment database, along with a default datasource, are available for configuration. Rules of different types can be added, deleted or edited.
|
||||
The Druid Coordinator exposes a web GUI for displaying cluster information and rule configuration. For more details, please see [coordinator console](../operations/web-consoles.html#coordinator-console).
|
||||
|
||||
### FAQ
|
||||
|
||||
1. **Do clients ever contact the Coordinator node?**
|
||||
1. **Do clients ever contact the Coordinator process?**
|
||||
|
||||
The Coordinator is not involved in a query.
|
||||
|
||||
Historical nodes never directly contact the Coordinator node. The Druid Coordinator tells the Historical nodes to load/drop data via Zookeeper, but the Historical nodes are completely unaware of the Coordinator.
|
||||
Historical processes never directly contact the Coordinator process. The Druid Coordinator tells the Historical processes to load/drop data via Zookeeper, but the Historical processes are completely unaware of the Coordinator.
|
||||
|
||||
Brokers also never contact the Coordinator. Brokers base their understanding of the data topology on metadata exposed by the Historical nodes via ZK and are completely unaware of the Coordinator.
|
||||
Brokers also never contact the Coordinator. Brokers base their understanding of the data topology on metadata exposed by the Historical processes via ZK and are completely unaware of the Coordinator.
|
||||
|
||||
2. **Does it matter if the Coordinator node starts up before or after other processes?**
|
||||
2. **Does it matter if the Coordinator process starts up before or after other processes?**
|
||||
|
||||
No. If the Druid Coordinator is not started up, no new segments will be loaded in the cluster and outdated segments will not be dropped. However, the Coordinator node can be started up at any time, and after a configurable delay, will start running Coordinator tasks.
|
||||
No. If the Druid Coordinator is not started up, no new segments will be loaded in the cluster and outdated segments will not be dropped. However, the Coordinator process can be started up at any time, and after a configurable delay, will start running Coordinator tasks.
|
||||
|
||||
This also means that if you have a working cluster and all of your Coordinators die, the cluster will continue to function, it just won’t experience any changes to its data topology.
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "Historical Node"
|
||||
title: "Historical Process"
|
||||
---
|
||||
|
||||
<!--
|
||||
|
@ -22,11 +22,11 @@ title: "Historical Node"
|
|||
~ under the License.
|
||||
-->
|
||||
|
||||
# Historical Node
|
||||
# Historical Process
|
||||
|
||||
### Configuration
|
||||
|
||||
For Historical Node Configuration, see [Historical Configuration](../configuration/index.html#historical).
|
||||
For Historical Process Configuration, see [Historical Configuration](../configuration/index.html#historical).
|
||||
|
||||
### HTTP Endpoints
|
||||
|
||||
|
@ -40,20 +40,20 @@ org.apache.druid.cli.Main server historical
|
|||
|
||||
### Loading and Serving Segments
|
||||
|
||||
Each Historical node maintains a constant connection to Zookeeper and watches a configurable set of Zookeeper paths for new segment information. Historical nodes do not communicate directly with each other or with the Coordinator nodes but instead rely on Zookeeper for coordination.
|
||||
Each Historical process maintains a constant connection to Zookeeper and watches a configurable set of Zookeeper paths for new segment information. Historical processes do not communicate directly with each other or with the Coordinator processes but instead rely on Zookeeper for coordination.
|
||||
|
||||
The [Coordinator](../design/coordinator.html) node is responsible for assigning new segments to Historical nodes. Assignment is done by creating an ephemeral Zookeeper entry under a load queue path associated with a Historical node. For more information on how the Coordinator assigns segments to Historical nodes, please see [Coordinator](../design/coordinator.html).
|
||||
The [Coordinator](../design/coordinator.html) process is responsible for assigning new segments to Historical processes. Assignment is done by creating an ephemeral Zookeeper entry under a load queue path associated with a Historical process. For more information on how the Coordinator assigns segments to Historical processes, please see [Coordinator](../design/coordinator.html).
|
||||
|
||||
When a Historical node notices a new load queue entry in its load queue path, it will first check a local disk directory (cache) for the information about segment. If no information about the segment exists in the cache, the Historical node will download metadata about the new segment to serve from Zookeeper. This metadata includes specifications about where the segment is located in deep storage and about how to decompress and process the segment. For more information about segment metadata and Druid segments in general, please see [Segments](../design/segments.html). Once a Historical node completes processing a segment, the segment is announced in Zookeeper under a served segments path associated with the node. At this point, the segment is available for querying.
|
||||
When a Historical process notices a new load queue entry in its load queue path, it will first check a local disk directory (cache) for the information about segment. If no information about the segment exists in the cache, the Historical process will download metadata about the new segment to serve from Zookeeper. This metadata includes specifications about where the segment is located in deep storage and about how to decompress and process the segment. For more information about segment metadata and Druid segments in general, please see [Segments](../design/segments.html). Once a Historical process completes processing a segment, the segment is announced in Zookeeper under a served segments path associated with the process. At this point, the segment is available for querying.
|
||||
|
||||
### Loading and Serving Segments From Cache
|
||||
|
||||
Recall that when a Historical node notices a new segment entry in its load queue path, the Historical node first checks a configurable cache directory on its local disk to see if the segment had been previously downloaded. If a local cache entry already exists, the Historical node will directly read the segment binary files from disk and load the segment.
|
||||
Recall that when a Historical process notices a new segment entry in its load queue path, the Historical process first checks a configurable cache directory on its local disk to see if the segment had been previously downloaded. If a local cache entry already exists, the Historical process will directly read the segment binary files from disk and load the segment.
|
||||
|
||||
The segment cache is also leveraged when a Historical node is first started. On startup, a Historical node will search through its cache directory and immediately load and serve all segments that are found. This feature allows Historical nodes to be queried as soon they come online.
|
||||
The segment cache is also leveraged when a Historical process is first started. On startup, a Historical process will search through its cache directory and immediately load and serve all segments that are found. This feature allows Historical processes to be queried as soon they come online.
|
||||
|
||||
### Querying Segments
|
||||
|
||||
Please see [Querying](../querying/querying.html) for more information on querying Historical nodes.
|
||||
Please see [Querying](../querying/querying.html) for more information on querying Historical processes.
|
||||
|
||||
A Historical can be configured to log and report metrics for every query it services.
|
||||
|
|
|
@ -29,7 +29,7 @@ The indexing service is a highly-available, distributed service that runs indexi
|
|||
Indexing [tasks](../ingestion/tasks.html) create (and sometimes destroy) Druid [segments](../design/segments.html). The indexing service has a master/slave like architecture.
|
||||
|
||||
The indexing service is composed of three main components: a [Peon](../design/peons.html) component that can run a single task, a [Middle Manager](../design/middlemanager.html) component that manages Peons, and an [Overlord](../design/overlord.html) component that manages task distribution to MiddleManagers.
|
||||
Overlords and MiddleManagers may run on the same node or across multiple nodes while MiddleManagers and Peons always run on the same node.
|
||||
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](../operations/api-reference.html#overlord-tasks) for more information.
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "MiddleManager Node"
|
||||
title: "MiddleManager Process"
|
||||
---
|
||||
|
||||
<!--
|
||||
|
@ -22,11 +22,11 @@ title: "MiddleManager Node"
|
|||
~ under the License.
|
||||
-->
|
||||
|
||||
# MiddleManager Node
|
||||
# MiddleManager Process
|
||||
|
||||
### Configuration
|
||||
|
||||
For Middlemanager Node Configuration, see [Indexing Service Configuration](../configuration/index.html#middlemanager-and-peons).
|
||||
For Middlemanager Process Configuration, see [Indexing Service Configuration](../configuration/index.html#middlemanager-and-peons).
|
||||
|
||||
### HTTP Endpoints
|
||||
|
||||
|
@ -34,7 +34,7 @@ For a list of API endpoints supported by the MiddleManager, please see the [API
|
|||
|
||||
### Overview
|
||||
|
||||
The MiddleManager node is a worker node that executes submitted tasks. Middle Managers forward tasks to Peons that run in separate JVMs.
|
||||
The MiddleManager process is a worker process that executes submitted tasks. Middle Managers forward tasks to Peons that run in separate JVMs.
|
||||
The reason we have separate JVMs for tasks is for resource and log isolation. Each [Peon](../design/peons.html) is capable of running only one task at a time, however, a MiddleManager may have multiple Peons.
|
||||
|
||||
### Running
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "Overlord Node"
|
||||
title: "Overlord Process"
|
||||
---
|
||||
|
||||
<!--
|
||||
|
@ -22,11 +22,11 @@ title: "Overlord Node"
|
|||
~ under the License.
|
||||
-->
|
||||
|
||||
# Overlord Node
|
||||
# Overlord Process
|
||||
|
||||
### Configuration
|
||||
|
||||
For Overlord Node Configuration, see [Overlord Configuration](../configuration/index.html#overlord).
|
||||
For Overlord Process Configuration, see [Overlord Configuration](../configuration/index.html#overlord).
|
||||
|
||||
### HTTP Endpoints
|
||||
|
||||
|
@ -34,22 +34,18 @@ For a list of API endpoints supported by the Overlord, please see the [API refer
|
|||
|
||||
### Overview
|
||||
|
||||
The Overlord node is responsible for accepting tasks, coordinating task distribution, creating locks around tasks, and returning statuses to callers. Overlord can be configured to run in one of two modes - local or remote (local being default).
|
||||
The Overlord process is responsible for accepting tasks, coordinating task distribution, creating locks around tasks, and returning statuses to callers. Overlord can be configured to run in one of two modes - local or remote (local being default).
|
||||
In local mode Overlord is also responsible for creating Peons for executing tasks. When running the Overlord in local mode, all MiddleManager and Peon configurations must be provided as well.
|
||||
Local mode is typically used for simple workflows. In remote mode, the Overlord and MiddleManager are run in separate processes and you can run each on a different server.
|
||||
This mode is recommended if you intend to use the indexing service as the single endpoint for all Druid indexing.
|
||||
|
||||
### Overlord Console
|
||||
|
||||
The Overlord console can be used to view pending tasks, running tasks, available workers, and recent worker creation and termination. The console can be accessed at:
|
||||
|
||||
```
|
||||
http://<OVERLORD_IP>:<port>/console.html
|
||||
```
|
||||
The Overlord provides a UI for managing tasks and workers. For more details, please see [overlord console](../operations/web-consoles.html#overlord-console).
|
||||
|
||||
### Blacklisted Workers
|
||||
|
||||
If the workers fail tasks above a threshold, the Overlord will blacklist these workers. No more than 20% of the nodes can be blacklisted. Blacklisted nodes will be periodically whitelisted.
|
||||
If a MiddleManager has task failures above a threshold, the Overlord will blacklist these MiddleManagers. No more than 20% of the MiddleManagers can be blacklisted. Blacklisted MiddleManagers will be periodically whitelisted.
|
||||
|
||||
The following vairables can be used to set the threshold and blacklist timeouts.
|
||||
|
||||
|
@ -62,6 +58,6 @@ druid.indexer.runner.maxPercentageBlacklistWorkers
|
|||
|
||||
### Autoscaling
|
||||
|
||||
The Autoscaling mechanisms currently in place are tightly coupled with our deployment infrastructure but the framework should be in place for other implementations. We are highly open to new implementations or extensions of the existing mechanisms. In our own deployments, MiddleManager nodes are Amazon AWS EC2 nodes and they are provisioned to register themselves in a [galaxy](https://github.com/ning/galaxy) environment.
|
||||
The Autoscaling mechanisms currently in place are tightly coupled with our deployment infrastructure but the framework should be in place for other implementations. We are highly open to new implementations or extensions of the existing mechanisms. In our own deployments, MiddleManager processes are Amazon AWS EC2 nodes and they are provisioned to register themselves in a [galaxy](https://github.com/ning/galaxy) environment.
|
||||
|
||||
If autoscaling is enabled, new MiddleManagers may be added when a task has been in pending state for too long. Middle managers may be terminated if they have not run any tasks for a period of time.
|
||||
If autoscaling is enabled, new MiddleManagers may be added when a task has been in pending state for too long. MiddleManagers may be terminated if they have not run any tasks for a period of time.
|
||||
|
|
|
@ -78,6 +78,8 @@ caller. End users typically query Brokers rather than querying Historicals or Mi
|
|||
Overlords, and Coordinators. They are optional since you can also simply contact the Druid Brokers, Overlords, and
|
||||
Coordinators directly.
|
||||
|
||||
The Router also runs the [Druid Console](../operations/management-uis.html#druid-console), a management UI for datasources, segments, tasks, data processes (Historicals and MiddleManagers), and coordinator dynamic configuration. The user can also run SQL and native Druid queries within the console.
|
||||
|
||||
### Data server
|
||||
|
||||
A Data server executes ingestion jobs and stores queryable data.
|
||||
|
@ -124,6 +126,6 @@ Please see [Coordinator Configuration: Operation](../configuration/index.html#co
|
|||
|
||||
### Historicals and MiddleManagers
|
||||
|
||||
With higher levels of ingestion or query load, it can make sense to deploy the Historical and MiddleManager processes on separate nodes to to avoid CPU and memory contention.
|
||||
With higher levels of ingestion or query load, it can make sense to deploy the Historical and MiddleManager processes on separate hosts to to avoid CPU and memory contention.
|
||||
|
||||
The Historical also benefits from having free memory for memory mapped segments, which can be another reason to deploy the Historical and MiddleManager processes separately.
|
|
@ -1,6 +1,6 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "Real-time Node"
|
||||
title: "Real-time Process"
|
||||
---
|
||||
|
||||
<!--
|
||||
|
@ -22,17 +22,17 @@ title: "Real-time Node"
|
|||
~ under the License.
|
||||
-->
|
||||
|
||||
# Real-time Node
|
||||
# Real-time Process
|
||||
|
||||
<div class="note info">
|
||||
NOTE: Realtime nodes are deprecated. Please use the <a href="../development/extensions-core/kafka-ingestion.html">Kafka Indexing Service</a> for stream pull use cases instead.
|
||||
NOTE: Realtime processes are deprecated. Please use the <a href="../development/extensions-core/kafka-ingestion.html">Kafka Indexing Service</a> for stream pull use cases instead.
|
||||
</div>
|
||||
|
||||
For Real-time Node Configuration, see [Realtime Configuration](../configuration/realtime.html).
|
||||
For Real-time Process Configuration, see [Realtime Configuration](../configuration/realtime.html).
|
||||
|
||||
For Real-time Ingestion, see [Realtime Ingestion](../ingestion/stream-ingestion.html).
|
||||
|
||||
Realtime nodes provide a realtime index. Data indexed via these nodes is immediately available for querying. Realtime nodes will periodically build segments representing the data they’ve collected over some span of time and transfer these segments off to [Historical](../design/historical.html) nodes. They use ZooKeeper to monitor the transfer and the metadata storage to store metadata about the transferred segment. Once transfered, segments are forgotten by the Realtime nodes.
|
||||
Realtime processes provide a realtime index. Data indexed via these processes is immediately available for querying. Realtime processes will periodically build segments representing the data they’ve collected over some span of time and transfer these segments off to [Historical](../design/historical.html) processes. They use ZooKeeper to monitor the transfer and the metadata storage to store metadata about the transferred segment. Once transfered, segments are forgotten by the Realtime processes.
|
||||
|
||||
### Running
|
||||
|
||||
|
@ -71,10 +71,10 @@ Given those expectations, adding a firehose is straightforward and completely en
|
|||
HTTP Endpoints
|
||||
--------------
|
||||
|
||||
The real-time node exposes several HTTP endpoints for interactions.
|
||||
The real-time process exposes several HTTP endpoints for interactions.
|
||||
|
||||
### GET
|
||||
|
||||
* `/status`
|
||||
|
||||
Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the node.
|
||||
Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the process.
|
||||
|
|
|
@ -36,4 +36,4 @@ To enable experimental features, include their artifacts in the configuration ru
|
|||
druid.extensions.loadList=["druid-histogram"]
|
||||
```
|
||||
|
||||
The configuration files for all the indexer and query nodes need to be updated with this.
|
||||
The configuration files for all the Druid processes need to be updated with this.
|
||||
|
|
|
@ -28,7 +28,7 @@ To use this extension, make sure to [include](../../operations/including-extensi
|
|||
|
||||
## Firehose
|
||||
|
||||
This is an experimental firehose to ingest data from kafka using kafka simple consumer api. Currently, this firehose would only work inside standalone realtime nodes.
|
||||
This is an experimental firehose to ingest data from kafka using kafka simple consumer api. Currently, this firehose would only work inside standalone realtime processes.
|
||||
The configuration for KafkaSimpleConsumerFirehose is similar to the Kafka Eight Firehose , except `firehose` should be replaced with `firehoseV2` like this:
|
||||
|
||||
```json
|
||||
|
|
|
@ -131,4 +131,4 @@ There are 2 parts in a view query:
|
|||
|queryType |The query type. This should always be view |yes|
|
||||
|query |The real query of this `view` query. The real query must be [groupBy](../../querying/groupbyquery.html), [topN](../../querying/topnquery.html), or [timeseries](../../querying/timeseriesquery.html) type.|yes|
|
||||
|
||||
**Note that Materialized View is currently designated as experimental. Please make sure the time of all nodes are the same and increase monotonically. Otherwise, some unexpected errors may happen on query results.**
|
||||
**Note that Materialized View is currently designated as experimental. Please make sure the time of all processes are the same and increase monotonically. Otherwise, some unexpected errors may happen on query results.**
|
||||
|
|
|
@ -42,7 +42,7 @@ All the configuration parameters for the opentsdb emitter are under `druid.emitt
|
|||
|`druid.emitter.opentsdb.readTimeout`|`Jersey client` read timeout(in milliseconds).|no|2000|
|
||||
|`druid.emitter.opentsdb.flushThreshold`|Queue flushing threshold.(Events will be sent as one batch)|no|100|
|
||||
|`druid.emitter.opentsdb.maxQueueSize`|Maximum size of the queue used to buffer events.|no|1000|
|
||||
|`druid.emitter.opentsdb.consumeDelay`|Queue consuming delay(in milliseconds). Actually, we use `ScheduledExecutorService` to schedule consuming events, so this `consumeDelay` means the delay between the termination of one execution and the commencement of the next. If your druid nodes produce metric events fast, then you should decrease this `consumeDelay` or increase the `maxQueueSize`.|no|10000|
|
||||
|`druid.emitter.opentsdb.consumeDelay`|Queue consuming delay(in milliseconds). Actually, we use `ScheduledExecutorService` to schedule consuming events, so this `consumeDelay` means the delay between the termination of one execution and the commencement of the next. If your druid processes produce metric events fast, then you should decrease this `consumeDelay` or increase the `maxQueueSize`.|no|10000|
|
||||
|`druid.emitter.opentsdb.metricMapPath`|JSON file defining the desired metrics and dimensions for every Druid metric|no|./src/main/resources/defaultMetrics.json|
|
||||
|
||||
### Druid to OpenTSDB Event Converter
|
||||
|
|
|
@ -29,9 +29,9 @@ A cache implementation for Druid based on [Redis](https://github.com/antirez/red
|
|||
# Configuration
|
||||
Below are the configuration options known to this module.
|
||||
|
||||
Note that just adding these properties does not enable the cache. You still need to add the `druid.<nodetype>.cache.useCache` and `druid.<nodetype>.cache.populateCache` properties for the nodes you want to enable the cache on as described in the [cache configuration docs](../../configuration/index.html#cache-configuration).
|
||||
Note that just adding these properties does not enable the cache. You still need to add the `druid.<process-type>.cache.useCache` and `druid.<process-type>.cache.populateCache` properties for the processes you want to enable the cache on as described in the [cache configuration docs](../../configuration/index.html#cache-configuration).
|
||||
|
||||
A possible configuration would be to keep the properties below in your `common.runtime.properties` file (present on all nodes) and then add `druid.<nodetype>.cache.useCache` and `druid.<nodetype>.cache.populateCache` in the `runtime.properties` file of the node types you want to enable caching on.
|
||||
A possible configuration would be to keep the properties below in your `common.runtime.properties` file (present on all processes) and then add `druid.<nodetype>.cache.useCache` and `druid.<nodetype>.cache.populateCache` in the `runtime.properties` file of the process types you want to enable caching on.
|
||||
|
||||
|
||||
|`common.runtime.properties`|Description|Default|Required|
|
||||
|
|
|
@ -30,6 +30,10 @@ This extension enables Druid to ingest thrift compact data online (`ByteBuffer`)
|
|||
|
||||
You may want to use another version of thrift, change the dependency in pom and compile yourself.
|
||||
|
||||
## LZO Support
|
||||
|
||||
If you plan to read LZO-compressed Thrift files, you will need to download version 0.4.19 of the [hadoop-lzo JAR](https://mvnrepository.com/artifact/com.hadoop.gplcompression/hadoop-lzo/0.4.19) and place it in your `extensions/druid-thrift-extensions` directory.
|
||||
|
||||
## Thrift Parser
|
||||
|
||||
|
||||
|
|
|
@ -43,7 +43,7 @@ These configuration properties should be added to the common runtime properties
|
|||
### Properties
|
||||
|Property|Description|Default|required|
|
||||
|--------|-----------|-------|--------|
|
||||
|`druid.auth.basic.common.pollingPeriod`|Defines in milliseconds how often nodes should poll the Coordinator for the current authenticator/authorizer database state.|60000|No|
|
||||
|`druid.auth.basic.common.pollingPeriod`|Defines in milliseconds how often processes should poll the Coordinator for the current authenticator/authorizer database state.|60000|No|
|
||||
|`druid.auth.basic.common.maxRandomDelay`|Defines in milliseconds the amount of random delay to add to the pollingPeriod, to spread polling requests across time.|6000|No|
|
||||
|`druid.auth.basic.common.maxSyncRetries`|Determines how many times a service will retry if the authentication/authorization database state sync with the Coordinator fails.|10|No|
|
||||
|`druid.auth.basic.common.cacheDirectory`|If defined, snapshots of the basic Authenticator and Authorizer database caches will be stored on disk in this directory. If this property is defined, when a service is starting, it will attempt to initialize its caches from these on-disk snapshots, if the service is unable to initialize its state by communicating with the Coordinator.|null|No|
|
||||
|
@ -74,8 +74,8 @@ The configuration examples in the rest of this document will use "MyBasicAuthent
|
|||
|Property|Description|Default|required|
|
||||
|--------|-----------|-------|--------|
|
||||
|`druid.auth.authenticator.MyBasicAuthenticator.initialAdminPassword`|Initial [Password Provider](../../operations/password-provider.html) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.|null|No|
|
||||
|`druid.auth.authenticator.MyBasicAuthenticator.initialInternalClientPassword`|Initial [Password Provider](../../operations/password-provider.html) for the default internal system user, used for internal node communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.|null|No|
|
||||
|`druid.auth.authenticator.MyBasicAuthenticator.enableCacheNotifications`|If true, the Coordinator will notify Druid nodes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.|true|No|
|
||||
|`druid.auth.authenticator.MyBasicAuthenticator.initialInternalClientPassword`|Initial [Password Provider](../../operations/password-provider.html) for the default internal system user, used for internal process communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.|null|No|
|
||||
|`druid.auth.authenticator.MyBasicAuthenticator.enableCacheNotifications`|If true, the Coordinator will notify Druid processes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.|true|No|
|
||||
|`druid.auth.authenticator.MyBasicAuthenticator.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No|
|
||||
|`druid.auth.authenticator.MyBasicAuthenticator.credentialIterations`|Number of iterations to use for password hashing.|10000|No|
|
||||
|`druid.auth.authenticator.MyBasicAuthenticator.authorizerName`|Authorizer that requests should be directed to|N/A|Yes|
|
||||
|
@ -116,7 +116,7 @@ druid.auth.authorizer.<authorizerName>.<authorizerProperty>
|
|||
#### Properties
|
||||
|Property|Description|Default|required|
|
||||
|--------|-----------|-------|--------|
|
||||
|`druid.auth.authorizer.MyBasicAuthorizer.enableCacheNotifications`|If true, the Coordinator will notify Druid nodes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.|true|No|
|
||||
|`druid.auth.authorizer.MyBasicAuthorizer.enableCacheNotifications`|If true, the Coordinator will notify Druid processes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.|true|No|
|
||||
|`druid.auth.authorizer.MyBasicAuthorizer.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No|
|
||||
|
||||
## Usage
|
||||
|
@ -260,7 +260,7 @@ There are two possible resource names for the "CONFIG" resource type, "CONFIG" a
|
|||
|
||||
"CONFIG" resource name covers the following endpoints:
|
||||
|
||||
|Endpoint|Node Type|
|
||||
|Endpoint|Process Type|
|
||||
|--------|---------|
|
||||
|`/druid/coordinator/v1/config`|coordinator|
|
||||
|`/druid/indexer/v1/worker`|overlord|
|
||||
|
@ -270,7 +270,7 @@ There are two possible resource names for the "CONFIG" resource type, "CONFIG" a
|
|||
|
||||
"security" resource name covers the following endpoint:
|
||||
|
||||
|Endpoint|Node Type|
|
||||
|Endpoint|Process Type|
|
||||
|--------|---------|
|
||||
|`/druid-ext/basic-security/authentication`|coordinator|
|
||||
|`/druid-ext/basic-security/authorization`|coordinator|
|
||||
|
@ -280,7 +280,7 @@ There is only one possible resource name for the "STATE" config resource type, "
|
|||
|
||||
"STATE" resource name covers the following endpoints:
|
||||
|
||||
|Endpoint|Node Type|
|
||||
|Endpoint|Process Type|
|
||||
|--------|---------|
|
||||
|`/druid/coordinator/v1`|coordinator|
|
||||
|`/druid/coordinator/v1/rules`|coordinator|
|
||||
|
@ -312,10 +312,10 @@ GET requires READ permission, while POST and DELETE require WRITE permission.
|
|||
|
||||
## Configuration Propagation
|
||||
|
||||
To prevent excessive load on the Coordinator, the Authenticator and Authorizer user/role database state is cached on each Druid node.
|
||||
To prevent excessive load on the Coordinator, the Authenticator and Authorizer user/role database state is cached on each Druid process.
|
||||
|
||||
Each node will periodically poll the Coordinator for the latest database state, controlled by the `druid.auth.basic.common.pollingPeriod` and `druid.auth.basic.common.maxRandomDelay` properties.
|
||||
Each process will periodically poll the Coordinator for the latest database state, controlled by the `druid.auth.basic.common.pollingPeriod` and `druid.auth.basic.common.maxRandomDelay` properties.
|
||||
|
||||
When a configuration update occurs, the Coordinator can optionally notify each node with the updated database state. This behavior is controlled by the `enableCacheNotifications` and `cacheNotificationTimeout` properties on Authenticators and Authorizers.
|
||||
When a configuration update occurs, the Coordinator can optionally notify each process with the updated database state. This behavior is controlled by the `enableCacheNotifications` and `cacheNotificationTimeout` properties on Authenticators and Authorizers.
|
||||
|
||||
Note that because of the caching, changes made to the user/role database may not be immediately reflected at each Druid node.
|
||||
Note that because of the caching, changes made to the user/role database may not be immediately reflected at each Druid process.
|
||||
|
|
|
@ -24,7 +24,7 @@ title: "Kerberos"
|
|||
|
||||
# Kerberos
|
||||
|
||||
Druid Extension to enable Authentication for Druid Nodes using Kerberos.
|
||||
Druid Extension to enable Authentication for Druid Processes using Kerberos.
|
||||
This extension adds an Authenticator which is used to protect HTTP Endpoints using the simple and protected GSSAPI negotiation mechanism [SPNEGO](https://en.wikipedia.org/wiki/SPNEGO).
|
||||
Make sure to [include](../../operations/including-extensions.html) `druid-kerberos` as an extension.
|
||||
|
||||
|
@ -51,14 +51,14 @@ The configuration examples in the rest of this document will use "kerberos" as t
|
|||
### Properties
|
||||
|Property|Possible Values|Description|Default|required|
|
||||
|--------|---------------|-----------|-------|--------|
|
||||
|`druid.auth.authenticator.kerberos.serverPrincipal`|`HTTP/_HOST@EXAMPLE.COM`| SPNego service principal used by druid nodes|empty|Yes|
|
||||
|`druid.auth.authenticator.kerberos.serverKeytab`|`/etc/security/keytabs/spnego.service.keytab`|SPNego service keytab used by druid nodes|empty|Yes|
|
||||
|`druid.auth.authenticator.kerberos.serverPrincipal`|`HTTP/_HOST@EXAMPLE.COM`| SPNego service principal used by druid processes|empty|Yes|
|
||||
|`druid.auth.authenticator.kerberos.serverKeytab`|`/etc/security/keytabs/spnego.service.keytab`|SPNego service keytab used by druid processes|empty|Yes|
|
||||
|`druid.auth.authenticator.kerberos.authToLocal`|`RULE:[1:$1@$0](druid@EXAMPLE.COM)s/.*/druid DEFAULT`|It allows you to set a general rule for mapping principal names to local user names. It will be used if there is not an explicit mapping for the principal name that is being translated.|DEFAULT|No|
|
||||
|`druid.auth.authenticator.kerberos.excludedPaths`|`['/status','/health']`| Array of HTTP paths which which does NOT need to be authenticated.|None|No|
|
||||
|`druid.auth.authenticator.kerberos.cookieSignatureSecret`|`secretString`| Secret used to sign authentication cookies. It is advisable to explicitly set it, if you have multiple druid ndoes running on same machine with different ports as the Cookie Specification does not guarantee isolation by port.|<Random value>|No|
|
||||
|`druid.auth.authenticator.kerberos.authorizerName`|Depends on available authorizers|Authorizer that requests should be directed to|Empty|Yes|
|
||||
|
||||
As a note, it is required that the SPNego principal in use by the druid nodes must start with HTTP (This specified by [RFC-4559](https://tools.ietf.org/html/rfc4559)) and must be of the form "HTTP/_HOST@REALM".
|
||||
As a note, it is required that the SPNego principal in use by the druid processes must start with HTTP (This specified by [RFC-4559](https://tools.ietf.org/html/rfc4559)) and must be of the form "HTTP/_HOST@REALM".
|
||||
The special string _HOST will be replaced automatically with the value of config `druid.host`
|
||||
|
||||
### Auth to Local Syntax
|
||||
|
@ -74,14 +74,14 @@ In such cases, max request header size that druid can handle can be increased by
|
|||
|
||||
## Configuring Kerberos Escalated Client
|
||||
|
||||
Druid internal nodes communicate with each other using an escalated http Client. A Kerberos enabled escalated HTTP Client can be configured by following properties -
|
||||
Druid internal processes communicate with each other using an escalated http Client. A Kerberos enabled escalated HTTP Client can be configured by following properties -
|
||||
|
||||
|
||||
|Property|Example Values|Description|Default|required|
|
||||
|--------|---------------|-----------|-------|--------|
|
||||
|`druid.escalator.type`|`kerberos`| Type of Escalator client used for internal node communication.|n/a|Yes|
|
||||
|`druid.escalator.internalClientPrincipal`|`druid@EXAMPLE.COM`| Principal user name, used for internal node communication|n/a|Yes|
|
||||
|`druid.escalator.internalClientKeytab`|`/etc/security/keytabs/druid.keytab`|Path to keytab file used for internal node communication|n/a|Yes|
|
||||
|`druid.escalator.type`|`kerberos`| Type of Escalator client used for internal process communication.|n/a|Yes|
|
||||
|`druid.escalator.internalClientPrincipal`|`druid@EXAMPLE.COM`| Principal user name, used for internal process communication|n/a|Yes|
|
||||
|`druid.escalator.internalClientKeytab`|`/etc/security/keytabs/druid.keytab`|Path to keytab file used for internal process communication|n/a|Yes|
|
||||
|`druid.escalator.authorizerName`|`MyBasicAuthorizer`|Authorizer that requests should be directed to.|n/a|Yes|
|
||||
|
||||
## Accessing Druid HTTP end points when kerberos security is enabled
|
||||
|
|
|
@ -192,7 +192,7 @@ For Roaring bitmaps:
|
|||
|`topic`|String|The Kafka topic to read from. This must be a specific topic as topic patterns are not supported.|yes|
|
||||
|`consumerProperties`|Map<String, Object>|A map of properties to be passed to the Kafka consumer. This must contain a property `bootstrap.servers` with a list of Kafka brokers in the form: `<BROKER_1>:<PORT_1>,<BROKER_2>:<PORT_2>,...`. For SSL connections, the `keystore`, `truststore` and `key` passwords can be provided as a [Password Provider](../../operations/password-provider.html) or String password.|yes|
|
||||
|`pollTimeout`|Long|The length of time to wait for the kafka consumer to poll records, in milliseconds|no (default == 100)|
|
||||
|`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against node failure.|no (default == 1)|
|
||||
|`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against process failure.|no (default == 1)|
|
||||
|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See 'Capacity Planning' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKafkaPartitions}`.|no (default == 1)|
|
||||
|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment.|no (default == PT1H)|
|
||||
|`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)|
|
||||
|
@ -278,7 +278,7 @@ in data loss (assuming the tasks run before Kafka purges those offsets).
|
|||
|
||||
A running task will normally be in one of two states: *reading* or *publishing*. A task will remain in reading state for
|
||||
`taskDuration`, at which point it will transition to publishing state. A task will remain in publishing state for as long
|
||||
as it takes to generate segments, push segments to deep storage, and have them be loaded and served by a Historical node
|
||||
as it takes to generate segments, push segments to deep storage, and have them be loaded and served by a Historical process
|
||||
(or until `completionTimeout` elapses).
|
||||
|
||||
The number of reading tasks is controlled by `replicas` and `taskCount`. In general, there will be `replicas * taskCount`
|
||||
|
@ -335,7 +335,7 @@ for this segment granularity is created for further events. Kafka Indexing Task
|
|||
means that all the segments created by a task will not be held up till the task duration is over. As soon as maxRowsPerSegment,
|
||||
maxTotalRows or intermediateHandoffPeriod limit is hit, all the segments held by the task at that point in time will be handed-off
|
||||
and new set of segments will be created for further events. This means that the task can run for longer durations of time
|
||||
without accumulating old segments locally on Middle Manager nodes and it is encouraged to do so.
|
||||
without accumulating old segments locally on Middle Manager processes and it is encouraged to do so.
|
||||
|
||||
Kafka Indexing Service may still produce some small segments. Lets say the task duration is 4 hours, segment granularity
|
||||
is set to an HOUR and Supervisor was started at 9:10 then after 4 hours at 13:10, new set of tasks will be started and
|
||||
|
|
|
@ -192,7 +192,7 @@ For Roaring bitmaps:
|
|||
|-----|----|-----------|--------|
|
||||
|`stream`|String|The Kinesis stream to read.|yes|
|
||||
|`endpoint`|String|The AWS Kinesis stream endpoint for a region. You can find a list of endpoints [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region).|no (default == kinesis.us-east-1.amazonaws.com)|
|
||||
|`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against node failure.|no (default == 1)|
|
||||
|`replicas`|Integer|The number of replica sets, where 1 means a single set of tasks (no replication). Replica tasks will always be assigned to different workers to provide resiliency against process failure.|no (default == 1)|
|
||||
|`taskCount`|Integer|The maximum number of *reading* tasks in a *replica set*. This means that the maximum number of reading tasks will be `taskCount * replicas` and the total number of tasks (*reading* + *publishing*) will be higher than this. See 'Capacity Planning' below for more details. The number of reading tasks will be less than `taskCount` if `taskCount > {numKinesisshards}`.|no (default == 1)|
|
||||
|`taskDuration`|ISO8601 Period|The length of time before tasks stop reading and begin publishing their segment.|no (default == PT1H)|
|
||||
|`startDelay`|ISO8601 Period|The period to wait before the supervisor starts managing tasks.|no (default == PT5S)|
|
||||
|
@ -282,7 +282,7 @@ in data loss (assuming the tasks run before Kinesis purges those sequence number
|
|||
|
||||
A running task will normally be in one of two states: *reading* or *publishing*. A task will remain in reading state for
|
||||
`taskDuration`, at which point it will transition to publishing state. A task will remain in publishing state for as long
|
||||
as it takes to generate segments, push segments to deep storage, and have them be loaded and served by a Historical node
|
||||
as it takes to generate segments, push segments to deep storage, and have them be loaded and served by a Historical process
|
||||
(or until `completionTimeout` elapses).
|
||||
|
||||
The number of reading tasks is controlled by `replicas` and `taskCount`. In general, there will be `replicas * taskCount`
|
||||
|
@ -339,7 +339,7 @@ for this segment granularity is created for further events. Kinesis Indexing Tas
|
|||
means that all the segments created by a task will not be held up till the task duration is over. As soon as maxRowsPerSegment,
|
||||
maxTotalRows or intermediateHandoffPeriod limit is hit, all the segments held by the task at that point in time will be handed-off
|
||||
and new set of segments will be created for further events. This means that the task can run for longer durations of time
|
||||
without accumulating old segments locally on Middle Manager nodes and it is encouraged to do so.
|
||||
without accumulating old segments locally on Middle Manager processes and it is encouraged to do so.
|
||||
|
||||
Kinesis Indexing Service may still produce some small segments. Lets say the task duration is 4 hours, segment granularity
|
||||
is set to an HOUR and Supervisor was started at 9:10 then after 4 hours at 13:10, new set of tasks will be started and
|
||||
|
|
|
@ -38,9 +38,9 @@ Static configuration is no longer supported. Lookups can be configured through
|
|||
|
||||
Globally cached lookups are appropriate for lookups which are not possible to pass at query time due to their size,
|
||||
or are not desired to be passed at query time because the data is to reside in and be handled by the Druid servers,
|
||||
and are small enough to reasonably populate on a node. This usually means tens to tens of thousands of entries per lookup.
|
||||
and are small enough to reasonably populate in-memory. This usually means tens to tens of thousands of entries per lookup.
|
||||
|
||||
Globally cached lookups all draw from the same cache pool, allowing each node to have a fixed cache pool that can be used by cached lookups.
|
||||
Globally cached lookups all draw from the same cache pool, allowing each process to have a fixed cache pool that can be used by cached lookups.
|
||||
|
||||
Globally cached lookups can be specified as part of the [cluster wide config for lookups](../../querying/lookups.html) as a type of `cachedNamespace`
|
||||
|
||||
|
@ -93,9 +93,9 @@ The parameters are as follows
|
|||
|`firstCacheTimeout`|How long to wait (in ms) for the first run of the cache to populate. 0 indicates to not wait|No|`0` (do not wait)|
|
||||
|`injective`|If the underlying map is [injective](../../querying/lookups.html#query-execution) (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`|
|
||||
|
||||
If `firstCacheTimeout` is set to a non-zero value, it should be less than `druid.manager.lookups.hostUpdateTimeout`. If `firstCacheTimeout` is NOT set, then management is essentially asynchronous and does not know if a lookup succeeded or failed in starting. In such a case logs from the lookup nodes should be monitored for repeated failures.
|
||||
If `firstCacheTimeout` is set to a non-zero value, it should be less than `druid.manager.lookups.hostUpdateTimeout`. If `firstCacheTimeout` is NOT set, then management is essentially asynchronous and does not know if a lookup succeeded or failed in starting. In such a case logs from the processes using lookups should be monitored for repeated failures.
|
||||
|
||||
Proper functionality of globally cached lookups requires the following extension to be loaded on the Broker, Peon, and Historical nodes:
|
||||
Proper functionality of globally cached lookups requires the following extension to be loaded on the Broker, Peon, and Historical processes:
|
||||
`druid-lookups-cached-global`
|
||||
|
||||
## Example configuration
|
||||
|
@ -160,7 +160,7 @@ Where the Coordinator endpoint `/druid/coordinator/v1/lookups/realtime_customer2
|
|||
|
||||
## Cache Settings
|
||||
|
||||
Lookups are cached locally on Historical nodes. The following are settings used by the nodes which service queries when
|
||||
Lookups are cached locally on Historical processes. The following are settings used by the processes which service queries when
|
||||
setting namespaces (Broker, Peon, Historical)
|
||||
|
||||
|Property|Description|Default|
|
||||
|
|
|
@ -25,7 +25,7 @@ title: "Simple SSLContext Provider Module"
|
|||
# Simple SSLContext Provider Module
|
||||
|
||||
This module contains a simple implementation of [SSLContext](http://docs.oracle.com/javase/8/docs/api/javax/net/ssl/SSLContext.html)
|
||||
that will be injected to be used with HttpClient that Druid nodes use internally to communicate with each other. To learn more about
|
||||
that will be injected to be used with HttpClient that Druid processes use internally to communicate with each other. To learn more about
|
||||
Java's SSL support, please refer to [this](http://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html) guide.
|
||||
|
||||
# Configuration
|
||||
|
|
|
@ -52,7 +52,7 @@ Core extensions are maintained by Druid committers.
|
|||
|druid-kafka-extraction-namespace|Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../development/extensions-core/kafka-extraction-namespace.html)|
|
||||
|druid-kafka-indexing-service|Supervised exactly-once Kafka ingestion for the indexing service.|[link](../development/extensions-core/kafka-ingestion.html)|
|
||||
|druid-kinesis-indexing-service|Supervised exactly-once Kinesis ingestion for the indexing service.|[link](../development/extensions-core/kinesis-ingestion.html)|
|
||||
|druid-kerberos|Kerberos authentication for druid nodes.|[link](../development/extensions-core/druid-kerberos.html)|
|
||||
|druid-kerberos|Kerberos authentication for druid processes.|[link](../development/extensions-core/druid-kerberos.html)|
|
||||
|druid-lookups-cached-global|A module for [lookups](../querying/lookups.html) providing a jvm-global eager caching for lookups. It provides JDBC and URI implementations for fetching lookup data.|[link](../development/extensions-core/lookups-cached-global.html)|
|
||||
|druid-lookups-cached-single| Per lookup caching module to support the use cases where a lookup need to be isolated from the global pool of lookups |[link](../development/extensions-core/druid-lookups.html)|
|
||||
|druid-parquet-extensions|Support for data in Apache Parquet data format. Requires druid-avro-extensions to be loaded.|[link](../development/extensions-core/parquet.html)|
|
||||
|
@ -61,7 +61,7 @@ Core extensions are maintained by Druid committers.
|
|||
|druid-stats|Statistics related module including variance and standard deviation.|[link](../development/extensions-core/stats.html)|
|
||||
|mysql-metadata-storage|MySQL metadata store.|[link](../development/extensions-core/mysql.html)|
|
||||
|postgresql-metadata-storage|PostgreSQL metadata store.|[link](../development/extensions-core/postgresql.html)|
|
||||
|simple-client-sslcontext|Simple SSLContext provider module to be used by internal HttpClient talking to other nodes over HTTPS.|[link](../development/extensions-core/simple-client-sslcontext.html)|
|
||||
|simple-client-sslcontext|Simple SSLContext provider module to be used by Druid's internal HttpClient when talking to other Druid processes over HTTPS.|[link](../development/extensions-core/simple-client-sslcontext.html)|
|
||||
|
||||
# Community Extensions
|
||||
|
||||
|
|
|
@ -59,7 +59,7 @@ unpredictable results if global variables are used.
|
|||
## Performance
|
||||
|
||||
Simple JavaScript functions typically have a slight performance penalty to native speed. More complex JavaScript
|
||||
functions can have steeper performance penalties. Druid compiles JavaScript functions once per node per query.
|
||||
functions can have steeper performance penalties. Druid compiles JavaScript functions once on each data process per query.
|
||||
|
||||
You may need to pay special attention to garbage collection when making heavy use of JavaScript functions, especially
|
||||
garbage collection of the compiled classes themselves. Be sure to use a garbage collector configuration that supports
|
||||
|
|
|
@ -110,7 +110,7 @@ In addition to DataSegmentPusher and DataSegmentPuller, you can also bind:
|
|||
|
||||
It's recommended to use batch ingestion tasks to validate your implementation.
|
||||
The segment will be automatically rolled up to Historical note after ~20 seconds.
|
||||
In this way, you can validate both push (at realtime node) and pull (at Historical node) segments.
|
||||
In this way, you can validate both push (at realtime process) and pull (at Historical process) segments.
|
||||
|
||||
* DataSegmentPusher
|
||||
|
||||
|
@ -118,9 +118,9 @@ Wherever your data storage (cloud storage service, distributed file system, etc.
|
|||
|
||||
* DataSegmentPuller
|
||||
|
||||
After ~20 secs your ingestion task ends, you should be able to see your Historical node trying to load the new segment.
|
||||
After ~20 secs your ingestion task ends, you should be able to see your Historical process trying to load the new segment.
|
||||
|
||||
The following example was retrieved from a Historical node configured to use Azure for deep storage:
|
||||
The following example was retrieved from a Historical process configured to use Azure for deep storage:
|
||||
|
||||
```
|
||||
2015-04-14T02:42:33,450 INFO [ZkCoordinator-0] org.apache.druid.server.coordination.ZkCoordinator - New request[LOAD: dde_2015-01-02T00:00:00.000Z_2015-01-03T00:00:00
|
||||
|
|
|
@ -48,7 +48,7 @@ the query logic is to start from `QueryResource.java`.
|
|||
|
||||
## Coordination
|
||||
|
||||
Most of the coordination logic for Historical nodes is on the Druid Coordinator. The starting point here is `DruidCoordinator.java`.
|
||||
Most of the coordination logic for Historical processes is on the Druid Coordinator. The starting point here is `DruidCoordinator.java`.
|
||||
Most of the coordination logic for (real-time) ingestion is in the Druid indexing service. The starting point here is `OverlordResource.java`.
|
||||
|
||||
## Real-time Ingestion
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "Router Node"
|
||||
title: "Router Process"
|
||||
---
|
||||
|
||||
<!--
|
||||
|
@ -22,9 +22,14 @@ title: "Router Node"
|
|||
~ under the License.
|
||||
-->
|
||||
|
||||
# Router Node
|
||||
# Router Process
|
||||
|
||||
The Router process can be used to route queries to different Broker processes. By default, the broker routes queries based on how [Rules](../operations/rule-configuration.html) are set up. For example, if 1 month of recent data is loaded into a `hot` cluster, queries that fall within the recent month can be routed to a dedicated set of brokers. Queries outside this range are routed to another set of brokers. This set up provides query isolation such that queries for more important data are not impacted by queries for less important data.
|
||||
|
||||
For query routing purposes, you should only ever need the Router process if you have a Druid cluster well into the terabyte range.
|
||||
|
||||
In addition to query routing, the Router also runs the [Druid Console](../operations/management-uis.html#druid-console), a management UI for datasources, segments, tasks, data processes (Historicals and MiddleManagers), and coordinator dynamic configuration. The user can also run SQL and native Druid queries within the console.
|
||||
|
||||
You should only ever need the Router node if you have a Druid cluster well into the terabyte range. The Router node can be used to route queries to different Broker nodes. By default, the broker routes queries based on how [Rules](../operations/rule-configuration.html) are set up. For example, if 1 month of recent data is loaded into a `hot` cluster, queries that fall within the recent month can be routed to a dedicated set of brokers. Queries outside this range are routed to another set of brokers. This set up provides query isolation such that queries for more important data are not impacted by queries for less important data.
|
||||
|
||||
Running
|
||||
-------
|
||||
|
@ -36,7 +41,7 @@ org.apache.druid.cli.Main server router
|
|||
Example Production Configuration
|
||||
--------------------------------
|
||||
|
||||
In this example, we have two tiers in our production cluster: `hot` and `_default_tier`. Queries for the `hot` tier are routed through the `broker-hot` set of Brokers, and queries for the `_default_tier` are routed through the `broker-cold` set of Brokers. If any exceptions or network problems occur, queries are routed to the `broker-cold` set of brokers. In our example, we are running with a c3.2xlarge EC2 node. We assume a `common.runtime.properties` already exists.
|
||||
In this example, we have two tiers in our production cluster: `hot` and `_default_tier`. Queries for the `hot` tier are routed through the `broker-hot` set of Brokers, and queries for the `_default_tier` are routed through the `broker-cold` set of Brokers. If any exceptions or network problems occur, queries are routed to the `broker-cold` set of brokers. In our example, we are running with a c3.2xlarge EC2 instance. We assume a `common.runtime.properties` already exists.
|
||||
|
||||
JVM settings:
|
||||
|
||||
|
@ -176,13 +181,13 @@ This is a non-default implementation that is provided for experimentation purpos
|
|||
HTTP Endpoints
|
||||
--------------
|
||||
|
||||
The Router node exposes several HTTP endpoints for interactions.
|
||||
The Router process exposes several HTTP endpoints for interactions.
|
||||
|
||||
### GET
|
||||
|
||||
* `/status`
|
||||
|
||||
Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the node.
|
||||
Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the process.
|
||||
|
||||
* `/druid/v2/datasources`
|
||||
|
||||
|
@ -203,7 +208,7 @@ Returns the metrics of the datasource.
|
|||
Router as Management Proxy
|
||||
--------------------------
|
||||
|
||||
The Router can be configured to forward requests to the active Coordinator or Overlord node. This may be useful for
|
||||
The Router can be configured to forward requests to the active Coordinator or Overlord process. This may be useful for
|
||||
setting up a highly available cluster in situations where the HTTP redirect mechanism of the inactive -> active
|
||||
Coordinator/Overlord does not function correctly (servers are behind a load balancer, the hostname used in the redirect
|
||||
is only resolvable internally, etc.).
|
||||
|
@ -224,10 +229,10 @@ determined from the original request path based on Druid API path conventions. F
|
|||
that using the management proxy does not require modifying the API request other than issuing the request to the Router
|
||||
instead of the Coordinator or Overlord. Most Druid API requests can be routed implicitly.
|
||||
|
||||
Explicit routes are those where the request to the Router contains a path prefix indicating which node the request
|
||||
Explicit routes are those where the request to the Router contains a path prefix indicating which process the request
|
||||
should be routed to. For the Coordinator this prefix is `/proxy/coordinator` and for the Overlord it is `/proxy/overlord`.
|
||||
This is required for API calls with an ambiguous destination. For example, the `/status` API is present on all Druid
|
||||
nodes, so explicit routing needs to be used to indicate the proxy destination.
|
||||
processes, so explicit routing needs to be used to indicate the proxy destination.
|
||||
|
||||
This is summarized in the table below:
|
||||
|
||||
|
|
|
@ -56,9 +56,9 @@ Other common reasons that hand-off fails are as follows:
|
|||
|
||||
1) Druid is unable to write to the metadata storage. Make sure your configurations are correct.
|
||||
|
||||
2) Historical nodes are out of capacity and cannot download any more segments. You'll see exceptions in the Coordinator logs if this occurs and the Coordinator console will show the Historicals are near capacity.
|
||||
2) Historical processes are out of capacity and cannot download any more segments. You'll see exceptions in the Coordinator logs if this occurs and the Coordinator console will show the Historicals are near capacity.
|
||||
|
||||
3) Segments are corrupt and cannot be downloaded. You'll see exceptions in your Historical nodes if this occurs.
|
||||
3) Segments are corrupt and cannot be downloaded. You'll see exceptions in your Historical processes if this occurs.
|
||||
|
||||
4) Deep storage is improperly configured. Make sure that your segment actually exists in deep storage and that the Coordinator logs have no errors.
|
||||
|
||||
|
@ -66,9 +66,9 @@ Other common reasons that hand-off fails are as follows:
|
|||
|
||||
Make sure to include the `druid-hdfs-storage` and all the hadoop configuration, dependencies (that can be obtained by running command `hadoop classpath` on a machine where hadoop has been setup) in the classpath. And, provide necessary HDFS settings as described in [Deep Storage](../dependencies/deep-storage.html) .
|
||||
|
||||
## I don't see my Druid segments on my Historical nodes
|
||||
## I don't see my Druid segments on my Historical processes
|
||||
|
||||
You can check the Coordinator console located at `<COORDINATOR_IP>:<PORT>`. Make sure that your segments have actually loaded on [Historical nodes](../design/historical.html). If your segments are not present, check the Coordinator logs for messages about capacity of replication errors. One reason that segments are not downloaded is because Historical nodes have maxSizes that are too small, making them incapable of downloading more data. You can change that with (for example):
|
||||
You can check the Coordinator console located at `<COORDINATOR_IP>:<PORT>`. Make sure that your segments have actually loaded on [Historical processes](../design/historical.html). If your segments are not present, check the Coordinator logs for messages about capacity of replication errors. One reason that segments are not downloaded is because Historical processes have maxSizes that are too small, making them incapable of downloading more data. You can change that with (for example):
|
||||
|
||||
```
|
||||
-Ddruid.segmentCache.locations=[{"path":"/tmp/druid/storageLocation","maxSize":"500000000000"}]
|
||||
|
@ -99,7 +99,7 @@ See [Update Existing Data](../ingestion/update-existing-data.html) for more deta
|
|||
|
||||
## Real-time ingestion seems to be stuck
|
||||
|
||||
There are a few ways this can occur. Druid will throttle ingestion to prevent out of memory problems if the intermediate persists are taking too long or if hand-off is taking too long. If your node logs indicate certain columns are taking a very long time to build (for example, if your segment granularity is hourly, but creating a single column takes 30 minutes), you should re-evaluate your configuration or scale up your real-time ingestion.
|
||||
There are a few ways this can occur. Druid will throttle ingestion to prevent out of memory problems if the intermediate persists are taking too long or if hand-off is taking too long. If your process logs indicate certain columns are taking a very long time to build (for example, if your segment granularity is hourly, but creating a single column takes 30 minutes), you should re-evaluate your configuration or scale up your real-time ingestion.
|
||||
|
||||
## More information
|
||||
|
||||
|
|
|
@ -191,8 +191,9 @@ When using this firehose, events can be sent by submitting a POST request to the
|
|||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|This should be "receiver"|yes|
|
||||
|serviceName|name used to announce the event receiver service endpoint|yes|
|
||||
|bufferSize| size of buffer used by firehose to store events|no default(100000)|
|
||||
|serviceName|Name used to announce the event receiver service endpoint|yes|
|
||||
|maxIdleTime|A firehose is automatically shut down after not receiving any events for this period of time, in milliseconds. If not specified, a firehose is never shut down due to being idle. Zero and negative values have the same effect.|no|
|
||||
|bufferSize|Size of buffer used by firehose to store events|no, default is 100000|
|
||||
|
||||
Shut down time for EventReceiverFirehose can be specified by submitting a POST request to
|
||||
|
||||
|
|
|
@ -38,6 +38,6 @@ ingestion method.
|
|||
| Supported [rollup modes](http://druid.io/docs/latest/ingestion/index.html#roll-up-modes) | Perfect rollup | Best-effort rollup | Both perfect and best-effort rollup |
|
||||
| Supported partitioning methods | [Both Hash-based and range partitioning](http://druid.io/docs/latest/ingestion/hadoop.html#partitioning-specification) | N/A | Hash-based partitioning (when `forceGuaranteedRollup` = true) |
|
||||
| Supported input locations | All locations accessible via HDFS client or Druid dataSource | All implemented [firehoses](./firehose.html) | All implemented [firehoses](./firehose.html) |
|
||||
| Supported file formats | All implemented Hadoop InputFormats | Currently only text file format (CSV, TSV, JSON) | Currently only text file format (CSV, TSV, JSON) |
|
||||
| Supported file formats | All implemented Hadoop InputFormats | Currently text file formats (CSV, TSV, JSON) by default. Additional formats can be added though a [custom extension](../development/modules.html) implementing [`FiniteFirehoseFactory`](https://github.com/apache/incubator-druid/blob/master/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java) | Currently text file formats (CSV, TSV, JSON) by default. Additional formats can be added though a [custom extension](../development/modules.html) implementing [`FiniteFirehoseFactory`](https://github.com/apache/incubator-druid/blob/master/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java) |
|
||||
| Saving parse exceptions in ingestion report | Currently not supported | Currently not supported | Supported |
|
||||
| Custom segment version | Supported, but this is NOT recommended | N/A | N/A |
|
||||
|
|
|
@ -162,13 +162,13 @@ to load that segment and instructs that Historical to do so.
|
|||
## Ingestion methods
|
||||
|
||||
In most ingestion methods, this work is done by Druid
|
||||
MiddleManager nodes. One exception is Hadoop-based ingestion, where this work is instead done using a Hadoop MapReduce
|
||||
job on YARN (although MiddleManager nodes are still involved in starting and monitoring the Hadoop jobs).
|
||||
MiddleManager processes. One exception is Hadoop-based ingestion, where this work is instead done using a Hadoop MapReduce
|
||||
job on YARN (although MiddleManager processes are still involved in starting and monitoring the Hadoop jobs).
|
||||
|
||||
Once segments have been generated and stored in [deep storage](../dependencies/deep-storage.html), they will be loaded by Druid Historical nodes. Some Druid
|
||||
ingestion methods additionally support _real-time queries_, meaning you can query in-flight data on MiddleManager nodes
|
||||
Once segments have been generated and stored in [deep storage](../dependencies/deep-storage.html), they will be loaded by Druid Historical processes. Some Druid
|
||||
ingestion methods additionally support _real-time queries_, meaning you can query in-flight data on MiddleManager processes
|
||||
before it is finished being converted and written to deep storage. In general, a small amount of data will be in-flight
|
||||
on MiddleManager nodes relative to the larger amount of historical data being served from Historical nodes.
|
||||
on MiddleManager processes relative to the larger amount of historical data being served from Historical processes.
|
||||
|
||||
See the [Design](../design/index.html) page for more details on how Druid stores and manages your data.
|
||||
|
||||
|
@ -291,9 +291,9 @@ For compaction documentation, please see [tasks](../ingestion/tasks.html).
|
|||
|
||||
Druid supports retention rules, which are used to define intervals of time where data should be preserved, and intervals where data should be discarded.
|
||||
|
||||
Druid also supports separating Historical nodes into tiers, and the retention rules can be configured to assign data for specific intervals to specific tiers.
|
||||
Druid also supports separating Historical processes into tiers, and the retention rules can be configured to assign data for specific intervals to specific tiers.
|
||||
|
||||
These features are useful for performance/cost management; a common use case is separating Historical nodes into a "hot" tier and a "cold" tier.
|
||||
These features are useful for performance/cost management; a common use case is separating Historical processes into a "hot" tier and a "cold" tier.
|
||||
|
||||
For more information, please see [Load rules](../operations/rule-configuration.html).
|
||||
|
||||
|
|
|
@ -26,7 +26,7 @@ title: "Task Locking & Priority"
|
|||
|
||||
## Locking
|
||||
|
||||
Once an Overlord node accepts a task, the task acquires locks for the data source and intervals specified in the task.
|
||||
Once an Overlord process accepts a task, the task acquires locks for the data source and intervals specified in the task.
|
||||
|
||||
There are two lock types, i.e., _shared lock_ and _exclusive lock_.
|
||||
|
||||
|
|
|
@ -25,7 +25,7 @@ title: "Native Index Tasks"
|
|||
# Native Index Tasks
|
||||
|
||||
Druid currently has two types of native batch indexing tasks, `index_parallel` which runs tasks
|
||||
in parallel on multiple MiddleManager nodes, and `index` which will run a single indexing task locally on a single
|
||||
in parallel on multiple MiddleManager processes, and `index` which will run a single indexing task locally on a single
|
||||
MiddleManager.
|
||||
|
||||
Please check [Hadoop-based Batch Ingestion VS Native Batch Ingestion](./hadoop-vs-native-batch.html) for differences between native batch ingestion and Hadoop-based ingestion.
|
||||
|
@ -577,12 +577,12 @@ the Index task supports two segment pushing modes, i.e., _bulk pushing mode_ and
|
|||
[perfect rollup and best-effort rollup](../ingestion/index.html#roll-up-modes), respectively.
|
||||
|
||||
In the bulk pushing mode, every segment is pushed at the very end of the index task. Until then, created segments
|
||||
are stored in the memory and local storage of the node running the index task. As a result, this mode might cause a
|
||||
are stored in the memory and local storage of the process running the index task. As a result, this mode might cause a
|
||||
problem due to limited storage capacity, and is not recommended to use in production.
|
||||
|
||||
On the contrary, in the incremental pushing mode, segments are incrementally pushed, that is they can be pushed
|
||||
in the middle of the index task. More precisely, the index task collects data and stores created segments in the memory
|
||||
and disks of the node running that task until the total number of collected rows exceeds `maxTotalRows`. Once it exceeds,
|
||||
and disks of the process running that task until the total number of collected rows exceeds `maxTotalRows`. Once it exceeds,
|
||||
the index task immediately pushes all segments created until that moment, cleans all pushed segments up, and
|
||||
continues to ingest remaining data.
|
||||
|
||||
|
|
|
@ -23,7 +23,7 @@ title: "Stream Pull Ingestion"
|
|||
-->
|
||||
|
||||
<div class="note info">
|
||||
NOTE: Realtime nodes are deprecated. Please use the <a href="../development/extensions-core/kafka-ingestion.html">Kafka Indexing Service</a> for stream pull use cases instead.
|
||||
NOTE: Realtime processes are deprecated. Please use the <a href="../development/extensions-core/kafka-ingestion.html">Kafka Indexing Service</a> for stream pull use cases instead.
|
||||
</div>
|
||||
|
||||
# Stream Pull Ingestion
|
||||
|
@ -32,20 +32,20 @@ If you have an external service that you want to pull data from, you have two op
|
|||
option is to set up a "copying" service that reads from the data source and writes to Druid using
|
||||
the [stream push method](stream-push.html).
|
||||
|
||||
Another option is *stream pull*. With this approach, a Druid Realtime Node ingests data from a
|
||||
Another option is *stream pull*. With this approach, a Druid Realtime Process ingests data from a
|
||||
[Firehose](../ingestion/firehose.html) connected to the data you want to
|
||||
read. The Druid quickstart and tutorials do not include information about how to set up standalone realtime nodes, but
|
||||
they can be used in place for Tranquility server and the indexing service. Please note that Realtime nodes have different properties and roles than the indexing service.
|
||||
read. The Druid quickstart and tutorials do not include information about how to set up standalone realtime processes, but
|
||||
they can be used in place for Tranquility server and the indexing service. Please note that Realtime processes have different properties and roles than the indexing service.
|
||||
|
||||
## Realtime Node Ingestion
|
||||
## Realtime Process Ingestion
|
||||
|
||||
Much of the configuration governing Realtime nodes and the ingestion of data is set in the Realtime spec file, discussed on this page.
|
||||
Much of the configuration governing Realtime processes and the ingestion of data is set in the Realtime spec file, discussed on this page.
|
||||
|
||||
For general Real-time Node information, see [here](../design/realtime.html).
|
||||
For general Real-time Process information, see [here](../design/realtime.html).
|
||||
|
||||
For Real-time Node Configuration, see [Realtime Configuration](../configuration/realtime.html).
|
||||
For Real-time Process Configuration, see [Realtime Configuration](../configuration/realtime.html).
|
||||
|
||||
For writing your own plugins to the real-time node, see [Firehose](../ingestion/firehose.html).
|
||||
For writing your own plugins to the real-time process, see [Firehose](../ingestion/firehose.html).
|
||||
|
||||
## Realtime "specFile"
|
||||
|
||||
|
@ -127,7 +127,7 @@ The property `druid.realtime.specFile` has the path of a file (absolute or relat
|
|||
]
|
||||
```
|
||||
|
||||
This is a JSON Array so you can give more than one realtime stream to a given node. The number you can put in the same process depends on the exact configuration. In general, it is best to think of each realtime stream handler as requiring 2-threads: 1 thread for data consumption and aggregation, 1 thread for incremental persists and other background tasks.
|
||||
This is a JSON Array so you can give more than one realtime stream to a given process. The number you can put in the same process depends on the exact configuration. In general, it is best to think of each realtime stream handler as requiring 2-threads: 1 thread for data consumption and aggregation, 1 thread for incremental persists and other background tasks.
|
||||
|
||||
There are three parts to a realtime stream specification, `dataSchema`, `IOConfig`, and `tuningConfig` which we will go into here.
|
||||
|
||||
|
@ -172,7 +172,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|
|||
|versioningPolicy|Object|How to version segments.|no (default == based on segment start time)|
|
||||
|rejectionPolicy|Object|Controls how data sets the data acceptance policy for creating and handing off segments. More on this below.|no (default == 'serverTime')|
|
||||
|maxPendingPersists|Integer|Maximum number of persists that can be pending, but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0; meaning one persist can be running concurrently with ingestion, and none can be queued up)|
|
||||
|shardSpec|Object|This describes the shard that is represented by this server. This must be specified properly in order to have multiple realtime nodes indexing the same data stream in a [sharded fashion](#sharding).|no (default == 'NoneShardSpec')|
|
||||
|shardSpec|Object|This describes the shard that is represented by this server. This must be specified properly in order to have multiple realtime processes indexing the same data stream in a [sharded fashion](#sharding).|no (default == 'NoneShardSpec')|
|
||||
|persistThreadPriority|int|If `-XX:+UseThreadPriorities` is properly enabled, this will set the thread priority of the persisting thread to `Thread.NORM_PRIORITY` plus this value within the bounds of `Thread.MIN_PRIORITY` and `Thread.MAX_PRIORITY`. A value of 0 indicates to not change the thread priority.|no (default == 0; inherit and do not override)|
|
||||
|mergeThreadPriority|int|If `-XX:+UseThreadPriorities` is properly enabled, this will set the thread priority of the merging thread to `Thread.NORM_PRIORITY` plus this value within the bounds of `Thread.MIN_PRIORITY` and `Thread.MAX_PRIORITY`. A value of 0 indicates to not change the thread priority.|no (default == 0; inherit and do not override)|
|
||||
|reportParseExceptions|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion. If false, unparseable rows and fields will be skipped. If an entire row is skipped, the "unparseable" counter will be incremented. If some fields in a row were parseable and some were not, the parseable fields will be indexed and the "unparseable" counter will not be incremented.|no (default == false)|
|
||||
|
@ -233,7 +233,7 @@ In small-data scenarios, sharding is unnecessary and can be set to none (the def
|
|||
"shardSpec": {"type": "none"}
|
||||
```
|
||||
|
||||
However, in scenarios with multiple realtime nodes, `none` is less useful as it cannot help with scaling data volume (see below). Note that for the batch indexing service, no explicit configuration is required; sharding is provided automatically.
|
||||
However, in scenarios with multiple realtime processes, `none` is less useful as it cannot help with scaling data volume (see below). Note that for the batch indexing service, no explicit configuration is required; sharding is provided automatically.
|
||||
|
||||
Druid uses sharding based on the `shardSpec` setting you configure. The recommended choices, `linear` and `numbered`, are discussed below; other types have been useful for internal Druid development but are not appropriate for production setups.
|
||||
|
||||
|
@ -243,7 +243,7 @@ Keep in mind, that sharding configuration has nothing to do with configured fire
|
|||
|
||||
This strategy provides following advantages:
|
||||
|
||||
* There is no need to update the fileSpec configurations of existing nodes when adding new nodes.
|
||||
* There is no need to update the fileSpec configurations of existing processes when adding new processes.
|
||||
* All unique shards are queried, regardless of whether the partition numbering is sequential or not (it allows querying of partitions 0 and 2, even if partition 1 is missing).
|
||||
|
||||
Configure `linear` under `schema`:
|
||||
|
@ -273,9 +273,9 @@ Configure `numbered` under `schema`:
|
|||
|
||||
##### Scale and Redundancy
|
||||
|
||||
The `shardSpec` configuration can be used to create redundancy by having the same `partitionNum` values on different nodes.
|
||||
The `shardSpec` configuration can be used to create redundancy by having the same `partitionNum` values on different processes.
|
||||
|
||||
For example, if RealTimeNode1 has:
|
||||
For example, if RealTimeProcess1 has:
|
||||
|
||||
```json
|
||||
"shardSpec": {
|
||||
|
@ -284,7 +284,7 @@ For example, if RealTimeNode1 has:
|
|||
}
|
||||
```
|
||||
|
||||
and RealTimeNode2 has:
|
||||
and RealTimeProcess2 has:
|
||||
|
||||
```json
|
||||
"shardSpec": {
|
||||
|
@ -293,9 +293,9 @@ and RealTimeNode2 has:
|
|||
}
|
||||
```
|
||||
|
||||
then two realtime nodes can store segments with the same datasource, version, time interval, and partition number. Brokers that query for data in such segments will assume that they hold the same data, and the query will target only one of the segments.
|
||||
then two realtime processes can store segments with the same datasource, version, time interval, and partition number. Brokers that query for data in such segments will assume that they hold the same data, and the query will target only one of the segments.
|
||||
|
||||
`shardSpec` can also help achieve scale. For this, add nodes with a different `partionNum`. Continuing with the example, if RealTimeNode3 has:
|
||||
`shardSpec` can also help achieve scale. For this, add processes with a different `partionNum`. Continuing with the example, if RealTimeProcess3 has:
|
||||
|
||||
```json
|
||||
"shardSpec": {
|
||||
|
@ -304,7 +304,7 @@ then two realtime nodes can store segments with the same datasource, version, ti
|
|||
}
|
||||
```
|
||||
|
||||
then it can store segments with the same datasource, time interval, and version as in the first two nodes, but with a different partition number. Brokers that query for data in such segments will assume that a segment from RealTimeNode3 holds *different* data, and the query will target it along with a segment from the first two nodes.
|
||||
then it can store segments with the same datasource, time interval, and version as in the first two processes, but with a different partition number. Brokers that query for data in such segments will assume that a segment from RealTimeProcess3 holds *different* data, and the query will target it along with a segment from the first two processes.
|
||||
|
||||
You can use type `numbered` similarly. Note that type `none` is essentially type `linear` with all shards having a fixed `partitionNum` of 0.
|
||||
|
||||
|
@ -327,45 +327,45 @@ The normal, expected use cases have the following overall constraints: `intermed
|
|||
|
||||
### Kafka
|
||||
|
||||
Standalone realtime nodes use the Kafka high level consumer, which imposes a few restrictions.
|
||||
Standalone realtime processes use the Kafka high level consumer, which imposes a few restrictions.
|
||||
|
||||
Druid replicates segment such that logically equivalent data segments are concurrently hosted on N nodes. If N–1 nodes go down,
|
||||
the data will still be available for querying. On real-time nodes, this process depends on maintaining logically equivalent
|
||||
data segments on each of the N nodes, which is not possible with standard Kafka consumer groups if your Kafka topic requires more than one consumer
|
||||
Druid replicates segment such that logically equivalent data segments are concurrently hosted on N processes. If N–1 processes go down,
|
||||
the data will still be available for querying. On real-time processes, this process depends on maintaining logically equivalent
|
||||
data segments on each of the N processes, which is not possible with standard Kafka consumer groups if your Kafka topic requires more than one consumer
|
||||
(because consumers in different consumer groups will split up the data differently).
|
||||
|
||||
For example, let's say your topic is split across Kafka partitions 1, 2, & 3 and you have 2 real-time nodes with linear shard specs 1 & 2.
|
||||
Both of the real-time nodes are in the same consumer group. Real-time node 1 may consume data from partitions 1 & 3, and real-time node 2 may consume data from partition 2.
|
||||
For example, let's say your topic is split across Kafka partitions 1, 2, & 3 and you have 2 real-time processes with linear shard specs 1 & 2.
|
||||
Both of the real-time processes are in the same consumer group. Real-time process 1 may consume data from partitions 1 & 3, and real-time process 2 may consume data from partition 2.
|
||||
Querying for your data through the Broker will yield correct results.
|
||||
|
||||
The problem arises if you want to replicate your data by creating real-time nodes 3 & 4. These new real-time nodes also
|
||||
The problem arises if you want to replicate your data by creating real-time processes 3 & 4. These new real-time processes also
|
||||
have linear shard specs 1 & 2, and they will consume data from Kafka using a different consumer group. In this case,
|
||||
real-time node 3 may consume data from partitions 1 & 2, and real-time node 4 may consume data from partition 2.
|
||||
From Druid's perspective, the segments hosted by real-time nodes 1 and 3 are the same, and the data hosted by real-time nodes
|
||||
real-time process 3 may consume data from partitions 1 & 2, and real-time process 4 may consume data from partition 2.
|
||||
From Druid's perspective, the segments hosted by real-time processes 1 and 3 are the same, and the data hosted by real-time processes
|
||||
2 and 4 are the same, although they are reading from different Kafka partitions. Querying for the data will yield inconsistent
|
||||
results.
|
||||
|
||||
Is this always a problem? No. If your data is small enough to fit on a single Kafka partition, you can replicate without issues.
|
||||
Otherwise, you can run real-time nodes without replication.
|
||||
Otherwise, you can run real-time processes without replication.
|
||||
|
||||
Please note that druid will skip over event that failed its checksum and it is corrupt.
|
||||
|
||||
### Locking
|
||||
|
||||
Using stream pull ingestion with Realtime nodes together batch ingestion may introduce data override issues. For example, if you
|
||||
Using stream pull ingestion with Realtime processes together batch ingestion may introduce data override issues. For example, if you
|
||||
are generating hourly segments for the current day, and run a daily batch job for the current day's data, the segments created by
|
||||
the batch job will have a more recent version than most of the segments generated by realtime ingestion. If your batch job is indexing
|
||||
data that isn't yet complete for the day, the daily segment created by the batch job can override recent segments created by
|
||||
realtime nodes. A portion of data will appear to be lost in this case.
|
||||
realtime processes. A portion of data will appear to be lost in this case.
|
||||
|
||||
### Schema changes
|
||||
|
||||
Standalone realtime nodes require stopping a node to update a schema, and starting it up again for the schema to take effect.
|
||||
Standalone realtime processes require stopping a process to update a schema, and starting it up again for the schema to take effect.
|
||||
This can be difficult to manage at scale, especially with multiple partitions.
|
||||
|
||||
### Log management
|
||||
|
||||
Each standalone realtime node has its own set of logs. Diagnosing errors across many partitions across many servers may be
|
||||
Each standalone realtime process has its own set of logs. Diagnosing errors across many partitions across many servers may be
|
||||
difficult to manage and track at scale.
|
||||
|
||||
## Deployment Notes
|
||||
|
|
|
@ -125,7 +125,7 @@ The windowPeriod is the slack time permitted for events. For example, a windowPe
|
|||
than ten minutes in the future, will be dropped.
|
||||
|
||||
These are important configurations because they influence how long tasks will be alive for, and how
|
||||
long data stays in the realtime system before being handed off to the Historical nodes. For example,
|
||||
long data stays in the realtime system before being handed off to the Historical processes. For example,
|
||||
if your configuration has segmentGranularity "hour" and windowPeriod ten minutes, tasks will stay
|
||||
around listening for events for an hour and ten minutes. For this reason, to prevent excessive
|
||||
buildup of tasks, it is recommended that your windowPeriod be less than your segmentGranularity.
|
||||
|
|
|
@ -40,15 +40,15 @@ This page documents all of the API endpoints for each Druid service type.
|
|||
|
||||
## Common
|
||||
|
||||
The following endpoints are supported by all nodes.
|
||||
The following endpoints are supported by all processes.
|
||||
|
||||
### Node information
|
||||
### Process information
|
||||
|
||||
#### GET
|
||||
|
||||
* `/status`
|
||||
|
||||
Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the node.
|
||||
Returns the Druid version, loaded extensions, memory used, total memory and other useful information about the process.
|
||||
|
||||
* `/status/health`
|
||||
|
||||
|
@ -56,7 +56,7 @@ An endpoint that always returns a boolean "true" value with a 200 OK response, u
|
|||
|
||||
* `/status/properties`
|
||||
|
||||
Returns the current configuration properties of the node.
|
||||
Returns the current configuration properties of the process.
|
||||
|
||||
## Master Server
|
||||
|
||||
|
@ -97,15 +97,15 @@ Returns the number of segments left to load in each tier until segments that sho
|
|||
|
||||
* `/druid/coordinator/v1/loadqueue`
|
||||
|
||||
Returns the ids of segments to load and drop for each Historical node.
|
||||
Returns the ids of segments to load and drop for each Historical process.
|
||||
|
||||
* `/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 node.
|
||||
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.
|
||||
|
||||
* `/druid/coordinator/v1/loadqueue?full`
|
||||
|
||||
Returns the serialized JSON of segments to load and drop for each Historical node.
|
||||
Returns the serialized JSON of segments to load and drop for each Historical process.
|
||||
|
||||
#### Metadata store information
|
||||
|
||||
|
@ -341,7 +341,8 @@ will be set for them.
|
|||
|
||||
* `/druid/coordinator/v1/config/compaction`
|
||||
|
||||
Creates or updates the compaction config for a dataSource. See [Compaction Configuration](../configuration/index.html#compaction-dynamic-configuration) for configuration details.
|
||||
Creates or updates the compaction config for a dataSource.
|
||||
See [Compaction Configuration](../configuration/index.html#compaction-dynamic-configuration) for configuration details.
|
||||
|
||||
|
||||
##### DELETE
|
||||
|
@ -357,14 +358,14 @@ Removes the compaction config for a dataSource.
|
|||
* `/druid/coordinator/v1/servers`
|
||||
|
||||
Returns a list of servers URLs using the format `{hostname}:{port}`. Note that
|
||||
nodes that run with different types will appear multiple times with different
|
||||
processes that run with different types will appear multiple times with different
|
||||
ports.
|
||||
|
||||
* `/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`: node type (`indexer-executor`, `historical`)
|
||||
* `type`: process type (`indexer-executor`, `historical`)
|
||||
* `currSize`: storage size currently used
|
||||
* `maxSize`: maximum storage size
|
||||
* `priority`
|
||||
|
@ -655,7 +656,7 @@ Retrieve an unparseable events report from a Peon. See [task reports](../ingesti
|
|||
* `/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 node is ready
|
||||
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.
|
||||
|
||||
* `/druid/historical/v1/readiness`
|
||||
|
|
|
@ -0,0 +1,78 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "Management UIs"
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
# Management UIs
|
||||
|
||||
## Druid Console
|
||||
|
||||
Druid provides a console for managing datasources, segments, tasks, data processes (Historicals and MiddleManagers), and coordinator dynamic configuration. The user can also run SQL and native Druid queries within the console.
|
||||
|
||||
The Druid Console is hosted by the [Router](../development/router.html) process. We recommend running the Router process on your [Query server](../design/processes.html).
|
||||
|
||||
In addition, the following cluster settings must be enabled:
|
||||
|
||||
- the Router's [management proxy](../development/router.html#enabling-the-management-proxy) must be enabled.
|
||||
- the Broker processes in the cluster must have [Druid SQL](../querying/sql.html) enabled.
|
||||
|
||||
After enabling Druid SQL on the Brokers and deploying a Router with the managment proxy enabled, the Druid console can be accessed at:
|
||||
|
||||
```
|
||||
http://<ROUTER_IP>:<ROUTER_PORT>
|
||||
```
|
||||
|
||||
The Druid Console contains all of the functionality provided by the older consoles described below, which are still available if needed. The legacy consoles may be replaced by the Druid Console in the future.
|
||||
|
||||
## Legacy Consoles
|
||||
|
||||
These older consoles provide a subset of the functionality of the Druid Console. We recommend using the Druid Console if possible.
|
||||
|
||||
### Coordinator Consoles
|
||||
|
||||
#### Version 2
|
||||
|
||||
The Druid Coordinator exposes a web console for displaying cluster information and rule configuration. After the Coordinator starts, the console can be accessed at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<COORDINATOR_PORT>
|
||||
```
|
||||
|
||||
There exists a full cluster view (which shows indexing tasks and Historical processes), as well as views for individual Historical processes, datasources and segments themselves. Segment information can be displayed in raw JSON form or as part of a sortable and filterable table.
|
||||
|
||||
The Coordinator console also exposes an interface to creating and editing rules. All valid datasources configured in the segment database, along with a default datasource, are available for configuration. Rules of different types can be added, deleted or edited.
|
||||
|
||||
#### Version 1
|
||||
|
||||
The oldest version of Druid's Coordinator console is still available for backwards compatibility at:
|
||||
|
||||
```
|
||||
http://<COORDINATOR_IP>:<COORDINATOR_PORT>/old-console
|
||||
```
|
||||
|
||||
### Overlord Console
|
||||
|
||||
The Overlord console can be used to view pending tasks, running tasks, available workers, and recent worker creation and termination. The console can be accessed at:
|
||||
|
||||
```
|
||||
http://<OVERLORD_IP>:<OVERLORD_PORT>/console.html
|
||||
```
|
|
@ -51,10 +51,10 @@ Available Metrics
|
|||
|------|-----------|----------|------------|
|
||||
|`query/time`|Milliseconds taken to complete a query.|Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.|< 1s|
|
||||
|`query/bytes`|number of bytes returned in query response.|Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.| |
|
||||
|`query/node/time`|Milliseconds taken to query individual historical/realtime nodes.|id, status, server.|< 1s|
|
||||
|`query/node/bytes`|number of bytes returned from querying individual historical/realtime nodes.|id, status, server.| |
|
||||
|`query/node/ttfb`|Time to first byte. Milliseconds elapsed until Broker starts receiving the response from individual historical/realtime nodes.|id, status, server.|< 1s|
|
||||
|`query/node/backpressure`|Milliseconds that the channel to this node has spent suspended due to backpressure.|id, status, server.| |
|
||||
|`query/node/time`|Milliseconds taken to query individual historical/realtime processes.|id, status, server.|< 1s|
|
||||
|`query/node/bytes`|number of bytes returned from querying individual historical/realtime processes.|id, status, server.| |
|
||||
|`query/node/ttfb`|Time to first byte. Milliseconds elapsed until Broker starts receiving the response from individual historical/realtime processes.|id, status, server.|< 1s|
|
||||
|`query/node/backpressure`|Milliseconds that the channel to this process has spent suspended due to backpressure.|id, status, server.| |
|
||||
|`query/intervalChunk/time`|Only emitted if interval chunking is enabled. Milliseconds required to query an interval chunk. This metric is deprecated and will be removed in the future because interval chunking is deprecated. See [Query Context](../querying/query-context.html).|id, status, chunkInterval (if interval chunking is enabled).|< 1s|
|
||||
|`query/count`|number of total queries|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/success/count`|number of queries successfully processed|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|
@ -69,7 +69,7 @@ Available Metrics
|
|||
|`query/segment/time`|Milliseconds taken to query individual segment. Includes time to page in the segment from disk.|id, status, segment.|several hundred milliseconds|
|
||||
|`query/wait/time`|Milliseconds spent waiting for a segment to be scanned.|id, segment.|< several hundred milliseconds|
|
||||
|`segment/scan/pending`|Number of segments in queue waiting to be scanned.||Close to 0|
|
||||
|`query/segmentAndCache/time`|Milliseconds taken to query individual segment or hit the cache (if it is enabled on the Historical node).|id, segment.|several hundred milliseconds|
|
||||
|`query/segmentAndCache/time`|Milliseconds taken to query individual segment or hit the cache (if it is enabled on the Historical process).|id, segment.|several hundred milliseconds|
|
||||
|`query/cpu/time`|Microseconds of CPU time taken to complete a query|Common: dataSource, type, interval, hasFilters, duration, context, remoteAddress, id. Aggregation Queries: numMetrics, numComplexMetrics. GroupBy: numDimensions. TopN: threshold, dimension.|Varies|
|
||||
|`query/count`|number of total queries|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|`query/success/count`|number of queries successfully processed|This metric is only available if the QueryCountStatsMonitor module is included.||
|
||||
|
@ -145,9 +145,9 @@ These metrics are applicable for the Kafka Indexing Service.
|
|||
|`ingest/kafka/maxLag`|Max lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute.|dataSource.|Greater than 0, should not be a very high number |
|
||||
|`ingest/kafka/avgLag`|Average lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute.|dataSource.|Greater than 0, should not be a very high number |
|
||||
|
||||
## Ingestion Metrics (Realtime node)
|
||||
## Ingestion Metrics (Realtime process)
|
||||
|
||||
These metrics are only available if the RealtimeMetricsMonitor is included in the monitors list for the Realtime node. These metrics are deltas for each emission period.
|
||||
These metrics are only available if the RealtimeMetricsMonitor is included in the monitors list for the Realtime process. These metrics are deltas for each emission period.
|
||||
|
||||
|Metric|Description|Dimensions|Normal Value|
|
||||
|------|-----------|----------|------------|
|
||||
|
|
|
@ -37,7 +37,7 @@ For best results, use these tips when configuring Druid to interact with your fa
|
|||
## Tip #1: Place Hadoop XMLs on Druid classpath
|
||||
|
||||
Place your Hadoop configuration XMLs (core-site.xml, hdfs-site.xml, yarn-site.xml, mapred-site.xml) on the classpath
|
||||
of your Druid nodes. You can do this by copying them into `conf/druid/_common/core-site.xml`,
|
||||
of your Druid processes. You can do this by copying them into `conf/druid/_common/core-site.xml`,
|
||||
`conf/druid/_common/hdfs-site.xml`, and so on. This allows Druid to find your Hadoop cluster and properly submit jobs.
|
||||
|
||||
## Tip #2: Classloader modification on Hadoop (Map/Reduce jobs only)
|
||||
|
|
|
@ -26,19 +26,19 @@ title: "Performance FAQ"
|
|||
|
||||
## I can't match your benchmarked results
|
||||
|
||||
Improper configuration is by far the largest problem we see people trying to deploy Druid. The example configurations listed in the tutorials are designed for a small volume of data where all nodes are on a single machine. The configs are extremely poor for actual production use.
|
||||
Improper configuration is by far the largest problem we see people trying to deploy Druid. The example configurations listed in the tutorials are designed for a small volume of data where all processes are on a single machine. The configs are extremely poor for actual production use.
|
||||
|
||||
## What should I set my JVM heap?
|
||||
|
||||
The size of the JVM heap really depends on the type of Druid node you are running. Below are a few considerations.
|
||||
The size of the JVM heap really depends on the type of Druid process you are running. Below are a few considerations.
|
||||
|
||||
[Broker nodes](../design/broker.html) uses the JVM heap mainly to merge results from Historicals and real-times. Brokers also use off-heap memory and processing threads for groupBy queries. We recommend 20G-30G of heap here.
|
||||
[Broker processes](../design/broker.html) uses the JVM heap mainly to merge results from Historicals and real-times. Brokers also use off-heap memory and processing threads for groupBy queries. We recommend 20G-30G of heap here.
|
||||
|
||||
[Historical nodes](../design/historical.html) use off-heap memory to store intermediate results, and by default, all segments are memory mapped before they can be queried. Typically, the more memory is available on a Historical node, the more segments can be served without the possibility of data being paged on to disk. On Historicals, the JVM heap is used for [GroupBy queries](../querying/groupbyquery.html), some data structures used for intermediate computation, and general processing. One way to calculate how much space there is for segments is: memory_for_segments = total_memory - heap - direct_memory - jvm_overhead. Note that total_memory here refers to the memory available to the cgroup (if running on Linux), which for default cases is going to be all the system memory.
|
||||
[Historical processes](../design/historical.html) use off-heap memory to store intermediate results, and by default, all segments are memory mapped before they can be queried. Typically, the more memory is available on a Historical process, the more segments can be served without the possibility of data being paged on to disk. On Historicals, the JVM heap is used for [GroupBy queries](../querying/groupbyquery.html), some data structures used for intermediate computation, and general processing. One way to calculate how much space there is for segments is: memory_for_segments = total_memory - heap - direct_memory - jvm_overhead. Note that total_memory here refers to the memory available to the cgroup (if running on Linux), which for default cases is going to be all the system memory.
|
||||
|
||||
We recommend 250mb * (processing.numThreads) for the heap.
|
||||
|
||||
[Coordinator nodes](../design/coordinator.html) do not require off-heap memory and the heap is used for loading information about all segments to determine what segments need to be loaded, dropped, moved, or replicated.
|
||||
[Coordinator processes](../design/coordinator.html) do not require off-heap memory and the heap is used for loading information about all segments to determine what segments need to be loaded, dropped, moved, or replicated.
|
||||
|
||||
## How much direct memory does Druid use?
|
||||
Any Druid process that handles queries (Brokers, Peons, and Historicals) uses two kinds of direct memory buffers with configurable size: processing buffers and merge buffers.
|
||||
|
@ -57,10 +57,10 @@ The `+1` is a fuzzy parameter meant to account for the decompression and diction
|
|||
Operators can ensure at least this amount of direct memory is available by providing `-XX:MaxDirectMemorySize=<VALUE>` at the command line.
|
||||
|
||||
## What is the intermediate computation buffer?
|
||||
The intermediate computation buffer specifies a buffer size for the storage of intermediate results. The computation engine in both the Historical and Realtime nodes 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. The default size is 1073741824 bytes (1GB).
|
||||
The intermediate computation buffer specifies a buffer size 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. The default size is 1073741824 bytes (1GB).
|
||||
|
||||
## What is server maxSize?
|
||||
Server maxSize sets the maximum cumulative segment size (in bytes) that a node can hold. Changing this parameter will affect performance by controlling the memory/disk ratio on a node. Setting this parameter to a value greater than the total memory capacity on a node and may cause disk paging to occur. This paging time introduces a query latency delay.
|
||||
Server maxSize sets the maximum cumulative segment size (in bytes) that a process can hold. Changing this parameter will affect performance by controlling the memory/disk ratio on a process. Setting this parameter to a value greater than the total memory capacity on a process and may cause disk paging to occur. This paging time introduces a query latency delay.
|
||||
|
||||
## My logs are really chatty, can I set them to asynchronously write?
|
||||
Yes, using a `log4j2.xml` similar to the following causes some of the more chatty classes to write asynchronously:
|
||||
|
|
|
@ -60,20 +60,20 @@ Please note that above flags are general guidelines only. Be cautious and feel f
|
|||
|
||||
Additionally, for large jvm heaps, here are a few Garbage Collection efficiency guidelines that have been known to help in some cases.
|
||||
- Mount /tmp on tmpfs ( See http://www.evanjones.ca/jvm-mmap-pause.html )
|
||||
- On Disk-IO intensive nodes (e.g. Historical and MiddleManager), GC and Druid logs should be written to a different disk than where data is written.
|
||||
- On Disk-IO intensive processes (e.g. Historical and MiddleManager), GC and Druid logs should be written to a different disk than where data is written.
|
||||
- Disable Transparent Huge Pages ( See https://blogs.oracle.com/linux/performance-issues-with-transparent-huge-pages-thp )
|
||||
- Try disabling biased locking by using `-XX:-UseBiasedLocking` jvm flag. ( See https://dzone.com/articles/logging-stop-world-pauses-jvm )
|
||||
|
||||
# Use UTC Timezone
|
||||
|
||||
We recommend using UTC timezone for all your events and across on your nodes, not just for Druid, but for all data infrastructure. This can greatly mitigate potential query problems with inconsistent timezones. To query in a non-UTC timezone see [query granularities](../querying/granularities.html#period-granularities)
|
||||
We recommend using UTC timezone for all your events and across your hosts, not just for Druid, but for all data infrastructure. This can greatly mitigate potential query problems with inconsistent timezones. To query in a non-UTC timezone see [query granularities](../querying/granularities.html#period-granularities)
|
||||
|
||||
# SSDs
|
||||
|
||||
SSDs are highly recommended for Historical and real-time nodes if you are not running a cluster that is entirely in memory. SSDs can greatly mitigate the time required to page data in and out of memory.
|
||||
SSDs are highly recommended for Historical and real-time processes if you are not running a cluster that is entirely in memory. SSDs can greatly mitigate the time required to page data in and out of memory.
|
||||
|
||||
# JBOD vs RAID
|
||||
Historical nodes store large number of segments on Disk and support specifying multiple paths for storing those. Typically, hosts have multiple disks configured with RAID which makes them look like a single disk to OS. RAID might have overheads specially if its not hardware controller based but software based. So, Historicals might get improved disk throughput with JBOD.
|
||||
Historical processes store large number of segments on Disk and support specifying multiple paths for storing those. Typically, hosts have multiple disks configured with RAID which makes them look like a single disk to OS. RAID might have overheads specially if its not hardware controller based but software based. So, Historicals might get improved disk throughput with JBOD.
|
||||
|
||||
# Use Timeseries and TopN Queries Instead of GroupBy Where Possible
|
||||
|
||||
|
|
|
@ -24,7 +24,7 @@ title: "Rolling Updates"
|
|||
|
||||
# Rolling Updates
|
||||
|
||||
For rolling Druid cluster updates with no downtime, we recommend updating Druid nodes in the
|
||||
For rolling Druid cluster updates with no downtime, we recommend updating Druid processes in the
|
||||
following order:
|
||||
|
||||
1. Historical
|
||||
|
@ -38,15 +38,15 @@ following order:
|
|||
|
||||
## Historical
|
||||
|
||||
Historical nodes can be updated one at a time. Each Historical node has a startup time to memory map
|
||||
Historical processes can be updated one at a time. Each Historical process has a startup time to memory map
|
||||
all the segments it was serving before the update. The startup time typically takes a few seconds to
|
||||
a few minutes, depending on the hardware of the node. As long as each Historical node is updated
|
||||
with a sufficient delay (greater than the time required to start a single node), you can rolling
|
||||
a few minutes, depending on the hardware of the host. As long as each Historical process is updated
|
||||
with a sufficient delay (greater than the time required to start a single process), you can rolling
|
||||
update the entire Historical cluster.
|
||||
|
||||
## Overlord
|
||||
|
||||
Overlord nodes can be updated one at a time in a rolling fashion.
|
||||
Overlord processes can be updated one at a time in a rolling fashion.
|
||||
|
||||
## Middle Managers
|
||||
|
||||
|
@ -80,23 +80,23 @@ to `<MiddleManager_IP:PORT>/druid/worker/v1/enable`.
|
|||
|
||||
### Autoscaling-based replacement
|
||||
|
||||
If autoscaling is enabled on your Overlord, then Overlord nodes can launch new Middle Manager nodes
|
||||
If autoscaling is enabled on your Overlord, then Overlord processes can launch new Middle Manager processes
|
||||
en masse and then gracefully terminate old ones as their tasks finish. This process is configured by
|
||||
setting `druid.indexer.runner.minWorkerVersion=#{VERSION}`. Each time you update your Overlord node,
|
||||
setting `druid.indexer.runner.minWorkerVersion=#{VERSION}`. Each time you update your Overlord process,
|
||||
the `VERSION` value should be increased, which will trigger a mass launch of new Middle Managers.
|
||||
|
||||
The config `druid.indexer.autoscale.workerVersion=#{VERSION}` also needs to be set.
|
||||
|
||||
## Standalone Real-time
|
||||
|
||||
Standalone real-time nodes can be updated one at a time in a rolling fashion.
|
||||
Standalone real-time processes can be updated one at a time in a rolling fashion.
|
||||
|
||||
## Broker
|
||||
|
||||
Broker nodes can be updated one at a time in a rolling fashion. There needs to be some delay between
|
||||
updating each node as Brokers must load the entire state of the cluster before they return valid
|
||||
Broker processes can be updated one at a time in a rolling fashion. There needs to be some delay between
|
||||
updating each process as Brokers must load the entire state of the cluster before they return valid
|
||||
results.
|
||||
|
||||
## Coordinator
|
||||
|
||||
Coordinator nodes can be updated one at a time in a rolling fashion.
|
||||
Coordinator processes can be updated one at a time in a rolling fashion.
|
||||
|
|
|
@ -24,14 +24,14 @@ title: "Retaining or Automatically Dropping Data"
|
|||
|
||||
# Retaining or Automatically Dropping Data
|
||||
|
||||
Coordinator nodes use rules to determine what data should be loaded to or dropped from the cluster. Rules are used for data retention and query execution, and are set on the Coordinator console (http://coordinator_ip:port).
|
||||
Coordinator processes use rules to determine what data should be loaded to or dropped from the cluster. Rules are used for data retention and query execution, and are set on the Coordinator console (http://coordinator_ip:port).
|
||||
|
||||
There are three types of rules, i.e., load rules, drop rules, and broadcast rules. Load rules indicate how segments should be assigned to different historical node tiers and how many replicas of a segment should exist in each tier.
|
||||
Drop rules indicate when segments should be dropped entirely from the cluster. Finally, broadcast rules indicate how segments of different data sources should be co-located in historical nodes.
|
||||
There are three types of rules, i.e., load rules, drop rules, and broadcast rules. Load rules indicate how segments should be assigned to different historical process tiers and how many replicas of a segment should exist in each tier.
|
||||
Drop rules indicate when segments should be dropped entirely from the cluster. Finally, broadcast rules indicate how segments of different data sources should be co-located in Historical processes.
|
||||
|
||||
The Coordinator loads a set of rules from the metadata storage. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The Coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule.
|
||||
|
||||
Note: It is recommended that the Coordinator console is used to configure rules. However, the Coordinator node does have HTTP endpoints to programmatically configure rules.
|
||||
Note: It is recommended that the Coordinator console is used to configure rules. However, the Coordinator process does have HTTP endpoints to programmatically configure rules.
|
||||
|
||||
When a rule is updated, the change may not be reflected until the next time the Coordinator runs. This will be fixed in the near future.
|
||||
|
||||
|
@ -173,7 +173,7 @@ The interval of a segment will be compared against the specified period. The per
|
|||
Broadcast Rules
|
||||
---------------
|
||||
|
||||
Broadcast rules indicate how segments of different data sources should be co-located in Historical nodes.
|
||||
Broadcast rules indicate how segments of different data sources should be co-located in Historical processes.
|
||||
Once a broadcast rule is configured for a data source, all segments of the data source are broadcasted to the servers holding _any segments_ of the co-located data sources.
|
||||
|
||||
### Forever Broadcast Rule
|
||||
|
@ -188,7 +188,7 @@ Forever broadcast rules are of the form:
|
|||
```
|
||||
|
||||
* `type` - this should always be "broadcastForever"
|
||||
* `colocatedDataSources` - A JSON List containing data source names to be co-located. `null` and empty list means broadcasting to every node in the cluster.
|
||||
* `colocatedDataSources` - A JSON List containing data source names to be co-located. `null` and empty list means broadcasting to every process in the cluster.
|
||||
|
||||
### Interval Broadcast Rule
|
||||
|
||||
|
@ -203,7 +203,7 @@ Interval broadcast rules are of the form:
|
|||
```
|
||||
|
||||
* `type` - this should always be "broadcastByInterval"
|
||||
* `colocatedDataSources` - A JSON List containing data source names to be co-located. `null` and empty list means broadcasting to every node in the cluster.
|
||||
* `colocatedDataSources` - A JSON List containing data source names to be co-located. `null` and empty list means broadcasting to every process in the cluster.
|
||||
* `interval` - A JSON Object representing ISO-8601 Periods. Only the segments of the interval will be broadcasted.
|
||||
|
||||
### Period Broadcast Rule
|
||||
|
@ -220,7 +220,7 @@ Period broadcast rules are of the form:
|
|||
```
|
||||
|
||||
* `type` - this should always be "broadcastByPeriod"
|
||||
* `colocatedDataSources` - A JSON List containing data source names to be co-located. `null` and empty list means broadcasting to every node in the cluster.
|
||||
* `colocatedDataSources` - A JSON List containing data source names to be co-located. `null` and empty list means broadcasting to every process in the cluster.
|
||||
* `period` - A JSON Object representing ISO-8601 Periods
|
||||
* `includeFuture` - A JSON Boolean indicating whether the load period should include the future. This property is optional, Default is true.
|
||||
|
||||
|
|
|
@ -1,6 +1,6 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "Segment size optimization"
|
||||
title: "Segment Size Optimization"
|
||||
---
|
||||
|
||||
<!--
|
||||
|
@ -22,25 +22,79 @@ title: "Segment size optimization"
|
|||
~ under the License.
|
||||
-->
|
||||
|
||||
# Segment size optimization
|
||||
# Segment Size Optimization
|
||||
|
||||
In Druid, it's important to optimize the segment size because
|
||||
|
||||
1. Druid stores data in segments. If you're using the [best-effort roll-up](../design/index.html#roll-up-modes) mode,
|
||||
increasing the segment size might introduce further aggregation which reduces the dataSource size.
|
||||
2. When a query is submitted, that query is distributed to all Historicals and realtimes
|
||||
which hold the input segments of the query. Each node has a processing threads pool and use one thread per segment to
|
||||
process it. If the segment size is too large, data might not be well distributed over the
|
||||
whole cluster, thereby decreasing the degree of parallelism. If the segment size is too small,
|
||||
each processing thread processes too small data. This might reduce the processing speed of other queries as well as
|
||||
the input query itself because the processing threads are shared for executing all queries.
|
||||
2. When a query is submitted, that query is distributed to all Historicals and realtime tasks
|
||||
which hold the input segments of the query. Each process and task picks a thread from its own processing thread pool
|
||||
to process a single segment. If segment sizes are too large, data might not be well distributed between data
|
||||
servers, decreasing the degree of parallelism possible during query processing.
|
||||
At the other extreme where segment sizes are too small, the scheduling
|
||||
overhead of processing a larger number of segments per query can reduce
|
||||
performance, as the threads that process each segment compete for the fixed
|
||||
slots of the processing pool.
|
||||
|
||||
It would be best if you can optimize the segment size at ingestion time, but sometimes it's not easy
|
||||
especially for the streaming ingestion because the amount of data ingested might vary over time. In this case,
|
||||
you can roughly set the segment size at ingestion time and optimize it later. You have two options:
|
||||
especially when it comes to stream ingestion because the amount of data ingested might vary over time. In this case,
|
||||
you can create segments with a sub-optimzed size first and optimize them later.
|
||||
|
||||
You may need to consider the followings to optimize your segments.
|
||||
|
||||
- Number of rows per segment: it's generally recommended for each segment to have around 5 million rows.
|
||||
This setting is usually _more_ important than the below "segment byte size".
|
||||
This is because Druid uses a single thread to process each segment,
|
||||
and thus this setting can directly control how many rows each thread processes,
|
||||
which in turn means how well the query execution is parallelized.
|
||||
- Segment byte size: it's recommended to set 300 ~ 700MB. If this value
|
||||
doesn't match with the "number of rows per segment", please consider optimizing
|
||||
number of rows per segment rather than this value.
|
||||
|
||||
<div class="note">
|
||||
The above recommendation works in general, but the optimal setting can
|
||||
vary based on your workload. For example, if most of your queries
|
||||
are heavy and take a long time to process each row, you may want to make
|
||||
segments smaller so that the query processing can be more parallelized.
|
||||
If you still see some performance issue after optimizing segment size,
|
||||
you may need to find the optimal settings for your workload.
|
||||
</div>
|
||||
|
||||
There might be several ways to check if the compaction is necessary. One way
|
||||
is using the [System Schema](../querying/sql.html#system-schema). The
|
||||
system schema provides several tables about the current system status including the `segments` table.
|
||||
By running the below query, you can get the average number of rows and average size for published segments.
|
||||
|
||||
```sql
|
||||
SELECT
|
||||
"start",
|
||||
"end",
|
||||
version,
|
||||
COUNT(*) AS num_segments,
|
||||
AVG("num_rows") AS avg_num_rows,
|
||||
SUM("num_rows") AS total_num_rows,
|
||||
AVG("size") AS avg_size,
|
||||
SUM("size") AS total_size
|
||||
FROM
|
||||
sys.segments A
|
||||
WHERE
|
||||
datasource = 'your_dataSource' AND
|
||||
is_published = 1
|
||||
GROUP BY 1, 2, 3
|
||||
ORDER BY 1, 2, 3 DESC;
|
||||
```
|
||||
|
||||
Please note that the query result might include overshadowed segments.
|
||||
In this case, you may want to see only rows of the max version per interval (pair of `start` and `end`).
|
||||
|
||||
Once you find your segments need compaction, you can consider the below two options:
|
||||
|
||||
- Turning on the [automatic compaction of Coordinators](../design/coordinator.html#compacting-segments).
|
||||
The Coordinator periodically submits [compaction tasks](../ingestion/tasks.html#compaction-task) to re-index small segments.
|
||||
To enable the automatic compaction, you need to configure it for each dataSource via Coordinator's dynamic configuration.
|
||||
See [Compaction Configuration API](../operations/api-reference.html#compaction-configuration)
|
||||
and [Compaction Configuration](../configuration/index.html#compaction-dynamic-configuration) for details.
|
||||
- Running periodic Hadoop batch ingestion jobs and using a `dataSource`
|
||||
inputSpec to read from the segments generated by the Kafka indexing tasks. This might be helpful if you want to compact a lot of segments in parallel.
|
||||
Details on how to do this can be found under ['Updating Existing Data'](../ingestion/update-existing-data.html).
|
||||
|
|
|
@ -32,7 +32,7 @@ title: "TLS Support"
|
|||
|`druid.enableTlsPort`|Enable/Disable HTTPS connector.|`false`|
|
||||
|
||||
Although not recommended but both HTTP and HTTPS connectors can be enabled at a time and respective ports are configurable using `druid.plaintextPort`
|
||||
and `druid.tlsPort` properties on each node. Please see `Configuration` section of individual nodes to check the valid and default values for these ports.
|
||||
and `druid.tlsPort` properties on each process. Please see `Configuration` section of individual processes to check the valid and default values for these ports.
|
||||
|
||||
# Jetty Server TLS Configuration
|
||||
|
||||
|
@ -75,7 +75,7 @@ The following table contains non-mandatory advanced configuration options, use c
|
|||
|
||||
# Druid's internal communication over TLS
|
||||
|
||||
Whenever possible Druid nodes will use HTTPS to talk to each other. To enable this communication Druid's HttpClient needs to
|
||||
Whenever possible Druid processes will use HTTPS to talk to each other. To enable this communication Druid's HttpClient needs to
|
||||
be configured with a proper [SSLContext](http://docs.oracle.com/javase/8/docs/api/javax/net/ssl/SSLContext.html) that is able
|
||||
to validate the Server Certificates, otherwise communication will fail.
|
||||
|
||||
|
@ -87,7 +87,7 @@ If this extension does not satisfy the requirements then please follow the exten
|
|||
to create your own extension.
|
||||
|
||||
# Upgrading Clients that interact with Overlord or Coordinator
|
||||
When Druid Coordinator/Overlord have both HTTP and HTTPS enabled and Client sends request to non-leader node, then Client is always redirected to the HTTPS endpoint on leader node.
|
||||
When Druid Coordinator/Overlord have both HTTP and HTTPS enabled and Client sends request to non-leader process, then Client is always redirected to the HTTPS endpoint on leader process.
|
||||
So, Clients should be first upgraded to be able to handle redirect to HTTPS. Then Druid Overlord/Coordinator should be upgraded and configured to run both HTTP and HTTPS ports. Then Client configuration should be changed to refer to Druid Coordinator/Overlord via the HTTPS endpoint and then HTTP port on Druid Coordinator/Overlord should be disabled.
|
||||
|
||||
# Custom TLS certificate checks
|
||||
|
|
|
@ -230,18 +230,18 @@ GroupBy queries can be executed using two different strategies. The default stra
|
|||
the query context. If neither the context field nor the property is set, the "v2" strategy will be used.
|
||||
|
||||
- "v2", the default, is designed to offer better performance and memory management. This strategy generates
|
||||
per-segment results using a fully off-heap map. Data nodes merge the per-segment results using a fully off-heap
|
||||
per-segment results using a fully off-heap map. Data processes merge the per-segment results using a fully off-heap
|
||||
concurrent facts map combined with an on-heap string dictionary. This may optionally involve spilling to disk. Data
|
||||
nodes return sorted results to the Broker, which merges result streams using an N-way merge. The broker materializes
|
||||
processes return sorted results to the Broker, which merges result streams using an N-way merge. The broker materializes
|
||||
the results if necessary (e.g. if the query sorts on columns other than its dimensions). Otherwise, it streams results
|
||||
back as they are merged.
|
||||
|
||||
- "v1", a legacy engine, generates per-segment results on data nodes (Historical, realtime, MiddleManager) using a map which
|
||||
is partially on-heap (dimension keys and the map itself) and partially off-heap (the aggregated values). Data nodes then
|
||||
- "v1", a legacy engine, generates per-segment results on data processes (Historical, realtime, MiddleManager) using a map which
|
||||
is partially on-heap (dimension keys and the map itself) and partially off-heap (the aggregated values). Data processes then
|
||||
merge the per-segment results using Druid's indexing mechanism. This merging is multi-threaded by default, but can
|
||||
optionally be single-threaded. The Broker merges the final result set using Druid's indexing mechanism again. The broker
|
||||
merging is always single-threaded. Because the Broker merges results using the indexing mechanism, it must materialize
|
||||
the full result set before returning any results. On both the data nodes and the Broker, the merging index is fully
|
||||
the full result set before returning any results. On both the data processes and the Broker, the merging index is fully
|
||||
on-heap by default, but it can optionally store aggregated values off-heap.
|
||||
|
||||
#### Differences between v1 and v2
|
||||
|
@ -257,8 +257,8 @@ that can complete successfully in one engine may exceed resource limits and fail
|
|||
- groupBy v1 imposes no limit on the number of concurrently running queries, whereas groupBy v2 controls memory usage
|
||||
by using a finite-sized merge buffer pool. By default, the number of merge buffers is 1/4 the number of processing
|
||||
threads. You can adjust this as necessary to balance concurrency and memory usage.
|
||||
- groupBy v1 supports caching on either the Broker or Historical nodes, whereas groupBy v2 only supports caching on
|
||||
Historical nodes.
|
||||
- groupBy v1 supports caching on either the Broker or Historical processes, whereas groupBy v2 only supports caching on
|
||||
Historical processes.
|
||||
- groupBy v1 supports using [chunkPeriod](query-context.html) to parallelize merging on the Broker, whereas groupBy v2
|
||||
ignores chunkPeriod.
|
||||
- groupBy v2 supports both array-based aggregation and hash-based aggregation. The array-based aggregation is used only
|
||||
|
@ -334,7 +334,7 @@ data is actually spilled (see [Memory tuning and resource limits](#memory-tuning
|
|||
|
||||
Once parallel combine is enabled, the groupBy v2 engine can create a combining tree for merging sorted aggregates. Each
|
||||
intermediate node of the tree is a thread merging aggregates from the child nodes. The leaf node threads read and merge
|
||||
aggregates from hash tables including spilled ones. Usually, leaf nodes are slower than intermediate nodes because they
|
||||
aggregates from hash tables including spilled ones. Usually, leaf processes are slower than intermediate nodes because they
|
||||
need to read data from disk. As a result, less threads are used for intermediate nodes by default. You can change the
|
||||
degree of intermediate nodes. See `intermediateCombineDegree` in [Advanced groupBy v2 configurations](#groupby-v2-configurations).
|
||||
|
||||
|
@ -364,7 +364,7 @@ strategy perform the outer query on the Broker in a single-threaded fashion.
|
|||
|
||||
#### Configurations
|
||||
|
||||
This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on Broker, Historical, and MiddleManager nodes. You can set the query context parameters through the [query context](query-context.html).
|
||||
This section describes the configurations for groupBy queries. You can set the runtime properties in the `runtime.properties` file on Broker, Historical, and MiddleManager processes. You can set the query context parameters through the [query context](query-context.html).
|
||||
|
||||
##### Configurations for groupBy v2
|
||||
|
||||
|
@ -424,7 +424,7 @@ Supported query contexts:
|
|||
|`intermediateCombineDegree`|Overrides the value of `druid.query.groupBy.intermediateCombineDegree`|None|
|
||||
|`numParallelCombineThreads`|Overrides the value of `druid.query.groupBy.numParallelCombineThreads`|None|
|
||||
|`sortByDimsFirst`|Sort the results first by dimension values and then by timestamp.|false|
|
||||
|`forceLimitPushDown`|When all fields in the orderby are part of the grouping key, the Broker will push limit application down to the Historical nodes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.|false|
|
||||
|`forceLimitPushDown`|When all fields in the orderby are part of the grouping key, the Broker will push limit application down to the Historical processes. When the sorting order uses fields that are not in the grouping key, applying this optimization can result in approximate results with unknown accuracy, so this optimization is disabled by default in that case. Enabling this context flag turns on limit push down for limit/orderbys that contain non-grouping key columns.|false|
|
||||
|
||||
|
||||
##### GroupBy v1 configurations
|
||||
|
|
|
@ -118,8 +118,8 @@ These endpoints will return one of the following results:
|
|||
|
||||
## Configuration propagation behavior
|
||||
The configuration is propagated to the query serving processes (Broker / Router / Peon / Historical) by the Coordinator.
|
||||
The query serving nodes have an internal API for managing lookups on the node and those are used by the Coordinator.
|
||||
The Coordinator periodically checks if any of the nodes need to load/drop lookups and updates them appropriately.
|
||||
The query serving processes have an internal API for managing lookups on the process and those are used by the Coordinator.
|
||||
The Coordinator periodically checks if any of the processes need to load/drop lookups and updates them appropriately.
|
||||
|
||||
# API for configuring lookups
|
||||
|
||||
|
@ -291,7 +291,7 @@ To discover a list of tiers currently active in the cluster **instead of** ones
|
|||
A `GET` to `/druid/coordinator/v1/lookups/config/{tier}` will return a list of known lookup names for that tier.
|
||||
|
||||
# Additional API related to status of configured lookups
|
||||
These end points can be used to get the propagation status of configured lookups to lookup nodes such as Historicals.
|
||||
These end points can be used to get the propagation status of configured lookups to processes using lookups such as Historicals.
|
||||
|
||||
## List load status of all lookups
|
||||
`GET /druid/coordinator/v1/lookups/status` with optional query parameter `detailed`.
|
||||
|
@ -302,25 +302,25 @@ These end points can be used to get the propagation status of configured lookups
|
|||
## List load status of single lookup
|
||||
`GET /druid/coordinator/v1/lookups/status/{tier}/{lookup}` with optional query parameter `detailed`.
|
||||
|
||||
## List lookup state of all nodes
|
||||
## List lookup state of all processes
|
||||
`GET /druid/coordinator/v1/lookups/nodeStatus` with optional query parameter `discover` to discover tiers from zookeeper or configured lookup tiers are listed.
|
||||
|
||||
## List lookup state of nodes in a tier
|
||||
## List lookup state of processes in a tier
|
||||
`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}`
|
||||
|
||||
## List lookup state of single node
|
||||
## List lookup state of single process
|
||||
`GET /druid/coordinator/v1/lookups/nodeStatus/{tier}/{host:port}`
|
||||
|
||||
# Internal API
|
||||
|
||||
The Peon, Router, Broker, and Historical nodes all have the ability to consume lookup configuration.
|
||||
There is an internal API these nodes use to list/load/drop their lookups starting at `/druid/listen/v1/lookups`.
|
||||
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 node at `/druid/listen/v1/lookups` will return a json map of all the lookups currently active on the node.
|
||||
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
|
||||
|
@ -339,7 +339,7 @@ The return value will be a json map of the lookups to their extractor factories.
|
|||
|
||||
## Get Lookup
|
||||
|
||||
A `GET` to the node at `/druid/listen/v1/lookups/some_lookup_name` will return the LookupExtractorFactory for the lookup identified by `some_lookup_name`.
|
||||
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
|
||||
|
@ -361,7 +361,7 @@ To configure a Broker / Router / Historical / Peon to announce itself as part of
|
|||
|
||||
|Property | Description | Default |
|
||||
|---------|-------------|---------|
|
||||
|`druid.lookup.lookupTier`| The tier for **lookups** for this node. This is independent of other tiers.|`__default`|
|
||||
|`druid.lookup.lookupTier`| The tier for **lookups** for this process. This is independent of other tiers.|`__default`|
|
||||
|`druid.lookup.lookupTierIsDatasource`|For some things like indexing service tasks, the datasource is passed in the runtime properties of a task. This option fetches the tierName from the same value as the datasource for the task. It is suggested to only use this as Peon options for the indexing service, if at all. If true, `druid.lookup.lookupTier` MUST NOT be specified|`"false"`|
|
||||
|
||||
To configure the behavior of the dynamic configuration manager, use the following properties on the Coordinator:
|
||||
|
@ -369,18 +369,18 @@ To configure the behavior of the dynamic configuration manager, use the followin
|
|||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.manager.lookups.hostTimeout`|Timeout (in ms) PER HOST for processing request|`2000`(2 seconds)|
|
||||
|`druid.manager.lookups.allHostTimeout`|Timeout (in ms) to finish lookup management on all the nodes.|`900000`(15 mins)|
|
||||
|`druid.manager.lookups.allHostTimeout`|Timeout (in ms) to finish lookup management on all the processes.|`900000`(15 mins)|
|
||||
|`druid.manager.lookups.period`|How long to pause between management cycles|`120000`(2 mins)|
|
||||
|`druid.manager.lookups.threadPoolSize`|Number of service nodes that can be managed concurrently|`10`|
|
||||
|`druid.manager.lookups.threadPoolSize`|Number of service processes that can be managed concurrently|`10`|
|
||||
|
||||
## Saving configuration across restarts
|
||||
|
||||
It is possible to save the configuration across restarts such that a node will not have to wait for Coordinator action to re-populate its lookups. To do this the following property is set:
|
||||
It is possible to save the configuration across restarts such that a process will not have to wait for Coordinator action to re-populate its lookups. To do this the following property is set:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.lookup.snapshotWorkingDir`|Working path used to store snapshot of current lookup configuration, leaving this property null will disable snapshot/bootstrap utility|null|
|
||||
|`druid.lookup.enableLookupSyncOnStartup`|Enable the lookup synchronization process with Coordinator on startup. The queryable nodes will fetch and load the lookups from the Coordinator instead of waiting for the Coordinator to load the lookups for them. Users may opt to disable this option if there are no lookups configured in the cluster.|true|
|
||||
|`druid.lookup.enableLookupSyncOnStartup`|Enable the lookup synchronization process with Coordinator on startup. The queryable processes will fetch and load the lookups from the Coordinator instead of waiting for the Coordinator to load the lookups for them. Users may opt to disable this option if there are no lookups configured in the cluster.|true|
|
||||
|`druid.lookup.numLookupLoadingThreads`|Number of threads for loading the lookups in parallel on startup. This thread pool is destroyed once startup is done. It is not kept during the lifetime of the JVM|Available Processors / 2|
|
||||
|`druid.lookup.coordinatorFetchRetries`|How many times to retry to fetch the lookup bean list from Coordinator, during the sync on startup.|3|
|
||||
|`druid.lookup.lookupStartRetries`|How many times to retry to start each lookup, either during the sync on startup, or during the runtime.|3|
|
||||
|
@ -437,7 +437,7 @@ ex: `GET /druid/v1/lookups/introspect/nato-phonetic/values`
|
|||
```
|
||||
|
||||
## Druid version 0.10.0 to 0.10.1 upgrade/downgrade
|
||||
Overall druid cluster lookups configuration is persisted in metadata store and also individual lookup nodes optionally persist a snapshot of loaded lookups on disk.
|
||||
Overall druid cluster lookups configuration is persisted in metadata store and also individual lookup processes optionally persist a snapshot of loaded lookups on disk.
|
||||
If upgrading from druid version 0.10.0 to 0.10.1, then migration for all persisted metadata is handled automatically.
|
||||
If downgrading from 0.10.1 to 0.9.0 then lookups updates done via Coordinator while 0.10.1 was running, would be lost.
|
||||
|
||||
|
|
|
@ -71,7 +71,7 @@ You can use this in concert with single-dimension partitioning to repartition yo
|
|||
|
||||
## Customizing data distribution
|
||||
|
||||
Druid additionally supports multitenancy by providing configurable means of distributing data. Druid's Historical nodes
|
||||
Druid additionally supports multitenancy by providing configurable means of distributing data. Druid's Historical processes
|
||||
can be configured into [tiers](../operations/rule-configuration.html), and [rules](../operations/rule-configuration.html)
|
||||
can be set that determines which segments go into which tiers. One use case of this is that recent data tends to be accessed
|
||||
more frequently than older data. Tiering enables more recent segments to be hosted on more powerful hardware for better performance.
|
||||
|
@ -80,8 +80,8 @@ stored on this tier.
|
|||
|
||||
## Supporting high query concurrency
|
||||
|
||||
Druid's fundamental unit of computation is a [segment](../design/segments.html). Nodes scan segments in parallel and a
|
||||
given node can scan `druid.processing.numThreads` concurrently. To
|
||||
Druid's fundamental unit of computation is a [segment](../design/segments.html). Processes scan segments in parallel and a
|
||||
given process can scan `druid.processing.numThreads` concurrently. To
|
||||
process more data in parallel and increase performance, more cores can be added to a cluster. Druid segments
|
||||
should be sized such that any computation over any given segment should complete in at most 500ms.
|
||||
|
||||
|
@ -94,6 +94,6 @@ that resources are constantly being yielded, and segments pertaining to differen
|
|||
Druid queries can optionally set a `priority` flag in the [query context](../querying/query-context.html). Queries known to be
|
||||
slow (download or reporting style queries) can be de-prioritized and more interactive queries can have higher priority.
|
||||
|
||||
Broker nodes can also be dedicated to a given tier. For example, one set of broker nodes can be dedicated to fast interactive queries,
|
||||
and a second set of Broker nodes can be dedicated to slower reporting queries. Druid also provides a [Router](../development/router.html)
|
||||
node that can route queries to different Brokers based on various query parameters (datasource, interval, etc.).
|
||||
Broker processes can also be dedicated to a given tier. For example, one set of Broker processes can be dedicated to fast interactive queries,
|
||||
and a second set of Broker processes can be dedicated to slower reporting queries. Druid also provides a [Router](../development/router.html)
|
||||
process that can route queries to different Brokers based on various query parameters (datasource, interval, etc.).
|
||||
|
|
|
@ -37,11 +37,11 @@ The query context is used for various query configuration parameters. The follow
|
|||
|populateResultLevelCache | `false` | Flag indicating whether to save the results of the query to the result level cache. Primarily used for debugging. When set to false, it disables saving the results of this query to the query cache. When set to true, Druid uses druid.broker.cache.populateCache to determine whether or not to save the results of this query to the query cache |
|
||||
|bySegment | `false` | Return "by segment" results. Primarily used for debugging, setting it to `true` returns results associated with the data segment they came from |
|
||||
|finalize | `true` | Flag indicating whether to "finalize" aggregation results. Primarily used for debugging. For instance, the `hyperUnique` aggregator will return the full HyperLogLog sketch instead of the estimated cardinality when this flag is set to `false` |
|
||||
|chunkPeriod | `P0D` (off) | At the Broker node level, long interval queries (of any type) may be broken into shorter interval queries to parallelize merging more than normal. Broken up queries will use a larger share of cluster resources, but, if you use groupBy "v1, it may be able to complete faster as a result. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller queries. The broker uses its query processing executor service to initiate processing for query chunks, so make sure "druid.processing.numThreads" is configured appropriately on the broker. [groupBy queries](groupbyquery.html) do not support chunkPeriod by default, although they do if using the legacy "v1" engine. This context is deprecated since it's only useful for groupBy "v1", and will be removed in the future releases.|
|
||||
|maxScatterGatherBytes| `druid.server.http.maxScatterGatherBytes` | Maximum number of bytes gathered from data nodes such as Historicals and realtime processes to execute a query. This parameter can be used to further reduce `maxScatterGatherBytes` limit at query time. See [Broker configuration](../configuration/index.html#broker) for more details.|
|
||||
|chunkPeriod | `P0D` (off) | At the Broker process level, long interval queries (of any type) may be broken into shorter interval queries to parallelize merging more than normal. Broken up queries will use a larger share of cluster resources, but, if you use groupBy "v1, it may be able to complete faster as a result. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller queries. The broker uses its query processing executor service to initiate processing for query chunks, so make sure "druid.processing.numThreads" is configured appropriately on the broker. [groupBy queries](groupbyquery.html) do not support chunkPeriod by default, although they do if using the legacy "v1" engine. This context is deprecated since it's only useful for groupBy "v1", and will be removed in the future releases.|
|
||||
|maxScatterGatherBytes| `druid.server.http.maxScatterGatherBytes` | Maximum number of bytes gathered from data processes such as Historicals and realtime processes to execute a query. This parameter can be used to further reduce `maxScatterGatherBytes` limit at query time. See [Broker configuration](../configuration/index.html#broker) for more details.|
|
||||
|maxQueuedBytes | `druid.broker.http.maxQueuedBytes` | Maximum number of bytes queued per query before exerting backpressure on the channel to the data server. Similar to `maxScatterGatherBytes`, except unlike that configuration, this one will trigger backpressure rather than query failure. Zero means disabled.|
|
||||
|serializeDateTimeAsLong| `false` | If true, DateTime is serialized as long in the result returned by Broker and the data transportation between Broker and compute node|
|
||||
|serializeDateTimeAsLongInner| `false` | If true, DateTime is serialized as long in the data transportation between Broker and compute node|
|
||||
|serializeDateTimeAsLong| `false` | If true, DateTime is serialized as long in the result returned by Broker and the data transportation between Broker and compute process|
|
||||
|serializeDateTimeAsLongInner| `false` | If true, DateTime is serialized as long in the data transportation between Broker and compute process|
|
||||
|
||||
In addition, some query types offer context parameters specific to that query type.
|
||||
|
||||
|
|
|
@ -24,11 +24,11 @@ title: "Querying"
|
|||
|
||||
# Querying
|
||||
|
||||
Queries are made using an HTTP REST style request to queryable nodes ([Broker](../design/broker.html),
|
||||
Queries are made using an HTTP REST style request to queryable processes ([Broker](../design/broker.html),
|
||||
[Historical](../design/historical.html). [Peons](../design/peons.html)) that are running stream ingestion tasks can also accept queries. The
|
||||
query is expressed in JSON and each of these node types expose the same
|
||||
REST query interface. For normal Druid operations, queries should be issued to the Broker nodes. Queries can be posted
|
||||
to the queryable nodes like this -
|
||||
query is expressed in JSON and each of these process types expose the same
|
||||
REST query interface. For normal Druid operations, queries should be issued to the Broker processes. Queries can be posted
|
||||
to the queryable processes like this -
|
||||
|
||||
```bash
|
||||
curl -X POST '<queryable_host>:<port>/druid/v2/?pretty' -H 'Content-Type:application/json' -H 'Accept:application/json' -d @<query_json_file>
|
||||
|
|
|
@ -191,7 +191,7 @@ you may have that is named "timestamp".
|
|||
- Timestamps are returned as ISO8601 time strings rather than integers (milliseconds since 1970-01-01 00:00:00 UTC).
|
||||
|
||||
Legacy mode can be triggered either by passing `"legacy" : true` in your query JSON, or by setting
|
||||
`druid.query.scan.legacy = true` on your Druid nodes. If you were previously using the scan-query contrib extension,
|
||||
`druid.query.scan.legacy = true` on your Druid processes. If you were previously using the scan-query contrib extension,
|
||||
the best way to migrate is to activate legacy mode during a rolling upgrade, then switch it off after the upgrade
|
||||
is complete.
|
||||
|
||||
|
|
|
@ -56,7 +56,7 @@ There are several main parts to a search query:
|
|||
|dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../querying/datasource.html) for more information.|yes|
|
||||
|granularity|Defines the granularity of the query. See [Granularities](../querying/granularities.html).|yes|
|
||||
|filter|See [Filters](../querying/filters.html).|no|
|
||||
|limit| Defines the maximum number per Historical node (parsed as int) of search results to return. |no (default to 1000)|
|
||||
|limit| Defines the maximum number per Historical process (parsed as int) of search results to return. |no (default to 1000)|
|
||||
|intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes|
|
||||
|searchDimensions|The dimensions to run the search over. Excluding this means the search is run over all dimensions.|no|
|
||||
|query|See [SearchQuerySpec](../querying/searchqueryspec.html).|yes|
|
||||
|
|
|
@ -31,7 +31,7 @@ subject to change.
|
|||
|
||||
Druid SQL is a built-in SQL layer and an alternative to Druid's native JSON-based query language, and is powered by a
|
||||
parser and planner based on [Apache Calcite](https://calcite.apache.org/). Druid SQL translates SQL into native Druid
|
||||
queries on the query Broker (the first node you query), which are then passed down to data nodes as native Druid
|
||||
queries on the query Broker (the first process you query), which are then passed down to data processes as native Druid
|
||||
queries. Other than the (slight) overhead of translating SQL on the Broker, there isn't an additional performance
|
||||
penalty versus native queries.
|
||||
|
||||
|
@ -91,7 +91,7 @@ ordinal position (like `ORDER BY 2` to order by the second selected column). For
|
|||
can only order by the `__time` column. For aggregation queries, ORDER BY can order by any column.
|
||||
|
||||
The LIMIT clause can be used to limit the number of rows returned. It can be used with any query type. It is pushed down
|
||||
to data nodes for queries that run with the native TopN query type, but not the native GroupBy query type. Future
|
||||
to data processes for queries that run with the native TopN query type, but not the native GroupBy query type. Future
|
||||
versions of Druid will support pushing down limits using the native GroupBy query type as well. If you notice that
|
||||
adding a limit doesn't change performance very much, then it's likely that Druid didn't push down the limit for your
|
||||
query.
|
||||
|
@ -327,7 +327,7 @@ computed in memory. See the TopN documentation for more details.
|
|||
- [GroupBy](groupbyquery.html) is used for all other aggregations, including any nested aggregation queries. Druid's
|
||||
GroupBy is a traditional aggregation engine: it delivers exact results and rankings and supports a wide variety of
|
||||
features. GroupBy aggregates in memory if it can, but it may spill to disk if it doesn't have enough memory to complete
|
||||
your query. Results are streamed back from data nodes through the Broker if you ORDER BY the same expressions in your
|
||||
your query. Results are streamed back from data processes through the Broker if you ORDER BY the same expressions in your
|
||||
GROUP BY clause, or if you don't have an ORDER BY at all. If your query has an ORDER BY referencing expressions that
|
||||
don't appear in the GROUP BY clause (like aggregation functions) then the Broker will materialize a list of results in
|
||||
memory, up to a max of your LIMIT, if any. See the GroupBy documentation for details about tuning performance and memory
|
||||
|
@ -345,7 +345,7 @@ of plan.
|
|||
|
||||
For all native query types, filters on the `__time` column will be translated into top-level query "intervals" whenever
|
||||
possible, which allows Druid to use its global time index to quickly prune the set of data that must be scanned. In
|
||||
addition, Druid will use indexes local to each data node to further speed up WHERE evaluation. This can typically be
|
||||
addition, Druid will use indexes local to each data process to further speed up WHERE evaluation. This can typically be
|
||||
done for filters that involve boolean combinations of references to and functions of single columns, like
|
||||
`WHERE col1 = 'a' AND col2 = 'b'`, but not `WHERE col1 = col2`.
|
||||
|
||||
|
@ -476,7 +476,7 @@ so avoid those.
|
|||
|
||||
Druid's JDBC server does not share connection state between Brokers. This means that if you're using JDBC and have
|
||||
multiple Druid Brokers, you should either connect to a specific Broker, or use a load balancer with sticky sessions
|
||||
enabled. The Druid Router node provides connection stickiness when balancing JDBC requests, and can be used to achieve
|
||||
enabled. The Druid Router process provides connection stickiness when balancing JDBC requests, and can be used to achieve
|
||||
the necessary stickiness even with a normal non-sticky load balancer. Please see the
|
||||
[Router](../development/router.html) documentation for more details.
|
||||
|
||||
|
|
|
@ -26,7 +26,7 @@ title: "TopN queries"
|
|||
|
||||
TopN queries return a sorted set of results for the values in a given dimension according to some criteria. Conceptually, they can be thought of as an approximate [GroupByQuery](../querying/groupbyquery.html) over a single dimension with an [Ordering](../querying/limitspec.html) spec. TopNs are much faster and resource efficient than GroupBys for this use case. These types of queries take a topN query object and return an array of JSON objects where each object represents a value asked for by the topN query.
|
||||
|
||||
TopNs are approximate in that each node will rank their top K results and only return those top K results to the Broker. K, by default in Druid, is `max(1000, threshold)`. In practice, this means that if you ask for the top 1000 items ordered, the correctness of the first ~900 items will be 100%, and the ordering of the results after that is not guaranteed. TopNs can be made more accurate by increasing the threshold.
|
||||
TopNs are approximate in that each data process will rank their top K results and only return those top K results to the Broker. K, by default in Druid, is `max(1000, threshold)`. In practice, this means that if you ask for the top 1000 items ordered, the correctness of the first ~900 items will be 100%, and the ordering of the results after that is not guaranteed. TopNs can be made more accurate by increasing the threshold.
|
||||
|
||||
A topN query object looks like:
|
||||
|
||||
|
|
|
@ -130,6 +130,7 @@ layout: toc
|
|||
* [Different Hadoop Versions](/docs/VERSION/operations/other-hadoop.html)
|
||||
* [High Availability](/docs/VERSION/operations/high-availability.html)
|
||||
* [Performance FAQ](/docs/VERSION/operations/performance-faq.html)
|
||||
* [Management UIs](/docs/VERSION/operations/management-uis.html)
|
||||
* [Dump Segment Tool](/docs/VERSION/operations/dump-segment.html)
|
||||
* [Insert Segment Tool](/docs/VERSION/operations/insert-segment-to-db.html)
|
||||
* [Pull Dependencies Tool](/docs/VERSION/operations/pull-deps.html)
|
||||
|
|
|
@ -182,7 +182,7 @@ druid.indexer.logs.directory=/druid/indexing-logs
|
|||
Also,
|
||||
|
||||
- Place your Hadoop configuration XMLs (core-site.xml, hdfs-site.xml, yarn-site.xml,
|
||||
mapred-site.xml) on the classpath of your Druid nodes. You can do this by copying them into
|
||||
mapred-site.xml) on the classpath of your Druid processes. You can do this by copying them into
|
||||
`conf/druid/_common/`.
|
||||
|
||||
## Configure Tranquility Server (optional)
|
||||
|
@ -207,7 +207,7 @@ a path on HDFS that you'd like to use for temporary files required during the in
|
|||
`druid.indexer.task.hadoopWorkingPath=/tmp/druid-indexing` is a common choice.
|
||||
|
||||
- Place your Hadoop configuration XMLs (core-site.xml, hdfs-site.xml, yarn-site.xml,
|
||||
mapred-site.xml) on the classpath of your Druid nodes. You can do this by copying them into
|
||||
mapred-site.xml) on the classpath of your Druid processes. You can do this by copying them into
|
||||
`conf/druid/_common/core-site.xml`, `conf/druid/_common/hdfs-site.xml`, and so on.
|
||||
|
||||
Note that you don't need to use HDFS deep storage in order to load data from Hadoop. For example, if
|
||||
|
@ -263,7 +263,7 @@ hardware. The most commonly adjusted configurations are:
|
|||
- `druid.processing.numThreads`
|
||||
- `druid.query.groupBy.maxIntermediateRows`
|
||||
- `druid.query.groupBy.maxResults`
|
||||
- `druid.server.maxSize` and `druid.segmentCache.locations` on Historical Nodes
|
||||
- `druid.server.maxSize` and `druid.segmentCache.locations` on Historical processes
|
||||
- `druid.worker.capacity` on MiddleManagers
|
||||
|
||||
<div class="note info">
|
||||
|
|
Before Width: | Height: | Size: 88 KiB After Width: | Height: | Size: 53 KiB |
Before Width: | Height: | Size: 220 KiB After Width: | Height: | Size: 54 KiB |
Before Width: | Height: | Size: 28 KiB After Width: | Height: | Size: 273 KiB |
After Width: | Height: | Size: 39 KiB |
After Width: | Height: | Size: 305 KiB |
After Width: | Height: | Size: 39 KiB |
After Width: | Height: | Size: 343 KiB |
After Width: | Height: | Size: 39 KiB |
After Width: | Height: | Size: 42 KiB |
Before Width: | Height: | Size: 108 KiB After Width: | Height: | Size: 70 KiB |
Before Width: | Height: | Size: 127 KiB After Width: | Height: | Size: 196 KiB |
Before Width: | Height: | Size: 214 KiB After Width: | Height: | Size: 53 KiB |
Before Width: | Height: | Size: 76 KiB After Width: | Height: | Size: 401 KiB |
Before Width: | Height: | Size: 135 KiB After Width: | Height: | Size: 43 KiB |
After Width: | Height: | Size: 66 KiB |
After Width: | Height: | Size: 60 KiB |
After Width: | Height: | Size: 228 KiB |
|
@ -96,13 +96,13 @@ This will bring up instances of Zookeeper and the Druid services, all running on
|
|||
|
||||
```bash
|
||||
bin/supervise -c quickstart/tutorial/conf/tutorial-cluster.conf
|
||||
[Thu Jul 26 12:16:23 2018] Running command[zk], logging to[/stage/apache-druid-#{DRUIDVERSION}/var/sv/zk.log]: bin/run-zk quickstart/tutorial/conf
|
||||
[Thu Jul 26 12:16:23 2018] Running command[coordinator], logging to[/stage/apache-druid-#{DRUIDVERSION}/var/sv/coordinator.log]: bin/run-druid coordinator quickstart/tutorial/conf
|
||||
[Thu Jul 26 12:16:23 2018] Running command[broker], logging to[//stage/apache-druid-#{DRUIDVERSION}/var/sv/broker.log]: bin/run-druid broker quickstart/tutorial/conf
|
||||
[Thu Jul 26 12:16:23 2018] Running command[historical], logging to[/stage/apache-druid-#{DRUIDVERSION}/var/sv/historical.log]: bin/run-druid historical quickstart/tutorial/conf
|
||||
[Thu Jul 26 12:16:23 2018] Running command[overlord], logging to[/stage/apache-druid-#{DRUIDVERSION}/var/sv/overlord.log]: bin/run-druid overlord quickstart/tutorial/conf
|
||||
[Thu Jul 26 12:16:23 2018] Running command[middleManager], logging to[/stage/apache-druid-#{DRUIDVERSION}/var/sv/middleManager.log]: bin/run-druid middleManager quickstart/tutorial/conf
|
||||
|
||||
[Wed Feb 27 12:46:13 2019] Running command[zk], logging to[/apache-druid-#{DRUIDVERSION}/var/sv/zk.log]: bin/run-zk quickstart/tutorial/conf
|
||||
[Wed Feb 27 12:46:13 2019] Running command[coordinator], logging to[/apache-druid-#{DRUIDVERSION}/var/sv/coordinator.log]: bin/run-druid coordinator quickstart/tutorial/conf
|
||||
[Wed Feb 27 12:46:13 2019] Running command[broker], logging to[/apache-druid-#{DRUIDVERSION}/var/sv/broker.log]: bin/run-druid broker quickstart/tutorial/conf
|
||||
[Wed Feb 27 12:46:13 2019] Running command[router], logging to[/apache-druid-#{DRUIDVERSION}/var/sv/router.log]: bin/run-druid router quickstart/tutorial/conf
|
||||
[Wed Feb 27 12:46:13 2019] Running command[historical], logging to[/apache-druid-#{DRUIDVERSION}/var/sv/historical.log]: bin/run-druid historical quickstart/tutorial/conf
|
||||
[Wed Feb 27 12:46:13 2019] Running command[overlord], logging to[/apache-druid-#{DRUIDVERSION}/var/sv/overlord.log]: bin/run-druid overlord quickstart/tutorial/conf
|
||||
[Wed Feb 27 12:46:13 2019] Running command[middleManager], logging to[/apache-druid-#{DRUIDVERSION}/var/sv/middleManager.log]: bin/run-druid middleManager quickstart/tutorial/conf
|
||||
```
|
||||
|
||||
All persistent state such as the cluster metadata store and segments for the services will be kept in the `var` directory under the apache-druid-#{DRUIDVERSION} package root. Logs for the services are located at `var/sv`.
|
||||
|
|
|
@ -163,16 +163,16 @@ Which will print the ID of the task if the submission was successful:
|
|||
{"task":"index_wikipedia_2018-06-09T21:30:32.802Z"}
|
||||
```
|
||||
|
||||
To view the status of the ingestion task, go to the Overlord console:
|
||||
[http://localhost:8090/console.html](http://localhost:8090/console.html). You can refresh the console periodically, and after
|
||||
the task is successful, you should see a "SUCCESS" status for the task.
|
||||
To view the status of the ingestion task, go to the Druid Console:
|
||||
[http://localhost:8888/](http://localhost:8888). You can refresh the console periodically, and after
|
||||
the task is successful, you should see a "SUCCESS" status for the task under the [Tasks view](http://localhost:8888/unified-console.html#tasks).
|
||||
|
||||
After the ingestion task finishes, the data will be loaded by Historical nodes and available for
|
||||
After the ingestion task finishes, the data will be loaded by Historical processes and available for
|
||||
querying within a minute or two. You can monitor the progress of loading the data in the
|
||||
Coordinator console, by checking whether there is a datasource "wikipedia" with a blue circle
|
||||
indicating "fully available": [http://localhost:8081/#/](http://localhost:8081/#/).
|
||||
Datasources view, by checking whether there is a datasource "wikipedia" with a green circle
|
||||
indicating "fully available": [http://localhost:8888/unified-console.html#datasources](http://localhost:8888/unified-console.html#datasources).
|
||||
|
||||
![Coordinator console](../tutorials/img/tutorial-batch-01.png "Wikipedia 100% loaded")
|
||||
![Druid Console](../tutorials/img/tutorial-batch-01.png "Wikipedia 100% loaded")
|
||||
|
||||
## Further reading
|
||||
|
||||
|
|
|
@ -27,6 +27,7 @@ title: "Tutorial: Compacting segments"
|
|||
This tutorial demonstrates how to compact existing segments into fewer but larger segments.
|
||||
|
||||
Because there is some per-segment memory and processing overhead, it can sometimes be beneficial to reduce the total number of segments.
|
||||
Please check [Segment size optimization](../operations/segment-optimization.html) for details.
|
||||
|
||||
For this tutorial, we'll assume you've already downloaded Druid as described in
|
||||
the [single-machine quickstart](index.html) and have it running on your local machine.
|
||||
|
@ -35,7 +36,7 @@ It will also be helpful to have finished [Tutorial: Loading a file](../tutorials
|
|||
|
||||
## Load the initial data
|
||||
|
||||
For this tutorial, we'll be using the Wikipedia edits sample data, with an ingestion task spec that will create a separate segment for each hour in the input data.
|
||||
For this tutorial, we'll be using the Wikipedia edits sample data, with an ingestion task spec that will create 1-3 segments per hour in the input data.
|
||||
|
||||
The ingestion spec can be found at `quickstart/tutorial/compaction-init-index.json`. Let's submit that spec, which will create a datasource called `compaction-tutorial`:
|
||||
|
||||
|
@ -43,11 +44,20 @@ The ingestion spec can be found at `quickstart/tutorial/compaction-init-index.js
|
|||
bin/post-index-task --file quickstart/tutorial/compaction-init-index.json
|
||||
```
|
||||
|
||||
After the ingestion completes, go to http://localhost:8081/#/datasources/compaction-tutorial in a browser to view information about the new datasource in the Coordinator console.
|
||||
<div class="note caution">
|
||||
Please note that `maxRowsPerSegment` in the ingestion spec is set to 1000. This is to generate multiple segments per hour and _NOT_ recommended in production.
|
||||
It's 5000000 by default and may need to be adjusted to make your segments optimized.
|
||||
</div>
|
||||
|
||||
There will be 24 segments for this datasource, one segment per hour in the input data:
|
||||
After the ingestion completes, go to [http://localhost:8888/unified-console.html#datasources](http://localhost:8888/unified-console.html#datasources) in a browser to see the new datasource in the Druid Console.
|
||||
|
||||
![Original segments](../tutorials/img/tutorial-retention-01.png "Original segments")
|
||||
![compaction-tutorial datasource](../tutorials/img/tutorial-compaction-01.png "compaction-tutorial datasource")
|
||||
|
||||
Click the `51 segments` link next to "Fully Available" for the `compaction-tutorial` datasource to view information about the datasource's segments:
|
||||
|
||||
There will be 51 segments for this datasource, 1-3 segments per hour in the input data:
|
||||
|
||||
![Original segments](../tutorials/img/tutorial-compaction-02.png "Original segments")
|
||||
|
||||
Running a COUNT(*) query on this datasource shows that there are 39,244 rows:
|
||||
|
||||
|
@ -63,9 +73,9 @@ Retrieved 1 row in 1.38s.
|
|||
|
||||
## Compact the data
|
||||
|
||||
Let's now combine these 24 segments into one segment.
|
||||
Let's now compact these 51 small segments.
|
||||
|
||||
We have included a compaction task spec for this tutorial datasource at `quickstart/tutorial/compaction-final-index.json`:
|
||||
We have included a compaction task spec for this tutorial datasource at `quickstart/tutorial/compaction-keep-granularity.json`:
|
||||
|
||||
```json
|
||||
{
|
||||
|
@ -85,23 +95,25 @@ This will compact all segments for the interval `2015-09-12/2015-09-13` in the `
|
|||
|
||||
The parameters in the `tuningConfig` control how many segments will be present in the compacted set of segments.
|
||||
|
||||
In this tutorial example, only one compacted segment will be created, as the 39244 rows in the input is less than the 5000000 `maxRowsPerSegment`.
|
||||
In this tutorial example, only one compacted segment will be created per hour, as each hour has less rows than the 5000000 `maxRowsPerSegment` (note that the total number of rows is 39244).
|
||||
|
||||
Let's submit this task now:
|
||||
|
||||
```bash
|
||||
bin/post-index-task --file quickstart/tutorial/compaction-final-index.json
|
||||
bin/post-index-task --file quickstart/tutorial/compaction-keep-granularity.json
|
||||
```
|
||||
|
||||
After the task finishes, refresh the http://localhost:8081/#/datasources/compaction-tutorial page.
|
||||
After the task finishes, refresh the [segments view](http://localhost:8888/unified-console.html#segments).
|
||||
|
||||
The original 24 segments will eventually be marked as "unused" by the Coordinator and removed, with the new compacted segment remaining.
|
||||
The original 51 segments will eventually be marked as "unused" by the Coordinator and removed, with the new compacted segments remaining.
|
||||
|
||||
By default, the Druid Coordinator will not mark segments as unused until the Coordinator process has been up for at least 15 minutes, so you may see the old segment set and the new compacted set at the same time in the Coordinator, e.g.:
|
||||
By default, the Druid Coordinator will not mark segments as unused until the Coordinator process has been up for at least 15 minutes, so you may see the old segment set and the new compacted set at the same time in the Druid Console, with 75 total segments:
|
||||
|
||||
![Compacted segments intermediate state](../tutorials/img/tutorial-compaction-01.png "Compacted segments intermediate state")
|
||||
![Compacted segments intermediate state 1](../tutorials/img/tutorial-compaction-03.png "Compacted segments intermediate state 1")
|
||||
|
||||
The new compacted segment has a more recent version than the original segments, so even when both sets of segments are shown by the Coordinator, queries will only read from the new compacted segment.
|
||||
![Compacted segments intermediate state 2](../tutorials/img/tutorial-compaction-04.png "Compacted segments intermediate state 2")
|
||||
|
||||
The new compacted segments have a more recent version than the original segments, so even when both sets of segments are shown in the Druid Console, queries will only read from the new compacted segments.
|
||||
|
||||
Let's try running a COUNT(*) on `compaction-tutorial` again, where the row count should still be 39,244:
|
||||
|
||||
|
@ -115,12 +127,50 @@ dsql> select count(*) from "compaction-tutorial";
|
|||
Retrieved 1 row in 1.30s.
|
||||
```
|
||||
|
||||
After the Coordinator has been running for at least 15 minutes, the http://localhost:8081/#/datasources/compaction-tutorial page should show there is only 1 segment:
|
||||
After the Coordinator has been running for at least 15 minutes, the [segments view](http://localhost:8888/unified-console.html#segments) should show there are 24 segments, one per hour:
|
||||
|
||||
![Compacted segments hourly granularity 1](../tutorials/img/tutorial-compaction-05.png "Compacted segments hourly granularity 1")
|
||||
|
||||
![Compacted segments hourly granularity 2](../tutorials/img/tutorial-compaction-06.png "Compacted segments hourly granularity 2")
|
||||
|
||||
## Compact the data with new segment granularity
|
||||
|
||||
The compaction task can also produce compacted segments with a granularity different from the granularity of the input segments.
|
||||
|
||||
We have included a compaction task spec that will create DAY granularity segments at `quickstart/tutorial/compaction-day-granularity.json`:
|
||||
|
||||
```json
|
||||
{
|
||||
"type": "compact",
|
||||
"dataSource": "compaction-tutorial",
|
||||
"interval": "2015-09-12/2015-09-13",
|
||||
"segmentGranularity": "DAY",
|
||||
"tuningConfig" : {
|
||||
"type" : "index",
|
||||
"maxRowsPerSegment" : 5000000,
|
||||
"maxRowsInMemory" : 25000,
|
||||
"forceExtendableShardSpecs" : true
|
||||
}
|
||||
}
|
||||
```
|
||||
|
||||
Note that `segmentGranularity` is set to `DAY` in this compaction task spec.
|
||||
|
||||
Let's submit this task now:
|
||||
|
||||
```bash
|
||||
bin/post-index-task --file quickstart/tutorial/compaction-day-granularity.json
|
||||
```
|
||||
|
||||
It will take a bit of time before the Coordinator marks the old input segments as unused, so you may see an intermediate state with 25 total segments. Eventually, there will only be one DAY granularity segment:
|
||||
|
||||
![Compacted segments day granularity 1](../tutorials/img/tutorial-compaction-07.png "Compacted segments day granularity 1")
|
||||
|
||||
![Compacted segments day granularity 2](../tutorials/img/tutorial-compaction-08.png "Compacted segments day granularity 2")
|
||||
|
||||
![Compacted segments final state](../tutorials/img/tutorial-compaction-02.png "Compacted segments final state")
|
||||
|
||||
## Further reading
|
||||
|
||||
[Task documentation](../ingestion/tasks.html)
|
||||
|
||||
[Segment optimization](../operations/segment-optimization.html)
|
||||
[Segment optimization](../operations/segment-optimization.html)
|
|
@ -41,7 +41,7 @@ Let's load this initial data:
|
|||
bin/post-index-task --file quickstart/tutorial/deletion-index.json
|
||||
```
|
||||
|
||||
When the load finishes, open http://localhost:8081/#/datasources/deletion-tutorial in a browser.
|
||||
When the load finishes, open [http://localhost:8888/unified-console.html#datasources](http://localhost:8888/unified-console.html#datasources) in a browser.
|
||||
|
||||
## How to permanently delete data
|
||||
|
||||
|
@ -56,15 +56,17 @@ Let's drop some segments now, first with load rules, then manually.
|
|||
|
||||
As with the previous retention tutorial, there are currently 24 segments in the `deletion-tutorial` datasource.
|
||||
|
||||
Click the `edit rules` button with a pencil icon at the upper left corner of the page.
|
||||
click the blue pencil icon next to `Cluster default: loadForever` for the `deletion-tutorial` datasource.
|
||||
|
||||
A rule configuration window will appear. Enter `tutorial` for both the user and changelog comment field.
|
||||
A rule configuration window will appear.
|
||||
|
||||
Now click the `+ Add a rule` button twice.
|
||||
Now click the `+ New rule` button twice.
|
||||
|
||||
In the `rule #1` box at the top, click `Load`, `Interval`, enter `2015-09-12T12:00:00.000Z/2015-09-13T00:00:00.000Z` in the interval box, and click `+ _default_tier replicant`.
|
||||
In the upper rule box, select `Load` and `by interval`, and then enter `2015-09-12T12:00:00.000Z/2015-09-13T00:00:00.000Z` in field next to `by interval`. Replicants can remain at 2 in the `_default_tier`.
|
||||
|
||||
In the `rule #2` box at the bottom, click `Drop` and `Forever`.
|
||||
In the lower rule box, select `Drop` and `forever`.
|
||||
|
||||
Now click `Next` and enter `tutorial` for both the user and changelog comment field.
|
||||
|
||||
This will cause the first 12 segments of `deletion-tutorial` to be dropped. However, these dropped segments are not removed from deep storage.
|
||||
|
||||
|
@ -102,11 +104,11 @@ $ ls -l1 var/druid/segments/deletion-tutorial/
|
|||
|
||||
Let's manually disable a segment now. This will mark a segment as "unused", but not remove it from deep storage.
|
||||
|
||||
On http://localhost:8081/#/datasources/deletion-tutorial, click one of the remaining segments on the left for full details about the segment:
|
||||
In the [segments view](http://localhost:8888/unified-console.html#segments), click the arrow on the left side of one of the remaining segments to expand the segment entry:
|
||||
|
||||
![Segments](../tutorials/img/tutorial-deletion-01.png "Segments")
|
||||
|
||||
The top of the info box shows the full segment ID, e.g. `deletion-tutorial_2016-06-27T14:00:00.000Z_2016-06-27T15:00:00.000Z_2018-07-27T22:57:00.110Z` for the segment of hour 14.
|
||||
The top of the info box shows the full segment ID, e.g. `deletion-tutorial_2015-09-12T14:00:00.000Z_2015-09-12T15:00:00.000Z_2019-02-28T01:11:51.606Z` for the segment of hour 14.
|
||||
|
||||
Let's disable the hour 14 segment by sending the following DELETE request to the Coordinator, where {SEGMENT-ID} is the full segment ID shown in the info box:
|
||||
|
||||
|
|
|
@ -70,6 +70,8 @@ If the supervisor was successfully created, you will get a response containing t
|
|||
For more details about what's going on here, check out the
|
||||
[Druid Kafka indexing service documentation](../development/extensions-core/kafka-ingestion.html).
|
||||
|
||||
You can view the current supervisors and tasks in the Druid Console: [http://localhost:8888/unified-console.html#tasks](http://localhost:8888/unified-console.html#tasks).
|
||||
|
||||
## Load data
|
||||
|
||||
Let's launch a console producer for our topic and send some data!
|
||||
|
|
|
@ -41,49 +41,54 @@ The ingestion spec can be found at `quickstart/tutorial/retention-index.json`. L
|
|||
bin/post-index-task --file quickstart/tutorial/retention-index.json
|
||||
```
|
||||
|
||||
After the ingestion completes, go to http://localhost:8081 in a browser to access the Coordinator console.
|
||||
After the ingestion completes, go to [http://localhost:8888/unified-console.html#datasources](http://localhost:8888/unified-console.html#datasources) in a browser to access the Druid Console's datasource view.
|
||||
|
||||
In the Coordinator console, go to the `datasources` tab at the top of the page.
|
||||
This view shows the available datasources and a summary of the retention rules for each datasource:
|
||||
|
||||
This tab shows the available datasources and a summary of the retention rules for each datasource:
|
||||
![Summary](../tutorials/img/tutorial-retention-01.png "Summary")
|
||||
|
||||
![Summary](../tutorials/img/tutorial-retention-00.png "Summary")
|
||||
Currently there are no rules set for the `retention-tutorial` datasource. Note that there are default rules for the cluster: load forever with 2 replicants in `_default_tier`.
|
||||
|
||||
Currently there are no rules set for the `retention-tutorial` datasource. Note that there are default rules, currently set to `load Forever 2 in _default_tier`.
|
||||
|
||||
This means that all data will be loaded regardless of timestamp, and each segment will be replicated to two nodes in the default tier.
|
||||
This means that all data will be loaded regardless of timestamp, and each segment will be replicated to two Historical processes in the default tier.
|
||||
|
||||
In this tutorial, we will ignore the tiering and redundancy concepts for now.
|
||||
|
||||
Let's click the `retention-tutorial` datasource on the left.
|
||||
Let's view the segments for the `retention-tutorial` datasource by clicking the "24 Segments" link next to "Fully Available".
|
||||
|
||||
The next page (http://localhost:8081/#/datasources/retention-tutorial) provides information about what segments a datasource contains. On the left, the page shows that there are 24 segments, each one containing data for a specific hour of 2015-09-12:
|
||||
The segments view ([http://localhost:8888/unified-console.html#segments](http://localhost:8888/unified-console.html#segments)) provides information about what segments a datasource contains. The page shows that there are 24 segments, each one containing data for a specific hour of 2015-09-12:
|
||||
|
||||
![Original segments](../tutorials/img/tutorial-retention-01.png "Original segments")
|
||||
![Original segments](../tutorials/img/tutorial-retention-02.png "Original segments")
|
||||
|
||||
## Set retention rules
|
||||
|
||||
Suppose we want to drop data for the first 12 hours of 2015-09-12 and keep data for the later 12 hours of 2015-09-12.
|
||||
|
||||
Click the `edit rules` button with a pencil icon at the upper left corner of the page.
|
||||
Go to the [datasources view](http://localhost:8888/unified-console.html#datasources) and click the blue pencil icon next to `Cluster default: loadForever` for the `retention-tutorial` datasource.
|
||||
|
||||
A rule configuration window will appear. Enter `tutorial` for both the user and changelog comment field.
|
||||
A rule configuration window will appear:
|
||||
|
||||
Now click the `+ Add a rule` button twice.
|
||||
![Rule configuration](../tutorials/img/tutorial-retention-03.png "Rule configuration")
|
||||
|
||||
In the `rule #1` box at the top, click `Load`, `Interval`, enter `2015-09-12T12:00:00.000Z/2015-09-13T00:00:00.000Z` in the interval box, and click `+ _default_tier replicant`.
|
||||
Now click the `+ New rule` button twice.
|
||||
|
||||
In the `rule #2` box at the bottom, click `Drop` and `Forever`.
|
||||
In the upper rule box, select `Load` and `by interval`, and then enter `2015-09-12T12:00:00.000Z/2015-09-13T00:00:00.000Z` in field next to `by interval`. Replicants can remain at 2 in the `_default_tier`.
|
||||
|
||||
In the lower rule box, select `Drop` and `forever`.
|
||||
|
||||
The rules should look like this:
|
||||
|
||||
![Set rules](../tutorials/img/tutorial-retention-02.png "Set rules")
|
||||
![Set rules](../tutorials/img/tutorial-retention-04.png "Set rules")
|
||||
|
||||
Now click `Save all rules`, wait for a few seconds, and refresh the page.
|
||||
Now click `Next`. The rule configuration process will ask for a user name and comment, for change logging purposes. You can enter `tutorial` for both.
|
||||
|
||||
Now click `Save`. You can see the new rules in the datasources view:
|
||||
|
||||
![New rules](../tutorials/img/tutorial-retention-05.png "New rules")
|
||||
|
||||
Give the cluster a few minutes to apply the rule change, and go to the [segments view](http://localhost:8888/unified-console.html#segments) in the Druid Console.
|
||||
The segments for the first 12 hours of 2015-09-12 are now gone:
|
||||
|
||||
![New segments](../tutorials/img/tutorial-retention-03.png "New segments")
|
||||
![New segments](../tutorials/img/tutorial-retention-06.png "New segments")
|
||||
|
||||
The resulting retention rule chain is the following:
|
||||
|
||||
|
@ -93,7 +98,6 @@ The resulting retention rule chain is the following:
|
|||
|
||||
3. loadForever (default rule)
|
||||
|
||||
|
||||
The rule chain is evaluated from top to bottom, with the default rule chain always added at the bottom.
|
||||
|
||||
The tutorial rule chain we just created loads data if it is within the specified 12 hour interval.
|
||||
|
|
|
@ -136,6 +136,10 @@ def raise_friendly_error(e):
|
|||
if error_obj['host']:
|
||||
error_text = error_text + ' (' + str(error_obj['host']) + ')'
|
||||
raise DruidSqlException(error_text)
|
||||
elif e.code == 405:
|
||||
error_text = 'HTTP Error {0}: {1}\n{2}'.format(e.code, e.reason + " - Are you using the correct broker URL and " +\
|
||||
"is druid.sql.enabled set to true on your broker?", text)
|
||||
raise DruidSqlException(error_text)
|
||||
else:
|
||||
raise DruidSqlException("HTTP Error {0}: {1}\n{2}".format(e.code, e.reason, text))
|
||||
else:
|
||||
|
|
|
@ -0,0 +1,12 @@
|
|||
{
|
||||
"type": "compact",
|
||||
"dataSource": "compaction-tutorial",
|
||||
"interval": "2015-09-12/2015-09-13",
|
||||
"segmentGranularity": "DAY",
|
||||
"tuningConfig" : {
|
||||
"type" : "index",
|
||||
"maxRowsPerSegment" : 5000000,
|
||||
"maxRowsInMemory" : 25000,
|
||||
"forceExtendableShardSpecs" : true
|
||||
}
|
||||
}
|