mirror of
https://github.com/apache/druid.git
synced 2025-02-23 03:03:02 +00:00
Add feature to automatically remove supervisor based on retention period (#11200)
* add auto clean up * add test * add test * fix test * Address comments * Address comments
This commit is contained in:
parent
351059ca43
commit
d73f72e508
@ -747,12 +747,15 @@ These Coordinator static configurations can be defined in the `coordinator/runti
|
||||
|Property|Description|Required?|Default|
|
||||
|--------|-----------|---------|-------|
|
||||
|`druid.coordinator.period.metadataStoreManagementPeriod`|How often to run metadata management tasks in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. |No | `PT1H`|
|
||||
|`druid.coordinator.kill.supervisor.on`| Boolean value for whether to enable automatic deletion of terminated supervisors. If set to true, Coordinator will periodically remove terminated supervisors from the supervisor table in metadata storage.| No | False|
|
||||
|`druid.coordinator.kill.supervisor.period`| How often to do automatic deletion of terminated supervisor in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Value must be equal to or greater than `druid.coordinator.period.metadataStoreManagementPeriod`. Only applies if `druid.coordinator.kill.supervisor.on` is set to "True".| No| `P1D`|
|
||||
|`druid.coordinator.kill.supervisor.durationToRetain`| Duration of terminated supervisor to be retained from created time in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Only applies if `druid.coordinator.kill.supervisor.on` is set to "True".| Yes if `druid.coordinator.kill.supervisor.on` is set to "True".| None|
|
||||
|`druid.coordinator.kill.audit.on`| Boolean value for whether to enable automatic deletion of audit logs. If set to true, Coordinator will periodically remove audit logs from the audit table entries in metadata storage.| No | False|
|
||||
|`druid.coordinator.kill.audit.period`| How often to do automatic deletion of audit logs in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Value must be greater than `druid.coordinator.period.metadataStoreManagementPeriod`. Only applies if `druid.coordinator.kill.audit.on` is set to True.| No| `P1D`|
|
||||
|`druid.coordinator.kill.audit.durationToRetain`| Duration of audit logs to be retained from created time in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Only applies if `druid.coordinator.kill.audit.on` is set to True.| Yes if `druid.coordinator.kill.audit.on` is set to True| None|
|
||||
|`druid.coordinator.kill.audit.period`| How often to do automatic deletion of audit logs in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Value must be equal to or greater than `druid.coordinator.period.metadataStoreManagementPeriod`. Only applies if `druid.coordinator.kill.audit.on` is set to "True".| No| `P1D`|
|
||||
|`druid.coordinator.kill.audit.durationToRetain`| Duration of audit logs to be retained from created time in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Only applies if `druid.coordinator.kill.audit.on` is set to "True".| Yes if `druid.coordinator.kill.audit.on` is set to "True".| None|
|
||||
|`druid.coordinator.kill.rule.on`| Boolean value for whether to enable automatic deletion of rules. If set to true, Coordinator will periodically remove rules of inactive datasource (datasource with no used and unused segments) from the rule table in metadata storage.| No | False|
|
||||
|`druid.coordinator.kill.rule.period`| How often to do automatic deletion of rules in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Value must be greater than `druid.coordinator.period.metadataStoreManagementPeriod`. Only applies if `druid.coordinator.kill.rule.on` is set to True.| No| `P1D`|
|
||||
|`druid.coordinator.kill.rule.durationToRetain`| Duration of rules to be retained from created time in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Only applies if `druid.coordinator.kill.rule.on` is set to True.| Yes if `druid.coordinator.kill.rule.on` is set to True| None|
|
||||
|`druid.coordinator.kill.rule.period`| How often to do automatic deletion of rules in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Value must be equal to or greater than `druid.coordinator.period.metadataStoreManagementPeriod`. Only applies if `druid.coordinator.kill.rule.on` is set to "True".| No| `P1D`|
|
||||
|`druid.coordinator.kill.rule.durationToRetain`| Duration of rules to be retained from created time in [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601) duration format. Only applies if `druid.coordinator.kill.rule.on` is set to "True".| Yes if `druid.coordinator.kill.rule.on` is set to "True".| None|
|
||||
|
||||
##### Segment Management
|
||||
|Property|Possible Values|Description|Default|
|
||||
|
@ -256,8 +256,9 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina
|
||||
|`interval/skipCompact/count`|Total number of intervals of this datasource that are skipped (not eligible for auto compaction) by the auto compaction.|datasource.|Varies.|
|
||||
|`coordinator/time`|Approximate Coordinator duty runtime in milliseconds. The duty dimension is the string alias of the Duty that is being run.|duty.|Varies.|
|
||||
|`coordinator/global/time`|Approximate runtime of a full coordination cycle in milliseconds. The `dutyGroup` dimension indicates what type of coordination this run was. i.e. Historical Management vs Indexing|`dutyGroup`|Varies.|
|
||||
|`metadata/kill/audit/count`|Total number of audit logs automatically deleted from metadata store audit table per each Coordinator kill audit duty run. This metric can help adjust `druid.coordinator.kill.audit.durationToRetain` configuration based on if more or less audit logs need to be deleted per cycle. Note that this metric is only emitted when `druid.coordinator.kill.audit.on` is set to true.| |Varies.|
|
||||
|`metadata/kill/rule/count`|Total number of rules automatically deleted from metadata store rule table per each Coordinator kill rule duty run. This metric can help adjust `druid.coordinator.kill.rule.durationToRetain` configuration based on if more or less rules need to be deleted per cycle. Note that this metric is only emitted when `druid.coordinator.kill.rule.on` is set to true.| |Varies.|
|
||||
|`metadata/kill/supervisor/count`|Total number of terminated supervisors that were automatically deleted from metadata store per each Coordinator kill supervisor duty run. This metric can help adjust `druid.coordinator.kill.supervisor.durationToRetain` configuration based on whether more or less terminated supervisors need to be deleted per cycle. Note that this metric is only emitted when `druid.coordinator.kill.supervisor.on` is set to true.| |Varies.|
|
||||
|`metadata/kill/audit/count`|Total number of audit logs that were automatically deleted from metadata store per each Coordinator kill audit duty run. This metric can help adjust `druid.coordinator.kill.audit.durationToRetain` configuration based on whether more or less audit logs need to be deleted per cycle. Note that this metric is only emitted when `druid.coordinator.kill.audit.on` is set to true.| |Varies.|
|
||||
|`metadata/kill/rule/count`|Total number of rules that were automatically deleted from metadata store per each Coordinator kill rule duty run. This metric can help adjust `druid.coordinator.kill.rule.durationToRetain` configuration based on whether more or less rules need to be deleted per cycle. Note that this metric is only emitted when `druid.coordinator.kill.rule.on` is set to true.| |Varies.|
|
||||
|
||||
|
||||
If `emitBalancingStats` is set to `true` in the Coordinator [dynamic configuration](../configuration/index.md#dynamic-configuration), then [log entries](../configuration/logging.md) for class
|
||||
|
@ -34,4 +34,12 @@ public interface MetadataSupervisorManager
|
||||
Map<String, List<VersionedSupervisorSpec>> getAll();
|
||||
|
||||
Map<String, SupervisorSpec> getLatest();
|
||||
|
||||
/**
|
||||
* Remove terminated supervisors created before the given timestamp.
|
||||
*
|
||||
* @param timestamp timestamp in milliseconds
|
||||
* @return number of supervisor removed
|
||||
*/
|
||||
int removeTerminatedSupervisorsOlderThan(long timestamp);
|
||||
}
|
||||
|
@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableMap;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.guice.ManageLifecycle;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.indexing.overlord.supervisor.NoopSupervisorSpec;
|
||||
import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
|
||||
import org.apache.druid.indexing.overlord.supervisor.VersionedSupervisorSpec;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
@ -35,10 +36,12 @@ import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.joda.time.DateTime;
|
||||
import org.skife.jdbi.v2.FoldController;
|
||||
import org.skife.jdbi.v2.Folder3;
|
||||
import org.skife.jdbi.v2.Handle;
|
||||
import org.skife.jdbi.v2.IDBI;
|
||||
import org.skife.jdbi.v2.PreparedBatch;
|
||||
import org.skife.jdbi.v2.StatementContext;
|
||||
import org.skife.jdbi.v2.tweak.HandleCallback;
|
||||
import org.skife.jdbi.v2.tweak.ResultSetMapper;
|
||||
@ -50,6 +53,7 @@ import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
@ManageLifecycle
|
||||
public class SQLMetadataSupervisorManager implements MetadataSupervisorManager
|
||||
@ -249,6 +253,34 @@ public class SQLMetadataSupervisorManager implements MetadataSupervisorManager
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int removeTerminatedSupervisorsOlderThan(long timestamp)
|
||||
{
|
||||
DateTime dateTime = DateTimes.utc(timestamp);
|
||||
Map<String, SupervisorSpec> supervisors = getLatest();
|
||||
return dbi.withHandle(
|
||||
handle -> {
|
||||
final PreparedBatch batch = handle.prepareBatch(
|
||||
StringUtils.format(
|
||||
"DELETE FROM %1$s WHERE spec_id = :spec_id AND created_date < '%2$s'",
|
||||
getSupervisorsTable(),
|
||||
dateTime.toString()
|
||||
)
|
||||
);
|
||||
for (Map.Entry<String, SupervisorSpec> supervisor : supervisors.entrySet()) {
|
||||
final SupervisorSpec spec = supervisor.getValue();
|
||||
// Terminated supervisor will have it's latest supervisorSpec as NoopSupervisorSpec
|
||||
// (NoopSupervisorSpec is used as a tombstone marker)
|
||||
if (spec instanceof NoopSupervisorSpec) {
|
||||
batch.bind("spec_id", supervisor.getKey()).add();
|
||||
}
|
||||
}
|
||||
int[] result = batch.execute();
|
||||
return IntStream.of(result).sum();
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private String getSupervisorsTable()
|
||||
{
|
||||
return dbTables.get().getSupervisorTable();
|
||||
|
@ -55,6 +55,14 @@ public abstract class DruidCoordinatorConfig
|
||||
@Default("0")
|
||||
public abstract int getCoordinatorKillMaxSegments();
|
||||
|
||||
@Config("druid.coordinator.kill.supervisor.period")
|
||||
@Default("P1D")
|
||||
public abstract Duration getCoordinatorSupervisorKillPeriod();
|
||||
|
||||
@Config("druid.coordinator.kill.supervisor.durationToRetain")
|
||||
@Default("PT-1s")
|
||||
public abstract Duration getCoordinatorSupervisorKillDurationToRetain();
|
||||
|
||||
@Config("druid.coordinator.kill.audit.period")
|
||||
@Default("P1D")
|
||||
public abstract Duration getCoordinatorAuditKillPeriod();
|
||||
|
@ -0,0 +1,84 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.server.coordinator.duty;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
|
||||
import org.apache.druid.metadata.MetadataSupervisorManager;
|
||||
import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
|
||||
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
|
||||
|
||||
/**
|
||||
* CoordinatorDuty for automatic deletion of terminated supervisors from the supervisor table in metadata storage.
|
||||
*/
|
||||
public class KillSupervisors implements CoordinatorDuty
|
||||
{
|
||||
private static final Logger log = new Logger(KillSupervisors.class);
|
||||
|
||||
private final long period;
|
||||
private final long retainDuration;
|
||||
private long lastKillTime = 0;
|
||||
|
||||
private final MetadataSupervisorManager metadataSupervisorManager;
|
||||
|
||||
@Inject
|
||||
public KillSupervisors(
|
||||
DruidCoordinatorConfig config,
|
||||
MetadataSupervisorManager metadataSupervisorManager
|
||||
)
|
||||
{
|
||||
this.metadataSupervisorManager = metadataSupervisorManager;
|
||||
this.period = config.getCoordinatorSupervisorKillPeriod().getMillis();
|
||||
Preconditions.checkArgument(
|
||||
this.period >= config.getCoordinatorMetadataStoreManagementPeriod().getMillis(),
|
||||
"Coordinator supervisor kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod"
|
||||
);
|
||||
this.retainDuration = config.getCoordinatorSupervisorKillDurationToRetain().getMillis();
|
||||
Preconditions.checkArgument(this.retainDuration >= 0, "Coordinator supervisor kill retainDuration must be >= 0");
|
||||
log.debug(
|
||||
"Supervisor Kill Task scheduling enabled with period [%s], retainDuration [%s]",
|
||||
this.period,
|
||||
this.retainDuration
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
|
||||
{
|
||||
if ((lastKillTime + period) < System.currentTimeMillis()) {
|
||||
lastKillTime = System.currentTimeMillis();
|
||||
|
||||
long timestamp = System.currentTimeMillis() - retainDuration;
|
||||
int supervisorRemoved = metadataSupervisorManager.removeTerminatedSupervisorsOlderThan(timestamp);
|
||||
ServiceEmitter emitter = params.getEmitter();
|
||||
emitter.emit(
|
||||
new ServiceMetricEvent.Builder().build(
|
||||
"metadata/kill/supervisor/count",
|
||||
supervisorRemoved
|
||||
)
|
||||
);
|
||||
log.info("Finished running KillSupervisors duty. Removed %,d supervisor", supervisorRemoved);
|
||||
}
|
||||
return params;
|
||||
}
|
||||
}
|
@ -21,12 +21,15 @@ package org.apache.druid.metadata;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.indexing.overlord.supervisor.NoopSupervisorSpec;
|
||||
import org.apache.druid.indexing.overlord.supervisor.Supervisor;
|
||||
import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
|
||||
import org.apache.druid.indexing.overlord.supervisor.VersionedSupervisorSpec;
|
||||
import org.apache.druid.indexing.overlord.supervisor.autoscaler.SupervisorTaskAutoScaler;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
@ -85,6 +88,83 @@ public class SQLMetadataSupervisorManagerTest
|
||||
supervisorManager = new SQLMetadataSupervisorManager(MAPPER, connector, Suppliers.ofInstance(tablesConfig));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveTerminatedSupervisorsOlderThanSupervisorActiveShouldNotBeDeleted()
|
||||
{
|
||||
final String supervisor1 = "test-supervisor-1";
|
||||
final Map<String, String> data1rev1 = ImmutableMap.of("key1-1", "value1-1-1", "key1-2", "value1-2-1");
|
||||
Assert.assertTrue(supervisorManager.getAll().isEmpty());
|
||||
supervisorManager.insert(supervisor1, new TestSupervisorSpec(supervisor1, data1rev1));
|
||||
// Test that supervisor was inserted
|
||||
Map<String, List<VersionedSupervisorSpec>> supervisorSpecs = supervisorManager.getAll();
|
||||
Assert.assertEquals(1, supervisorSpecs.size());
|
||||
Map<String, SupervisorSpec> latestSpecs = supervisorManager.getLatest();
|
||||
Assert.assertEquals(1, latestSpecs.size());
|
||||
// Try delete. Supervisor should not be deleted as it is still active
|
||||
int deleteCount = supervisorManager.removeTerminatedSupervisorsOlderThan(System.currentTimeMillis());
|
||||
// Test that supervisor was not deleted
|
||||
Assert.assertEquals(0, deleteCount);
|
||||
supervisorSpecs = supervisorManager.getAll();
|
||||
Assert.assertEquals(1, supervisorSpecs.size());
|
||||
latestSpecs = supervisorManager.getLatest();
|
||||
Assert.assertEquals(1, latestSpecs.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveTerminatedSupervisorsOlderThanWithSupervisorTerminatedAndOlderThanTimeShouldBeDeleted()
|
||||
{
|
||||
final String supervisor1 = "test-supervisor-1";
|
||||
final String datasource1 = "datasource-1";
|
||||
final Map<String, String> data1rev1 = ImmutableMap.of("key1-1", "value1-1-1", "key1-2", "value1-2-1");
|
||||
Assert.assertTrue(supervisorManager.getAll().isEmpty());
|
||||
supervisorManager.insert(supervisor1, new TestSupervisorSpec(supervisor1, data1rev1));
|
||||
supervisorManager.insert(supervisor1, new NoopSupervisorSpec(supervisor1, ImmutableList.of(datasource1)));
|
||||
// Test that supervisor was inserted
|
||||
Map<String, List<VersionedSupervisorSpec>> supervisorSpecs = supervisorManager.getAll();
|
||||
Assert.assertEquals(1, supervisorSpecs.size());
|
||||
Assert.assertEquals(2, supervisorSpecs.get(supervisor1).size());
|
||||
Map<String, SupervisorSpec> latestSpecs = supervisorManager.getLatest();
|
||||
Assert.assertEquals(1, latestSpecs.size());
|
||||
Assert.assertEquals(ImmutableList.of(datasource1), ((NoopSupervisorSpec) latestSpecs.get(supervisor1)).getDataSources());
|
||||
// Do delete. Supervisor should be deleted as it is terminated
|
||||
int deleteCount = supervisorManager.removeTerminatedSupervisorsOlderThan(System.currentTimeMillis());
|
||||
// Verify that supervisor was actually deleted
|
||||
Assert.assertEquals(2, deleteCount);
|
||||
supervisorSpecs = supervisorManager.getAll();
|
||||
Assert.assertEquals(0, supervisorSpecs.size());
|
||||
latestSpecs = supervisorManager.getLatest();
|
||||
Assert.assertEquals(0, latestSpecs.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveTerminatedSupervisorsOlderThanWithSupervisorTerminatedButNotOlderThanTimeShouldNotBeDeleted()
|
||||
{
|
||||
final String supervisor1 = "test-supervisor-1";
|
||||
final String datasource1 = "datasource-1";
|
||||
final Map<String, String> data1rev1 = ImmutableMap.of("key1-1", "value1-1-1", "key1-2", "value1-2-1");
|
||||
Assert.assertTrue(supervisorManager.getAll().isEmpty());
|
||||
supervisorManager.insert(supervisor1, new TestSupervisorSpec(supervisor1, data1rev1));
|
||||
supervisorManager.insert(supervisor1, new NoopSupervisorSpec(supervisor1, ImmutableList.of(datasource1)));
|
||||
// Test that supervisor was inserted
|
||||
Map<String, List<VersionedSupervisorSpec>> supervisorSpecs = supervisorManager.getAll();
|
||||
Assert.assertEquals(1, supervisorSpecs.size());
|
||||
Assert.assertEquals(2, supervisorSpecs.get(supervisor1).size());
|
||||
Map<String, SupervisorSpec> latestSpecs = supervisorManager.getLatest();
|
||||
Assert.assertEquals(1, latestSpecs.size());
|
||||
Assert.assertEquals(ImmutableList.of(datasource1), ((NoopSupervisorSpec) latestSpecs.get(supervisor1)).getDataSources());
|
||||
// Do delete. Supervisor should not be deleted. Supervisor is terminated but it was created just now so it's
|
||||
// created timestamp will be later than the timestamp 2012-01-01T00:00:00Z
|
||||
int deleteCount = supervisorManager.removeTerminatedSupervisorsOlderThan(DateTimes.of("2012-01-01T00:00:00Z").getMillis());
|
||||
// Verify that supervisor was not deleted
|
||||
Assert.assertEquals(0, deleteCount);
|
||||
supervisorSpecs = supervisorManager.getAll();
|
||||
Assert.assertEquals(1, supervisorSpecs.size());
|
||||
Assert.assertEquals(2, supervisorSpecs.get(supervisor1).size());
|
||||
latestSpecs = supervisorManager.getLatest();
|
||||
Assert.assertEquals(1, latestSpecs.size());
|
||||
Assert.assertEquals(ImmutableList.of(datasource1), ((NoopSupervisorSpec) latestSpecs.get(supervisor1)).getDataSources());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertAndGet()
|
||||
{
|
||||
|
@ -176,6 +176,8 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT0s")
|
||||
);
|
||||
|
@ -148,6 +148,8 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT0s")
|
||||
);
|
||||
|
@ -85,6 +85,8 @@ public class HttpLoadQueuePeonTest
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
Duration.ZERO
|
||||
)
|
||||
|
@ -100,6 +100,8 @@ public class LoadQueuePeonTest extends CuratorTestBase
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
Duration.millis(0)
|
||||
)
|
||||
@ -300,6 +302,8 @@ public class LoadQueuePeonTest extends CuratorTestBase
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT1s")
|
||||
)
|
||||
@ -357,6 +361,8 @@ public class LoadQueuePeonTest extends CuratorTestBase
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT1s")
|
||||
)
|
||||
|
@ -49,6 +49,8 @@ public class LoadQueuePeonTester extends CuratorLoadQueuePeon
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT1s")
|
||||
)
|
||||
|
@ -31,6 +31,8 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
||||
private final Duration loadTimeoutDelay;
|
||||
private final Duration coordinatorKillPeriod;
|
||||
private final Duration coordinatorKillDurationToRetain;
|
||||
private final Duration coordinatorSupervisorKillPeriod;
|
||||
private final Duration coordinatorSupervisorKillDurationToRetain;
|
||||
private final Duration coordinatorAuditKillPeriod;
|
||||
private final Duration coordinatorAuditKillDurationToRetain;
|
||||
private final Duration coordinatorRuleKillPeriod;
|
||||
@ -46,6 +48,8 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
||||
Duration loadTimeoutDelay,
|
||||
Duration coordinatorKillPeriod,
|
||||
Duration coordinatorKillDurationToRetain,
|
||||
Duration coordinatorSupervisorKillPeriod,
|
||||
Duration coordinatorSupervisorKillDurationToRetain,
|
||||
Duration coordinatorAuditKillPeriod,
|
||||
Duration coordinatorAuditKillDurationToRetain,
|
||||
Duration coordinatorRuleKillPeriod,
|
||||
@ -61,6 +65,8 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
||||
this.loadTimeoutDelay = loadTimeoutDelay;
|
||||
this.coordinatorKillPeriod = coordinatorKillPeriod;
|
||||
this.coordinatorKillDurationToRetain = coordinatorKillDurationToRetain;
|
||||
this.coordinatorSupervisorKillPeriod = coordinatorSupervisorKillPeriod;
|
||||
this.coordinatorSupervisorKillDurationToRetain = coordinatorSupervisorKillDurationToRetain;
|
||||
this.coordinatorAuditKillPeriod = coordinatorAuditKillPeriod;
|
||||
this.coordinatorAuditKillDurationToRetain = coordinatorAuditKillDurationToRetain;
|
||||
this.coordinatorRuleKillPeriod = coordinatorRuleKillPeriod;
|
||||
@ -105,6 +111,18 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
||||
return coordinatorKillDurationToRetain;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Duration getCoordinatorSupervisorKillPeriod()
|
||||
{
|
||||
return coordinatorSupervisorKillPeriod;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Duration getCoordinatorSupervisorKillDurationToRetain()
|
||||
{
|
||||
return coordinatorSupervisorKillDurationToRetain;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Duration getCoordinatorAuditKillPeriod()
|
||||
{
|
||||
|
@ -62,6 +62,8 @@ public class KillAuditLogTest
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration(Long.MAX_VALUE),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
@ -86,6 +88,8 @@ public class KillAuditLogTest
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
@ -110,6 +114,8 @@ public class KillAuditLogTest
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT3S"),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
@ -133,6 +139,8 @@ public class KillAuditLogTest
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT-1S"),
|
||||
null,
|
||||
|
@ -71,6 +71,8 @@ public class KillRulesTest
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration(Long.MAX_VALUE),
|
||||
new Duration("PT1S"),
|
||||
10,
|
||||
@ -95,6 +97,8 @@ public class KillRulesTest
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT1S"),
|
||||
10,
|
||||
@ -119,6 +123,8 @@ public class KillRulesTest
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT3S"),
|
||||
new Duration("PT1S"),
|
||||
10,
|
||||
@ -142,6 +148,8 @@ public class KillRulesTest
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT-1S"),
|
||||
10,
|
||||
|
@ -0,0 +1,155 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.server.coordinator.duty;
|
||||
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEventBuilder;
|
||||
import org.apache.druid.metadata.MetadataSupervisorManager;
|
||||
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
|
||||
import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.mockito.ArgumentMatchers;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.junit.MockitoJUnitRunner;
|
||||
|
||||
@RunWith(MockitoJUnitRunner.class)
|
||||
public class KillSupervisorsTest
|
||||
{
|
||||
@Mock
|
||||
private MetadataSupervisorManager mockMetadataSupervisorManager;
|
||||
|
||||
@Mock
|
||||
private DruidCoordinatorRuntimeParams mockDruidCoordinatorRuntimeParams;
|
||||
|
||||
@Mock
|
||||
private ServiceEmitter mockServiceEmitter;
|
||||
|
||||
@Rule
|
||||
public ExpectedException exception = ExpectedException.none();
|
||||
|
||||
private KillSupervisors killSupervisors;
|
||||
|
||||
@Test
|
||||
public void testRunSkipIfLastRunLessThanPeriod()
|
||||
{
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration(Long.MAX_VALUE),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
killSupervisors = new KillSupervisors(druidCoordinatorConfig, mockMetadataSupervisorManager);
|
||||
killSupervisors.run(mockDruidCoordinatorRuntimeParams);
|
||||
Mockito.verifyZeroInteractions(mockMetadataSupervisorManager);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunNotSkipIfLastRunMoreThanPeriod()
|
||||
{
|
||||
Mockito.when(mockDruidCoordinatorRuntimeParams.getEmitter()).thenReturn(mockServiceEmitter);
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
killSupervisors = new KillSupervisors(druidCoordinatorConfig, mockMetadataSupervisorManager);
|
||||
killSupervisors.run(mockDruidCoordinatorRuntimeParams);
|
||||
Mockito.verify(mockMetadataSupervisorManager).removeTerminatedSupervisorsOlderThan(ArgumentMatchers.anyLong());
|
||||
Mockito.verify(mockServiceEmitter).emit(ArgumentMatchers.any(ServiceEventBuilder.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstructorFailIfInvalidPeriod()
|
||||
{
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT3S"),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
exception.expect(IllegalArgumentException.class);
|
||||
exception.expectMessage("Coordinator supervisor kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod");
|
||||
killSupervisors = new KillSupervisors(druidCoordinatorConfig, mockMetadataSupervisorManager);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstructorFailIfInvalidRetainDuration()
|
||||
{
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT-1S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
exception.expect(IllegalArgumentException.class);
|
||||
exception.expectMessage("Coordinator supervisor kill retainDuration must be >= 0");
|
||||
killSupervisors = new KillSupervisors(druidCoordinatorConfig, mockMetadataSupervisorManager);
|
||||
}
|
||||
}
|
@ -113,6 +113,8 @@ public class KillUnusedSegmentsTest
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
1000,
|
||||
Duration.ZERO
|
||||
)
|
||||
|
@ -74,6 +74,7 @@ import org.apache.druid.server.coordinator.LoadQueueTaskMaster;
|
||||
import org.apache.druid.server.coordinator.duty.CoordinatorDuty;
|
||||
import org.apache.druid.server.coordinator.duty.KillAuditLog;
|
||||
import org.apache.druid.server.coordinator.duty.KillRules;
|
||||
import org.apache.druid.server.coordinator.duty.KillSupervisors;
|
||||
import org.apache.druid.server.coordinator.duty.KillUnusedSegments;
|
||||
import org.apache.druid.server.http.ClusterResource;
|
||||
import org.apache.druid.server.http.CompactionResource;
|
||||
@ -252,6 +253,11 @@ public class CliCoordinator extends ServerRunnable
|
||||
CoordinatorDuty.class,
|
||||
CoordinatorMetadataStoreManagementDuty.class
|
||||
);
|
||||
conditionalMetadataStoreManagementDutyMultibind.addConditionBinding(
|
||||
"druid.coordinator.kill.rule.on",
|
||||
Predicates.equalTo("true"),
|
||||
KillSupervisors.class
|
||||
);
|
||||
conditionalMetadataStoreManagementDutyMultibind.addConditionBinding(
|
||||
"druid.coordinator.kill.audit.on",
|
||||
Predicates.equalTo("true"),
|
||||
|
Loading…
x
Reference in New Issue
Block a user