diff --git a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/FileFetcher.java b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/FileFetcher.java index 2bdf6b79bb60..284547c2532f 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/FileFetcher.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/FileFetcher.java @@ -20,17 +20,13 @@ package org.apache.druid.data.input.impl.prefetch; import com.google.common.base.Predicate; -import org.apache.commons.io.IOUtils; -import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.StringUtils; import javax.annotation.Nullable; import java.io.Closeable; import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.List; import java.util.concurrent.ExecutorService; @@ -39,7 +35,6 @@ * See the javadoc of {@link PrefetchableTextFilesFirehoseFactory} for more details. */ public class FileFetcher extends Fetcher - { private static final int BUFFER_SIZE = 1024 * 4; private final ObjectOpenFunction openObjectFunction; @@ -48,7 +43,7 @@ public class FileFetcher extends Fetcher // maximum retry for fetching an object from the remote site private final int maxFetchRetry; - public int getMaxFetchRetry() + private int getMaxFetchRetry() { return maxFetchRetry; } @@ -61,7 +56,7 @@ public int getMaxFetchRetry() PrefetchConfig prefetchConfig, ObjectOpenFunction openObjectFunction, Predicate retryCondition, - Integer maxFetchRetries + int maxFetchRetries ) { @@ -91,23 +86,15 @@ public int getMaxFetchRetry() @Override protected long download(T object, File outFile) throws IOException { - try { - return RetryUtils.retry( - () -> { - try (final InputStream is = openObjectFunction.open(object); - final OutputStream os = new FileOutputStream(outFile)) { - return IOUtils.copyLarge(is, os, buffer); - } - }, - retryCondition, - outFile::delete, - maxFetchRetry + 1, - StringUtils.format("Failed to download object[%s]", object) - ); - } - catch (Exception e) { - throw new IOException(e); - } + return FileUtils.copyLarge( + object, + openObjectFunction, + outFile, + buffer, + retryCondition, + maxFetchRetry + 1, + StringUtils.format("Failed to download object[%s]", object) + ); } /** diff --git a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java index 6025ad6cf73f..e0d8898af8a2 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/FileUtils.java @@ -23,13 +23,17 @@ import com.google.common.collect.ImmutableList; import com.google.common.io.ByteSource; import com.google.common.io.Files; +import org.apache.commons.io.IOUtils; +import org.apache.druid.data.input.impl.prefetch.ObjectOpenFunction; import org.apache.druid.java.util.common.logger.Logger; import java.io.Closeable; import java.io.File; import java.io.FileNotFoundException; +import java.io.FileOutputStream; import java.io.FilterOutputStream; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.nio.MappedByteBuffer; import java.nio.channels.Channels; @@ -258,6 +262,50 @@ public void close() }; } + /** + * Copies data from the InputStream opened with objectOpenFunction to the given file. + * This method is supposed to be used for copying large files. + * The output file is deleted automatically if copy fails. + * + * @param object object to open + * @param objectOpenFunction function to open the given object + * @param outFile file to write data + * @param fetchBuffer a buffer to copy data from the input stream to the file + * @param retryCondition condition which should be satisfied for retry + * @param numRetries max number of retries + * @param messageOnRetry log message on retry + * + * @return the number of bytes copied + */ + public static long copyLarge( + T object, + ObjectOpenFunction objectOpenFunction, + File outFile, + byte[] fetchBuffer, + Predicate retryCondition, + int numRetries, + String messageOnRetry + ) throws IOException + { + try { + return RetryUtils.retry( + () -> { + try (InputStream inputStream = objectOpenFunction.open(object); + OutputStream out = new FileOutputStream(outFile)) { + return IOUtils.copyLarge(inputStream, out, fetchBuffer); + } + }, + retryCondition, + outFile::delete, + numRetries, + messageOnRetry + ); + } + catch (Exception e) { + throw new IOException(e); + } + } + public interface OutputStreamConsumer { T apply(OutputStream outputStream) throws IOException; diff --git a/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java b/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java index 8eb67196cc0f..b4520070fe9f 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java @@ -166,7 +166,7 @@ public static String toUpperCase(String s) * @return application/x-www-form-urlencoded format encoded String, but with "+" replaced with "%20". */ @Nullable - public static String urlEncode(String s) + public static String urlEncode(@Nullable String s) { if (s == null) { return null; diff --git a/docs/content/ingestion/hadoop-vs-native-batch.md b/docs/content/ingestion/hadoop-vs-native-batch.md index cde88e554fae..b90eb0043ce8 100644 --- a/docs/content/ingestion/hadoop-vs-native-batch.md +++ b/docs/content/ingestion/hadoop-vs-native-batch.md @@ -32,11 +32,11 @@ ingestion method. | |Hadoop-based ingestion|Native parallel ingestion|Native local ingestion| |---|----------------------|-------------------------|----------------------| -| Parallel indexing | Always parallel | Parallel if firehose is splittable | Always sequential | -| Supported indexing modes | Replacing mode | Both appending and replacing modes | Both appending and replacing modes | +| Parallel indexing | Always parallel | Parallel if firehose is splittable
& maxNumSubTasks > 1 in tuningConfig | Always sequential | +| Supported indexing modes | Overwriting mode | Both appending and overwriting modes | Both appending and overwriting modes | | External dependency | Hadoop (it internally submits Hadoop jobs) | No dependency | No dependency | -| Supported [rollup modes](./index.html#roll-up-modes) | Perfect rollup | Best-effort rollup | Both perfect and best-effort rollup | -| Supported partitioning methods | [Both Hash-based and range partitioning](./hadoop.html#partitioning-specification) | N/A | Hash-based partitioning (when `forceGuaranteedRollup` = true) | +| Supported [rollup modes](./index.html#roll-up-modes) | Perfect rollup | Both perfect and best-effort rollup | Both perfect and best-effort rollup | +| Supported partitioning methods | [Both Hash-based and range partitioning](./hadoop.html#partitioning-specification) | Hash-based partitioning (when `forceGuaranteedRollup` = true) | Hash-based partitioning (when `forceGuaranteedRollup` = true) | | Supported input locations | All locations accessible via HDFS client or Druid dataSource | All implemented [firehoses](./firehose.html) | All implemented [firehoses](./firehose.html) | | Supported file formats | All implemented Hadoop InputFormats | Currently text file formats (CSV, TSV, JSON) by default. Additional formats can be added though a [custom extension](../development/modules.html) implementing [`FiniteFirehoseFactory`](https://github.com/apache/incubator-druid/blob/master/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java) | Currently text file formats (CSV, TSV, JSON) by default. Additional formats can be added though a [custom extension](../development/modules.html) implementing [`FiniteFirehoseFactory`](https://github.com/apache/incubator-druid/blob/master/core/src/main/java/org/apache/druid/data/input/FiniteFirehoseFactory.java) | | Saving parse exceptions in ingestion report | Currently not supported | Currently not supported | Supported | diff --git a/docs/content/ingestion/native_tasks.md b/docs/content/ingestion/native_tasks.md index 76dfd2065f03..9d3b2caee458 100644 --- a/docs/content/ingestion/native_tasks.md +++ b/docs/content/ingestion/native_tasks.md @@ -34,35 +34,50 @@ To run either kind of native batch indexing task, write an ingestion spec as spe [`/druid/indexer/v1/task` endpoint on the Overlord](../operations/api-reference.html#tasks), or use the `post-index-task` script included with Druid. 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), [`IngestSegmentFirehose`](./firehose.html#ingestsegmentfirehose), [`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 +doesn't depend on other external systems like Hadoop. `index_parallel` task is a supervisor task which basically generates +multiple worker tasks and submits them to the Overlord. Each worker task reads input data and creates segments. Once they +successfully generate segments for all input data, they report the generated segment list to the supervisor task. +The supervisor task periodically checks the status of worker tasks. If one of them fails, it retries the failed task +until the number of retries reaches the configured limit. If all worker tasks succeed, then it publishes the reported segments at once. + +The parallel Index Task can run in two different modes depending on `forceGuaranteedRollup` in `tuningConfig`. +If `forceGuaranteedRollup` = false, it's executed in a single phase. In this mode, +each sub task creates segments individually and reports them to the supervisor task. + +If `forceGuaranteedRollup` = true, it's executed in two phases with data shuffle which is similar to [MapReduce](https://en.wikipedia.org/wiki/MapReduce). +In the first phase, each sub task partitions input data based on `segmentGranularity` (primary partition key) in `granularitySpec` +and `partitionDimensions` (secondary partition key) in `partitionsSpec`. The partitioned data is served by +the [middleManager](../design/middlemanager.html) or the [indexer](../development/indexer.html) +where the first phase tasks ran. In the second phase, each sub task fetches +partitioned data from middleManagers or indexers and merges them to create the final segments. +As in the single phase execution, the created segments are reported to the supervisor task to publish at once. + +To use this task, the `firehose` in `ioConfig` should be _splittable_ and `maxNumSubTasks` should be set something larger than 1 in `tuningConfig`. +Otherwise, this task runs sequentially. Here is the list of currently splittable fireshoses. + +- [`LocalFirehose`](./firehose.html#localfirehose) +- [`IngestSegmentFirehose`](./firehose.html#ingestsegmentfirehose) +- [`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) +- [`StaticCloudFilesFirehose`](../development/extensions-contrib/cloudfiles.html#staticcloudfilesfirehose) + +The splittable firehose is responsible for generating _splits_. The supervisor task generates _worker task specs_ containing 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. -You may want to consider the below points: +You may want to consider the below things: -- Since this task doesn't shuffle intermediate data, it isn't available for [perfect rollup](../ingestion/index.html#roll-up-modes). -- The number of tasks for parallel ingestion is decided by `maxNumSubTasks` in the tuningConfig. - Since the supervisor task creates up to `maxNumSubTasks` worker tasks regardless of the available task slots, - it may affect to other ingestion performance. As a result, it's important to set `maxNumSubTasks` properly. - See the below [Capacity Planning](#capacity-planning) section for more details. +- The number of concurrent tasks run in parallel ingestion is determined by `maxNumSubTasks` in the `tuningConfig`. + The supervisor task checks the number of current running sub tasks and creates more if it's smaller than `maxNumSubTasks` no matter how many task slots are currently available. + This may affect to other ingestion performance. See the below [Capacity Planning](#capacity-planning) section for more details. - By default, batch ingestion replaces all data in any segment that it writes to. If you'd like to add to the segment - instead, set the appendToExisting flag in ioConfig. Note that it only replaces data in segments where it actively adds + instead, set the `appendToExisting` flag in `ioConfig`. Note that it only replaces data in segments where it actively adds data: if there are segments in your granularitySpec's intervals that have no data written by this task, they will be left alone. @@ -182,15 +197,18 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |type|The task type, this should always be `index_parallel`.|none|yes| |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| -|partitionsSpec|Defines how to partition the segments in timeChunk, see [PartitionsSpec](#partitionsspec)|`dynamic_partitions`|no| +|partitionsSpec|Defines how to partition the segments in a timeChunk, see [PartitionsSpec](#partitionsspec)|`dynamic` if `forceGuaranteedRollup` = false, `hashed` if `forceGuaranteedRollup` = true|no| |indexSpec|Defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| |indexSpecForIntermediatePersists|Defines segment storage format options to be used at indexing time for intermediate persisted temporary segments. this can be used to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. however, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published, see [IndexSpec](#indexspec) for possible values.|same as indexSpec|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| +|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../ingestion/index.html#roll-up-modes). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. If this is set to true, `numShards` in `tuningConfig` and `intervals` in `granularitySpec` must be set. Note that the result segments would be hash-partitioned. This flag cannot be used with `appendToExisting` of IOConfig. 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 [SegmentWriteOutMediumFactory](#segmentWriteOutMediumFactory).|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type` is used|no| -|maxNumSubTasks|Maximum number of tasks which can be run at the same time. The supervisor task would spawn worker tasks up to `maxNumSubTasks` regardless of the available task slots. If this value is set to 1, the supervisor task processes data ingestion on its own instead of spawning worker tasks. If this value is set to too large, too many worker tasks can be created which might block other ingestion. Check [Capacity Planning](#capacity-planning) for more details.|1|no| +|maxNumSubTasks|Maximum number of tasks which can be run at the same time. The supervisor task would spawn worker tasks up to `maxNumSubTasks` regardless of the current available task slots. If this value is set to 1, the supervisor task processes data ingestion on its own instead of spawning worker tasks. If this value is set to too large, too many worker tasks can be created which might block other ingestion. Check [Capacity Planning](#capacity-planning) for more details.|1|no| |maxRetry|Maximum number of retries on task failures.|3|no| +|maxNumSegmentsToMerge|Max limit for the number of segments that a single task can merge at the same time in the second phase. Used only `forceGuaranteedRollup` is set.|100|no| +|totalNumMergeTasks|Total number of tasks to merge segments in the second phase when `forceGuaranteedRollup` is set.|10|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| @@ -198,12 +216,21 @@ The tuningConfig is optional and default parameters will be used if no tuningCon #### PartitionsSpec PartitionsSpec is to describe the secondary partitioning method. -Parallel Index Task supports only the best-effort rollup mode, -and thus `dynamic_partitions` is only available option currently. +You should use different partitionsSpec depending on the [rollup mode](../ingestion/index.html#roll-up-modes) you want. +For perfect rollup, you should use `hashed`. + +|property|description|default|required?| +|--------|-----------|-------|---------| +|type|This should always be `hashed`|none|yes| +|maxRowsPerSegment|Used in sharding. Determines how many rows are in each segment.|5000000|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 maxRowsPerSegment is set.|null|no| +|partitionDimensions|The dimensions to partition on. Leave blank to select all dimensions.|null|no| + +For best-effort rollup, you should use `dynamic`. |property|description|default|required?| |--------|-----------|-------|---------| -|type|This should always be `dynamic_partitions`|none|yes| +|type|This should always be `dynamic`|none|yes| |maxRowsPerSegment|Used in sharding. Determines how many rows are in each segment.|5000000|no| |maxTotalRows|Total number of rows in segments waiting for being pushed. Used in determining when intermediate segment push should occur.|20000000|no| @@ -215,6 +242,10 @@ The supervisor task provides some HTTP endpoints to get running status. 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}/phase` + +Returns the name of the current phase if the task running in the parallel mode. + * `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. @@ -570,11 +601,11 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |type|The task type, this should always be "index".|none|yes| |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| -|partitionsSpec|Defines how to partition the segments in timeChunk, see [PartitionsSpec](#partitionsspec)|`dynamic_partitions` if `forceGuaranteedRollup` = false, `hashed_partitions` if `forceGuaranteedRollup` = true|no| +|partitionsSpec|Defines how to partition the segments in a timeChunk, see [PartitionsSpec](#partitionsspec)|`dynamic` if `forceGuaranteedRollup` = false, `hashed` if `forceGuaranteedRollup` = true|no| |indexSpec|Defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| |indexSpecForIntermediatePersists|Defines segment storage format options to be used at indexing time for intermediate persisted temporary segments. this can be used to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. however, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published, see [IndexSpec](#indexspec) for possible values.|same as indexSpec|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| -|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../ingestion/index.html#roll-up-modes). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. If this is set to true, the index task will read the entire input data twice: one for finding the optimal number of partitions per time chunk and one for generating segments. Note that the result segments would be hash-partitioned. You can set `forceExtendableShardSpecs` if you plan to append more data to the same time range in the future. This flag cannot be used with `appendToExisting` of IOConfig. For more details, see the below __Segment pushing modes__ section.|false|no| +|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../ingestion/index.html#roll-up-modes). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. If this is set to true, the index task will read the entire input data twice: one for finding the optimal number of partitions per time chunk and one for generating segments. Note that the result segments would be hash-partitioned. This flag cannot be used with `appendToExisting` of IOConfig. For more details, see the below __Segment pushing modes__ section.|false|no| |reportParseExceptions|DEPRECATED. If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped. Setting `reportParseExceptions` to true will override existing configurations for `maxParseExceptions` and `maxSavedParseExceptions`, setting `maxParseExceptions` to 0 and limiting `maxSavedParseExceptions` to no more than 1.|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 [SegmentWriteOutMediumFactory](#segmentWriteOutMediumFactory).|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory.type` is used|no| @@ -586,20 +617,20 @@ The tuningConfig is optional and default parameters will be used if no tuningCon PartitionsSpec is to describe the secondary partitioning method. You should use different partitionsSpec depending on the [rollup mode](../ingestion/index.html#roll-up-modes) you want. -For perfect rollup, you should use `hashed_partitions`. +For perfect rollup, you should use `hashed`. |property|description|default|required?| |--------|-----------|-------|---------| -|type|This should always be `hashed_partitions`|none|yes| +|type|This should always be `hashed`|none|yes| |maxRowsPerSegment|Used in sharding. Determines how many rows are in each segment.|5000000|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 maxRowsPerSegment is set.|null|no| |partitionDimensions|The dimensions to partition on. Leave blank to select all dimensions.|null|no| -For best-effort rollup, you should use `dynamic_partitions`. +For best-effort rollup, you should use `dynamic`. |property|description|default|required?| |--------|-----------|-------|---------| -|type|This should always be `dynamic_partitions`|none|yes| +|type|This should always be `dynamic`|none|yes| |maxRowsPerSegment|Used in sharding. Determines how many rows are in each segment.|5000000|no| |maxTotalRows|Total number of rows in segments waiting for being pushed. Used in determining when intermediate segment push should occur.|20000000|no| diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index b5f3c6258f4d..bc019dec0ad3 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2733,7 +2733,8 @@ public void close() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), - new SingleFileTaskReportFileWriter(reportsFile) + new SingleFileTaskReportFileWriter(reportsFile), + null ); } diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index fdd92b278f9c..59fe45a5928b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -2913,7 +2913,8 @@ public void close() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), - new SingleFileTaskReportFileWriter(reportsFile) + new SingleFileTaskReportFileWriter(reportsFile), + null ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index eea396513eaf..3287925104b5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -36,6 +36,7 @@ import org.apache.druid.indexing.common.actions.SegmentInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.worker.IntermediaryDataManager; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.MonitorScheduler; import org.apache.druid.query.QueryRunnerFactoryConglomerate; @@ -99,6 +100,7 @@ public class TaskToolbox private final DruidNode druidNode; private final LookupNodeService lookupNodeService; private final DataNodeService dataNodeService; + private final IntermediaryDataManager intermediaryDataManager; public TaskToolbox( TaskConfig config, @@ -127,7 +129,8 @@ public TaskToolbox( DruidNode druidNode, LookupNodeService lookupNodeService, DataNodeService dataNodeService, - TaskReportFileWriter taskReportFileWriter + TaskReportFileWriter taskReportFileWriter, + IntermediaryDataManager intermediaryDataManager ) { this.config = config; @@ -158,6 +161,7 @@ public TaskToolbox( this.dataNodeService = dataNodeService; this.taskReportFileWriter = taskReportFileWriter; this.taskReportFileWriter.setObjectMapper(this.objectMapper); + this.intermediaryDataManager = intermediaryDataManager; } public TaskConfig getConfig() @@ -329,4 +333,9 @@ public TaskReportFileWriter getTaskReportFileWriter() { return taskReportFileWriter; } + + public IntermediaryDataManager getIntermediaryDataManager() + { + return intermediaryDataManager; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 83c958893d3d..830243b625bd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -35,6 +35,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.worker.IntermediaryDataManager; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.MonitorScheduler; import org.apache.druid.query.QueryRunnerFactoryConglomerate; @@ -83,6 +84,7 @@ public class TaskToolboxFactory private final LookupNodeService lookupNodeService; private final DataNodeService dataNodeService; private final TaskReportFileWriter taskReportFileWriter; + private final IntermediaryDataManager intermediaryDataManager; @Inject public TaskToolboxFactory( @@ -111,7 +113,8 @@ public TaskToolboxFactory( @RemoteChatHandler DruidNode druidNode, LookupNodeService lookupNodeService, DataNodeService dataNodeService, - TaskReportFileWriter taskReportFileWriter + TaskReportFileWriter taskReportFileWriter, + IntermediaryDataManager intermediaryDataManager ) { this.config = config; @@ -140,6 +143,7 @@ public TaskToolboxFactory( this.lookupNodeService = lookupNodeService; this.dataNodeService = dataNodeService; this.taskReportFileWriter = taskReportFileWriter; + this.intermediaryDataManager = intermediaryDataManager; } public TaskToolbox build(Task task) @@ -172,7 +176,8 @@ public TaskToolbox build(Task task) druidNode, lookupNodeService, dataNodeService, - taskReportFileWriter + taskReportFileWriter, + intermediaryDataManager ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index 3a96070f064d..ac00d60637f3 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -254,6 +254,7 @@ public int getMaxSavedParseExceptions() return maxSavedParseExceptions; } + @Override public RealtimeAppenderatorTuningConfig withBasePersistDirectory(File dir) { return new RealtimeAppenderatorTuningConfig( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index 56dccbe83515..c55d05ec0621 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -23,6 +23,8 @@ import com.google.common.collect.Iterables; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; @@ -31,9 +33,12 @@ import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; +import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory; import org.apache.druid.indexing.firehose.WindowedSegmentId; import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.logger.Logger; @@ -41,19 +46,24 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; import org.apache.druid.timeline.partition.PartitionChunk; +import org.apache.druid.timeline.partition.ShardSpecFactory; import org.joda.time.Interval; import org.joda.time.Period; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.SortedSet; import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -100,7 +110,7 @@ protected AbstractBatchIndexTask( } /** - * Run this task. Before running the task, it checks the the current task is already stopped and + * Run this task. Before running the task, it checks the current task is already stopped and * registers a cleaner to interrupt the thread running this task on abnormal exits. * * @see #runTask(TaskToolbox) @@ -288,7 +298,7 @@ private boolean tryLockWithDetermineResult(TaskActionClient client, LockGranular } } - private boolean tryTimeChunkLock(TaskActionClient client, List intervals) throws IOException + protected boolean tryTimeChunkLock(TaskActionClient client, List intervals) throws IOException { // In this case, the intervals to lock must be aligned with segmentGranularity if it's defined final Set uniqueIntervals = new HashSet<>(); @@ -360,6 +370,30 @@ private LockGranularityDetermineResult determineSegmentGranularity(List createShardSpecFactoryForGuaranteedRollup( + int numShards, + @Nullable List partitionDimensions + ) + { + return Pair.of(new HashBasedNumberedShardSpecFactory(partitionDimensions, numShards), numShards); + } + @Nullable static Granularity findGranularityFromSegments(List segments) { @@ -377,6 +411,45 @@ static Granularity findGranularityFromSegments(List segments) } } + /** + * Creates shard specs based on the given configurations. The return value is a map between intervals created + * based on the segment granularity and the shard specs to be created. + * Note that the shard specs to be created is a pair of {@link ShardSpecFactory} and number of segments per interval + * and filled only when {@link #isGuaranteedRollup} = true. Otherwise, the return value contains only the set of + * intervals generated based on the segment granularity. + */ + protected static Map> createShardSpecWithoutInputScan( + GranularitySpec granularitySpec, + IndexIOConfig ioConfig, + IndexTuningConfig tuningConfig, + @Nonnull PartitionsSpec nonNullPartitionsSpec + ) + { + final Map> allocateSpec = new HashMap<>(); + final SortedSet intervals = granularitySpec.bucketIntervals().get(); + + if (isGuaranteedRollup(ioConfig, tuningConfig)) { + // SingleDimensionPartitionsSpec or more partitionsSpec types will be supported in the future. + assert nonNullPartitionsSpec instanceof HashedPartitionsSpec; + // Overwrite mode, guaranteed rollup: shardSpecs must be known in advance. + final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) nonNullPartitionsSpec; + final int numShards = partitionsSpec.getNumShards() == null ? 1 : partitionsSpec.getNumShards(); + + for (Interval interval : intervals) { + allocateSpec.put( + interval, + createShardSpecFactoryForGuaranteedRollup(numShards, partitionsSpec.getPartitionDimensions()) + ); + } + } else { + for (Interval interval : intervals) { + allocateSpec.put(interval, null); + } + } + + return allocateSpec; + } + /** * If the given firehoseFactory is {@link IngestSegmentFirehoseFactory}, then it finds the segments to lock * from the firehoseFactory. This is because those segments will be read by this task no matter what segments would be diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java new file mode 100644 index 000000000000..13b9701e8a06 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; +import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; + +public final class BatchAppenderators +{ + public static Appenderator newAppenderator( + String taskId, + AppenderatorsManager appenderatorsManager, + FireDepartmentMetrics metrics, + TaskToolbox toolbox, + DataSchema dataSchema, + AppenderatorConfig appenderatorConfig + ) + { + return newAppenderator( + taskId, + appenderatorsManager, + metrics, + toolbox, + dataSchema, + appenderatorConfig, + toolbox.getSegmentPusher() + ); + } + + public static Appenderator newAppenderator( + String taskId, + AppenderatorsManager appenderatorsManager, + FireDepartmentMetrics metrics, + TaskToolbox toolbox, + DataSchema dataSchema, + AppenderatorConfig appenderatorConfig, + DataSegmentPusher segmentPusher + ) + { + return appenderatorsManager.createOfflineAppenderatorForTask( + taskId, + dataSchema, + appenderatorConfig.withBasePersistDirectory(toolbox.getPersistDir()), + metrics, + segmentPusher, + toolbox.getObjectMapper(), + toolbox.getIndexIO(), + toolbox.getIndexMergerV9() + ); + } + + public static BatchAppenderatorDriver newDriver( + final Appenderator appenderator, + final TaskToolbox toolbox, + final SegmentAllocator segmentAllocator + ) + { + return new BatchAppenderatorDriver( + appenderator, + segmentAllocator, + new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()), + toolbox.getDataSegmentKiller() + ); + } + + private BatchAppenderators() + { + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java index 8d5680e6ed11..80082ddea21b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java @@ -46,7 +46,7 @@ /** * Allocates all necessary segments locally at the beginning and reuse them. */ -class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator +public class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator { private final TaskToolbox toolbox; private final String taskId; @@ -58,7 +58,7 @@ class CachingLocalSegmentAllocator implements IndexTaskSegmentAllocator // sequenceName -> segmentId private final Map sequenceNameToSegmentId; - CachingLocalSegmentAllocator( + public CachingLocalSegmentAllocator( TaskToolbox toolbox, String taskId, String dataSource, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 4e550f4a8711..c7fb2a18a24e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -100,7 +100,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.TreeMap; -import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.StreamSupport; @@ -110,12 +109,6 @@ public class CompactionTask extends AbstractBatchIndexTask private static final Logger log = new Logger(CompactionTask.class); private static final String TYPE = "compact"; - /** - * A flag to indicate this task is already stopped and its child indexTasks shouldn't be created. - * See {@link #currentRunningTaskSpec} for more details. - */ - private static final Object SPECIAL_VALUE_STOPPED = new Object(); - private final Interval interval; private final List segments; @Nullable @@ -156,19 +149,15 @@ public class CompactionTask extends AbstractBatchIndexTask private final AppenderatorsManager appenderatorsManager; @JsonIgnore - private List indexTaskSpecs; + private final CurrentSubTaskHolder currentSubTaskHolder = new CurrentSubTaskHolder( + (taskObject, config) -> { + final IndexTask indexTask = (IndexTask) taskObject; + indexTask.stopGracefully(config); + } + ); - /** - * Reference to the sub-task that is currently running. - * - * When {@link #stopGracefully} is called, the compaction task gets the reference to the current running task, - * and calls {@link #stopGracefully} for that task. This reference will be updated to {@link #SPECIAL_VALUE_STOPPED}. - * - * Note that {@link #stopGracefully} can be called at any time during {@link #run}. Calling {@link #stopGracefully} - * on the current running task and setting this reference to {@link #SPECIAL_VALUE_STOPPED} should be done atomically. - */ - @Nullable - private final AtomicReference currentRunningTaskSpec = new AtomicReference<>(); + @JsonIgnore + private List indexTaskSpecs; @JsonCreator public CompactionTask( @@ -345,25 +334,17 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception log.warn("Interval[%s] has no segments, nothing to do.", interval); return TaskStatus.failure(getId()); } else { + registerResourceCloserOnAbnormalExit(currentSubTaskHolder); final int totalNumSpecs = indexTaskSpecs.size(); log.info("Generated [%d] compaction task specs", totalNumSpecs); int failCnt = 0; - registerResourceCloserOnAbnormalExit(config -> { - Object currentRunningTask = currentRunningTaskSpec.getAndSet(SPECIAL_VALUE_STOPPED); - if (currentRunningTask != null) { - ((IndexTask) currentRunningTask).stopGracefully(config); - } - }); for (IndexTask eachSpec : indexTaskSpecs) { - Object prevSpec = currentRunningTaskSpec.get(); - //noinspection ObjectEquality - if (prevSpec == SPECIAL_VALUE_STOPPED || !currentRunningTaskSpec.compareAndSet(prevSpec, eachSpec)) { + final String json = jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(eachSpec); + if (!currentSubTaskHolder.setTask(eachSpec)) { log.info("Task is asked to stop. Finish as failed."); return TaskStatus.failure(getId()); } - final String json = jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(eachSpec); - try { if (eachSpec.isReady(toolbox.getTaskActionClient())) { log.info("Running indexSpec: " + json); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CurrentSubTaskHolder.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CurrentSubTaskHolder.java new file mode 100644 index 000000000000..5175dbca6e61 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CurrentSubTaskHolder.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import org.apache.druid.indexing.common.config.TaskConfig; +import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; + +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +/** + * Some task types, e.g., {@link CompactionTask} or {@link ParallelIndexSupervisorTask} create sub tasks and execute + * them one at a time. This class holds the sub task that is currently running whatever its type is. + * + * This holder is supposed to be registered to {@link TaskResourceCleaner} so that {@link #accept} method is called + * properly when {@link Task#stopGracefully} is called. + */ +public class CurrentSubTaskHolder implements Consumer +{ + /** + * A flag to indicate the {@link Task} of this holder is already stopped and its child tasks shouldn't be + * created. See {@link #currentSubTaskReference} for more details. + */ + private static final Object SPECIAL_VALUE_STOPPED = new Object(); + + /** + * Reference to the sub-task that is currently running. + * + * When {@link #accept} is called, this class gets the reference to the current running task, + * and calls {@link #cleanFunction} for that task. This reference will be updated to {@link #SPECIAL_VALUE_STOPPED}. + */ + private final AtomicReference currentSubTaskReference = new AtomicReference<>(); + + private final BiConsumer cleanFunction; + + public CurrentSubTaskHolder(BiConsumer subTaskCleaner) + { + this.cleanFunction = subTaskCleaner; + } + + public boolean setTask(Object subTask) + { + final Object prevSpec = currentSubTaskReference.get(); + + //noinspection ObjectEquality + return prevSpec != SPECIAL_VALUE_STOPPED && currentSubTaskReference.compareAndSet(prevSpec, subTask); + } + + public T getTask() + { + //noinspection unchecked + return (T) currentSubTaskReference.get(); + } + + @Override + public void accept(TaskConfig taskConfig) + { + Object currentSubTask = currentSubTaskReference.getAndSet(SPECIAL_VALUE_STOPPED); + if (currentSubTask != null) { + cleanFunction.accept(currentSubTask, taskConfig); + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FiniteFirehoseProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FiniteFirehoseProcessor.java new file mode 100644 index 000000000000..98c447100fef --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/FiniteFirehoseProcessor.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import com.google.common.base.Optional; +import org.apache.druid.data.input.Firehose; +import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.indexing.common.stats.RowIngestionMeters; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; +import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; +import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import org.apache.druid.utils.CircularBuffer; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; + +public class FiniteFirehoseProcessor +{ + private static final Logger LOG = new Logger(FiniteFirehoseProcessor.class); + + private final RowIngestionMeters buildSegmentsMeters; + @Nullable + private final CircularBuffer buildSegmentsSavedParseExceptions; + private final boolean logParseExceptions; + private final int maxParseExceptions; + private final long pushTimeout; + + public FiniteFirehoseProcessor( + RowIngestionMeters buildSegmentsMeters, + @Nullable CircularBuffer buildSegmentsSavedParseExceptions, + boolean logParseExceptions, + int maxParseExceptions, + long pushTimeout + ) + { + this.buildSegmentsMeters = buildSegmentsMeters; + this.buildSegmentsSavedParseExceptions = buildSegmentsSavedParseExceptions; + this.logParseExceptions = logParseExceptions; + this.maxParseExceptions = maxParseExceptions; + this.pushTimeout = pushTimeout; + } + + /** + * This method connects the given {@link FirehoseFactory} and processes data from the connected {@link Firehose}. + * All read data is consumed by {@link BatchAppenderatorDriver} which creates new segments. + * All created segments are pushed when all input data is processed successfully. + * + * @return {@link SegmentsAndMetadata} for the pushed segments. + */ + public SegmentsAndMetadata process( + DataSchema dataSchema, + BatchAppenderatorDriver driver, + PartitionsSpec partitionsSpec, + FirehoseFactory firehoseFactory, + File firehoseTempDir, + IndexTaskSegmentAllocator segmentAllocator + ) throws IOException, InterruptedException, ExecutionException, TimeoutException + { + @Nullable + final DynamicPartitionsSpec dynamicPartitionsSpec = partitionsSpec instanceof DynamicPartitionsSpec + ? (DynamicPartitionsSpec) partitionsSpec + : null; + final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); + try ( + final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) + ) { + while (firehose.hasMore()) { + try { + final InputRow inputRow = firehose.nextRow(); + + if (inputRow == null) { + buildSegmentsMeters.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); + } + + final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); + if (!optInterval.isPresent()) { + buildSegmentsMeters.incrementThrownAway(); + continue; + } + + final Interval interval = optInterval.get(); + final String sequenceName = segmentAllocator.getSequenceName(interval, inputRow); + final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName); + + if (addResult.isOk()) { + + // incremental segment publishment is allowed only when rollup doesn't have to be perfect. + if (dynamicPartitionsSpec != null) { + final boolean isPushRequired = addResult.isPushRequired( + dynamicPartitionsSpec.getMaxRowsPerSegment(), + dynamicPartitionsSpec.getMaxTotalRows() + ); + if (isPushRequired) { + // There can be some segments waiting for being pushed even though no more rows will be added to them + // in the future. + // If those segments are not pushed here, the remaining available space in appenderator will be kept + // small which could lead to smaller segments. + final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout); + LOG.info("Pushed segments[%s]", pushed.getSegments()); + } + } + } else { + throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp()); + } + + if (addResult.getParseException() != null) { + handleParseException(addResult.getParseException()); + } else { + buildSegmentsMeters.incrementProcessed(); + } + } + catch (ParseException e) { + handleParseException(e); + } + } + + final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout); + LOG.info("Pushed segments[%s]", pushed.getSegments()); + return pushed; + } + } + + private void handleParseException(ParseException e) + { + if (e.isFromPartiallyValidRow()) { + buildSegmentsMeters.incrementProcessedWithError(); + } else { + buildSegmentsMeters.incrementUnparseable(); + } + + if (logParseExceptions) { + LOG.error(e, "Encountered parse exception:"); + } + + if (buildSegmentsSavedParseExceptions != null) { + buildSegmentsSavedParseExceptions.add(e); + } + + if (buildSegmentsMeters.getUnparseable() + buildSegmentsMeters.getProcessedWithError() > maxParseExceptions) { + LOG.error("Max parse exceptions exceeded, terminating task..."); + throw new RuntimeException("Max parse exceptions exceeded, terminating task...", e); + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index c314cef7f844..f8a175db068a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -42,7 +42,6 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; import org.apache.druid.indexing.common.LockGranularity; @@ -75,11 +74,9 @@ import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.appenderator.Appenderator; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; -import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; -import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.segment.realtime.firehose.ChatHandler; @@ -89,7 +86,6 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.ShardSpecFactory; @@ -116,7 +112,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.SortedSet; import java.util.TreeMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -163,13 +158,10 @@ private static String makeGroupId(boolean isAppendToExisting, String dataSource) private final RowIngestionMeters buildSegmentsMeters; @JsonIgnore - private FireDepartmentMetrics buildSegmentsFireDepartmentMetrics; + private final CircularBuffer buildSegmentsSavedParseExceptions; @JsonIgnore - private CircularBuffer buildSegmentsSavedParseExceptions; - - @JsonIgnore - private CircularBuffer determinePartitionsSavedParseExceptions; + private final CircularBuffer determinePartitionsSavedParseExceptions; @JsonIgnore private String errorMsg; @@ -234,6 +226,9 @@ public IndexTask( buildSegmentsSavedParseExceptions = new CircularBuffer<>( ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() ); + } else { + determinePartitionsSavedParseExceptions = null; + buildSegmentsSavedParseExceptions = null; } this.ingestionState = IngestionState.NOT_STARTED; this.determinePartitionsMeters = rowIngestionMetersFactory.createRowIngestionMeters(); @@ -542,15 +537,6 @@ private Map getTaskCompletionRowStats() return metrics; } - private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig) - { - Preconditions.checkState( - !tuningConfig.isForceGuaranteedRollup() || !ioConfig.isAppendToExisting(), - "Perfect rollup cannot be guaranteed when appending to existing dataSources" - ); - return tuningConfig.isForceGuaranteedRollup(); - } - /** * Determines intervals and shardSpecs for input data. This method first checks that it must determine intervals and * shardSpecs by itself. Intervals must be determined if they are not specified in {@link GranularitySpec}. @@ -609,37 +595,6 @@ private Map> determineShardSpecs( } } - private Map> createShardSpecWithoutInputScan( - GranularitySpec granularitySpec, - IndexIOConfig ioConfig, - IndexTuningConfig tuningConfig, - PartitionsSpec nonNullPartitionsSpec - ) - { - final Map> allocateSpec = new HashMap<>(); - final SortedSet intervals = granularitySpec.bucketIntervals().get(); - - if (isGuaranteedRollup(ioConfig, tuningConfig)) { - // Overwrite mode, guaranteed rollup: shardSpecs must be known in advance. - assert nonNullPartitionsSpec instanceof HashedPartitionsSpec; - final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) nonNullPartitionsSpec; - final int numShards = partitionsSpec.getNumShards() == null ? 1 : partitionsSpec.getNumShards(); - - for (Interval interval : intervals) { - allocateSpec.put( - interval, - createShardSpecFactoryForGuaranteedRollup(numShards, partitionsSpec.getPartitionDimensions()) - ); - } - } else { - for (Interval interval : intervals) { - allocateSpec.put(interval, null); - } - } - - return allocateSpec; - } - private Map> createShardSpecsFromInput( ObjectMapper jsonMapper, IndexIngestionSpec ingestionSchema, @@ -691,7 +646,7 @@ private Map> createShardSpecsFromInput log.info("Creating [%,d] shards for interval [%s]", numShards, interval); } - // Overwrite mode, guaranteed rollup: # of shards must be known in advance. + // Overwrite mode, guaranteed rollup: shardSpecs must be known in advance. allocateSpecs.put( interval, createShardSpecFactoryForGuaranteedRollup(numShards, partitionsSpec.getPartitionDimensions()) @@ -705,14 +660,6 @@ private Map> createShardSpecsFromInput return allocateSpecs; } - private Pair createShardSpecFactoryForGuaranteedRollup( - int numShards, - @Nullable List partitionDimensions - ) - { - return Pair.of(new HashBasedNumberedShardSpecFactory(partitionDimensions, numShards), numShards); - } - private Map> collectIntervalsAndShardSpecs( ObjectMapper jsonMapper, IndexIngestionSpec ingestionSchema, @@ -859,14 +806,9 @@ private TaskStatus generateAndPublishSegments( final PartitionsSpec partitionsSpec ) throws IOException, InterruptedException { - @Nullable - final DynamicPartitionsSpec dynamicPartitionsSpec = partitionsSpec instanceof DynamicPartitionsSpec - ? (DynamicPartitionsSpec) partitionsSpec - : null; - final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); final FireDepartment fireDepartmentForMetrics = new FireDepartment(dataSchema, new RealtimeIOConfig(null, null), null); - buildSegmentsFireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + FireDepartmentMetrics buildSegmentsFireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); if (toolbox.getMonitorScheduler() != null) { final TaskRealtimeMetricsMonitor metricsMonitor = TaskRealtimeMetricsMonitorBuilder.build( @@ -890,80 +832,33 @@ private TaskStatus generateAndPublishSegments( toolbox.getTaskActionClient() .submit(SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToPublish)); - final Appenderator appenderator = newAppenderator( + final Appenderator appenderator = BatchAppenderators.newAppenderator( + getId(), + appenderatorsManager, buildSegmentsFireDepartmentMetrics, toolbox, dataSchema, tuningConfig ); boolean exceptionOccurred = false; - try ( - final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator); - final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) - ) { - + try (final BatchAppenderatorDriver driver = BatchAppenderators.newDriver(appenderator, toolbox, segmentAllocator)) { driver.startJob(); - while (firehose.hasMore()) { - try { - final InputRow inputRow = firehose.nextRow(); - - if (inputRow == null) { - buildSegmentsMeters.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); - } - - final Optional optInterval = granularitySpec.bucketInterval(inputRow.getTimestamp()); - if (!optInterval.isPresent()) { - buildSegmentsMeters.incrementThrownAway(); - continue; - } - - final Interval interval = optInterval.get(); - final String sequenceName = segmentAllocator.getSequenceName(interval, inputRow); - final AppenderatorDriverAddResult addResult = driver.add(inputRow, sequenceName); - - if (addResult.isOk()) { - - // incremental segment publishment is allowed only when rollup don't have to be perfect. - if (dynamicPartitionsSpec != null) { - final boolean isPushRequired = addResult.isPushRequired( - dynamicPartitionsSpec.getMaxRowsPerSegment(), - dynamicPartitionsSpec.getMaxTotalRows() - ); - if (isPushRequired) { - // 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); - log.info("Pushed segments[%s]", pushed.getSegments()); - } - } - } else { - throw new ISE("Failed to add a row with timestamp[%s]", inputRow.getTimestamp()); - } - - if (addResult.getParseException() != null) { - handleParseException(addResult.getParseException()); - } else { - buildSegmentsMeters.incrementProcessed(); - } - } - catch (ParseException e) { - handleParseException(e); - } - } - - final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout); - log.info("Pushed segments[%s]", pushed.getSegments()); + final FiniteFirehoseProcessor firehoseProcessor = new FiniteFirehoseProcessor( + buildSegmentsMeters, + buildSegmentsSavedParseExceptions, + tuningConfig.isLogParseExceptions(), + tuningConfig.getMaxParseExceptions(), + pushTimeout + ); + firehoseProcessor.process( + dataSchema, + driver, + partitionsSpec, + firehoseFactory, + firehoseTempDir, + segmentAllocator + ); // If we use timeChunk lock, then we don't have to specify what segments will be overwritten because // it will just overwrite all segments overlapped with the new segments. @@ -1013,29 +908,6 @@ private TaskStatus generateAndPublishSegments( } } - private void handleParseException(ParseException e) - { - if (e.isFromPartiallyValidRow()) { - buildSegmentsMeters.incrementProcessedWithError(); - } else { - buildSegmentsMeters.incrementUnparseable(); - } - - if (ingestionSchema.tuningConfig.isLogParseExceptions()) { - log.error(e, "Encountered parse exception:"); - } - - if (buildSegmentsSavedParseExceptions != null) { - buildSegmentsSavedParseExceptions.add(e); - } - - if (buildSegmentsMeters.getUnparseable() - + buildSegmentsMeters.getProcessedWithError() > ingestionSchema.tuningConfig.getMaxParseExceptions()) { - log.error("Max parse exceptions exceeded, terminating task..."); - throw new RuntimeException("Max parse exceptions exceeded, terminating task...", e); - } - } - private static SegmentsAndMetadata awaitPublish( ListenableFuture publishFuture, long publishTimeout @@ -1048,39 +920,6 @@ private static SegmentsAndMetadata awaitPublish( } } - private Appenderator newAppenderator( - FireDepartmentMetrics metrics, - TaskToolbox toolbox, - DataSchema dataSchema, - IndexTuningConfig tuningConfig - ) - { - return appenderatorsManager.createOfflineAppenderatorForTask( - getId(), - 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() - ); - } - /** * This class represents a map of (Interval, ShardSpec) and is used for easy shardSpec generation. */ @@ -1391,6 +1230,7 @@ private IndexTuningConfig( : logParseExceptions; } + @Override public IndexTuningConfig withBasePersistDirectory(File dir) { return new IndexTuningConfig( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java index d997675f5962..ed52dcb45526 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTaskSegmentAllocator.java @@ -26,7 +26,7 @@ /** * Segment allocator interface for {@link IndexTask}. It has 3 different modes for allocating segments. */ -interface IndexTaskSegmentAllocator extends SegmentAllocator +public interface IndexTaskSegmentAllocator extends SegmentAllocator { /** * SequenceName is the key to create the segmentId. If previousSegmentId is given, {@link SegmentAllocator} allocates diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 097bf7474ba8..58612ce523ed 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -25,8 +25,10 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.config.TaskConfig; -import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSubTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialSegmentGenerateTask; +import org.apache.druid.indexing.common.task.batch.parallel.PartialSegmentMergeTask; +import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; @@ -52,7 +54,10 @@ @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 = SinglePhaseSubTask.TYPE, value = SinglePhaseSubTask.class), + @JsonSubTypes.Type(name = "index_sub", value = SinglePhaseSubTask.class), // for backward compatibility + @JsonSubTypes.Type(name = PartialSegmentGenerateTask.TYPE, value = PartialSegmentGenerateTask.class), + @JsonSubTypes.Type(name = PartialSegmentMergeTask.TYPE, value = PartialSegmentMergeTask.class), @JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class), @JsonSubTypes.Type(name = "index_realtime", value = RealtimeIndexTask.class), @JsonSubTypes.Type(name = "index_realtime_appenderator", value = AppenderatorDriverRealtimeIndexTask.class), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReport.java new file mode 100644 index 000000000000..d1fd0000d13f --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedPartitionsReport.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Objects; + +/** + * Report containing the {@link PartitionStat}s created by a {@link PartialSegmentGenerateTask}. + * This report is collected by {@link ParallelIndexSupervisorTask} and + * used to generate {@link PartialSegmentMergeIOConfig}. + */ +public class GeneratedPartitionsReport implements SubTaskReport +{ + public static final String TYPE = "generated_partitions"; + + private final String taskId; + private final List partitionStats; + + @JsonCreator + public GeneratedPartitionsReport( + @JsonProperty("taskId") String taskId, + @JsonProperty("partitionStats") List partitionStats + ) + { + this.taskId = taskId; + this.partitionStats = partitionStats; + } + + @Override + @JsonProperty + public String getTaskId() + { + return taskId; + } + + @JsonProperty + public List getPartitionStats() + { + return partitionStats; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + GeneratedPartitionsReport that = (GeneratedPartitionsReport) o; + return Objects.equals(taskId, that.taskId) && + Objects.equals(partitionStats, that.partitionStats); + } + + @Override + public int hashCode() + { + return Objects.hash(taskId, partitionStats); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java index 3c07cd6540f4..2e4ea8df5d31 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexIOConfig.java @@ -19,6 +19,7 @@ package org.apache.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 org.apache.druid.data.input.FirehoseFactory; @@ -32,6 +33,7 @@ @JsonTypeName("index_parallel") public class ParallelIndexIOConfig extends IndexIOConfig { + @JsonCreator public ParallelIndexIOConfig( @JsonProperty("firehose") FirehoseFactory firehoseFactory, @JsonProperty("appendToExisting") @Nullable Boolean appendToExisting diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java new file mode 100644 index 000000000000..1c697ab072f0 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexPhaseRunner.java @@ -0,0 +1,471 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.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 org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.FiniteFirehoseFactory; +import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.TaskStatusPlus; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.MonitorEntry; +import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.logger.Logger; + +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.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** + * Base class for different implementations of {@link ParallelIndexTaskRunner}. + * It creates sub tasks, schedule them, and monitor their status. + */ +public abstract class ParallelIndexPhaseRunner + implements ParallelIndexTaskRunner +{ + private static final Logger LOG = new Logger(ParallelIndexPhaseRunner.class); + + private final TaskToolbox toolbox; + private final String taskId; + private final String groupId; + private final ParallelIndexTuningConfig tuningConfig; + private final Map context; + + /** + * Max number of sub tasks which can be executed concurrently at the same time. + */ + private final int maxNumConcurrentSubTasks; + private final IndexingServiceClient indexingServiceClient; + + private final BlockingQueue> taskCompleteEvents = new LinkedBlockingDeque<>(); + + // subTaskId -> report + private final ConcurrentHashMap reportsMap = new ConcurrentHashMap<>(); + + private volatile boolean subTaskScheduleAndMonitorStopped; + private volatile TaskMonitor taskMonitor; + + private int nextSpecId = 0; + + ParallelIndexPhaseRunner( + TaskToolbox toolbox, + String taskId, + String groupId, + ParallelIndexTuningConfig tuningConfig, + Map context, + IndexingServiceClient indexingServiceClient + ) + { + this.toolbox = toolbox; + this.taskId = taskId; + this.groupId = groupId; + this.tuningConfig = tuningConfig; + this.context = context; + this.maxNumConcurrentSubTasks = tuningConfig.getMaxNumSubTasks(); + this.indexingServiceClient = Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient"); + } + + /** + * Returns an iterator for {@link SubTaskSpec}s of this phase. + */ + abstract Iterator> subTaskSpecIterator() throws IOException; + + /** + * Returns the total number of sub tasks required to execute this phase. + */ + abstract int getTotalNumSubTasks() throws IOException; + + @Override + public TaskState run() throws Exception + { + if (getTotalNumSubTasks() == 0) { + LOG.warn("There's no input split to process"); + return TaskState.SUCCESS; + } + + final Iterator> subTaskSpecIterator = subTaskSpecIterator(); + final long taskStatusCheckingPeriod = tuningConfig.getTaskStatusCheckPeriodMs(); + + taskMonitor = new TaskMonitor<>( + Preconditions.checkNotNull(indexingServiceClient, "indexingServiceClient"), + tuningConfig.getMaxRetry(), + getTotalNumSubTasks() + ); + TaskState state = TaskState.RUNNING; + + taskMonitor.start(taskStatusCheckingPeriod); + + try { + LOG.info("Submitting initial tasks"); + // Submit initial tasks + while (isRunning() && subTaskSpecIterator.hasNext() && taskMonitor.getNumRunningTasks() < maxNumConcurrentSubTasks) { + submitNewTask(taskMonitor, subTaskSpecIterator.next()); + } + + LOG.info("Waiting for subTasks to be completed"); + while (isRunning()) { + final SubTaskCompleteEvent 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 (!reportsMap.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) { + subTaskScheduleAndMonitorStopped = true; + if (taskMonitor.isSucceeded()) { + // 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() < maxNumConcurrentSubTasks) { + // 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; + subTaskScheduleAndMonitorStopped = true; + final TaskStatusPlus lastStatus = taskCompleteEvent.getLastStatus(); + if (lastStatus != null) { + LOG.error("Failed because of the failed sub task[%s]", lastStatus.getId()); + } else { + final SinglePhaseSubTaskSpec spec = + (SinglePhaseSubTaskSpec) 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 { + stopInternal(); + if (!state.isComplete()) { + state = TaskState.FAILED; + } + } + + return state; + } + + private boolean isRunning() + { + return !subTaskScheduleAndMonitorStopped && !Thread.currentThread().isInterrupted(); + } + + private void submitNewTask( + TaskMonitor taskMonitor, + SubTaskSpec spec + ) + { + LOG.info("Submit a new task for spec[%s] and inputSplit[%s]", spec.getId(), spec.getInputSplit()); + final ListenableFuture> future = taskMonitor.submit(spec); + Futures.addCallback( + future, + new FutureCallback>() + { + @Override + public void onSuccess(SubTaskCompleteEvent completeEvent) + { + // this callback is called if a task completed whether 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)); + } + } + ); + } + + private static List 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()); + } + } + + @Override + public void stopGracefully() + { + subTaskScheduleAndMonitorStopped = true; + stopInternal(); + } + + /** + * Stop task scheduling and monitoring, and kill all running tasks. + * This method is thread-safe. + */ + private void stopInternal() + { + LOG.info("Cleaning up resources"); + + taskCompleteEvents.clear(); + if (taskMonitor != null) { + taskMonitor.stop(); + } + } + + @Override + public void collectReport(SubTaskReportType 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. + reportsMap.compute(report.getTaskId(), (taskId, prevReport) -> { + if (prevReport != null) { + Preconditions.checkState( + prevReport.equals(report), + "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 Map getReports() + { + return reportsMap; + } + + @Override + public ParallelIndexingProgress getProgress() + { + return taskMonitor == null ? SinglePhaseParallelIndexingProgress.notRunning() : taskMonitor.getProgress(); + } + + @Override + public Set getRunningTaskIds() + { + return taskMonitor == null ? Collections.emptySet() : taskMonitor.getRunningTaskIds(); + } + + @Override + public List> getSubTaskSpecs() + { + if (taskMonitor != null) { + final List> runningSubTaskSpecs = taskMonitor.getRunningSubTaskSpecs(); + final List> completeSubTaskSpecs = taskMonitor + .getCompleteSubTaskSpecs(); + // Deduplicate subTaskSpecs because some subTaskSpec might exist both in runningSubTaskSpecs and + // completeSubTaskSpecs. + final Map> 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> getRunningSubTaskSpecs() + { + return taskMonitor == null ? Collections.emptyList() : taskMonitor.getRunningSubTaskSpecs(); + } + + @Override + public List> getCompleteSubTaskSpecs() + { + return taskMonitor == null ? Collections.emptyList() : taskMonitor.getCompleteSubTaskSpecs(); + } + + @Nullable + @Override + public SubTaskSpec 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 taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId); + final SubTaskSpec 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 taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId); + + final SubTaskSpecStatus subTaskSpecStatus; + + if (monitorEntry != null) { + subTaskSpecStatus = new SubTaskSpecStatus( + (SinglePhaseSubTaskSpec) monitorEntry.getSpec(), + monitorEntry.getRunningStatus(), + monitorEntry.getTaskHistory() + ); + } else { + if (taskHistory != null && !taskHistory.isEmpty()) { + subTaskSpecStatus = new SubTaskSpecStatus( + (SinglePhaseSubTaskSpec) taskHistory.getSpec(), + null, + taskHistory.getAttemptHistory() + ); + } else { + subTaskSpecStatus = null; + } + } + + return subTaskSpecStatus; + } + } + + @Nullable + @Override + public TaskHistory getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId) + { + if (taskMonitor == null) { + return null; + } else { + return taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId); + } + } + + String getTaskId() + { + return taskId; + } + + String getGroupId() + { + return groupId; + } + + Map getContext() + { + return context; + } + + ParallelIndexTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @VisibleForTesting + TaskToolbox getToolbox() + { + return toolbox; + } + + @VisibleForTesting + @Nullable + TaskMonitor getTaskMonitor() + { + return taskMonitor; + } + + @VisibleForTesting + int getAndIncrementNextSpecId() + { + return nextSpecId++; + } + + @VisibleForTesting + IndexingServiceClient getIndexingServiceClient() + { + return indexingServiceClient; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index e6ef2c485c9f..2fa30e34270c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -30,25 +30,31 @@ import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.indexer.TaskState; import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.Counters; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; +import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; +import org.apache.druid.indexing.common.task.CurrentSubTaskHolder; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.common.task.IndexTaskUtils; +import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTaskRunner.SubTaskSpecStatus; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.indexing.TuningConfig; @@ -56,6 +62,8 @@ import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; +import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker; import org.apache.druid.segment.realtime.firehose.ChatHandler; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.firehose.ChatHandlers; @@ -63,6 +71,7 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -79,12 +88,19 @@ import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; import java.io.IOException; +import java.util.ArrayList; +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.Set; import java.util.SortedSet; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; import java.util.stream.Collectors; /** @@ -124,10 +140,19 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen private final ConcurrentHashMap partitionNumCountersPerInterval = new ConcurrentHashMap<>(); - private volatile ParallelIndexTaskRunner runner; - private volatile IndexTask sequentialIndexTask; + /** + * A holder for the current phase runner (parallel mode) or index task (sequential mode). + * This variable is lazily initialized in {@link #initializeSubTaskCleaner}. + * Volatile since HTTP API calls can read this variable at any time while this task is running. + */ + @MonotonicNonNull + private volatile CurrentSubTaskHolder currentSubTaskHolder; - // toolbox is initlized when run() is called, and can be used for processing HTTP endpoint requests. + /** + * A variable to keep the given toolbox. This variable is lazily initialized in {@link #runTask}. + * Volatile since HTTP API calls can use this variable at any time while this task is running. + */ + @MonotonicNonNull private volatile TaskToolbox toolbox; @JsonCreator @@ -158,6 +183,15 @@ public ParallelIndexSupervisorTask( throw new IAE("[%s] should implement FiniteFirehoseFactory", firehoseFactory.getClass().getSimpleName()); } + if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup() + && (ingestionSchema.getTuningConfig().getNumShards() == null + || ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals().isEmpty())) { + throw new ISE( + "forceGuaranteedRollup is set " + + "but numShards is missing in partitionsSpec or intervals is missing in granularitySpec" + ); + } + this.baseFirehoseFactory = (FiniteFirehoseFactory) firehoseFactory; this.indexingServiceClient = indexingServiceClient; this.chatHandlerProvider = chatHandlerProvider; @@ -173,16 +207,6 @@ public ParallelIndexSupervisorTask( addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, true); } - if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() - != TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS) { - LOG.warn("maxSavedParseExceptions is not supported yet"); - } - if (ingestionSchema.getTuningConfig().getMaxParseExceptions() != TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS) { - LOG.warn("maxParseExceptions is not supported yet"); - } - if (ingestionSchema.getTuningConfig().isLogParseExceptions() != TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS) { - LOG.warn("logParseExceptions is not supported yet"); - } } @Override @@ -205,40 +229,87 @@ public ParallelIndexIngestionSpec getIngestionSchema() @VisibleForTesting @Nullable - ParallelIndexTaskRunner getRunner() + ParallelIndexTaskRunner getCurrentRunner() { - return runner; + if (isParallelMode()) { + return currentSubTaskHolder == null ? null : currentSubTaskHolder.getTask(); + } else { + return null; + } } @VisibleForTesting - AuthorizerMapper getAuthorizerMapper() + IndexingServiceClient getIndexingServiceClient() { - return authorizerMapper; + return indexingServiceClient; } - @VisibleForTesting - ParallelIndexTaskRunner createRunner(TaskToolbox toolbox) + @Nullable + private ParallelIndexTaskRunner createRunner( + TaskToolbox toolbox, + Function> runnerCreator + ) { - this.toolbox = toolbox; - if (ingestionSchema.getTuningConfig().isForceGuaranteedRollup()) { - throw new UnsupportedOperationException("Perfect roll-up is not supported yet"); + final ParallelIndexTaskRunner newRunner = runnerCreator.apply(toolbox); + if (currentSubTaskHolder.setTask(newRunner)) { + return newRunner; } else { - runner = new SinglePhaseParallelIndexTaskRunner( - toolbox, - getId(), - getGroupId(), - ingestionSchema, - getContext(), - indexingServiceClient - ); + return null; + } + } + + private TaskState runNextPhase(@Nullable ParallelIndexTaskRunner nextPhaseRunner) throws Exception + { + if (nextPhaseRunner == null) { + LOG.info("Task is asked to stop. Finish as failed"); + return TaskState.FAILED; + } else { + return nextPhaseRunner.run(); } - return runner; } @VisibleForTesting - void setRunner(ParallelIndexTaskRunner runner) + SinglePhaseParallelIndexTaskRunner createSinglePhaseTaskRunner(TaskToolbox toolbox) { - this.runner = runner; + return new SinglePhaseParallelIndexTaskRunner( + toolbox, + getId(), + getGroupId(), + ingestionSchema, + getContext(), + indexingServiceClient + ); + } + + @VisibleForTesting + public PartialSegmentGenerateParallelIndexTaskRunner createPartialSegmentGenerateRunner(TaskToolbox toolbox) + { + return new PartialSegmentGenerateParallelIndexTaskRunner( + toolbox, + getId(), + getGroupId(), + ingestionSchema, + getContext(), + indexingServiceClient + ); + } + + @VisibleForTesting + public PartialSegmentMergeParallelIndexTaskRunner createPartialSegmentMergeRunner( + TaskToolbox toolbox, + List ioConfigs + ) + { + return new PartialSegmentMergeParallelIndexTaskRunner( + toolbox, + getId(), + getGroupId(), + getIngestionSchema().getDataSchema(), + ioConfigs, + getIngestionSchema().getTuningConfig(), + getContext(), + indexingServiceClient + ); } @Override @@ -286,6 +357,16 @@ public Granularity getSegmentGranularity() @Override public TaskStatus runTask(TaskToolbox toolbox) throws Exception { + if (ingestionSchema.getTuningConfig().getMaxSavedParseExceptions() + != TuningConfig.DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS) { + LOG.warn("maxSavedParseExceptions is not supported yet"); + } + if (ingestionSchema.getTuningConfig().getMaxParseExceptions() != TuningConfig.DEFAULT_MAX_PARSE_EXCEPTIONS) { + LOG.warn("maxParseExceptions is not supported yet"); + } + if (ingestionSchema.getTuningConfig().isLogParseExceptions() != TuningConfig.DEFAULT_LOG_PARSE_EXCEPTIONS) { + LOG.warn("logParseExceptions is not supported yet"); + } if (missingIntervalsInOverwriteMode) { LOG.warn( "Intervals are missing in granularitySpec while this task is potentially overwriting existing segments. " @@ -299,8 +380,15 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception chatHandlerProvider.register(getId(), this, false); try { + initializeSubTaskCleaner(); + if (isParallelMode()) { - return runParallel(toolbox); + this.toolbox = toolbox; + if (getIngestionSchema().getTuningConfig().isForceGuaranteedRollup()) { + return runMultiPhaseParallel(toolbox); + } else { + return runSinglePhaseParallel(toolbox); + } } else { if (!baseFirehoseFactory.isSplittable()) { LOG.warn( @@ -324,31 +412,207 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception } } - private boolean isParallelMode() + private void initializeSubTaskCleaner() { - if (baseFirehoseFactory.isSplittable() && ingestionSchema.getTuningConfig().getMaxNumSubTasks() > 1) { - return true; + if (isParallelMode()) { + currentSubTaskHolder = new CurrentSubTaskHolder((currentRunnerObject, taskConfig) -> { + final ParallelIndexTaskRunner runner = (ParallelIndexTaskRunner) currentRunnerObject; + runner.stopGracefully(); + }); } else { - return false; + currentSubTaskHolder = new CurrentSubTaskHolder((taskObject, taskConfig) -> { + final IndexTask task = (IndexTask) taskObject; + task.stopGracefully(taskConfig); + }); } + registerResourceCloserOnAbnormalExit(currentSubTaskHolder); } - @VisibleForTesting - void setToolbox(TaskToolbox toolbox) + private boolean isParallelMode() { - this.toolbox = toolbox; + return baseFirehoseFactory.isSplittable() && ingestionSchema.getTuningConfig().getMaxNumSubTasks() > 1; } - private TaskStatus runParallel(TaskToolbox toolbox) throws Exception + /** + * Run the single phase parallel indexing for best-effort rollup. In this mode, each sub task created by + * the supervisor task reads data and generates segments individually. + */ + private TaskStatus runSinglePhaseParallel(TaskToolbox toolbox) throws Exception { - createRunner(toolbox); - registerResourceCloserOnAbnormalExit(config -> runner.stopGracefully()); - return TaskStatus.fromCode(getId(), Preconditions.checkNotNull(runner, "runner").run()); + final ParallelIndexTaskRunner runner = createRunner( + toolbox, + this::createSinglePhaseTaskRunner + ); + + final TaskState state = runNextPhase(runner); + if (state.isSuccess()) { + //noinspection ConstantConditions + publishSegments(toolbox, runner.getReports()); + } + return TaskStatus.fromCode(getId(), state); + } + + /** + * Run the multi phase parallel indexing for perfect rollup. In this mode, the parallel indexing is currently + * executed in two phases. + * + * - In the first phase, each task partitions input data and stores those partitions in local storage. + * - The partition is created based on the segment granularity (primary partition key) and the partition dimension + * values in {@link org.apache.druid.indexer.partitions.PartitionsSpec} (secondary partition key). + * - Partitioned data is maintained by {@link org.apache.druid.indexing.worker.IntermediaryDataManager}. + * - In the second phase, each task reads partitioned data from the intermediary data server (middleManager + * or indexer) and merges them to create the final segments. + */ + private TaskStatus runMultiPhaseParallel(TaskToolbox toolbox) throws Exception + { + // 1. Partial segment generation phase + final ParallelIndexTaskRunner indexingRunner = createRunner( + toolbox, + this::createPartialSegmentGenerateRunner + ); + + TaskState state = runNextPhase(indexingRunner); + if (state.isFailure()) { + return TaskStatus.failure(getId()); + } + + // 2. Partial segment merge phase + + // partition (interval, partitionId) -> partition locations + //noinspection ConstantConditions + Map, List> partitionToLocations = groupPartitionLocationsPerPartition( + indexingRunner.getReports() + ); + final List ioConfigs = createMergeIOConfigs( + ingestionSchema.getTuningConfig().getTotalNumMergeTasks(), + partitionToLocations + ); + + final ParallelIndexTaskRunner mergeRunner = createRunner( + toolbox, + tb -> createPartialSegmentMergeRunner(tb, ioConfigs) + ); + state = runNextPhase(mergeRunner); + if (state.isSuccess()) { + //noinspection ConstantConditions + publishSegments(toolbox, mergeRunner.getReports()); + } + + return TaskStatus.fromCode(getId(), state); + } + + private static Map, List> groupPartitionLocationsPerPartition( + // subTaskId -> report + Map reports + ) + { + // partition (interval, partitionId) -> partition locations + final Map, List> partitionToLocations = new HashMap<>(); + for (Entry entry : reports.entrySet()) { + final String subTaskId = entry.getKey(); + final GeneratedPartitionsReport report = entry.getValue(); + for (PartitionStat partitionStat : report.getPartitionStats()) { + final List locationsOfSamePartition = partitionToLocations.computeIfAbsent( + Pair.of(partitionStat.getInterval(), partitionStat.getPartitionId()), + k -> new ArrayList<>() + ); + locationsOfSamePartition.add( + new PartitionLocation( + partitionStat.getTaskExecutorHost(), + partitionStat.getTaskExecutorPort(), + partitionStat.isUseHttps(), + subTaskId, + partitionStat.getInterval(), + partitionStat.getPartitionId() + ) + ); + } + } + + return partitionToLocations; + } + + private static List createMergeIOConfigs( + int totalNumMergeTasks, + Map, List> partitionToLocations + ) + { + final int numMergeTasks = Math.min(totalNumMergeTasks, partitionToLocations.size()); + LOG.info( + "Number of merge tasks is set to [%d] based on totalNumMergeTasks[%d] and number of partitions[%d]", + numMergeTasks, + totalNumMergeTasks, + partitionToLocations.size() + ); + // Randomly shuffle partitionIds to evenly distribute partitions of potentially different sizes + // This will be improved once we collect partition stats properly. + // See PartitionStat in GeneratedPartitionsReport. + final List> partitions = new ArrayList<>(partitionToLocations.keySet()); + Collections.shuffle(partitions, ThreadLocalRandom.current()); + final int numPartitionsPerTask = (int) Math.round(partitions.size() / (double) numMergeTasks); + + final List assignedPartitionLocations = new ArrayList<>(numMergeTasks); + for (int i = 0; i < numMergeTasks - 1; i++) { + final List assignedToSameTask = partitions + .subList(i * numPartitionsPerTask, (i + 1) * numPartitionsPerTask) + .stream() + .flatMap(intervalAndPartitionId -> partitionToLocations.get(intervalAndPartitionId).stream()) + .collect(Collectors.toList()); + assignedPartitionLocations.add(new PartialSegmentMergeIOConfig(assignedToSameTask)); + } + + // The last task is assigned all remaining partitions. + final List assignedToSameTask = partitions + .subList((numMergeTasks - 1) * numPartitionsPerTask, partitions.size()) + .stream() + .flatMap(intervalAndPartitionId -> partitionToLocations.get(intervalAndPartitionId).stream()) + .collect(Collectors.toList()); + assignedPartitionLocations.add(new PartialSegmentMergeIOConfig(assignedToSameTask)); + + return assignedPartitionLocations; + } + + private static void publishSegments(TaskToolbox toolbox, Map reportsMap) + throws IOException + { + final UsedSegmentChecker usedSegmentChecker = new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()); + final Set oldSegments = new HashSet<>(); + final Set newSegments = new HashSet<>(); + reportsMap + .values() + .forEach(report -> { + oldSegments.addAll(report.getOldSegments()); + newSegments.addAll(report.getNewSegments()); + }); + final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> + toolbox.getTaskActionClient().submit( + SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToPublish) + ); + final boolean published = newSegments.isEmpty() + || publisher.publishSegments(oldSegments, newSegments, null).isSuccess(); + + if (published) { + LOG.info("Published [%d] segments", newSegments.size()); + } else { + LOG.info("Transaction failure while publishing segments, checking if someone else beat us to it."); + final Set segmentsIdentifiers = reportsMap + .values() + .stream() + .flatMap(report -> report.getNewSegments().stream()) + .map(SegmentIdWithShardSpec::fromDataSegment) + .collect(Collectors.toSet()); + if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers) + .equals(newSegments)) { + LOG.info("Our segments really do exist, awaiting handoff."); + } else { + throw new ISE("Failed to publish segments[%s]", newSegments); + } + } } private TaskStatus runSequential(TaskToolbox toolbox) throws Exception { - sequentialIndexTask = new IndexTask( + final IndexTask indexTask = new IndexTask( getId(), getGroupId(), getTaskResource(), @@ -364,10 +628,11 @@ private TaskStatus runSequential(TaskToolbox toolbox) throws Exception rowIngestionMetersFactory, appenderatorsManager ); - if (sequentialIndexTask.isReady(toolbox.getTaskActionClient())) { - registerResourceCloserOnAbnormalExit(config -> sequentialIndexTask.stopGracefully(config)); - return sequentialIndexTask.run(toolbox); + + if (currentSubTaskHolder.setTask(indexTask) && indexTask.isReady(toolbox.getTaskActionClient())) { + return indexTask.run(toolbox); } else { + LOG.info("Task is asked to stop. Finish as failed"); return TaskStatus.failure(getId()); } } @@ -387,7 +652,7 @@ private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningC tuningConfig.getIndexSpec(), tuningConfig.getIndexSpecForIntermediatePersists(), tuningConfig.getMaxPendingPersists(), - false, + tuningConfig.isForceGuaranteedRollup(), tuningConfig.isReportParseExceptions(), null, tuningConfig.getPushTimeout(), @@ -401,7 +666,7 @@ private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningC // Internal APIs /** - * Allocate a new {@link SegmentIdWithShardSpec} for a request from {@link ParallelIndexSubTask}. + * Allocate a new {@link SegmentIdWithShardSpec} for a request from {@link SinglePhaseSubTask}. * The returned segmentIdentifiers have different {@code partitionNum} (thereby different {@link NumberedShardSpec}) * per bucket interval. */ @@ -510,13 +775,13 @@ private static String findVersion(Map versions, Interval inter } /** - * {@link ParallelIndexSubTask}s call this API to report the segments they've generated and pushed. + * {@link SinglePhaseSubTask}s call this API to report the segments they generated and pushed. */ @POST @Path("/report") @Consumes(SmileMediaTypes.APPLICATION_JACKSON_SMILE) public Response report( - PushedSegmentsReport report, + SubTaskReport report, @Context final HttpServletRequest req ) { @@ -526,9 +791,11 @@ public Response report( getDataSource(), authorizerMapper ); - if (runner == null) { + if (currentSubTaskHolder == null || currentSubTaskHolder.getTask() == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { + final ParallelIndexTaskRunner runner = currentSubTaskHolder.getTask(); + //noinspection unchecked runner.collectReport(report); return Response.ok().build(); } @@ -545,16 +812,35 @@ public Response getMode(@Context final HttpServletRequest req) return Response.ok(isParallelMode() ? "parallel" : "sequential").build(); } + @GET + @Path("/phase") + @Produces(MediaType.APPLICATION_JSON) + public Response getPhaseName(@Context final HttpServletRequest req) + { + IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); + if (isParallelMode()) { + final ParallelIndexTaskRunner runner = getCurrentRunner(); + if (runner == null) { + return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running").build(); + } else { + return Response.ok(runner.getName()).build(); + } + } else { + return Response.status(Status.BAD_REQUEST).entity("task is running in the sequential mode").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) { + final ParallelIndexTaskRunner currentRunner = getCurrentRunner(); + if (currentRunner == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { - return Response.ok(runner.getProgress()).build(); + return Response.ok(currentRunner.getProgress()).build(); } } @@ -564,10 +850,11 @@ public Response getProgress(@Context final HttpServletRequest req) public Response getRunningTasks(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - if (runner == null) { + final ParallelIndexTaskRunner currentRunner = getCurrentRunner(); + if (currentRunner == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { - return Response.ok(runner.getRunningTaskIds()).build(); + return Response.ok(currentRunner.getRunningTaskIds()).build(); } } @@ -577,10 +864,11 @@ public Response getRunningTasks(@Context final HttpServletRequest req) public Response getSubTaskSpecs(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - if (runner == null) { + final ParallelIndexTaskRunner currentRunner = getCurrentRunner(); + if (currentRunner == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { - return Response.ok(runner.getSubTaskSpecs()).build(); + return Response.ok(currentRunner.getSubTaskSpecs()).build(); } } @@ -590,10 +878,11 @@ public Response getSubTaskSpecs(@Context final HttpServletRequest req) public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - if (runner == null) { + final ParallelIndexTaskRunner currentRunner = getCurrentRunner(); + if (currentRunner == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { - return Response.ok(runner.getRunningSubTaskSpecs()).build(); + return Response.ok(currentRunner.getRunningSubTaskSpecs()).build(); } } @@ -603,10 +892,11 @@ public Response getRunningSubTaskSpecs(@Context final HttpServletRequest req) public Response getCompleteSubTaskSpecs(@Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - if (runner == null) { + final ParallelIndexTaskRunner currentRunner = getCurrentRunner(); + if (currentRunner == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { - return Response.ok(runner.getCompleteSubTaskSpecs()).build(); + return Response.ok(currentRunner.getCompleteSubTaskSpecs()).build(); } } @@ -617,10 +907,11 @@ public Response getSubTaskSpec(@PathParam("id") String id, @Context final HttpSe { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - if (runner == null) { + final ParallelIndexTaskRunner currentRunner = getCurrentRunner(); + if (currentRunner == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { - final SubTaskSpec subTaskSpec = runner.getSubTaskSpec(id); + final SubTaskSpec subTaskSpec = currentRunner.getSubTaskSpec(id); if (subTaskSpec == null) { return Response.status(Response.Status.NOT_FOUND).build(); } else { @@ -635,10 +926,11 @@ public Response getSubTaskSpec(@PathParam("id") String id, @Context final HttpSe public Response getSubTaskState(@PathParam("id") String id, @Context final HttpServletRequest req) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - if (runner == null) { + final ParallelIndexTaskRunner currentRunner = getCurrentRunner(); + if (currentRunner == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { - final SubTaskSpecStatus subTaskSpecStatus = runner.getSubTaskState(id); + final SubTaskSpecStatus subTaskSpecStatus = currentRunner.getSubTaskState(id); if (subTaskSpecStatus == null) { return Response.status(Response.Status.NOT_FOUND).build(); } else { @@ -656,10 +948,11 @@ public Response getCompleteSubTaskSpecAttemptHistory( ) { IndexTaskUtils.datasourceAuthorizationCheck(req, Action.READ, getDataSource(), authorizerMapper); - if (runner == null) { + final ParallelIndexTaskRunner currentRunner = getCurrentRunner(); + if (currentRunner == null) { return Response.status(Response.Status.SERVICE_UNAVAILABLE).entity("task is not running yet").build(); } else { - final TaskHistory taskHistory = runner.getCompleteSubTaskSpecAttemptHistory(id); + final TaskHistory taskHistory = currentRunner.getCompleteSubTaskSpecAttemptHistory(id); if (taskHistory == null) { return Response.status(Status.NOT_FOUND).build(); } else { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java index 3d2ed6b2d1e0..4c2451605954 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskClient.java @@ -27,13 +27,11 @@ import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.response.FullResponseHolder; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.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.Set; public class ParallelIndexTaskClient extends IndexTaskClient { @@ -84,7 +82,7 @@ public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime } } - public void report(String supervisorTaskId, Set oldSegments, Set pushedSegments) + public void report(String supervisorTaskId, SubTaskReport report) { try { final FullResponseHolder response = submitSmileRequest( @@ -92,7 +90,7 @@ public void report(String supervisorTaskId, Set oldSegments, Set +public interface ParallelIndexTaskRunner { + /** + * Returns the name of this runner. + */ + String getName(); + /** * Runs the task. */ @@ -48,10 +60,15 @@ public interface ParallelIndexTaskRunner void stopGracefully(); /** - * {@link PushedSegmentsReport} is the report sent by {@link ParallelIndexSubTask}s. The subTasks call this method to + * {@link SubTaskReport} is the report sent by {@link SubTaskType}s. The subTasks call this method to * send their reports after pushing generated segments to deep storage. */ - void collectReport(PushedSegmentsReport report); + void collectReport(SubTaskReportType report); + + /** + * Returns a map between subTaskId and its report. + */ + Map getReports(); /** * Returns the current {@link ParallelIndexingProgress}. @@ -66,7 +83,7 @@ public interface ParallelIndexTaskRunner /** * Returns all {@link SubTaskSpec}s. */ - List> getSubTaskSpecs(); + List> getSubTaskSpecs(); /** * Returns running {@link SubTaskSpec}s. A {@link SubTaskSpec} is running if there is a running {@link Task} created @@ -74,7 +91,7 @@ public interface ParallelIndexTaskRunner * * @see SubTaskSpec#newSubTask */ - List> getRunningSubTaskSpecs(); + List> getRunningSubTaskSpecs(); /** * Returns complete {@link SubTaskSpec}s. A {@link SubTaskSpec} is complete if there is a succeeded or failed @@ -82,13 +99,13 @@ public interface ParallelIndexTaskRunner * * @see SubTaskSpec#newSubTask */ - List> getCompleteSubTaskSpecs(); + List> getCompleteSubTaskSpecs(); /** * Returns the {@link SubTaskSpec} of the given ID or null if it's not found. */ @Nullable - SubTaskSpec getSubTaskSpec(String subTaskSpecId); + SubTaskSpec getSubTaskSpec(String subTaskSpecId); /** * Returns {@link SubTaskSpecStatus} of the given ID or null if it's not found. @@ -100,18 +117,18 @@ public interface ParallelIndexTaskRunner * Returns {@link TaskHistory} of the given ID or null if it's not found. */ @Nullable - TaskHistory getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId); + TaskHistory getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId); class SubTaskSpecStatus { - private final ParallelIndexSubTaskSpec spec; + private final SinglePhaseSubTaskSpec spec; @Nullable private final TaskStatusPlus currentStatus; // null if there is no running task for the spec private final List taskHistory; // can be empty if there is no history @JsonCreator public SubTaskSpecStatus( - @JsonProperty("spec") ParallelIndexSubTaskSpec spec, + @JsonProperty("spec") SinglePhaseSubTaskSpec spec, @JsonProperty("currentStatus") @Nullable TaskStatusPlus currentStatus, @JsonProperty("taskHistory") List taskHistory ) @@ -122,7 +139,7 @@ public SubTaskSpecStatus( } @JsonProperty - public ParallelIndexSubTaskSpec getSpec() + public SinglePhaseSubTaskSpec getSpec() { return spec; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java index 86f1e3fdcff4..4ec7198a7671 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; -import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.segment.IndexSpec; @@ -43,6 +42,8 @@ public class ParallelIndexTuningConfig extends IndexTuningConfig private static final Duration DEFAULT_CHAT_HANDLER_TIMEOUT = new Period("PT10S").toStandardDuration(); private static final int DEFAULT_CHAT_HANDLER_NUM_RETRIES = 5; + private static final int DEFAULT_MAX_NUM_SEGMENTS_TO_MERGE = 100; + private static final int DEFAULT_TOTAL_NUM_MERGE_TASKS = 10; private final int maxNumSubTasks; private final int maxRetry; @@ -51,7 +52,21 @@ public class ParallelIndexTuningConfig extends IndexTuningConfig private final Duration chatHandlerTimeout; private final int chatHandlerNumRetries; - public static ParallelIndexTuningConfig defaultConfig() + /** + * Max number of segments to merge at the same time. + * Used only by {@link PartialSegmentMergeTask}. + * This configuration was temporarily added to avoid using too much memory while merging segments, + * and will be removed once {@link org.apache.druid.segment.IndexMerger} is improved to not use much memory. + */ + private final int maxNumSegmentsToMerge; + + /** + * Total number of tasks for partial segment merge (that is, number of {@link PartialSegmentMergeTask}s). + * Used only when this task runs with shuffle. + */ + private final int totalNumMergeTasks; + + static ParallelIndexTuningConfig defaultConfig() { return new ParallelIndexTuningConfig( null, @@ -75,6 +90,8 @@ public static ParallelIndexTuningConfig defaultConfig() null, null, null, + null, + null, null ); } @@ -100,6 +117,8 @@ public ParallelIndexTuningConfig( @JsonProperty("taskStatusCheckPeriodMs") @Nullable Integer taskStatusCheckPeriodMs, @JsonProperty("chatHandlerTimeout") @Nullable Duration chatHandlerTimeout, @JsonProperty("chatHandlerNumRetries") @Nullable Integer chatHandlerNumRetries, + @JsonProperty("maxNumSegmentsToMerge") @Nullable Integer maxNumSegmentsToMerge, + @JsonProperty("totalNumMergeTasks") @Nullable Integer totalNumMergeTasks, @JsonProperty("logParseExceptions") @Nullable Boolean logParseExceptions, @JsonProperty("maxParseExceptions") @Nullable Integer maxParseExceptions, @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions @@ -114,7 +133,7 @@ public ParallelIndexTuningConfig( null, numShards, null, - getValidPartitionsSpec(maxRowsPerSegment, maxTotalRows, partitionsSpec), + partitionsSpec, indexSpec, indexSpecForIntermediatePersists, maxPendingPersists, @@ -139,23 +158,17 @@ public ParallelIndexTuningConfig( ? DEFAULT_CHAT_HANDLER_NUM_RETRIES : chatHandlerNumRetries; - Preconditions.checkArgument(this.maxNumSubTasks > 0, "maxNumSubTasks must be positive"); - } + this.maxNumSegmentsToMerge = maxNumSegmentsToMerge == null + ? DEFAULT_MAX_NUM_SEGMENTS_TO_MERGE + : maxNumSegmentsToMerge; - private static PartitionsSpec getValidPartitionsSpec( - @Nullable Integer maxRowsPerSegment, - @Nullable Long maxTotalRows, - @Nullable PartitionsSpec partitionsSpec - ) - { - if (partitionsSpec != null) { - if (!(partitionsSpec instanceof DynamicPartitionsSpec)) { - throw new UnsupportedOperationException("Parallel index task supports only dynamic partitionsSpec yet"); - } - return partitionsSpec; - } else { - return new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); - } + this.totalNumMergeTasks = totalNumMergeTasks == null + ? DEFAULT_TOTAL_NUM_MERGE_TASKS + : totalNumMergeTasks; + + Preconditions.checkArgument(this.maxNumSubTasks > 0, "maxNumSubTasks must be positive"); + Preconditions.checkArgument(this.maxNumSegmentsToMerge > 0, "maxNumSegmentsToMerge must be positive"); + Preconditions.checkArgument(this.totalNumMergeTasks > 0, "totalNumMergeTasks must be positive"); } @JsonProperty @@ -188,6 +201,18 @@ public int getChatHandlerNumRetries() return chatHandlerNumRetries; } + @JsonProperty + public int getMaxNumSegmentsToMerge() + { + return maxNumSegmentsToMerge; + } + + @JsonProperty + public int getTotalNumMergeTasks() + { + return totalNumMergeTasks; + } + @Override public boolean equals(Object o) { @@ -205,6 +230,8 @@ public boolean equals(Object o) maxRetry == that.maxRetry && taskStatusCheckPeriodMs == that.taskStatusCheckPeriodMs && chatHandlerNumRetries == that.chatHandlerNumRetries && + maxNumSegmentsToMerge == that.maxNumSegmentsToMerge && + totalNumMergeTasks == that.totalNumMergeTasks && Objects.equals(chatHandlerTimeout, that.chatHandlerTimeout); } @@ -217,7 +244,9 @@ public int hashCode() maxRetry, taskStatusCheckPeriodMs, chatHandlerTimeout, - chatHandlerNumRetries + chatHandlerNumRetries, + maxNumSegmentsToMerge, + totalNumMergeTasks ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java new file mode 100644 index 000000000000..35061a1df524 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateParallelIndexTaskRunner.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.FiniteFirehoseFactory; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.indexing.common.TaskToolbox; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + +/** + * {@link ParallelIndexTaskRunner} for the phase to create partitioned segments in multi-phase parallel indexing. + * + * @see PartialSegmentMergeParallelIndexTaskRunner + */ +class PartialSegmentGenerateParallelIndexTaskRunner + extends ParallelIndexPhaseRunner +{ + private final ParallelIndexIngestionSpec ingestionSchema; + private final FiniteFirehoseFactory baseFirehoseFactory; + + PartialSegmentGenerateParallelIndexTaskRunner( + TaskToolbox toolbox, + String taskId, + String groupId, + ParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient + ) + { + super( + toolbox, + taskId, + groupId, + ingestionSchema.getTuningConfig(), + context, + indexingServiceClient + ); + this.ingestionSchema = ingestionSchema; + this.baseFirehoseFactory = (FiniteFirehoseFactory) ingestionSchema.getIOConfig().getFirehoseFactory(); + } + + @Override + public String getName() + { + return PartialSegmentGenerateTask.TYPE; + } + + @Override + Iterator> subTaskSpecIterator() throws IOException + { + return baseFirehoseFactory.getSplits().map(this::newTaskSpec).iterator(); + } + + @Override + int getTotalNumSubTasks() throws IOException + { + return baseFirehoseFactory.getNumSplits(); + } + + @VisibleForTesting + ParallelIndexIngestionSpec getIngestionSchema() + { + return ingestionSchema; + } + + @VisibleForTesting + FiniteFirehoseFactory getBaseFirehoseFactory() + { + return baseFirehoseFactory; + } + + SubTaskSpec newTaskSpec(InputSplit split) + { + final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec( + ingestionSchema.getDataSchema(), + new ParallelIndexIOConfig( + baseFirehoseFactory.withSplit(split), + ingestionSchema.getIOConfig().isAppendToExisting() + ), + ingestionSchema.getTuningConfig() + ); + return new SubTaskSpec( + getTaskId() + "_" + getAndIncrementNextSpecId(), + getGroupId(), + getTaskId(), + getContext(), + split + ) + { + @Override + public PartialSegmentGenerateTask newSubTask(int numAttempts) + { + return new PartialSegmentGenerateTask( + null, + getGroupId(), + null, + getSupervisorTaskId(), + numAttempts, + subTaskIngestionSpec, + getContext(), + getIndexingServiceClient(), + null, + null + ); + } + }; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java new file mode 100644 index 000000000000..d78e337c5dd7 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -0,0 +1,327 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.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.Preconditions; +import org.apache.commons.io.FileUtils; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.FirehoseFactory; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters; +import org.apache.druid.indexing.common.stats.RowIngestionMeters; +import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; +import org.apache.druid.indexing.common.task.BatchAppenderators; +import org.apache.druid.indexing.common.task.CachingLocalSegmentAllocator; +import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; +import org.apache.druid.indexing.common.task.FiniteFirehoseProcessor; +import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.IndexTaskSegmentAllocator; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.DruidMetrics; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.RealtimeIOConfig; +import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.realtime.FireDepartment; +import org.apache.druid.segment.realtime.FireDepartmentMetrics; +import org.apache.druid.segment.realtime.RealtimeMetricsMonitor; +import org.apache.druid.segment.realtime.appenderator.Appenderator; +import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; +import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; +import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.ShardSpecFactory; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +/** + * The worker task of {@link PartialSegmentGenerateParallelIndexTaskRunner}. This task partitions input data by + * the segment granularity and partition dimensions in {@link org.apache.druid.indexer.partitions.PartitionsSpec}. + * Partitioned segments are stored in local storage using {@link ShuffleDataSegmentPusher}. + */ +public class PartialSegmentGenerateTask extends AbstractBatchIndexTask +{ + public static final String TYPE = "partial_index_generate"; + + private final int numAttempts; + private final ParallelIndexIngestionSpec ingestionSchema; + private final String supervisorTaskId; + private final IndexingServiceClient indexingServiceClient; + private final IndexTaskClientFactory taskClientFactory; + private final AppenderatorsManager appenderatorsManager; + + @JsonCreator + public PartialSegmentGenerateTask( + // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask + @JsonProperty("id") @Nullable 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 context, + @JacksonInject IndexingServiceClient indexingServiceClient, + @JacksonInject IndexTaskClientFactory taskClientFactory, + @JacksonInject AppenderatorsManager appenderatorsManager + ) + { + super( + getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), + groupId, + taskResource, + ingestionSchema.getDataSchema().getDataSource(), + context + ); + + Preconditions.checkArgument( + ingestionSchema.getTuningConfig().isForceGuaranteedRollup(), + "forceGuaranteedRollup must be set" + ); + Preconditions.checkArgument( + ingestionSchema.getTuningConfig().getPartitionsSpec() == null + || ingestionSchema.getTuningConfig().getPartitionsSpec() instanceof HashedPartitionsSpec, + "Please use hashed_partitions for perfect rollup" + ); + Preconditions.checkArgument( + !ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals().isEmpty(), + "Missing intervals in granularitySpec" + ); + + this.numAttempts = numAttempts; + this.ingestionSchema = ingestionSchema; + this.supervisorTaskId = supervisorTaskId; + this.indexingServiceClient = indexingServiceClient; + this.taskClientFactory = taskClientFactory; + this.appenderatorsManager = appenderatorsManager; + } + + @JsonProperty + public int getNumAttempts() + { + return numAttempts; + } + + @JsonProperty("spec") + public ParallelIndexIngestionSpec getIngestionSchema() + { + return ingestionSchema; + } + + @JsonProperty + public String getSupervisorTaskId() + { + return supervisorTaskId; + } + + @Override + public int getPriority() + { + return getContextValue(Tasks.PRIORITY_KEY, Tasks.DEFAULT_BATCH_INDEX_TASK_PRIORITY); + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public boolean requireLockExistingSegments() + { + return true; + } + + @Override + public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) + { + throw new UnsupportedOperationException( + "This method should be never called because ParallelIndexGeneratingTask always uses timeChunk locking" + + " but this method is supposed to be called only with segment locking." + ); + } + + @Override + public boolean isPerfectRollup() + { + return true; + } + + @Nullable + @Override + public Granularity getSegmentGranularity() + { + final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + if (granularitySpec instanceof ArbitraryGranularitySpec) { + return null; + } else { + return granularitySpec.getSegmentGranularity(); + } + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) throws Exception + { + return tryTimeChunkLock( + taskActionClient, + getIngestionSchema().getDataSchema().getGranularitySpec().inputIntervals() + ); + } + + @Override + public TaskStatus runTask(TaskToolbox toolbox) throws Exception + { + final FirehoseFactory firehoseFactory = ingestionSchema.getIOConfig().getFirehoseFactory(); + + 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 segments = generateSegments(toolbox, firehoseFactory, firehoseTempDir); + final List partitionStats = segments + .stream() + .map(segment -> new PartitionStat( + toolbox.getTaskExecutorNode().getHost(), + toolbox.getTaskExecutorNode().getPortToUse(), + toolbox.getTaskExecutorNode().isEnableTlsPort(), + segment.getInterval(), + segment.getShardSpec().getPartitionNum(), + null, // numRows is not supported yet + null // sizeBytes is not supported yet + )) + .collect(Collectors.toList()); + taskClient.report(supervisorTaskId, new GeneratedPartitionsReport(getId(), partitionStats)); + + return TaskStatus.success(getId()); + } + + private List generateSegments( + final TaskToolbox toolbox, + final FirehoseFactory firehoseFactory, + final File firehoseTempDir + ) throws IOException, InterruptedException, ExecutionException, TimeoutException + { + final DataSchema dataSchema = ingestionSchema.getDataSchema(); + final GranularitySpec granularitySpec = dataSchema.getGranularitySpec(); + final FireDepartment fireDepartmentForMetrics = new FireDepartment( + dataSchema, + new RealtimeIOConfig(null, null), + null + ); + final FireDepartmentMetrics fireDepartmentMetrics = fireDepartmentForMetrics.getMetrics(); + final RowIngestionMeters buildSegmentsMeters = new DropwizardRowIngestionMeters(); + + 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 HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec(); + final long pushTimeout = tuningConfig.getPushTimeout(); + + final Map> shardSpecs = createShardSpecWithoutInputScan( + granularitySpec, + ingestionSchema.getIOConfig(), + tuningConfig, + partitionsSpec + ); + + final IndexTaskSegmentAllocator segmentAllocator = new CachingLocalSegmentAllocator( + toolbox, + getId(), + getDataSource(), + shardSpecs + ); + + final Appenderator appenderator = BatchAppenderators.newAppenderator( + getId(), + appenderatorsManager, + fireDepartmentMetrics, + toolbox, + dataSchema, + tuningConfig, + new ShuffleDataSegmentPusher(supervisorTaskId, getId(), toolbox.getIntermediaryDataManager()) + ); + boolean exceptionOccurred = false; + try (final BatchAppenderatorDriver driver = BatchAppenderators.newDriver(appenderator, toolbox, segmentAllocator)) { + driver.startJob(); + + final FiniteFirehoseProcessor firehoseProcessor = new FiniteFirehoseProcessor( + buildSegmentsMeters, + null, + tuningConfig.isLogParseExceptions(), + tuningConfig.getMaxParseExceptions(), + pushTimeout + ); + final SegmentsAndMetadata pushed = firehoseProcessor.process( + dataSchema, + driver, + partitionsSpec, + firehoseFactory, + firehoseTempDir, + segmentAllocator + ); + + return pushed.getSegments(); + } + catch (Exception e) { + exceptionOccurred = true; + throw e; + } + finally { + if (exceptionOccurred) { + appenderator.closeNow(); + } else { + appenderator.close(); + } + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfig.java new file mode 100644 index 000000000000..4b5b4c3689a1 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIOConfig.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import org.apache.druid.segment.indexing.IOConfig; + +import java.util.List; + +@JsonTypeName(PartialSegmentMergeTask.TYPE) +public class PartialSegmentMergeIOConfig implements IOConfig +{ + private final List partitionLocations; + + @JsonCreator + public PartialSegmentMergeIOConfig(@JsonProperty("partitionLocations") List partitionLocations) + { + Preconditions.checkState( + partitionLocations != null && !partitionLocations.isEmpty(), + "Empty partition locations" + ); + this.partitionLocations = partitionLocations; + } + + @JsonProperty + public List getPartitionLocations() + { + return partitionLocations; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpec.java new file mode 100644 index 000000000000..e4c6e260c949 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeIngestionSpec.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.IngestionSpec; + +public class PartialSegmentMergeIngestionSpec + extends IngestionSpec +{ + @JsonCreator + public PartialSegmentMergeIngestionSpec( + @JsonProperty("dataSchema") DataSchema dataSchema, + @JsonProperty("ioConfig") PartialSegmentMergeIOConfig ioConfig, + @JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig + ) + { + super(dataSchema, ioConfig, tuningConfig); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeParallelIndexTaskRunner.java new file mode 100644 index 000000000000..860b619b4d91 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeParallelIndexTaskRunner.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.segment.indexing.DataSchema; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * {@link ParallelIndexTaskRunner} for the phase to merge partitioned segments in multi-phase parallel indexing. + * + * @see PartialSegmentGenerateParallelIndexTaskRunner + */ +class PartialSegmentMergeParallelIndexTaskRunner + extends ParallelIndexPhaseRunner +{ + private final DataSchema dataSchema; + private final List mergeIOConfigs; + + PartialSegmentMergeParallelIndexTaskRunner( + TaskToolbox toolbox, + String taskId, + String groupId, + DataSchema dataSchema, + List mergeIOConfigs, + ParallelIndexTuningConfig tuningConfig, + Map context, + IndexingServiceClient indexingServiceClient + ) + { + super(toolbox, taskId, groupId, tuningConfig, context, indexingServiceClient); + + this.dataSchema = dataSchema; + this.mergeIOConfigs = mergeIOConfigs; + } + + @Override + public String getName() + { + return PartialSegmentMergeTask.TYPE; + } + + @Override + Iterator> subTaskSpecIterator() + { + return mergeIOConfigs.stream().map(this::newTaskSpec).iterator(); + } + + @Override + int getTotalNumSubTasks() + { + return mergeIOConfigs.size(); + } + + @VisibleForTesting + SubTaskSpec newTaskSpec(PartialSegmentMergeIOConfig ioConfig) + { + final PartialSegmentMergeIngestionSpec ingestionSpec = new PartialSegmentMergeIngestionSpec( + dataSchema, + ioConfig, + getTuningConfig() + ); + return new SubTaskSpec( + getTaskId() + "_" + getAndIncrementNextSpecId(), + getGroupId(), + getTaskId(), + getContext(), + new InputSplit<>(ioConfig.getPartitionLocations()) + ) + { + @Override + public PartialSegmentMergeTask newSubTask(int numAttempts) + { + return new PartialSegmentMergeTask( + null, + getGroupId(), + null, + getSupervisorTaskId(), + numAttempts, + ingestionSpec, + getContext(), + null, + null + ); + } + }; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java new file mode 100644 index 000000000000..bac0415c1364 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java @@ -0,0 +1,452 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.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.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Stopwatch; +import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; +import org.apache.commons.io.FileUtils; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexing.common.TaskLock; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.actions.LockListAction; +import org.apache.druid.indexing.common.actions.SurrogateAction; +import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; +import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; +import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMerger; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; +import org.apache.druid.segment.loading.DataSegmentPusher; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.utils.CompressionUtils; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +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.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** + * The worker task of {@link PartialSegmentMergeParallelIndexTaskRunner}. This task reads partitioned segments created + * by {@link PartialSegmentGenerateTask}s, merges them, and pushes to the deep storage. The pushed segments are reported + * to {@link PartialSegmentMergeParallelIndexTaskRunner}. + */ +public class PartialSegmentMergeTask extends AbstractBatchIndexTask +{ + public static final String TYPE = "partial_index_merge"; + + private static final Logger LOG = new Logger(PartialSegmentMergeTask.class); + private static final int BUFFER_SIZE = 1024 * 4; + private static final int NUM_FETCH_RETRIES = 3; + + private final byte[] buffer = new byte[BUFFER_SIZE]; + + private final int numAttempts; + private final PartialSegmentMergeIngestionSpec ingestionSchema; + private final String supervisorTaskId; + private final IndexingServiceClient indexingServiceClient; + private final IndexTaskClientFactory taskClientFactory; + + @JsonCreator + public PartialSegmentMergeTask( + // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask + @JsonProperty("id") @Nullable 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 PartialSegmentMergeIngestionSpec ingestionSchema, + @JsonProperty("context") final Map context, + @JacksonInject IndexingServiceClient indexingServiceClient, + @JacksonInject IndexTaskClientFactory taskClientFactory + ) + { + super( + getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), + groupId, + taskResource, + ingestionSchema.getDataSchema().getDataSource(), + context + ); + + Preconditions.checkArgument( + ingestionSchema.getTuningConfig().isForceGuaranteedRollup(), + "forceGuaranteedRollup must be set" + ); + Preconditions.checkArgument( + ingestionSchema.getTuningConfig().getPartitionsSpec() == null + || ingestionSchema.getTuningConfig().getPartitionsSpec() instanceof HashedPartitionsSpec, + "Please use hashed_partitions for perfect rollup" + ); + Preconditions.checkArgument( + !ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals().isEmpty(), + "Missing intervals in granularitySpec" + ); + + this.numAttempts = numAttempts; + this.ingestionSchema = ingestionSchema; + this.supervisorTaskId = supervisorTaskId; + this.indexingServiceClient = indexingServiceClient; + this.taskClientFactory = taskClientFactory; + } + + @JsonProperty + public int getNumAttempts() + { + return numAttempts; + } + + @JsonProperty("spec") + public PartialSegmentMergeIngestionSpec getIngestionSchema() + { + return ingestionSchema; + } + + @JsonProperty + public String getSupervisorTaskId() + { + return supervisorTaskId; + } + + @Override + public boolean requireLockExistingSegments() + { + return true; + } + + @Override + public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) + { + throw new UnsupportedOperationException( + "This method should be never called because PartialSegmentMergeTask always uses timeChunk locking" + + " but this method is supposed to be called only with segment locking." + ); + } + + @Override + public boolean isPerfectRollup() + { + return true; + } + + @Nullable + @Override + public Granularity getSegmentGranularity() + { + final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); + if (granularitySpec instanceof ArbitraryGranularitySpec) { + return null; + } else { + return granularitySpec.getSegmentGranularity(); + } + } + + @Override + public String getType() + { + return TYPE; + } + + @Override + public boolean isReady(TaskActionClient taskActionClient) + { + return true; + } + + @Override + public TaskStatus runTask(TaskToolbox toolbox) throws Exception + { + // Group partitionLocations by interval and partitionId + final Map>> intervalToPartitions = new HashMap<>(); + for (PartitionLocation location : ingestionSchema.getIOConfig().getPartitionLocations()) { + intervalToPartitions.computeIfAbsent(location.getInterval(), k -> new Int2ObjectOpenHashMap<>()) + .computeIfAbsent(location.getPartitionId(), k -> new ArrayList<>()) + .add(location); + } + + final List locks = toolbox.getTaskActionClient().submit( + new SurrogateAction<>(supervisorTaskId, new LockListAction()) + ); + final Map intervalToVersion = new HashMap<>(locks.size()); + locks.forEach(lock -> { + if (lock.isRevoked()) { + throw new ISE("Lock[%s] is revoked"); + } + final String mustBeNull = intervalToVersion.put(lock.getInterval(), lock.getVersion()); + if (mustBeNull != null) { + throw new ISE( + "WTH? Two versions([%s], [%s]) for the same interval[%s]?", + lock.getVersion(), + mustBeNull, + lock.getInterval() + ); + } + }); + + LOG.info("locks: [%s]", locks); + + final Stopwatch fetchStopwatch = Stopwatch.createStarted(); + final Map>> intervalToUnzippedFiles = fetchSegmentFiles( + toolbox, + intervalToPartitions + ); + final long fetchTime = fetchStopwatch.elapsed(TimeUnit.SECONDS); + fetchStopwatch.stop(); + LOG.info("Fetch took [%s] seconds", fetchTime); + + final ParallelIndexTaskClient taskClient = taskClientFactory.build( + new ClientBasedTaskInfoProvider(indexingServiceClient), + getId(), + 1, // always use a single http thread + ingestionSchema.getTuningConfig().getChatHandlerTimeout(), + ingestionSchema.getTuningConfig().getChatHandlerNumRetries() + ); + + final HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) ingestionSchema + .getTuningConfig().getGivenOrDefaultPartitionsSpec(); + + final File persistDir = toolbox.getPersistDir(); + FileUtils.deleteQuietly(persistDir); + FileUtils.forceMkdir(persistDir); + + final Set pushedSegments = mergeAndPushSegments( + toolbox, + partitionsSpec, + persistDir, + intervalToVersion, + intervalToUnzippedFiles + ); + + taskClient.report(supervisorTaskId, new PushedSegmentsReport(getId(), Collections.emptySet(), pushedSegments)); + + return TaskStatus.success(getId()); + } + + private Map>> fetchSegmentFiles( + TaskToolbox toolbox, + Map>> intervalToPartitions + ) throws IOException + { + final File tempDir = toolbox.getFirehoseTemporaryDir(); + FileUtils.deleteQuietly(tempDir); + FileUtils.forceMkdir(tempDir); + + final Map>> intervalToUnzippedFiles = new HashMap<>(); + // Fetch partition files + for (Entry>> entryPerInterval : intervalToPartitions.entrySet()) { + final Interval interval = entryPerInterval.getKey(); + for (Int2ObjectMap.Entry> entryPerPartitionId : + entryPerInterval.getValue().int2ObjectEntrySet()) { + final int partitionId = entryPerPartitionId.getIntKey(); + final File partitionDir = FileUtils.getFile( + tempDir, + interval.getStart().toString(), + interval.getEnd().toString(), + Integer.toString(partitionId) + ); + FileUtils.forceMkdir(partitionDir); + for (PartitionLocation location : entryPerPartitionId.getValue()) { + final File zippedFile = fetchSegmentFile(partitionDir, location); + try { + final File unzippedDir = new File(partitionDir, StringUtils.format("unzipped_%s", location.getSubTaskId())); + FileUtils.forceMkdir(unzippedDir); + CompressionUtils.unzip(zippedFile, unzippedDir); + intervalToUnzippedFiles.computeIfAbsent(interval, k -> new Int2ObjectOpenHashMap<>()) + .computeIfAbsent(partitionId, k -> new ArrayList<>()) + .add(unzippedDir); + } + finally { + if (!zippedFile.delete()) { + LOG.warn("Failed to delete temp file[%s]", zippedFile); + } + } + } + } + } + return intervalToUnzippedFiles; + } + + @VisibleForTesting + File fetchSegmentFile(File partitionDir, PartitionLocation location) throws IOException + { + final File zippedFile = new File(partitionDir, StringUtils.format("temp_%s", location.getSubTaskId())); + final URI uri = location.toIntermediaryDataServerURI(supervisorTaskId); + org.apache.druid.java.util.common.FileUtils.copyLarge( + uri, + u -> u.toURL().openStream(), + zippedFile, + buffer, + t -> t instanceof IOException, + NUM_FETCH_RETRIES, + StringUtils.format("Failed to fetch file[%s]", uri) + ); + return zippedFile; + } + + private Set mergeAndPushSegments( + TaskToolbox toolbox, + HashedPartitionsSpec partitionsSpec, + File persistDir, + Map intervalToVersion, + Map>> intervalToUnzippedFiles + ) throws Exception + { + final DataSegmentPusher segmentPusher = toolbox.getSegmentPusher(); + final Set pushedSegments = new HashSet<>(); + for (Entry>> entryPerInterval : intervalToUnzippedFiles.entrySet()) { + final Interval interval = entryPerInterval.getKey(); + for (Int2ObjectMap.Entry> entryPerPartitionId : entryPerInterval.getValue().int2ObjectEntrySet()) { + final int partitionId = entryPerPartitionId.getIntKey(); + final List segmentFilesToMerge = entryPerPartitionId.getValue(); + final Pair> mergedFileAndDimensionNames = mergeSegmentsInSamePartition( + ingestionSchema, + toolbox.getIndexIO(), + toolbox.getIndexMergerV9(), + segmentFilesToMerge, + ingestionSchema.getTuningConfig().getMaxNumSegmentsToMerge(), + persistDir, + 0 + ); + final List metricNames = Arrays.stream(ingestionSchema.getDataSchema().getAggregators()) + .map(AggregatorFactory::getName) + .collect(Collectors.toList()); + + // Retry pushing segments because uploading to deep storage might fail especially for cloud storage types + final DataSegment segment = RetryUtils.retry( + () -> segmentPusher.push( + mergedFileAndDimensionNames.lhs, + new DataSegment( + getDataSource(), + interval, + Preconditions.checkNotNull(intervalToVersion.get(interval), "version for interval[%s]", interval), + null, // will be filled in the segmentPusher + mergedFileAndDimensionNames.rhs, + metricNames, + new HashBasedNumberedShardSpec( + partitionId, + Preconditions.checkNotNull(partitionsSpec.getNumShards(), "numShards"), + partitionsSpec.getPartitionDimensions(), + toolbox.getObjectMapper() + ), + null, // will be filled in the segmentPusher + 0 // will be filled in the segmentPusher + ), + false + ), + exception -> exception instanceof Exception, + 5 + ); + pushedSegments.add(segment); + } + } + return pushedSegments; + } + + private static Pair> mergeSegmentsInSamePartition( + PartialSegmentMergeIngestionSpec ingestionSpec, + IndexIO indexIO, + IndexMergerV9 merger, + List indexes, + int maxNumSegmentsToMerge, + File baseOutDir, + int outDirSuffix + ) throws IOException + { + int suffix = outDirSuffix; + final List mergedFiles = new ArrayList<>(); + List dimensionNames = null; + for (int i = 0; i < indexes.size(); i += maxNumSegmentsToMerge) { + final List filesToMerge = indexes.subList(i, Math.min(i + maxNumSegmentsToMerge, indexes.size())); + final List indexesToMerge = new ArrayList<>(filesToMerge.size()); + final Closer indexCleaner = Closer.create(); + for (File file : filesToMerge) { + final QueryableIndex queryableIndex = indexIO.loadIndex(file); + indexesToMerge.add(queryableIndex); + indexCleaner.register(() -> { + queryableIndex.close(); + file.delete(); + }); + } + if (maxNumSegmentsToMerge >= indexes.size()) { + dimensionNames = IndexMerger.getMergedDimensionsFromQueryableIndexes(indexesToMerge); + } + final File outDir = new File(baseOutDir, StringUtils.format("merged_%d", suffix++)); + mergedFiles.add( + merger.mergeQueryableIndex( + indexesToMerge, + ingestionSpec.getDataSchema().getGranularitySpec().isRollup(), + ingestionSpec.getDataSchema().getAggregators(), + outDir, + ingestionSpec.getTuningConfig().getIndexSpec(), + ingestionSpec.getTuningConfig().getSegmentWriteOutMediumFactory() + ) + ); + + indexCleaner.close(); + } + + if (mergedFiles.size() == 1) { + return Pair.of(mergedFiles.get(0), Preconditions.checkNotNull(dimensionNames, "dimensionNames")); + } else { + return mergeSegmentsInSamePartition( + ingestionSpec, + indexIO, + merger, + mergedFiles, + maxNumSegmentsToMerge, + baseOutDir, + suffix + ); + } + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java new file mode 100644 index 000000000000..b7155367dd16 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.StringUtils; +import org.joda.time.Interval; + +import java.net.URI; + +/** + * This class represents the intermediary data server where the partition of {@link #interval} and {@link #partitionId} + * is stored. + */ +public class PartitionLocation +{ + private final String host; + private final int port; + private final boolean useHttps; + private final String subTaskId; + private final Interval interval; + private final int partitionId; + + @JsonCreator + public PartitionLocation( + @JsonProperty("host") String host, + @JsonProperty("port") int port, + @JsonProperty("useHttps") boolean useHttps, + @JsonProperty("subTaskId") String subTaskId, + @JsonProperty("interval") Interval interval, + @JsonProperty("partitionId") int partitionId + ) + { + this.host = host; + this.port = port; + this.useHttps = useHttps; + this.subTaskId = subTaskId; + this.interval = interval; + this.partitionId = partitionId; + } + + @JsonProperty + public String getHost() + { + return host; + } + + @JsonProperty + public int getPort() + { + return port; + } + + @JsonProperty + public boolean isUseHttps() + { + return useHttps; + } + + @JsonProperty + public String getSubTaskId() + { + return subTaskId; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public int getPartitionId() + { + return partitionId; + } + + URI toIntermediaryDataServerURI(String supervisorTaskId) + { + return URI.create( + StringUtils.format( + "%s://%s:%d/druid/worker/v1/shuffle/task/%s/%s/partition?startTime=%s&endTime=%s&partitionId=%d", + useHttps ? "https" : "http", + host, + port, + StringUtils.urlEncode(supervisorTaskId), + StringUtils.urlEncode(subTaskId), + interval.getStart(), + interval.getEnd(), + partitionId + ) + ); + } + + @Override + public String toString() + { + return "PartitionLocation{" + + "host='" + host + '\'' + + ", port=" + port + + ", useHttps=" + useHttps + + ", subTaskId='" + subTaskId + '\'' + + ", interval=" + interval + + ", partitionId=" + partitionId + + '}'; + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java new file mode 100644 index 000000000000..010e84f7f8c1 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * Statistics about a partition created by {@link PartialSegmentGenerateTask}. Each partition is a set of data + * of the same time chunk (primary partition key) and the same partitionId (secondary partition key). This class + * holds the statistics of a single partition created by a task. + */ +public class PartitionStat +{ + // Host and port of the task executor + private final String taskExecutorHost; + private final int taskExecutorPort; + private final boolean useHttps; + + // Primary partition key + private final Interval interval; + // Secondary partition key + private final int partitionId; + + // numRows and sizeBytes are always null currently and will be filled properly in the future. + @Nullable + private final Integer numRows; + @Nullable + private final Long sizeBytes; + + @JsonCreator + public PartitionStat( + @JsonProperty("taskExecutorHost") String taskExecutorHost, + @JsonProperty("taskExecutorPort") int taskExecutorPort, + @JsonProperty("useHttps") boolean useHttps, + @JsonProperty("interval") Interval interval, + @JsonProperty("partitionId") int partitionId, + @JsonProperty("numRows") @Nullable Integer numRows, + @JsonProperty("sizeBytes") @Nullable Long sizeBytes + ) + { + this.taskExecutorHost = taskExecutorHost; + this.taskExecutorPort = taskExecutorPort; + this.useHttps = useHttps; + this.interval = interval; + this.partitionId = partitionId; + this.numRows = numRows == null ? 0 : numRows; + this.sizeBytes = sizeBytes == null ? 0 : sizeBytes; + } + + @JsonProperty + public String getTaskExecutorHost() + { + return taskExecutorHost; + } + + @JsonProperty + public int getTaskExecutorPort() + { + return taskExecutorPort; + } + + @JsonProperty + public boolean isUseHttps() + { + return useHttps; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + + @JsonProperty + public int getPartitionId() + { + return partitionId; + } + + @Nullable + @JsonProperty + public Integer getNumRows() + { + return numRows; + } + + @Nullable + @JsonProperty + public Long getSizeBytes() + { + return sizeBytes; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PartitionStat that = (PartitionStat) o; + return taskExecutorPort == that.taskExecutorPort && + useHttps == that.useHttps && + partitionId == that.partitionId && + Objects.equals(taskExecutorHost, that.taskExecutorHost) && + Objects.equals(interval, that.interval) && + Objects.equals(numRows, that.numRows) && + Objects.equals(sizeBytes, that.sizeBytes); + } + + @Override + public int hashCode() + { + return Objects.hash(taskExecutorHost, taskExecutorPort, useHttps, interval, partitionId, numRows, sizeBytes); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java index e83d9b025c09..8654172c572c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PushedSegmentsReport.java @@ -24,15 +24,18 @@ import com.google.common.base.Preconditions; import org.apache.druid.timeline.DataSegment; +import java.util.Objects; import java.util.Set; /** - * 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. + * In the last phase of native parallel batch indexing, each sub task 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 +public class PushedSegmentsReport implements SubTaskReport { + public static final String TYPE = "pushed_segments"; + private final String taskId; private final Set oldSegments; private final Set newSegments; @@ -49,6 +52,7 @@ public PushedSegmentsReport( this.newSegments = Preconditions.checkNotNull(newSegments, "newSegments"); } + @Override @JsonProperty public String getTaskId() { @@ -66,4 +70,25 @@ public Set getNewSegments() { return newSegments; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PushedSegmentsReport that = (PushedSegmentsReport) o; + return Objects.equals(taskId, that.taskId) && + Objects.equals(oldSegments, that.oldSegments) && + Objects.equals(newSegments, that.newSegments); + } + + @Override + public int hashCode() + { + return Objects.hash(taskId, oldSegments, newSegments); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index 79648a3f6301..9139e797def4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -20,77 +20,27 @@ package org.apache.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 org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.data.input.InputSplit; -import org.apache.druid.indexer.TaskState; -import org.apache.druid.indexer.TaskStatusPlus; -import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction; -import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.MonitorEntry; -import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; -import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker; -import org.apache.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.HashSet; 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.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. + * monitor multiple {@link SinglePhaseSubTask}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 +class SinglePhaseParallelIndexTaskRunner + extends ParallelIndexPhaseRunner { - 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 context; private final FiniteFirehoseFactory baseFirehoseFactory; - private final int maxNumTasks; - private final IndexingServiceClient indexingServiceClient; - - private final BlockingQueue> taskCompleteEvents = - new LinkedBlockingDeque<>(); - - /** - * subTaskId -> report - */ - private final ConcurrentHashMap segmentsMap = new ConcurrentHashMap<>(); - - private volatile boolean subTaskScheduleAndMonitorStopped; - private volatile TaskMonitor taskMonitor; - - private int nextSpecId = 0; SinglePhaseParallelIndexTaskRunner( TaskToolbox toolbox, @@ -101,154 +51,22 @@ public class SinglePhaseParallelIndexTaskRunner implements ParallelIndexTaskRunn IndexingServiceClient indexingServiceClient ) { - this.toolbox = toolbox; - this.taskId = taskId; - this.groupId = groupId; + super( + toolbox, + taskId, + groupId, + ingestionSchema.getTuningConfig(), + context, + indexingServiceClient + ); 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 - { - if (baseFirehoseFactory.getNumSplits() == 0) { - log.warn("There's no input split to process"); - return TaskState.SUCCESS; - } - - final Iterator 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 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) { - subTaskScheduleAndMonitorStopped = 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; - subTaskScheduleAndMonitorStopped = 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 { - stopInternal(); - if (!state.isComplete()) { - state = TaskState.FAILED; - } - } - - return state; } @Override - public void stopGracefully() + public String getName() { - subTaskScheduleAndMonitorStopped = true; - stopInternal(); - } - - /** - * Stop task scheduling and monitoring, and kill all running tasks. - * This method is thread-safe. - */ - private void stopInternal() - { - log.info("Cleaning up resources"); - - taskCompleteEvents.clear(); - if (taskMonitor != null) { - taskMonitor.stop(); - } - } - - private boolean isRunning() - { - return !subTaskScheduleAndMonitorStopped && !Thread.currentThread().isInterrupted(); - } - - @VisibleForTesting - TaskToolbox getToolbox() - { - return toolbox; + return SinglePhaseSubTask.TYPE; } @VisibleForTesting @@ -258,234 +76,25 @@ ParallelIndexIngestionSpec getIngestionSchema() } @VisibleForTesting - @Nullable - TaskMonitor getTaskMonitor() - { - return taskMonitor; - } - - @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.getNewSegments().equals(report.getNewSegments()), - "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 getRunningTaskIds() - { - return taskMonitor == null ? Collections.emptySet() : taskMonitor.getRunningTaskIds(); - } - - @Override - public List> getSubTaskSpecs() - { - if (taskMonitor != null) { - final List> runningSubTaskSpecs = taskMonitor.getRunningSubTaskSpecs(); - final List> completeSubTaskSpecs = taskMonitor - .getCompleteSubTaskSpecs(); - // Deduplicate subTaskSpecs because some subTaskSpec might exist both in runningSubTaskSpecs and - // completeSubTaskSpecs. - final Map> 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> getRunningSubTaskSpecs() - { - return taskMonitor == null ? Collections.emptyList() : taskMonitor.getRunningSubTaskSpecs(); - } - - @Override - public List> getCompleteSubTaskSpecs() - { - return taskMonitor == null ? Collections.emptyList() : taskMonitor.getCompleteSubTaskSpecs(); - } - - @Nullable - @Override - public SubTaskSpec 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 taskHistory = taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId); - final SubTaskSpec 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) + Iterator> subTaskSpecIterator() throws IOException { - 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 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; - } + return baseFirehoseFactory.getSplits().map(this::newTaskSpec).iterator(); } - @Nullable @Override - public TaskHistory getCompleteSubTaskSpecAttemptHistory(String subTaskSpecId) + int getTotalNumSubTasks() throws IOException { - if (taskMonitor == null) { - return null; - } else { - return taskMonitor.getCompleteSubTaskSpecHistory(subTaskSpecId); - } - } - - private void publish(TaskToolbox toolbox) throws IOException - { - final UsedSegmentChecker usedSegmentChecker = new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()); - final Set oldSegments = new HashSet<>(); - final Set newSegments = new HashSet<>(); - segmentsMap - .values() - .forEach(report -> { - oldSegments.addAll(report.getOldSegments()); - newSegments.addAll(report.getNewSegments()); - }); - final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToPublish, commitMetadata) -> - toolbox.getTaskActionClient().submit( - SegmentTransactionalInsertAction.overwriteAction(segmentsToBeOverwritten, segmentsToPublish) - ); - final boolean published = newSegments.isEmpty() - || publisher.publishSegments(oldSegments, newSegments, null).isSuccess(); - - if (published) { - log.info("Published [%d] segments", newSegments.size()); - } else { - log.info("Transaction failure while publishing segments, checking if someone else beat us to it."); - final Set segmentsIdentifiers = segmentsMap - .values() - .stream() - .flatMap(report -> report.getNewSegments().stream()) - .map(SegmentIdWithShardSpec::fromDataSegment) - .collect(Collectors.toSet()); - if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers) - .equals(newSegments)) { - log.info("Our segments really do exist, awaiting handoff."); - } else { - throw new ISE("Failed to publish segments[%s]", newSegments); - } - } - } - - private void submitNewTask( - TaskMonitor taskMonitor, - ParallelIndexSubTaskSpec spec - ) - { - log.info("Submit a new task for spec[%s] and inputSplit[%s]", spec.getId(), spec.getInputSplit()); - final ListenableFuture> future = taskMonitor.submit(spec); - Futures.addCallback( - future, - new FutureCallback>() - { - @Override - public void onSuccess(SubTaskCompleteEvent 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)); - } - } - ); + return baseFirehoseFactory.getNumSplits(); } @VisibleForTesting - int getAndIncrementNextSpecId() + SubTaskSpec newTaskSpec(InputSplit split) { - return nextSpecId++; - } - - @VisibleForTesting - Stream subTaskSpecIterator() throws IOException - { - return baseFirehoseFactory.getSplits().map(this::newTaskSpec); - } - - @VisibleForTesting - ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) - { - return new ParallelIndexSubTaskSpec( - taskId + "_" + getAndIncrementNextSpecId(), - groupId, - taskId, + return new SinglePhaseSubTaskSpec( + getTaskId() + "_" + getAndIncrementNextSpecId(), + getGroupId(), + getTaskId(), new ParallelIndexIngestionSpec( ingestionSchema.getDataSchema(), new ParallelIndexIOConfig( @@ -494,18 +103,8 @@ ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) ), ingestionSchema.getTuningConfig() ), - context, + getContext(), split ); } - - private static List 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()); - } - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java similarity index 92% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 3e3cf80bc180..e55c4731fa76 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -32,13 +32,13 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; -import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.SegmentAllocateAction; import org.apache.druid.indexing.common.actions.SurrogateAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.task.AbstractBatchIndexTask; +import org.apache.druid.indexing.common.task.BatchAppenderators; import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.IndexTask; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; @@ -89,15 +89,15 @@ import java.util.stream.Collectors; /** - * 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 + * The worker task of {@link SinglePhaseParallelIndexTaskRunner}. Similar to {@link IndexTask}, but this task + * generates and pushes segments, and reports them to the {@link SinglePhaseParallelIndexTaskRunner} instead of * publishing on its own. */ -public class ParallelIndexSubTask extends AbstractBatchIndexTask +public class SinglePhaseSubTask extends AbstractBatchIndexTask { - public static final String TYPE = "index_sub"; + public static final String TYPE = "single_phase_sub_task"; - private static final Logger LOG = new Logger(ParallelIndexSubTask.class); + private static final Logger LOG = new Logger(SinglePhaseSubTask.class); private final int numAttempts; private final ParallelIndexIngestionSpec ingestionSchema; @@ -121,7 +121,7 @@ public class ParallelIndexSubTask extends AbstractBatchIndexTask private final boolean missingIntervalsInOverwriteMode; @JsonCreator - public ParallelIndexSubTask( + public SinglePhaseSubTask( // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask @JsonProperty("id") @Nullable final String id, @JsonProperty("groupId") final String groupId, @@ -237,7 +237,7 @@ public TaskStatus runTask(final TaskToolbox toolbox) throws Exception .flatMap(holder -> holder.getObject().stream()) .map(PartitionChunk::getObject) .collect(Collectors.toSet()); - taskClient.report(supervisorTaskId, oldSegments, pushedSegments); + taskClient.report(supervisorTaskId, new PushedSegmentsReport(getId(), oldSegments, pushedSegments)); return TaskStatus.success(getId()); } @@ -405,17 +405,23 @@ private Set generateAndPushSegments( ); } - // Initialize maxRowsPerSegment and maxTotalRows lazily final ParallelIndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); final DynamicPartitionsSpec partitionsSpec = (DynamicPartitionsSpec) tuningConfig.getGivenOrDefaultPartitionsSpec(); final long pushTimeout = tuningConfig.getPushTimeout(); final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent(); final SegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient); - final Appenderator appenderator = newAppenderator(fireDepartmentMetrics, toolbox, dataSchema, tuningConfig); + final Appenderator appenderator = BatchAppenderators.newAppenderator( + getId(), + appenderatorsManager, + fireDepartmentMetrics, + toolbox, + dataSchema, + tuningConfig + ); boolean exceptionOccurred = false; try ( - final BatchAppenderatorDriver driver = newDriver(appenderator, toolbox, segmentAllocator); + final BatchAppenderatorDriver driver = BatchAppenderators.newDriver(appenderator, toolbox, segmentAllocator); final Firehose firehose = firehoseFactory.connect(dataSchema.getParser(), firehoseTempDir) ) { driver.startJob(); @@ -503,37 +509,4 @@ private Set generateAndPushSegments( } } } - - private Appenderator newAppenderator( - FireDepartmentMetrics metrics, - TaskToolbox toolbox, - DataSchema dataSchema, - ParallelIndexTuningConfig tuningConfig - ) - { - return appenderatorsManager.createOfflineAppenderatorForTask( - getId(), - 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() - ); - } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpec.java similarity index 90% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpec.java index 13938651b32c..8df941c806ca 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTaskSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpec.java @@ -26,12 +26,12 @@ import java.util.Map; -class ParallelIndexSubTaskSpec extends SubTaskSpec +class SinglePhaseSubTaskSpec extends SubTaskSpec { private final ParallelIndexIngestionSpec ingestionSpec; @JsonCreator - ParallelIndexSubTaskSpec( + SinglePhaseSubTaskSpec( String id, String groupId, String supervisorTaskId, @@ -51,9 +51,9 @@ public ParallelIndexIngestionSpec getIngestionSpec() } @Override - public ParallelIndexSubTask newSubTask(int numAttempts) + public SinglePhaseSubTask newSubTask(int numAttempts) { - return new ParallelIndexSubTask( + return new SinglePhaseSubTask( null, getGroupId(), null, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java new file mode 100644 index 000000000000..ec26cc8c1103 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonSubTypes.Type; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeInfo.Id; + +/** + * Each sub task of {@link ParallelIndexSupervisorTask} reports the result of indexing using this class. + */ +@JsonTypeInfo(use = Id.NAME, property = "type", defaultImpl = PushedSegmentsReport.class) +@JsonSubTypes(value = { + @Type(name = PushedSegmentsReport.TYPE, value = PushedSegmentsReport.class), + @Type(name = GeneratedPartitionsReport.TYPE, value = GeneratedPartitionsReport.class) +}) +public interface SubTaskReport +{ + String getTaskId(); +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java index ff1683668b25..33cc804d0565 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java @@ -136,7 +136,7 @@ public void start(long taskStatusCheckingPeriod) log.warn("task[%s] failed!", taskId); if (monitorEntry.numTries() < maxRetry) { log.info( - "We still have chances[%d/%d] to complete for spec[%s].", + "We still have more chances[%d/%d] to process the spec[%s].", monitorEntry.numTries(), maxRetry, monitorEntry.spec.getId() @@ -181,33 +181,41 @@ public void start(long taskStatusCheckingPeriod) public void stop() { synchronized (startStopLock) { - running = false; - taskStatusChecker.shutdownNow(); - - if (numRunningTasks > 0) { - final Iterator iterator = runningTasks.values().iterator(); - while (iterator.hasNext()) { - final MonitorEntry entry = iterator.next(); - iterator.remove(); - final String taskId = entry.runningTask.getId(); - log.info("Request to kill subtask[%s]", taskId); - indexingServiceClient.killTask(taskId); - numRunningTasks--; - numKilledTasks++; - } + if (running) { + running = false; + taskStatusChecker.shutdownNow(); if (numRunningTasks > 0) { - log.warn( - "Inconsistent state: numRunningTasks[%d] is still not zero after trying to kill all running tasks.", - numRunningTasks - ); + final Iterator iterator = runningTasks.values().iterator(); + while (iterator.hasNext()) { + final MonitorEntry entry = iterator.next(); + iterator.remove(); + final String taskId = entry.runningTask.getId(); + log.info("Request to kill subtask[%s]", taskId); + indexingServiceClient.killTask(taskId); + numRunningTasks--; + numKilledTasks++; + } + + if (numRunningTasks > 0) { + log.warn( + "Inconsistent state: numRunningTasks[%d] is still not zero after trying to kill all running tasks.", + numRunningTasks + ); + } } - } - log.info("Stopped taskMonitor"); + log.info("Stopped taskMonitor"); + } } } + /** + * Submits a {@link SubTaskSpec} to process to this TaskMonitor. TaskMonitor can issue one or more tasks to process + * the given spec. The returned future is done when + * 1) a sub task successfully processed the given spec or + * 2) the last sub task for the spec failed after all retries were exhausted. + */ public ListenableFuture> submit(SubTaskSpec spec) { synchronized (startStopLock) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java index 3af2ebb102d1..009e57c9fd39 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ThreadingTaskRunner.java @@ -61,7 +61,6 @@ import javax.annotation.Nullable; import java.io.File; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -126,7 +125,7 @@ public ThreadingTaskRunner( } @Override - public Optional streamTaskLog(String taskid, long offset) throws IOException + public Optional streamTaskLog(String taskid, long offset) { // task logs will appear in the main indexer log, streaming individual task logs is not supported return Optional.absent(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index bcd4e312f682..c21e2849574b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -267,6 +267,7 @@ public boolean isSkipSequenceNumberAvailabilityCheck() return skipSequenceNumberAvailabilityCheck; } + @Override public abstract SeekableStreamIndexTaskTuningConfig withBasePersistDirectory(File dir); @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java index 95acb22536ff..5b6078f74f9b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java @@ -49,8 +49,6 @@ import java.io.IOException; import java.nio.file.Path; import java.nio.file.Paths; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -202,7 +200,7 @@ private void discoverSupervisorTaskPartitions() } } numDiscovered.increment(); - return DateTimes.nowUtc().plus(intermediaryPartitionTimeout); + return getExpiryTimeFromNow(); } ); } @@ -236,21 +234,23 @@ private void deleteExpiredSuprevisorTaskPartitionsIfNotRunning() throws Interrup LOG.info("Found [%s] expired supervisor tasks", expiredSupervisorTasks.size()); - final Map taskStatuses = indexingServiceClient.getTaskStatuses(expiredSupervisorTasks); - for (Entry entry : taskStatuses.entrySet()) { - final String supervisorTaskId = entry.getKey(); - final TaskStatus status = entry.getValue(); - if (status.getStatusCode().isComplete()) { - // If it's finished, clean up all partitions for the supervisor task. - try { - deletePartitions(supervisorTaskId); - } - catch (IOException e) { - LOG.warn(e, "Failed to delete partitions for task[%s]", supervisorTaskId); + if (!expiredSupervisorTasks.isEmpty()) { + final Map taskStatuses = indexingServiceClient.getTaskStatuses(expiredSupervisorTasks); + for (Entry entry : taskStatuses.entrySet()) { + final String supervisorTaskId = entry.getKey(); + final TaskStatus status = entry.getValue(); + if (status.getStatusCode().isComplete()) { + // If it's finished, clean up all partitions for the supervisor task. + try { + deletePartitions(supervisorTaskId); + } + catch (IOException e) { + LOG.warn(e, "Failed to delete partitions for task[%s]", supervisorTaskId); + } + } else { + // If it's still running, update last access time. + supervisorTaskCheckTimes.put(supervisorTaskId, getExpiryTimeFromNow()); } - } else { - // If it's still running, update last access time. - supervisorTaskCheckTimes.put(supervisorTaskId, DateTimes.nowUtc()); } } } @@ -328,18 +328,33 @@ long addSegment(String supervisorTaskId, String subTaskId, DataSegment segment, } } - public List findPartitionFiles(String supervisorTaskId, Interval interval, int partitionId) + @Nullable + public File findPartitionFile(String supervisorTaskId, String subTaskId, Interval interval, int partitionId) { for (StorageLocation location : shuffleDataLocations) { final File partitionDir = new File(location.getPath(), getPartitionDir(supervisorTaskId, interval, partitionId)); if (partitionDir.exists()) { - supervisorTaskCheckTimes.put(supervisorTaskId, DateTimes.nowUtc()); + supervisorTaskCheckTimes.put(supervisorTaskId, getExpiryTimeFromNow()); final File[] segmentFiles = partitionDir.listFiles(); - return segmentFiles == null ? Collections.emptyList() : Arrays.asList(segmentFiles); + if (segmentFiles == null) { + return null; + } else { + for (File segmentFile : segmentFiles) { + if (segmentFile.getName().equals(subTaskId)) { + return segmentFile; + } + } + return null; + } } } - return Collections.emptyList(); + return null; + } + + private DateTime getExpiryTimeFromNow() + { + return DateTimes.nowUtc().plus(intermediaryPartitionTimeout); } public void deletePartitions(String supervisorTaskId) throws IOException diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java index 9bf197c20f58..d1adcb9f2d22 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java @@ -42,7 +42,6 @@ import java.io.File; import java.io.FileInputStream; import java.io.IOException; -import java.util.List; /** * HTTP endpoints for shuffle system. The MiddleManager and Indexer use this resource to serve intermediary shuffle @@ -69,26 +68,29 @@ public ShuffleResource(IntermediaryDataManager intermediaryDataManager) } @GET - @Path("/task/{supervisorTaskId}/partition") + @Path("/task/{supervisorTaskId}/{subTaskId}/partition") @Produces(MediaType.APPLICATION_OCTET_STREAM) public Response getPartition( @PathParam("supervisorTaskId") String supervisorTaskId, + @PathParam("subTaskId") String subTaskId, @QueryParam("startTime") String startTime, @QueryParam("endTime") String endTime, @QueryParam("partitionId") int partitionId ) { final Interval interval = new Interval(DateTimes.of(startTime), DateTimes.of(endTime)); - final List partitionFiles = intermediaryDataManager.findPartitionFiles( + final File partitionFile = intermediaryDataManager.findPartitionFile( supervisorTaskId, + subTaskId, interval, partitionId ); - if (partitionFiles.isEmpty()) { + if (partitionFile == null) { final String errorMessage = StringUtils.format( - "Can't find the partition for supervisor[%s], interval[%s], and partitionId[%s]", + "Can't find the partition for supervisorTask[%s], subTask[%s], interval[%s], and partitionId[%s]", supervisorTaskId, + subTaskId, interval, partitionId ); @@ -96,10 +98,8 @@ public Response getPartition( } else { return Response.ok( (StreamingOutput) output -> { - for (File partitionFile : partitionFiles) { - try (final FileInputStream fileInputStream = new FileInputStream(partitionFile)) { - ByteStreams.copy(fileInputStream, output); - } + try (final FileInputStream fileInputStream = new FileInputStream(partitionFile)) { + ByteStreams.copy(fileInputStream, output); } } ).build(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index 548d1a94aae2..9330396d3249 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -121,7 +121,8 @@ public void setUp() throws IOException null, null, null, - new NoopTestTaskReportFileWriter() + new NoopTestTaskReportFileWriter(), + null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 898411ae892b..691022e7bc9b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1626,7 +1626,8 @@ public List getLocations() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new SingleFileTaskReportFileWriter(reportsFile) + new SingleFileTaskReportFileWriter(reportsFile), + null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index c1d2598fa099..fbac1e42c40b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -722,7 +722,8 @@ public List getLocations() null, null, null, - new NoopTestTaskReportFileWriter() + new NoopTestTaskReportFileWriter(), + null ); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 6f6f871ac931..5d6ec71f49d9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1186,7 +1186,8 @@ private static class TestTaskToolbox extends TaskToolbox null, null, null, - new NoopTestTaskReportFileWriter() + new NoopTestTaskReportFileWriter(), + null ); this.segmentFileMap = segmentFileMap; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java index 2f36270cd399..f49a90f27b10 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java @@ -304,7 +304,8 @@ public ListenableFuture run(Task task) null, null, null, - new SingleFileTaskReportFileWriter(taskReportsFile) + new SingleFileTaskReportFileWriter(taskReportsFile), + null ); if (task.isReady(box.getTaskActionClient())) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index fbfa7a907280..7456ed446737 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -1000,7 +1000,8 @@ public List getLocations() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new NoopTestTaskReportFileWriter() + new NoopTestTaskReportFileWriter(), + null ); return toolboxFactory.build(task); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index f780428f7021..945316bb1769 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -20,14 +20,13 @@ package org.apache.druid.indexing.common.task.batch.parallel; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.client.indexing.TaskStatusResponse; -import org.apache.druid.data.input.FiniteFirehoseFactory; -import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.ParseSpec; @@ -40,6 +39,7 @@ import org.apache.druid.indexing.common.TaskInfoProvider; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMetersFactory; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.IngestionTestBase; @@ -47,19 +47,21 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; +import org.apache.druid.indexing.worker.IntermediaryDataManager; +import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; import org.apache.druid.segment.loading.NoopDataSegmentKiller; +import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider; import org.apache.druid.server.DruidNode; import org.apache.druid.server.security.AllowAllAuthorizer; import org.apache.druid.server.security.Authorizer; import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Duration; import org.junit.Rule; @@ -70,13 +72,12 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Iterator; import java.util.Map; -import java.util.Set; 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 { @@ -105,6 +106,27 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + private IntermediaryDataManager intermediaryDataManager; + + protected void initializeIntermeidaryDataManager() throws IOException + { + intermediaryDataManager = new IntermediaryDataManager( + new WorkerConfig(), + new TaskConfig( + null, + null, + null, + null, + null, + false, + null, + null, + ImmutableList.of(new StorageLocationConfig(temporaryFolder.newFolder(), null, null)) + ), + null + ); + } + class LocalIndexingServiceClient extends NoopIndexingServiceClient { private final ConcurrentMap> tasks = new ConcurrentHashMap<>(); @@ -115,7 +137,7 @@ class LocalIndexingServiceClient extends NoopIndexingServiceClient @Override public String runTask(Object taskObject) { - final ParallelIndexSubTask subTask = (ParallelIndexSubTask) taskObject; + final Task subTask = (Task) taskObject; tasks.put(subTask.getId(), service.submit(() -> { try { final TaskToolbox toolbox = createTaskToolbox(subTask); @@ -144,7 +166,7 @@ public TaskStatusResponse getTaskStatus(String taskId) taskId, new TaskStatusPlus( taskId, - "index_sub", + SinglePhaseSubTask.TYPE, DateTimes.EPOCH, DateTimes.EPOCH, taskStatus.getStatusCode(), @@ -160,7 +182,7 @@ public TaskStatusResponse getTaskStatus(String taskId) taskId, new TaskStatusPlus( taskId, - "index_sub", + SinglePhaseSubTask.TYPE, DateTimes.EPOCH, DateTimes.EPOCH, TaskState.RUNNING, @@ -174,14 +196,14 @@ public TaskStatusResponse getTaskStatus(String taskId) } } catch (InterruptedException | ExecutionException e) { - // We don't have a way to pass this exception to the supervisorTask yet.. + // We don't have a way to propagate 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", + SinglePhaseSubTask.TYPE, DateTimes.EPOCH, DateTimes.EPOCH, TaskState.FAILED, @@ -254,7 +276,8 @@ public File getStorageDirectory() null, null, null, - new NoopTestTaskReportFileWriter() + new NoopTestTaskReportFileWriter(), + intermediaryDataManager ); } @@ -289,9 +312,9 @@ public Authorizer getAuthorizer(String name) } } - static class TestParallelIndexTaskRunner extends SinglePhaseParallelIndexTaskRunner + static class TestSinglePhaseParallelIndexTaskRunner extends SinglePhaseParallelIndexTaskRunner { - TestParallelIndexTaskRunner( + TestSinglePhaseParallelIndexTaskRunner( TaskToolbox toolbox, String taskId, String groupId, @@ -311,21 +334,29 @@ static class TestParallelIndexTaskRunner extends SinglePhaseParallelIndexTaskRun } @Override - Stream subTaskSpecIterator() throws IOException + Iterator> 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); + final Iterator> iterator = super.subTaskSpecIterator(); + return new Iterator>() + { + @Override + public boolean hasNext() + { + return iterator.hasNext(); } - catch (InterruptedException e) { - throw new RuntimeException(e); + + @Override + public SubTaskSpec next() + { + try { + Thread.sleep(10); + return iterator.next(); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } } - return newTaskSpec((InputSplit) split); - }); + }; } } @@ -368,9 +399,9 @@ public SegmentIdWithShardSpec allocateSegment(String supervisorTaskId, DateTime } @Override - public void report(String supervisorTaskId, Set oldSegments, Set pushedSegments) + public void report(String supervisorTaskId, SubTaskReport report) { - supervisorTask.getRunner().collectReport(new PushedSegmentsReport(getSubtaskId(), oldSegments, pushedSegments)); + supervisorTask.getCurrentRunner().collectReport(report); } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java new file mode 100644 index 000000000000..b342adaeb74c --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingTest.java @@ -0,0 +1,530 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task.batch.parallel; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.common.task.TestAppenderatorsManager; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.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.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +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.List; +import java.util.Map; + +@RunWith(Parameterized.class) +public class MultiPhaseParallelIndexingTest extends AbstractParallelIndexSupervisorTaskTest +{ + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{LockGranularity.TIME_CHUNK}, + new Object[]{LockGranularity.SEGMENT} + ); + } + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private final LockGranularity lockGranularity; + private File inputDir; + + public MultiPhaseParallelIndexingTest(LockGranularity lockGranularity) + { + this.lockGranularity = lockGranularity; + } + + @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"); + initializeIntermeidaryDataManager(); + } + + @After + public void teardown() + { + indexingServiceClient.shutdown(); + temporaryFolder.delete(); + } + + @Test + public void testRun() throws Exception + { + runTestTask(Intervals.of("2017/2018"), Granularities.DAY); + } + + @Test + public void testMissingIntervals() throws Exception + { + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage( + "forceGuaranteedRollup is set but numShards is missing in partitionsSpec " + + "or intervals is missing in granularitySpec" + ); + runTestTask(null, Granularities.DAY); + } + + @Test + public void testMissingNumShards() + { + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage( + "forceGuaranteedRollup is set but numShards is missing in partitionsSpec " + + "or intervals is missing in granularitySpec" + ); + newTask( + Intervals.of("2017/2018"), + Granularities.DAY, + new ParallelIndexIOConfig(new LocalFirehoseFactory(inputDir, "test_*", null), false), + new ParallelIndexTuningConfig( + null, + null, + null, + null, + null, + null, + new HashedPartitionsSpec(null, null, null), + null, + null, + null, + true, + null, + null, + null, + 2, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + ); + } + + private void runTestTask(Interval interval, Granularity segmentGranularity) throws Exception + { + final ParallelIndexSupervisorTask task = newTask( + interval, + segmentGranularity, + new ParallelIndexIOConfig( + new LocalFirehoseFactory(inputDir, "test_*", null), + false + ) + ); + actionClient = createActionClient(task); + toolbox = createTaskToolbox(task); + + prepareTaskForLocking(task); + task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); + Assert.assertTrue(task.isReady(actionClient)); + Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); + shutdownTask(task); + } + + private ParallelIndexSupervisorTask newTask( + Interval interval, + Granularity segmentGranularity, + ParallelIndexIOConfig ioConfig + ) + { + return newTask( + interval, + segmentGranularity, + ioConfig, + new ParallelIndexTuningConfig( + null, + null, + null, + null, + null, + null, + new HashedPartitionsSpec(null, 2, null), + null, + null, + null, + true, + null, + null, + null, + 2, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + ); + } + + private ParallelIndexSupervisorTask newTask( + Interval interval, + Granularity segmentGranularity, + ParallelIndexIOConfig ioConfig, + ParallelIndexTuningConfig tuningConfig + ) + { + // set up ingestion spec + //noinspection unchecked + 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( + segmentGranularity, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ), + null, + getObjectMapper() + ), + ioConfig, + tuningConfig + ); + + // set up test tools + return new TestSupervisorTask( + null, + null, + ingestionSpec, + new HashMap<>(), + indexingServiceClient + ); + } + + + private static class TestSupervisorTask extends TestParallelIndexSupervisorTask + { + TestSupervisorTask( + String id, + TaskResource taskResource, + ParallelIndexIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient + ) + { + super(id, taskResource, ingestionSchema, context, indexingServiceClient); + } + + @Override + public PartialSegmentGenerateParallelIndexTaskRunner createPartialSegmentGenerateRunner(TaskToolbox toolbox) + { + return new TestPartialSegmentGenerateRunner(toolbox, this, getIndexingServiceClient()); + } + + @Override + public PartialSegmentMergeParallelIndexTaskRunner createPartialSegmentMergeRunner( + TaskToolbox toolbox, + List ioConfigs + ) + { + return new TestPartialSegmentMergeParallelIndexTaskRunner(toolbox, this, ioConfigs, getIndexingServiceClient()); + } + } + + private static class TestPartialSegmentGenerateRunner extends PartialSegmentGenerateParallelIndexTaskRunner + { + private final ParallelIndexSupervisorTask supervisorTask; + + private TestPartialSegmentGenerateRunner( + TaskToolbox toolbox, + ParallelIndexSupervisorTask supervisorTask, + IndexingServiceClient indexingServiceClient + ) + { + super( + toolbox, + supervisorTask.getId(), + supervisorTask.getGroupId(), + supervisorTask.getIngestionSchema(), + supervisorTask.getContext(), + indexingServiceClient + ); + this.supervisorTask = supervisorTask; + } + + @Override + Iterator> subTaskSpecIterator() throws IOException + { + final Iterator> iterator = super.subTaskSpecIterator(); + return new Iterator>() + { + @Override + public boolean hasNext() + { + return iterator.hasNext(); + } + + @Override + public SubTaskSpec next() + { + try { + Thread.sleep(10); + return iterator.next(); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + }; + } + + @Override + SubTaskSpec newTaskSpec(InputSplit split) + { + final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec( + getIngestionSchema().getDataSchema(), + new ParallelIndexIOConfig( + getBaseFirehoseFactory().withSplit(split), + getIngestionSchema().getIOConfig().isAppendToExisting() + ), + getIngestionSchema().getTuningConfig() + ); + return new SubTaskSpec( + getTaskId() + "_" + getAndIncrementNextSpecId(), + getGroupId(), + getTaskId(), + getContext(), + split + ) + { + @Override + public PartialSegmentGenerateTask newSubTask(int numAttempts) + { + return new PartialSegmentGenerateTask( + null, + getGroupId(), + null, + getSupervisorTaskId(), + numAttempts, + subTaskIngestionSpec, + getContext(), + getIndexingServiceClient(), + new LocalParallelIndexTaskClientFactory(supervisorTask), + new TestAppenderatorsManager() + ); + } + }; + } + } + + private static class TestPartialSegmentMergeParallelIndexTaskRunner extends PartialSegmentMergeParallelIndexTaskRunner + { + private final ParallelIndexSupervisorTask supervisorTask; + + private TestPartialSegmentMergeParallelIndexTaskRunner( + TaskToolbox toolbox, + ParallelIndexSupervisorTask supervisorTask, + List mergeIOConfigs, + IndexingServiceClient indexingServiceClient + ) + { + super( + toolbox, + supervisorTask.getId(), + supervisorTask.getGroupId(), + supervisorTask.getIngestionSchema().getDataSchema(), + mergeIOConfigs, + supervisorTask.getIngestionSchema().getTuningConfig(), + supervisorTask.getContext(), + indexingServiceClient + ); + this.supervisorTask = supervisorTask; + } + + @Override + Iterator> subTaskSpecIterator() + { + final Iterator> iterator = super.subTaskSpecIterator(); + return new Iterator>() + { + @Override + public boolean hasNext() + { + return iterator.hasNext(); + } + + @Override + public SubTaskSpec next() + { + try { + Thread.sleep(10); + return iterator.next(); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + }; + } + + @Override + SubTaskSpec newTaskSpec(PartialSegmentMergeIOConfig ioConfig) + { + final PartialSegmentMergeIngestionSpec ingestionSpec = new PartialSegmentMergeIngestionSpec( + supervisorTask.getIngestionSchema().getDataSchema(), + ioConfig, + getTuningConfig() + ); + return new SubTaskSpec( + getTaskId() + "_" + getAndIncrementNextSpecId(), + getGroupId(), + getTaskId(), + getContext(), + new InputSplit<>(ioConfig.getPartitionLocations()) + ) + { + @Override + public PartialSegmentMergeTask newSubTask(int numAttempts) + { + return new TestPartialSegmentMergeTask( + null, + getGroupId(), + null, + getSupervisorTaskId(), + numAttempts, + ingestionSpec, + getContext(), + getIndexingServiceClient(), + new LocalParallelIndexTaskClientFactory(supervisorTask), + getToolbox() + ); + } + }; + } + } + + private static class TestPartialSegmentMergeTask extends PartialSegmentMergeTask + { + private final TaskToolbox toolbox; + + private TestPartialSegmentMergeTask( + @Nullable String id, + String groupId, + TaskResource taskResource, + String supervisorTaskId, + int numAttempts, + PartialSegmentMergeIngestionSpec ingestionSchema, + Map context, + IndexingServiceClient indexingServiceClient, + IndexTaskClientFactory taskClientFactory, + TaskToolbox toolboxo + ) + { + super( + id, + groupId, + taskResource, + supervisorTaskId, + numAttempts, + ingestionSchema, + context, + indexingServiceClient, + taskClientFactory + ); + this.toolbox = toolboxo; + } + + @Override + File fetchSegmentFile(File partitionDir, PartitionLocation location) + { + final File zippedFile = toolbox.getIntermediaryDataManager().findPartitionFile( + getSupervisorTaskId(), + location.getSubTaskId(), + location.getInterval(), + location.getPartitionId() + ); + if (zippedFile == null) { + throw new ISE("Can't find segment file for location[%s] at path[%s]", location); + } + return zippedFile; + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index 8ddf6cc299fb..3f2e4970ae71 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -101,7 +101,7 @@ public void testStopGracefully() throws Exception Assert.assertTrue(task.isReady(actionClient)); final Future future = service.submit(() -> task.run(toolbox).getStatusCode()); - while (task.getRunner() == null) { + while (task.getCurrentRunner() == null) { Thread.sleep(100); } task.stopGracefully(null); @@ -109,7 +109,7 @@ public void testStopGracefully() throws Exception expectedException.expectCause(CoreMatchers.instanceOf(InterruptedException.class)); future.get(); - final TestParallelIndexTaskRunner runner = (TestParallelIndexTaskRunner) task.getRunner(); + final TestSinglePhaseParallelIndexTaskRunner runner = (TestSinglePhaseParallelIndexTaskRunner) task.getCurrentRunner(); Assert.assertTrue(runner.getRunningTaskIds().isEmpty()); // completeSubTaskSpecs should be empty because no task has reported its status to TaskMonitor Assert.assertTrue(runner.getCompleteSubTaskSpecs().isEmpty()); @@ -139,11 +139,11 @@ public void testSubTaskFail() throws Exception final TaskState state = task.run(toolbox).getStatusCode(); Assert.assertEquals(TaskState.FAILED, state); - final TestParallelIndexTaskRunner runner = (TestParallelIndexTaskRunner) task.getRunner(); + final TestSinglePhaseParallelIndexTaskRunner runner = (TestSinglePhaseParallelIndexTaskRunner) task.getCurrentRunner(); Assert.assertTrue(runner.getRunningTaskIds().isEmpty()); - final List> completeSubTaskSpecs = runner.getCompleteSubTaskSpecs(); + final List> completeSubTaskSpecs = runner.getCompleteSubTaskSpecs(); Assert.assertEquals(1, completeSubTaskSpecs.size()); - final TaskHistory history = runner.getCompleteSubTaskSpecAttemptHistory( + final TaskHistory history = runner.getCompleteSubTaskSpecAttemptHistory( completeSubTaskSpecs.get(0).getId() ); Assert.assertNotNull(history); @@ -207,6 +207,8 @@ private ParallelIndexSupervisorTask newTask( null, null, null, + null, + null, null ) ); @@ -292,20 +294,17 @@ private TestSupervisorTask( } @Override - ParallelIndexTaskRunner createRunner(TaskToolbox toolbox) + SinglePhaseParallelIndexTaskRunner createSinglePhaseTaskRunner(TaskToolbox toolbox) { - setRunner( - new TestRunner( - toolbox, - this, - indexingServiceClient - ) + return new TestRunner( + toolbox, + this, + indexingServiceClient ); - return getRunner(); } } - private static class TestRunner extends TestParallelIndexTaskRunner + private static class TestRunner extends TestSinglePhaseParallelIndexTaskRunner { private final ParallelIndexSupervisorTask supervisorTask; @@ -327,12 +326,12 @@ private TestRunner( } @Override - ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) + SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) { final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() .getIOConfig() .getFirehoseFactory(); - return new TestParallelIndexSubTaskSpec( + return new TestSinglePhaseSubTaskSpec( supervisorTask.getId() + "_" + getAndIncrementNextSpecId(), supervisorTask.getGroupId(), supervisorTask, @@ -350,11 +349,11 @@ ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) } } - private static class TestParallelIndexSubTaskSpec extends ParallelIndexSubTaskSpec + private static class TestSinglePhaseSubTaskSpec extends SinglePhaseSubTaskSpec { private final ParallelIndexSupervisorTask supervisorTask; - private TestParallelIndexSubTaskSpec( + private TestSinglePhaseSubTaskSpec( String id, String groupId, ParallelIndexSupervisorTask supervisorTask, @@ -368,9 +367,9 @@ private TestParallelIndexSubTaskSpec( } @Override - public ParallelIndexSubTask newSubTask(int numAttempts) + public SinglePhaseSubTask newSubTask(int numAttempts) { - return new TestParallelIndexSubTask( + return new TestSinglePhaseSubTask( null, getGroupId(), null, @@ -384,9 +383,9 @@ public ParallelIndexSubTask newSubTask(int numAttempts) } } - private static class TestParallelIndexSubTask extends ParallelIndexSubTask + private static class TestSinglePhaseSubTask extends SinglePhaseSubTask { - private TestParallelIndexSubTask( + private TestSinglePhaseSubTask( @Nullable String id, String groupId, TaskResource taskResource, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index f2b151e2521a..16ab8b105186 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -85,7 +85,7 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd private static final int NUM_SUB_TASKS = 10; /** specId -> spec */ - private final ConcurrentMap subTaskSpecs = new ConcurrentHashMap<>(); + private final ConcurrentMap subTaskSpecs = new ConcurrentHashMap<>(); /** specId -> taskStatusPlus */ private final ConcurrentMap runningSpecs = new ConcurrentHashMap<>(); @@ -94,7 +94,7 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd private final ConcurrentHashMap> taskHistories = new ConcurrentHashMap<>(); /** taskId -> subTaskSpec */ - private final ConcurrentMap taskIdToSpec = new ConcurrentHashMap<>(); + private final ConcurrentMap taskIdToSpec = new ConcurrentHashMap<>(); /** taskId -> task */ private final CopyOnWriteArrayList runningTasks = new CopyOnWriteArrayList<>(); @@ -137,7 +137,7 @@ public void testAPIs() throws Exception final Future supervisorTaskFuture = service.submit(() -> task.run(toolbox)); Thread.sleep(1000); - final SinglePhaseParallelIndexTaskRunner runner = (SinglePhaseParallelIndexTaskRunner) task.getRunner(); + final SinglePhaseParallelIndexTaskRunner runner = (SinglePhaseParallelIndexTaskRunner) task.getCurrentRunner(); Assert.assertNotNull("runner is null", runner); // test getMode @@ -306,8 +306,8 @@ private void checkState( // subTaskSpecs response = task.getSubTaskSpecs(newRequest()); Assert.assertEquals(200, response.getStatus()); - List> actualSubTaskSpecMap = - (List>) response.getEntity(); + List> actualSubTaskSpecMap = + (List>) response.getEntity(); Assert.assertEquals( subTaskSpecs.keySet(), actualSubTaskSpecMap.stream().map(SubTaskSpec::getId).collect(Collectors.toSet()) @@ -317,14 +317,14 @@ private void checkState( response = task.getRunningSubTaskSpecs(newRequest()); Assert.assertEquals(200, response.getStatus()); actualSubTaskSpecMap = - (List>) response.getEntity(); + (List>) response.getEntity(); Assert.assertEquals( runningSpecs.keySet(), actualSubTaskSpecMap.stream().map(SubTaskSpec::getId).collect(Collectors.toSet()) ); // completeSubTaskSpecs - final List> completeSubTaskSpecs = expectedSubTaskStateResponses + final List> completeSubTaskSpecs = expectedSubTaskStateResponses .entrySet() .stream() .filter(entry -> !runningSpecs.containsKey(entry.getKey())) @@ -339,8 +339,8 @@ private void checkState( final String subTaskId = runningSpecs.keySet().iterator().next(); response = task.getSubTaskSpec(subTaskId, newRequest()); Assert.assertEquals(200, response.getStatus()); - final SubTaskSpec subTaskSpec = - (SubTaskSpec) response.getEntity(); + final SubTaskSpec subTaskSpec = + (SubTaskSpec) response.getEntity(); Assert.assertEquals(subTaskId, subTaskSpec.getId()); // subTaskState @@ -438,6 +438,8 @@ private TestSupervisorTask newTask( null, null, null, + null, + null, null ) ); @@ -500,21 +502,17 @@ private class TestSupervisorTask extends TestParallelIndexSupervisorTask } @Override - ParallelIndexTaskRunner createRunner(TaskToolbox toolbox) + SinglePhaseParallelIndexTaskRunner createSinglePhaseTaskRunner(TaskToolbox toolbox) { - setToolbox(toolbox); - setRunner( - new TestRunner( - toolbox, - this, - indexingServiceClient - ) + return new TestRunner( + toolbox, + this, + indexingServiceClient ); - return getRunner(); } } - private class TestRunner extends TestParallelIndexTaskRunner + private class TestRunner extends TestSinglePhaseParallelIndexTaskRunner { private final ParallelIndexSupervisorTask supervisorTask; @@ -536,7 +534,7 @@ private class TestRunner extends TestParallelIndexTaskRunner } @Override - ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) + SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) { final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() .getIOConfig() @@ -561,7 +559,7 @@ ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) } } - private class TestSubTaskSpec extends ParallelIndexSubTaskSpec + private class TestSubTaskSpec extends SinglePhaseSubTaskSpec { private final ParallelIndexSupervisorTask supervisorTask; @@ -579,7 +577,7 @@ private class TestSubTaskSpec extends ParallelIndexSubTaskSpec } @Override - public ParallelIndexSubTask newSubTask(int numAttempts) + public SinglePhaseSubTask 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 @@ -622,7 +620,7 @@ public ParallelIndexSubTask newSubTask(int numAttempts) } } - private class TestSubTask extends ParallelIndexSubTask + private class TestSubTask extends SinglePhaseSubTask { private final IndexTaskClientFactory taskClientFactory; private volatile TaskState state = TaskState.RUNNING; @@ -682,7 +680,10 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception 1L ); - taskClient.report(getId(), Collections.emptySet(), Collections.singleton(segment)); + taskClient.report( + getSupervisorTaskId(), + new PushedSegmentsReport(getId(), Collections.emptySet(), Collections.singleton(segment)) + ); return TaskStatus.fromCode(getId(), state); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index 2f1c24f5e424..766651bdcfd0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -148,6 +148,8 @@ private ParallelIndexSupervisorTask newTask( null, null, null, + null, + null, null ) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java index d96cf544e497..7c6ad17da156 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java @@ -84,6 +84,8 @@ public void testSerdeWithMaxRowsPerSegment() 20, new Duration(3600), 128, + null, + null, false, null, null diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java similarity index 91% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index 7ff4bacead9c..1af7cc97cabd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -65,7 +65,7 @@ import java.util.stream.Collectors; @RunWith(Parameterized.class) -public class ParallelIndexSupervisorTaskTest extends AbstractParallelIndexSupervisorTaskTest +public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSupervisorTaskTest { @Parameterized.Parameters(name = "{0}") public static Iterable constructorFeeder() @@ -79,7 +79,7 @@ public static Iterable constructorFeeder() private final LockGranularity lockGranularity; private File inputDir; - public ParallelIndexSupervisorTaskTest(LockGranularity lockGranularity) + public SinglePhaseParallelIndexingTest(LockGranularity lockGranularity) { this.lockGranularity = lockGranularity; } @@ -131,12 +131,12 @@ public void testIsReady() throws Exception prepareTaskForLocking(task); Assert.assertTrue(task.isReady(actionClient)); - final SinglePhaseParallelIndexTaskRunner runner = (SinglePhaseParallelIndexTaskRunner) task.createRunner(toolbox); - final Iterator subTaskSpecIterator = runner.subTaskSpecIterator().iterator(); + final SinglePhaseParallelIndexTaskRunner runner = task.createSinglePhaseTaskRunner(toolbox); + final Iterator> subTaskSpecIterator = runner.subTaskSpecIterator(); while (subTaskSpecIterator.hasNext()) { - final ParallelIndexSubTaskSpec spec = subTaskSpecIterator.next(); - final ParallelIndexSubTask subTask = new ParallelIndexSubTask( + final SinglePhaseSubTaskSpec spec = (SinglePhaseSubTaskSpec) subTaskSpecIterator.next(); + final SinglePhaseSubTask subTask = new SinglePhaseSubTask( null, spec.getGroupId(), null, @@ -307,6 +307,8 @@ public void testWith1MaxNumSubTasks() throws Exception null, null, null, + null, + null, null ) ); @@ -317,7 +319,7 @@ public void testWith1MaxNumSubTasks() throws Exception task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertTrue(task.isReady(actionClient)); Assert.assertEquals(TaskState.SUCCESS, task.run(toolbox).getStatusCode()); - Assert.assertNull("Runner must be null if the task was in the sequential mode", task.getRunner()); + Assert.assertNull("Runner must be null if the task was in the sequential mode", task.getCurrentRunner()); } @Test @@ -376,6 +378,8 @@ private ParallelIndexSupervisorTask newTask( null, null, null, + null, + null, null ) ); @@ -427,8 +431,6 @@ private ParallelIndexSupervisorTask newTask( private static class TestSupervisorTask extends TestParallelIndexSupervisorTask { - private final IndexingServiceClient indexingServiceClient; - TestSupervisorTask( String id, TaskResource taskResource, @@ -437,36 +439,21 @@ private static class TestSupervisorTask extends TestParallelIndexSupervisorTask IndexingServiceClient indexingServiceClient ) { - super( - id, - taskResource, - ingestionSchema, - context, - indexingServiceClient - ); - this.indexingServiceClient = indexingServiceClient; + super(id, taskResource, ingestionSchema, context, indexingServiceClient); } @Override - ParallelIndexTaskRunner createRunner(TaskToolbox toolbox) + SinglePhaseParallelIndexTaskRunner createSinglePhaseTaskRunner(TaskToolbox toolbox) { - setToolbox(toolbox); - setRunner( - new TestRunner( - toolbox, - this, - indexingServiceClient - ) - ); - return getRunner(); + return new TestSinglePhaseRunner(toolbox, this, getIndexingServiceClient()); } } - private static class TestRunner extends TestParallelIndexTaskRunner + private static class TestSinglePhaseRunner extends TestSinglePhaseParallelIndexTaskRunner { private final ParallelIndexSupervisorTask supervisorTask; - TestRunner( + TestSinglePhaseRunner( TaskToolbox toolbox, ParallelIndexSupervisorTask supervisorTask, @Nullable IndexingServiceClient indexingServiceClient @@ -484,12 +471,12 @@ private static class TestRunner extends TestParallelIndexTaskRunner } @Override - ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) + SinglePhaseSubTaskSpec newTaskSpec(InputSplit split) { final FiniteFirehoseFactory baseFirehoseFactory = (FiniteFirehoseFactory) getIngestionSchema() .getIOConfig() .getFirehoseFactory(); - return new TestParallelIndexSubTaskSpec( + return new TestSinglePhaseSubTaskSpec( supervisorTask.getId() + "_" + getAndIncrementNextSpecId(), supervisorTask.getGroupId(), supervisorTask, @@ -507,11 +494,11 @@ ParallelIndexSubTaskSpec newTaskSpec(InputSplit split) } } - private static class TestParallelIndexSubTaskSpec extends ParallelIndexSubTaskSpec + private static class TestSinglePhaseSubTaskSpec extends SinglePhaseSubTaskSpec { private final ParallelIndexSupervisorTask supervisorTask; - TestParallelIndexSubTaskSpec( + TestSinglePhaseSubTaskSpec( String id, String groupId, ParallelIndexSupervisorTask supervisorTask, @@ -525,9 +512,9 @@ private static class TestParallelIndexSubTaskSpec extends ParallelIndexSubTaskSp } @Override - public ParallelIndexSubTask newSubTask(int numAttempts) + public SinglePhaseSubTask newSubTask(int numAttempts) { - return new ParallelIndexSubTask( + return new SinglePhaseSubTask( null, getGroupId(), null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 4ee4adef77df..eb7c6de297bb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -106,7 +106,8 @@ public void setup() throws IOException node, null, null, - new SingleFileTaskReportFileWriter(new File("fake")) + new SingleFileTaskReportFileWriter(new File("fake")), + null ); runner = new SingleTaskBackgroundRunner( toolboxFactory, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 475b8ef6f68f..af553fe11347 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -631,7 +631,8 @@ public void unannounceSegments(Iterable segments) EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new NoopTestTaskReportFileWriter() + new NoopTestTaskReportFileWriter(), + null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/IntermediaryDataManagerAutoCleanupTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/IntermediaryDataManagerAutoCleanupTest.java index 6a5f611fdf5a..7d0233b6b16d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/IntermediaryDataManagerAutoCleanupTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/IntermediaryDataManagerAutoCleanupTest.java @@ -115,13 +115,14 @@ public void teardown() throws InterruptedException public void testCleanup() throws IOException, InterruptedException { final String supervisorTaskId = "supervisorTaskId"; + final String subTaskId = "subTaskId"; final Interval interval = Intervals.of("2018/2019"); final File segmentFile = generateSegmentDir("test"); final DataSegment segment = newSegment(interval, 0); - intermediaryDataManager.addSegment(supervisorTaskId, "subTaskId", segment, segmentFile); + intermediaryDataManager.addSegment(supervisorTaskId, subTaskId, segment, segmentFile); Thread.sleep(3000); - Assert.assertTrue(intermediaryDataManager.findPartitionFiles(supervisorTaskId, interval, 0).isEmpty()); + Assert.assertNull(intermediaryDataManager.findPartitionFile(supervisorTaskId, subTaskId, interval, 0)); } private File generateSegmentDir(String fileName) throws IOException diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/IntermediaryDataManagerManualAddAndDeleteTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/IntermediaryDataManagerManualAddAndDeleteTest.java index 7d322d080d30..1e1eab4e7ff0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/IntermediaryDataManagerManualAddAndDeleteTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/IntermediaryDataManagerManualAddAndDeleteTest.java @@ -41,8 +41,6 @@ import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.Comparator; -import java.util.List; public class IntermediaryDataManagerManualAddAndDeleteTest { @@ -107,11 +105,14 @@ public void testFindPartitionFiles() throws IOException final DataSegment segment = newSegment(interval, partitionId); intermediaryDataManager.addSegment(supervisorTaskId, "subTaskId_" + i, segment, segmentFile); } - final List files = intermediaryDataManager.findPartitionFiles(supervisorTaskId, interval, partitionId); - Assert.assertEquals(4, files.size()); - files.sort(Comparator.comparing(File::getName)); for (int i = 0; i < 4; i++) { - Assert.assertEquals("subTaskId_" + i, files.get(i).getName()); + final File file = intermediaryDataManager.findPartitionFile( + supervisorTaskId, + "subTaskId_" + i, + interval, + partitionId + ); + Assert.assertNotNull(file); } } @@ -131,7 +132,11 @@ public void deletePartitions() throws IOException intermediaryDataManager.deletePartitions(supervisorTaskId); for (int partitionId = 0; partitionId < 2; partitionId++) { - Assert.assertTrue(intermediaryDataManager.findPartitionFiles(supervisorTaskId, interval, partitionId).isEmpty()); + for (int subTaskId = 0; subTaskId < 2; subTaskId++) { + Assert.assertNull( + intermediaryDataManager.findPartitionFile(supervisorTaskId, "subTaskId_" + subTaskId, interval, partitionId) + ); + } } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/ShuffleDataSegmentPusherTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/ShuffleDataSegmentPusherTest.java index c3dfcdd66e60..153192633967 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/ShuffleDataSegmentPusherTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/ShuffleDataSegmentPusherTest.java @@ -93,13 +93,13 @@ public void testPush() throws IOException Assert.assertEquals(9, pushed.getBinaryVersion().intValue()); Assert.assertEquals(14, pushed.getSize()); // 10 bytes data + 4 bytes version - final List files = intermediaryDataManager.findPartitionFiles( + final File zippedSegment = intermediaryDataManager.findPartitionFile( "supervisorTaskId", + "subTaskId", segment.getInterval(), segment.getShardSpec().getPartitionNum() ); - Assert.assertEquals(1, files.size()); - final File zippedSegment = files.get(0); + Assert.assertNotNull(zippedSegment); final File tempDir = temporaryFolder.newFolder(); final FileCopyResult result = CompressionUtils.unzip(zippedSegment, tempDir); final List unzippedFiles = new ArrayList<>(result.getFiles()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index c4ff006005a9..88a8dd00517b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -132,7 +132,8 @@ public List getLocations() null, null, null, - new NoopTestTaskReportFileWriter() + new NoopTestTaskReportFileWriter(), + null ), taskConfig, location diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index 022830ec8264..023251d1e15a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -180,7 +180,8 @@ private WorkerTaskMonitor createTaskMonitor() null, null, null, - new NoopTestTaskReportFileWriter() + new NoopTestTaskReportFileWriter(), + null ), taskConfig, new NoopServiceEmitter(), diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java index 4f9c0fcce176..6237ec031e7e 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java @@ -21,6 +21,7 @@ import com.google.inject.Inject; import org.apache.commons.io.IOUtils; +import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -222,7 +223,7 @@ private long countCompleteSubTasks(final String dataSource) { return indexer.getCompleteTasksForDataSource(dataSource) .stream() - .filter(t -> t.getType().equals("index_sub")) + .filter(t -> t.getType().equals(SinglePhaseSubTask.TYPE)) .count(); } } diff --git a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java index 84edad165068..c38469fdfc12 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java @@ -326,6 +326,7 @@ public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) ); } + @Override public RealtimeTuningConfig withBasePersistDirectory(File dir) { return new RealtimeTuningConfig( diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java index 2889a988a4d7..dd9a63071282 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java @@ -71,6 +71,8 @@ default Long getMaxTotalRows() File getBasePersistDirectory(); + AppenderatorConfig withBasePersistDirectory(File basePersistDirectory); + @Nullable SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory(); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index 11d46c2fa363..aff64d8e0658 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -331,6 +331,12 @@ public File getBasePersistDirectory() return baseConfig.getBasePersistDirectory(); } + @Override + public AppenderatorConfig withBasePersistDirectory(File basePersistDirectory) + { + return this; + } + @Nullable @Override public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java index eeec2686cc9a..a9d758e7cad4 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/HttpFirehoseFactory.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Strings; @@ -205,8 +204,7 @@ public FiniteFirehoseFactory withSplit(InputSplit