diff --git a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java index a45a9a7865a9..cd295ce2e5a0 100644 --- a/api/src/main/java/io/druid/indexer/TaskStatusPlus.java +++ b/api/src/main/java/io/druid/indexer/TaskStatusPlus.java @@ -29,6 +29,7 @@ public class TaskStatusPlus { private final String id; + private final String type; private final DateTime createdTime; private final DateTime queueInsertionTime; private final TaskState state; @@ -38,6 +39,7 @@ public class TaskStatusPlus @JsonCreator public TaskStatusPlus( @JsonProperty("id") String id, + @JsonProperty("type") String type, @JsonProperty("createdTime") DateTime createdTime, @JsonProperty("queueInsertionTime") DateTime queueInsertionTime, @JsonProperty("state") @Nullable TaskState state, @@ -49,6 +51,7 @@ public TaskStatusPlus( Preconditions.checkNotNull(duration, "duration"); } this.id = Preconditions.checkNotNull(id, "id"); + this.type = Preconditions.checkNotNull(type, "type"); this.createdTime = Preconditions.checkNotNull(createdTime, "createdTime"); this.queueInsertionTime = Preconditions.checkNotNull(queueInsertionTime, "queueInsertionTime"); this.state = state; @@ -62,6 +65,12 @@ public String getId() return id; } + @JsonProperty + public String getType() + { + return type; + } + @JsonProperty public DateTime getCreatedTime() { diff --git a/benchmarks/src/main/java/io/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/main/java/io/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java new file mode 100644 index 000000000000..4e5469872569 --- /dev/null +++ b/benchmarks/src/main/java/io/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -0,0 +1,144 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.coordinator; + +import com.google.common.collect.ImmutableList; +import io.druid.java.util.common.DateTimes; +import io.druid.server.coordinator.helper.CompactionSegmentIterator; +import io.druid.server.coordinator.helper.CompactionSegmentSearchPolicy; +import io.druid.server.coordinator.helper.NewestSegmentFirstPolicy; +import io.druid.timeline.DataSegment; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.NumberedShardSpec; +import io.druid.timeline.partition.ShardSpec; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.infra.Blackhole; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.MICROSECONDS) +@Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") +public class NewestSegmentFirstPolicyBenchmark +{ + private static final String DATA_SOURCE_PREFIX = "dataSource_"; + + private final CompactionSegmentSearchPolicy policy = new NewestSegmentFirstPolicy(); + + @Param("100") + private int numDataSources; + + @Param("10000") + private int numDayIntervalsPerDataSource; + + @Param("10") + private int numPartitionsPerDayInterval; + + @Param("800000000") + private long targetCompactionSizeBytes; + + @Param("1000000") + private long segmentSizeBytes; + + @Param("10") + private int numCompactionTaskSlots; + + private Map compactionConfigs; + private Map> dataSources; + + @Setup(Level.Trial) + public void setup() + { + compactionConfigs = new HashMap<>(); + for (int i = 0; i < numDataSources; i++) { + final String dataSource = DATA_SOURCE_PREFIX + i; + compactionConfigs.put( + dataSource, + new CoordinatorCompactionConfig( + dataSource, + 0, + targetCompactionSizeBytes, + null, + null, + null, + null + ) + ); + } + + dataSources = new HashMap<>(); + for (int i = 0; i < numDataSources; i++) { + final String dataSource = DATA_SOURCE_PREFIX + i; + + VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( + String.CASE_INSENSITIVE_ORDER + ); + + final int startYear = ThreadLocalRandom.current().nextInt(2000, 2040); + DateTime date = DateTimes.of(startYear, 1, 1, 0, 0); + + for (int j = 0; j < numDayIntervalsPerDataSource; j++, date = date.plusDays(1)) { + for (int k = 0; k < numPartitionsPerDayInterval; k++) { + final ShardSpec shardSpec = new NumberedShardSpec(numPartitionsPerDayInterval, k); + final DataSegment segment = new DataSegment( + dataSource, + new Interval(date, date.plusDays(1)), + "version", + null, + ImmutableList.of(), + ImmutableList.of(), + shardSpec, + 0, + segmentSizeBytes + ); + timeline.add(segment.getInterval(), segment.getVersion(), shardSpec.createChunk(segment)); + } + } + + dataSources.put(dataSource, timeline); + } + } + + @Benchmark + public void measureNewestSegmentFirstPolicy(Blackhole blackhole) + { + final CompactionSegmentIterator iterator = policy.reset(compactionConfigs, dataSources); + for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) { + final List segments = iterator.next(); + blackhole.consume(segments); + } + } +} diff --git a/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java b/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java index aa5a7e81e93c..8e5588042102 100644 --- a/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java +++ b/common/src/main/java/io/druid/timeline/VersionedIntervalTimeline.java @@ -224,6 +224,48 @@ public List> lookupWithIncompleteP } } + public boolean isEmpty() + { + try { + lock.readLock().lock(); + return completePartitionsTimeline.isEmpty(); + } + finally { + lock.readLock().unlock(); + } + } + + public TimelineObjectHolder first() + { + try { + lock.readLock().lock(); + return timelineEntryToObjectHolder(completePartitionsTimeline.firstEntry().getValue()); + } + finally { + lock.readLock().unlock(); + } + } + + public TimelineObjectHolder last() + { + try { + lock.readLock().lock(); + return timelineEntryToObjectHolder(completePartitionsTimeline.lastEntry().getValue()); + } + finally { + lock.readLock().unlock(); + } + } + + private TimelineObjectHolder timelineEntryToObjectHolder(TimelineEntry entry) + { + return new TimelineObjectHolder<>( + entry.getTrueInterval(), + entry.getVersion(), + new PartitionHolder<>(entry.getPartitionHolder()) + ); + } + public Set> findOvershadowed() { try { @@ -260,13 +302,7 @@ public Set> findOvershadowed() for (Map.Entry> versionEntry : overShadowed.entrySet()) { for (Map.Entry entry : versionEntry.getValue().entrySet()) { TimelineEntry object = entry.getValue(); - retVal.add( - new TimelineObjectHolder( - object.getTrueInterval(), - object.getVersion(), - new PartitionHolder(object.getPartitionHolder()) - ) - ); + retVal.add(timelineEntryToObjectHolder(object)); } } @@ -557,14 +593,14 @@ public class TimelineEntry private final VersionType version; private final PartitionHolder partitionHolder; - public TimelineEntry(Interval trueInterval, VersionType version, PartitionHolder partitionHolder) + TimelineEntry(Interval trueInterval, VersionType version, PartitionHolder partitionHolder) { this.trueInterval = Preconditions.checkNotNull(trueInterval); this.version = Preconditions.checkNotNull(version); this.partitionHolder = Preconditions.checkNotNull(partitionHolder); } - public Interval getTrueInterval() + Interval getTrueInterval() { return trueInterval; } diff --git a/docs/content/configuration/coordinator.md b/docs/content/configuration/coordinator.md index 2dfd97098df6..c04cfce8642f 100644 --- a/docs/content/configuration/coordinator.md +++ b/docs/content/configuration/coordinator.md @@ -24,7 +24,7 @@ The coordinator node uses several of the global configs in [Configuration](../co |Property|Description|Default| |--------|-----------|-------| |`druid.coordinator.period`|The run period for the coordinator. The coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S| -|`druid.coordinator.period.indexingPeriod`|How often to send indexing tasks to the indexing service. Only applies if merge or conversion is turned on.|PT1800S (30 mins)| +|`druid.coordinator.period.indexingPeriod`|How often to send compact/merge/conversion tasks to the indexing service. It's recommended to be longer than `druid.manager.segments.pollDuration`|PT1800S (30 mins)| |`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data.|PT300S| |`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|false| |`druid.coordinator.conversion.on`|Boolean flag for converting old segment indexing versions to the latest segment indexing version.|false| @@ -106,6 +106,9 @@ Issuing a GET request at the same URL will return the spec that is currently in |`killDataSourceWhitelist`|List of dataSources for which kill tasks are sent if property `druid.coordinator.kill.on` is true.|none| |`killAllDataSources`|Send kill tasks for ALL dataSources if property `druid.coordinator.kill.on` is true. If this is set to true then `killDataSourceWhitelist` must not be specified or be empty list.|false| |`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 0 (loading queue is unbounded) |0| +|`compactionConfigs`|Compaction config list. See the below [Compaction Config](#compaction-config)|none| +|`compactionTaskSlotRatio`|The ratio of the total task slots to the copmaction task slots. The actual max number of compaction tasks is `min(maxCompactionTaskSlots, compactionTaskSlotRatio * total task slots)`.|0.1| +|`maxCompactionTaskSlots`|The maximum number of task slots for compaction task. The actual max number of compaction tasks is `min(maxCompactionTaskSlots, compactionTaskSlotRatio * total task slots)`.|Unbounded| To view the audit history of coordinator dynamic config issue a GET request to the URL - @@ -121,6 +124,41 @@ To view last entries of the audit history of coordinator dynamic config issu http://:/druid/coordinator/v1/config/history?count= ``` +# Compaction Config + +|Property|Description|Required| +|--------|-----------|--------| +|`dataSource`|dataSource name to be compacted.|yes| +|`taskPriority`|[Priority](../ingestion/tasks.html#task-priorities) of compact task.|no (default = 25)| +|`targetCompactionSizeBytes`|The target segment size of compaction. The actual size of a compact segment might be slightly larger or smaller than this value.|no (default = 838860800)| +|`numTargetCompactionSegments`|Max number of segments to compact together.|no (default = 150)| +|`skipOffsetFromLatest`|The offset for searching segments to be compacted. Strongly recommended to set for realtime dataSources. |no (default = "P1D")| +|`tuningConfig`|Tuning config for compact tasks. See below [Compact Task TuningConfig](#compact-task-tuningconfig).|no| +|`taskContext`|[Task context](../ingestion/tasks.html#task-context) for compact tasks.|no| + +An example of compaction config is: + +```json +{ + "dataSource": "wikiticker", + "targetCompactionSizeBytes": 800000000, + "skipOffsetFromLatest": "P1D" +} +``` + +For the realtime dataSources, it's recommended to set `skipOffsetFromLatest` to some sufficiently large values to avoid frequent compact task failures. + +## Compact Task TuningConfig + +Compact task tuning config is a subset of the tuningConfig of IndexTask. See [TuningConfig](../ingestion/tasks.html#tuningconfig) for more details. + +|Property|Required| +|--------|--------| +|`maxRowsInMemory`|no| +|`maxTotalRows`|no| +|`indexSpec`|no| +|`maxPendingPersists`|no| +|`publishTimeout`|no| # Lookups Dynamic Config (EXPERIMENTAL) These configuration options control the behavior of the Lookup dynamic configuration described in the [lookups page](../querying/lookups.html) diff --git a/docs/content/design/coordinator.md b/docs/content/design/coordinator.md index 5c4aa430c606..b6aa862b584a 100644 --- a/docs/content/design/coordinator.md +++ b/docs/content/design/coordinator.md @@ -38,6 +38,45 @@ Balancing Segment Load To ensure an even distribution of segments across historical nodes in the cluster, the coordinator node will find the total size of all segments being served by every historical node each time the coordinator runs. For every historical node tier in the cluster, the coordinator node will determine the historical node with the highest utilization and the historical node with the lowest utilization. The percent difference in utilization between the two nodes is computed, and if the result exceeds a certain threshold, a number of segments will be moved from the highest utilized node to the lowest utilized node. There is a configurable limit on the number of segments that can be moved from one node to another each time the coordinator runs. Segments to be moved are selected at random and only moved if the resulting utilization calculation indicates the percentage difference between the highest and lowest servers has decreased. +Compacting Segments +------------------- + +Each run, the Druid coordinator compacts small segments abutting each other. This is useful when you have a lot of small +segments which may degrade the query performance as well as increasing the disk usage. Note that the data for an interval +cannot be compacted across the segments. + +The coordinator first finds the segments to compact together based on the [segment search policy](#segment-search-policy). +Once it finds some segments, it launches a [compact task](../ingestion/tasks.html#compaction-task) to compact those segments. +The maximum number of running compact tasks is `max(sum of worker capacity * compactionTaskSlotRatio, 1)`. (See [Dynamic Configuration](../configuration/coordinator.html#dynamic-configuration) for `compactionTaskSlotRatio`.) + +Compact tasks might fail due to some reasons. + +- If the input segments of a compact task are removed or overshadowed before it starts, that compact task fails immediately. +- If a task of a higher priority acquires a lock for an interval overlapping with the interval of a compact task, the compact task fails. + +Once a compact task fails, the coordinator simply finds the segments for the interval of the failed task again, and launches a new compact task in the next run. + +To use this feature, you need to set some configurations for dataSources you want to compact. +Please see [Compaction Config](../configuration/coordinator.html#compaction-config) for details. + +### Segment Search Policy + +#### Newest Segment First Policy + +This policy searches the segments of _all dataSources_ in inverse order of their intervals. +For example, let me assume there are 3 dataSources (`ds1`, `ds2`, `ds3`) and 5 segments (`seg_ds1_2017-10-01_2017-10-02`, `seg_ds1_2017-11-01_2017-11-02`, `seg_ds2_2017-08-01_2017-08-02`, `seg_ds3_2017-07-01_2017-07-02`, `seg_ds3_2017-12-01_2017-12-02`) for those dataSources. +The segment name indicates its dataSource and interval. The search result of newestSegmentFirstPolicy is [`seg_ds3_2017-12-01_2017-12-02`, `seg_ds1_2017-11-01_2017-11-02`, `seg_ds1_2017-10-01_2017-10-02`, `seg_ds2_2017-08-01_2017-08-02`, `seg_ds3_2017-07-01_2017-07-02`]. + +Every run, this policy starts searching from the (very latest interval - [skipOffsetFromLatest](../configuration/coordinator.html#compaction-config)). +This is to handle the late segments ingested to realtime dataSources. + +
+This policy currently cannot handle the situation when there are a lot of small segments which have the same interval, +and their total size exceeds targetCompactionSizebytes. +If it finds such segments, it simply skips compacting them. +
+ + HTTP Endpoints -------------- @@ -241,6 +280,11 @@ Returns total size and count for each interval within given isointerval. Returns total size and count for each datasource for each interval within given isointerval. +#### Compaction Configs + +* `/druid/coordinator/v1/config/compaction/{dataSource}` + +Returns a compaction config of a dataSource. ### POST @@ -267,6 +311,12 @@ Optional Header Parameters for auditing the config change can also be specified. |`X-Druid-Author`| author making the config change|""| |`X-Druid-Comment`| comment describing the change being done|""| +#### Compaction Configs + +* `/druid/coordinator/v1/config/compaction/{dataSource}` + +Creates or updates the compaction config for a dataSource. + ### DELETE #### Datasources @@ -286,6 +336,12 @@ Note that {interval} parameters are delimited by a `_` instead of a `/` (e.g., 2 Disables a segment. +#### Compaction Configs + +* `/druid/coordinator/v1/config/compaction/{dataSource}` + +Removes the compaction config for a dataSource. + The Coordinator Console ------------------ diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index b32c479fe7ba..d858b50f4407 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -339,12 +339,7 @@ keeping the number of segments low are: In many production installations which have been ingesting events for a long period of time, these suggestions alone will not be sufficient to keep the number of segments at an optimal level. It is recommended that scheduled re-indexing tasks be run to merge segments together into new segments of an ideal size (in the range of ~500-700 MB per segment). -Currently, the recommended way of doing this is by running periodic Hadoop batch ingestion jobs and using a `dataSource` -inputSpec to read from the segments generated by the Kafka indexing tasks. Details on how to do this can be found under -['Updating Existing Data'](../../ingestion/update-existing-data.html). Note that the Merge Task and Append Task described -[here](../../ingestion/tasks.html) will not work as they require unsharded segments while Kafka indexing tasks always -generated sharded segments. - -There is ongoing work to support automatic segment compaction of sharded segments as well as compaction not requiring -Hadoop (see [here](https://github.com/druid-io/druid/pull/1998) and [here](https://github.com/druid-io/druid/pull/3611) -for related PRs). +Details on how to optimize the segment size can be found on [Segment size optimization](../../operations/segment-optimization.html). + +Note that the Merge Task and Append Task described [here](../../ingestion/tasks.html) will not work as they require +unsharded segments while Kafka indexing tasks always generated sharded segments. diff --git a/docs/content/ingestion/stream-pull.md b/docs/content/ingestion/stream-pull.md index 48e011ec0466..e57975276871 100644 --- a/docs/content/ingestion/stream-pull.md +++ b/docs/content/ingestion/stream-pull.md @@ -335,3 +335,10 @@ This can be difficult to manage at scale, especially with multiple partitions. Each standalone realtime node has its own set of logs. Diagnosing errors across many partitions across many servers may be difficult to manage and track at scale. + +## Deployment Notes + +Stream ingestion may generate a large number of small segments because it's difficult to optimize the segment size at +ingestion time. The number of segments will increase over time, and this might cause the query performance issue. + +Details on how to optimize the segment size can be found on [Segment size optimization](../../operations/segment-optimization.html). \ No newline at end of file diff --git a/docs/content/ingestion/stream-push.md b/docs/content/ingestion/stream-push.md index 1ea8f9f7a636..ca0f7013bbac 100644 --- a/docs/content/ingestion/stream-push.md +++ b/docs/content/ingestion/stream-push.md @@ -135,6 +135,13 @@ Under normal operation, these risks are minimal. But if you need absolute 100% f historical data, we recommend a [hybrid batch/streaming](../tutorials/ingestion.html#hybrid-batch-streaming) architecture. +### Deployment Notes + +Stream ingestion may generate a large number of small segments because it's difficult to optimize the segment size at +ingestion time. The number of segments will increase over time, and this might cuase the query performance issue. + +Details on how to optimize the segment size can be found on [Segment size optimization](../../operations/segment-optimization.html). + ## Documentation Tranquility documentation be found [here](https://github.com/druid-io/tranquility/blob/master/README.md). diff --git a/docs/content/operations/segment-optimization.md b/docs/content/operations/segment-optimization.md new file mode 100644 index 000000000000..0e43982d2949 --- /dev/null +++ b/docs/content/operations/segment-optimization.md @@ -0,0 +1,25 @@ +--- +layout: doc_page +--- +# Segment size optimization + +In Druid, it's important to optimize the segment size because + + 1. Druid stores data in segments. If you're using the [best-effort roll-up](../design/index.html#roll-up-modes) mode, + increasing the segment size might introduce further aggregation which reduces the dataSource size. + 2. When a query is submitted, that query is distributed to all historicals and realtimes + which hold the input segments of the query. Each node has a processing threads pool and use one thread per segment to + process it. If the segment size is too large, data might not be well distributed over the + whole cluster, thereby decreasing the degree of parallelism. If the segment size is too small, + each processing thread processes too small data. This might reduce the processing speed of other queries as well as + the input query itself because the processing threads are shared for executing all queries. + +It would be best if you can optimize the segment size at ingestion time, but sometimes it's not easy +especially for the streaming ingestion because the amount of data ingested might vary over time. In this case, +you can roughly set the segment size at ingestion time and optimize it later. You have two options: + + - Turning on the [automatic compaction of coordinators](../design/coordinator.html#compacting-segments). + The coordinator periodically submits [compaction tasks](../ingestion/tasks.html#compaction-task) to re-index small segments. + - Running periodic Hadoop batch ingestion jobs and using a `dataSource` + inputSpec to read from the segments generated by the Kafka indexing tasks. This might be helpful if you want to compact a lot of segments in parallel. + Details on how to do this can be found under ['Updating Existing Data'](../ingestion/update-existing-data.html). diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 29041e27de58..a53d6758f8ab 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -1019,7 +1019,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception List tasks = captured.getValues(); Collection workItems = new ArrayList<>(); for (Task task : tasks) { - workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); + workItems.add(new TestTaskRunnerWorkItem(task, null, location)); } reset(taskStorage, taskRunner, taskClient, taskQueue); @@ -1102,7 +1102,7 @@ public void testDiscoverExistingPublishingTask() throws Exception ); Collection workItems = new ArrayList<>(); - workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); + workItems.add(new TestTaskRunnerWorkItem(task, null, location)); Capture captured = Capture.newInstance(); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -1197,7 +1197,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation() ); Collection workItems = new ArrayList<>(); - workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); + workItems.add(new TestTaskRunnerWorkItem(task, null, location)); Capture captured = Capture.newInstance(); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); @@ -1300,8 +1300,8 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception ); Collection workItems = new ArrayList<>(); - workItems.add(new TestTaskRunnerWorkItem(id1.getId(), null, location1)); - workItems.add(new TestTaskRunnerWorkItem(id2.getId(), null, location2)); + workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); @@ -1456,7 +1456,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception List tasks = captured.getValues(); Collection workItems = new ArrayList<>(); for (Task task : tasks) { - workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); + workItems.add(new TestTaskRunnerWorkItem(task, null, location)); } reset(taskStorage, taskRunner, taskClient, taskQueue); @@ -1533,7 +1533,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception List tasks = captured.getValues(); Collection workItems = new ArrayList<>(); for (Task task : tasks) { - workItems.add(new TestTaskRunnerWorkItem(task.getId(), null, location)); + workItems.add(new TestTaskRunnerWorkItem(task, null, location)); } reset(taskStorage, taskRunner, taskClient, taskQueue); @@ -1653,8 +1653,8 @@ public void testStopGracefully() throws Exception ); Collection workItems = new ArrayList<>(); - workItems.add(new TestTaskRunnerWorkItem(id1.getId(), null, location1)); - workItems.add(new TestTaskRunnerWorkItem(id2.getId(), null, location2)); + workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); @@ -1855,8 +1855,8 @@ public void testResetRunningTasks() throws Exception ); Collection workItems = new ArrayList<>(); - workItems.add(new TestTaskRunnerWorkItem(id1.getId(), null, location1)); - workItems.add(new TestTaskRunnerWorkItem(id2.getId(), null, location2)); + workItems.add(new TestTaskRunnerWorkItem(id1, null, location1)); + workItems.add(new TestTaskRunnerWorkItem(id2, null, location2)); expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); @@ -2057,12 +2057,13 @@ private KafkaIndexTask createKafkaIndexTask( private static class TestTaskRunnerWorkItem extends TaskRunnerWorkItem { + private final String taskType; + private final TaskLocation location; - private TaskLocation location; - - public TestTaskRunnerWorkItem(String taskId, ListenableFuture result, TaskLocation location) + public TestTaskRunnerWorkItem(Task task, ListenableFuture result, TaskLocation location) { - super(taskId, result); + super(task.getId(), result); + this.taskType = task.getType(); this.location = location; } @@ -2071,6 +2072,12 @@ public TaskLocation getLocation() { return location; } + + @Override + public String getTaskType() + { + return taskType; + } } private static class TestableKafkaSupervisor extends KafkaSupervisor diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java index 96f3889982b9..66202395f8cf 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/CompactionTask.java @@ -475,12 +475,21 @@ List checkAndGetSegments(TaskToolbox toolbox) throws IOException if (segments != null) { Collections.sort(usedSegments); Collections.sort(segments); - Preconditions.checkState( - usedSegments.equals(segments), - "Specified segments[%s] are different from the current used segments[%s]", - segments, - usedSegments - ); + + if (!usedSegments.equals(segments)) { + final List unknownSegments = segments.stream() + .filter(segment -> !usedSegments.contains(segment)) + .collect(Collectors.toList()); + final List missingSegments = usedSegments.stream() + .filter(segment -> !segments.contains(segment)) + .collect(Collectors.toList()); + throw new ISE( + "Specified segments in the spec are different from the current used segments. " + + "There are unknown segments[%s] and missing segments[%s] in the spec.", + unknownSegments, + missingSegments + ); + } } return usedSegments; } diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java index ee103420207d..c69d14226ebb 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ForkingTaskRunner.java @@ -766,6 +766,12 @@ public TaskLocation getLocation() return TaskLocation.create(processHolder.host, processHolder.port, processHolder.tlsPort); } } + + @Override + public String getTaskType() + { + return task.getType(); + } } private static class ProcessHolder diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java index c3e26bd5fac4..28117e7b829e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunner.java @@ -624,7 +624,12 @@ private URL makeWorkerURL(Worker worker, String path) private RemoteTaskRunnerWorkItem addPendingTask(final Task task) { log.info("Added pending task %s", task.getId()); - final RemoteTaskRunnerWorkItem taskRunnerWorkItem = new RemoteTaskRunnerWorkItem(task.getId(), null, null); + final RemoteTaskRunnerWorkItem taskRunnerWorkItem = new RemoteTaskRunnerWorkItem( + task.getId(), + task.getType(), + null, + null + ); pendingTaskPayloads.put(task.getId(), task); pendingTasks.put(task.getId(), taskRunnerWorkItem); runPendingTasks(); @@ -959,6 +964,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th } else { final RemoteTaskRunnerWorkItem newTaskRunnerWorkItem = new RemoteTaskRunnerWorkItem( taskId, + announcement.getTaskType(), zkWorker.getWorker(), TaskLocation.unknown() ); diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java index 87f722d2e981..a62990f1678d 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/RemoteTaskRunnerWorkItem.java @@ -30,31 +30,23 @@ public class RemoteTaskRunnerWorkItem extends TaskRunnerWorkItem { private final SettableFuture result; + private String taskType; private Worker worker; private TaskLocation location; public RemoteTaskRunnerWorkItem( String taskId, + String taskType, Worker worker, TaskLocation location ) { - this(taskId, SettableFuture.create(), worker, location); - } - - public RemoteTaskRunnerWorkItem( - String taskId, - DateTime createdTime, - DateTime queueInsertionTime, - Worker worker, - TaskLocation location - ) - { - this(taskId, SettableFuture.create(), createdTime, queueInsertionTime, worker, location); + this(taskId, taskType, SettableFuture.create(), worker, location); } private RemoteTaskRunnerWorkItem( String taskId, + String taskType, SettableFuture result, Worker worker, TaskLocation location @@ -62,12 +54,14 @@ private RemoteTaskRunnerWorkItem( { super(taskId, result); this.result = result; + this.taskType = taskType; this.worker = worker; this.location = location == null ? TaskLocation.unknown() : location; } private RemoteTaskRunnerWorkItem( String taskId, + String taskType, SettableFuture result, DateTime createdTime, DateTime queueInsertionTime, @@ -77,6 +71,7 @@ private RemoteTaskRunnerWorkItem( { super(taskId, result, createdTime, queueInsertionTime); this.result = result; + this.taskType = taskType; this.worker = worker; this.location = location == null ? TaskLocation.unknown() : location; } @@ -92,6 +87,17 @@ public TaskLocation getLocation() return location; } + public void setTaskType(String taskType) + { + this.taskType = taskType; + } + + @Override + public String getTaskType() + { + return taskType; + } + public void setWorker(Worker worker) { this.worker = worker; @@ -109,13 +115,14 @@ public void setResult(TaskStatus status) public RemoteTaskRunnerWorkItem withQueueInsertionTime(DateTime time) { - return new RemoteTaskRunnerWorkItem(getTaskId(), result, getCreatedTime(), time, worker, location); + return new RemoteTaskRunnerWorkItem(getTaskId(), taskType, result, getCreatedTime(), time, worker, location); } public RemoteTaskRunnerWorkItem withWorker(Worker theWorker, TaskLocation location) { return new RemoteTaskRunnerWorkItem( getTaskId(), + taskType, result, getCreatedTime(), getQueueInsertionTime(), diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java index 2ec4a1473331..d2d8f6daf1ca 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/TaskRunnerWorkItem.java @@ -85,6 +85,7 @@ public DateTime getQueueInsertionTime() } public abstract TaskLocation getLocation(); + public abstract String getTaskType(); @Override public String toString() diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java index b7d7008b0937..37b02e63c2d8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/ThreadPoolTaskRunner.java @@ -411,6 +411,12 @@ public TaskLocation getLocation() { return location; } + + @Override + public String getTaskType() + { + return task.getType(); + } } private class ThreadPoolTaskRunnerCallable implements Callable diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java b/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java index d6d73a12f3f2..81cc223ede58 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java @@ -79,6 +79,7 @@ import org.jboss.netty.handler.codec.http.HttpMethod; import org.joda.time.Period; +import javax.annotation.Nullable; import java.io.IOException; import java.io.InputStream; import java.net.URL; @@ -1335,11 +1336,11 @@ enum State String taskId, Worker worker, TaskLocation location, - Task task, + @Nullable Task task, State state ) { - super(taskId, worker, location); + super(taskId, task == null ? null : task.getType(), worker, location); this.state = Preconditions.checkNotNull(state); // It is possible to have it null when the TaskRunner is just started and discovered this taskId from a worker, @@ -1355,6 +1356,7 @@ public Task getTask() public void setTask(Task task) { this.task = task; + setTaskType(task.getType()); } public State getState() diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/WorkerHolder.java b/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/WorkerHolder.java index ff91743be57e..6326b9959a8c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/WorkerHolder.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/hrtr/WorkerHolder.java @@ -389,6 +389,7 @@ public void fullSync(List changes) ); delta.add(TaskAnnouncement.create( announcement.getTaskId(), + announcement.getTaskType(), announcement.getTaskResource(), TaskStatus.failure(announcement.getTaskId()), announcement.getTaskLocation() @@ -424,6 +425,7 @@ public void deltaSync(List changes) ); delta.add(TaskAnnouncement.create( announcement.getTaskId(), + announcement.getTaskType(), announcement.getTaskResource(), TaskStatus.failure(announcement.getTaskId()), announcement.getTaskLocation() diff --git a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java index 83fc0d305f15..b22a5626e317 100644 --- a/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/io/druid/indexing/overlord/http/OverlordResource.java @@ -28,6 +28,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.io.ByteSource; +import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; @@ -431,11 +432,10 @@ public String apply(final TaskRunnerWorkItem workItem) if (!runnersKnownTasks.contains(task.getId())) { waitingTasks.add( // Would be nice to include the real created date, but the TaskStorage API doesn't yet allow it. - new TaskRunnerWorkItem( + new WaitingTask( task.getId(), - SettableFuture.create(), - DateTimes.EPOCH, - DateTimes.EPOCH + task.getType(), + SettableFuture.create() ) { @Override @@ -453,6 +453,33 @@ public TaskLocation getLocation() ); } + private static class WaitingTask extends TaskRunnerWorkItem + { + private final String taskType; + + WaitingTask( + String taskId, + String taskType, + ListenableFuture result + ) + { + super(taskId, result, DateTimes.EPOCH, DateTimes.EPOCH); + this.taskType = taskType; + } + + @Override + public TaskLocation getLocation() + { + return TaskLocation.unknown(); + } + + @Override + public String getTaskType() + { + return taskType; + } + } + @GET @Path("/pendingTasks") @Produces(MediaType.APPLICATION_JSON) @@ -473,7 +500,10 @@ public Collection apply(TaskRunner taskRunner) @GET @Path("/runningTasks") @Produces(MediaType.APPLICATION_JSON) - public Response getRunningTasks(@Context final HttpServletRequest req) + public Response getRunningTasks( + @QueryParam("type") String taskType, + @Context final HttpServletRequest req + ) { return workItemsResponse( new Function>() @@ -481,7 +511,17 @@ public Response getRunningTasks(@Context final HttpServletRequest req) @Override public Collection apply(TaskRunner taskRunner) { - return securedTaskRunnerWorkItem(taskRunner.getRunningTasks(), req); + final Collection workItems; + if (taskType == null) { + workItems = taskRunner.getRunningTasks(); + } else { + workItems = taskRunner.getRunningTasks() + .stream() + .filter(workitem -> workitem.getTaskType().equals(taskType)) + .collect(Collectors.toList()); + } + + return securedTaskRunnerWorkItem(workItems, req); } } ); @@ -495,8 +535,7 @@ public Response getCompleteTasks( @Context final HttpServletRequest req ) { - Function> raGenerator = taskStatus -> { - final String taskId = taskStatus.getId(); + final Function taskFunction = taskId -> { final Optional optionalTask = taskStorageQueryAdapter.getTask(taskId); if (!optionalTask.isPresent()) { throw new WebApplicationException( @@ -505,10 +544,15 @@ public Response getCompleteTasks( ).build() ); } + return optionalTask.get(); + }; + + Function> raGenerator = taskStatus -> { + final Task task = taskFunction.apply(taskStatus.getId()); return Lists.newArrayList( new ResourceAction( - new Resource(optionalTask.get().getDataSource(), ResourceType.DATASOURCE), + new Resource(task.getDataSource(), ResourceType.DATASOURCE), Action.READ ) ); @@ -527,6 +571,7 @@ public Response getCompleteTasks( .stream() .map(status -> new TaskStatusPlus( status.getId(), + taskFunction.apply(status.getId()).getType(), taskStorageQueryAdapter.getCreatedTime(status.getId()), // Would be nice to include the real queue insertion time, but the TaskStorage API doesn't yet allow it. DateTimes.EPOCH, @@ -663,6 +708,7 @@ public TaskStatusPlus apply(TaskRunnerWorkItem workItem) { return new TaskStatusPlus( workItem.getTaskId(), + workItem.getTaskType(), workItem.getCreatedTime(), workItem.getQueueInsertionTime(), null, diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java b/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java index a0c804238040..5079d8b04b48 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/TaskAnnouncement.java @@ -33,30 +33,39 @@ */ public class TaskAnnouncement { + private final String taskType; private final TaskStatus taskStatus; private final TaskResource taskResource; private final TaskLocation taskLocation; public static TaskAnnouncement create(Task task, TaskStatus status, TaskLocation location) { - return create(task.getId(), task.getTaskResource(), status, location); + return create(task.getId(), task.getType(), task.getTaskResource(), status, location); } - public static TaskAnnouncement create(String taskId, TaskResource resource, TaskStatus status, TaskLocation location) + public static TaskAnnouncement create( + String taskId, + String taskType, + TaskResource resource, + TaskStatus status, + TaskLocation location + ) { Preconditions.checkArgument(status.getId().equals(taskId), "task id == status id"); - return new TaskAnnouncement(null, null, status, resource, location); + return new TaskAnnouncement(null, taskType, null, status, resource, location); } @JsonCreator private TaskAnnouncement( @JsonProperty("id") String taskId, + @JsonProperty("type") String taskType, @JsonProperty("status") TaskState status, @JsonProperty("taskStatus") TaskStatus taskStatus, @JsonProperty("taskResource") TaskResource taskResource, @JsonProperty("taskLocation") TaskLocation taskLocation ) { + this.taskType = taskType; if (taskStatus != null) { this.taskStatus = taskStatus; } else { @@ -73,6 +82,12 @@ public String getTaskId() return taskStatus.getId(); } + @JsonProperty("type") + public String getTaskType() + { + return taskType; + } + @JsonProperty("status") public TaskState getStatus() { diff --git a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java index eeaff13212a2..2c086fa0ea1b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/io/druid/indexing/worker/WorkerTaskMonitor.java @@ -134,6 +134,7 @@ private void cleanupStaleAnnouncements() throws Exception workerCuratorCoordinator.updateTaskStatusAnnouncement( TaskAnnouncement.create( announcement.getTaskStatus().getId(), + announcement.getTaskType(), announcement.getTaskResource(), completionStatus, TaskLocation.unknown() diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java index c2a199f872d3..f8fd89917475 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/RemoteTaskRunnerTest.java @@ -603,11 +603,11 @@ public boolean isValid() @Test public void testSortByInsertionTime() throws Exception { - RemoteTaskRunnerWorkItem item1 = new RemoteTaskRunnerWorkItem("b", null, null) + RemoteTaskRunnerWorkItem item1 = new RemoteTaskRunnerWorkItem("b", "t", null, null) .withQueueInsertionTime(DateTimes.of("2015-01-01T00:00:03Z")); - RemoteTaskRunnerWorkItem item2 = new RemoteTaskRunnerWorkItem("a", null, null) + RemoteTaskRunnerWorkItem item2 = new RemoteTaskRunnerWorkItem("a", "t", null, null) .withQueueInsertionTime(DateTimes.of("2015-01-01T00:00:02Z")); - RemoteTaskRunnerWorkItem item3 = new RemoteTaskRunnerWorkItem("c", null, null) + RemoteTaskRunnerWorkItem item3 = new RemoteTaskRunnerWorkItem("c", "t", null, null) .withQueueInsertionTime(DateTimes.of("2015-01-01T00:00:01Z")); ArrayList workItems = Lists.newArrayList(item1, item2, item3); RemoteTaskRunner.sortByInsertionTime(workItems); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java index 3713b00cf1f2..0392e1537570 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/PendingTaskBasedProvisioningStrategyTest.java @@ -337,6 +337,7 @@ public void testDoSuccessfulTerminate() throws Exception Arrays.asList( new RemoteTaskRunnerWorkItem( testTask.getId(), + testTask.getType(), null, TaskLocation.unknown() ).withQueueInsertionTime(DateTimes.nowUtc()) diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java index f917d359c64a..145af608c730 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/autoscaling/SimpleProvisioningStrategyTest.java @@ -28,7 +28,6 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; import io.druid.common.guava.DSuppliers; -import io.druid.java.util.common.concurrent.Execs; import io.druid.indexer.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TestTasks; @@ -38,12 +37,13 @@ import io.druid.indexing.overlord.RemoteTaskRunner; import io.druid.indexing.overlord.RemoteTaskRunnerWorkItem; import io.druid.indexing.overlord.ZkWorker; -import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; import io.druid.indexing.overlord.setup.DefaultWorkerBehaviorConfig; +import io.druid.indexing.overlord.setup.WorkerBehaviorConfig; import io.druid.indexing.worker.TaskAnnouncement; import io.druid.indexing.worker.Worker; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.concurrent.Execs; import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Period; @@ -125,7 +125,8 @@ public void testSuccessfulProvision() throws Exception RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); EasyMock.expect(runner.getPendingTasks()).andReturn( Collections.singletonList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(DateTimes.nowUtc()) + new RemoteTaskRunnerWorkItem(testTask.getId(), testTask.getType(), null, null) + .withQueueInsertionTime(DateTimes.nowUtc()) ) ); EasyMock.expect(runner.getWorkers()).andReturn( @@ -162,7 +163,8 @@ public void testSomethingProvisioning() throws Exception RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); EasyMock.expect(runner.getPendingTasks()).andReturn( Collections.singletonList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(DateTimes.nowUtc()) + new RemoteTaskRunnerWorkItem(testTask.getId(), testTask.getType(), null, null) + .withQueueInsertionTime(DateTimes.nowUtc()) ) ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( @@ -220,7 +222,8 @@ public void testProvisionAlert() throws Exception RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); EasyMock.expect(runner.getPendingTasks()).andReturn( Collections.singletonList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(DateTimes.nowUtc()) + new RemoteTaskRunnerWorkItem(testTask.getId(), testTask.getType(), null, null) + .withQueueInsertionTime(DateTimes.nowUtc()) ) ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( @@ -272,7 +275,8 @@ public void testDoSuccessfulTerminate() throws Exception RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); EasyMock.expect(runner.getPendingTasks()).andReturn( Collections.singletonList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(DateTimes.nowUtc()) + new RemoteTaskRunnerWorkItem(testTask.getId(), testTask.getType(), null, null) + .withQueueInsertionTime(DateTimes.nowUtc()) ) ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( @@ -312,7 +316,8 @@ public void testSomethingTerminating() throws Exception RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); EasyMock.expect(runner.getPendingTasks()).andReturn( Collections.singletonList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(DateTimes.nowUtc()) + new RemoteTaskRunnerWorkItem(testTask.getId(), testTask.getType(), null, null) + .withQueueInsertionTime(DateTimes.nowUtc()) ) ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( @@ -359,7 +364,8 @@ public void testNoActionNeeded() throws Exception RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); EasyMock.expect(runner.getPendingTasks()).andReturn( Collections.singletonList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(DateTimes.nowUtc()) + new RemoteTaskRunnerWorkItem(testTask.getId(), testTask.getType(), null, null) + .withQueueInsertionTime(DateTimes.nowUtc()) ) ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( @@ -462,7 +468,8 @@ public void testNullWorkerConfig() throws Exception RemoteTaskRunner runner = EasyMock.createMock(RemoteTaskRunner.class); EasyMock.expect(runner.getPendingTasks()).andReturn( Collections.singletonList( - new RemoteTaskRunnerWorkItem(testTask.getId(), null, null).withQueueInsertionTime(DateTimes.nowUtc()) + new RemoteTaskRunnerWorkItem(testTask.getId(), testTask.getType(), null, null) + .withQueueInsertionTime(DateTimes.nowUtc()) ) ).times(2); EasyMock.expect(runner.getWorkers()).andReturn( diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java index 78242e8f8603..817d6743df90 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordResourceTest.java @@ -212,18 +212,27 @@ public TaskStatus apply(String input) ) ).once(); - EasyMock.expect(taskStorageQueryAdapter.getTask(tasksIds.get(0))).andReturn( + EasyMock.expect(taskStorageQueryAdapter.getTask(EasyMock.eq(tasksIds.get(0)))).andReturn( Optional.of(getTaskWithIdAndDatasource(tasksIds.get(0), "deny")) ).once(); - EasyMock.expect(taskStorageQueryAdapter.getTask(tasksIds.get(1))).andReturn( + EasyMock.expect(taskStorageQueryAdapter.getTask(EasyMock.eq(tasksIds.get(1)))).andReturn( Optional.of(getTaskWithIdAndDatasource(tasksIds.get(1), "allow")) ).once(); - EasyMock.expect(taskStorageQueryAdapter.getTask(tasksIds.get(2))).andReturn( + EasyMock.expect(taskStorageQueryAdapter.getTask(EasyMock.eq(tasksIds.get(2)))).andReturn( Optional.of(getTaskWithIdAndDatasource(tasksIds.get(2), "allow")) ).once(); + EasyMock.expect(taskStorageQueryAdapter.getTask(EasyMock.eq(tasksIds.get(1)))).andReturn( + Optional.of(getTaskWithIdAndDatasource(tasksIds.get(1), "allow")) + ).once(); EasyMock.expect(taskStorageQueryAdapter.getCreatedTime(EasyMock.anyString())) .andReturn(DateTimes.EPOCH) - .anyTimes(); + .once(); + EasyMock.expect(taskStorageQueryAdapter.getTask(EasyMock.eq(tasksIds.get(2)))).andReturn( + Optional.of(getTaskWithIdAndDatasource(tasksIds.get(2), "allow")) + ).once(); + EasyMock.expect(taskStorageQueryAdapter.getCreatedTime(EasyMock.anyString())) + .andReturn(DateTimes.EPOCH) + .once(); EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); List responseObjects = (List) overlordResource.getCompleteTasks(null, req) @@ -255,7 +264,7 @@ public void testSecuredGetRunningTasks() EasyMock.replay(taskRunner, taskMaster, taskStorageQueryAdapter, indexerMetadataStorageAdapter, req); - List responseObjects = (List) overlordResource.getRunningTasks(req) + List responseObjects = (List) overlordResource.getRunningTasks(null, req) .getEntity(); Assert.assertEquals(1, responseObjects.size()); @@ -304,7 +313,7 @@ private Task getTaskWithIdAndDatasource(String id, String datasource) @Override public String getType() { - return null; + return "test"; } @Override @@ -336,6 +345,12 @@ public TaskLocation getLocation() { return TaskLocation.unknown(); } + + @Override + public String getTaskType() + { + return "test"; + } } } diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java index b88556cf4fc3..874f891e87f2 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/http/OverlordTest.java @@ -264,7 +264,7 @@ public void testOverlordRun() throws Exception // Wait for task runner to run task_1 runTaskCountDownLatches[Integer.parseInt(taskId_1)].await(); - response = overlordResource.getRunningTasks(req); + response = overlordResource.getRunningTasks(null, req); // 1 task that was manually inserted should be in running state Assert.assertEquals(1, (((List) response.getEntity()).size())); final TaskStatusPlus taskResponseObject = ((List) response @@ -389,6 +389,12 @@ public TaskLocation getLocation() { return TASK_LOCATION; } + + @Override + public String getTaskType() + { + return task.getType(); + } }; taskRunnerWorkItems.put(taskId, taskRunnerWorkItem); return future; diff --git a/java-util/src/main/java/io/druid/java/util/common/DateTimes.java b/java-util/src/main/java/io/druid/java/util/common/DateTimes.java index e5a08595c252..6045fff3034d 100644 --- a/java-util/src/main/java/io/druid/java/util/common/DateTimes.java +++ b/java-util/src/main/java/io/druid/java/util/common/DateTimes.java @@ -77,6 +77,17 @@ public static DateTime of(String instant) return new DateTime(instant, ISOChronology.getInstanceUTC()); } + public static DateTime of( + int year, + int monthOfYear, + int dayOfMonth, + int hourOfDay, + int minuteOfHour + ) + { + return new DateTime(year, monthOfYear, dayOfMonth, hourOfDay, minuteOfHour, ISOChronology.getInstanceUTC()); + } + public static DateTime nowUtc() { return DateTime.now(ISOChronology.getInstanceUTC()); diff --git a/java-util/src/main/java/io/druid/java/util/common/Intervals.java b/java-util/src/main/java/io/druid/java/util/common/Intervals.java index 6a1e8b9c8b5a..c81ed0d2ffe9 100644 --- a/java-util/src/main/java/io/druid/java/util/common/Intervals.java +++ b/java-util/src/main/java/io/druid/java/util/common/Intervals.java @@ -38,6 +38,11 @@ public static Interval of(String interval) return new Interval(interval, ISOChronology.getInstanceUTC()); } + public static boolean isEmpty(Interval interval) + { + return interval.getStart().equals(interval.getEnd()); + } + private Intervals() { } diff --git a/server/src/main/java/io/druid/client/DruidDataSource.java b/server/src/main/java/io/druid/client/DruidDataSource.java index 84b04b3b146d..b1b6e1596760 100644 --- a/server/src/main/java/io/druid/client/DruidDataSource.java +++ b/server/src/main/java/io/druid/client/DruidDataSource.java @@ -70,6 +70,12 @@ public DruidDataSource addSegment(DataSegment dataSegment) return this; } + public DruidDataSource addSegments(Map partitionMap) + { + idToSegmentMap.putAll(partitionMap); + return this; + } + public DruidDataSource removePartition(String segmentId) { idToSegmentMap.remove(segmentId); diff --git a/server/src/main/java/io/druid/client/indexing/ClientCompactQuery.java b/server/src/main/java/io/druid/client/indexing/ClientCompactQuery.java new file mode 100644 index 000000000000..4c739448cd43 --- /dev/null +++ b/server/src/main/java/io/druid/client/indexing/ClientCompactQuery.java @@ -0,0 +1,90 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.client.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.timeline.DataSegment; + +import java.util.List; +import java.util.Map; + +public class ClientCompactQuery +{ + private final String dataSource; + private final List segments; + private final ClientCompactQueryTuningConfig tuningConfig; + private final Map context; + + @JsonCreator + public ClientCompactQuery( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segments") List segments, + @JsonProperty("tuningConfig") ClientCompactQueryTuningConfig tuningConfig, + @JsonProperty("context") Map context + ) + { + this.dataSource = dataSource; + this.segments = segments; + this.tuningConfig = tuningConfig; + this.context = context; + } + + @JsonProperty + public String getType() + { + return "compact"; + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public List getSegments() + { + return segments; + } + + @JsonProperty + public ClientCompactQueryTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @JsonProperty + public Map getContext() + { + return context; + } + + @Override + public String toString() + { + return "ClientCompactQuery{" + + "dataSource=" + dataSource + "'" + + ", segments=" + segments + + ", tuningConfig=" + tuningConfig + + ", contexts=" + context + + "}"; + } +} diff --git a/server/src/main/java/io/druid/client/indexing/ClientCompactQueryTuningConfig.java b/server/src/main/java/io/druid/client/indexing/ClientCompactQueryTuningConfig.java new file mode 100644 index 000000000000..5ca7027f4980 --- /dev/null +++ b/server/src/main/java/io/druid/client/indexing/ClientCompactQueryTuningConfig.java @@ -0,0 +1,139 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.client.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.segment.IndexSpec; + +import javax.annotation.Nullable; +import java.util.Objects; + +public class ClientCompactQueryTuningConfig +{ + // These default values should be synchronized with those of IndexTuningConfig + private static final int DEFAULT_MAX_ROWS_IN_MEMORY = 75_000; + private static final int DEFAULT_MAX_TOTAL_ROWS = 20_000_000; + private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); + private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; + private static final long DEFAULT_PUBLISH_TIMEOUT = 0; + + private final int maxRowsInMemory; + private final int maxTotalRows; + private final IndexSpec indexSpec; + private final int maxPendingPersists; + private final long publishTimeout; + + @JsonCreator + public ClientCompactQueryTuningConfig( + @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxTotalRows") @Nullable Integer maxTotalRows, + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, + @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, + @JsonProperty("publishTimeout") @Nullable Long publishTimeout + ) + { + this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; + this.maxTotalRows = maxTotalRows == null ? DEFAULT_MAX_TOTAL_ROWS : maxTotalRows; + this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; + this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists; + this.publishTimeout = publishTimeout == null ? DEFAULT_PUBLISH_TIMEOUT : publishTimeout; + } + + @JsonProperty + public int getMaxRowsInMemory() + { + return maxRowsInMemory; + } + + @JsonProperty + public int getMaxTotalRows() + { + return maxTotalRows; + } + + @JsonProperty + public IndexSpec getIndexSpec() + { + return indexSpec; + } + + @JsonProperty + public int getMaxPendingPersists() + { + return maxPendingPersists; + } + + @JsonProperty + public long getPublishTimeout() + { + return publishTimeout; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final ClientCompactQueryTuningConfig that = (ClientCompactQueryTuningConfig) o; + + if (maxRowsInMemory != that.maxRowsInMemory) { + return false; + } + + if (maxTotalRows != that.maxTotalRows) { + return false; + } + + if (!indexSpec.equals(that.indexSpec)) { + return false; + } + + if (maxPendingPersists != that.maxPendingPersists) { + return false; + } + + return publishTimeout == that.publishTimeout; + } + + @Override + public int hashCode() + { + return Objects.hash(maxRowsInMemory, maxTotalRows, indexSpec, maxPendingPersists, publishTimeout); + } + + @Override + public String toString() + { + return "ClientCompactQueryTuningConfig{" + + "maxRowsInMemory='" + maxRowsInMemory + + ", maxTotalRows='" + maxTotalRows + + ", indexSpec='" + indexSpec + + ", maxPendingPersists='" + maxPendingPersists + + ", publishTimeout='" + publishTimeout + + "}"; + } +} diff --git a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java index 39aa1e037df3..a5a47aa029c3 100644 --- a/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/io/druid/client/indexing/IndexingServiceClient.java @@ -41,6 +41,8 @@ import javax.annotation.Nullable; import javax.ws.rs.core.MediaType; import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -88,9 +90,78 @@ public void upgradeSegment(DataSegment dataSegment) runQuery(new ClientConversionQuery(dataSegment)); } - public void upgradeSegments(String dataSource, Interval interval) + public String compactSegments( + List segments, + int compactionTaskPriority, + @Nullable ClientCompactQueryTuningConfig tuningConfig, + @Nullable Map context + ) + { + Preconditions.checkArgument(segments.size() > 1, "Expect two or more segments to compact"); + + final String dataSource = segments.get(0).getDataSource(); + Preconditions.checkArgument( + segments.stream().allMatch(segment -> segment.getDataSource().equals(dataSource)), + "Segments must have the same dataSource" + ); + + context = context == null ? new HashMap<>() : context; + context.put("priority", compactionTaskPriority); + + return runQuery(new ClientCompactQuery(dataSource, segments, tuningConfig, context)); + } + + private String runQuery(Object queryObject) + { + try { + final FullResponseHolder response = druidLeaderClient.go( + druidLeaderClient.makeRequest( + HttpMethod.POST, + "/druid/indexer/v1/task" + ).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(queryObject)) + ); + + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE("Failed to post query[%s]", queryObject); + } + + final Map resultMap = jsonMapper.readValue( + response.getContent(), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + final String taskId = (String) resultMap.get("task"); + return Preconditions.checkNotNull(taskId, "Null task id for query[%s]", queryObject); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + public int getTotalWorkerCapacity() { - runQuery(new ClientConversionQuery(dataSource, interval)); + try { + final FullResponseHolder response = druidLeaderClient.go( + druidLeaderClient.makeRequest(HttpMethod.GET, "/druid/indexer/v1/workers") + .setHeader("Content-Type", MediaType.APPLICATION_JSON) + ); + + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while getting available cluster capacity. status[%s] content[%s]", + response.getStatus(), + response.getContent() + ); + } + final Collection workers = jsonMapper.readValue( + response.getContent(), + new TypeReference>() {} + ); + + return workers.stream().mapToInt(workerInfo -> workerInfo.getWorker().getCapacity()).sum(); + } + catch (Exception e) { + throw new RuntimeException(e); + } } public List getRunningTasks() @@ -166,19 +237,4 @@ public int killPendingSegments(String dataSource, DateTime end) throw new RuntimeException(e); } } - - private void runQuery(Object queryObject) - { - try { - druidLeaderClient.go( - druidLeaderClient.makeRequest( - HttpMethod.POST, - "/druid/indexer/v1/task" - ).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(queryObject)) - ); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } } diff --git a/server/src/main/java/io/druid/client/indexing/IndexingWorker.java b/server/src/main/java/io/druid/client/indexing/IndexingWorker.java new file mode 100644 index 000000000000..a91c2f8f40e2 --- /dev/null +++ b/server/src/main/java/io/druid/client/indexing/IndexingWorker.java @@ -0,0 +1,81 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.client.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Should be synchronized with io.druid.indexing.worker.Worker + */ +public class IndexingWorker +{ + private final String scheme; + private final String host; + private final String ip; + private final int capacity; + private final String version; + + @JsonCreator + public IndexingWorker( + @JsonProperty("scheme") String scheme, + @JsonProperty("host") String host, + @JsonProperty("ip") String ip, + @JsonProperty("capacity") int capacity, + @JsonProperty("version") String version + ) + { + this.scheme = scheme; + this.host = host; + this.ip = ip; + this.capacity = capacity; + this.version = version; + } + + @JsonProperty + public String getScheme() + { + return scheme; + } + + @JsonProperty + public String getHost() + { + return host; + } + + @JsonProperty + public String getIp() + { + return ip; + } + + @JsonProperty + public int getCapacity() + { + return capacity; + } + + @JsonProperty + public String getVersion() + { + return version; + } +} diff --git a/server/src/main/java/io/druid/client/indexing/IndexingWorkerInfo.java b/server/src/main/java/io/druid/client/indexing/IndexingWorkerInfo.java new file mode 100644 index 000000000000..c53feb6f30d1 --- /dev/null +++ b/server/src/main/java/io/druid/client/indexing/IndexingWorkerInfo.java @@ -0,0 +1,100 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.client.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.Set; + +/** + * Should be synchronized with io.druid.indexing.overlord.ImmutableWorkerInfo + */ +public class IndexingWorkerInfo +{ + private final IndexingWorker worker; + private final int currCapacityUsed; + private final Set availabilityGroups; + private final Collection runningTasks; + private final DateTime lastCompletedTaskTime; + private final DateTime blacklistedUntil; + + @JsonCreator + public IndexingWorkerInfo( + @JsonProperty("worker") IndexingWorker worker, + @JsonProperty("currCapacityUsed") int currCapacityUsed, + @JsonProperty("availabilityGroups") Set availabilityGroups, + @JsonProperty("runningTasks") Collection runningTasks, + @JsonProperty("lastCompletedTaskTime") DateTime lastCompletedTaskTime, + @Nullable @JsonProperty("blacklistedUntil") DateTime blacklistedUntil + ) + { + this.worker = worker; + this.currCapacityUsed = currCapacityUsed; + this.availabilityGroups = availabilityGroups; + this.runningTasks = runningTasks; + this.lastCompletedTaskTime = lastCompletedTaskTime; + this.blacklistedUntil = blacklistedUntil; + } + + @JsonProperty("worker") + public IndexingWorker getWorker() + { + return worker; + } + + @JsonProperty("currCapacityUsed") + public int getCurrCapacityUsed() + { + return currCapacityUsed; + } + + @JsonProperty("availabilityGroups") + public Set getAvailabilityGroups() + { + return availabilityGroups; + } + + public int getAvailableCapacity() + { + return getWorker().getCapacity() - getCurrCapacityUsed(); + } + + @JsonProperty("runningTasks") + public Collection getRunningTasks() + { + return runningTasks; + } + + @JsonProperty("lastCompletedTaskTime") + public DateTime getLastCompletedTaskTime() + { + return lastCompletedTaskTime; + } + + @JsonProperty + public DateTime getBlacklistedUntil() + { + return blacklistedUntil; + } +} diff --git a/server/src/main/java/io/druid/client/indexing/QueryStatus.java b/server/src/main/java/io/druid/client/indexing/QueryStatus.java new file mode 100644 index 000000000000..71ff7592a403 --- /dev/null +++ b/server/src/main/java/io/druid/client/indexing/QueryStatus.java @@ -0,0 +1,89 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.client.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; + +/** + * Should be synchronized with io.druid.indexing.common.TaskStatus. + */ +public class QueryStatus +{ + public enum Status + { + RUNNING, + SUCCESS, + FAILED + } + + private final String id; + private final Status status; + private final long duration; + + @JsonCreator + public QueryStatus( + @JsonProperty("id") String id, + @JsonProperty("status") Status status, + @JsonProperty("duration") long duration + ) + { + this.id = Preconditions.checkNotNull(id, "id"); + this.status = Preconditions.checkNotNull(status, "status"); + this.duration = duration; + } + + @JsonProperty("id") + public String getId() + { + return id; + } + + @JsonProperty("status") + public Status getStatusCode() + { + return status; + } + + @JsonProperty("duration") + public long getDuration() + { + return duration; + } + + @JsonIgnore + public boolean isComplete() + { + return status != Status.RUNNING; + } + + @Override + public String toString() + { + return Objects.toStringHelper(this) + .add("id", id) + .add("status", status) + .add("duration", duration) + .toString(); + } +} diff --git a/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java index eca6620f17a4..9a6ad7f11d52 100644 --- a/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/io/druid/metadata/SQLMetadataSegmentManager.java @@ -108,9 +108,7 @@ public SQLMetadataSegmentManager( this.jsonMapper = jsonMapper; this.config = config; this.dbTables = dbTables; - this.dataSourcesRef = new AtomicReference<>( - new ConcurrentHashMap() - ); + this.dataSourcesRef = new AtomicReference<>(new ConcurrentHashMap<>()); this.connector = connector; } diff --git a/server/src/main/java/io/druid/server/coordinator/CoordinatorCompactionConfig.java b/server/src/main/java/io/druid/server/coordinator/CoordinatorCompactionConfig.java new file mode 100644 index 000000000000..bcf4e4844e01 --- /dev/null +++ b/server/src/main/java/io/druid/server/coordinator/CoordinatorCompactionConfig.java @@ -0,0 +1,178 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.coordinator; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import io.druid.client.indexing.ClientCompactQueryTuningConfig; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import java.util.Map; +import java.util.Objects; + +public class CoordinatorCompactionConfig +{ + // should be synchronized with Tasks.DEFAULT_MERGE_TASK_PRIORITY + private static final int DEFAULT_COMPACTION_TASK_PRIORITY = 25; + private static final long DEFAULT_TARGET_COMPACTION_SIZE_BYTES = 800 * 1024 * 1024; // 800MB + private static final int DEFAULT_NUM_TARGET_COMPACTION_SEGMENTS = 150; + private static final Period DEFAULT_SKIP_OFFSET_FROM_LATEST = new Period("P1D"); + + private final String dataSource; + private final int taskPriority; + private final long targetCompactionSizeBytes; + // The number of compaction segments is limited because the byte size of a serialized task spec is limited by + // RemoteTaskRunnerConfig.maxZnodeBytes. + private final int numTargetCompactionSegments; + private final Period skipOffsetFromLatest; + private final ClientCompactQueryTuningConfig tuningConfig; + private final Map taskContext; + + @JsonCreator + public CoordinatorCompactionConfig( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("taskPriority") Integer taskPriority, + @JsonProperty("targetCompactionSizeBytes") Long targetCompactionSizeBytes, + @JsonProperty("numTargetCompactionSegments") Integer numTargetCompactionSegments, + @JsonProperty("skipOffsetFromLatest") Period skipOffsetFromLatest, + @JsonProperty("tuningConfig") ClientCompactQueryTuningConfig tuningConfig, + @JsonProperty("taskContext") Map taskContext + ) + { + this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); + this.taskPriority = taskPriority == null ? + DEFAULT_COMPACTION_TASK_PRIORITY : + taskPriority; + this.targetCompactionSizeBytes = targetCompactionSizeBytes == null ? + DEFAULT_TARGET_COMPACTION_SIZE_BYTES : + targetCompactionSizeBytes; + this.numTargetCompactionSegments = numTargetCompactionSegments == null ? + DEFAULT_NUM_TARGET_COMPACTION_SEGMENTS : + numTargetCompactionSegments; + this.skipOffsetFromLatest = skipOffsetFromLatest == null ? DEFAULT_SKIP_OFFSET_FROM_LATEST : skipOffsetFromLatest; + this.tuningConfig = tuningConfig; + this.taskContext = taskContext; + + Preconditions.checkArgument( + this.numTargetCompactionSegments > 1, + "numTargetCompactionSegments should be larger than 1" + ); + } + + @JsonProperty + public String getDataSource() + { + return dataSource; + } + + @JsonProperty + public int getTaskPriority() + { + return taskPriority; + } + + @JsonProperty + public long getTargetCompactionSizeBytes() + { + return targetCompactionSizeBytes; + } + + @JsonProperty + public int getNumTargetCompactionSegments() + { + return numTargetCompactionSegments; + } + + @JsonProperty + public Period getSkipOffsetFromLatest() + { + return skipOffsetFromLatest; + } + + @JsonProperty + @Nullable + public ClientCompactQueryTuningConfig getTuningConfig() + { + return tuningConfig; + } + + @JsonProperty + @Nullable + public Map getTaskContext() + { + return taskContext; + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + final CoordinatorCompactionConfig that = (CoordinatorCompactionConfig) o; + + if (!dataSource.equals(that.dataSource)) { + return false; + } + + if (taskPriority != that.taskPriority) { + return false; + } + + if (targetCompactionSizeBytes != that.targetCompactionSizeBytes) { + return false; + } + + if (numTargetCompactionSegments != that.numTargetCompactionSegments) { + return false; + } + + if (!skipOffsetFromLatest.equals(that.skipOffsetFromLatest)) { + return false; + } + + if (!Objects.equals(tuningConfig, that.tuningConfig)) { + return false; + } + + return Objects.equals(taskContext, that.taskContext); + } + + @Override + public int hashCode() + { + return Objects.hash( + dataSource, + taskPriority, + targetCompactionSizeBytes, + numTargetCompactionSegments, + skipOffsetFromLatest, + tuningConfig, + taskContext + ); + } +} diff --git a/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java index 9a71ba30fd7d..158db06c616d 100644 --- a/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/io/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -25,6 +25,7 @@ import java.util.Collection; import java.util.HashSet; +import java.util.List; import java.util.Objects; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -46,6 +47,7 @@ public class CoordinatorDynamicConfig // The pending segments of the dataSources in this list are not killed. private final Set killPendingSegmentsSkipList; + /** * The maximum number of segments that could be queued for loading to any given server. * Default values is 0 with the meaning of "unbounded" (any number of @@ -53,6 +55,9 @@ public class CoordinatorDynamicConfig * See {@link LoadQueuePeon}, {@link io.druid.server.coordinator.rules.LoadRule#run} */ private final int maxSegmentsInNodeLoadingQueue; + private final List compactionConfigs; + private final double compactionTaskSlotRatio; + private final int maxCompactionTaskSlots; @JsonCreator public CoordinatorDynamicConfig( @@ -69,9 +74,12 @@ public CoordinatorDynamicConfig( // coordinator console can not send array of strings in the update request. // See https://github.com/druid-io/druid/issues/3055 @JsonProperty("killDataSourceWhitelist") Object killDataSourceWhitelist, - @JsonProperty("killPendingSegmentsSkipList") Object killPendingSegmentsSkipList, @JsonProperty("killAllDataSources") boolean killAllDataSources, - @JsonProperty("maxSegmentsInNodeLoadingQueue") int maxSegmentsInNodeLoadingQueue + @JsonProperty("killPendingSegmentsSkipList") Object killPendingSegmentsSkipList, + @JsonProperty("maxSegmentsInNodeLoadingQueue") int maxSegmentsInNodeLoadingQueue, + @JsonProperty("compactionConfigs") List compactionConfigs, + @JsonProperty("compactionTaskSlotRatio") double compactionTaskSlotRatio, + @JsonProperty("maxCompactionTaskSlots") int maxCompactionTaskSlots ) { this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; @@ -86,6 +94,9 @@ public CoordinatorDynamicConfig( this.killDataSourceWhitelist = parseJsonStringOrArray(killDataSourceWhitelist); this.killPendingSegmentsSkipList = parseJsonStringOrArray(killPendingSegmentsSkipList); this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; + this.compactionConfigs = compactionConfigs; + this.compactionTaskSlotRatio = compactionTaskSlotRatio; + this.maxCompactionTaskSlots = maxCompactionTaskSlots; if (this.killAllDataSources && !this.killDataSourceWhitelist.isEmpty()) { throw new IAE("can't have killAllDataSources and non-empty killDataSourceWhitelist"); @@ -166,15 +177,15 @@ public Set getKillDataSourceWhitelist() } @JsonProperty - public Set getKillPendingSegmentsSkipList() + public boolean isKillAllDataSources() { - return killPendingSegmentsSkipList; + return killAllDataSources; } @JsonProperty - public boolean isKillAllDataSources() + public Set getKillPendingSegmentsSkipList() { - return killAllDataSources; + return killPendingSegmentsSkipList; } @JsonProperty @@ -183,6 +194,24 @@ public int getMaxSegmentsInNodeLoadingQueue() return maxSegmentsInNodeLoadingQueue; } + @JsonProperty + public List getCompactionConfigs() + { + return compactionConfigs; + } + + @JsonProperty + public double getCompactionTaskSlotRatio() + { + return compactionTaskSlotRatio; + } + + @JsonProperty + public int getMaxCompactionTaskSlots() + { + return maxCompactionTaskSlots; + } + @Override public String toString() { @@ -196,9 +225,12 @@ public String toString() ", balancerComputeThreads=" + balancerComputeThreads + ", emitBalancingStats=" + emitBalancingStats + ", killDataSourceWhitelist=" + killDataSourceWhitelist + - ", killPendingSegmentsSkipList=" + killPendingSegmentsSkipList + ", killAllDataSources=" + killAllDataSources + + ", killPendingSegmentsSkipList=" + killPendingSegmentsSkipList + ", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue + + ", compactionConfigs=" + compactionConfigs + + ", compactionTaskSlotRatio=" + compactionTaskSlotRatio + + ", maxCompactionTaskSlots=" + maxCompactionTaskSlots + '}'; } @@ -247,8 +279,17 @@ public boolean equals(Object o) if (!Objects.equals(killDataSourceWhitelist, that.killDataSourceWhitelist)) { return false; } + if (!Objects.equals(killPendingSegmentsSkipList, that.killPendingSegmentsSkipList)) { + return false; + } + if (!Objects.equals(compactionConfigs, that.compactionConfigs)) { + return false; + } + if (compactionTaskSlotRatio != that.compactionTaskSlotRatio) { + return false; + } - return Objects.equals(killPendingSegmentsSkipList, that.killPendingSegmentsSkipList); + return maxCompactionTaskSlots == that.maxCompactionTaskSlots; } @Override @@ -266,7 +307,10 @@ public int hashCode() killAllDataSources, maxSegmentsInNodeLoadingQueue, killDataSourceWhitelist, - killPendingSegmentsSkipList + killPendingSegmentsSkipList, + compactionConfigs, + compactionTaskSlotRatio, + maxCompactionTaskSlots ); } @@ -287,6 +331,8 @@ public static class Builder private static final boolean DEFAULT_EMIT_BALANCING_STATS = false; private static final boolean DEFAULT_KILL_ALL_DATA_SOURCES = false; private static final int DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE = 0; + private static final double DEFAULT_COMPACTION_TASK_RATIO = 0.1; + private static final int DEFAILT_MAX_COMPACTION_TASK_SLOTS = Integer.MAX_VALUE; private Long millisToWaitBeforeDeleting; private Long mergeBytesLimit; @@ -297,9 +343,12 @@ public static class Builder private Boolean emitBalancingStats; private Integer balancerComputeThreads; private Object killDataSourceWhitelist; - private Object killPendingSegmentsSkipList; private Boolean killAllDataSources; + private Object killPendingSegmentsSkipList; private Integer maxSegmentsInNodeLoadingQueue; + private List compactionConfigs; + private Double compactionTaskRatio; + private Integer maxCompactionTaskSlots; public Builder() { @@ -316,9 +365,12 @@ public Builder( @JsonProperty("balancerComputeThreads") Integer balancerComputeThreads, @JsonProperty("emitBalancingStats") Boolean emitBalancingStats, @JsonProperty("killDataSourceWhitelist") Object killDataSourceWhitelist, - @JsonProperty("killPendingSegmentsSkipList") Object killPendingSegmentsSkipList, @JsonProperty("killAllDataSources") Boolean killAllDataSources, - @JsonProperty("maxSegmentsInNodeLoadingQueue") Integer maxSegmentsInNodeLoadingQueue + @JsonProperty("killPendingSegmentsSkipList") Object killPendingSegmentsSkipList, + @JsonProperty("maxSegmentsInNodeLoadingQueue") Integer maxSegmentsInNodeLoadingQueue, + @JsonProperty("compactionConfigs") List compactionConfigs, + @JsonProperty("compactionTaskSlotRatio") Double compactionTaskRatio, + @JsonProperty("maxCompactionTaskSlots") Integer maxCompactionTaskSlots ) { this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; @@ -333,6 +385,9 @@ public Builder( this.killDataSourceWhitelist = killDataSourceWhitelist; this.killPendingSegmentsSkipList = killPendingSegmentsSkipList; this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; + this.compactionConfigs = compactionConfigs; + this.compactionTaskRatio = compactionTaskRatio; + this.maxCompactionTaskSlots = maxCompactionTaskSlots; } public Builder withMillisToWaitBeforeDeleting(long millisToWaitBeforeDeleting) @@ -401,6 +456,24 @@ public Builder withMaxSegmentsInNodeLoadingQueue(int maxSegmentsInNodeLoadingQue return this; } + public Builder withCompactionConfigs(List compactionConfigs) + { + this.compactionConfigs = compactionConfigs; + return this; + } + + public Builder withCompactionTaskRatio(double compactionTaskRatio) + { + this.compactionTaskRatio = compactionTaskRatio; + return this; + } + + public Builder withMaxCompactionTaskSlots(int maxCompactionTaskSlots) + { + this.maxCompactionTaskSlots = maxCompactionTaskSlots; + return this; + } + public CoordinatorDynamicConfig build() { return new CoordinatorDynamicConfig( @@ -413,9 +486,12 @@ public CoordinatorDynamicConfig build() balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads, emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats, killDataSourceWhitelist, - killPendingSegmentsSkipList, killAllDataSources == null ? DEFAULT_KILL_ALL_DATA_SOURCES : killAllDataSources, - maxSegmentsInNodeLoadingQueue == null ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE : maxSegmentsInNodeLoadingQueue + killPendingSegmentsSkipList, + maxSegmentsInNodeLoadingQueue == null ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE : maxSegmentsInNodeLoadingQueue, + compactionConfigs, + compactionTaskRatio == null ? DEFAULT_COMPACTION_TASK_RATIO : compactionTaskRatio, + maxCompactionTaskSlots == null ? DEFAILT_MAX_COMPACTION_TASK_SLOTS : maxCompactionTaskSlots ); } @@ -431,9 +507,12 @@ public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads, emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats, killDataSourceWhitelist == null ? defaults.getKillDataSourceWhitelist() : killDataSourceWhitelist, - killPendingSegmentsSkipList == null ? defaults.getKillPendingSegmentsSkipList() : killPendingSegmentsSkipList, killAllDataSources == null ? defaults.isKillAllDataSources() : killAllDataSources, - maxSegmentsInNodeLoadingQueue == null ? defaults.getMaxSegmentsInNodeLoadingQueue() : maxSegmentsInNodeLoadingQueue + killPendingSegmentsSkipList == null ? defaults.getKillPendingSegmentsSkipList() : killPendingSegmentsSkipList, + maxSegmentsInNodeLoadingQueue == null ? defaults.getMaxSegmentsInNodeLoadingQueue() : maxSegmentsInNodeLoadingQueue, + compactionConfigs == null ? defaults.getCompactionConfigs() : compactionConfigs, + compactionTaskRatio == null ? defaults.getCompactionTaskSlotRatio() : compactionTaskRatio, + maxCompactionTaskSlots == null ? defaults.getMaxCompactionTaskSlots() : maxCompactionTaskSlots ); } } diff --git a/server/src/main/java/io/druid/server/coordinator/CoordinatorStats.java b/server/src/main/java/io/druid/server/coordinator/CoordinatorStats.java index 4589b141fc2e..52cb80771c90 100644 --- a/server/src/main/java/io/druid/server/coordinator/CoordinatorStats.java +++ b/server/src/main/java/io/druid/server/coordinator/CoordinatorStats.java @@ -19,11 +19,12 @@ package io.druid.server.coordinator; -import com.google.common.collect.Maps; import it.unimi.dsi.fastutil.objects.Object2LongMap; +import it.unimi.dsi.fastutil.objects.Object2LongMap.Entry; import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.Set; import java.util.function.ObjLongConsumer; @@ -33,11 +34,13 @@ public class CoordinatorStats { private final Map> perTierStats; + private final Map> perDataSourceStats; private final Object2LongOpenHashMap globalStats; public CoordinatorStats() { - perTierStats = Maps.newHashMap(); + perTierStats = new HashMap<>(); + perDataSourceStats = new HashMap<>(); globalStats = new Object2LongOpenHashMap<>(); } @@ -46,6 +49,11 @@ public boolean hasPerTierStats() return !perTierStats.isEmpty(); } + public boolean hasPerDataSourceStats() + { + return !perDataSourceStats.isEmpty(); + } + public Set getTiers(final String statName) { final Object2LongOpenHashMap theStat = perTierStats.get(statName); @@ -55,6 +63,15 @@ public Set getTiers(final String statName) return Collections.unmodifiableSet(theStat.keySet()); } + public Set getDataSources(String statName) + { + final Object2LongOpenHashMap stat = perDataSourceStats.get(statName); + if (stat == null) { + return Collections.emptySet(); + } + return Collections.unmodifiableSet(stat.keySet()); + } + /** * * @param statName the name of the statistics @@ -77,6 +94,21 @@ public void forEachTieredStat(final String statName, final ObjLongConsumer consumer) + { + final Object2LongOpenHashMap stat = perDataSourceStats.get(statName); + if (stat != null) { + for (Entry entry : stat.object2LongEntrySet()) { + consumer.accept(entry.getKey(), entry.getLongValue()); + } + } + } + public long getGlobalStat(final String statName) { return globalStats.getLong(statName); @@ -88,6 +120,12 @@ public void addToTieredStat(final String statName, final String tier, final long .addTo(tier, value); } + public void addToDataSourceStat(String statName, String dataSource, long value) + { + perDataSourceStats.computeIfAbsent(statName, k -> new Object2LongOpenHashMap<>()) + .addTo(dataSource, value); + } + public void addToGlobalStat(final String statName, final long value) { globalStats.addTo(statName, value); @@ -108,6 +146,19 @@ public CoordinatorStats accumulate(final CoordinatorStats stats) } ); + stats.perDataSourceStats.forEach( + (statName, urStat) -> { + final Object2LongOpenHashMap myStat = perDataSourceStats.computeIfAbsent( + statName, + k -> new Object2LongOpenHashMap<>() + ); + + for (Entry entry : urStat.object2LongEntrySet()) { + myStat.addTo(entry.getKey(), entry.getLongValue()); + } + } + ); + for (final Object2LongMap.Entry entry : stats.globalStats.object2LongEntrySet()) { globalStats.addTo(entry.getKey(), entry.getLongValue()); } diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java index 7239aca5b223..9c266fcbbe4a 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinator.java @@ -65,6 +65,7 @@ import io.druid.server.coordinator.helper.DruidCoordinatorHelper; import io.druid.server.coordinator.helper.DruidCoordinatorLogger; import io.druid.server.coordinator.helper.DruidCoordinatorRuleRunner; +import io.druid.server.coordinator.helper.DruidCoordinatorSegmentCompactor; import io.druid.server.coordinator.helper.DruidCoordinatorSegmentInfoLoader; import io.druid.server.coordinator.rules.LoadRule; import io.druid.server.coordinator.rules.Rule; @@ -317,7 +318,7 @@ public CoordinatorDynamicConfig getDynamicConfigs() return configManager.watch( CoordinatorDynamicConfig.CONFIG_KEY, CoordinatorDynamicConfig.class, - new CoordinatorDynamicConfig.Builder().build() + CoordinatorDynamicConfig.builder().build() ).get(); } @@ -590,9 +591,13 @@ private List makeIndexingServiceHelpers() { List helpers = Lists.newArrayList(); helpers.add(new DruidCoordinatorSegmentInfoLoader(DruidCoordinator.this)); + helpers.add(new DruidCoordinatorSegmentCompactor(indexingServiceClient)); helpers.addAll(indexingServiceHelpers); - log.info("Done making indexing service helpers [%s]", helpers); + log.info( + "Done making indexing service helpers [%s]", + helpers.stream().map(helper -> helper.getClass().getCanonicalName()).collect(Collectors.toList()) + ); return ImmutableList.copyOf(helpers); } @@ -643,7 +648,7 @@ public void run() DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams.newBuilder() .withStartTime(startTime) - .withDatasources(metadataSegmentManager.getInventory()) + .withDataSources(metadataSegmentManager.getInventory()) .withDynamicConfigs(getDynamicConfigs()) .withEmitter(emitter) .withBalancerStrategy(balancerStrategy) diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java index 40af4260324e..ed999843437d 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java @@ -21,7 +21,6 @@ import com.google.common.base.Preconditions; import com.google.inject.Inject; -import io.druid.client.ImmutableDruidDataSource; import io.druid.client.indexing.IndexingServiceClient; import io.druid.indexer.TaskStatusPlus; import io.druid.java.util.common.DateTimes; @@ -89,11 +88,11 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // If there is no running/pending/waiting/complete tasks, pendingSegmentsCleanupEndTime is // (DateTimes.nowUtc() - KEEP_PENDING_SEGMENTS_OFFSET). final DateTime pendingSegmentsCleanupEndTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); - for (ImmutableDruidDataSource dataSource : params.getDataSources()) { - if (!params.getCoordinatorDynamicConfig().getKillPendingSegmentsSkipList().contains(dataSource.getName())) { + for (String dataSource : params.getDataSources().keySet()) { + if (!params.getCoordinatorDynamicConfig().getKillPendingSegmentsSkipList().contains(dataSource)) { log.info( "Killed [%d] pendingSegments created until [%s] for dataSource[%s]", - indexingServiceClient.killPendingSegments(dataSource.getName(), pendingSegmentsCleanupEndTime), + indexingServiceClient.killPendingSegments(dataSource, pendingSegmentsCleanupEndTime), pendingSegmentsCleanupEndTime, dataSource ); diff --git a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index e2b51771c175..512ab41a47cd 100644 --- a/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorRuntimeParams.java +++ b/server/src/main/java/io/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -26,11 +26,12 @@ import io.druid.java.util.common.DateTimes; import io.druid.metadata.MetadataRuleManager; import io.druid.timeline.DataSegment; +import io.druid.timeline.VersionedIntervalTimeline; import org.joda.time.DateTime; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; +import java.util.HashMap; import java.util.Map; import java.util.Set; import java.util.TreeSet; @@ -43,7 +44,7 @@ public class DruidCoordinatorRuntimeParams private final DruidCluster druidCluster; private final MetadataRuleManager databaseRuleManager; private final SegmentReplicantLookup segmentReplicantLookup; - private final Set dataSources; + private final Map> dataSources; private final Set availableSegments; private final Map loadManagementPeons; private final ReplicationThrottler replicationManager; @@ -58,7 +59,7 @@ private DruidCoordinatorRuntimeParams( DruidCluster druidCluster, MetadataRuleManager databaseRuleManager, SegmentReplicantLookup segmentReplicantLookup, - Set dataSources, + Map> dataSources, Set availableSegments, Map loadManagementPeons, ReplicationThrottler replicationManager, @@ -104,7 +105,7 @@ public SegmentReplicantLookup getSegmentReplicantLookup() return segmentReplicantLookup; } - public Set getDataSources() + public Map> getDataSources() { return dataSources; } @@ -203,7 +204,7 @@ public static class Builder private DruidCluster druidCluster; private MetadataRuleManager databaseRuleManager; private SegmentReplicantLookup segmentReplicantLookup; - private final Set dataSources; + private Map> dataSources; private final Set availableSegments; private final Map loadManagementPeons; private ReplicationThrottler replicationManager; @@ -219,13 +220,13 @@ public static class Builder this.druidCluster = null; this.databaseRuleManager = null; this.segmentReplicantLookup = null; - this.dataSources = new HashSet<>(); + this.dataSources = new HashMap<>(); this.availableSegments = new TreeSet<>(DruidCoordinator.SEGMENT_COMPARATOR); this.loadManagementPeons = Maps.newHashMap(); this.replicationManager = null; this.emitter = null; this.stats = new CoordinatorStats(); - this.coordinatorDynamicConfig = new CoordinatorDynamicConfig.Builder().build(); + this.coordinatorDynamicConfig = CoordinatorDynamicConfig.builder().build(); this.balancerReferenceTimestamp = DateTimes.nowUtc(); } @@ -234,7 +235,7 @@ public static class Builder DruidCluster cluster, MetadataRuleManager databaseRuleManager, SegmentReplicantLookup segmentReplicantLookup, - Set dataSources, + Map> dataSources, Set availableSegments, Map loadManagementPeons, ReplicationThrottler replicationManager, @@ -303,9 +304,30 @@ public Builder withSegmentReplicantLookup(SegmentReplicantLookup lookup) return this; } - public Builder withDatasources(Collection dataSourcesCollection) + public Builder withDataSources(Map> dataSources) { - dataSources.addAll(Collections.unmodifiableCollection(dataSourcesCollection)); + this.dataSources = dataSources; + return this; + } + + public Builder withDataSources(Collection dataSourcesCollection) + { + dataSourcesCollection.forEach( + dataSource -> { + VersionedIntervalTimeline timeline = dataSources.computeIfAbsent( + dataSource.getName(), + k -> new VersionedIntervalTimeline<>(String.CASE_INSENSITIVE_ORDER) + ); + + dataSource.getSegments().forEach( + segment -> timeline.add( + segment.getInterval(), + segment.getVersion(), + segment.getShardSpec().createChunk(segment) + ) + ); + } + ); return this; } diff --git a/server/src/main/java/io/druid/server/coordinator/helper/CompactionSegmentIterator.java b/server/src/main/java/io/druid/server/coordinator/helper/CompactionSegmentIterator.java new file mode 100644 index 000000000000..3acc478543ea --- /dev/null +++ b/server/src/main/java/io/druid/server/coordinator/helper/CompactionSegmentIterator.java @@ -0,0 +1,35 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.coordinator.helper; + +import io.druid.timeline.DataSegment; +import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; + +import java.util.Iterator; +import java.util.List; + +public interface CompactionSegmentIterator extends Iterator> +{ + /** + * Return a map of (dataSource, number of remaining segments) for all dataSources. + * This method should consider all segments except the segments returned by {@link #next()}. + */ + Object2LongOpenHashMap remainingSegments(); +} diff --git a/server/src/main/java/io/druid/server/coordinator/helper/CompactionSegmentSearchPolicy.java b/server/src/main/java/io/druid/server/coordinator/helper/CompactionSegmentSearchPolicy.java new file mode 100644 index 000000000000..b3b97be317de --- /dev/null +++ b/server/src/main/java/io/druid/server/coordinator/helper/CompactionSegmentSearchPolicy.java @@ -0,0 +1,40 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.coordinator.helper; + +import io.druid.server.coordinator.CoordinatorCompactionConfig; +import io.druid.timeline.DataSegment; +import io.druid.timeline.VersionedIntervalTimeline; + +import java.util.Map; + +/** + * Segment searching policy used by {@link DruidCoordinatorSegmentCompactor}. + */ +public interface CompactionSegmentSearchPolicy +{ + /** + * Reset the current states of this policy. This method should be called whenever iterating starts. + */ + CompactionSegmentIterator reset( + Map compactionConfigs, + Map> dataSources + ); +} diff --git a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java index ced5efb1dc78..b7655b70a13e 100644 --- a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java +++ b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorLogger.java @@ -31,11 +31,13 @@ import io.druid.server.coordinator.LoadQueuePeon; import io.druid.server.coordinator.ServerHolder; import io.druid.timeline.DataSegment; +import io.druid.timeline.partition.PartitionChunk; import it.unimi.dsi.fastutil.objects.Object2LongMap; import java.util.List; import java.util.stream.Collectors; import java.util.stream.Stream; +import java.util.stream.StreamSupport; /** */ @@ -254,11 +256,33 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) } ); + emitter.emit( + new ServiceMetricEvent.Builder().build( + "compact/task/count", + stats.getGlobalStat("compactTaskCount") + ) + ); + + stats.forEachDataSourceStat( + "segmentsWaitCompact", + (final String dataSource, final long count) -> { + emitter.emit( + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.DATASOURCE, dataSource) + .build("segment/waitCompact/count", count) + ); + } + ); + // Emit segment metrics final Stream allSegments = params .getDataSources() + .values() .stream() - .flatMap(dataSource -> dataSource.getSegments().stream()); + .flatMap(timeline -> timeline.getAllTimelineEntries().values().stream()) + .flatMap(entryMap -> entryMap.values().stream()) + .flatMap(entry -> StreamSupport.stream(entry.getPartitionHolder().spliterator(), false)) + .map(PartitionChunk::getObject); allSegments .collect(Collectors.groupingBy(DataSegment::getDataSource)) diff --git a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java new file mode 100644 index 000000000000..a703ad4c5b62 --- /dev/null +++ b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java @@ -0,0 +1,153 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.coordinator.helper; + +import com.google.inject.Inject; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.logger.Logger; +import io.druid.server.coordinator.CoordinatorCompactionConfig; +import io.druid.server.coordinator.CoordinatorDynamicConfig; +import io.druid.server.coordinator.CoordinatorStats; +import io.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import io.druid.timeline.DataSegment; +import io.druid.timeline.VersionedIntervalTimeline; + +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class DruidCoordinatorSegmentCompactor implements DruidCoordinatorHelper +{ + static final String COMPACT_TASK_COUNT = "compactTaskCount"; + static final String SEGMENTS_WAIT_COMPACT = "segmentsWaitCompact"; + + // Should be synced with CompactionTask.TYPE + private static final String COMPACT_TASK_TYPE = "compact"; + private static final Logger LOG = new Logger(DruidCoordinatorSegmentCompactor.class); + + private final CompactionSegmentSearchPolicy policy = new NewestSegmentFirstPolicy(); + private final IndexingServiceClient indexingServiceClient; + + @Inject + public DruidCoordinatorSegmentCompactor(IndexingServiceClient indexingServiceClient) + { + this.indexingServiceClient = indexingServiceClient; + } + + @Override + public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) + { + LOG.info("Run coordinator segment compactor"); + + final CoordinatorDynamicConfig dynamicConfig = params.getCoordinatorDynamicConfig(); + final CoordinatorStats stats = new CoordinatorStats(); + + if (dynamicConfig.getMaxCompactionTaskSlots() > 0) { + Map> dataSources = params.getDataSources(); + List compactionConfigList = dynamicConfig.getCompactionConfigs(); + + if (compactionConfigList != null && !compactionConfigList.isEmpty()) { + Map compactionConfigs = compactionConfigList + .stream() + .collect(Collectors.toMap(CoordinatorCompactionConfig::getDataSource, Function.identity())); + final int numRunningCompactTasks = indexingServiceClient + .getRunningTasks() + .stream() + .filter(status -> status.getType().equals(COMPACT_TASK_TYPE)) + .collect(Collectors.toList()) + .size(); + final CompactionSegmentIterator iterator = policy.reset(compactionConfigs, dataSources); + + final int compactionTaskCapacity = (int) Math.min( + indexingServiceClient.getTotalWorkerCapacity() * dynamicConfig.getCompactionTaskSlotRatio(), + dynamicConfig.getMaxCompactionTaskSlots() + ); + final int numAvailableCompactionTaskSlots = numRunningCompactTasks > 0 ? + compactionTaskCapacity - numRunningCompactTasks : + // compactionTaskCapacity might be 0 if totalWorkerCapacity is low. + // This guarantees that at least one slot is available if + // compaction is enabled and numRunningCompactTasks is 0. + Math.max(1, compactionTaskCapacity); + LOG.info("Running tasks [%d/%d]", numRunningCompactTasks, compactionTaskCapacity); + if (numAvailableCompactionTaskSlots > 0) { + stats.accumulate(doRun(compactionConfigs, numAvailableCompactionTaskSlots, iterator)); + } else { + stats.accumulate(makeStats(0, iterator)); + } + } else { + LOG.info("compactionConfig is empty. Skip."); + } + } else { + LOG.info("maxCompactionTaskSlots was set to 0. Skip compaction"); + } + + return params.buildFromExisting() + .withCoordinatorStats(stats) + .build(); + } + + private CoordinatorStats doRun( + Map compactionConfigs, + int numAvailableCompactionTaskSlots, + CompactionSegmentIterator iterator + ) + { + int numSubmittedTasks = 0; + + for (; iterator.hasNext() && numSubmittedTasks < numAvailableCompactionTaskSlots; numSubmittedTasks++) { + final List segmentsToCompact = iterator.next(); + + final String dataSourceName = segmentsToCompact.get(0).getDataSource(); + + if (segmentsToCompact.size() > 1) { + final CoordinatorCompactionConfig config = compactionConfigs.get(dataSourceName); + final String taskId = indexingServiceClient.compactSegments( + segmentsToCompact, + config.getTaskPriority(), + config.getTuningConfig(), + config.getTaskContext() + ); + LOG.info("Submitted a compactTask[%s] for segments[%s]", taskId, segmentsToCompact); + } else if (segmentsToCompact.size() == 1) { + throw new ISE("Found one segments[%s] to compact", segmentsToCompact); + } else { + throw new ISE("Failed to find segments for dataSource[%s]", dataSourceName); + } + } + + return makeStats(numSubmittedTasks, iterator); + } + + private CoordinatorStats makeStats(int numCompactionTasks, CompactionSegmentIterator iterator) + { + final CoordinatorStats stats = new CoordinatorStats(); + stats.addToGlobalStat(COMPACT_TASK_COUNT, numCompactionTasks); + iterator.remainingSegments().object2LongEntrySet().fastForEach( + entry -> { + final String dataSource = entry.getKey(); + final long numSegmentsWaitCompact = entry.getLongValue(); + stats.addToDataSourceStat(SEGMENTS_WAIT_COMPACT, dataSource, numSegmentsWaitCompact); + } + ); + return stats; + } +} diff --git a/server/src/main/java/io/druid/server/coordinator/helper/NewestSegmentFirstIterator.java b/server/src/main/java/io/druid/server/coordinator/helper/NewestSegmentFirstIterator.java new file mode 100644 index 000000000000..a66db31a1f3f --- /dev/null +++ b/server/src/main/java/io/druid/server/coordinator/helper/NewestSegmentFirstIterator.java @@ -0,0 +1,464 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.coordinator.helper; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.Pair; +import io.druid.java.util.common.guava.Comparators; +import io.druid.java.util.common.logger.Logger; +import io.druid.server.coordinator.CoordinatorCompactionConfig; +import io.druid.timeline.DataSegment; +import io.druid.timeline.TimelineObjectHolder; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.PartitionChunk; +import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.joda.time.Period; + +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.Map.Entry; +import java.util.NoSuchElementException; +import java.util.PriorityQueue; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +/** + * This class iterates all segments of the dataSources configured for compaction from the newest to the oldest. + */ +public class NewestSegmentFirstIterator implements CompactionSegmentIterator +{ + private static final Logger log = new Logger(NewestSegmentFirstIterator.class); + + private final Map compactionConfigs; + private final Map> dataSources; + + // dataSource -> intervalToFind + // searchIntervals keeps track of the current state of which interval should be considered to search segments to + // compact. + private final Map searchIntervals; + + // dataSource -> end dateTime of the initial searchInterval + // searchEndDates keeps the endDate of the initial searchInterval (the entire searchInterval). It's immutable and not + // changed once it's initialized. + // This is used to determine that we can expect more segments to be added for an interval in the future. If the end of + // the interval is same with searchEndDate, we can expect more segments to be added and discard the found segments for + // compaction in this run to further optimize the size of compact segments. See checkCompactableSizeForLastSegmentOrReturn(). + private final Map searchEndDates; + private final PriorityQueue queue = new PriorityQueue<>( + (o1, o2) -> Comparators.intervalsByStartThenEnd().compare(o2.interval, o1.interval) + ); + + NewestSegmentFirstIterator( + Map compactionConfigs, + Map> dataSources + ) + { + this.compactionConfigs = compactionConfigs; + this.dataSources = dataSources; + this.searchIntervals = new HashMap<>(dataSources.size()); + this.searchEndDates = new HashMap<>(dataSources.size()); + + for (Entry> entry : dataSources.entrySet()) { + final String dataSource = entry.getKey(); + final VersionedIntervalTimeline timeline = entry.getValue(); + final CoordinatorCompactionConfig config = compactionConfigs.get(dataSource); + + if (config != null && !timeline.isEmpty()) { + final Interval searchInterval = findInitialSearchInterval(timeline, config.getSkipOffsetFromLatest()); + searchIntervals.put(dataSource, searchInterval); + searchEndDates.put(dataSource, searchInterval.getEnd()); + } + } + + for (Entry entry : compactionConfigs.entrySet()) { + final String dataSourceName = entry.getKey(); + final CoordinatorCompactionConfig config = entry.getValue(); + + if (config == null) { + throw new ISE("Unknown dataSource[%s]", dataSourceName); + } + + updateQueue(dataSourceName, config); + } + } + + @Override + public Object2LongOpenHashMap remainingSegments() + { + final Object2LongOpenHashMap resultMap = new Object2LongOpenHashMap<>(); + final Iterator iterator = queue.iterator(); + while (iterator.hasNext()) { + final QueueEntry entry = iterator.next(); + final VersionedIntervalTimeline timeline = dataSources.get(entry.getDataSource()); + final Interval interval = new Interval(timeline.first().getInterval().getStart(), entry.interval.getEnd()); + + final List> holders = timeline.lookup(interval); + + resultMap.put( + entry.getDataSource(), + holders.stream() + .flatMap(holder -> StreamSupport.stream(holder.getObject().spliterator(), false)) + .count() + ); + } + return resultMap; + } + + @Override + public boolean hasNext() + { + return !queue.isEmpty(); + } + + @Override + public List next() + { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + final QueueEntry entry = queue.poll(); + + if (entry == null) { + throw new NoSuchElementException(); + } + + final List resultSegments = entry.segments; + + Preconditions.checkState(!resultSegments.isEmpty(), "Queue entry must not be empty"); + + final String dataSource = resultSegments.get(0).getDataSource(); + updateQueue(dataSource, compactionConfigs.get(dataSource)); + + return resultSegments; + } + + /** + * Find the next segments to compact for the given dataSource and add them to the queue. + * {@link #searchIntervals} is updated according to the found segments. That is, the interval of the found segments + * are removed from the searchInterval of the given dataSource. + */ + private void updateQueue(String dataSourceName, CoordinatorCompactionConfig config) + { + VersionedIntervalTimeline timeline = dataSources.get(dataSourceName); + + if (timeline == null || timeline.isEmpty()) { + log.warn("Cannot find timeline for dataSource[%s]. Continue to the next dataSource", dataSourceName); + return; + } + + final Interval searchInterval = Preconditions.checkNotNull( + searchIntervals.get(dataSourceName), + "Cannot find intervals to find for dataSource[%s]", + dataSourceName + ); + final DateTime searchEnd = Preconditions.checkNotNull( + searchEndDates.get(dataSourceName), + "searchEndDate for dataSource[%s]", + dataSourceName + ); + + final Pair pair = findSegmentsToCompact( + timeline, + searchInterval, + searchEnd, + config + ); + final List segmentsToCompact = pair.rhs.getSegments(); + final Interval remainingSearchInterval = pair.lhs; + + searchIntervals.put(dataSourceName, remainingSearchInterval); + if (!segmentsToCompact.isEmpty()) { + queue.add(new QueueEntry(segmentsToCompact)); + } + } + + /** + * Find segments to compact together for the given intervalToSearch. It progressively searches the given + * intervalToSearch in time order (latest first). The timeline lookup duration is one day. It means, the timeline is + * looked up for the last one day of the given intervalToSearch, and the next day is searched again if the size of + * found segments are not enough to compact. This is repeated until enough amount of segments are found. + * + * @param timeline timeline of a dataSource + * @param intervalToSearch interval to search + * @param searchEnd the end of the whole searchInterval + * @param config compaction config + * + * @return a pair of the reduced interval of (intervalToSearch - interval of found segments) and segments to compact + */ + @VisibleForTesting + static Pair findSegmentsToCompact( + final VersionedIntervalTimeline timeline, + final Interval intervalToSearch, + final DateTime searchEnd, + final CoordinatorCompactionConfig config + ) + { + final long targetCompactionSize = config.getTargetCompactionSizeBytes(); + final int numTargetSegments = config.getNumTargetCompactionSegments(); + final List segmentsToCompact = new ArrayList<>(); + Interval searchInterval = intervalToSearch; + long totalSegmentsToCompactBytes = 0; + + // Finds segments to compact together while iterating searchInterval from latest to oldest + while (!Intervals.isEmpty(searchInterval) + && totalSegmentsToCompactBytes < targetCompactionSize + && segmentsToCompact.size() < numTargetSegments) { + final Interval lookupInterval = SegmentCompactorUtil.getNextLoopupInterval(searchInterval); + // holders are sorted by their interval + final List> holders = timeline.lookup(lookupInterval); + + if (holders.isEmpty()) { + // We found nothing. Continue to the next interval. + searchInterval = SegmentCompactorUtil.removeIntervalFromEnd(searchInterval, lookupInterval); + continue; + } + + for (int i = holders.size() - 1; i >= 0; i--) { + final TimelineObjectHolder holder = holders.get(i); + final List> chunks = Lists.newArrayList(holder.getObject().iterator()); + final long partitionBytes = chunks.stream().mapToLong(chunk -> chunk.getObject().getSize()).sum(); + if (chunks.size() == 0 || partitionBytes == 0) { + log.warn("Skip empty shard[%s]", holder); + continue; + } + + if (!intervalToSearch.contains(chunks.get(0).getObject().getInterval())) { + searchInterval = SegmentCompactorUtil.removeIntervalFromEnd( + searchInterval, + new Interval(chunks.get(0).getObject().getInterval().getStart(), searchInterval.getEnd()) + ); + continue; + } + + // Addition of the segments of a partition should be atomic. + if (SegmentCompactorUtil.isCompactible(targetCompactionSize, totalSegmentsToCompactBytes, partitionBytes) && + segmentsToCompact.size() + chunks.size() <= numTargetSegments) { + chunks.forEach(chunk -> segmentsToCompact.add(chunk.getObject())); + totalSegmentsToCompactBytes += partitionBytes; + } else { + if (segmentsToCompact.size() > 1) { + // We found some segmens to compact and cannot add more. End here. + return checkCompactableSizeForLastSegmentOrReturn( + segmentsToCompact, + totalSegmentsToCompactBytes, + timeline, + searchInterval, + searchEnd, + config + ); + } else { + // (*) Discard segments found so far because we can't compact it anyway. + final int numSegmentsToCompact = segmentsToCompact.size(); + segmentsToCompact.clear(); + + if (!SegmentCompactorUtil.isCompactible(targetCompactionSize, 0, partitionBytes)) { + // TODO: this should be changed to compact many small segments into a few large segments + final DataSegment segment = chunks.get(0).getObject(); + log.warn( + "shardSize[%d] for dataSource[%s] and interval[%s] is larger than targetCompactionSize[%d]." + + " Contitnue to the next shard.", + partitionBytes, + segment.getDataSource(), + segment.getInterval(), + targetCompactionSize + ); + } else if (numTargetSegments < chunks.size()) { + final DataSegment segment = chunks.get(0).getObject(); + log.warn( + "The number of segments[%d] for dataSource[%s] and interval[%s] is larger than " + + "numTargetCompactSegments[%d]. If you see lots of shards are being skipped due to too many " + + "segments, consider increasing 'numTargetCompactionSegments' and " + + "'druid.indexer.runner.maxZnodeBytes'. Contitnue to the next shard.", + chunks.size(), + segment.getDataSource(), + segment.getInterval(), + numTargetSegments + ); + } else { + if (numSegmentsToCompact == 1) { + // We found a segment which is smaller than targetCompactionSize but too large to compact with other + // segments. Skip this one. + // Note that segmentsToCompact is already cleared at (*). + chunks.forEach(chunk -> segmentsToCompact.add(chunk.getObject())); + totalSegmentsToCompactBytes = partitionBytes; + } else { + throw new ISE( + "Cannot compact segments[%s]. shardBytes[%s], numSegments[%s]", + chunks.stream().map(PartitionChunk::getObject).collect(Collectors.toList()), + partitionBytes, + chunks.size() + ); + } + } + } + } + + // Update searchInterval + searchInterval = SegmentCompactorUtil.removeIntervalFromEnd( + searchInterval, + new Interval(chunks.get(0).getObject().getInterval().getStart(), searchInterval.getEnd()) + ); + } + } + + if (segmentsToCompact.size() == 0 || segmentsToCompact.size() == 1) { + if (Intervals.isEmpty(searchInterval)) { + // We found nothing to compact. End here. + return Pair.of(intervalToSearch, new SegmentsToCompact(ImmutableList.of())); + } else { + // We found only 1 segment. Further find segments for the remaining interval. + return findSegmentsToCompact(timeline, searchInterval, searchEnd, config); + } + } + + return checkCompactableSizeForLastSegmentOrReturn( + segmentsToCompact, + totalSegmentsToCompactBytes, + timeline, + searchInterval, + searchEnd, + config + ); + } + + /** + * Check the found segments are enough to compact. If it's expected that more data will be added in the future for the + * interval of found segments, the found segments are skipped and remained to be considered again in the next + * coordinator run. Otherwise, simply returns a pair of the given searchInterval and found segments. + */ + private static Pair checkCompactableSizeForLastSegmentOrReturn( + final List segmentsToCompact, + final long totalSegmentsToCompactBytes, + final VersionedIntervalTimeline timeline, + final Interval searchInterval, + final DateTime searchEnd, + final CoordinatorCompactionConfig config + ) + { + if (segmentsToCompact.size() > 0) { + // Check we have enough segments to compact. For realtime dataSources, we can expect more data to be added in the + // future, so we skip compaction for segments in this run if their size is not sufficiently large. + final DataSegment lastSegment = segmentsToCompact.get(segmentsToCompact.size() - 1); + if (lastSegment.getInterval().getEnd().equals(searchEnd) && + !SegmentCompactorUtil.isProperCompactionSize( + config.getTargetCompactionSizeBytes(), + totalSegmentsToCompactBytes + ) && + config.getNumTargetCompactionSegments() > segmentsToCompact.size()) { + // Ignore found segments and find again for the remaininig searchInterval. + return findSegmentsToCompact(timeline, searchInterval, searchEnd, config); + } + } + + return Pair.of(searchInterval, new SegmentsToCompact(segmentsToCompact)); + } + + /** + * Returns the initial searchInterval which is {@code (timeline.first().start, timeline.last().end - skipOffset)}. + * + * @param timeline timeline of a dataSource + * @param skipOffset skipOFfset + * + * @return found searchInterval + */ + private static Interval findInitialSearchInterval( + VersionedIntervalTimeline timeline, + Period skipOffset + ) + { + Preconditions.checkArgument(timeline != null && !timeline.isEmpty(), "timeline should not be null or empty"); + Preconditions.checkNotNull(skipOffset, "skipOffset"); + + final TimelineObjectHolder first = Preconditions.checkNotNull(timeline.first(), "first"); + final TimelineObjectHolder last = Preconditions.checkNotNull(timeline.last(), "last"); + + final Interval skipInterval = new Interval(skipOffset, last.getInterval().getEnd()); + + final List> holders = timeline.lookup( + new Interval(first.getInterval().getStart(), last.getInterval().getEnd().minus(skipOffset)) + ); + + final List segments = holders + .stream() + .flatMap(holder -> StreamSupport.stream(holder.getObject().spliterator(), false)) + .map(PartitionChunk::getObject) + .filter(segment -> !segment.getInterval().overlaps(skipInterval)) + .sorted((s1, s2) -> Comparators.intervalsByStartThenEnd().compare(s1.getInterval(), s2.getInterval())) + .collect(Collectors.toList()); + + if (segments.isEmpty()) { + return new Interval(first.getInterval().getStart(), first.getInterval().getStart()); + } else { + return new Interval( + segments.get(0).getInterval().getStart(), + segments.get(segments.size() - 1).getInterval().getEnd() + ); + } + } + + private static class QueueEntry + { + private final Interval interval; // whole interval for all segments + private final List segments; + + QueueEntry(List segments) + { + Preconditions.checkArgument(segments != null && !segments.isEmpty()); + Collections.sort(segments); + this.interval = new Interval( + segments.get(0).getInterval().getStart(), + segments.get(segments.size() - 1).getInterval().getEnd() + ); + this.segments = segments; + } + + String getDataSource() + { + return segments.get(0).getDataSource(); + } + } + + private static class SegmentsToCompact + { + private final List segments; + + SegmentsToCompact(List segments) + { + this.segments = segments; + } + + public List getSegments() + { + return segments; + } + } +} diff --git a/server/src/main/java/io/druid/server/coordinator/helper/NewestSegmentFirstPolicy.java b/server/src/main/java/io/druid/server/coordinator/helper/NewestSegmentFirstPolicy.java new file mode 100644 index 000000000000..9f0047404e83 --- /dev/null +++ b/server/src/main/java/io/druid/server/coordinator/helper/NewestSegmentFirstPolicy.java @@ -0,0 +1,41 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.coordinator.helper; + +import io.druid.server.coordinator.CoordinatorCompactionConfig; +import io.druid.timeline.DataSegment; +import io.druid.timeline.VersionedIntervalTimeline; + +import java.util.Map; + +/** + * This policy searches segments for compaction from the newest one to oldest one. + */ +public class NewestSegmentFirstPolicy implements CompactionSegmentSearchPolicy +{ + @Override + public CompactionSegmentIterator reset( + Map compactionConfigs, + Map> dataSources + ) + { + return new NewestSegmentFirstIterator(compactionConfigs, dataSources); + } +} diff --git a/server/src/main/java/io/druid/server/coordinator/helper/SegmentCompactorUtil.java b/server/src/main/java/io/druid/server/coordinator/helper/SegmentCompactorUtil.java new file mode 100644 index 000000000000..54f9d77a34e4 --- /dev/null +++ b/server/src/main/java/io/druid/server/coordinator/helper/SegmentCompactorUtil.java @@ -0,0 +1,78 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.coordinator.helper; + +import com.google.common.base.Preconditions; +import org.joda.time.Duration; +import org.joda.time.Interval; +import org.joda.time.Period; + +/** + * Util class used by {@link DruidCoordinatorSegmentCompactor} and {@link CompactionSegmentSearchPolicy}. + */ +class SegmentCompactorUtil +{ + private static final Period LOOKUP_PERIOD = new Period("P1D"); + private static final Duration LOOKUP_DURATION = LOOKUP_PERIOD.toStandardDuration(); + // Allow compaction of segments if totalSize(segments) <= remainingBytes * ALLOWED_MARGIN_OF_COMPACTION_SIZE + private static final double ALLOWED_MARGIN_OF_COMPACTION_SIZE = .1; + + static boolean isCompactible(long remainingBytes, long currentTotalBytes, long additionalBytes) + { + return remainingBytes * (1 + ALLOWED_MARGIN_OF_COMPACTION_SIZE) >= currentTotalBytes + additionalBytes; + } + + static boolean isProperCompactionSize(long targetCompactionSizeBytes, long totalBytesOfSegmentsToCompact) + { + return targetCompactionSizeBytes * (1 - ALLOWED_MARGIN_OF_COMPACTION_SIZE) <= totalBytesOfSegmentsToCompact && + targetCompactionSizeBytes * (1 + ALLOWED_MARGIN_OF_COMPACTION_SIZE) >= totalBytesOfSegmentsToCompact; + } + + /** + * Return an interval for looking up for timeline. + * If {@code totalInterval} is larger than {@link #LOOKUP_PERIOD}, it returns an interval of {@link #LOOKUP_PERIOD} + * and the end of {@code totalInterval}. + */ + static Interval getNextLoopupInterval(Interval totalInterval) + { + final Duration givenDuration = totalInterval.toDuration(); + return givenDuration.isLongerThan(LOOKUP_DURATION) ? + new Interval(LOOKUP_PERIOD, totalInterval.getEnd()) : + totalInterval; + } + + /** + * Removes {@code smallInterval} from {@code largeInterval}. The end of both intervals should be same. + * + * @return an interval of {@code largeInterval} - {@code smallInterval}. + */ + static Interval removeIntervalFromEnd(Interval largeInterval, Interval smallInterval) + { + Preconditions.checkArgument( + largeInterval.getEnd().equals(smallInterval.getEnd()), + "end should be same. largeInterval[%s] smallInterval[%s]", + largeInterval, + smallInterval + ); + return new Interval(largeInterval.getStart(), smallInterval.getStart()); + } + + private SegmentCompactorUtil() {} +} diff --git a/server/src/main/java/io/druid/server/http/CoordinatorDynamicConfigsResource.java b/server/src/main/java/io/druid/server/http/CoordinatorDynamicConfigsResource.java index 208dcf387be5..ee2d3c3830e6 100644 --- a/server/src/main/java/io/druid/server/http/CoordinatorDynamicConfigsResource.java +++ b/server/src/main/java/io/druid/server/http/CoordinatorDynamicConfigsResource.java @@ -19,12 +19,16 @@ package io.druid.server.http; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.sun.jersey.spi.container.ResourceFilters; import io.druid.audit.AuditInfo; import io.druid.audit.AuditManager; import io.druid.common.config.JacksonConfigManager; +import io.druid.guice.annotations.Json; import io.druid.java.util.common.Intervals; +import io.druid.server.coordinator.CoordinatorCompactionConfig; import io.druid.server.coordinator.CoordinatorDynamicConfig; import io.druid.server.http.security.ConfigResourceFilter; import org.joda.time.Interval; @@ -32,16 +36,21 @@ import javax.inject.Inject; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; import javax.ws.rs.HeaderParam; import javax.ws.rs.POST; import javax.ws.rs.Path; +import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; /** */ @@ -51,15 +60,18 @@ public class CoordinatorDynamicConfigsResource { private final JacksonConfigManager manager; private final AuditManager auditManager; + private final ObjectMapper jsonMapper; @Inject public CoordinatorDynamicConfigsResource( JacksonConfigManager manager, - AuditManager auditManager + AuditManager auditManager, + @Json ObjectMapper jsonMapper ) { this.manager = manager; this.auditManager = auditManager; + this.jsonMapper = jsonMapper; } @GET @@ -98,6 +110,116 @@ public Response setDynamicConfigs(final CoordinatorDynamicConfig.Builder dynamic return Response.ok().build(); } + @POST + @Path("/compaction/{dataSource}") + @Consumes(MediaType.APPLICATION_JSON) + public Response addOrUpdateCompactionConfig( + final CoordinatorCompactionConfig newConfig, + @PathParam("dataSource") String dataSource, + @HeaderParam(AuditManager.X_DRUID_AUTHOR) @DefaultValue("") final String author, + @HeaderParam(AuditManager.X_DRUID_COMMENT) @DefaultValue("") final String comment, + @Context HttpServletRequest req + ) + { + CoordinatorDynamicConfig current = manager.watch( + CoordinatorDynamicConfig.CONFIG_KEY, + CoordinatorDynamicConfig.class + ).get(); + + final CoordinatorDynamicConfig newDynamicConfig; + if (current != null) { + final Map newConfigs = current + .getCompactionConfigs() + .stream() + .collect(Collectors.toMap(CoordinatorCompactionConfig::getDataSource, Function.identity())); + newConfigs.put(dataSource, newConfig); + newDynamicConfig = CoordinatorDynamicConfig.builder() + .withCompactionConfigs(ImmutableList.copyOf(newConfigs.values())) + .build(current); + } else { + newDynamicConfig = CoordinatorDynamicConfig.builder() + .withCompactionConfigs(ImmutableList.of(newConfig)) + .build(); + } + + if (!manager.set( + CoordinatorDynamicConfig.CONFIG_KEY, + newDynamicConfig, + new AuditInfo(author, comment, req.getRemoteAddr()) + )) { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + return Response.ok().build(); + } + + @GET + @Path("/compaction/{dataSource}") + @Produces(MediaType.APPLICATION_JSON) + public Response getCompactionConfig(@PathParam("dataSource") String dataSource) + { + CoordinatorDynamicConfig current = manager.watch( + CoordinatorDynamicConfig.CONFIG_KEY, + CoordinatorDynamicConfig.class + ).get(); + + if (current == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + final Map configs = current + .getCompactionConfigs() + .stream() + .collect(Collectors.toMap(CoordinatorCompactionConfig::getDataSource, Function.identity())); + + final CoordinatorCompactionConfig config = configs.get(dataSource); + if (config == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + return Response.ok().entity(config).build(); + } + + @DELETE + @Path("/compaction/{dataSource}") + @Produces(MediaType.APPLICATION_JSON) + public Response deleteCompactionConfig( + @PathParam("dataSource") String dataSource, + @HeaderParam(AuditManager.X_DRUID_AUTHOR) @DefaultValue("") final String author, + @HeaderParam(AuditManager.X_DRUID_COMMENT) @DefaultValue("") final String comment, + @Context HttpServletRequest req + ) + { + CoordinatorDynamicConfig current = manager.watch( + CoordinatorDynamicConfig.CONFIG_KEY, + CoordinatorDynamicConfig.class + ).get(); + + if (current == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + final Map configs = current + .getCompactionConfigs() + .stream() + .collect(Collectors.toMap(CoordinatorCompactionConfig::getDataSource, Function.identity())); + + final CoordinatorCompactionConfig config = configs.remove(dataSource); + if (config == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + if (!manager.set( + CoordinatorDynamicConfig.CONFIG_KEY, + CoordinatorDynamicConfig.builder() + .withCompactionConfigs(ImmutableList.copyOf(configs.values())) + .build(current), + new AuditInfo(author, comment, req.getRemoteAddr()) + )) { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + return Response.ok().build(); + } + @GET @Path("/history") @Produces(MediaType.APPLICATION_JSON) diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java index b49445905f47..2dc5c89de565 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java @@ -154,7 +154,7 @@ public void bigProfiler() ) .withAvailableSegments(segmentMap.values()) .withDynamicConfigs( - new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove( + CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( MAX_SEGMENTS_TO_MOVE ).withReplicantLifetime(500) .withReplicationThrottleLimit(5) @@ -252,7 +252,7 @@ public void profileRun() ) .withAvailableSegments(segments.values()) .withDynamicConfigs( - new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove( + CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( MAX_SEGMENTS_TO_MOVE ).build() ) diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTest.java index ec0b841fbcd2..b70c2ca0a028 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorBalancerTest.java @@ -222,7 +222,7 @@ public void testMoveSameSegmentTwice() throws Exception ) .withBalancerStrategy(predefinedPickOrderStrategy) .withDynamicConfigs( - new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove( + CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( 2 ).build() ) @@ -304,7 +304,7 @@ private DruidCoordinatorRuntimeParams.Builder defaullRuntimeParamsBuilder( ) .withAvailableSegments(segments.values()) .withDynamicConfigs( - new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove( + CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( MAX_SEGMENTS_TO_MOVE ).build() ) diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java index 7f2515415099..cc9bc647b89e 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java @@ -204,7 +204,7 @@ public void testRunThreeTiersOneReplicant() throws Exception .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withBalancerStrategy(balancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove(5).build()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -608,7 +608,7 @@ public void testDropRemove() throws Exception DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -694,7 +694,7 @@ public void testDropTooManyInSameTier() throws Exception DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -785,7 +785,7 @@ public void testDropTooManyInDifferentTiers() throws Exception DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -872,7 +872,7 @@ public void testDontDropInDifferentTiers() throws Exception DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -974,7 +974,7 @@ public void testDropServerActuallyServesSegment() throws Exception DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(availableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -1289,7 +1289,7 @@ public void testDropReplicantThrottle() throws Exception DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) .withAvailableSegments(longerAvailableSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -1380,7 +1380,7 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() throws Exception .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withBalancerStrategy(balancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMaxSegmentsToMove(5).build()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java index 686d7399d57e..69731a99235e 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorSegmentMergerTest.java @@ -471,7 +471,7 @@ public void mergeSegments(List segmentsToMerge) final DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams.newBuilder() .withAvailableSegments(ImmutableSet.copyOf(segments)) - .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMergeBytesLimit( + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMergeBytesLimit( mergeBytesLimit).withMergeSegmentsLimit(mergeSegmentsLimit).build()) .withEmitter(EasyMock.createMock(ServiceEmitter.class)) .build(); diff --git a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java index d2f871c4e779..99a19104e90d 100644 --- a/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/io/druid/server/coordinator/DruidCoordinatorTest.java @@ -114,7 +114,7 @@ public void setUp() throws Exception EasyMock.anyObject(Class.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference(new CoordinatorDynamicConfig.Builder().build())).anyTimes(); + ).andReturn(new AtomicReference(CoordinatorDynamicConfig.builder().build())).anyTimes(); EasyMock.replay(configManager); setupServerAndCurator(); curator.start(); diff --git a/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java b/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java new file mode 100644 index 000000000000..fac0a01377cd --- /dev/null +++ b/server/src/test/java/io/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java @@ -0,0 +1,364 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.coordinator.helper; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import io.druid.client.indexing.ClientCompactQueryTuningConfig; +import io.druid.client.indexing.IndexingServiceClient; +import io.druid.indexer.TaskStatusPlus; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.StringUtils; +import io.druid.server.coordinator.CoordinatorCompactionConfig; +import io.druid.server.coordinator.CoordinatorDynamicConfig; +import io.druid.server.coordinator.CoordinatorStats; +import io.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import io.druid.timeline.DataSegment; +import io.druid.timeline.TimelineObjectHolder; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.NoneShardSpec; +import io.druid.timeline.partition.NumberedShardSpec; +import io.druid.timeline.partition.PartitionChunk; +import io.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; + +public class DruidCoordinatorSegmentCompactorTest +{ + private static final String DATA_SOURCE_PREFIX = "dataSource_"; + + private final IndexingServiceClient indexingServiceClient = new IndexingServiceClient(null, null) + { + private int compactVersionSuffix = 0; + private int idSuffix = 0; + + @Override + public String compactSegments( + List segments, + int compactionTaskPriority, + ClientCompactQueryTuningConfig tuningConfig, + Map context + ) + { + Preconditions.checkArgument(segments.size() > 1); + Collections.sort(segments); + Interval compactInterval = new Interval( + segments.get(0).getInterval().getStart(), + segments.get(segments.size() - 1).getInterval().getEnd() + ); + DataSegment compactSegment = new DataSegment( + segments.get(0).getDataSource(), + compactInterval, + "newVersion_" + compactVersionSuffix++, + null, + segments.get(0).getDimensions(), + segments.get(0).getMetrics(), + NoneShardSpec.instance(), + 1, + segments.stream().mapToLong(DataSegment::getSize).sum() + ); + + final VersionedIntervalTimeline timeline = dataSources.get(segments.get(0).getDataSource()); + segments.forEach( + segment -> timeline.remove( + segment.getInterval(), + segment.getVersion(), + segment.getShardSpec().createChunk(segment) + ) + ); + timeline.add( + compactInterval, + compactSegment.getVersion(), + compactSegment.getShardSpec().createChunk(compactSegment) + ); + return "task_" + idSuffix++; + } + + @Override + public List getRunningTasks() + { + return ImmutableList.of(); + } + + @Override + public int getTotalWorkerCapacity() + { + return 10; + } + }; + + private List compactionConfigs; + private Map> dataSources; + + @Before + public void setup() + { + compactionConfigs = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + final String dataSource = DATA_SOURCE_PREFIX + i; + compactionConfigs.add( + new CoordinatorCompactionConfig( + dataSource, + 0, + 50L, + null, + new Period("PT1H"), // smaller than segment interval + null, + null + ) + ); + } + + dataSources = new HashMap<>(); + for (int i = 0; i < 3; i++) { + final String dataSource = DATA_SOURCE_PREFIX + i; + + VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( + String.CASE_INSENSITIVE_ORDER + ); + + for (int j = 0; j < 4; j++) { + for (int k = 0; k < 2; k++) { + DataSegment segment = createSegment(dataSource, j, true, k); + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + segment = createSegment(dataSource, j, false, k); + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + } + } + + for (int j = 7; j < 9; j++) { + for (int k = 0; k < 2; k++) { + DataSegment segment = createSegment(dataSource, j, true, k); + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + segment = createSegment(dataSource, j, false, k); + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + } + } + + dataSources.put(dataSource, timeline); + } + } + + private static DataSegment createSegment(String dataSource, int startDay, boolean beforeNoon, int partition) + { + final ShardSpec shardSpec = new NumberedShardSpec(partition, 2); + final Interval interval = beforeNoon ? + Intervals.of( + StringUtils.format( + "2017-01-%02dT00:00:00/2017-01-%02dT12:00:00", + startDay + 1, + startDay + 1 + ) + ) : + Intervals.of( + StringUtils.format( + "2017-01-%02dT12:00:00/2017-01-%02dT00:00:00", + startDay + 1, + startDay + 2 + ) + ); + return new DataSegment( + dataSource, + interval, + "version", + null, + ImmutableList.of(), + ImmutableList.of(), + shardSpec, + 0, + 10L + ); + } + + @Test + public void testRun() + { + final DruidCoordinatorSegmentCompactor compactor = new DruidCoordinatorSegmentCompactor(indexingServiceClient); + + final Supplier expectedVersionSupplier = new Supplier() + { + private int i = 0; + + @Override + public String get() + { + return "newVersion_" + i++; + } + }; + int expectedCompactTaskCount = 1; + int expectedRemainingSegments = 18; + + // compact for 2017-01-08T12:00:00.000Z/2017-01-09T12:00:00.000Z + assertCompactSegments( + compactor, + Intervals.of(StringUtils.format("2017-01-%02dT12:00:00/2017-01-%02dT12:00:00", 8, 9)), + expectedRemainingSegments, + expectedCompactTaskCount, + expectedVersionSupplier + ); + + // compact for 2017-01-07T12:00:00.000Z/2017-01-08T12:00:00.000Z + expectedRemainingSegments -= 4; + assertCompactSegments( + compactor, + Intervals.of(StringUtils.format("2017-01-%02dT12:00:00/2017-01-%02dT12:00:00", 4, 8)), + expectedRemainingSegments, + expectedCompactTaskCount, + expectedVersionSupplier + ); + + for (int endDay = 4; endDay > 1; endDay -= 1) { + expectedRemainingSegments -= 4; + assertCompactSegments( + compactor, + Intervals.of(StringUtils.format("2017-01-%02dT12:00:00/2017-01-%02dT12:00:00", endDay - 1, endDay)), + expectedRemainingSegments, + expectedCompactTaskCount, + expectedVersionSupplier + ); + } + + // Segments of the latest interval should not be compacted + for (int i = 0; i < 3; i++) { + final String dataSource = DATA_SOURCE_PREFIX + i; + final Interval interval = Intervals.of(StringUtils.format("2017-01-09T12:00:00/2017-01-10")); + List> holders = dataSources.get(dataSource).lookup(interval); + Assert.assertEquals(1, holders.size()); + for (TimelineObjectHolder holder : holders) { + List> chunks = Lists.newArrayList(holder.getObject()); + Assert.assertEquals(2, chunks.size()); + for (PartitionChunk chunk : chunks) { + DataSegment segment = chunk.getObject(); + Assert.assertEquals(interval, segment.getInterval()); + Assert.assertEquals("version", segment.getVersion()); + } + } + } + + // Emulating realtime dataSource + final String dataSource = DATA_SOURCE_PREFIX + 0; + addMoreData(dataSource, 9); + + CoordinatorStats stats = runCompactor(compactor); + Assert.assertEquals( + 1, + stats.getGlobalStat(DruidCoordinatorSegmentCompactor.COMPACT_TASK_COUNT) + ); + + addMoreData(dataSource, 10); + + stats = runCompactor(compactor); + Assert.assertEquals( + 1, + stats.getGlobalStat(DruidCoordinatorSegmentCompactor.COMPACT_TASK_COUNT) + ); + } + + private CoordinatorStats runCompactor(DruidCoordinatorSegmentCompactor compactor) + { + DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + .newBuilder() + .withDataSources(dataSources) + .withDynamicConfigs( + CoordinatorDynamicConfig.builder().withCompactionConfigs(compactionConfigs).build() + ) + .build(); + return compactor.run(params).getCoordinatorStats(); + } + + private void assertCompactSegments( + DruidCoordinatorSegmentCompactor compactor, + Interval expectedInterval, + int expectedRemainingSegments, + int expectedCompactTaskCount, + Supplier expectedVersionSupplier + ) + { + for (int i = 0; i < 3; i++) { + final CoordinatorStats stats = runCompactor(compactor); + Assert.assertEquals( + expectedCompactTaskCount, + stats.getGlobalStat(DruidCoordinatorSegmentCompactor.COMPACT_TASK_COUNT) + ); + + // One of dataSource is compacted + if (expectedRemainingSegments > 0) { + // If expectedRemainingSegments is positive, we check how many dataSources have the segments waiting + // compaction. + long numDataSourceOfExpectedRemainingSegments = stats + .getDataSources(DruidCoordinatorSegmentCompactor.SEGMENTS_WAIT_COMPACT) + .stream() + .mapToLong(dataSource -> stats.getDataSourceStat( + DruidCoordinatorSegmentCompactor.SEGMENTS_WAIT_COMPACT, + dataSource) + ) + .filter(stat -> stat == expectedRemainingSegments) + .count(); + Assert.assertEquals(i + 1, numDataSourceOfExpectedRemainingSegments); + } else { + // Otherwise, we check how many dataSources are in the coordinator stats. + Assert.assertEquals( + 2 - i, + stats.getDataSources(DruidCoordinatorSegmentCompactor.SEGMENTS_WAIT_COMPACT).size() + ); + } + } + + for (int i = 0; i < 3; i++) { + final String dataSource = DATA_SOURCE_PREFIX + i; + List> holders = dataSources.get(dataSource).lookup(expectedInterval); + Assert.assertEquals(1, holders.size()); + List> chunks = Lists.newArrayList(holders.get(0).getObject()); + Assert.assertEquals(1, chunks.size()); + DataSegment segment = chunks.get(0).getObject(); + Assert.assertEquals(expectedInterval, segment.getInterval()); + Assert.assertEquals(expectedVersionSupplier.get(), segment.getVersion()); + } + } + + private void addMoreData(String dataSource, int day) + { + for (int i = 0; i < 2; i++) { + DataSegment newSegment = createSegment(dataSource, day, true, i); + dataSources.get(dataSource).add( + newSegment.getInterval(), + newSegment.getVersion(), + newSegment.getShardSpec().createChunk(newSegment) + ); + newSegment = createSegment(dataSource, day, false, i); + dataSources.get(dataSource).add( + newSegment.getInterval(), + newSegment.getVersion(), + newSegment.getShardSpec().createChunk(newSegment) + ); + } + } +} diff --git a/server/src/test/java/io/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java b/server/src/test/java/io/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java new file mode 100644 index 000000000000..300b0df7c33f --- /dev/null +++ b/server/src/test/java/io/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java @@ -0,0 +1,488 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.server.coordinator.helper; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.guava.Comparators; +import io.druid.server.coordinator.CoordinatorCompactionConfig; +import io.druid.timeline.DataSegment; +import io.druid.timeline.VersionedIntervalTimeline; +import io.druid.timeline.partition.NumberedShardSpec; +import io.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class NewestSegmentFirstPolicyTest +{ + private static final String DATA_SOURCE = "dataSource"; + private static final long DEFAULT_SEGMENT_SIZE = 1000; + private static final int DEFAULT_NUM_SEGMENTS_PER_SHARD = 4; + + private final NewestSegmentFirstPolicy policy = new NewestSegmentFirstPolicy(); + + @Test + public void testLargeOffsetAndSmallSegmentInterval() + { + final Period segmentPeriod = new Period("PT1H"); + final CompactionSegmentIterator iterator = policy.reset( + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, 100, new Period("P2D"))), + ImmutableMap.of( + DATA_SOURCE, + createTimeline( + new SegmentGenerateSpec(Intervals.of("2017-11-16T20:00:00/2017-11-17T04:00:00"), segmentPeriod), + new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-16T07:00:00"), segmentPeriod) + ) + ) + ); + + assertCompactSegmentIntervals( + iterator, + segmentPeriod, + Intervals.of("2017-11-14T00:00:00/2017-11-14T01:00:00"), + Intervals.of("2017-11-15T03:00:00/2017-11-15T04:00:00"), + true + ); + } + + @Test + public void testSmallOffsetAndLargeSegmentInterval() + { + final Period segmentPeriod = new Period("PT1H"); + final CompactionSegmentIterator iterator = policy.reset( + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, 100, new Period("PT1M"))), + ImmutableMap.of( + DATA_SOURCE, + createTimeline( + new SegmentGenerateSpec(Intervals.of("2017-11-16T20:00:00/2017-11-17T04:00:00"), segmentPeriod), + new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-16T07:00:00"), segmentPeriod) + ) + ) + ); + + assertCompactSegmentIntervals( + iterator, + segmentPeriod, + Intervals.of("2017-11-16T21:00:00/2017-11-16T22:00:00"), + Intervals.of("2017-11-17T02:00:00/2017-11-17T03:00:00"), + false + ); + + final List segments = iterator.next(); + Assert.assertNotNull(segments); + Assert.assertEquals(8, segments.size()); + + final List expectedIntervals = new ArrayList<>(segments.size()); + for (int i = 0; i < 4; i++) { + expectedIntervals.add(Intervals.of("2017-11-16T06:00:00/2017-11-16T07:00:00")); + } + for (int i = 0; i < 4; i++) { + expectedIntervals.add(Intervals.of("2017-11-16T20:00:00/2017-11-16T21:00:00")); + } + expectedIntervals.sort(Comparators.intervalsByStartThenEnd()); + + Assert.assertEquals( + expectedIntervals, + segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) + ); + + assertCompactSegmentIntervals( + iterator, + segmentPeriod, + Intervals.of("2017-11-14T00:00:00/2017-11-14T01:00:00"), + Intervals.of("2017-11-16T05:00:00/2017-11-16T06:00:00"), + true + ); + } + + @Test + public void testLargeGapInData() + { + final Period segmentPeriod = new Period("PT1H"); + final CompactionSegmentIterator iterator = policy.reset( + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, 100, new Period("PT1H1M"))), + ImmutableMap.of( + DATA_SOURCE, + createTimeline( + new SegmentGenerateSpec(Intervals.of("2017-11-16T20:00:00/2017-11-17T04:00:00"), segmentPeriod), + // larger gap than SegmentCompactorUtil.LOOKUP_PERIOD (1 day) + new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-15T07:00:00"), segmentPeriod) + ) + ) + ); + + assertCompactSegmentIntervals( + iterator, + segmentPeriod, + Intervals.of("2017-11-16T20:00:00/2017-11-16T21:00:00"), + Intervals.of("2017-11-17T01:00:00/2017-11-17T02:00:00"), + false + ); + + assertCompactSegmentIntervals( + iterator, + segmentPeriod, + Intervals.of("2017-11-14T00:00:00/2017-11-14T01:00:00"), + Intervals.of("2017-11-15T06:00:00/2017-11-15T07:00:00"), + true + ); + } + + @Test + public void testSmallNumTargetCompactionSegments() + { + final Period segmentPeriod = new Period("PT1H"); + final CompactionSegmentIterator iterator = policy.reset( + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, 5, new Period("PT1H1M"))), + ImmutableMap.of( + DATA_SOURCE, + createTimeline( + new SegmentGenerateSpec(Intervals.of("2017-11-16T20:00:00/2017-11-17T04:00:00"), segmentPeriod), + // larger gap than SegmentCompactorUtil.LOOKUP_PERIOD (1 day) + new SegmentGenerateSpec(Intervals.of("2017-11-14T00:00:00/2017-11-15T07:00:00"), segmentPeriod) + ) + ) + ); + + assertCompactSegmentIntervals( + iterator, + segmentPeriod, + Intervals.of("2017-11-16T20:00:00/2017-11-16T21:00:00"), + // The last interval is not "2017-11-17T01:00:00/2017-11-17T02:00:00". This is because more segments are + // expected to be added for that interval. See NewestSegmentFirstIterator.returnIfCompactibleSize(). + Intervals.of("2017-11-17T00:00:00/2017-11-17T01:00:00"), + false + ); + + assertCompactSegmentIntervals( + iterator, + segmentPeriod, + Intervals.of("2017-11-14T00:00:00/2017-11-14T01:00:00"), + Intervals.of("2017-11-15T06:00:00/2017-11-15T07:00:00"), + true + ); + } + + @Test + public void testHugeShard() + { + final CompactionSegmentIterator iterator = policy.reset( + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(10000, 100, new Period("P1D"))), + ImmutableMap.of( + DATA_SOURCE, + createTimeline( + new SegmentGenerateSpec( + Intervals.of("2017-11-17T00:00:00/2017-11-18T03:00:00"), + new Period("PT1H"), + 200, + DEFAULT_NUM_SEGMENTS_PER_SHARD + ), + new SegmentGenerateSpec( + Intervals.of("2017-11-09T00:00:00/2017-11-17T00:00:00"), + new Period("P2D"), + 13000, // larger than target compact segment size + 1 + ), + new SegmentGenerateSpec( + Intervals.of("2017-11-05T00:00:00/2017-11-09T00:00:00"), + new Period("PT1H"), + 200, + DEFAULT_NUM_SEGMENTS_PER_SHARD + ) + ) + ) + ); + + Interval lastInterval = null; + while (iterator.hasNext()) { + final List segments = iterator.next(); + lastInterval = segments.get(0).getInterval(); + + Interval prevInterval = null; + for (DataSegment segment : segments) { + if (prevInterval != null && !prevInterval.getStart().equals(segment.getInterval().getStart())) { + Assert.assertEquals(prevInterval.getEnd(), segment.getInterval().getStart()); + } + + prevInterval = segment.getInterval(); + } + } + + Assert.assertNotNull(lastInterval); + Assert.assertEquals(Intervals.of("2017-11-05T00:00:00/2017-11-05T01:00:00"), lastInterval); + } + + @Test + public void testManySegmentsPerShard() + { + final CompactionSegmentIterator iterator = policy.reset( + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(800000, 100, new Period("P1D"))), + ImmutableMap.of( + DATA_SOURCE, + createTimeline( + new SegmentGenerateSpec( + Intervals.of("2017-12-04T01:00:00/2017-12-05T03:00:00"), + new Period("PT1H"), + 375, + 80 + ), + new SegmentGenerateSpec( + Intervals.of("2017-12-04T00:00:00/2017-12-04T01:00:00"), + new Period("PT1H"), + 200, + 150 + ), + new SegmentGenerateSpec( + Intervals.of("2017-12-03T18:00:00/2017-12-04T00:00:00"), + new Period("PT6H"), + 200000, + 1 + ), + new SegmentGenerateSpec( + Intervals.of("2017-12-03T11:00:00/2017-12-03T18:00:00"), + new Period("PT1H"), + 375, + 80 + ) + ) + ) + ); + + Interval lastInterval = null; + while (iterator.hasNext()) { + final List segments = iterator.next(); + lastInterval = segments.get(0).getInterval(); + + Interval prevInterval = null; + for (DataSegment segment : segments) { + if (prevInterval != null && !prevInterval.getStart().equals(segment.getInterval().getStart())) { + Assert.assertEquals(prevInterval.getEnd(), segment.getInterval().getStart()); + } + + prevInterval = segment.getInterval(); + } + } + + Assert.assertNotNull(lastInterval); + Assert.assertEquals(Intervals.of("2017-12-03T11:00:00/2017-12-03T12:00:00"), lastInterval); + } + + @Test + public void testManySegmentsPerShard2() + { + final CompactionSegmentIterator iterator = policy.reset( + ImmutableMap.of(DATA_SOURCE, createCompactionConfig(800000, 100, new Period("P1D"))), + ImmutableMap.of( + DATA_SOURCE, + createTimeline( + new SegmentGenerateSpec( + Intervals.of("2017-12-04T11:00:00/2017-12-05T05:00:00"), + new Period("PT1H"), + 200, + 150 + ), + new SegmentGenerateSpec( + Intervals.of("2017-12-04T06:00:00/2017-12-04T11:00:00"), + new Period("PT1H"), + 375, + 80 + ), + new SegmentGenerateSpec( + Intervals.of("2017-12-03T18:00:00/2017-12-04T06:00:00"), + new Period("PT12H"), + 257000, + 1 + ), + new SegmentGenerateSpec( + Intervals.of("2017-12-03T11:00:00/2017-12-03T18:00:00"), + new Period("PT1H"), + 200, + 150 + ), + new SegmentGenerateSpec( + Intervals.of("2017-12-02T19:00:00/2017-12-03T11:00:00"), + new Period("PT16H"), + 257000, + 1 + ), + new SegmentGenerateSpec( + Intervals.of("2017-12-02T11:00:00/2017-12-02T19:00:00"), + new Period("PT1H"), + 200, + 150 + ), + new SegmentGenerateSpec( + Intervals.of("2017-12-01T18:00:00/2017-12-02T11:00:00"), + new Period("PT17H"), + 257000, + 1 + ), + new SegmentGenerateSpec( + Intervals.of("2017-12-01T09:00:00/2017-12-01T18:00:00"), + new Period("PT1H"), + 200, + 150 + ) + ) + ) + ); + + Assert.assertFalse(iterator.hasNext()); + } + + private static void assertCompactSegmentIntervals( + CompactionSegmentIterator iterator, + Period segmentPeriod, + Interval from, + Interval to, + boolean assertLast + ) + { + Interval expectedSegmentIntervalStart = to; + while (iterator.hasNext()) { + final List segments = iterator.next(); + + final List expectedIntervals = new ArrayList<>(segments.size()); + for (int i = 0; i < segments.size(); i++) { + if (i > 0 && i % DEFAULT_NUM_SEGMENTS_PER_SHARD == 0) { + expectedSegmentIntervalStart = new Interval(segmentPeriod, expectedSegmentIntervalStart.getStart()); + } + expectedIntervals.add(expectedSegmentIntervalStart); + } + expectedIntervals.sort(Comparators.intervalsByStartThenEnd()); + + Assert.assertEquals( + expectedIntervals, + segments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) + ); + + if (expectedSegmentIntervalStart.equals(from)) { + break; + } + expectedSegmentIntervalStart = new Interval(segmentPeriod, expectedSegmentIntervalStart.getStart()); + } + + if (assertLast) { + Assert.assertFalse(iterator.hasNext()); + } + } + + private static VersionedIntervalTimeline createTimeline( + SegmentGenerateSpec... specs + ) + { + VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( + String.CASE_INSENSITIVE_ORDER + ); + + final String version = DateTimes.nowUtc().toString(); + + final List orderedSpecs = Lists.newArrayList(specs); + orderedSpecs.sort((s1, s2) -> Comparators.intervalsByStartThenEnd().compare(s1.totalInterval, s2.totalInterval)); + Collections.reverse(orderedSpecs); + + for (SegmentGenerateSpec spec: orderedSpecs) { + Interval remaininInterval = spec.totalInterval; + + while (!Intervals.isEmpty(remaininInterval)) { + final Interval segmentInterval; + if (remaininInterval.toDuration().isLongerThan(spec.segmentPeriod.toStandardDuration())) { + segmentInterval = new Interval(spec.segmentPeriod, remaininInterval.getEnd()); + } else { + segmentInterval = remaininInterval; + } + + for (int i = 0; i < spec.numSegmentsPerShard; i++) { + final ShardSpec shardSpec = new NumberedShardSpec(spec.numSegmentsPerShard, i); + final DataSegment segment = new DataSegment( + DATA_SOURCE, + segmentInterval, + version, + null, + ImmutableList.of(), + ImmutableList.of(), + shardSpec, + 0, + spec.segmentSize + ); + timeline.add( + segmentInterval, + version, + shardSpec.createChunk(segment) + ); + } + + remaininInterval = SegmentCompactorUtil.removeIntervalFromEnd(remaininInterval, segmentInterval); + } + } + + return timeline; + } + + private static CoordinatorCompactionConfig createCompactionConfig( + long targetCompactionSizeBytes, + int numTargetCompactionSegments, + Period skipOffsetFromLatest + ) + { + return new CoordinatorCompactionConfig( + DATA_SOURCE, + 0, + targetCompactionSizeBytes, + numTargetCompactionSegments, + skipOffsetFromLatest, + null, + null + ); + } + + private static class SegmentGenerateSpec + { + private final Interval totalInterval; + private final Period segmentPeriod; + private final long segmentSize; + private final int numSegmentsPerShard; + + SegmentGenerateSpec(Interval totalInterval, Period segmentPeriod) + { + this(totalInterval, segmentPeriod, DEFAULT_SEGMENT_SIZE, DEFAULT_NUM_SEGMENTS_PER_SHARD); + } + + SegmentGenerateSpec(Interval totalInterval, Period segmentPeriod, long segmentSize, int numSegmentsPerShard) + { + Preconditions.checkArgument(numSegmentsPerShard >= 1); + this.totalInterval = totalInterval; + this.segmentPeriod = segmentPeriod; + this.segmentSize = segmentSize; + this.numSegmentsPerShard = numSegmentsPerShard; + } + } +} diff --git a/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java index 76174172d9fe..df8fefa0c5e8 100644 --- a/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/io/druid/server/coordinator/rules/LoadRuleTest.java @@ -280,7 +280,7 @@ public void testLoadPriority() throws Exception } @Test - public void testDrop() throws Exception + public void testDrop() { final LoadQueuePeon mockPeon = createEmptyPeon(); mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); @@ -543,7 +543,7 @@ public void testMaxLoadingQueueSize() throws Exception .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .withAvailableSegments(Arrays.asList(dataSegment1, dataSegment2, dataSegment3)) - .withDynamicConfigs(new CoordinatorDynamicConfig.Builder().withMaxSegmentsInNodeLoadingQueue(2).build()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsInNodeLoadingQueue(2).build()) .build(); CoordinatorStats stats1 = rule.run(null, params, dataSegment1); diff --git a/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java index 094e42630815..e5a60f3ba727 100644 --- a/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/io/druid/server/http/CoordinatorDynamicConfigTest.java @@ -76,7 +76,7 @@ public void testSerdeWithStringinKillDataSourceWhitelist() throws Exception + " \"replicationThrottleLimit\": 1,\n" + " \"balancerComputeThreads\": 2, \n" + " \"emitBalancingStats\": true,\n" - + " \"killDataSourceWhitelist\": \" test1 ,test2 \", \n" + + " \"killDataSourceWhitelist\": \"test1, test2\", \n" + " \"maxSegmentsInNodeLoadingQueue\": 1\n" + "}\n"; @@ -183,7 +183,8 @@ public void testUpdate() Assert.assertEquals( current, - new CoordinatorDynamicConfig.Builder(null, null, null, null, null, null, null, null, null, null, null, null) + new CoordinatorDynamicConfig + .Builder(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null) .build(current) ); }