Native parallel batch indexing without shuffle (#5492)

* Native parallel indexing without shuffle

* fix build

* fix ci

* fix ingestion without intervals

* fix retry

* fix retry

* add it test

* use chat handler

* fix build

* add docs

* fix ITUnionQueryTest

* fix failures

* disable metrics reporting

* working

* Fix split of static-s3 firehose

* Add endpoints to supervisor task and a unit test for endpoints

* increase timeout in test

* Added doc

* Address comments

* Fix overlapping locks

* address comments

* Fix static s3 firehose

* Fix test

* fix build

* fix test

* fix typo in docs

* add missing maxBytesInMemory to doc

* address comments

* fix race in test

* fix test

* Rename to ParallelIndexSupervisorTask

* fix teamcity

* address comments

* Fix license

* addressing comments

* addressing comments

* indexTaskClient-based segmentAllocator instead of CountingActionBasedSegmentAllocator

* Fix race in TaskMonitor and move HTTP endpoints to supervisorTask from runner

* Add more javadocs

* use StringUtils.nonStrictFormat for logging

* fix typo and remove unused class

* fix tests

* change package

* fix strict build

* tmp

* Fix overlord api according to the recent change in master

* Fix it test
This commit is contained in:
Jihoon Son 2018-08-06 23:59:42 -07:00 committed by GitHub
parent 62677212cc
commit 56ab4363ea
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
118 changed files with 8263 additions and 947 deletions

View File

@ -0,0 +1,68 @@
/*
* 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 io.druid.data.input;
import com.fasterxml.jackson.annotation.JsonIgnore;
import io.druid.data.input.impl.InputRowParser;
import java.io.IOException;
import java.util.stream.Stream;
/**
* {@link FiniteFirehoseFactory} designed for batch processing. Its implementations assume that the amount of inputs is
* limited.
*
* @param <T> parser type
* @param <S> input split type
*/
public interface FiniteFirehoseFactory<T extends InputRowParser, S> extends FirehoseFactory<T>
{
/**
* Returns true if this {@link FiniteFirehoseFactory} supports parallel batch indexing.
*/
@JsonIgnore
@Override
default boolean isSplittable()
{
return true;
}
/**
* Returns a {@link Stream} for {@link InputSplit}s. In parallel batch indexing, each {@link InputSplit} is processed
* by a sub task.
*
* Listing splits may cause high overhead in some implementations. In this case, {@link InputSplit}s should be listed
* lazily so that the listing overhead could be amortized.
*/
@JsonIgnore
Stream<InputSplit<S>> getSplits() throws IOException;
/**
* Returns number of splits returned by {@link #getSplits()}.
*/
@JsonIgnore
int getNumSplits() throws IOException;
/**
* Returns the same {@link FiniteFirehoseFactory} but with the given {@link InputSplit}. The returned
* {@link FiniteFirehoseFactory} is used by sub tasks in parallel batch indexing.
*/
FiniteFirehoseFactory<T, S> withSplit(InputSplit<S> split);
}

View File

@ -72,4 +72,9 @@ public interface FirehoseFactory<T extends InputRowParser>
{
return connect(parser);
}
default boolean isSplittable()
{
return false;
}
}

View File

@ -0,0 +1,52 @@
/*
* 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 io.druid.data.input;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
/**
* Input unit for distributed batch ingestion. Used in {@link FiniteFirehoseFactory}.
* An {@link InputSplit} represents the input data processed by a {@code io.druid.indexing.common.task.Task}.
*/
public class InputSplit<T>
{
private final T split;
@JsonCreator
public InputSplit(@JsonProperty("split") T split)
{
this.split = split;
}
@JsonProperty("split")
public T get()
{
return split;
}
@Override
public String toString()
{
return "InputSplit{" +
"split=" + split +
"}";
}
}

View File

@ -22,20 +22,22 @@ package io.druid.data.input.impl;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.InputSplit;
import io.druid.java.util.common.logger.Logger;
import org.apache.commons.io.Charsets;
import org.apache.commons.io.IOUtils;
import org.apache.commons.io.LineIterator;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.stream.Stream;
/**
* This is an abstract class for firehose factory for making firehoses reading text files.
@ -44,7 +46,7 @@ import java.util.NoSuchElementException;
* @param <T> object type representing input data
*/
public abstract class AbstractTextFilesFirehoseFactory<T>
implements FirehoseFactory<StringInputRowParser>
implements FiniteFirehoseFactory<StringInputRowParser, T>
{
private static final Logger LOG = new Logger(AbstractTextFilesFirehoseFactory.class);
@ -53,9 +55,7 @@ public abstract class AbstractTextFilesFirehoseFactory<T>
@Override
public Firehose connect(StringInputRowParser firehoseParser, File temporaryDirectory) throws IOException
{
if (objects == null) {
objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects"));
}
initializeObjectsIfNeeded();
final Iterator<T> iterator = objects.iterator();
return new FileIteratingFirehose(
new Iterator<LineIterator>()
@ -74,7 +74,7 @@ public abstract class AbstractTextFilesFirehoseFactory<T>
}
final T object = iterator.next();
try {
return IOUtils.lineIterator(wrapObjectStream(object, openObjectStream(object)), Charsets.UTF_8);
return IOUtils.lineIterator(wrapObjectStream(object, openObjectStream(object)), StandardCharsets.UTF_8);
}
catch (Exception e) {
LOG.error(
@ -90,6 +90,32 @@ public abstract class AbstractTextFilesFirehoseFactory<T>
);
}
protected void initializeObjectsIfNeeded() throws IOException
{
if (objects == null) {
objects = ImmutableList.copyOf(Preconditions.checkNotNull(initObjects(), "initObjects"));
}
}
public List<T> getObjects()
{
return objects;
}
@Override
public Stream<InputSplit<T>> getSplits() throws IOException
{
initializeObjectsIfNeeded();
return getObjects().stream().map(InputSplit::new);
}
@Override
public int getNumSplits() throws IOException
{
initializeObjectsIfNeeded();
return getObjects().size();
}
/**
* Initialize objects to be read by this firehose. Since firehose factories are constructed whenever
* io.druid.indexing.common.task.Task objects are deserialized, actual initialization of objects is deferred

View File

@ -25,5 +25,5 @@ public enum RunnerTaskState
WAITING,
PENDING,
RUNNING,
NONE; // is used for a completed task
NONE // is used for a completed task
}

View File

@ -172,4 +172,20 @@ public class TaskStatusPlus
getErrorMsg()
);
}
@Override
public String toString()
{
return "TaskStatusPlus{" +
"id='" + id + '\'' +
", type='" + type + '\'' +
", createdTime=" + createdTime +
", queueInsertionTime=" + queueInsertionTime +
", state=" + state +
", duration=" + duration +
", location=" + location +
", dataSource='" + dataSource + '\'' +
", errorMsg='" + errorMsg + '\'' +
'}';
}
}

View File

@ -23,7 +23,9 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.Lists;
import com.google.common.io.CountingOutputStream;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.Firehose;
import io.druid.data.input.InputSplit;
import io.druid.data.input.Row;
import io.druid.data.input.impl.CSVParseSpec;
import io.druid.data.input.impl.DimensionsSpec;
@ -605,6 +607,12 @@ public class PrefetchableTextFilesFirehoseFactoryTest
private int readCount;
private int numConnectionResets;
@Override
public FiniteFirehoseFactory<StringInputRowParser, File> withSplit(InputSplit<File> split)
{
throw new UnsupportedOperationException();
}
private class TestInputStream extends InputStream
{
private static final int NUM_READ_COUNTS_BEFORE_ERROR = 10;

View File

@ -78,6 +78,9 @@ public interface MetadataStorageActionHandler<EntryType, StatusType, LogType, Lo
*/
Optional<StatusType> getStatus(String entryId);
@Nullable
TaskInfo<EntryType, StatusType> getTaskInfo(String entryId);
/**
* Return up to {@code maxNumStatuses} {@link TaskInfo} objects for all inactive entries
* created on or later than the given timestamp

View File

@ -33,6 +33,9 @@ The storage account is shared with the one used for Azure deep storage functiona
As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz
This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task).
Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object.
Sample spec:
```json

View File

@ -32,6 +32,9 @@ The storage account is shared with the one used for Racksapce's Cloud Files deep
As with the Azure blobstore, it is assumed to be gzipped if the extension ends in .gz
This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task).
Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object.
Sample spec:
```json

View File

@ -27,6 +27,9 @@ This firehose ingests events, similar to the StaticS3Firehose, but from an Googl
As with the S3 blobstore, it is assumed to be gzipped if the extension ends in .gz
This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task).
Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object.
Sample spec:
```json

View File

@ -43,6 +43,8 @@ You can enable [server-side encryption](https://docs.aws.amazon.com/AmazonS3/lat
## StaticS3Firehose
This firehose ingests events from a predefined list of S3 objects.
This firehose is _splittable_ and can be used by [native parallel index tasks](../../ingestion/native_tasks.html#parallel-index-task).
Since each split represents an object in this firehose, each worker task of `index_parallel` will read an object.
Sample spec:

View File

@ -20,6 +20,8 @@ For additional firehoses, please see our [extensions list](../development/extens
This Firehose can be used to read the data from files on local disk.
It can be used for POCs to ingest data on disk.
This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#parallel-index-task).
Since each split represents a file in this firehose, each worker task of `index_parallel` will read a file.
A sample local firehose spec is shown below:
```json
@ -39,6 +41,8 @@ A sample local firehose spec is shown below:
#### HttpFirehose
This Firehose can be used to read the data from remote sites via HTTP.
This firehose is _splittable_ and can be used by [native parallel index tasks](./native_tasks.html#parallel-index-task).
Since each split represents a file in this firehose, each worker task of `index_parallel` will read a file.
A sample http firehose spec is shown below:
```json

View File

@ -0,0 +1,530 @@
---
layout: doc_page
---
# Native Index Tasks
Druid currently has two types of native batch indexing tasks, `index_parallel` which runs tasks
in parallel on multiple middle manager nodes, and `index` which will run a single indexing task locally on a single
middle manager.
Parallel Index Task
--------------------------------
The Parallel Index Task is a task for parallel batch indexing. This task only uses Druid's resource and
doesn't depend on other external systems like Hadoop. This task currently works in a single phase without shuffling intermediate
data. `index_parallel` task is a supervisor task which basically generates multiple worker tasks and submits
them to overlords. Each worker task reads input data and makes segments. Once they successfully generate segments for all
input, they report the generated segment list to the supervisor task. The supervisor task periodically checks the worker
task statuses. If one of them fails, it retries the failed task until the retrying number reaches the configured limit.
If all worker tasks succeed, then it collects the reported list of generated segments and publishes those segments at once.
To use this task, the `firehose` in `ioConfig` should be _splittable_. If it's not, this task runs sequentially. The
current splittable fireshoses are [`LocalFirehose`](./firehose.html#localfirehose), [`HttpFirehose`](./firehose.html#httpfirehose)
, [`StaticS3Firehose`](../development/extensions-core/s3.html#statics3firehose), [`StaticAzureBlobStoreFirehose`](../development/extensions-contrib/azure.html#staticazureblobstorefirehose)
, [`StaticGoogleBlobStoreFirehose`](../development/extensions-contrib/google.html#staticgoogleblobstorefirehose), and [`StaticCloudFilesFirehose`](../development/extensions-contrib/cloudfiles.html#staticcloudfilesfirehose).
The splittable firehose is responsible for generating _splits_. The supervisor task generates _worker task specs_ each of
which specifies a split and submits worker tasks using those specs. As a result, the number of worker tasks depends on
the implementation of splittable firehoses. Please note that multiple tasks can be created for the same worker task spec
if one of them fails.
Since this task doesn't shuffle intermediate data, it isn't available for [perfect rollup](../design/index.html).
An example ingestion spec is:
```json
{
"type": "index_parallel",
"spec": {
"dataSchema": {
"dataSource": "wikipedia_parallel_index_test",
"metricsSpec": [
{
"type": "count",
"name": "count"
},
{
"type": "doubleSum",
"name": "added",
"fieldName": "added"
},
{
"type": "doubleSum",
"name": "deleted",
"fieldName": "deleted"
},
{
"type": "doubleSum",
"name": "delta",
"fieldName": "delta"
}
],
"granularitySpec": {
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
"language",
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
}
},
"ioConfig": {
"type": "index_parallel",
"firehose": {
"type": "local",
"baseDir": "examples/indexing/",
"filter": "wikipedia_index_data*"
}
}
}
}
```
#### Task Properties
|property|description|required?|
|--------|-----------|---------|
|type|The task type, this should always be `index_parallel`.|yes|
|id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no|
|spec|The ingestion spec including the data schema, IOConfig, and TuningConfig. See below for more details. |yes|
|context|Context containing various task configuration parameters. See below for more details.|no|
#### DataSchema
This field is required.
See [Ingestion](../ingestion/index.html)
#### IOConfig
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|The task type, this should always be `index_parallel`.|none|yes|
|firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes|
|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no|
#### TuningConfig
The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. See below for more details.
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|The task type, this should always be `index_parallel`.|none|yes|
|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no|
|maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no|
|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no|
|maxTotalRows|Total number of rows in segments waiting for being pushed. Used in determining when intermediate pushing should occur.|150000|no|
|numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no|
|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no|
|maxPendingPersists|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).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no|
|forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no|
|reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no|
|pushTimeout|Milliseconds to wait for pushing segments. It must be >= 0, where 0 means to wait forever.|0|no|
|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no|
|maxNumSubTasks|Maximum number of tasks which can be run at the same time.|Integer.MAX_VALUE|no|
|maxRetry|Maximum number of retries on task failures.|3|no|
|taskStatusCheckPeriodMs|Polling period in milleseconds to check running task statuses.|1000|no|
|chatHandlerTimeout|Timeout for reporting the pushed segments in worker tasks.|PT10S|no|
|chatHandlerNumRetries|Retries for reporting the pushed segments in worker tasks.|5|no|
#### HTTP Endpoints
The supervisor task provides some HTTP endpoints to get running status.
* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/mode`
Returns 'parallel' if the indexing task is running in parallel. Otherwise, it returns 'sequential'.
* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/progress`
Returns the current progress if the supervisor task is running in the parallel mode.
An example of the result is
```json
{
"running":10,
"succeeded":0,
"failed":0,
"complete":0,
"total":10,
"expectedSucceeded":10
}
```
* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtasks/running`
Returns the task IDs of running worker tasks, or an empty list if the supervisor task is running in the sequential mode.
* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs`
Returns all worker task specs, or an empty list if the supervisor task is running in the sequential mode.
* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs/running`
Returns running worker task specs, or an empty list if the supervisor task is running in the sequential mode.
* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspecs/complete`
Returns complete worker task specs, or an empty list if the supervisor task is running in the sequential mode.
* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}`
Returns the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode.
* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}/state`
Returns the state of the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode.
The returned result contains the worker task spec, a current task status if exists, and task attempt history.
An example of the result is
```json
{
"spec": {
"id": "index_parallel_lineitem_2018-04-20T22:12:43.610Z_2",
"groupId": "index_parallel_lineitem_2018-04-20T22:12:43.610Z",
"supervisorTaskId": "index_parallel_lineitem_2018-04-20T22:12:43.610Z",
"context": null,
"inputSplit": {
"split": "/path/to/data/lineitem.tbl.5"
},
"ingestionSpec": {
"dataSchema": {
"dataSource": "lineitem",
"parser": {
"type": "hadoopyString",
"parseSpec": {
"format": "tsv",
"delimiter": "|",
"timestampSpec": {
"column": "l_shipdate",
"format": "yyyy-MM-dd"
},
"dimensionsSpec": {
"dimensions": [
"l_orderkey",
"l_partkey",
"l_suppkey",
"l_linenumber",
"l_returnflag",
"l_linestatus",
"l_shipdate",
"l_commitdate",
"l_receiptdate",
"l_shipinstruct",
"l_shipmode",
"l_comment"
]
},
"columns": [
"l_orderkey",
"l_partkey",
"l_suppkey",
"l_linenumber",
"l_quantity",
"l_extendedprice",
"l_discount",
"l_tax",
"l_returnflag",
"l_linestatus",
"l_shipdate",
"l_commitdate",
"l_receiptdate",
"l_shipinstruct",
"l_shipmode",
"l_comment"
]
}
},
"metricsSpec": [
{
"type": "count",
"name": "count"
},
{
"type": "longSum",
"name": "l_quantity",
"fieldName": "l_quantity",
"expression": null
},
{
"type": "doubleSum",
"name": "l_extendedprice",
"fieldName": "l_extendedprice",
"expression": null
},
{
"type": "doubleSum",
"name": "l_discount",
"fieldName": "l_discount",
"expression": null
},
{
"type": "doubleSum",
"name": "l_tax",
"fieldName": "l_tax",
"expression": null
}
],
"granularitySpec": {
"type": "uniform",
"segmentGranularity": "YEAR",
"queryGranularity": {
"type": "none"
},
"rollup": true,
"intervals": [
"1980-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z"
]
},
"transformSpec": {
"filter": null,
"transforms": []
}
},
"ioConfig": {
"type": "index_parallel",
"firehose": {
"type": "local",
"baseDir": "/path/to/data/",
"filter": "lineitem.tbl.5",
"parser": null
},
"appendToExisting": false
},
"tuningConfig": {
"type": "index_parallel",
"targetPartitionSize": 5000000,
"maxRowsInMemory": 1000000,
"maxTotalRows": 20000000,
"numShards": null,
"indexSpec": {
"bitmap": {
"type": "concise"
},
"dimensionCompression": "lz4",
"metricCompression": "lz4",
"longEncoding": "longs"
},
"maxPendingPersists": 0,
"forceExtendableShardSpecs": false,
"reportParseExceptions": false,
"pushTimeout": 0,
"segmentWriteOutMediumFactory": null,
"maxNumSubTasks": 2147483647,
"maxRetry": 3,
"taskStatusCheckPeriodMs": 1000,
"chatHandlerTimeout": "PT10S",
"chatHandlerNumRetries": 5,
"logParseExceptions": false,
"maxParseExceptions": 2147483647,
"maxSavedParseExceptions": 0,
"forceGuaranteedRollup": false,
"buildV9Directly": true
}
}
},
"currentStatus": {
"id": "index_sub_lineitem_2018-04-20T22:16:29.922Z",
"type": "index_sub",
"createdTime": "2018-04-20T22:16:29.925Z",
"queueInsertionTime": "2018-04-20T22:16:29.929Z",
"statusCode": "RUNNING",
"duration": -1,
"location": {
"host": null,
"port": -1,
"tlsPort": -1
},
"dataSource": "lineitem",
"errorMsg": null
},
"taskHistory": []
}
```
* `http://{PEON_IP}:{PEON_PORT}/druid/worker/v1/chat/{SUPERVISOR_TASK_ID}/subtaskspec/{SUB_TASK_SPEC_ID}/history`
Returns the task attempt history of the worker task spec of the given id, or HTTP 404 Not Found error if the supervisor task is running in the sequential mode.
Local Index Task
----------------
The Local Index Task is designed to be used for smaller data sets. The task executes within the indexing service. The grammar of the index task is as follows:
```json
{
"type" : "index",
"spec" : {
"dataSchema" : {
"dataSource" : "wikipedia",
"parser" : {
"type" : "string",
"parseSpec" : {
"format" : "json",
"timestampSpec" : {
"column" : "timestamp",
"format" : "auto"
},
"dimensionsSpec" : {
"dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"],
"dimensionExclusions" : [],
"spatialDimensions" : []
}
}
},
"metricsSpec" : [
{
"type" : "count",
"name" : "count"
},
{
"type" : "doubleSum",
"name" : "added",
"fieldName" : "added"
},
{
"type" : "doubleSum",
"name" : "deleted",
"fieldName" : "deleted"
},
{
"type" : "doubleSum",
"name" : "delta",
"fieldName" : "delta"
}
],
"granularitySpec" : {
"type" : "uniform",
"segmentGranularity" : "DAY",
"queryGranularity" : "NONE",
"intervals" : [ "2013-08-31/2013-09-01" ]
}
},
"ioConfig" : {
"type" : "index",
"firehose" : {
"type" : "local",
"baseDir" : "examples/indexing/",
"filter" : "wikipedia_data.json"
}
},
"tuningConfig" : {
"type" : "index",
"targetPartitionSize" : 5000000,
"maxRowsInMemory" : 1000000
}
}
}
```
#### Task Properties
|property|description|required?|
|--------|-----------|---------|
|type|The task type, this should always be "index".|yes|
|id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no|
|spec|The ingestion spec including the data schema, IOConfig, and TuningConfig. See below for more details. |yes|
|context|Context containing various task configuration parameters. See below for more details.|no|
#### DataSchema
This field is required.
See [Ingestion](../ingestion/index.html)
#### IOConfig
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|The task type, this should always be "index".|none|yes|
|firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes|
|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no|
#### TuningConfig
The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. See below for more details.
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|The task type, this should always be "index".|none|yes|
|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no|
|maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no|
|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no|
|maxTotalRows|Total number of rows in segments waiting for being pushed. Used in determining when intermediate pushing should occur.|20000000|no|
|numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no|
|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no|
|maxPendingPersists|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).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no|
|forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no|
|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment pushing modes__ section.|false|no|
|reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no|
|pushTimeout|Milliseconds to wait for pushing segments. It must be >= 0, where 0 means to wait forever.|0|no|
|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no|
#### IndexSpec
The indexSpec defines segment storage format options to be used at indexing time, such as bitmap type and column
compression formats. The indexSpec is optional and default parameters will be used if not specified.
|Field|Type|Description|Required|
|-----|----|-----------|--------|
|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)|
|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)|
|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)|
|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)|
##### Bitmap types
For Concise bitmaps:
|Field|Type|Description|Required|
|-----|----|-----------|--------|
|type|String|Must be `concise`.|yes|
For Roaring bitmaps:
|Field|Type|Description|Required|
|-----|----|-----------|--------|
|type|String|Must be `roaring`.|yes|
|compressRunOnSerialization|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)|
#### Segment pushing modes
While ingesting data using the Index task, it creates segments from the input data and pushes them. For segment pushing,
the Index task supports two segment pushing modes, i.e., _bulk pushing mode_ and _incremental pushing mode_ for
[perfect rollup and best-effort rollup](./design/index.html), 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
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,
the index task immediately pushes all segments created until that moment, cleans all pushed segments up, and
continues to ingest remaining data.
To enable bulk pushing mode, `forceGuaranteedRollup` should be set in the TuningConfig. Note that this option cannot
be used with either `forceExtendableShardSpecs` of TuningConfig or `appendToExisting` of IOConfig.

View File

@ -13,178 +13,10 @@ Segment Creation Tasks
See [batch ingestion](../ingestion/batch-ingestion.html).
### Index Task
### Native Index Tasks
The Index Task is a simpler variation of the Index Hadoop task that is designed to be used for smaller data sets. The task executes within the indexing service and does not require an external Hadoop setup to use. The grammar of the index task is as follows:
```json
{
"type" : "index",
"spec" : {
"dataSchema" : {
"dataSource" : "wikipedia",
"parser" : {
"type" : "string",
"parseSpec" : {
"format" : "json",
"timestampSpec" : {
"column" : "timestamp",
"format" : "auto"
},
"dimensionsSpec" : {
"dimensions": ["page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","country","region","city"],
"dimensionExclusions" : [],
"spatialDimensions" : []
}
}
},
"metricsSpec" : [
{
"type" : "count",
"name" : "count"
},
{
"type" : "doubleSum",
"name" : "added",
"fieldName" : "added"
},
{
"type" : "doubleSum",
"name" : "deleted",
"fieldName" : "deleted"
},
{
"type" : "doubleSum",
"name" : "delta",
"fieldName" : "delta"
}
],
"granularitySpec" : {
"type" : "uniform",
"segmentGranularity" : "DAY",
"queryGranularity" : "NONE",
"intervals" : [ "2013-08-31/2013-09-01" ]
}
},
"ioConfig" : {
"type" : "index",
"firehose" : {
"type" : "local",
"baseDir" : "examples/indexing/",
"filter" : "wikipedia_data.json"
}
},
"tuningConfig" : {
"type" : "index",
"targetPartitionSize" : 5000000,
"maxRowsInMemory" : 1000000
}
}
}
```
#### Task Properties
|property|description|required?|
|--------|-----------|---------|
|type|The task type, this should always be "index".|yes|
|id|The task ID. If this is not explicitly specified, Druid generates the task ID using task type, data source name, interval, and date-time stamp. |no|
|spec|The ingestion spec including the data schema, IOConfig, and TuningConfig. See below for more details. |yes|
|context|Context containing various task configuration parameters. See below for more details.|no|
#### Task Priority
Druid's indexing tasks use locks for atomic data ingestion. Each lock is acquired for the combination of a dataSource and an interval. Once a task acquires a lock, it can write data for the dataSource and the interval of the acquired lock unless the lock is released or preempted. Please see [the below Locking section](#locking)
Each task has a priority which is used for lock acquisition. The locks of higher-priority tasks can preempt the locks of lower-priority tasks if they try to acquire for the same dataSource and interval. If some locks of a task are preempted, the behavior of the preempted task depends on the task implementation. Usually, most tasks finish as failed if they are preempted.
Tasks can have different default priorities depening on their types. Here are a list of default priorities. Higher the number, higher the priority.
|task type|default priority|
|---------|----------------|
|Realtime index task|75|
|Batch index task|50|
|Merge/Append/Compaction task|25|
|Other tasks|0|
You can override the task priority by setting your priority in the task context like below.
```json
"context" : {
"priority" : 100
}
```
#### DataSchema
This field is required.
See [Ingestion](../ingestion/index.html)
#### IOConfig
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|The task type, this should always be "index".|none|yes|
|firehose|Specify a [Firehose](../ingestion/firehose.html) here.|none|yes|
|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This will only work if the existing segment set has extendable-type shardSpecs (which can be forced by setting 'forceExtendableShardSpecs' in the tuning config).|false|no|
#### TuningConfig
The tuningConfig is optional and default parameters will be used if no tuningConfig is specified. See below for more details.
|property|description|default|required?|
|--------|-----------|-------|---------|
|type|The task type, this should always be "index".|none|yes|
|targetPartitionSize|Used in sharding. Determines how many rows are in each segment.|5000000|no|
|maxRowsInMemory|Used in determining when intermediate persists to disk should occur. Normally user does not need to set this, but depending on the nature of data, if rows are short in terms of bytes, user may not want to store a million rows in memory and this value should be set.|1000000|no|
|maxBytesInMemory|Used in determining when intermediate persists to disk should occur. Normally this is computed internally and user does not need to set it. This value represents number of bytes to aggregate in heap memory before persisting. This is based on a rough estimate of memory usage and not actual usage. The maximum heap memory usage for indexing is maxBytesInMemory * (2 + maxPendingPersists)|1/6 of max JVM memory|no|
|maxTotalRows|Total number of rows in segments waiting for being published. Used in determining when intermediate publish should occur.|20000000|no|
|numShards|Directly specify the number of shards to create. If this is specified and 'intervals' is specified in the granularitySpec, the index task can skip the determine intervals/partitions pass through the data. numShards cannot be specified if targetPartitionSize is set.|null|no|
|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no|
|maxPendingPersists|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).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no|
|forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no|
|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no|
|reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no|
|publishTimeout|Milliseconds to wait for publishing segments. It must be >= 0, where 0 means to wait forever.|0|no|
|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no|
#### IndexSpec
The indexSpec defines segment storage format options to be used at indexing time, such as bitmap type and column
compression formats. The indexSpec is optional and default parameters will be used if not specified.
|Field|Type|Description|Required|
|-----|----|-----------|--------|
|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)|
|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)|
|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)|
|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)|
##### Bitmap types
For Concise bitmaps:
|Field|Type|Description|Required|
|-----|----|-----------|--------|
|type|String|Must be `concise`.|yes|
For Roaring bitmaps:
|Field|Type|Description|Required|
|-----|----|-----------|--------|
|type|String|Must be `roaring`.|yes|
|compressRunOnSerialization|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)|
#### Segment publishing modes
While ingesting data using the Index task, it creates segments from the input data and publishes them. For segment publishing, the Index task supports two segment publishing modes, i.e., _bulk publishing mode_ and _incremental publishing mode_ for [perfect rollup and best-effort rollup](./design/index.html), respectively.
In the bulk publishing mode, every segment is published 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 problem due to limited storage capacity, and is not recommended to use in production.
On the contrary, in the incremental publishing mode, segments are incrementally published, that is they can be published 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, the index task immediately publishes all segments created until that moment, cleans all published segments up, and continues to ingest remaining data.
To enable bulk publishing mode, `forceGuaranteedRollup` should be set in the TuningConfig. Note that this option cannot be used with either `forceExtendableShardSpecs` of TuningConfig or `appendToExisting` of IOConfig.
Druid provides a native index task which doesn't need any dependencies on other systems.
See [native index tasks](./native_tasks.html) for more details.
Segment Merging Tasks
---------------------
@ -382,6 +214,33 @@ The values are described below.
Unlike the hadoop convert task, the indexing service task draws its output path from the indexing service's configuration.
#### IndexSpec
The indexSpec defines segment storage format options to be used at indexing time, such as bitmap type and column
compression formats. The indexSpec is optional and default parameters will be used if not specified.
|Field|Type|Description|Required|
|-----|----|-----------|--------|
|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)|
|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)|
|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)|
|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)|
##### Bitmap types
For Concise bitmaps:
|Field|Type|Description|Required|
|-----|----|-----------|--------|
|type|String|Must be `concise`.|yes|
For Roaring bitmaps:
|Field|Type|Description|Required|
|-----|----|-----------|--------|
|type|String|Must be `roaring`.|yes|
|compressRunOnSerialization|Boolean|Use a run-length encoding where it is estimated as more space efficient.|no (default == `true`)|
### Noop Task
These tasks start, sleep for a time and are used only for testing. The available grammar is:
@ -428,3 +287,27 @@ A task writing data into a dataSource must acquire exclusive locks for target in
Tasks do not need to explicitly release locks, they are released upon task completion. Tasks may potentially release
locks early if they desire. Task ids are unique by naming them using UUIDs or the timestamp in which the task was created.
Tasks are also part of a "task group", which is a set of tasks that can share interval locks.
Task Priority
-------------
Druid's indexing tasks use locks for atomic data ingestion. Each lock is acquired for the combination of a dataSource and an interval. Once a task acquires a lock, it can write data for the dataSource and the interval of the acquired lock unless the lock is released or preempted. Please see [the below Locking section](#locking)
Each task has a priority which is used for lock acquisition. The locks of higher-priority tasks can preempt the locks of lower-priority tasks if they try to acquire for the same dataSource and interval. If some locks of a task are preempted, the behavior of the preempted task depends on the task implementation. Usually, most tasks finish as failed if they are preempted.
Tasks can have different default priorities depening on their types. Here are a list of default priorities. Higher the number, higher the priority.
|task type|default priority|
|---------|----------------|
|Realtime index task|75|
|Batch index task|50|
|Merge/Append/Compaction task|25|
|Other tasks|0|
You can override the task priority by setting your priority in the task context like below.
```json
"context" : {
"priority" : 100
}
```

View File

@ -24,6 +24,9 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.InputSplit;
import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
import io.druid.java.util.common.CompressionUtils;
import io.druid.storage.azure.AzureByteSource;
@ -33,6 +36,7 @@ import io.druid.storage.azure.AzureUtils;
import java.io.IOException;
import java.io.InputStream;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
@ -143,4 +147,18 @@ public class StaticAzureBlobStoreFirehoseFactory extends PrefetchableTextFilesFi
{
return AzureUtils.AZURE_RETRY;
}
@Override
public FiniteFirehoseFactory<StringInputRowParser, AzureBlob> withSplit(InputSplit<AzureBlob> split)
{
return new StaticAzureBlobStoreFirehoseFactory(
azureStorage,
Collections.singletonList(split.get()),
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
}

View File

@ -23,6 +23,9 @@ import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Predicate;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.InputSplit;
import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.logger.Logger;
@ -34,6 +37,7 @@ import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
import java.io.IOException;
import java.io.InputStream;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
@ -142,4 +146,18 @@ public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFireho
{
return CloudFilesUtils.CLOUDFILESRETRY;
}
@Override
public FiniteFirehoseFactory<StringInputRowParser, CloudFilesBlob> withSplit(InputSplit<CloudFilesBlob> split)
{
return new StaticCloudFilesFirehoseFactory(
cloudFilesApi,
Collections.singletonList(split.get()),
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
}

View File

@ -23,6 +23,9 @@ import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Predicate;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.InputSplit;
import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
import io.druid.java.util.common.CompressionUtils;
import io.druid.storage.google.GoogleByteSource;
@ -32,6 +35,7 @@ import io.druid.storage.google.GoogleUtils;
import java.io.IOException;
import java.io.InputStream;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
@ -134,5 +138,19 @@ public class StaticGoogleBlobStoreFirehoseFactory extends PrefetchableTextFilesF
{
return GoogleUtils::isRetryable;
}
@Override
public FiniteFirehoseFactory<StringInputRowParser, GoogleBlob> withSplit(InputSplit<GoogleBlob> split)
{
return new StaticGoogleBlobStoreFirehoseFactory(
storage,
Collections.singletonList(split.get()),
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
}

View File

@ -33,6 +33,7 @@ import io.druid.indexer.TaskStatus;
import io.druid.indexing.appenderator.ActionBasedSegmentAllocator;
import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.SegmentAllocateAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.stats.RowIngestionMetersFactory;
import io.druid.indexing.common.task.AbstractTask;
@ -258,7 +259,19 @@ public class KafkaIndexTask extends AbstractTask implements ChatHandler
{
return new StreamAppenderatorDriver(
appenderator,
new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema),
new ActionBasedSegmentAllocator(
toolbox.getTaskActionClient(),
dataSchema,
(schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction(
schema.getDataSource(),
row.getTimestamp(),
schema.getGranularitySpec().getQueryGranularity(),
schema.getGranularitySpec().getSegmentGranularity(),
sequenceName,
previousSegmentId,
skipSegmentLineageCheck
)
),
toolbox.getSegmentHandoffNotifierFactory(),
new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
toolbox.getDataSegmentKiller(),

View File

@ -21,80 +21,31 @@ package io.druid.indexing.kafka;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import io.druid.indexer.TaskLocation;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.RetryPolicy;
import io.druid.indexing.common.RetryPolicyConfig;
import io.druid.indexing.common.RetryPolicyFactory;
import io.druid.indexing.common.IndexTaskClient;
import io.druid.indexing.common.TaskInfoProvider;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.concurrent.Execs;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.emitter.EmittingLogger;
import io.druid.java.util.http.client.HttpClient;
import io.druid.java.util.http.client.Request;
import io.druid.java.util.http.client.response.FullResponseHandler;
import io.druid.java.util.http.client.response.FullResponseHolder;
import io.druid.segment.realtime.firehose.ChatHandlerResource;
import org.jboss.netty.channel.ChannelException;
import org.jboss.netty.handler.codec.http.HttpMethod;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Period;
import javax.ws.rs.core.MediaType;
import javax.annotation.Nullable;
import java.io.IOException;
import java.net.Socket;
import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.Map;
import java.util.TreeMap;
import java.util.concurrent.Callable;
public class KafkaIndexTaskClient
public class KafkaIndexTaskClient extends IndexTaskClient
{
public static class NoTaskLocationException extends RuntimeException
{
public NoTaskLocationException(String message)
{
super(message);
}
}
public static class TaskNotRunnableException extends RuntimeException
{
public TaskNotRunnableException(String message)
{
super(message);
}
}
public static final int MAX_RETRY_WAIT_SECONDS = 10;
private static final int MIN_RETRY_WAIT_SECONDS = 2;
private static final EmittingLogger log = new EmittingLogger(KafkaIndexTaskClient.class);
private static final String BASE_PATH = "/druid/worker/v1/chat";
private static final int TASK_MISMATCH_RETRY_DELAY_SECONDS = 5;
private static final TreeMap EMPTY_TREE_MAP = new TreeMap();
private final HttpClient httpClient;
private final ObjectMapper jsonMapper;
private final TaskInfoProvider taskInfoProvider;
private final Duration httpTimeout;
private final RetryPolicyFactory retryPolicyFactory;
private final ListeningExecutorService executorService;
private final long numRetries;
private static final TreeMap<Integer, Map<Integer, Long>> EMPTY_TREE_MAP = new TreeMap<>();
public KafkaIndexTaskClient(
HttpClient httpClient,
@ -106,27 +57,7 @@ public class KafkaIndexTaskClient
long numRetries
)
{
this.httpClient = httpClient;
this.jsonMapper = jsonMapper;
this.taskInfoProvider = taskInfoProvider;
this.httpTimeout = httpTimeout;
this.numRetries = numRetries;
this.retryPolicyFactory = createRetryPolicyFactory();
this.executorService = MoreExecutors.listeningDecorator(
Execs.multiThreaded(
numThreads,
StringUtils.format(
"KafkaIndexTaskClient-%s-%%d",
dataSource
)
)
);
}
public void close()
{
executorService.shutdownNow();
super(httpClient, jsonMapper, taskInfoProvider, httpTimeout, dataSource, numThreads, numRetries);
}
public boolean stop(final String id, final boolean publish)
@ -134,10 +65,10 @@ public class KafkaIndexTaskClient
log.debug("Stop task[%s] publish[%s]", id, publish);
try {
final FullResponseHolder response = submitRequest(
final FullResponseHolder response = submitRequestWithEmptyContent(
id, HttpMethod.POST, "stop", publish ? "publish=true" : null, true
);
return response.getStatus().getCode() / 100 == 2;
return isSuccess(response);
}
catch (NoTaskLocationException e) {
return false;
@ -157,10 +88,11 @@ public class KafkaIndexTaskClient
log.debug("Resume task[%s]", id);
try {
final FullResponseHolder response = submitRequest(id, HttpMethod.POST, "resume", null, true);
return response.getStatus().getCode() / 100 == 2;
final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.POST, "resume", null, true);
return isSuccess(response);
}
catch (NoTaskLocationException e) {
catch (NoTaskLocationException | IOException e) {
log.warn(e, "Exception while stopping task [%s]", id);
return false;
}
}
@ -170,7 +102,7 @@ public class KafkaIndexTaskClient
log.debug("Pause task[%s]", id);
try {
final FullResponseHolder response = submitRequest(
final FullResponseHolder response = submitRequestWithEmptyContent(
id,
HttpMethod.POST,
"pause",
@ -180,18 +112,17 @@ public class KafkaIndexTaskClient
if (response.getStatus().equals(HttpResponseStatus.OK)) {
log.info("Task [%s] paused successfully", id);
return jsonMapper.readValue(response.getContent(), new TypeReference<Map<Integer, Long>>()
return deserialize(response.getContent(), new TypeReference<Map<Integer, Long>>()
{
});
}
final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy();
while (true) {
if (getStatus(id) == KafkaIndexTask.Status.PAUSED) {
return getCurrentOffsets(id, true);
}
final Duration delay = retryPolicy.getAndIncrementRetryDelay();
final Duration delay = newRetryPolicy().getAndIncrementRetryDelay();
if (delay == null) {
log.error("Task [%s] failed to pause, aborting", id);
throw new ISE("Task [%s] failed to pause, aborting", id);
@ -223,8 +154,8 @@ public class KafkaIndexTaskClient
log.debug("GetStatus task[%s]", id);
try {
final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "status", null, true);
return jsonMapper.readValue(response.getContent(), KafkaIndexTask.Status.class);
final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "status", null, true);
return deserialize(response.getContent(), KafkaIndexTask.Status.class);
}
catch (NoTaskLocationException e) {
return KafkaIndexTask.Status.NOT_STARTED;
@ -234,15 +165,16 @@ public class KafkaIndexTaskClient
}
}
@Nullable
public DateTime getStartTime(final String id)
{
log.debug("GetStartTime task[%s]", id);
try {
final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "time/start", null, true);
final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "time/start", null, true);
return response.getContent() == null || response.getContent().isEmpty()
? null
: jsonMapper.readValue(response.getContent(), DateTime.class);
: deserialize(response.getContent(), DateTime.class);
}
catch (NoTaskLocationException e) {
return null;
@ -257,7 +189,7 @@ public class KafkaIndexTaskClient
log.debug("GetMovingAverages task[%s]", id);
try {
final FullResponseHolder response = submitRequest(
final FullResponseHolder response = submitRequestWithEmptyContent(
id,
HttpMethod.GET,
"rowStats",
@ -266,7 +198,7 @@ public class KafkaIndexTaskClient
);
return response.getContent() == null || response.getContent().isEmpty()
? Collections.emptyMap()
: jsonMapper.readValue(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
: deserialize(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT);
}
catch (NoTaskLocationException e) {
return Collections.emptyMap();
@ -276,27 +208,19 @@ public class KafkaIndexTaskClient
}
}
public ListenableFuture<Map<String, Object>> getMovingAveragesAsync(final String id)
{
return executorService.submit(
new Callable<Map<String, Object>>()
{
@Override
public Map<String, Object> call()
{
return getMovingAverages(id);
}
}
);
}
public Map<Integer, Long> getCurrentOffsets(final String id, final boolean retry)
{
log.debug("GetCurrentOffsets task[%s] retry[%s]", id, retry);
try {
final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/current", null, retry);
return jsonMapper.readValue(response.getContent(), new TypeReference<Map<Integer, Long>>()
final FullResponseHolder response = submitRequestWithEmptyContent(
id,
HttpMethod.GET,
"offsets/current",
null,
retry
);
return deserialize(response.getContent(), new TypeReference<Map<Integer, Long>>()
{
});
}
@ -312,10 +236,13 @@ public class KafkaIndexTaskClient
{
log.debug("GetCheckpoints task[%s] retry[%s]", id, retry);
try {
final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "checkpoints", null, retry);
return jsonMapper.readValue(response.getContent(), new TypeReference<TreeMap<Integer, TreeMap<Integer, Long>>>()
{
});
final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "checkpoints", null, retry);
return deserialize(
response.getContent(),
new TypeReference<TreeMap<Integer, Map<Integer, Long>>>()
{
}
);
}
catch (NoTaskLocationException e) {
return EMPTY_TREE_MAP;
@ -330,9 +257,7 @@ public class KafkaIndexTaskClient
final boolean retry
)
{
return executorService.submit(
() -> getCheckpoints(id, retry)
);
return doAsync(() -> getCheckpoints(id, retry));
}
public Map<Integer, Long> getEndOffsets(final String id)
@ -340,8 +265,8 @@ public class KafkaIndexTaskClient
log.debug("GetEndOffsets task[%s]", id);
try {
final FullResponseHolder response = submitRequest(id, HttpMethod.GET, "offsets/end", null, true);
return jsonMapper.readValue(response.getContent(), new TypeReference<Map<Integer, Long>>()
final FullResponseHolder response = submitRequestWithEmptyContent(id, HttpMethod.GET, "offsets/end", null, true);
return deserialize(response.getContent(), new TypeReference<Map<Integer, Long>>()
{
});
}
@ -357,322 +282,72 @@ public class KafkaIndexTaskClient
final String id,
final Map<Integer, Long> endOffsets,
final boolean finalize
)
) throws IOException
{
log.debug("SetEndOffsets task[%s] endOffsets[%s] finalize[%s]", id, endOffsets, finalize);
try {
final FullResponseHolder response = submitRequest(
final FullResponseHolder response = submitJsonRequest(
id,
HttpMethod.POST,
"offsets/end",
StringUtils.format("finish=%s", finalize),
jsonMapper.writeValueAsBytes(endOffsets),
serialize(endOffsets),
true
);
return response.getStatus().getCode() / 100 == 2;
return isSuccess(response);
}
catch (NoTaskLocationException e) {
return false;
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
public ListenableFuture<Boolean> stopAsync(final String id, final boolean publish)
{
return executorService.submit(
new Callable<Boolean>()
{
@Override
public Boolean call()
{
return stop(id, publish);
}
}
);
return doAsync(() -> stop(id, publish));
}
public ListenableFuture<Boolean> resumeAsync(final String id)
{
return executorService.submit(
new Callable<Boolean>()
{
@Override
public Boolean call()
{
return resume(id);
}
}
);
return doAsync(() -> resume(id));
}
public ListenableFuture<Map<Integer, Long>> pauseAsync(final String id)
{
return executorService.submit(
new Callable<Map<Integer, Long>>()
{
@Override
public Map<Integer, Long> call()
{
return pause(id);
}
}
);
return doAsync(() -> pause(id));
}
public ListenableFuture<KafkaIndexTask.Status> getStatusAsync(final String id)
{
return executorService.submit(
new Callable<KafkaIndexTask.Status>()
{
@Override
public KafkaIndexTask.Status call()
{
return getStatus(id);
}
}
);
return doAsync(() -> getStatus(id));
}
public ListenableFuture<DateTime> getStartTimeAsync(final String id)
{
return executorService.submit(
new Callable<DateTime>()
{
@Override
public DateTime call()
{
return getStartTime(id);
}
}
);
return doAsync(() -> getStartTime(id));
}
public ListenableFuture<Map<Integer, Long>> getCurrentOffsetsAsync(final String id, final boolean retry)
{
return executorService.submit(
new Callable<Map<Integer, Long>>()
{
@Override
public Map<Integer, Long> call()
{
return getCurrentOffsets(id, retry);
}
}
);
return doAsync(() -> getCurrentOffsets(id, retry));
}
public ListenableFuture<Map<Integer, Long>> getEndOffsetsAsync(final String id)
{
return executorService.submit(
new Callable<Map<Integer, Long>>()
{
@Override
public Map<Integer, Long> call()
{
return getEndOffsets(id);
}
}
);
return doAsync(() -> getEndOffsets(id));
}
public ListenableFuture<Boolean> setEndOffsetsAsync(
final String id, final Map<Integer, Long> endOffsets, final boolean finalize
final String id,
final Map<Integer, Long> endOffsets,
final boolean finalize
)
{
return executorService.submit(
new Callable<Boolean>()
{
@Override
public Boolean call()
{
return setEndOffsets(id, endOffsets, finalize);
}
}
);
return doAsync(() -> setEndOffsets(id, endOffsets, finalize));
}
@VisibleForTesting
RetryPolicyFactory createRetryPolicyFactory()
public ListenableFuture<Map<String, Object>> getMovingAveragesAsync(final String id)
{
// Retries [numRetries] times before giving up; this should be set long enough to handle any temporary
// unresponsiveness such as network issues, if a task is still in the process of starting up, or if the task is in
// the middle of persisting to disk and doesn't respond immediately.
return new RetryPolicyFactory(
new RetryPolicyConfig()
.setMinWait(Period.seconds(MIN_RETRY_WAIT_SECONDS))
.setMaxWait(Period.seconds(MAX_RETRY_WAIT_SECONDS))
.setMaxRetryCount(numRetries)
);
}
@VisibleForTesting
void checkConnection(String host, int port) throws IOException
{
new Socket(host, port).close();
}
private FullResponseHolder submitRequest(String id, HttpMethod method, String pathSuffix, String query, boolean retry)
{
return submitRequest(id, method, pathSuffix, query, new byte[0], retry);
}
private FullResponseHolder submitRequest(
String id,
HttpMethod method,
String pathSuffix,
String query,
byte[] content,
boolean retry
)
{
final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy();
while (true) {
FullResponseHolder response = null;
Request request = null;
TaskLocation location = TaskLocation.unknown();
String path = StringUtils.format("%s/%s/%s", BASE_PATH, id, pathSuffix);
Optional<TaskStatus> status = taskInfoProvider.getTaskStatus(id);
if (!status.isPresent() || !status.get().isRunnable()) {
throw new TaskNotRunnableException(StringUtils.format(
"Aborting request because task [%s] is not runnable",
id
));
}
String host = location.getHost();
String scheme = "";
int port = -1;
try {
location = taskInfoProvider.getTaskLocation(id);
if (location.equals(TaskLocation.unknown())) {
throw new NoTaskLocationException(StringUtils.format("No TaskLocation available for task [%s]", id));
}
host = location.getHost();
scheme = location.getTlsPort() >= 0 ? "https" : "http";
port = location.getTlsPort() >= 0 ? location.getTlsPort() : location.getPort();
// Netty throws some annoying exceptions if a connection can't be opened, which happens relatively frequently
// for tasks that happen to still be starting up, so test the connection first to keep the logs clean.
checkConnection(host, port);
try {
URI serviceUri = new URI(
scheme,
null,
host,
port,
path,
query,
null
);
request = new Request(method, serviceUri.toURL());
// used to validate that we are talking to the correct worker
request.addHeader(ChatHandlerResource.TASK_ID_HEADER, id);
if (content.length > 0) {
request.setContent(MediaType.APPLICATION_JSON, content);
}
log.debug("HTTP %s: %s", method.getName(), serviceUri.toString());
response = httpClient.go(request, new FullResponseHandler(StandardCharsets.UTF_8), httpTimeout).get();
}
catch (IOException | ChannelException ioce) {
throw ioce;
}
catch (InterruptedException ie) {
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
}
catch (Exception e) {
throw new RuntimeException(e);
}
int responseCode = response.getStatus().getCode();
if (responseCode / 100 == 2) {
return response;
} else if (responseCode == 400) { // don't bother retrying if it's a bad request
throw new IAE("Received 400 Bad Request with body: %s", response.getContent());
} else {
throw new IOE("Received status [%d]", responseCode);
}
}
catch (IOException | ChannelException e) {
// Since workers are free to move tasks around to different ports, there is a chance that a task may have been
// moved but our view of its location has not been updated yet from ZK. To detect this case, we send a header
// identifying our expected recipient in the request; if this doesn't correspond to the worker we messaged, the
// worker will return an HTTP 404 with its ID in the response header. If we get a mismatching task ID, then
// we will wait for a short period then retry the request indefinitely, expecting the task's location to
// eventually be updated.
final Duration delay;
if (response != null && response.getStatus().equals(HttpResponseStatus.NOT_FOUND)) {
String headerId = response.getResponse().headers().get(ChatHandlerResource.TASK_ID_HEADER);
if (headerId != null && !headerId.equals(id)) {
log.warn(
"Expected worker to have taskId [%s] but has taskId [%s], will retry in [%d]s",
id, headerId, TASK_MISMATCH_RETRY_DELAY_SECONDS
);
delay = Duration.standardSeconds(TASK_MISMATCH_RETRY_DELAY_SECONDS);
} else {
delay = retryPolicy.getAndIncrementRetryDelay();
}
} else {
delay = retryPolicy.getAndIncrementRetryDelay();
}
String urlForLog = (request != null
? request.getUrl().toString()
: StringUtils.format(
"%s://%s:%d%s",
scheme,
host,
port,
path
));
if (!retry) {
// if retry=false, we probably aren't too concerned if the operation doesn't succeed (i.e. the request was
// for informational purposes only) so don't log a scary stack trace
log.info("submitRequest failed for [%s], with message [%s]", urlForLog, e.getMessage());
throw new RuntimeException(e);
} else if (delay == null) {
log.warn(e, "Retries exhausted for [%s], last exception:", urlForLog);
throw new RuntimeException(e);
} else {
try {
final long sleepTime = delay.getMillis();
log.debug(
"Bad response HTTP [%s] from [%s]; will try again in [%s] (body/exception: [%s])",
(response != null ? response.getStatus().getCode() : "no response"),
urlForLog,
new Duration(sleepTime).toString(),
(response != null ? response.getContent() : e.getMessage())
);
Thread.sleep(sleepTime);
}
catch (InterruptedException e2) {
Thread.currentThread().interrupt();
e.addSuppressed(e2);
throw new RuntimeException(e);
}
}
}
catch (NoTaskLocationException e) {
log.info("No TaskLocation available for task [%s], this task may not have been assigned to a worker yet or "
+ "may have already completed", id);
throw e;
}
catch (Exception e) {
log.warn(e, "Exception while sending request");
throw e;
}
}
return doAsync(() -> getMovingAverages(id));
}
}

View File

@ -21,13 +21,14 @@ package io.druid.indexing.kafka;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import io.druid.java.util.http.client.HttpClient;
import io.druid.guice.annotations.EscalatedGlobal;
import io.druid.guice.annotations.Json;
import io.druid.indexing.common.TaskInfoProvider;
import io.druid.indexing.common.task.IndexTaskClientFactory;
import io.druid.java.util.http.client.HttpClient;
import org.joda.time.Duration;
public class KafkaIndexTaskClientFactory
public class KafkaIndexTaskClientFactory implements IndexTaskClientFactory<KafkaIndexTaskClient>
{
private HttpClient httpClient;
private ObjectMapper mapper;
@ -42,6 +43,7 @@ public class KafkaIndexTaskClientFactory
this.mapper = mapper;
}
@Override
public KafkaIndexTaskClient build(
TaskInfoProvider taskInfoProvider,
String dataSource,

View File

@ -24,6 +24,9 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.TypeLiteral;
import io.druid.guice.LazySingleton;
import io.druid.indexing.common.task.IndexTaskClientFactory;
import io.druid.indexing.kafka.supervisor.KafkaSupervisorSpec;
import io.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig;
import io.druid.initialization.DruidModule;
@ -50,5 +53,10 @@ public class KafkaIndexTaskModule implements DruidModule
@Override
public void configure(Binder binder)
{
binder.bind(
new TypeLiteral<IndexTaskClientFactory<KafkaIndexTaskClient>>()
{
}
).to(KafkaIndexTaskClientFactory.class).in(LazySingleton.class);
}
}

View File

@ -29,6 +29,7 @@ import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import io.druid.indexer.TaskLocation;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.IndexTaskClient;
import io.druid.indexing.common.TaskInfoProvider;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
@ -56,6 +57,7 @@ import org.junit.rules.ExpectedException;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.IOException;
import java.net.URL;
import java.util.Collections;
import java.util.List;
@ -134,7 +136,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
}
@Test
public void testNoTaskLocation()
public void testNoTaskLocation() throws IOException
{
reset(taskInfoProvider);
expect(taskInfoProvider.getTaskLocation(TEST_ID)).andReturn(TaskLocation.unknown()).anyTimes();
@ -158,7 +160,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
@Test
public void testTaskNotRunnableException()
{
expectedException.expect(KafkaIndexTaskClient.TaskNotRunnableException.class);
expectedException.expect(IndexTaskClient.TaskNotRunnableException.class);
expectedException.expectMessage("Aborting request because task [test-id] is not runnable");
reset(taskInfoProvider);
@ -950,7 +952,7 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
}
@Override
void checkConnection(String host, int port)
protected void checkConnection(String host, int port)
{
}
}

View File

@ -50,6 +50,7 @@ import io.druid.discovery.DruidNodeAnnouncer;
import io.druid.discovery.LookupNodeService;
import io.druid.indexer.TaskState;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.Counters;
import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexing.common.TaskLock;
@ -2050,6 +2051,7 @@ public class KafkaIndexTaskTest
taskLockbox = new TaskLockbox(taskStorage);
final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
taskLockbox,
taskStorage,
metadataStorageCoordinator,
emitter,
new SupervisorManager(null)
@ -2073,7 +2075,8 @@ public class KafkaIndexTaskTest
);
return true;
}
}
},
new Counters()
);
final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
taskStorage,

View File

@ -30,11 +30,15 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.Lists;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.InputSplit;
import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.storage.s3.ServerSideEncryptingAmazonS3;
import io.druid.storage.s3.S3Utils;
@ -44,6 +48,7 @@ import java.io.InputStream;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
@ -52,7 +57,7 @@ import java.util.stream.Collectors;
/**
* Builds firehoses that read from a predefined list of S3 objects and then dry up.
*/
public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory<S3ObjectSummary>
public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory<URI>
{
private static final Logger log = new Logger(StaticS3FirehoseFactory.class);
private static final int MAX_LISTING_LENGTH = 1024;
@ -108,20 +113,12 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
}
@Override
protected Collection<S3ObjectSummary> initObjects() throws IOException
protected Collection<URI> initObjects() throws IOException
{
// Here, the returned s3 objects contain minimal information without data.
// Getting data is deferred until openObjectStream() is called for each object.
if (!uris.isEmpty()) {
return uris.stream()
.map(
uri -> {
final String s3Bucket = uri.getAuthority();
final String key = S3Utils.extractS3Key(uri);
return S3Utils.getSingleObjectSummary(s3Client, s3Bucket, key);
}
)
.collect(Collectors.toList());
return uris;
} else {
final List<S3ObjectSummary> objects = new ArrayList<>();
for (URI uri : prefixes) {
@ -167,18 +164,21 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
}
}
}
return objects;
return objects.stream().map(StaticS3FirehoseFactory::toUri).collect(Collectors.toList());
}
}
@Override
protected InputStream openObjectStream(S3ObjectSummary object) throws IOException
protected InputStream openObjectStream(URI object) throws IOException
{
try {
// Get data of the given object and open an input stream
final S3Object s3Object = s3Client.getObject(object.getBucketName(), object.getKey());
final String bucket = object.getAuthority();
final String key = S3Utils.extractS3Key(object);
final S3Object s3Object = s3Client.getObject(bucket, key);
if (s3Object == null) {
throw new ISE("Failed to get an s3 object for bucket[%s] and key[%s]", object.getBucketName(), object.getKey());
throw new ISE("Failed to get an s3 object for bucket[%s] and key[%s]", bucket, key);
}
return s3Object.getObjectContent();
}
@ -188,17 +188,20 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
}
@Override
protected InputStream openObjectStream(S3ObjectSummary object, long start) throws IOException
protected InputStream openObjectStream(URI object, long start) throws IOException
{
final GetObjectRequest request = new GetObjectRequest(object.getBucketName(), object.getKey());
final String bucket = object.getAuthority();
final String key = S3Utils.extractS3Key(object);
final GetObjectRequest request = new GetObjectRequest(bucket, key);
request.setRange(start);
try {
final S3Object s3Object = s3Client.getObject(request);
if (s3Object == null) {
throw new ISE(
"Failed to get an s3 object for bucket[%s], key[%s], and start[%d]",
object.getBucketName(),
object.getKey(),
bucket,
key,
start
);
}
@ -210,9 +213,9 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
}
@Override
protected InputStream wrapObjectStream(S3ObjectSummary object, InputStream stream) throws IOException
protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException
{
return CompressionUtils.decompress(stream, object.getKey());
return CompressionUtils.decompress(stream, S3Utils.extractS3Key(object));
}
@Override
@ -255,4 +258,40 @@ public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactor
{
return S3Utils.S3RETRY;
}
@Override
public FiniteFirehoseFactory<StringInputRowParser, URI> withSplit(InputSplit<URI> split)
{
return new StaticS3FirehoseFactory(
s3Client,
Collections.singletonList(split.get()),
null,
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
/**
* Create an {@link URI} from the given {@link S3ObjectSummary}. The result URI is composed as below.
*
* <pre>
* {@code s3://{BUCKET_NAME}/{OBJECT_KEY}}
* </pre>
*/
private static URI toUri(S3ObjectSummary object)
{
final String originalAuthority = object.getBucketName();
final String originalPath = object.getKey();
final String authority = originalAuthority.endsWith("/") ?
originalAuthority.substring(0, originalAuthority.length() - 1) :
originalAuthority;
final String path = originalPath.startsWith("/") ?
originalPath.substring(1, originalPath.length()) :
originalPath;
return URI.create(StringUtils.format("s3://%s/%s", authority, path));
}
}

View File

@ -21,6 +21,9 @@ package io.druid.firehose.s3;
import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.AmazonS3Client;
import com.amazonaws.services.s3.model.ListObjectsV2Request;
import com.amazonaws.services.s3.model.ListObjectsV2Result;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.databind.DeserializationContext;
import com.fasterxml.jackson.databind.Module;
@ -33,22 +36,30 @@ import com.google.inject.Binder;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.Provides;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.impl.StringInputRowParser;
import io.druid.initialization.DruidModule;
import io.druid.storage.s3.S3Utils;
import org.easymock.EasyMock;
import io.druid.storage.s3.ServerSideEncryptingAmazonS3;
import io.druid.storage.s3.NoopServerSideEncryption;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.net.URI;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.stream.Collectors;
/**
*/
public class StaticS3FirehoseFactoryTest
{
private static final AmazonS3Client S3_ClIENT = EasyMock.createNiceMock(AmazonS3Client.class);
private static final ServerSideEncryptingAmazonS3 SERVICE = new ServerSideEncryptingAmazonS3(
new AmazonS3Client(),
S3_ClIENT,
new NoopServerSideEncryption()
);
@ -81,6 +92,60 @@ public class StaticS3FirehoseFactoryTest
Assert.assertEquals(factory, outputFact);
}
@Test
public void testWithSplit() throws IOException
{
final List<URI> uris = Arrays.asList(
URI.create("s3://foo/bar/file.gz"),
URI.create("s3://bar/foo/file2.gz")
);
uris.sort(Comparator.comparing(URI::toString));
uris.forEach(StaticS3FirehoseFactoryTest::addExpectedObjject);
EasyMock.replay(S3_ClIENT);
final StaticS3FirehoseFactory factory = new StaticS3FirehoseFactory(
SERVICE,
uris,
null,
2048L,
1024L,
512L,
100L,
5
);
final List<FiniteFirehoseFactory<StringInputRowParser, URI>> subFactories = factory
.getSplits()
.map(factory::withSplit)
.sorted(Comparator.comparing(eachFactory -> {
final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) eachFactory;
return staticS3FirehoseFactory.getUris().toString();
}))
.collect(Collectors.toList());
Assert.assertEquals(uris.size(), subFactories.size());
for (int i = 0; i < uris.size(); i++) {
final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) subFactories.get(i);
final List<URI> subFactoryUris = staticS3FirehoseFactory.getUris();
Assert.assertEquals(1, subFactoryUris.size());
Assert.assertEquals(uris.get(i), subFactoryUris.get(0));
}
}
private static void addExpectedObjject(URI uri)
{
final String s3Bucket = uri.getAuthority();
final String key = S3Utils.extractS3Key(uri);
final S3ObjectSummary objectSummary = new S3ObjectSummary();
objectSummary.setBucketName(s3Bucket);
objectSummary.setKey(key);
final ListObjectsV2Result result = new ListObjectsV2Result();
result.setBucketName(s3Bucket);
result.setKeyCount(1);
result.getObjectSummaries().add(objectSummary);
EasyMock.expect(SERVICE.listObjectsV2(EasyMock.anyObject(ListObjectsV2Request.class))).andReturn(result);
}
private static ObjectMapper createObjectMapper(DruidModule baseModule)
{
final Injector injector = Guice.createInjector(

View File

@ -20,7 +20,6 @@
package io.druid.indexing.appenderator;
import io.druid.data.input.InputRow;
import io.druid.indexing.common.actions.SegmentAllocateAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.realtime.appenderator.SegmentAllocator;
@ -32,14 +31,17 @@ public class ActionBasedSegmentAllocator implements SegmentAllocator
{
private final TaskActionClient taskActionClient;
private final DataSchema dataSchema;
private final SegmentAllocateActionGenerator actionGenerator;
public ActionBasedSegmentAllocator(
TaskActionClient taskActionClient,
DataSchema dataSchema
DataSchema dataSchema,
SegmentAllocateActionGenerator actionGenerator
)
{
this.taskActionClient = taskActionClient;
this.dataSchema = dataSchema;
this.actionGenerator = actionGenerator;
}
@Override
@ -51,15 +53,7 @@ public class ActionBasedSegmentAllocator implements SegmentAllocator
) throws IOException
{
return taskActionClient.submit(
new SegmentAllocateAction(
dataSchema.getDataSource(),
row.getTimestamp(),
dataSchema.getGranularitySpec().getQueryGranularity(),
dataSchema.getGranularitySpec().getSegmentGranularity(),
sequenceName,
previousSegmentId,
skipSegmentLineageCheck
)
actionGenerator.generate(dataSchema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck)
);
}
}

View File

@ -0,0 +1,44 @@
/*
* 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 io.druid.indexing.appenderator;
import io.druid.data.input.InputRow;
import io.druid.indexing.common.actions.TaskAction;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.realtime.appenderator.SegmentIdentifier;
/**
* This class is used in {@link ActionBasedSegmentAllocator} and expected to generate a
* {@link TaskAction<SegmentIdentifier>} which is submitted to overlords to allocate a new segment.
* The {@link #generate} method can return any implementation of {@link TaskAction<SegmentIdentifier>}.
*
* @see io.druid.indexing.common.actions.SegmentAllocateAction
* @see io.druid.indexing.common.actions.SurrogateAction
*/
public interface SegmentAllocateActionGenerator
{
TaskAction<SegmentIdentifier> generate(
DataSchema dataSchema,
InputRow row,
String sequenceName,
String previousSegmentId,
boolean skipSegmentLineageCheck
);
}

View File

@ -0,0 +1,72 @@
/*
* 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 io.druid.indexing.common;
import com.google.common.util.concurrent.AtomicDouble;
import javax.annotation.Nullable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BinaryOperator;
public class Counters
{
private final ConcurrentMap<String, AtomicInteger> intCounters = new ConcurrentHashMap<>();
private final ConcurrentMap<String, AtomicDouble> doubleCounters = new ConcurrentHashMap<>();
private final ConcurrentMap<String, AtomicReference> objectCounters = new ConcurrentHashMap<>();
public int increment(String key, int val)
{
return intCounters.computeIfAbsent(key, k -> new AtomicInteger()).addAndGet(val);
}
public double increment(String key, double val)
{
return doubleCounters.computeIfAbsent(key, k -> new AtomicDouble()).addAndGet(val);
}
public Object increment(String key, Object obj, BinaryOperator mergeFunction)
{
return objectCounters.computeIfAbsent(key, k -> new AtomicReference()).accumulateAndGet(obj, mergeFunction);
}
@Nullable
public Integer getIntCounter(String key)
{
final AtomicInteger atomicInteger = intCounters.get(key);
return atomicInteger == null ? null : atomicInteger.get();
}
@Nullable
public Double getDoubleCounter(String key)
{
final AtomicDouble atomicDouble = doubleCounters.get(key);
return atomicDouble == null ? null : atomicDouble.get();
}
@Nullable
public Object getObjectCounter(String key)
{
final AtomicReference atomicReference = objectCounters.get(key);
return atomicReference == null ? null : atomicReference.get();
}
}

View File

@ -0,0 +1,381 @@
/*
* 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 io.druid.indexing.common;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import io.druid.indexer.TaskLocation;
import io.druid.indexer.TaskStatus;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.IOE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.concurrent.Execs;
import io.druid.java.util.emitter.EmittingLogger;
import io.druid.java.util.http.client.HttpClient;
import io.druid.java.util.http.client.Request;
import io.druid.java.util.http.client.response.FullResponseHandler;
import io.druid.java.util.http.client.response.FullResponseHolder;
import io.druid.segment.realtime.firehose.ChatHandlerResource;
import org.jboss.netty.channel.ChannelException;
import org.jboss.netty.handler.codec.http.HttpMethod;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.joda.time.Duration;
import org.joda.time.Period;
import javax.annotation.Nullable;
import javax.ws.rs.core.MediaType;
import java.io.IOException;
import java.net.Socket;
import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.util.concurrent.Callable;
/**
* Abstract class to communicate with index tasks via HTTP. This class provides interfaces to serialize/deserialize
* data and send an HTTP request.
*/
public abstract class IndexTaskClient implements AutoCloseable
{
public static class NoTaskLocationException extends RuntimeException
{
public NoTaskLocationException(String message)
{
super(message);
}
}
public static class TaskNotRunnableException extends RuntimeException
{
public TaskNotRunnableException(String message)
{
super(message);
}
}
public static final int MAX_RETRY_WAIT_SECONDS = 10;
private static final EmittingLogger log = new EmittingLogger(IndexTaskClient.class);
private static final String BASE_PATH = "/druid/worker/v1/chat";
private static final int MIN_RETRY_WAIT_SECONDS = 2;
private static final int TASK_MISMATCH_RETRY_DELAY_SECONDS = 5;
private final HttpClient httpClient;
private final ObjectMapper objectMapper;
private final TaskInfoProvider taskInfoProvider;
private final Duration httpTimeout;
private final RetryPolicyFactory retryPolicyFactory;
private final ListeningExecutorService executorService;
public IndexTaskClient(
HttpClient httpClient,
ObjectMapper objectMapper,
TaskInfoProvider taskInfoProvider,
Duration httpTimeout,
String callerId,
int numThreads,
long numRetries
)
{
this.httpClient = httpClient;
this.objectMapper = objectMapper;
this.taskInfoProvider = taskInfoProvider;
this.httpTimeout = httpTimeout;
this.retryPolicyFactory = initializeRetryPolicyFactory(numRetries);
this.executorService = MoreExecutors.listeningDecorator(
Execs.multiThreaded(
numThreads,
StringUtils.format(
"IndexTaskClient-%s-%%d",
callerId
)
)
);
}
private static RetryPolicyFactory initializeRetryPolicyFactory(long numRetries)
{
// Retries [numRetries] times before giving up; this should be set long enough to handle any temporary
// unresponsiveness such as network issues, if a task is still in the process of starting up, or if the task is in
// the middle of persisting to disk and doesn't respond immediately.
return new RetryPolicyFactory(
new RetryPolicyConfig()
.setMinWait(Period.seconds(MIN_RETRY_WAIT_SECONDS))
.setMaxWait(Period.seconds(MAX_RETRY_WAIT_SECONDS))
.setMaxRetryCount(numRetries)
);
}
protected HttpClient getHttpClient()
{
return httpClient;
}
protected RetryPolicy newRetryPolicy()
{
return retryPolicyFactory.makeRetryPolicy();
}
protected <T> T deserialize(String content, TypeReference<T> typeReference) throws IOException
{
return objectMapper.readValue(content, typeReference);
}
protected <T> T deserialize(String content, Class<T> typeReference) throws IOException
{
return objectMapper.readValue(content, typeReference);
}
protected byte[] serialize(Object value) throws JsonProcessingException
{
return objectMapper.writeValueAsBytes(value);
}
protected <T> ListenableFuture<T> doAsync(Callable<T> callable)
{
return executorService.submit(callable);
}
protected boolean isSuccess(FullResponseHolder responseHolder)
{
return responseHolder.getStatus().getCode() / 100 == 2;
}
@VisibleForTesting
protected void checkConnection(String host, int port) throws IOException
{
new Socket(host, port).close();
}
protected FullResponseHolder submitRequestWithEmptyContent(
String taskId,
HttpMethod method,
String pathSuffix,
@Nullable String query,
boolean retry
) throws IOException, ChannelException, NoTaskLocationException
{
return submitRequest(taskId, null, method, pathSuffix, query, new byte[0], retry);
}
/**
* To use this method, {@link #objectMapper} should be a jsonMapper.
*/
protected FullResponseHolder submitJsonRequest(
String taskId,
HttpMethod method,
String pathSuffix,
@Nullable String query,
byte[] content,
boolean retry
) throws IOException, ChannelException, NoTaskLocationException
{
return submitRequest(taskId, MediaType.APPLICATION_JSON, method, pathSuffix, query, content, retry);
}
/**
* To use this method, {@link #objectMapper} should be a smileMapper.
*/
protected FullResponseHolder submitSmileRequest(
String taskId,
HttpMethod method,
String pathSuffix,
@Nullable String query,
byte[] content,
boolean retry
) throws IOException, ChannelException, NoTaskLocationException
{
return submitRequest(taskId, SmileMediaTypes.APPLICATION_JACKSON_SMILE, method, pathSuffix, query, content, retry);
}
/**
* Sends an HTTP request to the task of the specified {@code taskId} and returns a response if it succeeded.
*/
private FullResponseHolder submitRequest(
String taskId,
@Nullable String mediaType, // nullable if content is empty
HttpMethod method,
String pathSuffix,
@Nullable String query,
byte[] content,
boolean retry
) throws IOException, ChannelException, NoTaskLocationException
{
final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy();
while (true) {
FullResponseHolder response = null;
Request request = null;
TaskLocation location = TaskLocation.unknown();
String path = StringUtils.format("%s/%s/%s", BASE_PATH, taskId, pathSuffix);
Optional<TaskStatus> status = taskInfoProvider.getTaskStatus(taskId);
if (!status.isPresent() || !status.get().isRunnable()) {
throw new TaskNotRunnableException(StringUtils.format(
"Aborting request because task [%s] is not runnable",
taskId
));
}
String host = location.getHost();
String scheme = "";
int port = -1;
try {
location = taskInfoProvider.getTaskLocation(taskId);
if (location.equals(TaskLocation.unknown())) {
throw new NoTaskLocationException(StringUtils.format("No TaskLocation available for task [%s]", taskId));
}
host = location.getHost();
scheme = location.getTlsPort() >= 0 ? "https" : "http";
port = location.getTlsPort() >= 0 ? location.getTlsPort() : location.getPort();
// Netty throws some annoying exceptions if a connection can't be opened, which happens relatively frequently
// for tasks that happen to still be starting up, so test the connection first to keep the logs clean.
checkConnection(host, port);
try {
URI serviceUri = new URI(
scheme,
null,
host,
port,
path,
query,
null
);
request = new Request(method, serviceUri.toURL());
// used to validate that we are talking to the correct worker
request.addHeader(ChatHandlerResource.TASK_ID_HEADER, taskId);
if (content.length > 0) {
request.setContent(Preconditions.checkNotNull(mediaType, "mediaType"), content);
}
log.debug("HTTP %s: %s", method.getName(), serviceUri.toString());
response = httpClient.go(request, new FullResponseHandler(StandardCharsets.UTF_8), httpTimeout).get();
}
catch (IOException | ChannelException ioce) {
throw ioce;
}
catch (InterruptedException ie) {
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
}
catch (Exception e) {
throw new RuntimeException(e);
}
int responseCode = response.getStatus().getCode();
if (responseCode / 100 == 2) {
return response;
} else if (responseCode == 400) { // don't bother retrying if it's a bad request
throw new IAE("Received 400 Bad Request with body: %s", response.getContent());
} else {
throw new IOE("Received status [%d]", responseCode);
}
}
catch (IOException | ChannelException e) {
// Since workers are free to move tasks around to different ports, there is a chance that a task may have been
// moved but our view of its location has not been updated yet from ZK. To detect this case, we send a header
// identifying our expected recipient in the request; if this doesn't correspond to the worker we messaged, the
// worker will return an HTTP 404 with its ID in the response header. If we get a mismatching task ID, then
// we will wait for a short period then retry the request indefinitely, expecting the task's location to
// eventually be updated.
final Duration delay;
if (response != null && response.getStatus().equals(HttpResponseStatus.NOT_FOUND)) {
String headerId = response.getResponse().headers().get(ChatHandlerResource.TASK_ID_HEADER);
if (headerId != null && !headerId.equals(taskId)) {
log.warn(
"Expected worker to have taskId [%s] but has taskId [%s], will retry in [%d]s",
taskId, headerId, TASK_MISMATCH_RETRY_DELAY_SECONDS
);
delay = Duration.standardSeconds(TASK_MISMATCH_RETRY_DELAY_SECONDS);
} else {
delay = retryPolicy.getAndIncrementRetryDelay();
}
} else {
delay = retryPolicy.getAndIncrementRetryDelay();
}
String urlForLog = (request != null
? request.getUrl().toString()
: StringUtils.nonStrictFormat(
"%s://%s:%d%s",
scheme,
host,
port,
path
));
if (!retry) {
// if retry=false, we probably aren't too concerned if the operation doesn't succeed (i.e. the request was
// for informational purposes only) so don't log a scary stack trace
log.info("submitRequest failed for [%s], with message [%s]", urlForLog, e.getMessage());
throw e;
} else if (delay == null) {
log.warn(e, "Retries exhausted for [%s], last exception:", urlForLog);
throw e;
} else {
try {
final long sleepTime = delay.getMillis();
log.debug(
"Bad response HTTP [%s] from [%s]; will try again in [%s] (body/exception: [%s])",
(response != null ? response.getStatus().getCode() : "no response"),
urlForLog,
new Duration(sleepTime).toString(),
(response != null ? response.getContent() : e.getMessage())
);
Thread.sleep(sleepTime);
}
catch (InterruptedException e2) {
Thread.currentThread().interrupt();
e.addSuppressed(e2);
throw new RuntimeException(e);
}
}
}
catch (NoTaskLocationException e) {
log.info("No TaskLocation available for task [%s], this task may not have been assigned to a worker yet or "
+ "may have already completed", taskId);
throw e;
}
catch (Exception e) {
log.warn(e, "Exception while sending request");
throw e;
}
}
}
@Override
public void close()
{
executorService.shutdownNow();
}
}

View File

@ -0,0 +1,90 @@
/*
* 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 io.druid.indexing.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Optional;
import io.druid.indexing.common.task.Task;
import io.druid.java.util.common.ISE;
/**
* Perform the given action using {@link #surrogateId} on behalf of the caller task.
*/
public class SurrogateAction<ReturnType, ActionType extends TaskAction<ReturnType>> implements TaskAction<ReturnType>
{
private final String surrogateId;
private final ActionType taskAction;
@JsonCreator
public SurrogateAction(
@JsonProperty("surrogateId") String surrogateId,
@JsonProperty("taskAction") ActionType taskAction
)
{
this.surrogateId = surrogateId;
this.taskAction = taskAction;
}
@JsonProperty
public String getSurrogateId()
{
return surrogateId;
}
@JsonProperty
public ActionType getTaskAction()
{
return taskAction;
}
@Override
public TypeReference<ReturnType> getReturnTypeReference()
{
return taskAction.getReturnTypeReference();
}
@Override
public ReturnType perform(Task task, TaskActionToolbox toolbox)
{
final Optional<Task> maybeSurrogateTask = toolbox.getTaskStorage().getTask(surrogateId);
if (maybeSurrogateTask.isPresent()) {
return taskAction.perform(maybeSurrogateTask.get(), toolbox);
} else {
throw new ISE("Can't find surrogate task[%s]", surrogateId);
}
}
@Override
public boolean isAudited()
{
return taskAction.isAudited();
}
@Override
public String toString()
{
return "SurrogateAction{" +
"surrogateId='" + surrogateId + '\'' +
", taskAction=" + taskAction +
'}';
}
}

View File

@ -38,11 +38,15 @@ import io.druid.indexing.common.task.Task;
@JsonSubTypes.Type(name = "segmentMetadataUpdate", value = SegmentMetadataUpdateAction.class),
@JsonSubTypes.Type(name = "segmentAllocate", value = SegmentAllocateAction.class),
@JsonSubTypes.Type(name = "resetDataSourceMetadata", value = ResetDataSourceMetadataAction.class),
@JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class)
@JsonSubTypes.Type(name = "checkPointDataSourceMetadata", value = CheckPointDataSourceMetadataAction.class),
@JsonSubTypes.Type(name = "surrogateAction", value = SurrogateAction.class)
})
public interface TaskAction<RetType>
{
TypeReference<RetType> getReturnTypeReference(); // T_T
RetType perform(Task task, TaskActionToolbox toolbox);
boolean isAudited();
@Override
String toString();
}

View File

@ -42,7 +42,12 @@ public class TaskActionPreconditions
)
{
if (!isLockCoversSegments(task, taskLockbox, segments)) {
throw new ISE("Segments not covered by locks for task: %s", task.getId());
throw new ISE(
"Segments[%s] are not covered by locks[%s] for task[%s]",
segments,
taskLockbox.findLocksForTask(task),
task.getId()
);
}
}

View File

@ -20,30 +20,38 @@
package io.druid.indexing.common.actions;
import com.google.inject.Inject;
import io.druid.java.util.emitter.service.ServiceEmitter;
import io.druid.indexing.common.Counters;
import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import io.druid.indexing.overlord.TaskLockbox;
import io.druid.indexing.overlord.TaskStorage;
import io.druid.indexing.overlord.supervisor.SupervisorManager;
import io.druid.java.util.emitter.service.ServiceEmitter;
public class TaskActionToolbox
{
private final TaskLockbox taskLockbox;
private final TaskStorage taskStorage;
private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
private final ServiceEmitter emitter;
private final SupervisorManager supervisorManager;
private final Counters counters;
@Inject
public TaskActionToolbox(
TaskLockbox taskLockbox,
TaskStorage taskStorage,
IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
ServiceEmitter emitter,
SupervisorManager supervisorManager
SupervisorManager supervisorManager,
Counters counters
)
{
this.taskLockbox = taskLockbox;
this.taskStorage = taskStorage;
this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
this.emitter = emitter;
this.supervisorManager = supervisorManager;
this.counters = counters;
}
public TaskLockbox getTaskLockbox()
@ -51,6 +59,11 @@ public class TaskActionToolbox
return taskLockbox;
}
public TaskStorage getTaskStorage()
{
return taskStorage;
}
public IndexerMetadataStorageCoordinator getIndexerMetadataStorageCoordinator()
{
return indexerMetadataStorageCoordinator;
@ -65,4 +78,9 @@ public class TaskActionToolbox
{
return supervisorManager;
}
public Counters getCounters()
{
return counters;
}
}

View File

@ -78,7 +78,7 @@ public abstract class AbstractTask implements Task
this.context = context == null ? new HashMap<>() : context;
}
static String getOrMakeId(String id, final String typeName, String dataSource)
public static String getOrMakeId(String id, final String typeName, String dataSource)
{
return getOrMakeId(id, typeName, dataSource, null);
}
@ -227,7 +227,7 @@ public abstract class AbstractTask implements Task
return Objects.hashCode(id, groupId, dataSource, context);
}
static List<TaskLock> getTaskLocks(TaskActionClient client) throws IOException
public static List<TaskLock> getTaskLocks(TaskActionClient client) throws IOException
{
return client.submit(new LockListAction());
}

View File

@ -47,6 +47,7 @@ import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
import io.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
import io.druid.indexing.common.TaskReport;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.SegmentAllocateAction;
import io.druid.indexing.common.actions.SegmentTransactionalInsertAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.index.RealtimeAppenderatorIngestionSpec;
@ -718,7 +719,19 @@ public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements
{
return new StreamAppenderatorDriver(
appenderator,
new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema),
new ActionBasedSegmentAllocator(
toolbox.getTaskActionClient(),
dataSchema,
(schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction(
schema.getDataSource(),
row.getTimestamp(),
schema.getGranularitySpec().getQueryGranularity(),
schema.getGranularitySpec().getSegmentGranularity(),
sequenceName,
previousSegmentId,
skipSegmentLineageCheck
)
),
toolbox.getSegmentHandoffNotifierFactory(),
new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
toolbox.getDataSegmentKiller(),

View File

@ -0,0 +1,55 @@
/*
* 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 io.druid.indexing.common.task;
import com.google.common.base.Optional;
import com.google.inject.Inject;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.client.indexing.TaskStatusResponse;
import io.druid.indexer.TaskLocation;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.TaskInfoProvider;
public class ClientBasedTaskInfoProvider implements TaskInfoProvider
{
private final IndexingServiceClient client;
@Inject
public ClientBasedTaskInfoProvider(IndexingServiceClient client)
{
this.client = client;
}
@Override
public TaskLocation getTaskLocation(String id)
{
final TaskStatusResponse response = client.getTaskStatus(id);
return response == null ? TaskLocation.unknown() : response.getStatus().getLocation();
}
@Override
public Optional<TaskStatus> getTaskStatus(String id)
{
final TaskStatusResponse response = client.getTaskStatus(id);
return response == null ?
Optional.absent() :
Optional.of(TaskStatus.fromCode(id, response.getStatus().getState()));
}
}

View File

@ -50,6 +50,7 @@ import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskRealtimeMetricsMonitorBuilder;
import io.druid.indexing.common.TaskReport;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.SegmentAllocateAction;
import io.druid.indexing.common.actions.SegmentTransactionalInsertAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.stats.RowIngestionMeters;
@ -204,7 +205,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
);
}
IndexTask(
public IndexTask(
String id,
String groupId,
TaskResource resource,
@ -800,8 +801,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
}
determinePartitionsMeters.incrementUnparseable();
if (determinePartitionsMeters.getUnparseable() > ingestionSchema.getTuningConfig()
.getMaxParseExceptions()) {
if (determinePartitionsMeters.getUnparseable() > ingestionSchema.getTuningConfig().getMaxParseExceptions()) {
throw new RuntimeException("Max parse exceptions exceeded, terminating task...");
}
}
@ -854,7 +854,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
final TaskToolbox toolbox,
final DataSchema dataSchema,
final ShardSpecs shardSpecs,
Map<Interval, String> versions,
final Map<Interval, String> versions,
final FirehoseFactory firehoseFactory,
final File firehoseTempDir
) throws IOException, InterruptedException
@ -908,7 +908,19 @@ public class IndexTask extends AbstractTask implements ChatHandler
segmentAllocator = (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> lookup.get(sequenceName);
} else if (ioConfig.isAppendToExisting()) {
// Append mode: Allocate segments as needed using Overlord APIs.
segmentAllocator = new ActionBasedSegmentAllocator(toolbox.getTaskActionClient(), dataSchema);
segmentAllocator = new ActionBasedSegmentAllocator(
toolbox.getTaskActionClient(),
dataSchema,
(schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SegmentAllocateAction(
schema.getDataSource(),
row.getTimestamp(),
schema.getGranularitySpec().getQueryGranularity(),
schema.getGranularitySpec().getSegmentGranularity(),
sequenceName,
previousSegmentId,
skipSegmentLineageCheck
)
);
} else {
// Overwrite mode, non-guaranteed rollup: We can make up our own segment ids but we don't know them in advance.
final Map<Interval, AtomicInteger> counters = new HashMap<>();

View File

@ -0,0 +1,29 @@
/*
* 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 io.druid.indexing.common.task;
import io.druid.indexing.common.IndexTaskClient;
import io.druid.indexing.common.TaskInfoProvider;
import org.joda.time.Duration;
public interface IndexTaskClientFactory<T extends IndexTaskClient>
{
T build(TaskInfoProvider taskInfoProvider, String callerId, int numThreads, Duration httpTimeout, long numRetries);
}

View File

@ -24,6 +24,8 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.task.batch.parallel.ParallelIndexSubTask;
import io.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
@ -50,6 +52,8 @@ import java.util.Map;
@JsonSubTypes.Type(name = "archive", value = ArchiveTask.class),
@JsonSubTypes.Type(name = "restore", value = RestoreTask.class),
@JsonSubTypes.Type(name = "index", value = IndexTask.class),
@JsonSubTypes.Type(name = ParallelIndexSupervisorTask.TYPE, value = ParallelIndexSupervisorTask.class),
@JsonSubTypes.Type(name = ParallelIndexSubTask.TYPE, value = ParallelIndexSubTask.class),
@JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class),
@JsonSubTypes.Type(name = "hadoop_convert_segment", value = HadoopConverterTask.class),
@JsonSubTypes.Type(name = "hadoop_convert_segment_sub", value = HadoopConverterTask.ConverterSubTask.class),

View File

@ -0,0 +1,42 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import io.druid.data.input.FirehoseFactory;
import io.druid.indexing.common.task.IndexTask.IndexIOConfig;
import javax.annotation.Nullable;
/**
* Same with {@link IndexIOConfig} except its JSON type name.
*/
@JsonTypeName("index_parallel")
public class ParallelIndexIOConfig extends IndexIOConfig
{
public ParallelIndexIOConfig(
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
@JsonProperty("appendToExisting") @Nullable Boolean appendToExisting
)
{
super(firehoseFactory, appendToExisting);
}
}

View File

@ -0,0 +1,67 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.IngestionSpec;
public class ParallelIndexIngestionSpec extends IngestionSpec<ParallelIndexIOConfig, ParallelIndexTuningConfig>
{
private final DataSchema dataSchema;
private final ParallelIndexIOConfig ioConfig;
private final ParallelIndexTuningConfig tuningConfig;
@JsonCreator
public ParallelIndexIngestionSpec(
@JsonProperty("dataSchema") DataSchema dataSchema,
@JsonProperty("ioConfig") ParallelIndexIOConfig ioConfig,
@JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig
)
{
super(dataSchema, ioConfig, tuningConfig);
this.dataSchema = dataSchema;
this.ioConfig = ioConfig;
this.tuningConfig = tuningConfig == null ? ParallelIndexTuningConfig.defaultConfig() : tuningConfig;
}
@Override
@JsonProperty("dataSchema")
public DataSchema getDataSchema()
{
return dataSchema;
}
@Override
@JsonProperty("ioConfig")
public ParallelIndexIOConfig getIOConfig()
{
return ioConfig;
}
@Override
@JsonProperty("tuningConfig")
public ParallelIndexTuningConfig getTuningConfig()
{
return tuningConfig;
}
}

View File

@ -0,0 +1,437 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.data.input.Firehose;
import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.InputRow;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.appenderator.ActionBasedSegmentAllocator;
import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
import io.druid.indexing.common.TaskLockType;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.LockTryAcquireAction;
import io.druid.indexing.common.actions.SegmentAllocateAction;
import io.druid.indexing.common.actions.SurrogateAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.task.AbstractTask;
import io.druid.indexing.common.task.ClientBasedTaskInfoProvider;
import io.druid.indexing.common.task.IndexTask;
import io.druid.indexing.common.task.IndexTaskClientFactory;
import io.druid.indexing.common.task.TaskResource;
import io.druid.indexing.common.task.Tasks;
import io.druid.indexing.firehose.IngestSegmentFirehoseFactory;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.logger.Logger;
import io.druid.java.util.common.parsers.ParseException;
import io.druid.query.DruidMetrics;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeIOConfig;
import io.druid.segment.indexing.granularity.GranularitySpec;
import io.druid.segment.realtime.FireDepartment;
import io.druid.segment.realtime.FireDepartmentMetrics;
import io.druid.segment.realtime.RealtimeMetricsMonitor;
import io.druid.segment.realtime.appenderator.Appenderator;
import io.druid.segment.realtime.appenderator.AppenderatorDriverAddResult;
import io.druid.segment.realtime.appenderator.Appenderators;
import io.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
import io.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
import io.druid.segment.realtime.appenderator.SegmentAllocator;
import io.druid.segment.realtime.appenderator.SegmentsAndMetadata;
import io.druid.timeline.DataSegment;
import org.apache.commons.io.FileUtils;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.SortedSet;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeoutException;
/**
* A worker task of {@link ParallelIndexSupervisorTask}. Similar to {@link IndexTask}, but this task
* generates and pushes segments, and reports them to the {@link ParallelIndexSupervisorTask} instead of
* publishing on its own.
*/
public class ParallelIndexSubTask extends AbstractTask
{
public static final String TYPE = "index_sub";
private static final Logger log = new Logger(ParallelIndexSubTask.class);
private final int numAttempts;
private final ParallelIndexIngestionSpec ingestionSchema;
private final String supervisorTaskId;
private final IndexingServiceClient indexingServiceClient;
private final IndexTaskClientFactory<ParallelIndexTaskClient> taskClientFactory;
@JsonCreator
public ParallelIndexSubTask(
// id shouldn't be null except when this task is created by ParallelIndexSupervisorTask
@JsonProperty("id") @Nullable final String id,
@JsonProperty("groupId") final String groupId,
@JsonProperty("resource") final TaskResource taskResource,
@JsonProperty("supervisorTaskId") final String supervisorTaskId,
@JsonProperty("numAttempts") final int numAttempts, // zero-based counting
@JsonProperty("spec") final ParallelIndexIngestionSpec ingestionSchema,
@JsonProperty("context") final Map<String, Object> context,
@JacksonInject IndexingServiceClient indexingServiceClient,
@JacksonInject IndexTaskClientFactory<ParallelIndexTaskClient> taskClientFactory
)
{
super(
getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()),
groupId,
taskResource,
ingestionSchema.getDataSchema().getDataSource(),
context
);
if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) {
throw new UnsupportedOperationException("Guaranteed rollup is not supported");
}
this.numAttempts = numAttempts;
this.ingestionSchema = ingestionSchema;
this.supervisorTaskId = supervisorTaskId;
this.indexingServiceClient = indexingServiceClient;
this.taskClientFactory = taskClientFactory;
}
@Override
public int getPriority()
{
return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY);
}
@Override
public String getType()
{
return TYPE;
}
@Override
public boolean isReady(TaskActionClient taskActionClient)
{
final Optional<SortedSet<Interval>> intervals = ingestionSchema.getDataSchema()
.getGranularitySpec()
.bucketIntervals();
return !intervals.isPresent() || checkLockAcquired(taskActionClient, intervals.get());
}
private boolean checkLockAcquired(TaskActionClient actionClient, SortedSet<Interval> intervals)
{
try {
tryAcquireExclusiveSurrogateLocks(actionClient, intervals);
return true;
}
catch (Exception e) {
log.error(e, "Failed to acquire locks for intervals[%s]", intervals);
return false;
}
}
@JsonProperty
public int getNumAttempts()
{
return numAttempts;
}
@JsonProperty("spec")
public ParallelIndexIngestionSpec getIngestionSchema()
{
return ingestionSchema;
}
@JsonProperty
public String getSupervisorTaskId()
{
return supervisorTaskId;
}
@Override
public TaskStatus run(final TaskToolbox toolbox) throws Exception
{
final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory();
if (firehoseFactory instanceof IngestSegmentFirehoseFactory) {
// pass toolbox to Firehose
((IngestSegmentFirehoseFactory) firehoseFactory).setTaskToolbox(toolbox);
}
final File firehoseTempDir = toolbox.getFirehoseTemporaryDir();
// Firehose temporary directory is automatically removed when this IndexTask completes.
FileUtils.forceMkdir(firehoseTempDir);
final ParallelIndexTaskClient taskClient = taskClientFactory.build(
new ClientBasedTaskInfoProvider(indexingServiceClient),
getId(),
1, // always use a single http thread
ingestionSchema.getTuningConfig().getChatHandlerTimeout(),
ingestionSchema.getTuningConfig().getChatHandlerNumRetries()
);
final List<DataSegment> pushedSegments = generateAndPushSegments(
toolbox,
taskClient,
firehoseFactory,
firehoseTempDir
);
taskClient.report(supervisorTaskId, pushedSegments);
return TaskStatus.success(getId());
}
private void tryAcquireExclusiveSurrogateLocks(
TaskActionClient client,
SortedSet<Interval> intervals
)
throws IOException
{
for (Interval interval : Tasks.computeCompactIntervals(intervals)) {
Preconditions.checkNotNull(
client.submit(
new SurrogateAction<>(supervisorTaskId, new LockTryAcquireAction(TaskLockType.EXCLUSIVE, interval))
),
"Cannot acquire a lock for interval[%s]", interval
);
}
}
private SegmentAllocator createSegmentAllocator(
TaskToolbox toolbox,
ParallelIndexTaskClient taskClient,
ParallelIndexIngestionSpec ingestionSchema
)
{
final DataSchema dataSchema = ingestionSchema.getDataSchema();
final boolean explicitIntervals = dataSchema.getGranularitySpec().bucketIntervals().isPresent();
final ParallelIndexIOConfig ioConfig = ingestionSchema.getIOConfig();
if (ioConfig.isAppendToExisting() || !explicitIntervals) {
return new ActionBasedSegmentAllocator(
toolbox.getTaskActionClient(),
dataSchema,
(schema, row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> new SurrogateAction<>(
supervisorTaskId,
new SegmentAllocateAction(
schema.getDataSource(),
row.getTimestamp(),
schema.getGranularitySpec().getQueryGranularity(),
schema.getGranularitySpec().getSegmentGranularity(),
sequenceName,
previousSegmentId,
skipSegmentLineageCheck
)
)
);
} else {
return (row, sequenceName, previousSegmentId, skipSegmentLineageCheck) -> taskClient.allocateSegment(
supervisorTaskId,
row.getTimestamp()
);
}
}
/**
* This method reads input data row by row and adds the read row to a proper segment using {@link BaseAppenderatorDriver}.
* If there is no segment for the row, a new one is created. Segments can be published in the middle of reading inputs
* if one of below conditions are satisfied.
*
* <ul>
* <li>
* If the number of rows in a segment exceeds {@link ParallelIndexTuningConfig#targetPartitionSize}
* </li>
* <li>
* If the number of rows added to {@link BaseAppenderatorDriver} so far exceeds {@link ParallelIndexTuningConfig#maxTotalRows}
* </li>
* </ul>
*
* At the end of this method, all the remaining segments are published.
*
* @return true if generated segments are successfully published, otherwise false
*/
private List<DataSegment> generateAndPushSegments(
final TaskToolbox toolbox,
final ParallelIndexTaskClient taskClient,
final FirehoseFactory firehoseFactory,
final File firehoseTempDir
) throws IOException, InterruptedException
{
final DataSchema dataSchema = ingestionSchema.getDataSchema();
final GranularitySpec granularitySpec = dataSchema.getGranularitySpec();
final FireDepartment fireDepartmentForMetrics = new FireDepartment(
dataSchema, new RealtimeIOConfig(null, null, null), null
);
final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics();
if (toolbox.getMonitorScheduler() != null) {
toolbox.getMonitorScheduler().addMonitor(
new RealtimeMetricsMonitor(
Collections.singletonList(fireDepartmentForMetrics),
Collections.singletonMap(DruidMetrics.TASK_ID, new String[]{getId()})
)
);
}
final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig();
final long pushTimeout = tuningConfig.getPushTimeout();
final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent();
final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient, ingestionSchema);
try (
final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig);
final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator);
final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir)
) {
driver.startJob();
final List<DataSegment> pushedSegments = new ArrayList<>();
while (firehose.hasMore()) {
try {
final InputRow inputRow = firehose.nextRow();
if (inputRow == null) {
fireDepartmentMetrics.incrementThrownAway();
continue;
}
if (!Intervals.ETERNITY.contains(inputRow.getTimestamp())) {
final String errorMsg = StringUtils.format(
"Encountered row with timestamp that cannot be represented as a long: [%s]",
inputRow
);
throw new ParseException(errorMsg);
}
if (explicitIntervals) {
final Optional<Interval> optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp());
if (!optInterval.isPresent()) {
fireDepartmentMetrics.incrementThrownAway();
continue;
}
}
// Segments are created as needed, using a single sequence name. They may be allocated from the overlord
// (in append mode) or may be created on our own authority (in overwrite mode).
final String sequenceName = getId();
final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName);
if (addResult.isOk()) {
if (exceedMaxRowsInSegment(addResult.getNumRowsInSegment(), tuningConfig) ||
exceedMaxRowsInAppenderator(addResult.getTotalNumRowsInAppenderator(), tuningConfig)) {
// There can be some segments waiting for being published even though any rows won't be added to them.
// If those segments are not published here, the available space in appenderator will be kept to be small
// which makes the size of segments smaller.
final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout);
pushedSegments.addAll(pushed.getSegments());
log.info("Pushed segments[%s]", pushed.getSegments());
}
} else {
throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp());
}
fireDepartmentMetrics.incrementProcessed();
}
catch (ParseException e) {
if (tuningConfig.isReportParseExceptions()) {
throw e;
} else {
fireDepartmentMetrics.incrementUnparseable();
}
}
}
final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout);
pushedSegments.addAll(pushed.getSegments());
log.info("Pushed segments[%s]", pushed.getSegments());
return pushedSegments;
}
catch (TimeoutException | ExecutionException e) {
throw new RuntimeException(e);
}
}
private static boolean exceedMaxRowsInSegment(
int numRowsInSegment,
ParallelIndexTuningConfig indexTuningConfig
)
{
// maxRowsInSegment should be null if numShards is set in indexTuningConfig
final Integer maxRowsInSegment = indexTuningConfig.getTargetPartitionSize();
return maxRowsInSegment != null && maxRowsInSegment <= numRowsInSegment;
}
private static boolean exceedMaxRowsInAppenderator(
long numRowsInAppenderator,
ParallelIndexTuningConfig indexTuningConfig
)
{
// maxRowsInAppenderator should be null if numShards is set in indexTuningConfig
final Long maxRowsInAppenderator = indexTuningConfig.getMaxTotalRows();
return maxRowsInAppenderator != null && maxRowsInAppenderator <= numRowsInAppenderator;
}
private static Appenderator newAppenderator(
FireDepartmentMetrics metrics,
TaskToolbox toolbox,
DataSchema dataSchema,
ParallelIndexTuningConfig tuningConfig
)
{
return Appenderators.createOffline(
dataSchema,
tuningConfig.withBasePersistDirectory(toolbox.getPersistDir()),
metrics,
toolbox.getSegmentPusher(),
toolbox.getObjectMapper(),
toolbox.getIndexIO(),
toolbox.getIndexMergerV9()
);
}
private static BatchAppenderatorDriver newDriver(
final Appenderator appenderator,
final TaskToolbox toolbox,
final SegmentAllocator segmentAllocator
)
{
return new BatchAppenderatorDriver(
appenderator,
segmentAllocator,
new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
toolbox.getDataSegmentKiller()
);
}
}

View File

@ -0,0 +1,67 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.data.input.InputSplit;
import java.util.Map;
class ParallelIndexSubTaskSpec extends SubTaskSpec<ParallelIndexSubTask>
{
private final ParallelIndexIngestionSpec ingestionSpec;
@JsonCreator
ParallelIndexSubTaskSpec(
String id,
String groupId,
String supervisorTaskId,
ParallelIndexIngestionSpec indexIngestionSpec,
Map<String, Object> context,
InputSplit inputSplit
)
{
super(id, groupId, supervisorTaskId, context, inputSplit);
this.ingestionSpec = indexIngestionSpec;
}
@JsonProperty
public ParallelIndexIngestionSpec getIngestionSpec()
{
return ingestionSpec;
}
@Override
public ParallelIndexSubTask newSubTask(int numAttempts)
{
return new ParallelIndexSubTask(
null,
getGroupId(),
null,
getSupervisorTaskId(),
numAttempts,
getIngestionSpec(),
getContext(),
null,
null
);
}
}

View File

@ -0,0 +1,558 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.FirehoseFactory;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.Counters;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.LockListAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.stats.RowIngestionMetersFactory;
import io.druid.indexing.common.task.AbstractTask;
import io.druid.indexing.common.task.IndexTask;
import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec;
import io.druid.indexing.common.task.IndexTask.IndexTuningConfig;
import io.druid.indexing.common.task.IndexTaskUtils;
import io.druid.indexing.common.task.TaskResource;
import io.druid.indexing.common.task.Tasks;
import io.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.indexing.granularity.GranularitySpec;
import io.druid.segment.realtime.appenderator.SegmentIdentifier;
import io.druid.segment.realtime.firehose.ChatHandler;
import io.druid.segment.realtime.firehose.ChatHandlerProvider;
import io.druid.segment.realtime.firehose.ChatHandlers;
import io.druid.server.security.Action;
import io.druid.server.security.AuthorizerMapper;
import io.druid.timeline.partition.NumberedShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.Consumes;
import javax.ws.rs.GET;
import javax.ws.rs.POST;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import javax.ws.rs.core.Response.Status;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.SortedSet;
import java.util.stream.Collectors;
/**
* ParallelIndexSupervisorTask is capable of running multiple subTasks for parallel indexing. This is
* applicable if the input {@link FiniteFirehoseFactory} is splittable. While this task is running, it can submit
* multiple child tasks to overlords. This task succeeds only when all its child tasks succeed; otherwise it fails.
*
* @see ParallelIndexTaskRunner
*/
public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHandler
{
public static final String TYPE = "index_parallel";
private static final Logger log = new Logger(ParallelIndexSupervisorTask.class);
private final ParallelIndexIngestionSpec ingestionSchema;
private final FiniteFirehoseFactory<?, ?> baseFirehoseFactory;
private final IndexingServiceClient indexingServiceClient;
private final ChatHandlerProvider chatHandlerProvider;
private final AuthorizerMapper authorizerMapper;
private final RowIngestionMetersFactory rowIngestionMetersFactory;
private final Counters counters = new Counters();
private volatile ParallelIndexTaskRunner runner;
// toolbox is initlized when run() is called, and can be used for processing HTTP endpoint requests.
private volatile TaskToolbox toolbox;
@JsonCreator
public ParallelIndexSupervisorTask(
@JsonProperty("id") String id,
@JsonProperty("resource") TaskResource taskResource,
@JsonProperty("spec") ParallelIndexIngestionSpec ingestionSchema,
@JsonProperty("context") Map<String, Object> context,
@JacksonInject @Nullable IndexingServiceClient indexingServiceClient, // null in overlords
@JacksonInject @Nullable ChatHandlerProvider chatHandlerProvider, // null in overlords
@JacksonInject AuthorizerMapper authorizerMapper,
@JacksonInject RowIngestionMetersFactory rowIngestionMetersFactory
)
{
super(
getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()),
null,
taskResource,
ingestionSchema.getDataSchema().getDataSource(),
context
);
this.ingestionSchema = ingestionSchema;
final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory();
if (!(firehoseFactory instanceof FiniteFirehoseFactory)) {
throw new IAE("[%s] should implement FiniteFirehoseFactory", firehoseFactory.getClass().getSimpleName());
}
this.baseFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory;
this.indexingServiceClient = indexingServiceClient;
this.chatHandlerProvider = chatHandlerProvider;
this.authorizerMapper = authorizerMapper;
this.rowIngestionMetersFactory = rowIngestionMetersFactory;
if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() > 0) {
log.warn("maxSavedParseExceptions is not supported yet");
}
if (ingestionSchema.getTuningConfig().getMaxParseExceptions() > 0) {
log.warn("maxParseExceptions is not supported yet");
}
if (ingestionSchema.getTuningConfig().isLogParseExceptions()) {
log.warn("logParseExceptions is not supported yet");
}
}
@Override
public int getPriority()
{
return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY);
}
@Override
public String getType()
{
return TYPE;
}
@JsonProperty("spec")
public ParallelIndexIngestionSpec getIngestionSchema()
{
return ingestionSchema;
}
@VisibleForTesting
@Nullable
ParallelIndexTaskRunner getRunner()
{
return runner;
}
@VisibleForTesting
AuthorizerMapper getAuthorizerMapper()
{
return authorizerMapper;
}
@VisibleForTesting
ParallelIndexTaskRunner createRunner(TaskToolbox toolbox)
{
if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) {
throw new UnsupportedOperationException("Perfect roll-up is not supported yet");
} else {
runner = new SinglePhaseParallelIndexTaskRunner(
toolbox,
getId(),
getGroupId(),
ingestionSchema,
getContext(),
indexingServiceClient
);
}
return runner;
}
@VisibleForTesting
void setRunner(ParallelIndexTaskRunner runner)
{
this.runner = runner;
}
@Override
public boolean isReady(TaskActionClient taskActionClient) throws Exception
{
final Optional<SortedSet<Interval>> intervals = ingestionSchema.getDataSchema()
.getGranularitySpec()
.bucketIntervals();
return !intervals.isPresent() || isReady(taskActionClient, intervals.get());
}
static boolean isReady(TaskActionClient actionClient, SortedSet<Interval> intervals) throws IOException
{
final List<TaskLock> locks = getTaskLocks(actionClient);
if (locks.isEmpty()) {
try {
Tasks.tryAcquireExclusiveLocks(actionClient, intervals);
}
catch (Exception e) {
log.error(e, "Failed to acquire locks for intervals[%s]", intervals);
return false;
}
}
return true;
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
setToolbox(toolbox);
log.info(
"Found chat handler of class[%s]",
Preconditions.checkNotNull(chatHandlerProvider, "chatHandlerProvider").getClass().getName()
);
chatHandlerProvider.register(getId(), this, false);
try {
if (baseFirehoseFactory.isSplittable()) {
return runParallel(toolbox);
} else {
log.warn(
"firehoseFactory[%s] is not splittable. Running sequentially",
baseFirehoseFactory.getClass().getSimpleName()
);
return runSequential(toolbox);
}
}
finally {
chatHandlerProvider.unregister(getId());
}
}
@VisibleForTesting
void setToolbox(TaskToolbox toolbox)
{
this.toolbox = toolbox;
}
private TaskStatus runParallel(TaskToolbox toolbox) throws Exception
{
createRunner(toolbox);
return TaskStatus.fromCode(getId(), runner.run());
}
private TaskStatus runSequential(TaskToolbox toolbox) throws Exception
{
return new IndexTask(
getId(),
getGroupId(),
getTaskResource(),
getDataSource(),
new IndexIngestionSpec(
getIngestionSchema().getDataSchema(),
getIngestionSchema().getIOConfig(),
convertToIndexTuningConfig(getIngestionSchema().getTuningConfig())
),
getContext(),
authorizerMapper,
chatHandlerProvider,
rowIngestionMetersFactory
).run(toolbox);
}
private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningConfig tuningConfig)
{
return new IndexTuningConfig(
tuningConfig.getTargetPartitionSize(),
tuningConfig.getMaxRowsInMemory(),
tuningConfig.getMaxBytesInMemory(),
tuningConfig.getMaxTotalRows(),
null,
tuningConfig.getNumShards(),
tuningConfig.getIndexSpec(),
tuningConfig.getMaxPendingPersists(),
true,
tuningConfig.isForceExtendableShardSpecs(),
false,
tuningConfig.isReportParseExceptions(),
null,
tuningConfig.getPushTimeout(),
tuningConfig.getSegmentWriteOutMediumFactory(),
tuningConfig.isLogParseExceptions(),
tuningConfig.getMaxParseExceptions(),
tuningConfig.getMaxSavedParseExceptions()
);
}
// Internal APIs
/**
* Allocate a new {@link SegmentIdentifier} for a request from {@link ParallelIndexSubTask}.
* The returned segmentIdentifiers have different {@code partitionNum} (thereby different {@link NumberedShardSpec})
* per bucket interval.
*/
@POST
@Path("/segment/allocate")
@Produces(SmileMediaTypes.APPLICATION_JACKSON_SMILE)
public Response allocateSegment(
DateTime timestamp,
@Context final HttpServletRequest req
)
{
ChatHandlers.authorizationCheck(
req,
Action.READ,
getDataSource(),
authorizerMapper
);
if (toolbox == null) {
return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
}
try {
final SegmentIdentifier segmentIdentifier = allocateNewSegment(timestamp);
return Response.ok(toolbox.getObjectMapper().writeValueAsBytes(segmentIdentifier)).build();
}
catch (IOException | IllegalStateException e) {
return Response.serverError().entity(Throwables.getStackTraceAsString(e)).build();
}
catch (IllegalArgumentException e) {
return Response.status(Response.Status.BAD_REQUEST).entity(Throwables.getStackTraceAsString(e)).build();
}
}
@VisibleForTesting
SegmentIdentifier allocateNewSegment(DateTime timestamp) throws IOException
{
final String dataSource = getDataSource();
final GranularitySpec granularitySpec = getIngestionSchema().getDataSchema().getGranularitySpec();
final SortedSet<Interval> bucketIntervals = Preconditions.checkNotNull(
granularitySpec.bucketIntervals().orNull(),
"bucketIntervals"
);
// List locks whenever allocating a new segment because locks might be revoked and no longer valid.
final Map<Interval, String> versions = toolbox
.getTaskActionClient()
.submit(new LockListAction())
.stream()
.collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion));
final Optional<Interval> maybeInterval = granularitySpec.bucketInterval(timestamp);
if (!maybeInterval.isPresent()) {
throw new IAE("Could not find interval for timestamp [%s]", timestamp);
}
final Interval interval = maybeInterval.get();
if (!bucketIntervals.contains(interval)) {
throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec);
}
final int partitionNum = counters.increment(interval.toString(), 1);
return new SegmentIdentifier(
dataSource,
interval,
findVersion(versions, interval),
new NumberedShardSpec(partitionNum, 0)
);
}
private static String findVersion(Map<Interval, String> versions, Interval interval)
{
return versions.entrySet().stream()
.filter(entry -> entry.getKey().contains(interval))
.map(Entry::getValue)
.findFirst()
.orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval));
}
/**
* {@link ParallelIndexSubTask}s call this API to report the segments they've generated and pushed.
*/
@POST
@Path("/report")
@Consumes(SmileMediaTypes.APPLICATION_JACKSON_SMILE)
public Response report(
PushedSegmentsReport report,
@Context final HttpServletRequest req
)
{
ChatHandlers.authorizationCheck(
req,
Action.WRITE,
getDataSource(),
authorizerMapper
);
if (runner == null) {
return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
} else {
runner.collectReport(report);
return Response.ok().build();
}
}
// External APIs to get running status
@GET
@Path("/mode")
@Produces(MediaType.APPLICATION_JSON)
public Response getMode(@Context final HttpServletRequest req)
{
IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
if (runner == null) {
return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
} else {
return Response.ok(baseFirehoseFactory.isSplittable() ? "parallel" : "sequential").build();
}
}
@GET
@Path("/progress")
@Produces(MediaType.APPLICATION_JSON)
public Response getProgress(@Context final HttpServletRequest req)
{
IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
if (runner == null) {
return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
} else {
return Response.ok(runner.getProgress()).build();
}
}
@GET
@Path("/subtasks/running")
@Produces(MediaType.APPLICATION_JSON)
public Response getRunningTasks(@Context final HttpServletRequest req)
{
IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
if (runner == null) {
return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
} else {
return Response.ok(runner.getRunningTaskIds()).build();
}
}
@GET
@Path("/subtaskspecs")
@Produces(MediaType.APPLICATION_JSON)
public Response getSubTaskSpecs(@Context final HttpServletRequest req)
{
IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
if (runner == null) {
return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
} else {
return Response.ok(runner.getSubTaskSpecs()).build();
}
}
@GET
@Path("/subtaskspecs/running")
@Produces(MediaType.APPLICATION_JSON)
public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req)
{
IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
if (runner == null) {
return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
} else {
return Response.ok(runner.getRunningSubTaskSpecs()).build();
}
}
@GET
@Path("/subtaskspecs/complete")
@Produces(MediaType.APPLICATION_JSON)
public Response getCompleteSubTaskSpecs(@Context final HttpServletRequest req)
{
IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
if (runner == null) {
return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
} else {
return Response.ok(runner.getCompleteSubTaskSpecs()).build();
}
}
@GET
@Path("/subtaskspec/{id}")
@Produces(MediaType.APPLICATION_JSON)
public Response getSubTaskSpec(@PathParam("id") String id, @Context final HttpServletRequest req)
{
IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
if (runner == null) {
return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
} else {
final SubTaskSpec subTaskSpec = runner.getSubTaskSpec(id);
if (subTaskSpec == null) {
return Response.status(Response.Status.NOT_FOUND).build();
} else {
return Response.ok(subTaskSpec).build();
}
}
}
@GET
@Path("/subtaskspec/{id}/state")
@Produces(MediaType.APPLICATION_JSON)
public Response getSubTaskState(@PathParam("id") String id, @Context final HttpServletRequest req)
{
IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
if (runner == null) {
return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
} else {
final SubTaskSpecStatus subTaskSpecStatus = runner.getSubTaskState(id);
if (subTaskSpecStatus == null) {
return Response.status(Response.Status.NOT_FOUND).build();
} else {
return Response.ok(subTaskSpecStatus).build();
}
}
}
@GET
@Path("/subtaskspec/{id}/history")
@Produces(MediaType.APPLICATION_JSON)
public Response getCompleteSubTaskSpecAttemptHistory(
@PathParam("id") String id,
@Context final HttpServletRequest req
)
{
IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper);
if (runner == null) {
return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build();
} else {
final TaskHistory taskHistory = runner.getCompleteSubTaskSpecAttemptHistory(id);
if (taskHistory == null) {
return Response.status(Status.NOT_FOUND).build();
} else {
return Response.ok(taskHistory.getAttemptHistory()).build();
}
}
}
}

View File

@ -0,0 +1,110 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.indexing.common.IndexTaskClient;
import io.druid.indexing.common.TaskInfoProvider;
import io.druid.java.util.common.ISE;
import io.druid.java.util.http.client.HttpClient;
import io.druid.java.util.http.client.response.FullResponseHolder;
import io.druid.segment.realtime.appenderator.SegmentIdentifier;
import io.druid.timeline.DataSegment;
import org.jboss.netty.handler.codec.http.HttpMethod;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import java.io.IOException;
import java.util.List;
public class ParallelIndexTaskClient extends IndexTaskClient
{
private final String subtaskId;
ParallelIndexTaskClient(
HttpClient httpClient,
ObjectMapper objectMapper,
TaskInfoProvider taskInfoProvider,
Duration httpTimeout,
String callerId,
long numRetries
)
{
super(httpClient, objectMapper, taskInfoProvider, httpTimeout, callerId, 1, numRetries);
this.subtaskId = callerId;
}
String getSubtaskId()
{
return subtaskId;
}
public SegmentIdentifier allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
{
final FullResponseHolder response = submitSmileRequest(
supervisorTaskId,
HttpMethod.POST,
"segment/allocate",
null,
serialize(timestamp),
true
);
if (!isSuccess(response)) {
throw new ISE(
"task[%s] failed to allocate a new segment identifier with the HTTP code[%d] and content[%s]",
supervisorTaskId,
response.getStatus().getCode(),
response.getContent()
);
} else {
return deserialize(
response.getContent(),
new TypeReference<SegmentIdentifier>()
{
}
);
}
}
public void report(String supervisorTaskId, List<DataSegment> pushedSegments)
{
try {
final FullResponseHolder response = submitSmileRequest(
supervisorTaskId,
HttpMethod.POST,
"report",
null,
serialize(new PushedSegmentsReport(subtaskId, pushedSegments)),
true
);
if (!isSuccess(response)) {
throw new ISE(
"Failed to send taskReports to task[%s] with the HTTP code [%d]",
supervisorTaskId,
response.getStatus().getCode()
);
}
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
}

View File

@ -0,0 +1,66 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import io.druid.guice.annotations.EscalatedGlobal;
import io.druid.guice.annotations.Smile;
import io.druid.indexing.common.TaskInfoProvider;
import io.druid.indexing.common.task.IndexTaskClientFactory;
import io.druid.java.util.http.client.HttpClient;
import org.joda.time.Duration;
public class ParallelIndexTaskClientFactory implements IndexTaskClientFactory<ParallelIndexTaskClient>
{
private final HttpClient httpClient;
private final ObjectMapper mapper;
@Inject
public ParallelIndexTaskClientFactory(
@EscalatedGlobal HttpClient httpClient,
@Smile ObjectMapper mapper
)
{
this.httpClient = httpClient;
this.mapper = mapper;
}
@Override
public ParallelIndexTaskClient build(
TaskInfoProvider taskInfoProvider,
String callerId,
int numThreads,
Duration httpTimeout,
long numRetries
)
{
Preconditions.checkState(numThreads == 1, "expect numThreads to be 1");
return new ParallelIndexTaskClient(
httpClient,
mapper,
taskInfoProvider,
httpTimeout,
callerId,
numRetries
);
}
}

View File

@ -0,0 +1,137 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.indexer.TaskState;
import io.druid.indexer.TaskStatusPlus;
import io.druid.indexing.common.task.Task;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Set;
/**
* ParallelIndexTaskRunner is the actual task runner of {@link ParallelIndexSupervisorTask}. There is currently a single
* implementation, i.e. {@link SinglePhaseParallelIndexTaskRunner} which supports only best-effort roll-up. We can add
* more implementations for different distributed indexing algorithms in the future.
*/
public interface ParallelIndexTaskRunner<T extends Task>
{
/**
* Runs the task.
*/
TaskState run() throws Exception;
/**
* {@link PushedSegmentsReport} is the report sent by {@link ParallelIndexSubTask}s. The subTasks call this method to
* send their reports after pushing generated segments to deep storage.
*/
void collectReport(PushedSegmentsReport report);
/**
* Returns the current {@link ParallelIndexingProgress}.
*/
ParallelIndexingProgress getProgress();
/**
* Returns the IDs of current running tasks.
*/
Set<String> getRunningTaskIds();
/**
* Returns all {@link SubTaskSpec}s.
*/
List<SubTaskSpec<T>> getSubTaskSpecs();
/**
* Returns running {@link SubTaskSpec}s. A {@link SubTaskSpec} is running if there is a running {@link Task} created
* using that subTaskSpec.
*
* @see SubTaskSpec#newSubTask
*/
List<SubTaskSpec<T>> getRunningSubTaskSpecs();
/**
* Returns complete {@link SubTaskSpec}s. A {@link SubTaskSpec} is complete if there is a succeeded or failed
* {@link Task} created using that subTaskSpec.
*
* @see SubTaskSpec#newSubTask
*/
List<SubTaskSpec<T>> getCompleteSubTaskSpecs();
/**
* Returns the {@link SubTaskSpec} of the given ID or null if it's not found.
*/
@Nullable
SubTaskSpec<T> getSubTaskSpec(String subTaskSpecId);
/**
* Returns {@link SubTaskSpecStatus} of the given ID or null if it's not found.
*/
@Nullable
SubTaskSpecStatus getSubTaskState(String subTaskSpecId);
/**
* Returns {@link TaskHistory} of the given ID or null if it's not found.
*/
@Nullable
TaskHistory<T> getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId);
class SubTaskSpecStatus
{
private final ParallelIndexSubTaskSpec spec;
@Nullable
private final TaskStatusPlus currentStatus; // null if there is no running task for the spec
private final List<TaskStatusPlus> taskHistory; // can be empty if there is no history
@JsonCreator
public SubTaskSpecStatus(
@JsonProperty("spec") ParallelIndexSubTaskSpec spec,
@JsonProperty("currentStatus") @Nullable TaskStatusPlus currentStatus,
@JsonProperty("taskHistory") List<TaskStatusPlus> taskHistory
)
{
this.spec = spec;
this.currentStatus = currentStatus;
this.taskHistory = taskHistory;
}
@JsonProperty
public ParallelIndexSubTaskSpec getSpec()
{
return spec;
}
@JsonProperty
@Nullable
public TaskStatusPlus getCurrentStatus()
{
return currentStatus;
}
@JsonProperty
public List<TaskStatusPlus> getTaskHistory()
{
return taskHistory;
}
}
}

View File

@ -0,0 +1,195 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import io.druid.indexing.common.task.IndexTask.IndexTuningConfig;
import io.druid.segment.IndexSpec;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import org.joda.time.Duration;
import org.joda.time.Period;
import javax.annotation.Nullable;
import java.util.Objects;
@JsonTypeName("index_parallel")
public class ParallelIndexTuningConfig extends IndexTuningConfig
{
private static final int DEFAULT_MAX_NUM_BATCH_TASKS = Integer.MAX_VALUE; // unlimited
private static final int DEFAULT_MAX_RETRY = 3;
private static final long DEFAULT_TASK_STATUS_CHECK_PERIOD_MS = 1000;
private static final Duration DEFAULT_CHAT_HANDLER_TIMEOUT = new Period("PT10S").toStandardDuration();
private static final int DEFAULT_CHAT_HANDLER_NUM_RETRIES = 5;
private final int maxNumSubTasks;
private final int maxRetry;
private final long taskStatusCheckPeriodMs;
private final Duration chatHandlerTimeout;
private final int chatHandlerNumRetries;
public static ParallelIndexTuningConfig defaultConfig()
{
return new ParallelIndexTuningConfig(
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null
);
}
@JsonCreator
public ParallelIndexTuningConfig(
@JsonProperty("targetPartitionSize") @Nullable Integer targetPartitionSize,
@JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory,
@JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory,
@JsonProperty("maxTotalRows") @Nullable Long maxTotalRows,
@JsonProperty("numShards") @Nullable Integer numShards,
@JsonProperty("indexSpec") @Nullable IndexSpec indexSpec,
@JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists,
@JsonProperty("forceExtendableShardSpecs") @Nullable Boolean forceExtendableShardSpecs,
@JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup,
@JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions,
@JsonProperty("pushTimeout") @Nullable Long pushTimeout,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
@JsonProperty("maxNumSubTasks") @Nullable Integer maxNumSubTasks,
@JsonProperty("maxRetry") @Nullable Integer maxRetry,
@JsonProperty("taskStatusCheckPeriodMs") @Nullable Integer taskStatusCheckPeriodMs,
@JsonProperty("chatHandlerTimeout") @Nullable Duration chatHandlerTimeout,
@JsonProperty("chatHandlerNumRetries") @Nullable Integer chatHandlerNumRetries,
@JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions,
@JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions,
@JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions
)
{
super(
targetPartitionSize,
maxRowsInMemory,
maxBytesInMemory,
maxTotalRows,
null,
numShards,
indexSpec,
maxPendingPersists,
null,
forceExtendableShardSpecs,
forceGuaranteedRollup,
reportParseExceptions,
null,
pushTimeout,
segmentWriteOutMediumFactory,
logParseExceptions,
maxParseExceptions,
maxSavedParseExceptions
);
this.maxNumSubTasks = maxNumSubTasks == null ? DEFAULT_MAX_NUM_BATCH_TASKS : maxNumSubTasks;
this.maxRetry = maxRetry == null ? DEFAULT_MAX_RETRY : maxRetry;
this.taskStatusCheckPeriodMs = taskStatusCheckPeriodMs == null ?
DEFAULT_TASK_STATUS_CHECK_PERIOD_MS :
taskStatusCheckPeriodMs;
this.chatHandlerTimeout = DEFAULT_CHAT_HANDLER_TIMEOUT;
this.chatHandlerNumRetries = DEFAULT_CHAT_HANDLER_NUM_RETRIES;
}
@JsonProperty
public int getMaxNumSubTasks()
{
return maxNumSubTasks;
}
@JsonProperty
public int getMaxRetry()
{
return maxRetry;
}
@JsonProperty
public long getTaskStatusCheckPeriodMs()
{
return taskStatusCheckPeriodMs;
}
@JsonProperty
public Duration getChatHandlerTimeout()
{
return chatHandlerTimeout;
}
@JsonProperty
public int getChatHandlerNumRetries()
{
return chatHandlerNumRetries;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
if (!super.equals(o)) {
return false;
}
ParallelIndexTuningConfig that = (ParallelIndexTuningConfig) o;
return maxNumSubTasks == that.maxNumSubTasks &&
maxRetry == that.maxRetry &&
taskStatusCheckPeriodMs == that.taskStatusCheckPeriodMs &&
chatHandlerNumRetries == that.chatHandlerNumRetries &&
Objects.equals(chatHandlerTimeout, that.chatHandlerTimeout);
}
@Override
public int hashCode()
{
return Objects.hash(
super.hashCode(),
maxNumSubTasks,
maxRetry,
taskStatusCheckPeriodMs,
chatHandlerTimeout,
chatHandlerNumRetries
);
}
}

View File

@ -0,0 +1,27 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
/**
* Represents the current progress of {@link ParallelIndexSupervisorTask}. Implementations can be different depending on
* the distributed indexing algorithm.
*/
interface ParallelIndexingProgress
{
}

View File

@ -0,0 +1,60 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import io.druid.timeline.DataSegment;
import java.util.List;
/**
* This class is used in native parallel batch indexing, currently only in {@link SinglePhaseParallelIndexTaskRunner}.
* In native parallel batch indexing, each subTask generates and pushes segments and sends a report to the
* supervisorTask. Once the supervisorTask collects all reports, it publishes all the pushed segments at once.
*/
public class PushedSegmentsReport
{
private final String taskId;
private final List<DataSegment> segments;
@JsonCreator
public PushedSegmentsReport(
@JsonProperty("taskId") String taskId,
@JsonProperty("segments") List<DataSegment> segments
)
{
this.taskId = Preconditions.checkNotNull(taskId, "taskId");
this.segments = Preconditions.checkNotNull(segments, "segments");
}
@JsonProperty
public String getTaskId()
{
return taskId;
}
@JsonProperty
public List<DataSegment> getSegments()
{
return segments;
}
}

View File

@ -0,0 +1,484 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.InputSplit;
import io.druid.indexer.TaskState;
import io.druid.indexer.TaskStatusPlus;
import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.SegmentTransactionalInsertAction;
import io.druid.indexing.common.task.batch.parallel.TaskMonitor.MonitorEntry;
import io.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.realtime.appenderator.SegmentIdentifier;
import io.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import io.druid.segment.realtime.appenderator.UsedSegmentChecker;
import io.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* An implementation of {@link ParallelIndexTaskRunner} to support best-effort roll-up. This runner can submit and
* monitor multiple {@link ParallelIndexSubTask}s.
*
* As its name indicates, distributed indexing is done in a single phase, i.e., without shuffling intermediate data. As
* a result, this task can't be used for perfect rollup.
*/
public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunner<ParallelIndexSubTask>
{
private static final Logger log = new Logger(SinglePhaseParallelIndexTaskRunner.class);
private final TaskToolbox toolbox;
private final String taskId;
private final String groupId;
private final ParallelIndexIngestionSpec ingestionSchema;
private final Map<String, Object> context;
private final FiniteFirehoseFactory<?, ?> baseFirehoseFactory;
private final int maxNumTasks;
private final IndexingServiceClient indexingServiceClient;
private final BlockingQueue<SubTaskCompleteEvent<ParallelIndexSubTask>> taskCompleteEvents =
new LinkedBlockingDeque<>();
// subTaskId -> report
private final ConcurrentMap<String, PushedSegmentsReport> segmentsMap = new ConcurrentHashMap<>();
private volatile boolean stopped;
private volatile TaskMonitor<ParallelIndexSubTask> taskMonitor;
private int nextSpecId = 0;
SinglePhaseParallelIndexTaskRunner(
TaskToolbox toolbox,
String taskId,
String groupId,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient
)
{
this.toolbox = toolbox;
this.taskId = taskId;
this.groupId = groupId;
this.ingestionSchema = ingestionSchema;
this.context = context;
this.baseFirehoseFactory = (FiniteFirehoseFactory) ingestionSchema.getIOConfig().getFirehoseFactory();
this.maxNumTasks = ingestionSchema.getTuningConfig().getMaxNumSubTasks();
this.indexingServiceClient = Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient");
}
@Override
public TaskState run() throws Exception
{
final Iterator<ParallelIndexSubTaskSpec> subTaskSpecIterator = subTaskSpecIterator().iterator();
final long taskStatusCheckingPeriod = ingestionSchema.getTuningConfig().getTaskStatusCheckPeriodMs();
taskMonitor = new TaskMonitor<>(
Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient"),
ingestionSchema.getTuningConfig().getMaxRetry(),
baseFirehoseFactory.getNumSplits()
);
TaskState state = TaskState.RUNNING;
taskMonitor.start(taskStatusCheckingPeriod);
try {
log.info("Submitting initial tasks");
// Submit initial tasks
while (isRunning() && subTaskSpecIterator.hasNext() && taskMonitor.getNumRunningTasks() < maxNumTasks) {
submitNewTask(taskMonitor, subTaskSpecIterator.next());
}
log.info("Waiting for subTasks to be completed");
while (isRunning()) {
final SubTaskCompleteEvent<ParallelIndexSubTask> taskCompleteEvent = taskCompleteEvents.poll(
taskStatusCheckingPeriod,
TimeUnit.MILLISECONDS
);
if (taskCompleteEvent != null) {
final TaskState completeState = taskCompleteEvent.getLastState();
switch (completeState) {
case SUCCESS:
final TaskStatusPlus completeStatus = taskCompleteEvent.getLastStatus();
if (completeStatus == null) {
throw new ISE("Last status of complete task is missing!");
}
// Pushed segments of complete tasks are supposed to be already reported.
if (!segmentsMap.containsKey(completeStatus.getId())) {
throw new ISE("Missing reports from task[%s]!", completeStatus.getId());
}
if (!subTaskSpecIterator.hasNext()) {
// We have no more subTasks to run
if (taskMonitor.getNumRunningTasks() == 0 && taskCompleteEvents.size() == 0) {
stopped = true;
if (taskMonitor.isSucceeded()) {
// Publishing all segments reported so far
publish(toolbox);
// Succeeded
state = TaskState.SUCCESS;
} else {
// Failed
final SinglePhaseParallelIndexingProgress monitorStatus = taskMonitor.getProgress();
throw new ISE(
"Expected for [%d] tasks to succeed, but we got [%d] succeeded tasks and [%d] failed tasks",
monitorStatus.getExpectedSucceeded(),
monitorStatus.getSucceeded(),
monitorStatus.getFailed()
);
}
}
} else if (taskMonitor.getNumRunningTasks() < maxNumTasks) {
// We have more subTasks to run
submitNewTask(taskMonitor, subTaskSpecIterator.next());
} else {
// We have more subTasks to run, but don't have enough available task slots
// do nothing
}
break;
case FAILED:
// TaskMonitor already tried everything it can do for failed tasks. We failed.
state = TaskState.FAILED;
stopped = true;
final TaskStatusPlus lastStatus = taskCompleteEvent.getLastStatus();
if (lastStatus != null) {
log.error("Failed because of the failed sub task[%s]", lastStatus.getId());
} else {
final ParallelIndexSubTaskSpec spec =
(ParallelIndexSubTaskSpec) taskCompleteEvent.getSpec();
log.error(
"Failed to run sub tasks for inputSplits[%s]",
getSplitsIfSplittable(spec.getIngestionSpec().getIOConfig().getFirehoseFactory())
);
}
break;
default:
throw new ISE("spec[%s] is in an invalid state[%s]", taskCompleteEvent.getSpec().getId(), completeState);
}
}
}
}
finally {
log.info("Cleaning up resources");
// Cleanup resources
taskCompleteEvents.clear();
taskMonitor.stop();
if (state != TaskState.SUCCESS) {
log.info(
"This task is finished with [%s] state. Killing [%d] remaining subtasks.",
state,
taskMonitor.getNumRunningTasks()
);
// if this fails, kill all sub tasks
// Note: this doesn't work when this task is killed by users. We need a way for gracefully shutting down tasks
// for resource cleanup.
taskMonitor.killAll();
}
}
return state;
}
private boolean isRunning()
{
return !stopped && !Thread.currentThread().isInterrupted();
}
@VisibleForTesting
TaskToolbox getToolbox()
{
return toolbox;
}
@VisibleForTesting
ParallelIndexIngestionSpec getIngestionSchema()
{
return ingestionSchema;
}
@Override
public void collectReport(PushedSegmentsReport report)
{
// subTasks might send their reports multiple times because of the HTTP retry.
// Here, we simply make sure the current report is exactly same with the previous one.
segmentsMap.compute(report.getTaskId(), (taskId, prevReport) -> {
if (prevReport != null) {
Preconditions.checkState(
prevReport.getSegments().equals(report.getSegments()),
"task[%s] sent two or more reports and previous report[%s] is different from the current one[%s]",
taskId,
prevReport,
report
);
}
return report;
});
}
@Override
public SinglePhaseParallelIndexingProgress getProgress()
{
return taskMonitor == null ? SinglePhaseParallelIndexingProgress.notRunning() : taskMonitor.getProgress();
}
@Override
public Set<String> getRunningTaskIds()
{
return taskMonitor == null ? Collections.emptySet() : taskMonitor.getRunningTaskIds();
}
@Override
public List<SubTaskSpec<ParallelIndexSubTask>> getSubTaskSpecs()
{
if (taskMonitor != null) {
final List<SubTaskSpec<ParallelIndexSubTask>> runningSubTaskSpecs = taskMonitor.getRunningSubTaskSpecs();
final List<SubTaskSpec<ParallelIndexSubTask>> completeSubTaskSpecs = taskMonitor
.getCompleteSubTaskSpecs();
// Deduplicate subTaskSpecs because some subTaskSpec might exist both in runningSubTaskSpecs and
// completeSubTaskSpecs.
final Map<String, SubTaskSpec<ParallelIndexSubTask>> subTaskSpecMap = new HashMap<>(
runningSubTaskSpecs.size() + completeSubTaskSpecs.size()
);
runningSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec));
completeSubTaskSpecs.forEach(spec -> subTaskSpecMap.put(spec.getId(), spec));
return new ArrayList<>(subTaskSpecMap.values());
} else {
return Collections.emptyList();
}
}
@Override
public List<SubTaskSpec<ParallelIndexSubTask>> getRunningSubTaskSpecs()
{
return taskMonitor == null ? Collections.emptyList() : taskMonitor.getRunningSubTaskSpecs();
}
@Override
public List<SubTaskSpec<ParallelIndexSubTask>> getCompleteSubTaskSpecs()
{
return taskMonitor == null ? Collections.emptyList() : taskMonitor.getCompleteSubTaskSpecs();
}
@Nullable
@Override
public SubTaskSpec<ParallelIndexSubTask> getSubTaskSpec(String subTaskSpecId)
{
if (taskMonitor != null) {
// Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after
// adding them to taskHistory.
final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntry(subTaskSpecId);
final TaskHistory<ParallelIndexSubTask> taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId);
final SubTaskSpec<ParallelIndexSubTask> subTaskSpec;
if (monitorEntry != null) {
subTaskSpec = monitorEntry.getSpec();
} else {
if (taskHistory != null) {
subTaskSpec = taskHistory.getSpec();
} else {
subTaskSpec = null;
}
}
return subTaskSpec;
} else {
return null;
}
}
@Nullable
@Override
public SubTaskSpecStatus getSubTaskState(String subTaskSpecId)
{
if (taskMonitor == null) {
return null;
} else {
// Running tasks should be checked first because, in taskMonitor, subTaskSpecs are removed from runningTasks after
// adding them to taskHistory.
final MonitorEntry monitorEntry = taskMonitor.getRunningTaskMonitorEntry(subTaskSpecId);
final TaskHistory<ParallelIndexSubTask> taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId);
final SubTaskSpecStatus subTaskSpecStatus;
if (monitorEntry != null) {
subTaskSpecStatus = new SubTaskSpecStatus(
(ParallelIndexSubTaskSpec) monitorEntry.getSpec(),
monitorEntry.getRunningStatus(),
monitorEntry.getTaskHistory()
);
} else {
if (taskHistory != null && !taskHistory.isEmpty()) {
subTaskSpecStatus = new SubTaskSpecStatus(
(ParallelIndexSubTaskSpec) taskHistory.getSpec(),
null,
taskHistory.getAttemptHistory()
);
} else {
subTaskSpecStatus = null;
}
}
return subTaskSpecStatus;
}
}
@Nullable
@Override
public TaskHistory<ParallelIndexSubTask> getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId)
{
if (taskMonitor == null) {
return null;
} else {
return taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId);
}
}
private void publish(TaskToolbox toolbox) throws IOException
{
final TransactionalSegmentPublisher publisher = (segments, commitMetadata) -> {
final SegmentTransactionalInsertAction action = new SegmentTransactionalInsertAction(segments);
return toolbox.getTaskActionClient().submit(action).isSuccess();
};
final UsedSegmentChecker usedSegmentChecker = new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient());
final Set<DataSegment> segmentsToPublish = segmentsMap
.values()
.stream()
.flatMap(report -> report.getSegments().stream())
.collect(Collectors.toSet());
final boolean published = publisher.publishSegments(segmentsToPublish, null);
if (published) {
log.info("Published segments");
} else {
log.info("Transaction failure while publishing segments, checking if someone else beat us to it.");
final Set<SegmentIdentifier> segmentsIdentifiers = segmentsMap
.values()
.stream()
.flatMap(report -> report.getSegments().stream())
.map(SegmentIdentifier::fromDataSegment)
.collect(Collectors.toSet());
if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers)
.equals(segmentsToPublish)) {
log.info("Our segments really do exist, awaiting handoff.");
} else {
throw new ISE("Failed to publish segments[%s]", segmentsToPublish);
}
}
}
private void submitNewTask(
TaskMonitor<ParallelIndexSubTask> taskMonitor,
ParallelIndexSubTaskSpec spec
)
{
log.info("Submit a new task for spec[%s] and inputSplit[%s]", spec.getId(), spec.getInputSplit());
final ListenableFuture<SubTaskCompleteEvent<ParallelIndexSubTask>> future = taskMonitor.submit(spec);
Futures.addCallback(
future,
new FutureCallback<SubTaskCompleteEvent<ParallelIndexSubTask>>()
{
@Override
public void onSuccess(SubTaskCompleteEvent<ParallelIndexSubTask> completeEvent)
{
// this callback is called if a task completed wheter it succeeded or not.
taskCompleteEvents.offer(completeEvent);
}
@Override
public void onFailure(Throwable t)
{
// this callback is called only when there were some problems in TaskMonitor.
log.error(t, "Error while running a task for subTaskSpec[%s]", spec);
taskCompleteEvents.offer(SubTaskCompleteEvent.fail(spec, t));
}
}
);
}
@VisibleForTesting
int getAndIncrementNextSpecId()
{
return nextSpecId++;
}
@VisibleForTesting
Stream<ParallelIndexSubTaskSpec> subTaskSpecIterator() throws IOException
{
return baseFirehoseFactory.getSplits().map(this::newTaskSpec);
}
@VisibleForTesting
ParallelIndexSubTaskSpec newTaskSpec(InputSplit split)
{
return new ParallelIndexSubTaskSpec(
taskId + "_" + getAndIncrementNextSpecId(),
groupId,
taskId,
new ParallelIndexIngestionSpec(
ingestionSchema.getDataSchema(),
new ParallelIndexIOConfig(
baseFirehoseFactory.withSplit(split),
ingestionSchema.getIOConfig().isAppendToExisting()
),
ingestionSchema.getTuningConfig()
),
context,
split
);
}
private static List<InputSplit> getSplitsIfSplittable(FirehoseFactory firehoseFactory) throws IOException
{
if (firehoseFactory instanceof FiniteFirehoseFactory) {
final FiniteFirehoseFactory<?, ?> finiteFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory;
return finiteFirehoseFactory.getSplits().collect(Collectors.toList());
} else {
throw new ISE("firehoseFactory[%s] is not splittable", firehoseFactory.getClass().getSimpleName());
}
}
}

View File

@ -0,0 +1,117 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
/**
* Represents the current progress of {@link SinglePhaseParallelIndexTaskRunner}.
*/
class SinglePhaseParallelIndexingProgress implements ParallelIndexingProgress
{
/**
* Number of running tasks.
*/
private final int running;
/**
* Number of succeeded tasks.
*/
private final int succeeded;
/**
* Number of failed tasks.
*/
private final int failed;
/**
* Number of complete (succeeded + failed) tasks.
*/
private final int complete;
/**
* Number of total (running + complete) tasks.
*/
private final int total;
/**
* Number of succeeded tasks for {@link SinglePhaseParallelIndexTaskRunner} to succeed.
*/
private final int expectedSucceeded;
static SinglePhaseParallelIndexingProgress notRunning()
{
return new SinglePhaseParallelIndexingProgress(0, 0, 0, 0, 0, -1);
}
@JsonCreator
SinglePhaseParallelIndexingProgress(
@JsonProperty("running") int running,
@JsonProperty("succeeded") int succeeded,
@JsonProperty("failed") int failed,
@JsonProperty("complete") int complete,
@JsonProperty("total") int total,
@JsonProperty("expectedSucceeded") int expectedSucceeded
)
{
this.running = running;
this.succeeded = succeeded;
this.failed = failed;
this.complete = complete;
this.total = total;
this.expectedSucceeded = expectedSucceeded;
}
@JsonProperty
public int getRunning()
{
return running;
}
@JsonProperty
public int getSucceeded()
{
return succeeded;
}
@JsonProperty
public int getFailed()
{
return failed;
}
@JsonProperty
public int getComplete()
{
return complete;
}
@JsonProperty
public int getTotal()
{
return total;
}
@JsonProperty
public int getExpectedSucceeded()
{
return expectedSucceeded;
}
}

View File

@ -0,0 +1,84 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.data.input.InputSplit;
import io.druid.indexing.common.task.Task;
import java.util.Map;
public abstract class SubTaskSpec<T extends Task>
{
private final String id;
private final String groupId;
private final String supervisorTaskId;
private final Map<String, Object> context;
private final InputSplit inputSplit;
@JsonCreator
public SubTaskSpec(
String id,
String groupId,
String supervisorTaskId,
Map<String, Object> context,
InputSplit inputSplit
)
{
this.id = id;
this.groupId = groupId;
this.supervisorTaskId = supervisorTaskId;
this.context = context;
this.inputSplit = inputSplit;
}
@JsonProperty
public String getId()
{
return id;
}
@JsonProperty
public String getGroupId()
{
return groupId;
}
@JsonProperty
public String getSupervisorTaskId()
{
return supervisorTaskId;
}
@JsonProperty
public Map<String, Object> getContext()
{
return context;
}
@JsonProperty
public InputSplit getInputSplit()
{
return inputSplit;
}
public abstract T newSubTask(int numAttempts);
}

View File

@ -0,0 +1,66 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import io.druid.indexer.TaskState;
import io.druid.indexer.TaskStatusPlus;
import io.druid.indexing.common.task.Task;
import java.util.List;
/**
* Task attempt history for complete {@link SubTaskSpec}s. A {@link SubTaskSpec} is complete if its last status is
* succeeded or failed.
*/
class TaskHistory<T extends Task>
{
private final SubTaskSpec<T> spec;
private final List<TaskStatusPlus> attemptHistory; // old attempts to recent ones
TaskHistory(SubTaskSpec<T> spec, List<TaskStatusPlus> attemptHistory)
{
attemptHistory.forEach(status -> {
Preconditions.checkState(
status.getState() == TaskState.SUCCESS || status.getState() == TaskState.FAILED,
"Complete tasks should be recorded, but the state of task[%s] is [%s]",
status.getId(),
status.getState()
);
});
this.spec = spec;
this.attemptHistory = ImmutableList.copyOf(attemptHistory);
}
SubTaskSpec<T> getSpec()
{
return spec;
}
List<TaskStatusPlus> getAttemptHistory()
{
return attemptHistory;
}
boolean isEmpty()
{
return attemptHistory.isEmpty();
}
}

View File

@ -0,0 +1,477 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.client.indexing.TaskStatusResponse;
import io.druid.indexer.TaskState;
import io.druid.indexer.TaskStatusPlus;
import io.druid.indexing.common.task.Task;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.concurrent.Execs;
import io.druid.java.util.common.logger.Logger;
import javax.annotation.Nullable;
import java.util.Iterator;
import java.util.List;
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
/**
* Responsible for submitting tasks, monitoring task statuses, resubmitting failed tasks, and returning the final task
* status.
*/
public class TaskMonitor<T extends Task>
{
private static final Logger log = new Logger(TaskMonitor.class);
private final ScheduledExecutorService taskStatusChecker = Execs.scheduledSingleThreaded(("task-monitor-%d"));
/**
* A map of subTaskSpecId to {@link MonitorEntry}. This map stores the state of running {@link SubTaskSpec}s. This is
* read in {@link java.util.concurrent.Callable} executed by {@link #taskStatusChecker} and updated in {@link #submit}
* and {@link #retry}. This can also be read by calling {@link #getRunningTaskMonitorEntry},
* {@link #getRunningTaskIds}, and {@link #getRunningSubTaskSpecs}.
*/
private final ConcurrentMap<String, MonitorEntry> runningTasks = new ConcurrentHashMap<>();
/**
* A map of subTaskSpecId to {@link TaskHistory}. This map stores the history of complete {@link SubTaskSpec}s
* whether their final state is succeeded or failed. This is updated in {@link MonitorEntry#setLastStatus} which is
* called by the {@link java.util.concurrent.Callable} executed by {@link #taskStatusChecker} and can be
* read by outside of this class.
*/
private final ConcurrentMap<String, TaskHistory<T>> taskHistories = new ConcurrentHashMap<>();
// lock for updating numRunningTasks, numSucceededTasks, and numFailedTasks
private final Object taskCountLock = new Object();
// lock for updating running state
private final Object startStopLock = new Object();
// overlord client
private final IndexingServiceClient indexingServiceClient;
private final int maxRetry;
private final int expectedNumSucceededTasks;
private int numRunningTasks;
private int numSucceededTasks;
private int numFailedTasks;
private boolean running = false;
TaskMonitor(IndexingServiceClient indexingServiceClient, int maxRetry, int expectedNumSucceededTasks)
{
this.indexingServiceClient = Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient");
this.maxRetry = maxRetry;
this.expectedNumSucceededTasks = expectedNumSucceededTasks;
log.info("TaskMonitor is initialized with expectedNumSucceededTasks[%d]", expectedNumSucceededTasks);
}
public void start(long taskStatusCheckingPeriod)
{
synchronized (startStopLock) {
running = true;
log.info("Starting taskMonitor");
// NOTE: This polling can be improved to event-driven pushing by registering TaskRunnerListener to TaskRunner.
// That listener should be able to send the events reported to TaskRunner to this TaskMonitor.
taskStatusChecker.scheduleAtFixedRate(
() -> {
try {
final Iterator<Entry<String, MonitorEntry>> iterator = runningTasks.entrySet().iterator();
while (iterator.hasNext()) {
final Entry<String, MonitorEntry> entry = iterator.next();
final String specId = entry.getKey();
final MonitorEntry monitorEntry = entry.getValue();
final String taskId = monitorEntry.runningTask.getId();
final TaskStatusResponse taskStatusResponse = indexingServiceClient.getTaskStatus(taskId);
final TaskStatusPlus taskStatus = taskStatusResponse.getStatus();
if (taskStatus != null) {
switch (Preconditions.checkNotNull(taskStatus.getState(), "taskState")) {
case SUCCESS:
incrementNumSucceededTasks();
// Remote the current entry after updating taskHistories to make sure that task history
// exists either runningTasks or taskHistories.
monitorEntry.setLastStatus(taskStatus);
iterator.remove();
break;
case FAILED:
incrementNumFailedTasks();
log.warn("task[%s] failed!", taskId);
if (monitorEntry.numTries() < maxRetry) {
log.info(
"We still have chances[%d/%d] to complete for spec[%s].",
monitorEntry.numTries(),
maxRetry,
monitorEntry.spec.getId()
);
retry(specId, monitorEntry, taskStatus);
} else {
log.error(
"spec[%s] failed after [%d] tries",
monitorEntry.spec.getId(),
monitorEntry.numTries()
);
// Remote the current entry after updating taskHistories to make sure that task history
// exists either runningTasks or taskHistories.
monitorEntry.setLastStatus(taskStatus);
iterator.remove();
}
break;
case RUNNING:
monitorEntry.updateStatus(taskStatus);
break;
default:
throw new ISE("Unknown taskStatus[%s] for task[%s[", taskStatus.getState(), taskId);
}
}
}
}
catch (Throwable t) {
log.error(t, "Error while monitoring");
throw t;
}
},
taskStatusCheckingPeriod,
taskStatusCheckingPeriod,
TimeUnit.MILLISECONDS
);
}
}
public void stop()
{
synchronized (startStopLock) {
running = false;
taskStatusChecker.shutdownNow();
log.info("Stopped taskMonitor");
}
}
public ListenableFuture<SubTaskCompleteEvent<T>> submit(SubTaskSpec<T> spec)
{
synchronized (startStopLock) {
if (!running) {
return Futures.immediateFailedFuture(new ISE("TaskMonitore is not running"));
}
final T task = spec.newSubTask(0);
log.info("Submitting a new task[%s] for spec[%s]", task.getId(), spec.getId());
indexingServiceClient.runTask(task);
incrementNumRunningTasks();
final SettableFuture<SubTaskCompleteEvent<T>> taskFuture = SettableFuture.create();
runningTasks.put(
spec.getId(),
new MonitorEntry(spec, task, indexingServiceClient.getTaskStatus(task.getId()).getStatus(), taskFuture)
);
return taskFuture;
}
}
/**
* Submit a retry task for a failed spec. This method should be called inside of the
* {@link java.util.concurrent.Callable} executed by {@link #taskStatusChecker}.
*/
private void retry(String subTaskSpecId, MonitorEntry monitorEntry, TaskStatusPlus lastFailedTaskStatus)
{
synchronized (startStopLock) {
if (running) {
final SubTaskSpec<T> spec = monitorEntry.spec;
final T task = spec.newSubTask(monitorEntry.taskHistory.size() + 1);
log.info("Submitting a new task[%s] for retrying spec[%s]", task.getId(), spec.getId());
indexingServiceClient.runTask(task);
incrementNumRunningTasks();
runningTasks.put(
subTaskSpecId,
monitorEntry.withNewRunningTask(
task,
indexingServiceClient.getTaskStatus(task.getId()).getStatus(),
lastFailedTaskStatus
)
);
}
}
}
/**
* This method should be called after {@link #stop()} to make sure no additional tasks are submitted.
*/
void killAll()
{
runningTasks.values().forEach(entry -> {
final String taskId = entry.runningTask.getId();
log.info("Request to kill subtask[%s]", taskId);
indexingServiceClient.killTask(taskId);
});
runningTasks.clear();
}
void incrementNumRunningTasks()
{
synchronized (taskCountLock) {
numRunningTasks++;
}
}
void incrementNumSucceededTasks()
{
synchronized (taskCountLock) {
numRunningTasks--;
numSucceededTasks++;
log.info("[%d/%d] tasks succeeded", numSucceededTasks, expectedNumSucceededTasks);
}
}
void incrementNumFailedTasks()
{
synchronized (taskCountLock) {
numRunningTasks--;
numFailedTasks++;
}
}
boolean isSucceeded()
{
synchronized (taskCountLock) {
return numSucceededTasks == expectedNumSucceededTasks;
}
}
int getNumRunningTasks()
{
synchronized (taskCountLock) {
return numRunningTasks;
}
}
SinglePhaseParallelIndexingProgress getProgress()
{
synchronized (taskCountLock) {
return new SinglePhaseParallelIndexingProgress(
numRunningTasks,
numSucceededTasks,
numFailedTasks,
numSucceededTasks + numFailedTasks,
numRunningTasks + numSucceededTasks + numFailedTasks,
expectedNumSucceededTasks
);
}
}
Set<String> getRunningTaskIds()
{
return runningTasks.values().stream().map(entry -> entry.runningTask.getId()).collect(Collectors.toSet());
}
List<SubTaskSpec<T>> getRunningSubTaskSpecs()
{
return runningTasks.values().stream().map(monitorEntry -> monitorEntry.spec).collect(Collectors.toList());
}
@Nullable
MonitorEntry getRunningTaskMonitorEntry(String subTaskSpecId)
{
return runningTasks.values()
.stream()
.filter(monitorEntry -> monitorEntry.spec.getId().equals(subTaskSpecId))
.findFirst()
.orElse(null);
}
List<SubTaskSpec<T>> getCompleteSubTaskSpecs()
{
return taskHistories.values().stream().map(TaskHistory::getSpec).collect(Collectors.toList());
}
@Nullable
TaskHistory<T> getCompleteSubTaskSpecHistory(String subTaskSpecId)
{
return taskHistories.get(subTaskSpecId);
}
class MonitorEntry
{
private final SubTaskSpec<T> spec;
private final T runningTask;
// old tasks to recent tasks. running task is not included
private final CopyOnWriteArrayList<TaskStatusPlus> taskHistory;
private final SettableFuture<SubTaskCompleteEvent<T>> completeEventFuture;
/**
* This variable is updated inside of the {@link java.util.concurrent.Callable} executed by
* {@link #taskStatusChecker}, and can be read by calling {@link #getRunningStatus}.
*/
@Nullable
private volatile TaskStatusPlus runningStatus;
MonitorEntry(
SubTaskSpec<T> spec,
T runningTask,
@Nullable TaskStatusPlus runningStatus,
SettableFuture<SubTaskCompleteEvent<T>> completeEventFuture
)
{
this(spec, runningTask, runningStatus, new CopyOnWriteArrayList<>(), completeEventFuture);
}
private MonitorEntry(
SubTaskSpec<T> spec,
T runningTask,
@Nullable TaskStatusPlus runningStatus,
CopyOnWriteArrayList<TaskStatusPlus> taskHistory,
SettableFuture<SubTaskCompleteEvent<T>> completeEventFuture
)
{
this.spec = spec;
this.runningTask = runningTask;
this.runningStatus = runningStatus;
this.taskHistory = taskHistory;
this.completeEventFuture = completeEventFuture;
}
MonitorEntry withNewRunningTask(T newTask, @Nullable TaskStatusPlus newStatus, TaskStatusPlus statusOfLastTask)
{
taskHistory.add(statusOfLastTask);
return new MonitorEntry(
spec,
newTask,
newStatus,
taskHistory,
completeEventFuture
);
}
int numTries()
{
return taskHistory.size() + 1; // count runningTask as well
}
void updateStatus(TaskStatusPlus statusPlus)
{
if (!runningTask.getId().equals(statusPlus.getId())) {
throw new ISE(
"Task id[%s] of lastStatus is different from the running task[%s]",
statusPlus.getId(),
runningTask.getId()
);
}
this.runningStatus = statusPlus;
}
void setLastStatus(TaskStatusPlus lastStatus)
{
if (!runningTask.getId().equals(lastStatus.getId())) {
throw new ISE(
"Task id[%s] of lastStatus is different from the running task[%s]",
lastStatus.getId(),
runningTask.getId()
);
}
this.runningStatus = lastStatus;
taskHistory.add(lastStatus);
taskHistories.put(spec.getId(), new TaskHistory<>(spec, taskHistory));
completeEventFuture.set(SubTaskCompleteEvent.success(spec, lastStatus));
}
SubTaskSpec<T> getSpec()
{
return spec;
}
@Nullable
TaskStatusPlus getRunningStatus()
{
return runningStatus;
}
List<TaskStatusPlus> getTaskHistory()
{
return taskHistory;
}
}
static class SubTaskCompleteEvent<T extends Task>
{
private final SubTaskSpec<T> spec;
@Nullable
private final TaskStatusPlus lastStatus;
@Nullable
private final Throwable throwable;
static <T extends Task> SubTaskCompleteEvent<T> success(SubTaskSpec<T> spec, TaskStatusPlus lastStatus)
{
return new SubTaskCompleteEvent<>(spec, Preconditions.checkNotNull(lastStatus, "lastStatus"), null);
}
static <T extends Task> SubTaskCompleteEvent<T> fail(SubTaskSpec<T> spec, Throwable t)
{
return new SubTaskCompleteEvent<>(spec, null, t);
}
private SubTaskCompleteEvent(
SubTaskSpec<T> spec,
@Nullable TaskStatusPlus lastStatus,
@Nullable Throwable throwable
)
{
this.spec = Preconditions.checkNotNull(spec, "spec");
this.lastStatus = lastStatus;
this.throwable = throwable;
}
SubTaskSpec<T> getSpec()
{
return spec;
}
TaskState getLastState()
{
return lastStatus == null ? TaskState.FAILED : lastStatus.getState();
}
@Nullable
TaskStatusPlus getLastStatus()
{
return lastStatus;
}
@Nullable
Throwable getThrowable()
{
return throwable;
}
}
}

View File

@ -42,6 +42,7 @@ import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.inject.Inject;
import io.druid.guice.annotations.Self;
import io.druid.indexer.RunnerTaskState;
import io.druid.indexer.TaskLocation;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.config.TaskConfig;
@ -68,6 +69,7 @@ import org.apache.commons.io.FileUtils;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy;
import java.io.File;
import java.io.IOException;
@ -666,6 +668,24 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogStreamer
}
}
@Nullable
@Override
public RunnerTaskState getRunnerTaskState(String taskId)
{
final ForkingTaskRunnerWorkItem workItem = tasks.get(taskId);
if (workItem == null) {
return null;
} else {
if (workItem.processHolder == null) {
return RunnerTaskState.PENDING;
} else if (workItem.processHolder.process.isAlive()) {
return RunnerTaskState.RUNNING;
} else {
return RunnerTaskState.NONE;
}
}
}
@Override
public Optional<ScalingStats> getScalingStats()
{

View File

@ -149,6 +149,32 @@ public class HeapMemoryTaskStorage implements TaskStorage
}
}
@Nullable
@Override
public TaskInfo<Task, TaskStatus> getTaskInfo(String taskId)
{
giant.lock();
try {
Preconditions.checkNotNull(taskId, "taskId");
final TaskStuff taskStuff = tasks.get(taskId);
if (taskStuff != null) {
return new TaskInfo<>(
taskStuff.getTask().getId(),
taskStuff.getCreatedDate(),
taskStuff.getStatus(),
taskStuff.getDataSource(),
taskStuff.getTask()
);
} else {
return null;
}
}
finally {
giant.unlock();
}
}
@Override
public List<Task> getActiveTasks()
{

View File

@ -180,6 +180,13 @@ public class MetadataTaskStorage implements TaskStorage
return handler.getStatus(taskId);
}
@Nullable
@Override
public TaskInfo<Task, TaskStatus> getTaskInfo(String taskId)
{
return handler.getTaskInfo(taskId);
}
@Override
public List<Task> getActiveTasks()
{

View File

@ -45,6 +45,7 @@ import com.google.common.util.concurrent.SettableFuture;
import io.druid.concurrent.LifecycleLock;
import io.druid.curator.CuratorUtils;
import io.druid.curator.cache.PathChildrenCacheFactory;
import io.druid.indexer.RunnerTaskState;
import io.druid.indexer.TaskLocation;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.task.Task;
@ -87,6 +88,7 @@ import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.joda.time.Duration;
import org.joda.time.Period;
import javax.annotation.Nullable;
import java.io.IOException;
import java.io.InputStream;
import java.net.MalformedURLException;
@ -465,6 +467,23 @@ public class RemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
return ImmutableList.copyOf(Iterables.concat(pendingTasks.values(), runningTasks.values(), completeTasks.values()));
}
@Nullable
@Override
public RunnerTaskState getRunnerTaskState(String taskId)
{
if (pendingTasks.containsKey(taskId)) {
return RunnerTaskState.PENDING;
}
if (runningTasks.containsKey(taskId)) {
return RunnerTaskState.RUNNING;
}
if (completeTasks.containsKey(taskId)) {
return RunnerTaskState.NONE;
}
return null;
}
@Override
public Optional<ScalingStats> getScalingStats()
{

View File

@ -994,8 +994,20 @@ public class TaskLockbox
boolean addTask(Task task)
{
Preconditions.checkArgument(taskLock.getGroupId().equals(task.getGroupId()));
Preconditions.checkArgument(taskLock.getNonNullPriority() == task.getPriority());
Preconditions.checkArgument(
taskLock.getGroupId().equals(task.getGroupId()),
"groupId[%s] of task[%s] is different from the existing lockPosse's groupId[%s]",
task.getGroupId(),
task.getId(),
taskLock.getGroupId()
);
Preconditions.checkArgument(
taskLock.getNonNullPriority() == task.getPriority(),
"priority[%s] of task[%s] is different from the existing lockPosse's priority[%s]",
task.getPriority(),
task.getId(),
taskLock.getNonNullPriority()
);
return taskIds.add(task.getId());
}

View File

@ -22,11 +22,13 @@ package io.druid.indexing.overlord;
import com.google.common.base.Optional;
import com.google.common.util.concurrent.ListenableFuture;
import io.druid.guice.annotations.PublicApi;
import io.druid.indexer.RunnerTaskState;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.autoscaling.ScalingStats;
import io.druid.java.util.common.Pair;
import javax.annotation.Nullable;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.Executor;
@ -94,6 +96,12 @@ public interface TaskRunner
Collection<? extends TaskRunnerWorkItem> getKnownTasks();
@Nullable
default RunnerTaskState getRunnerTaskState(String taskId)
{
return null;
}
/**
* Some runners are able to scale up and down their capacity in a dynamic manner. This returns stats on those activities
*

View File

@ -95,6 +95,9 @@ public interface TaskStorage
*/
Optional<TaskStatus> getStatus(String taskid);
@Nullable
TaskInfo<Task, TaskStatus> getTaskInfo(String taskId);
/**
* Add an action taken by a task to the audit log.
*
@ -122,8 +125,8 @@ public interface TaskStorage
List<Task> getActiveTasks();
/**
* Returns a list of currently running or pending tasks as stored in the storage facility as {@link TaskInfo}. No particular order
* is guaranteed, but implementations are encouraged to return tasks in ascending order of creation.
* Returns a list of currently running or pending tasks as stored in the storage facility as {@link TaskInfo}. No
* particular order is guaranteed, but implementations are encouraged to return tasks in ascending order of creation.
*
* @param dataSource datasource
*
@ -132,9 +135,10 @@ public interface TaskStorage
List<TaskInfo<Task, TaskStatus>> getActiveTaskInfo(@Nullable String dataSource);
/**
* Returns up to {@code maxTaskStatuses} {@link TaskInfo} objects of recently finished tasks as stored in the storage facility. No
* particular order is guaranteed, but implementations are encouraged to return tasks in descending order of creation.
* No particular standard of "recent" is guaranteed, and in fact, this method is permitted to simply return nothing.
* Returns up to {@code maxTaskStatuses} {@link TaskInfo} objects of recently finished tasks as stored in the storage
* facility. No particular order is guaranteed, but implementations are encouraged to return tasks in descending order
* of creation. No particular standard of "recent" is guaranteed, and in fact, this method is permitted to simply
* return nothing.
*
* @param maxTaskStatuses maxTaskStatuses
* @param duration duration

View File

@ -77,6 +77,12 @@ public class TaskStorageQueryAdapter
return storage.getStatus(taskid);
}
@Nullable
public TaskInfo<Task, TaskStatus> getTaskInfo(String taskId)
{
return storage.getTaskInfo(taskId);
}
/**
* Returns all segments created by this task.
*

View File

@ -41,6 +41,7 @@ import io.druid.discovery.DiscoveryDruidNode;
import io.druid.discovery.DruidNodeDiscovery;
import io.druid.discovery.DruidNodeDiscoveryProvider;
import io.druid.discovery.WorkerNodeService;
import io.druid.indexer.RunnerTaskState;
import io.druid.indexer.TaskLocation;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.task.Task;
@ -1120,6 +1121,27 @@ public class HttpRemoteTaskRunner implements WorkerTaskRunner, TaskLogStreamer
.collect(Collectors.toList());
}
@Nullable
@Override
public RunnerTaskState getRunnerTaskState(String taskId)
{
final HttpRemoteTaskRunnerWorkItem workItem = tasks.get(taskId);
if (workItem == null) {
return null;
} else {
switch (workItem.state) {
case PENDING:
return RunnerTaskState.PENDING;
case RUNNING:
return RunnerTaskState.RUNNING;
case COMPLETE:
return RunnerTaskState.NONE;
default:
throw new ISE("Unknown state[%s]", workItem.state);
}
}
}
public List<String> getBlacklistedWorkers()
{
return blackListedWorkers.values().stream().map(

View File

@ -101,6 +101,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiFunction;
import java.util.stream.Collectors;
/**
@ -243,10 +244,61 @@ public class OverlordResource
@ResourceFilters(TaskResourceFilter.class)
public Response getTaskStatus(@PathParam("taskid") String taskid)
{
final TaskStatusResponse response = new TaskStatusResponse(
taskid,
taskStorageQueryAdapter.getStatus(taskid).orNull()
);
final TaskInfo<Task, TaskStatus> taskInfo = taskStorageQueryAdapter.getTaskInfo(taskid);
TaskStatusResponse response = null;
if (taskInfo != null) {
if (taskMaster.getTaskRunner().isPresent()) {
final TaskRunner taskRunner = taskMaster.getTaskRunner().get();
final TaskRunnerWorkItem workItem = taskRunner
.getKnownTasks()
.stream()
.filter(item -> item.getTaskId().equals(taskid))
.findAny()
.orElse(null);
if (workItem != null) {
response = new TaskStatusResponse(
workItem.getTaskId(),
new TaskStatusPlus(
taskInfo.getId(),
taskInfo.getTask() == null ? null : taskInfo.getTask().getType(),
taskInfo.getCreatedTime(),
// Would be nice to include the real queue insertion time, but the
// TaskStorage API doesn't yet allow it.
DateTimes.EPOCH,
taskInfo.getStatus().getStatusCode(),
taskRunner.getRunnerTaskState(workItem.getTaskId()),
taskInfo.getStatus().getDuration(),
workItem.getLocation(),
taskInfo.getDataSource(),
taskInfo.getStatus().getErrorMsg()
)
);
}
}
if (response == null) {
response = new TaskStatusResponse(
taskid,
new TaskStatusPlus(
taskInfo.getId(),
taskInfo.getTask() == null ? null : taskInfo.getTask().getType(),
taskInfo.getCreatedTime(),
// Would be nice to include the real queue insertion time, but the
// TaskStorage API doesn't yet allow it.
DateTimes.EPOCH,
taskInfo.getStatus().getStatusCode(),
RunnerTaskState.WAITING,
taskInfo.getStatus().getDuration(),
TaskLocation.unknown(),
taskInfo.getDataSource(),
taskInfo.getStatus().getErrorMsg()
)
);
}
} else {
response = new TaskStatusResponse(taskid, null);
}
final Response.Status status = response.getStatus() == null
? Response.Status.NOT_FOUND
@ -626,7 +678,9 @@ public class OverlordResource
final List<TaskInfo<Task, TaskStatus>> taskInfoList = taskStorageQueryAdapter.getRecentlyCompletedTaskInfo(
maxCompletedTasks, duration, dataSource
);
final List<TaskStatusPlus> completedTasks = Lists.transform(taskInfoList, completeTaskTransformFunc);
final List<TaskStatusPlus> completedTasks = taskInfoList.stream()
.map(completeTaskTransformFunc::apply)
.collect(Collectors.toList());
finalTaskList.addAll(completedTasks);
}
@ -651,17 +705,23 @@ public class OverlordResource
}
if (state == null || "waiting".equals(StringUtils.toLowerCase(state))) {
final List<AnyTask> waitingWorkItems = filterActiveTasks(RunnerTaskState.WAITING, allActiveTasks);
List<TaskStatusPlus> transformedWaitingList = Lists.transform(waitingWorkItems, activeTaskTransformFunc);
List<TaskStatusPlus> transformedWaitingList = waitingWorkItems.stream()
.map(activeTaskTransformFunc::apply)
.collect(Collectors.toList());
finalTaskList.addAll(transformedWaitingList);
}
if (state == null || "pending".equals(StringUtils.toLowerCase(state))) {
final List<AnyTask> pendingWorkItems = filterActiveTasks(RunnerTaskState.PENDING, allActiveTasks);
List<TaskStatusPlus> transformedPendingList = Lists.transform(pendingWorkItems, activeTaskTransformFunc);
List<TaskStatusPlus> transformedPendingList = pendingWorkItems.stream()
.map(activeTaskTransformFunc::apply)
.collect(Collectors.toList());
finalTaskList.addAll(transformedPendingList);
}
if (state == null || "running".equals(StringUtils.toLowerCase(state))) {
final List<AnyTask> runningWorkItems = filterActiveTasks(RunnerTaskState.RUNNING, allActiveTasks);
List<TaskStatusPlus> transformedRunningList = Lists.transform(runningWorkItems, activeTaskTransformFunc);
List<TaskStatusPlus> transformedRunningList = runningWorkItems.stream()
.map(activeTaskTransformFunc::apply)
.collect(Collectors.toList());
finalTaskList.addAll(transformedRunningList);
}
final List<TaskStatusPlus> authorizedList = securedTaskStatusPlus(
@ -673,6 +733,24 @@ public class OverlordResource
return Response.ok(authorizedList).build();
}
private static BiFunction<TaskInfo<Task, TaskStatus>, RunnerTaskState, TaskStatusPlus> newTaskInfo2TaskStatusPlusFn()
{
return (taskInfo, runnerTaskState) -> new TaskStatusPlus(
taskInfo.getId(),
taskInfo.getTask() == null ? null : taskInfo.getTask().getType(),
taskInfo.getCreatedTime(),
// Would be nice to include the real queue insertion time, but the
// TaskStorage API doesn't yet allow it.
DateTimes.EPOCH,
taskInfo.getStatus().getStatusCode(),
runnerTaskState,
taskInfo.getStatus().getDuration(),
TaskLocation.unknown(),
taskInfo.getDataSource(),
taskInfo.getStatus().getErrorMsg()
);
}
private List<AnyTask> filterActiveTasks(
RunnerTaskState state,
List<AnyTask> allTasks

View File

@ -21,19 +21,21 @@ package io.druid.indexing.overlord.http;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.indexer.TaskStatus;
import io.druid.indexer.TaskStatusPlus;
import javax.annotation.Nullable;
import java.util.Objects;
public class TaskStatusResponse
{
private final String task; // Task ID, named "task" in the JSONification of this class.
private final TaskStatus status;
@Nullable
private final TaskStatusPlus status;
@JsonCreator
public TaskStatusResponse(
@JsonProperty("task") final String task,
@JsonProperty("status") final TaskStatus status
@JsonProperty("status") @Nullable final TaskStatusPlus status
)
{
this.task = task;
@ -47,7 +49,8 @@ public class TaskStatusResponse
}
@JsonProperty
public TaskStatus getStatus()
@Nullable
public TaskStatusPlus getStatus()
{
return status;
}
@ -69,7 +72,6 @@ public class TaskStatusResponse
@Override
public int hashCode()
{
return Objects.hash(task, status);
}

View File

@ -23,6 +23,9 @@ import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Stopwatch;
import com.google.common.collect.ImmutableMap;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.client.indexing.NoopIndexingServiceClient;
import io.druid.guice.ServerModule;
import io.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory;
import io.druid.indexing.common.stats.RowIngestionMetersFactory;
@ -81,7 +84,7 @@ public class TestUtils
jsonMapper.setInjectableValues(
new InjectableValues.Std()
.addValue(ExprMacroTable.class.getName(), LookupEnabledTestExprMacroTable.INSTANCE)
.addValue(ExprMacroTable.class, LookupEnabledTestExprMacroTable.INSTANCE)
.addValue(IndexIO.class, indexIO)
.addValue(ObjectMapper.class, jsonMapper)
.addValue(ChatHandlerProvider.class, new NoopChatHandlerProvider())
@ -89,6 +92,8 @@ public class TestUtils
.addValue(AuthorizerMapper.class, null)
.addValue(RowIngestionMetersFactory.class, rowIngestionMetersFactory)
.addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT)
.addValue(IndexingServiceClient.class, new NoopIndexingServiceClient())
.addValue(AuthorizerMapper.class, new AuthorizerMapper(ImmutableMap.of()))
);
}

View File

@ -127,7 +127,7 @@ public class SegmentInsertActionTest
actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000);
thrown.expect(IllegalStateException.class);
thrown.expectMessage(CoreMatchers.startsWith("Segments not covered by locks for task"));
thrown.expectMessage(CoreMatchers.containsString("are not covered by locks"));
final Set<DataSegment> segments = actionTestKit.getTaskLockbox().doInCriticalSection(
task,
Collections.singletonList(INTERVAL),

View File

@ -155,7 +155,7 @@ public class SegmentTransactionalInsertActionTest
actionTestKit.getTaskLockbox().lock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000);
thrown.expect(IllegalStateException.class);
thrown.expectMessage(CoreMatchers.startsWith("Segments not covered by locks for task"));
thrown.expectMessage(CoreMatchers.containsString("are not covered by locks"));
SegmentPublishResult result = action.perform(task, actionTestKit.getTaskActionToolbox());
Assert.assertEquals(new SegmentPublishResult(ImmutableSet.of(SEGMENT3), true), result);
}

View File

@ -0,0 +1,45 @@
/*
* 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 io.druid.indexing.common.actions;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskLockType;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.Intervals;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
public class SurrogateActionTest
{
@Test
public void testSerde() throws IOException
{
final ObjectMapper objectMapper = new DefaultObjectMapper();
final SurrogateAction<TaskLock, LockTryAcquireAction> surrogateAction = new SurrogateAction<>(
"testId", new LockTryAcquireAction(TaskLockType.EXCLUSIVE, Intervals.of("2018-01-01/2019-01-01"))
);
final String json = objectMapper.writeValueAsString(surrogateAction);
Assert.assertEquals(surrogateAction.toString(), objectMapper.readValue(json, TaskAction.class).toString());
}
}

View File

@ -21,6 +21,7 @@ package io.druid.indexing.common.actions;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Suppliers;
import io.druid.indexing.common.Counters;
import io.druid.indexing.common.TestUtils;
import io.druid.indexing.common.config.TaskStorageConfig;
import io.druid.indexing.overlord.HeapMemoryTaskStorage;
@ -94,9 +95,11 @@ public class TaskActionTestKit extends ExternalResource
);
taskActionToolbox = new TaskActionToolbox(
taskLockbox,
taskStorage,
metadataStorageCoordinator,
new NoopServiceEmitter(),
EasyMock.createMock(SupervisorManager.class)
EasyMock.createMock(SupervisorManager.class),
new Counters()
);
testDerbyConnector.createDataSourceTable();
testDerbyConnector.createPendingSegmentsTable();

View File

@ -48,6 +48,7 @@ import io.druid.discovery.DruidNodeAnnouncer;
import io.druid.discovery.LookupNodeService;
import io.druid.indexer.IngestionState;
import io.druid.indexer.TaskState;
import io.druid.indexing.common.Counters;
import io.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexing.common.TaskReport;
@ -1393,9 +1394,11 @@ public class AppenderatorDriverRealtimeIndexTaskTest
final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
taskLockbox,
taskStorage,
mdc,
emitter,
EasyMock.createMock(SupervisorManager.class)
EasyMock.createMock(SupervisorManager.class),
new Counters()
);
final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
taskStorage,

View File

@ -0,0 +1,108 @@
/*
* 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 io.druid.indexing.common.task;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.Counters;
import io.druid.indexing.common.TestUtils;
import io.druid.indexing.common.actions.LocalTaskActionClient;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.actions.TaskActionToolbox;
import io.druid.indexing.common.config.TaskStorageConfig;
import io.druid.indexing.overlord.HeapMemoryTaskStorage;
import io.druid.indexing.overlord.TaskLockbox;
import io.druid.indexing.overlord.TaskStorage;
import io.druid.metadata.EntryExistsException;
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
import io.druid.metadata.TestDerbyConnector;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMergerV9;
import io.druid.server.metrics.NoopServiceEmitter;
import org.junit.Rule;
public abstract class IngestionTestBase
{
@Rule
public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule();
private final TestUtils testUtils = new TestUtils();
private final ObjectMapper objectMapper = testUtils.getTestObjectMapper();
private final TaskStorage taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(null));
private final TaskLockbox lockbox = new TaskLockbox(taskStorage);
public IngestionTestBase()
{
}
public TaskActionClient createActionClient(Task task)
{
return new LocalTaskActionClient(task, taskStorage, createTaskActionToolbox());
}
public void prepareTaskForLocking(Task task) throws EntryExistsException
{
lockbox.add(task);
taskStorage.insert(task, TaskStatus.running(task.getId()));
}
public ObjectMapper getObjectMapper()
{
return objectMapper;
}
public TaskStorage getTaskStorage()
{
return taskStorage;
}
public TaskLockbox getLockbox()
{
return lockbox;
}
public TaskActionToolbox createTaskActionToolbox()
{
final IndexerSQLMetadataStorageCoordinator storageCoordinator = new IndexerSQLMetadataStorageCoordinator(
objectMapper,
derbyConnectorRule.metadataTablesConfigSupplier().get(),
derbyConnectorRule.getConnector()
);
storageCoordinator.start();
return new TaskActionToolbox(
lockbox,
taskStorage,
storageCoordinator,
new NoopServiceEmitter(),
null,
new Counters()
);
}
public IndexIO getIndexIO()
{
return testUtils.getTestIndexIO();
}
public IndexMergerV9 getIndexMerger()
{
return testUtils.getTestIndexMergerV9();
}
}

View File

@ -45,6 +45,7 @@ import io.druid.discovery.DataNodeService;
import io.druid.discovery.DruidNodeAnnouncer;
import io.druid.discovery.LookupNodeService;
import io.druid.indexer.TaskState;
import io.druid.indexing.common.Counters;
import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
@ -976,9 +977,11 @@ public class RealtimeIndexTaskTest
taskLockbox.syncFromStorage();
final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
taskLockbox,
taskStorage,
mdc,
emitter,
EasyMock.createMock(SupervisorManager.class)
EasyMock.createMock(SupervisorManager.class),
new Counters()
);
final TaskActionClientFactory taskActionClientFactory = new LocalTaskActionClientFactory(
taskStorage,

View File

@ -0,0 +1,399 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.client.indexing.NoopIndexingServiceClient;
import io.druid.client.indexing.TaskStatusResponse;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.InputSplit;
import io.druid.data.input.impl.CSVParseSpec;
import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.ParseSpec;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.indexer.RunnerTaskState;
import io.druid.indexer.TaskLocation;
import io.druid.indexer.TaskState;
import io.druid.indexer.TaskStatus;
import io.druid.indexer.TaskStatusPlus;
import io.druid.indexing.common.TaskInfoProvider;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory;
import io.druid.indexing.common.task.IndexTaskClientFactory;
import io.druid.indexing.common.task.IngestionTestBase;
import io.druid.indexing.common.task.NoopTestTaskFileWriter;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.common.task.TaskResource;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.concurrent.Execs;
import io.druid.segment.loading.DataSegmentKiller;
import io.druid.segment.loading.LocalDataSegmentPusher;
import io.druid.segment.loading.LocalDataSegmentPusherConfig;
import io.druid.segment.realtime.appenderator.SegmentIdentifier;
import io.druid.segment.realtime.firehose.NoopChatHandlerProvider;
import io.druid.server.security.AllowAllAuthorizer;
import io.druid.server.security.Authorizer;
import io.druid.server.security.AuthorizerMapper;
import io.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.junit.Rule;
import org.junit.rules.TemporaryFolder;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.stream.Stream;
public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
{
static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec(
new TimestampSpec(
"ts",
"auto",
null
),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")),
Lists.newArrayList(),
Lists.newArrayList()
),
null,
Arrays.asList("ts", "dim", "val"),
false,
0
);
TaskActionClient actionClient;
LocalIndexingServiceClient indexingServiceClient;
TaskToolbox toolbox;
File localDeepStorage;
@Rule
public final TemporaryFolder temporaryFolder = new TemporaryFolder();
class LocalIndexingServiceClient extends NoopIndexingServiceClient
{
private final ConcurrentMap<String, Future<TaskStatus>> tasks = new ConcurrentHashMap<>();
private final ListeningExecutorService service = MoreExecutors.listeningDecorator(
Execs.multiThreaded(5, "parallel-index-supervisor-task-test-%d")
);
@Override
public String runTask(Object taskObject)
{
final ParallelIndexSubTask subTask = (ParallelIndexSubTask) taskObject;
tasks.put(subTask.getId(), service.submit(() -> {
try {
final TaskToolbox toolbox = createTaskToolbox(subTask);
if (subTask.isReady(toolbox.getTaskActionClient())) {
return subTask.run(toolbox);
} else {
throw new ISE("task[%s] is not ready", subTask.getId());
}
}
catch (Exception e) {
throw new RuntimeException(e);
}
}));
return subTask.getId();
}
@Override
public TaskStatusResponse getTaskStatus(String taskId)
{
final Future<TaskStatus> taskStatusFuture = tasks.get(taskId);
if (taskStatusFuture != null) {
try {
if (taskStatusFuture.isDone()) {
final TaskStatus taskStatus = taskStatusFuture.get();
return new TaskStatusResponse(
taskId,
new TaskStatusPlus(
taskId,
"index_sub",
DateTimes.EPOCH,
DateTimes.EPOCH,
taskStatus.getStatusCode(),
RunnerTaskState.NONE,
-1L,
TaskLocation.unknown(),
null,
null
)
);
} else {
return new TaskStatusResponse(
taskId,
new TaskStatusPlus(
taskId,
"index_sub",
DateTimes.EPOCH,
DateTimes.EPOCH,
TaskState.RUNNING,
RunnerTaskState.RUNNING,
-1L,
TaskLocation.unknown(),
null,
null
)
);
}
}
catch (InterruptedException | ExecutionException e) {
// We don't have a way to pass this exception to the supervisorTask yet..
// So, let's print it here.
System.err.println(Throwables.getStackTraceAsString(e));
return new TaskStatusResponse(
taskId,
new TaskStatusPlus(
taskId,
"index_sub",
DateTimes.EPOCH,
DateTimes.EPOCH,
TaskState.FAILED,
RunnerTaskState.NONE,
-1L,
TaskLocation.unknown(),
null,
null
)
);
}
} else {
return new TaskStatusResponse(taskId, null);
}
}
@Override
public String killTask(String taskId)
{
final Future<TaskStatus> taskStatusFuture = tasks.remove(taskId);
if (taskStatusFuture != null) {
taskStatusFuture.cancel(true);
return taskId;
} else {
return null;
}
}
void shutdown()
{
service.shutdownNow();
}
}
TaskToolbox createTaskToolbox(Task task) throws IOException
{
return new TaskToolbox(
null,
actionClient,
null,
new LocalDataSegmentPusher(
new LocalDataSegmentPusherConfig()
{
@Override
public File getStorageDirectory()
{
return localDeepStorage;
}
},
getObjectMapper()
),
new DataSegmentKiller()
{
@Override
public void kill(DataSegment segment)
{
}
@Override
public void killAll()
{
}
},
null,
null,
null,
null,
null,
null,
null,
null,
null,
getObjectMapper(),
temporaryFolder.newFolder(task.getId()),
getIndexIO(),
null,
null,
getIndexMerger(),
null,
null,
null,
null,
new NoopTestTaskFileWriter()
);
}
static class TestParallelIndexSupervisorTask extends ParallelIndexSupervisorTask
{
TestParallelIndexSupervisorTask(
String id,
TaskResource taskResource,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient
)
{
super(
id,
taskResource,
ingestionSchema,
context,
indexingServiceClient,
new NoopChatHandlerProvider(),
new AuthorizerMapper(ImmutableMap.of())
{
@Override
public Authorizer getAuthorizer(String name)
{
return new AllowAllAuthorizer();
}
},
new DropwizardRowIngestionMetersFactory()
);
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
return TaskStatus.fromCode(
getId(),
new TestParallelIndexTaskRunner(
toolbox,
getId(),
getGroupId(),
getIngestionSchema(),
getContext(),
new NoopIndexingServiceClient()
).run()
);
}
}
static class TestParallelIndexTaskRunner extends SinglePhaseParallelIndexTaskRunner
{
TestParallelIndexTaskRunner(
TaskToolbox toolbox,
String taskId,
String groupId,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
@Nullable IndexingServiceClient indexingServiceClient
)
{
super(
toolbox,
taskId,
groupId,
ingestionSchema,
context,
indexingServiceClient
);
}
@Override
Stream<ParallelIndexSubTaskSpec> subTaskSpecIterator() throws IOException
{
final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema()
.getIOConfig()
.getFirehoseFactory();
return baseFirehoseFactory.getSplits().map(split -> {
try {
// taskId is suffixed by the current time and this sleep is to make sure that every sub task has different id
Thread.sleep(10);
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
return newTaskSpec((InputSplit<?>) split);
});
}
}
static class LocalParallelIndexTaskClientFactory implements IndexTaskClientFactory<ParallelIndexTaskClient>
{
private final ParallelIndexSupervisorTask supervisorTask;
LocalParallelIndexTaskClientFactory(ParallelIndexSupervisorTask supervisorTask)
{
this.supervisorTask = supervisorTask;
}
@Override
public ParallelIndexTaskClient build(
TaskInfoProvider taskInfoProvider,
String callerId,
int numThreads,
Duration httpTimeout,
long numRetries
)
{
return new LocalParallelIndexTaskClient(callerId, supervisorTask);
}
}
static class LocalParallelIndexTaskClient extends ParallelIndexTaskClient
{
private final ParallelIndexSupervisorTask supervisorTask;
LocalParallelIndexTaskClient(String callerId, ParallelIndexSupervisorTask supervisorTask)
{
super(null, null, null, null, callerId, 0);
this.supervisorTask = supervisorTask;
}
@Override
public SegmentIdentifier allocateSegment(String supervisorTaskId, DateTime timestamp) throws IOException
{
return supervisorTask.allocateNewSegment(timestamp);
}
@Override
public void report(String supervisorTaskId, List<DataSegment> pushedSegments)
{
supervisorTask.getRunner().collectReport(new PushedSegmentsReport(getSubtaskId(), pushedSegments));
}
}
}

View File

@ -0,0 +1,727 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.InputSplit;
import io.druid.data.input.impl.StringInputRowParser;
import io.druid.indexer.RunnerTaskState;
import io.druid.indexer.TaskLocation;
import io.druid.indexer.TaskState;
import io.druid.indexer.TaskStatus;
import io.druid.indexer.TaskStatusPlus;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.LockListAction;
import io.druid.indexing.common.actions.SurrogateAction;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.task.AbstractTask;
import io.druid.indexing.common.task.IndexTaskClientFactory;
import io.druid.indexing.common.task.TaskResource;
import io.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.concurrent.Execs;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.server.security.AuthConfig;
import io.druid.server.security.AuthenticationResult;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NumberedShardSpec;
import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import javax.annotation.Nullable;
import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.core.Response;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelIndexSupervisorTaskTest
{
private static final int NUM_SUB_TASKS = 10;
// specId -> spec
private final ConcurrentMap<String, ParallelIndexSubTaskSpec> subTaskSpecs = new ConcurrentHashMap<>();
// specId -> taskStatusPlus
private final ConcurrentMap<String, TaskStatusPlus> runningSpecs = new ConcurrentHashMap<>();
// specId -> taskStatusPlus list
private final ConcurrentMap<String, List<TaskStatusPlus>> taskHistories = new ConcurrentHashMap<>();
// taskId -> subTaskSpec
private final ConcurrentMap<String, ParallelIndexSubTaskSpec> taskIdToSpec = new ConcurrentHashMap<>();
// taskId -> task
private final CopyOnWriteArrayList<TestSubTask> runningTasks = new CopyOnWriteArrayList<>();
private ExecutorService service;
private TestSupervisorTask task;
@Before
public void setup() throws IOException
{
service = Execs.singleThreaded("parallel-index-supervisor-task-resource-test-%d");
indexingServiceClient = new LocalIndexingServiceClient();
localDeepStorage = temporaryFolder.newFolder("localStorage");
}
@After
public void teardown()
{
indexingServiceClient.shutdown();
temporaryFolder.delete();
service.shutdownNow();
}
@Test(timeout = 20000L)
public void testAPIs() throws Exception
{
task = newTask(
Intervals.of("2017/2018"),
new ParallelIndexIOConfig(
new TestFirehose(IntStream.range(0, NUM_SUB_TASKS).boxed().collect(Collectors.toList())),
false
)
);
actionClient = createActionClient(task);
toolbox = createTaskToolbox(task);
prepareTaskForLocking(task);
Assert.assertTrue(task.isReady(actionClient));
final Future<TaskStatus> supervisorTaskFuture = service.submit(() -> task.run(toolbox));
Thread.sleep(1000);
final SinglePhaseParallelIndexTaskRunner runner = (SinglePhaseParallelIndexTaskRunner) task.getRunner();
Assert.assertNotNull("runner is null", runner);
// test getMode
Response response = task.getMode(newRequest());
Assert.assertEquals(200, response.getStatus());
Assert.assertEquals("parallel", response.getEntity());
// test expectedNumSucceededTasks
response = task.getProgress(newRequest());
Assert.assertEquals(200, response.getStatus());
Assert.assertEquals(NUM_SUB_TASKS, ((SinglePhaseParallelIndexingProgress) response.getEntity()).getExpectedSucceeded());
// Since taskMonitor works based on polling, it's hard to use a fancier way to check its state.
// We use polling to check the state of taskMonitor in this test.
while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getRunning) < NUM_SUB_TASKS) {
Thread.sleep(100);
}
int succeededTasks = 0;
int failedTasks = 0;
checkState(
succeededTasks,
failedTasks,
buildStateMap()
);
// numRunningTasks and numSucceededTasks after some successful subTasks
succeededTasks += 2;
for (int i = 0; i < succeededTasks; i++) {
runningTasks.get(0).setState(TaskState.SUCCESS);
}
while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getSucceeded) < succeededTasks) {
Thread.sleep(100);
}
checkState(
succeededTasks,
failedTasks,
buildStateMap()
);
// numRunningTasks and numSucceededTasks after some failed subTasks
failedTasks += 3;
for (int i = 0; i < failedTasks; i++) {
runningTasks.get(0).setState(TaskState.FAILED);
}
// Wait for new tasks to be started
while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getFailed) < failedTasks || runningTasks.size() < NUM_SUB_TASKS - succeededTasks) {
Thread.sleep(100);
}
checkState(
succeededTasks,
failedTasks,
buildStateMap()
);
// Make sure only one subTask is running
succeededTasks += 7;
for (int i = 0; i < 7; i++) {
runningTasks.get(0).setState(TaskState.SUCCESS);
}
while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getSucceeded) < succeededTasks) {
Thread.sleep(100);
}
checkState(
succeededTasks,
failedTasks,
buildStateMap()
);
Assert.assertEquals(1, runningSpecs.size());
final String lastRunningSpecId = runningSpecs.keySet().iterator().next();
final List<TaskStatusPlus> taskHistory = taskHistories.get(lastRunningSpecId);
// This should be a failed task history because new tasks appear later in runningTasks.
Assert.assertEquals(1, taskHistory.size());
// Test one more failure
runningTasks.get(0).setState(TaskState.FAILED);
failedTasks++;
while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getFailed) < failedTasks) {
Thread.sleep(100);
}
while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getRunning) < 1) {
Thread.sleep(100);
}
checkState(
succeededTasks,
failedTasks,
buildStateMap()
);
Assert.assertEquals(2, taskHistory.size());
runningTasks.get(0).setState(TaskState.SUCCESS);
succeededTasks++;
while (getNumSubTasks(SinglePhaseParallelIndexingProgress::getSucceeded) < succeededTasks) {
Thread.sleep(100);
}
Assert.assertEquals(TaskState.SUCCESS, supervisorTaskFuture.get(1000, TimeUnit.MILLISECONDS).getStatusCode());
}
@SuppressWarnings({"ConstantConditions"})
private int getNumSubTasks(Function<SinglePhaseParallelIndexingProgress, Integer> func)
{
final Response response = task.getProgress(newRequest());
Assert.assertEquals(200, response.getStatus());
return func.apply((SinglePhaseParallelIndexingProgress) response.getEntity());
}
private Map<String, SubTaskSpecStatus> buildStateMap()
{
final Map<String, SubTaskSpecStatus> stateMap = new HashMap<>();
subTaskSpecs.forEach((specId, spec) -> {
final List<TaskStatusPlus> taskHistory = taskHistories.get(specId);
final TaskStatusPlus runningTaskStatus = runningSpecs.get(specId);
stateMap.put(
specId,
new SubTaskSpecStatus(spec, runningTaskStatus, taskHistory == null ? Collections.emptyList() : taskHistory)
);
});
return stateMap;
}
/**
* Test all endpoints of {@link ParallelIndexSupervisorTask}.
*/
private void checkState(
int expectedSucceededTasks,
int expectedFailedTask,
Map<String, SubTaskSpecStatus> expectedSubTaskStateResponses // subTaskSpecId -> response
)
{
Response response = task.getProgress(newRequest());
Assert.assertEquals(200, response.getStatus());
final SinglePhaseParallelIndexingProgress monitorStatus = (SinglePhaseParallelIndexingProgress) response.getEntity();
// numRunningTasks
Assert.assertEquals(runningTasks.size(), monitorStatus.getRunning());
// numSucceededTasks
Assert.assertEquals(expectedSucceededTasks, monitorStatus.getSucceeded());
// numFailedTasks
Assert.assertEquals(expectedFailedTask, monitorStatus.getFailed());
// numCompleteTasks
Assert.assertEquals(expectedSucceededTasks + expectedFailedTask, monitorStatus.getComplete());
// numTotalTasks
Assert.assertEquals(runningTasks.size() + expectedSucceededTasks + expectedFailedTask, monitorStatus.getTotal());
// runningSubTasks
response = task.getRunningTasks(newRequest());
Assert.assertEquals(200, response.getStatus());
Assert.assertEquals(
runningTasks.stream().map(AbstractTask::getId).collect(Collectors.toSet()),
new HashSet<>((Collection<String>) response.getEntity())
);
// subTaskSpecs
response = task.getSubTaskSpecs(newRequest());
Assert.assertEquals(200, response.getStatus());
List<SubTaskSpec<ParallelIndexSubTask>> actualSubTaskSpecMap =
(List<SubTaskSpec<ParallelIndexSubTask>>) response.getEntity();
Assert.assertEquals(
subTaskSpecs.keySet(),
actualSubTaskSpecMap.stream().map(SubTaskSpec::getId).collect(Collectors.toSet())
);
// runningSubTaskSpecs
response = task.getRunningSubTaskSpecs(newRequest());
Assert.assertEquals(200, response.getStatus());
actualSubTaskSpecMap =
(List<SubTaskSpec<ParallelIndexSubTask>>) response.getEntity();
Assert.assertEquals(
runningSpecs.keySet(),
actualSubTaskSpecMap.stream().map(SubTaskSpec::getId).collect(Collectors.toSet())
);
// completeSubTaskSpecs
final List<SubTaskSpec<ParallelIndexSubTask>> completeSubTaskSpecs = expectedSubTaskStateResponses
.entrySet()
.stream()
.filter(entry -> !runningSpecs.containsKey(entry.getKey()))
.map(entry -> entry.getValue().getSpec())
.collect(Collectors.toList());
response = task.getCompleteSubTaskSpecs(newRequest());
Assert.assertEquals(200, response.getStatus());
Assert.assertEquals(completeSubTaskSpecs, response.getEntity());
// subTaskSpec
final String subTaskId = runningSpecs.keySet().iterator().next();
response = task.getSubTaskSpec(subTaskId, newRequest());
Assert.assertEquals(200, response.getStatus());
final SubTaskSpec<ParallelIndexSubTask> subTaskSpec =
(SubTaskSpec<ParallelIndexSubTask>) response.getEntity();
Assert.assertEquals(subTaskId, subTaskSpec.getId());
// subTaskState
response = task.getSubTaskState(subTaskId, newRequest());
Assert.assertEquals(200, response.getStatus());
final SubTaskSpecStatus expectedResponse = Preconditions.checkNotNull(
expectedSubTaskStateResponses.get(subTaskId),
"response for task[%s]",
subTaskId
);
final SubTaskSpecStatus actualResponse = (SubTaskSpecStatus) response.getEntity();
Assert.assertEquals(expectedResponse.getSpec().getId(), actualResponse.getSpec().getId());
Assert.assertEquals(expectedResponse.getCurrentStatus(), actualResponse.getCurrentStatus());
Assert.assertEquals(expectedResponse.getTaskHistory(), actualResponse.getTaskHistory());
// completeSubTaskSpecAttemptHistory
final String completeSubTaskSpecId = expectedSubTaskStateResponses
.entrySet()
.stream()
.filter(entry -> {
final TaskStatusPlus currentStatus = entry.getValue().getCurrentStatus();
return currentStatus != null &&
(currentStatus.getState() == TaskState.SUCCESS || currentStatus.getState() == TaskState.FAILED);
})
.map(Entry::getKey)
.findFirst()
.orElse(null);
if (completeSubTaskSpecId != null) {
response = task.getCompleteSubTaskSpecAttemptHistory(completeSubTaskSpecId, newRequest());
Assert.assertEquals(200, response.getStatus());
Assert.assertEquals(
expectedSubTaskStateResponses.get(completeSubTaskSpecId).getTaskHistory(),
response.getEntity()
);
}
}
private static HttpServletRequest newRequest()
{
final HttpServletRequest request = EasyMock.niceMock(HttpServletRequest.class);
EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null);
EasyMock.expect(request.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT))
.andReturn(new AuthenticationResult("test", "test", "test", Collections.emptyMap()));
EasyMock.replay(request);
return request;
}
private TestSupervisorTask newTask(
Interval interval,
ParallelIndexIOConfig ioConfig
)
{
// set up ingestion spec
final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
getObjectMapper().convertValue(
new StringInputRowParser(
DEFAULT_PARSE_SPEC,
null
),
Map.class
),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
),
null,
getObjectMapper()
),
ioConfig,
new ParallelIndexTuningConfig(
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
NUM_SUB_TASKS,
null,
null,
null,
null,
null,
null,
null
)
);
// set up test tools
return new TestSupervisorTask(
null,
null,
ingestionSpec,
Collections.emptyMap(),
indexingServiceClient
);
}
private static class TestFirehose implements FiniteFirehoseFactory<StringInputRowParser, Integer>
{
private final List<Integer> ids;
TestFirehose(List<Integer> ids)
{
this.ids = ids;
}
@Override
public Stream<InputSplit<Integer>> getSplits()
{
return ids.stream().map(InputSplit::new);
}
@Override
public int getNumSplits()
{
return ids.size();
}
@Override
public FiniteFirehoseFactory<StringInputRowParser, Integer> withSplit(InputSplit<Integer> split)
{
return new TestFirehose(Collections.singletonList(split.get()));
}
}
private class TestSupervisorTask extends TestParallelIndexSupervisorTask
{
TestSupervisorTask(
String id,
TaskResource taskResource,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient
)
{
super(
id,
taskResource,
ingestionSchema,
context,
indexingServiceClient
);
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
setRunner(
new TestRunner(
toolbox,
this,
indexingServiceClient
)
);
return TaskStatus.fromCode(
getId(),
getRunner().run()
);
}
}
private class TestRunner extends TestParallelIndexTaskRunner
{
private final ParallelIndexSupervisorTask supervisorTask;
TestRunner(
TaskToolbox toolbox,
ParallelIndexSupervisorTask supervisorTask,
@Nullable IndexingServiceClient indexingServiceClient
)
{
super(
toolbox,
supervisorTask.getId(),
supervisorTask.getGroupId(),
supervisorTask.getIngestionSchema(),
supervisorTask.getContext(),
indexingServiceClient
);
this.supervisorTask = supervisorTask;
}
@Override
ParallelIndexSubTaskSpec newTaskSpec(InputSplit split)
{
final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema()
.getIOConfig()
.getFirehoseFactory();
final TestSubTaskSpec spec = new TestSubTaskSpec(
supervisorTask.getId() + "_" + getAndIncrementNextSpecId(),
supervisorTask.getGroupId(),
supervisorTask,
this,
new ParallelIndexIngestionSpec(
getIngestionSchema().getDataSchema(),
new ParallelIndexIOConfig(
baseFirehoseFactory.withSplit(split),
getIngestionSchema().getIOConfig().isAppendToExisting()
),
getIngestionSchema().getTuningConfig()
),
supervisorTask.getContext(),
split
);
subTaskSpecs.put(spec.getId(), spec);
return spec;
}
}
private class TestSubTaskSpec extends ParallelIndexSubTaskSpec
{
private final ParallelIndexSupervisorTask supervisorTask;
TestSubTaskSpec(
String id,
String groupId,
ParallelIndexSupervisorTask supervisorTask,
SinglePhaseParallelIndexTaskRunner runner,
ParallelIndexIngestionSpec ingestionSpec,
Map<String, Object> context,
InputSplit inputSplit
)
{
super(id, groupId, supervisorTask.getId(), ingestionSpec, context, inputSplit);
this.supervisorTask = supervisorTask;
}
@Override
public ParallelIndexSubTask newSubTask(int numAttempts)
{
try {
// taskId is suffixed by the current time and this sleep is to make sure that every sub task has different id
Thread.sleep(10);
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
final TestSubTask subTask = new TestSubTask(
getGroupId(),
getSupervisorTaskId(),
numAttempts,
getIngestionSpec(),
getContext(),
new LocalParallelIndexTaskClientFactory(supervisorTask)
);
final TestFirehose firehose = (TestFirehose) getIngestionSpec().getIOConfig().getFirehoseFactory();
final InputSplit<Integer> split = firehose.getSplits().findFirst().orElse(null);
if (split == null) {
throw new ISE("Split is null");
}
runningTasks.add(subTask);
taskIdToSpec.put(subTask.getId(), this);
runningSpecs.put(
getId(),
new TaskStatusPlus(
subTask.getId(),
subTask.getType(),
DateTimes.EPOCH,
DateTimes.EPOCH,
TaskState.RUNNING,
RunnerTaskState.RUNNING,
-1L,
TaskLocation.unknown(),
null,
null
)
);
return subTask;
}
}
private class TestSubTask extends ParallelIndexSubTask
{
private volatile TaskState state = TaskState.RUNNING;
TestSubTask(
String groupId,
String supervisorTaskId,
int numAttempts,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexTaskClientFactory<ParallelIndexTaskClient> taskClientFactory
)
{
super(
null,
groupId,
null,
supervisorTaskId,
numAttempts,
ingestionSchema,
context,
null,
taskClientFactory
);
}
@Override
public boolean isReady(TaskActionClient taskActionClient)
{
return true;
}
@Override
public TaskStatus run(final TaskToolbox toolbox) throws Exception
{
while (state == TaskState.RUNNING) {
Thread.sleep(100);
}
final TestFirehose firehose = (TestFirehose) getIngestionSchema().getIOConfig().getFirehoseFactory();
final List<TaskLock> locks = toolbox.getTaskActionClient()
.submit(new SurrogateAction<>(getSupervisorTaskId(), new LockListAction()));
Preconditions.checkState(locks.size() == 1, "There should be a single lock");
task.getRunner().collectReport(
new PushedSegmentsReport(
getId(),
Collections.singletonList(
new DataSegment(
getDataSource(),
Intervals.of("2017/2018"),
locks.get(0).getVersion(),
null,
null,
null,
new NumberedShardSpec(firehose.ids.get(0), NUM_SUB_TASKS),
0,
1L
)
)
)
);
return TaskStatus.fromCode(getId(), state);
}
void setState(TaskState state)
{
Preconditions.checkArgument(
state == TaskState.SUCCESS || state == TaskState.FAILED,
"state[%s] should be SUCCESS of FAILED",
state
);
this.state = state;
final int taskIndex = IntStream.range(0, runningTasks.size())
.filter(i -> runningTasks.get(i).getId().equals(getId())).findAny()
.orElse(-1);
if (taskIndex == -1) {
throw new ISE("Can't find an index for task[%s]", getId());
}
runningTasks.remove(taskIndex);
final String specId = Preconditions.checkNotNull(taskIdToSpec.get(getId()), "spec for task[%s]", getId()).getId();
runningSpecs.remove(specId);
taskHistories.computeIfAbsent(specId, k -> new ArrayList<>()).add(
new TaskStatusPlus(
getId(),
getType(),
DateTimes.EPOCH,
DateTimes.EPOCH,
state,
RunnerTaskState.NONE,
-1L,
TaskLocation.unknown(),
null,
null
)
);
}
}
}

View File

@ -0,0 +1,164 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.collect.Lists;
import io.druid.client.indexing.NoopIndexingServiceClient;
import io.druid.data.input.impl.CSVParseSpec;
import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.ParseSpec;
import io.druid.data.input.impl.StringInputRowParser;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.indexing.common.TestUtils;
import io.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.common.task.TaskResource;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.segment.realtime.firehose.LocalFirehoseFactory;
import io.druid.segment.realtime.firehose.NoopChatHandlerProvider;
import io.druid.server.security.AuthorizerMapper;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
public class ParallelIndexSupervisorTaskSerdeTest
{
@Rule
public final TemporaryFolder temporaryFolder = new TemporaryFolder();
private static final ParseSpec DEFAULT_PARSE_SPEC = new CSVParseSpec(
new TimestampSpec(
"ts",
"auto",
null
),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")),
Lists.newArrayList(),
Lists.newArrayList()
),
null,
Arrays.asList("ts", "dim", "val"),
false,
0
);
private final TestUtils testUtils = new TestUtils();
@Test
public void testSerde() throws IOException
{
final ObjectMapper objectMapper = testUtils.getTestObjectMapper();
objectMapper.registerSubtypes(
new NamedType(LocalFirehoseFactory.class, "local")
);
final ParallelIndexSupervisorTask task = newTask(
objectMapper,
Intervals.of("2018/2019")
);
final String json = objectMapper.writeValueAsString(task);
Assert.assertEquals(task, objectMapper.readValue(json, Task.class));
}
private ParallelIndexSupervisorTask newTask(
ObjectMapper objectMapper,
Interval interval
)
{
// set up ingestion spec
final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
objectMapper.convertValue(
new StringInputRowParser(
DEFAULT_PARSE_SPEC,
null
),
Map.class
),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
),
null,
objectMapper
),
new ParallelIndexIOConfig(
new LocalFirehoseFactory(new File("tmp"), "test_*", null),
false
),
new ParallelIndexTuningConfig(
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
2,
null,
null,
null,
null,
null,
null,
null
)
);
// set up test tools
return new ParallelIndexSupervisorTask(
"taskId",
new TaskResource("group", 1),
ingestionSpec,
new HashMap<>(),
new NoopIndexingServiceClient(),
new NoopChatHandlerProvider(),
new AuthorizerMapper(Collections.emptyMap()),
new DropwizardRowIngestionMetersFactory()
);
}
}

View File

@ -0,0 +1,370 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.data.input.FiniteFirehoseFactory;
import io.druid.data.input.InputSplit;
import io.druid.data.input.impl.StringInputRowParser;
import io.druid.indexer.TaskState;
import io.druid.indexer.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.task.TaskResource;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import io.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.joda.time.Interval;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.io.Writer;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
public class ParallelIndexSupervisorTaskTest extends AbstractParallelIndexSupervisorTaskTest
{
private File inputDir;
@Before
public void setup() throws IOException
{
inputDir = temporaryFolder.newFolder("data");
// set up data
for (int i = 0; i < 5; i++) {
try (final Writer writer =
Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) {
writer.write(StringUtils.format("2017-12-%d,%d th test file\n", 24 + i, i));
writer.write(StringUtils.format("2017-12-%d,%d th test file\n", 25 + i, i));
}
}
for (int i = 0; i < 5; i++) {
try (final Writer writer =
Files.newBufferedWriter(new File(inputDir, "filtered_" + i).toPath(), StandardCharsets.UTF_8)) {
writer.write(StringUtils.format("2017-12-%d,%d th test file\n", 25 + i, i));
}
}
indexingServiceClient = new LocalIndexingServiceClient();
localDeepStorage = temporaryFolder.newFolder("localStorage");
}
@After
public void teardown()
{
indexingServiceClient.shutdown();
temporaryFolder.delete();
}
@Test
public void testIsReady() throws Exception
{
final ParallelIndexSupervisorTask task = newTask(
Intervals.of("2017/2018"),
new ParallelIndexIOConfig(
new LocalFirehoseFactory(inputDir, "test_*", null),
false
)
);
actionClient = createActionClient(task);
toolbox = createTaskToolbox(task);
prepareTaskForLocking(task);
Assert.assertTrue(task.isReady(actionClient));
final SinglePhaseParallelIndexTaskRunner runner = (SinglePhaseParallelIndexTaskRunner) task.createRunner(toolbox);
final Iterator<ParallelIndexSubTaskSpec> subTaskSpecIterator = runner.subTaskSpecIterator().iterator();
while (subTaskSpecIterator.hasNext()) {
final ParallelIndexSubTaskSpec spec = subTaskSpecIterator.next();
final ParallelIndexSubTask subTask = new ParallelIndexSubTask(
null,
spec.getGroupId(),
null,
spec.getSupervisorTaskId(),
0,
spec.getIngestionSpec(),
spec.getContext(),
indexingServiceClient,
null
);
final TaskActionClient subTaskActionClient = createActionClient(subTask);
prepareTaskForLocking(subTask);
Assert.assertTrue(subTask.isReady(subTaskActionClient));
}
}
@Test
public void testWithoutInterval() throws Exception
{
final ParallelIndexSupervisorTask task = newTask(
null,
new ParallelIndexIOConfig(
new LocalFirehoseFactory(inputDir, "test_*", null),
false
)
);
actionClient = createActionClient(task);
toolbox = createTaskToolbox(task);
prepareTaskForLocking(task);
Assert.assertTrue(task.isReady(actionClient));
Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode());
}
@Test()
public void testRunInParallel() throws Exception
{
final ParallelIndexSupervisorTask task = newTask(
Intervals.of("2017/2018"),
new ParallelIndexIOConfig(
new LocalFirehoseFactory(inputDir, "test_*", null),
false
)
);
actionClient = createActionClient(task);
toolbox = createTaskToolbox(task);
prepareTaskForLocking(task);
Assert.assertTrue(task.isReady(actionClient));
Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode());
}
@Test
public void testRunInSequential() throws Exception
{
final ParallelIndexSupervisorTask task = newTask(
Intervals.of("2017/2018"),
new ParallelIndexIOConfig(
new LocalFirehoseFactory(inputDir, "test_*", null)
{
@Override
public boolean isSplittable()
{
return false;
}
},
false
)
);
actionClient = createActionClient(task);
toolbox = createTaskToolbox(task);
prepareTaskForLocking(task);
Assert.assertTrue(task.isReady(actionClient));
Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode());
}
private ParallelIndexSupervisorTask newTask(
Interval interval,
ParallelIndexIOConfig ioConfig
)
{
// set up ingestion spec
final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
getObjectMapper().convertValue(
new StringInputRowParser(
DEFAULT_PARSE_SPEC,
null
),
Map.class
),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
),
null,
getObjectMapper()
),
ioConfig,
new ParallelIndexTuningConfig(
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
2,
null,
null,
null,
null,
null,
null,
null
)
);
// set up test tools
return new TestSupervisorTask(
null,
null,
ingestionSpec,
new HashMap<>(),
indexingServiceClient
);
}
private static class TestSupervisorTask extends TestParallelIndexSupervisorTask
{
private final IndexingServiceClient indexingServiceClient;
TestSupervisorTask(
String id,
TaskResource taskResource,
ParallelIndexIngestionSpec ingestionSchema,
Map<String, Object> context,
IndexingServiceClient indexingServiceClient
)
{
super(
id,
taskResource,
ingestionSchema,
context,
indexingServiceClient
);
this.indexingServiceClient = indexingServiceClient;
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
setToolbox(toolbox);
setRunner(
new TestRunner(
toolbox,
this,
indexingServiceClient
)
);
return TaskStatus.fromCode(
getId(),
getRunner().run()
);
}
}
private static class TestRunner extends TestParallelIndexTaskRunner
{
private final ParallelIndexSupervisorTask supervisorTask;
TestRunner(
TaskToolbox toolbox,
ParallelIndexSupervisorTask supervisorTask,
@Nullable IndexingServiceClient indexingServiceClient
)
{
super(
toolbox,
supervisorTask.getId(),
supervisorTask.getGroupId(),
supervisorTask.getIngestionSchema(),
supervisorTask.getContext(),
indexingServiceClient
);
this.supervisorTask = supervisorTask;
}
@Override
ParallelIndexSubTaskSpec newTaskSpec(InputSplit split)
{
final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema()
.getIOConfig()
.getFirehoseFactory();
return new TestParallelIndexSubTaskSpec(
supervisorTask.getId() + "_" + getAndIncrementNextSpecId(),
supervisorTask.getGroupId(),
supervisorTask,
new ParallelIndexIngestionSpec(
getIngestionSchema().getDataSchema(),
new ParallelIndexIOConfig(
baseFirehoseFactory.withSplit(split),
getIngestionSchema().getIOConfig().isAppendToExisting()
),
getIngestionSchema().getTuningConfig()
),
supervisorTask.getContext(),
split
);
}
}
private static class TestParallelIndexSubTaskSpec extends ParallelIndexSubTaskSpec
{
private final ParallelIndexSupervisorTask supervisorTask;
TestParallelIndexSubTaskSpec(
String id,
String groupId,
ParallelIndexSupervisorTask supervisorTask,
ParallelIndexIngestionSpec ingestionSpec,
Map<String, Object> context,
InputSplit inputSplit
)
{
super(id, groupId, supervisorTask.getId(), ingestionSpec, context, inputSplit);
this.supervisorTask = supervisorTask;
}
@Override
public ParallelIndexSubTask newSubTask(int numAttempts)
{
return new ParallelIndexSubTask(
null,
getGroupId(),
null,
getSupervisorTaskId(),
numAttempts,
getIngestionSpec(),
getContext(),
null,
new LocalParallelIndexTaskClientFactory(supervisorTask)
);
}
}
}

View File

@ -0,0 +1,220 @@
/*
* 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 io.druid.indexing.common.task.batch.parallel;
import com.google.common.util.concurrent.ListenableFuture;
import io.druid.client.indexing.NoopIndexingServiceClient;
import io.druid.client.indexing.TaskStatusResponse;
import io.druid.data.input.InputSplit;
import io.druid.indexer.RunnerTaskState;
import io.druid.indexer.TaskLocation;
import io.druid.indexer.TaskState;
import io.druid.indexer.TaskStatus;
import io.druid.indexer.TaskStatusPlus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.task.NoopTask;
import io.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.concurrent.Execs;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
public class TaskMonitorTest
{
private static final int SPLIT_NUM = 10;
private final ExecutorService taskRunner = Execs.multiThreaded(5, "task-monitor-test-%d");
private final ConcurrentMap<String, TaskState> tasks = new ConcurrentHashMap<>();
private final TaskMonitor<TestTask> monitor = new TaskMonitor<>(new TestIndexingServiceClient(), 3, SPLIT_NUM);
@Before
public void setup()
{
tasks.clear();
monitor.start(100);
}
@After
public void teardown()
{
monitor.stop();
taskRunner.shutdownNow();
}
@Test
public void testBasic() throws InterruptedException, ExecutionException, TimeoutException
{
final List<ListenableFuture<SubTaskCompleteEvent<TestTask>>> futures = IntStream
.range(0, 10)
.mapToObj(i -> monitor.submit(
new TestTaskSpec("specId" + i, "groupId", "supervisorId", null, new IntegerInputSplit(i), 100L, 0)
))
.collect(Collectors.toList());
for (int i = 0; i < futures.size(); i++) {
// # of threads of taskRunner is 5, so the expected max timeout is 2 sec. We additionally wait three more seconds
// here to make sure the test passes.
final SubTaskCompleteEvent<TestTask> result = futures.get(i).get(1, TimeUnit.SECONDS);
Assert.assertEquals("supervisorId", result.getSpec().getSupervisorTaskId());
Assert.assertEquals("specId" + i, result.getSpec().getId());
Assert.assertNotNull(result.getLastStatus());
Assert.assertEquals(TaskState.SUCCESS, result.getLastStatus().getState());
Assert.assertEquals(TaskState.SUCCESS, result.getLastState());
}
}
@Test
public void testRetry() throws InterruptedException, ExecutionException, TimeoutException
{
final List<TestTaskSpec> specs = IntStream
.range(0, 10)
.mapToObj(
i -> new TestTaskSpec("specId" + i, "groupId", "supervisorId", null, new IntegerInputSplit(i), 100L, 2)
)
.collect(Collectors.toList());
final List<ListenableFuture<SubTaskCompleteEvent<TestTask>>> futures = specs
.stream()
.map(monitor::submit)
.collect(Collectors.toList());
for (int i = 0; i < futures.size(); i++) {
// # of threads of taskRunner is 5, and each task is expected to be run 3 times (with 2 retries), so the expected
// max timeout is 6 sec. We additionally wait 4 more seconds here to make sure the test passes.
final SubTaskCompleteEvent<TestTask> result = futures.get(i).get(2, TimeUnit.SECONDS);
Assert.assertEquals("supervisorId", result.getSpec().getSupervisorTaskId());
Assert.assertEquals("specId" + i, result.getSpec().getId());
Assert.assertNotNull(result.getLastStatus());
Assert.assertEquals(TaskState.SUCCESS, result.getLastStatus().getState());
Assert.assertEquals(TaskState.SUCCESS, result.getLastState());
final TaskHistory<TestTask> taskHistory = monitor.getCompleteSubTaskSpecHistory(specs.get(i).getId());
Assert.assertNotNull(taskHistory);
final List<TaskStatusPlus> attemptHistory = taskHistory.getAttemptHistory();
Assert.assertNotNull(attemptHistory);
Assert.assertEquals(3, attemptHistory.size());
Assert.assertEquals(TaskState.FAILED, attemptHistory.get(0).getState());
Assert.assertEquals(TaskState.FAILED, attemptHistory.get(1).getState());
}
}
private static class TestTaskSpec extends SubTaskSpec<TestTask>
{
private final long runTime;
private final int numMaxFails;
private int numFails;
public TestTaskSpec(
String id,
String groupId,
String supervisorTaskId,
Map<String, Object> context,
InputSplit inputSplit,
long runTime,
int numMaxFails
)
{
super(id, groupId, supervisorTaskId, context, inputSplit);
this.runTime = runTime;
this.numMaxFails = numMaxFails;
}
@Override
public TestTask newSubTask(int numAttempts)
{
return new TestTask(getId(), runTime, numFails++ < numMaxFails);
}
}
private static class TestTask extends NoopTask
{
private final boolean shouldFail;
TestTask(String id, long runTime, boolean shouldFail)
{
super(id, "testDataSource", runTime, 0, null, null, null);
this.shouldFail = shouldFail;
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
if (shouldFail) {
Thread.sleep(getRunTime());
return TaskStatus.failure(getId());
} else {
return super.run(toolbox);
}
}
}
private class TestIndexingServiceClient extends NoopIndexingServiceClient
{
@Override
public String runTask(Object taskObject)
{
final TestTask task = (TestTask) taskObject;
tasks.put(task.getId(), TaskState.RUNNING);
taskRunner.submit(() -> tasks.put(task.getId(), task.run(null).getStatusCode()));
return task.getId();
}
@Override
public TaskStatusResponse getTaskStatus(String taskId)
{
return new TaskStatusResponse(
taskId,
new TaskStatusPlus(
taskId,
"testTask",
DateTimes.EPOCH,
DateTimes.EPOCH,
tasks.get(taskId),
RunnerTaskState.RUNNING,
-1L,
TaskLocation.unknown(),
"testDataSource",
null
)
);
}
}
private static class IntegerInputSplit extends InputSplit<Integer>
{
public IntegerInputSplit(int split)
{
super(split);
}
}
}

View File

@ -41,6 +41,7 @@ import io.druid.data.input.impl.TimestampSpec;
import io.druid.guice.GuiceAnnotationIntrospector;
import io.druid.guice.GuiceInjectableValues;
import io.druid.guice.GuiceInjectors;
import io.druid.indexing.common.Counters;
import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexing.common.TaskToolboxFactory;
import io.druid.indexing.common.TestUtils;
@ -215,7 +216,14 @@ public class IngestSegmentFirehoseFactoryTest
};
final LocalTaskActionClientFactory tac = new LocalTaskActionClientFactory(
TASK_STORAGE,
new TaskActionToolbox(TASK_LOCKBOX, mdc, newMockEmitter(), EasyMock.createMock(SupervisorManager.class))
new TaskActionToolbox(
TASK_LOCKBOX,
TASK_STORAGE,
mdc,
newMockEmitter(),
EasyMock.createMock(SupervisorManager.class),
new Counters()
)
);
SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class);
EasyMock.replay(notifierFactory);

View File

@ -44,6 +44,7 @@ import io.druid.discovery.DataNodeService;
import io.druid.discovery.DruidNodeAnnouncer;
import io.druid.discovery.LookupNodeService;
import io.druid.indexer.TaskState;
import io.druid.indexing.common.Counters;
import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexing.common.TaskLock;
import io.druid.indexer.TaskStatus;
@ -527,8 +528,17 @@ public class TaskLifecycleTest
Preconditions.checkNotNull(emitter);
taskLockbox = new TaskLockbox(taskStorage);
tac = new LocalTaskActionClientFactory(taskStorage, new TaskActionToolbox(taskLockbox, mdc, emitter, EasyMock.createMock(
SupervisorManager.class)));
tac = new LocalTaskActionClientFactory(
taskStorage,
new TaskActionToolbox(
taskLockbox,
taskStorage,
mdc,
emitter,
EasyMock.createMock(SupervisorManager.class),
new Counters()
)
);
File tmpDir = temporaryFolder.newFolder();
taskConfig = new TaskConfig(tmpDir.toString(), null, null, 50000, null, false, null, null);

View File

@ -23,8 +23,10 @@ import com.google.common.base.Optional;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.ListenableFuture;
import io.druid.indexer.RunnerTaskState;
import io.druid.indexer.TaskInfo;
import io.druid.indexer.TaskLocation;
import io.druid.indexer.TaskState;
import io.druid.indexer.TaskStatus;
import io.druid.indexer.TaskStatusPlus;
import io.druid.indexing.common.TaskToolbox;
@ -860,11 +862,17 @@ public class OverlordResourceTest
public void testGetTaskStatus() throws Exception
{
expectAuthorizationTokenCheck();
EasyMock.expect(taskStorageQueryAdapter.getStatus("mytask"))
.andReturn(Optional.of(TaskStatus.success("mytask")));
final Task task = NoopTask.create("mytask", 0);
final TaskStatus status = TaskStatus.running("mytask");
EasyMock.expect(taskStorageQueryAdapter.getStatus("othertask"))
.andReturn(Optional.absent());
EasyMock.expect(taskStorageQueryAdapter.getTaskInfo("mytask"))
.andReturn(new TaskInfo<>(task.getId(), DateTimes.of("2018-01-01"), status, task.getDataSource(), task));
EasyMock.expect(taskStorageQueryAdapter.getTaskInfo("othertask"))
.andReturn(null);
EasyMock.<Collection<? extends TaskRunnerWorkItem>>expect(taskRunner.getKnownTasks())
.andReturn(ImmutableList.of());
EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req);
@ -873,7 +881,24 @@ public class OverlordResourceTest
TestHelper.makeJsonMapper().writeValueAsString(response1.getEntity()),
TaskStatusResponse.class
);
Assert.assertEquals(new TaskStatusResponse("mytask", TaskStatus.success("mytask")), taskStatusResponse1);
Assert.assertEquals(
new TaskStatusResponse(
"mytask",
new TaskStatusPlus(
"mytask",
"noop",
DateTimes.of("2018-01-01"),
DateTimes.EPOCH,
TaskState.RUNNING,
RunnerTaskState.RUNNING,
-1L,
TaskLocation.unknown(),
task.getDataSource(),
null
)
),
taskStatusResponse1
);
final Response response2 = overlordResource.getTaskStatus("othertask");
final TaskStatusResponse taskStatusResponse2 = TestHelper.makeJsonMapper().readValue(

View File

@ -248,7 +248,7 @@ public class OverlordTest
Assert.assertEquals(taskId_0, ((TaskStatusResponse) response.getEntity()).getTask());
Assert.assertEquals(
TaskStatus.running(taskId_0).getStatusCode(),
((TaskStatusResponse) response.getEntity()).getStatus().getStatusCode()
((TaskStatusResponse) response.getEntity()).getStatus().getState()
);
// Simulate completion of task_0
@ -296,7 +296,7 @@ public class OverlordTest
{
while (true) {
Response response = overlordResource.getTaskStatus(taskId);
if (status.equals(((TaskStatusResponse) response.getEntity()).getStatus().getStatusCode())) {
if (status.equals(((TaskStatusResponse) response.getEntity()).getStatus().getState())) {
break;
}
Thread.sleep(10);

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Predicates;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import io.druid.client.indexing.TaskStatusResponse;
import io.druid.indexer.TaskState;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.RetryUtils;
@ -125,12 +126,13 @@ public class OverlordResourceTestClient
);
LOG.info("Index status response" + response.getContent());
Map<String, Object> responseData = jsonMapper.readValue(
response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
TaskStatusResponse taskStatusResponse = jsonMapper.readValue(
response.getContent(),
new TypeReference<TaskStatusResponse>()
{
}
);
//TODO: figure out a better way to parse the response...
String status = (String) ((Map) responseData.get("status")).get("status");
return TaskState.valueOf(status);
return taskStatusResponse.getStatus().getState();
}
catch (Exception e) {
throw Throwables.propagate(e);

View File

@ -0,0 +1,91 @@
/*
* 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 io.druid.tests.indexer;
import com.google.inject.Inject;
import io.druid.java.util.common.logger.Logger;
import io.druid.testing.IntegrationTestingConfig;
import io.druid.testing.clients.ClientInfoResourceTestClient;
import io.druid.testing.utils.RetryUtil;
import org.junit.Assert;
import java.io.IOException;
import java.util.List;
public class AbstractITBatchIndexTest extends AbstractIndexerTest
{
private static final Logger LOG = new Logger(AbstractITBatchIndexTest.class);
@Inject
IntegrationTestingConfig config;
@Inject
ClientInfoResourceTestClient clientInfoResourceTestClient;
void doIndexTestTest(
String dataSource,
String indexTaskFilePath,
String queryFilePath
) throws IOException
{
submitTaskAndWait(indexTaskFilePath, dataSource);
try {
queryHelper.testQueriesFromFile(queryFilePath, 2);
}
catch (Exception e) {
LOG.error(e, "Error while testing");
throw new RuntimeException(e);
}
}
void doReindexTest(
String reindexDataSource,
String reindexTaskFilePath,
String queryFilePath
) throws IOException
{
submitTaskAndWait(reindexTaskFilePath, reindexDataSource);
try {
queryHelper.testQueriesFromFile(queryFilePath, 2);
// verify excluded dimension is not reIndexed
final List<String> dimensions = clientInfoResourceTestClient.getDimensions(
reindexDataSource,
"2013-08-31T00:00:00.000Z/2013-09-10T00:00:00.000Z"
);
Assert.assertFalse("dimensions : " + dimensions, dimensions.contains("robot"));
}
catch (Exception e) {
LOG.error(e, "Error while testing");
throw new RuntimeException(e);
}
}
private void submitTaskAndWait(String indexTaskFilePath, String dataSourceName) throws IOException
{
final String taskID = indexer.submitTask(getTaskAsString(indexTaskFilePath));
LOG.info("TaskID for loading index task %s", taskID);
indexer.waitUntilTaskCompletes(taskID);
RetryUtil.retryUntilTrue(
() -> coordinator.areSegmentsLoaded(dataSourceName), "Segment Load"
);
}
}

View File

@ -19,97 +19,37 @@
package io.druid.tests.indexer;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import io.druid.java.util.common.logger.Logger;
import io.druid.testing.IntegrationTestingConfig;
import io.druid.testing.clients.ClientInfoResourceTestClient;
import io.druid.testing.guice.DruidTestModuleFactory;
import io.druid.testing.utils.RetryUtil;
import org.junit.Assert;
import org.testng.annotations.Guice;
import org.testng.annotations.Test;
import java.util.List;
import java.util.concurrent.Callable;
@Guice(moduleFactory = DruidTestModuleFactory.class)
public class ITIndexerTest extends AbstractIndexerTest
public class ITIndexerTest extends AbstractITBatchIndexTest
{
private static final Logger LOG = new Logger(ITIndexerTest.class);
private static String INDEX_TASK = "/indexer/wikipedia_index_task.json";
private static String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
private static String INDEX_DATASOURCE = "wikipedia_index_test";
private static String REINDEX_TASK = "/indexer/wikipedia_reindex_task.json";
private static String REINDEX_DATASOURCE = "wikipedia_reindex_test";
@Inject
private IntegrationTestingConfig config;
@Inject
ClientInfoResourceTestClient clientInfoResourceTestClient;
@Test
public void testIndexData() throws Exception
{
loadData();
try {
queryHelper.testQueriesFromFile(INDEX_QUERIES_RESOURCE, 2);
reIndexData();
queryHelper.testQueriesFromFile(INDEX_QUERIES_RESOURCE, 2);
// verify excluded dimension is not reIndexed
final List<String> dimensions = clientInfoResourceTestClient.getDimensions(
REINDEX_DATASOURCE,
"2013-08-31T00:00:00.000Z/2013-09-10T00:00:00.000Z"
doIndexTestTest(
INDEX_DATASOURCE,
INDEX_TASK,
INDEX_QUERIES_RESOURCE
);
doReindexTest(
REINDEX_DATASOURCE,
REINDEX_TASK,
INDEX_QUERIES_RESOURCE
);
Assert.assertFalse("dimensions : " + dimensions, dimensions.contains("robot"));
}
catch (Exception e) {
LOG.error(e, "Error while testing");
throw Throwables.propagate(e);
}
finally {
unloadAndKillData(INDEX_DATASOURCE);
unloadAndKillData(REINDEX_DATASOURCE);
}
}
private void loadData() throws Exception
{
final String taskID = indexer.submitTask(getTaskAsString(INDEX_TASK));
LOG.info("TaskID for loading index task %s", taskID);
indexer.waitUntilTaskCompletes(taskID);
RetryUtil.retryUntilTrue(
new Callable<Boolean>()
{
@Override
public Boolean call()
{
return coordinator.areSegmentsLoaded(INDEX_DATASOURCE);
}
}, "Segment Load"
);
}
private void reIndexData() throws Exception
{
final String taskID = indexer.submitTask(getTaskAsString(REINDEX_TASK));
LOG.info("TaskID for loading index task %s", taskID);
indexer.waitUntilTaskCompletes(taskID);
RetryUtil.retryUntilTrue(
new Callable<Boolean>()
{
@Override
public Boolean call()
{
return coordinator.areSegmentsLoaded(REINDEX_DATASOURCE);
}
}, "Segment Load"
);
}
}

View File

@ -0,0 +1,47 @@
/*
* 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 io.druid.tests.indexer;
import io.druid.testing.guice.DruidTestModuleFactory;
import org.testng.annotations.Guice;
import org.testng.annotations.Test;
@Guice(moduleFactory = DruidTestModuleFactory.class)
public class ITParallelIndexTest extends AbstractITBatchIndexTest
{
private static String INDEX_TASK = "/indexer/wikipedia_parallel_index_task.json";
private static String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_parallel_index_queries.json";
private static String INDEX_DATASOURCE = "wikipedia_parallel_index_test";
@Test
public void testIndexData() throws Exception
{
try {
doIndexTestTest(
INDEX_DATASOURCE,
INDEX_TASK,
INDEX_QUERIES_RESOURCE
);
}
finally {
unloadAndKillData(INDEX_DATASOURCE);
}
}
}

View File

@ -47,7 +47,7 @@ public class ITUnionQueryTest extends AbstractIndexerTest
private static final Logger LOG = new Logger(ITUnionQueryTest.class);
private static final String UNION_TASK_RESOURCE = "/indexer/wikipedia_union_index_task.json";
private static final String EVENT_RECEIVER_SERVICE_PREFIX = "eventReceiverServiceName";
private static final String UNION_DATA_FILE = "/indexer/wikipedia_index_data.json";
private static final String UNION_DATA_FILE = "/data/union_query/wikipedia_index_data.json";
private static final String UNION_QUERIES_RESOURCE = "/indexer/union_queries.json";
private static final String UNION_DATASOURCE = "wikipedia_index_test";
@ -91,7 +91,9 @@ public class ITUnionQueryTest extends AbstractIndexerTest
RetryUtil.retryUntil(
() -> {
for (int i = 0; i < numTasks; i++) {
if (queryHelper.countRows(UNION_DATASOURCE + i, "2013-08-31/2013-09-01") < 5) {
final int countRows = queryHelper.countRows(UNION_DATASOURCE + i, "2013-08-31/2013-09-01");
if (countRows < 5) {
LOG.warn("%d events have been ingested to %s so far", countRows, UNION_DATASOURCE + i);
return false;
}
}

View File

@ -0,0 +1,3 @@
{"timestamp": "2013-08-31T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}
{"timestamp": "2013-08-31T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
{"timestamp": "2013-08-31T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}

View File

@ -0,0 +1,3 @@
{"timestamp": "2013-08-31T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
{"timestamp": "2013-08-31T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}
{"timestamp": "2013-09-01T01:02:33Z", "page": "Gypsy Danger", "language" : "en", "user" : "nuclear", "unpatrolled" : "true", "newPage" : "true", "robot": "false", "anonymous": "false", "namespace":"article", "continent":"North America", "country":"United States", "region":"Bay Area", "city":"San Francisco", "added": 57, "deleted": 200, "delta": -143}

View File

@ -0,0 +1,4 @@
{"timestamp": "2013-09-01T03:32:45Z", "page": "Striker Eureka", "language" : "en", "user" : "speed", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Australia", "country":"Australia", "region":"Cantebury", "city":"Syndey", "added": 459, "deleted": 129, "delta": 330}
{"timestamp": "2013-09-01T07:11:21Z", "page": "Cherno Alpha", "language" : "ru", "user" : "masterYi", "unpatrolled" : "false", "newPage" : "true", "robot": "true", "anonymous": "false", "namespace":"article", "continent":"Asia", "country":"Russia", "region":"Oblast", "city":"Moscow", "added": 123, "deleted": 12, "delta": 111}
{"timestamp": "2013-09-01T11:58:39Z", "page": "Crimson Typhoon", "language" : "zh", "user" : "triplets", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"China", "region":"Shanxi", "city":"Taiyuan", "added": 905, "deleted": 5, "delta": 900}
{"timestamp": "2013-09-01T12:41:27Z", "page": "Coyote Tango", "language" : "ja", "user" : "stringer", "unpatrolled" : "true", "newPage" : "false", "robot": "true", "anonymous": "false", "namespace":"wikipedia", "continent":"Asia", "country":"Japan", "region":"Kanto", "city":"Tokyo", "added": 1, "deleted": 10, "delta": -9}

View File

@ -58,8 +58,8 @@
"type": "index",
"firehose": {
"type": "local",
"baseDir": "/resources/indexer",
"filter": "wikipedia_index_data.json"
"baseDir": "/resources/data/batch_index",
"filter": "wikipedia_index_data*"
}
},
"tuningConfig": {

View File

@ -0,0 +1,71 @@
[
{
"description": "timeseries, 1 agg, all",
"query":{
"queryType" : "timeBoundary",
"dataSource": "wikipedia_parallel_index_test"
},
"expectedResults":[
{
"timestamp" : "2013-08-31T01:02:33.000Z",
"result" : {
"minTime" : "2013-08-31T01:02:33.000Z",
"maxTime" : "2013-09-01T12:41:27.000Z"
}
}
]
},
{
"description":"having spec on post aggregation",
"query":{
"queryType":"groupBy",
"dataSource":"wikipedia_parallel_index_test",
"granularity":"day",
"dimensions":[
"page"
],
"filter":{
"type":"selector",
"dimension":"language",
"value":"zh"
},
"aggregations":[
{
"type":"count",
"name":"rows"
},
{
"type":"longSum",
"fieldName":"added",
"name":"added_count"
}
],
"postAggregations": [
{
"type":"arithmetic",
"name":"added_count_times_ten",
"fn":"*",
"fields":[
{"type":"fieldAccess", "name":"added_count", "fieldName":"added_count"},
{"type":"constant", "name":"const", "value":10}
]
}
],
"having":{"type":"greaterThan", "aggregation":"added_count_times_ten", "value":9000},
"intervals":[
"2013-08-31T00:00/2013-09-01T00:00"
]
},
"expectedResults":[ {
"version" : "v1",
"timestamp" : "2013-08-31T00:00:00.000Z",
"event" : {
"added_count_times_ten" : 9050.0,
"page" : "Crimson Typhoon",
"added_count" : 905,
"rows" : 1
}
} ]
}
]

View File

@ -0,0 +1,66 @@
{
"type": "index_parallel",
"spec": {
"dataSchema": {
"dataSource": "wikipedia_parallel_index_test",
"metricsSpec": [
{
"type": "count",
"name": "count"
},
{
"type": "doubleSum",
"name": "added",
"fieldName": "added"
},
{
"type": "doubleSum",
"name": "deleted",
"fieldName": "deleted"
},
{
"type": "doubleSum",
"name": "delta",
"fieldName": "delta"
}
],
"granularitySpec": {
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
}
},
"ioConfig": {
"type": "index_parallel",
"firehose": {
"type": "local",
"baseDir": "/resources/data/batch_index",
"filter": "wikipedia_index_data*"
}
}
}
}

View File

@ -0,0 +1,305 @@
/*
* 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 io.druid.client.indexing;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import io.druid.discovery.DruidLeaderClient;
import io.druid.indexer.TaskStatusPlus;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.http.client.response.FullResponseHolder;
import io.druid.timeline.DataSegment;
import org.jboss.netty.handler.codec.http.HttpMethod;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import javax.ws.rs.core.MediaType;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
public class HttpIndexingServiceClient implements IndexingServiceClient
{
private final DruidLeaderClient druidLeaderClient;
private final ObjectMapper jsonMapper;
@Inject
public HttpIndexingServiceClient(
ObjectMapper jsonMapper,
@IndexingService DruidLeaderClient druidLeaderClient
)
{
this.jsonMapper = jsonMapper;
this.druidLeaderClient = druidLeaderClient;
}
@Override
public void mergeSegments(List<DataSegment> segments)
{
final Iterator<DataSegment> segmentsIter = segments.iterator();
if (!segmentsIter.hasNext()) {
return;
}
final String dataSource = segmentsIter.next().getDataSource();
while (segmentsIter.hasNext()) {
DataSegment next = segmentsIter.next();
if (!dataSource.equals(next.getDataSource())) {
throw new IAE("Cannot merge segments of different dataSources[%s] and [%s]", dataSource, next.getDataSource());
}
}
runTask(new ClientAppendQuery(dataSource, segments));
}
@Override
public void killSegments(String dataSource, Interval interval)
{
runTask(new ClientKillQuery(dataSource, interval));
}
@Override
public void upgradeSegment(DataSegment dataSegment)
{
runTask(new ClientConversionQuery(dataSegment));
}
@Override
public String compactSegments(
List<DataSegment> segments,
int compactionTaskPriority,
@Nullable ClientCompactQueryTuningConfig tuningConfig,
@Nullable Map<String, Object> context
)
{
Preconditions.checkArgument(segments.size() > 1, "Expect two or more segments to compact");
final String dataSource = segments.get(0).getDataSource();
Preconditions.checkArgument(
segments.stream().allMatch(segment -> segment.getDataSource().equals(dataSource)),
"Segments must have the same dataSource"
);
context = context == null ? new HashMap<>() : context;
context.put("priority", compactionTaskPriority);
return runTask(new ClientCompactQuery(dataSource, segments, tuningConfig, context));
}
@Override
public String runTask(Object taskObject)
{
try {
final FullResponseHolder response = druidLeaderClient.go(
druidLeaderClient.makeRequest(
HttpMethod.POST,
"/druid/indexer/v1/task"
).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(taskObject))
);
if (!response.getStatus().equals(HttpResponseStatus.OK)) {
throw new ISE("Failed to post task[%s]", taskObject);
}
final Map<String, Object> resultMap = jsonMapper.readValue(
response.getContent(),
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
final String taskId = (String) resultMap.get("task");
return Preconditions.checkNotNull(taskId, "Null task id for task[%s]", taskObject);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Override
public String killTask(String taskId)
{
try {
final FullResponseHolder response = druidLeaderClient.go(
druidLeaderClient.makeRequest(
HttpMethod.POST,
StringUtils.format("/druid/indexer/v1/task/%s/shutdown", taskId)
)
);
if (!response.getStatus().equals(HttpResponseStatus.OK)) {
throw new ISE("Failed to kill task[%s]", taskId);
}
final Map<String, Object> resultMap = jsonMapper.readValue(
response.getContent(),
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
final String killedTaskId = (String) resultMap.get("task");
Preconditions.checkNotNull(killedTaskId, "Null task id returned for task[%s]", taskId);
Preconditions.checkState(
taskId.equals(killedTaskId),
"Requested to kill task[%s], but another task[%s] was killed!",
taskId,
killedTaskId
);
return killedTaskId;
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Override
public int getTotalWorkerCapacity()
{
try {
final FullResponseHolder response = druidLeaderClient.go(
druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/workers")
.setHeader("Content-Type", MediaType.APPLICATION_JSON)
);
if (!response.getStatus().equals(HttpResponseStatus.OK)) {
throw new ISE(
"Error while getting available cluster capacity. status[%s] content[%s]",
response.getStatus(),
response.getContent()
);
}
final Collection<IndexingWorkerInfo> workers = jsonMapper.readValue(
response.getContent(),
new TypeReference<Collection<IndexingWorkerInfo>>() {}
);
return workers.stream().mapToInt(workerInfo -> workerInfo.getWorker().getCapacity()).sum();
}
catch (Exception e) {
throw new RuntimeException(e);
}
}
@Override
public List<TaskStatusPlus> getRunningTasks()
{
return getTasks("runningTasks");
}
@Override
public List<TaskStatusPlus> getPendingTasks()
{
return getTasks("pendingTasks");
}
@Override
public List<TaskStatusPlus> getWaitingTasks()
{
return getTasks("waitingTasks");
}
private List<TaskStatusPlus> getTasks(String endpointSuffix)
{
try {
final FullResponseHolder responseHolder = druidLeaderClient.go(
druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/%s", endpointSuffix))
);
if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) {
throw new ISE("Error while fetching the status of the last complete task");
}
return jsonMapper.readValue(
responseHolder.getContent(),
new TypeReference<List<TaskStatusPlus>>()
{
}
);
}
catch (IOException | InterruptedException e) {
throw new RuntimeException(e);
}
}
@Override
public TaskStatusResponse getTaskStatus(String taskId)
{
try {
final FullResponseHolder responseHolder = druidLeaderClient.go(
druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/task/%s/status", taskId))
);
return jsonMapper.readValue(
responseHolder.getContent(),
new TypeReference<TaskStatusResponse>()
{
}
);
}
catch (IOException | InterruptedException e) {
throw new RuntimeException(e);
}
}
@Override
@Nullable
public TaskStatusPlus getLastCompleteTask()
{
final List<TaskStatusPlus> completeTaskStatuses = getTasks("completeTasks?n=1");
return completeTaskStatuses.isEmpty() ? null : completeTaskStatuses.get(0);
}
@Override
public int killPendingSegments(String dataSource, DateTime end)
{
final String endPoint = StringUtils.format(
"/druid/indexer/v1/pendingSegments/%s?interval=%s",
dataSource,
new Interval(DateTimes.MIN, end)
);
try {
final FullResponseHolder responseHolder = druidLeaderClient.go(
druidLeaderClient.makeRequest(HttpMethod.DELETE, endPoint)
);
if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) {
throw new ISE("Error while killing pendingSegments of dataSource[%s] created until [%s]", dataSource, end);
}
final Map<String, Object> resultMap = jsonMapper.readValue(
responseHolder.getContent(),
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
final Object numDeletedObject = resultMap.get("numDeleted");
return (Integer) Preconditions.checkNotNull(numDeletedObject, "numDeletedObject");
}
catch (Exception e) {
throw new RuntimeException(e);
}
}
}

View File

@ -19,222 +19,46 @@
package io.druid.client.indexing;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import io.druid.discovery.DruidLeaderClient;
import io.druid.indexer.TaskStatusPlus;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.jackson.JacksonUtils;
import io.druid.java.util.http.client.response.FullResponseHolder;
import io.druid.timeline.DataSegment;
import org.jboss.netty.handler.codec.http.HttpMethod;
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import javax.ws.rs.core.MediaType;
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
public class IndexingServiceClient
public interface IndexingServiceClient
{
private final DruidLeaderClient druidLeaderClient;
private final ObjectMapper jsonMapper;
void killSegments(String dataSource, Interval interval);
@Inject
public IndexingServiceClient(
ObjectMapper jsonMapper,
@IndexingService DruidLeaderClient druidLeaderClient
)
{
this.jsonMapper = jsonMapper;
this.druidLeaderClient = druidLeaderClient;
}
int killPendingSegments(String dataSource, DateTime end);
public void mergeSegments(List<DataSegment> segments)
{
final Iterator<DataSegment> segmentsIter = segments.iterator();
if (!segmentsIter.hasNext()) {
return;
}
void upgradeSegment(DataSegment dataSegment);
final String dataSource = segmentsIter.next().getDataSource();
while (segmentsIter.hasNext()) {
DataSegment next = segmentsIter.next();
if (!dataSource.equals(next.getDataSource())) {
throw new IAE("Cannot merge segments of different dataSources[%s] and [%s]", dataSource, next.getDataSource());
}
}
void mergeSegments(List<DataSegment> segments);
runQuery(new ClientAppendQuery(dataSource, segments));
}
public void killSegments(String dataSource, Interval interval)
{
runQuery(new ClientKillQuery(dataSource, interval));
}
public void upgradeSegment(DataSegment dataSegment)
{
runQuery(new ClientConversionQuery(dataSegment));
}
public String compactSegments(
String compactSegments(
List<DataSegment> segments,
int compactionTaskPriority,
@Nullable ClientCompactQueryTuningConfig tuningConfig,
@Nullable Map<String, Object> context
)
{
Preconditions.checkArgument(segments.size() > 1, "Expect two or more segments to compact");
);
final String dataSource = segments.get(0).getDataSource();
Preconditions.checkArgument(
segments.stream().allMatch(segment -> segment.getDataSource().equals(dataSource)),
"Segments must have the same dataSource"
);
int getTotalWorkerCapacity();
context = context == null ? new HashMap<>() : context;
context.put("priority", compactionTaskPriority);
String runTask(Object taskObject);
return runQuery(new ClientCompactQuery(dataSource, segments, tuningConfig, context));
}
String killTask(String taskId);
private String runQuery(Object queryObject)
{
try {
final FullResponseHolder response = druidLeaderClient.go(
druidLeaderClient.makeRequest(
HttpMethod.POST,
"/druid/indexer/v1/task"
).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(queryObject))
);
List<TaskStatusPlus> getRunningTasks();
if (!response.getStatus().equals(HttpResponseStatus.OK)) {
throw new ISE("Failed to post query[%s]", queryObject);
}
List<TaskStatusPlus> getPendingTasks();
final Map<String, Object> resultMap = jsonMapper.readValue(
response.getContent(),
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
final String taskId = (String) resultMap.get("task");
return Preconditions.checkNotNull(taskId, "Null task id for query[%s]", queryObject);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
List<TaskStatusPlus> getWaitingTasks();
public int getTotalWorkerCapacity()
{
try {
final FullResponseHolder response = druidLeaderClient.go(
druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/workers")
.setHeader("Content-Type", MediaType.APPLICATION_JSON)
);
if (!response.getStatus().equals(HttpResponseStatus.OK)) {
throw new ISE(
"Error while getting available cluster capacity. status[%s] content[%s]",
response.getStatus(),
response.getContent()
);
}
final Collection<IndexingWorkerInfo> workers = jsonMapper.readValue(
response.getContent(),
new TypeReference<Collection<IndexingWorkerInfo>>() {}
);
return workers.stream().mapToInt(workerInfo -> workerInfo.getWorker().getCapacity()).sum();
}
catch (Exception e) {
throw new RuntimeException(e);
}
}
public List<TaskStatusPlus> getRunningTasks()
{
return getTasks("runningTasks");
}
public List<TaskStatusPlus> getPendingTasks()
{
return getTasks("pendingTasks");
}
public List<TaskStatusPlus> getWaitingTasks()
{
return getTasks("waitingTasks");
}
private List<TaskStatusPlus> getTasks(String endpointSuffix)
{
try {
final FullResponseHolder responseHolder = druidLeaderClient.go(
druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/%s", endpointSuffix))
);
if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) {
throw new ISE("Error while fetching the status of the last complete task");
}
return jsonMapper.readValue(
responseHolder.getContent(),
new TypeReference<List<TaskStatusPlus>>()
{
}
);
}
catch (IOException | InterruptedException e) {
throw new RuntimeException(e);
}
}
TaskStatusResponse getTaskStatus(String taskId);
@Nullable
public TaskStatusPlus getLastCompleteTask()
{
final List<TaskStatusPlus> completeTaskStatuses = getTasks("completeTasks?n=1");
return completeTaskStatuses.isEmpty() ? null : completeTaskStatuses.get(0);
}
public int killPendingSegments(String dataSource, DateTime end)
{
final String endPoint = StringUtils.format(
"/druid/indexer/v1/pendingSegments/%s?interval=%s",
dataSource,
new Interval(DateTimes.MIN, end)
);
try {
final FullResponseHolder responseHolder = druidLeaderClient.go(
druidLeaderClient.makeRequest(HttpMethod.DELETE, endPoint)
);
if (!responseHolder.getStatus().equals(HttpResponseStatus.OK)) {
throw new ISE("Error while killing pendingSegments of dataSource[%s] created until [%s]", dataSource, end);
}
final Map<String, Object> resultMap = jsonMapper.readValue(
responseHolder.getContent(),
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
final Object numDeletedObject = resultMap.get("numDeleted");
return (Integer) Preconditions.checkNotNull(numDeletedObject, "numDeletedObject");
}
catch (Exception e) {
throw new RuntimeException(e);
}
}
TaskStatusPlus getLastCompleteTask();
}

View File

@ -0,0 +1,101 @@
/*
* 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 io.druid.client.indexing;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import io.druid.indexer.TaskState;
/**
* Should be synced with io.druid.indexing.common.TaskStatus
*/
public class TaskStatus
{
private final String id;
private final TaskState status;
private final long duration;
@JsonCreator
public TaskStatus(
@JsonProperty("id") String id,
@JsonProperty("status") TaskState status,
@JsonProperty("duration") long duration
)
{
this.id = id;
this.status = status;
this.duration = duration;
// Check class invariants.
Preconditions.checkNotNull(id, "id");
Preconditions.checkNotNull(status, "status");
}
@JsonProperty("id")
public String getId()
{
return id;
}
@JsonProperty("status")
public TaskState getStatusCode()
{
return status;
}
@JsonProperty("duration")
public long getDuration()
{
return duration;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TaskStatus that = (TaskStatus) o;
return duration == that.duration &&
java.util.Objects.equals(id, that.id) &&
status == that.status;
}
@Override
public int hashCode()
{
return java.util.Objects.hash(id, status, duration);
}
@Override
public String toString()
{
return Objects.toStringHelper(this)
.add("id", id)
.add("status", status)
.add("duration", duration)
.toString();
}
}

View File

@ -0,0 +1,89 @@
/*
* 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 io.druid.client.indexing;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.indexer.TaskStatusPlus;
import javax.annotation.Nullable;
import java.util.Objects;
/**
* Should be synced with io.druid.indexing.overlord.http.TaskStatusResponse
*/
public class TaskStatusResponse
{
private final String task; // Task ID, named "task" in the JSONification of this class.
@Nullable
private final TaskStatusPlus status;
@JsonCreator
public TaskStatusResponse(
@JsonProperty("task") final String task,
@JsonProperty("status") @Nullable final TaskStatusPlus status
)
{
this.task = task;
this.status = status;
}
@JsonProperty
public String getTask()
{
return task;
}
@JsonProperty
@Nullable
public TaskStatusPlus getStatus()
{
return status;
}
@Override
public boolean equals(final Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final TaskStatusResponse that = (TaskStatusResponse) o;
return Objects.equals(task, that.task) &&
Objects.equals(status, that.status);
}
@Override
public int hashCode()
{
return Objects.hash(task, status);
}
@Override
public String toString()
{
return "TaskstatusResponse{" +
"task='" + task + '\'' +
", status=" + status +
'}';
}
}

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