mirror of https://github.com/apache/druid.git
Add feature to automatically remove rules based on retention period (#11164)
* Add feature to automatically remove rules based on retention period * Add feature to automatically remove rules based on retention period * address comments
This commit is contained in:
parent
809e001939
commit
84aac4832d
|
@ -750,6 +750,9 @@ These Coordinator static configurations can be defined in the `coordinator/runti
|
|||
|`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.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|
|
||||
|
||||
##### Segment Management
|
||||
|Property|Possible Values|Description|Default|
|
||||
|
|
|
@ -257,6 +257,7 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina
|
|||
|`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.|
|
||||
|
||||
|
||||
If `emitBalancingStats` is set to `true` in the Coordinator [dynamic configuration](../configuration/index.md#dynamic-configuration), then [log entries](../configuration/logging.md) for class
|
||||
|
|
|
@ -42,4 +42,12 @@ public interface MetadataRuleManager
|
|||
List<Rule> getRulesWithDefault(String dataSource);
|
||||
|
||||
boolean overrideRule(String dataSource, List<Rule> rulesConfig, AuditInfo auditInfo);
|
||||
|
||||
/**
|
||||
* Remove rules for non-existence datasource (datasource with no segment) created older than the given timestamp.
|
||||
*
|
||||
* @param timestamp timestamp in milliseconds
|
||||
* @return number of rules removed
|
||||
*/
|
||||
int removeRulesForEmptyDatasourcesOlderThan(long timestamp);
|
||||
}
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.skife.jdbi.v2.IDBI;
|
|||
import org.skife.jdbi.v2.StatementContext;
|
||||
import org.skife.jdbi.v2.TransactionCallback;
|
||||
import org.skife.jdbi.v2.TransactionStatus;
|
||||
import org.skife.jdbi.v2.Update;
|
||||
import org.skife.jdbi.v2.tweak.HandleCallback;
|
||||
import org.skife.jdbi.v2.tweak.ResultSetMapper;
|
||||
|
||||
|
@ -389,6 +390,8 @@ public class SQLMetadataRuleManager implements MetadataRuleManager
|
|||
.build(),
|
||||
handle
|
||||
);
|
||||
// Note that the method removeRulesForEmptyDatasourcesOlderThan depends on the version field
|
||||
// to be a timestamp
|
||||
String version = auditTime.toString();
|
||||
handle.createStatement(
|
||||
StringUtils.format(
|
||||
|
@ -421,8 +424,40 @@ public class SQLMetadataRuleManager implements MetadataRuleManager
|
|||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int removeRulesForEmptyDatasourcesOlderThan(long timestamp)
|
||||
{
|
||||
// Note that this DELETE SQL depends on the version field to be a timestamp. Hence, this
|
||||
// method depends on overrideRule method to set version to timestamp when the rule entry is created
|
||||
DateTime dateTime = DateTimes.utc(timestamp);
|
||||
synchronized (lock) {
|
||||
return dbi.withHandle(
|
||||
handle -> {
|
||||
Update sql = handle.createStatement(
|
||||
// Note that this query could be expensive when the segments table is large
|
||||
// However, since currently this query is run very infrequent (by default once a day by the KillRules Coordinator duty)
|
||||
// and the inner query on segment table is a READ (no locking), it is keep this way.
|
||||
StringUtils.format(
|
||||
"DELETE FROM %1$s WHERE datasource NOT IN (SELECT DISTINCT datasource from %2$s) and datasource!=:default_rule and version < :date_time",
|
||||
getRulesTable(),
|
||||
getSegmentsTable()
|
||||
)
|
||||
);
|
||||
return sql.bind("default_rule", config.getDefaultRule())
|
||||
.bind("date_time", dateTime.toString())
|
||||
.execute();
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private String getRulesTable()
|
||||
{
|
||||
return dbTables.getRulesTable();
|
||||
}
|
||||
|
||||
private String getSegmentsTable()
|
||||
{
|
||||
return dbTables.getSegmentsTable();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -63,6 +63,14 @@ public abstract class DruidCoordinatorConfig
|
|||
@Default("PT-1s")
|
||||
public abstract Duration getCoordinatorAuditKillDurationToRetain();
|
||||
|
||||
@Config("druid.coordinator.kill.rule.period")
|
||||
@Default("P1D")
|
||||
public abstract Duration getCoordinatorRuleKillPeriod();
|
||||
|
||||
@Config("druid.coordinator.kill.rule.durationToRetain")
|
||||
@Default("PT-1s")
|
||||
public abstract Duration getCoordinatorRuleKillDurationToRetain();
|
||||
|
||||
@Config("druid.coordinator.load.timeout")
|
||||
public Duration getLoadTimeoutDelay()
|
||||
{
|
||||
|
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* 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.server.coordinator.DruidCoordinatorConfig;
|
||||
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
|
||||
|
||||
public class KillRules implements CoordinatorDuty
|
||||
{
|
||||
private static final Logger log = new Logger(KillRules.class);
|
||||
|
||||
private final long period;
|
||||
private final long retainDuration;
|
||||
private long lastKillTime = 0;
|
||||
|
||||
@Inject
|
||||
public KillRules(
|
||||
DruidCoordinatorConfig config
|
||||
)
|
||||
{
|
||||
this.period = config.getCoordinatorRuleKillPeriod().getMillis();
|
||||
Preconditions.checkArgument(
|
||||
this.period >= config.getCoordinatorMetadataStoreManagementPeriod().getMillis(),
|
||||
"coordinator rule kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod"
|
||||
);
|
||||
this.retainDuration = config.getCoordinatorRuleKillDurationToRetain().getMillis();
|
||||
Preconditions.checkArgument(this.retainDuration >= 0, "coordinator rule kill retainDuration must be >= 0");
|
||||
log.debug(
|
||||
"Rule 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 ruleRemoved = params.getDatabaseRuleManager().removeRulesForEmptyDatasourcesOlderThan(timestamp);
|
||||
ServiceEmitter emitter = params.getEmitter();
|
||||
emitter.emit(
|
||||
new ServiceMetricEvent.Builder().build(
|
||||
"metadata/kill/rule/count",
|
||||
ruleRemoved
|
||||
)
|
||||
);
|
||||
log.info("Finished running KillRules duty. Removed %,d rule", ruleRemoved);
|
||||
}
|
||||
return params;
|
||||
}
|
||||
}
|
|
@ -23,19 +23,24 @@ package org.apache.druid.metadata;
|
|||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
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.audit.AuditEntry;
|
||||
import org.apache.druid.audit.AuditInfo;
|
||||
import org.apache.druid.audit.AuditManager;
|
||||
import org.apache.druid.client.DruidServer;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.server.audit.SQLAuditManager;
|
||||
import org.apache.druid.server.audit.SQLAuditManagerConfig;
|
||||
import org.apache.druid.server.coordinator.rules.IntervalLoadRule;
|
||||
import org.apache.druid.server.coordinator.rules.Rule;
|
||||
import org.apache.druid.server.metrics.NoopServiceEmitter;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.NoneShardSpec;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
|
@ -55,8 +60,9 @@ public class SQLMetadataRuleManagerTest
|
|||
private MetadataStorageTablesConfig tablesConfig;
|
||||
private SQLMetadataRuleManager ruleManager;
|
||||
private AuditManager auditManager;
|
||||
private SQLMetadataSegmentPublisher publisher;
|
||||
private final ObjectMapper mapper = new DefaultObjectMapper();
|
||||
|
||||
private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
|
@ -80,6 +86,12 @@ public class SQLMetadataRuleManagerTest
|
|||
connector,
|
||||
auditManager
|
||||
);
|
||||
connector.createSegmentTable();
|
||||
publisher = new SQLMetadataSegmentPublisher(
|
||||
jsonMapper,
|
||||
derbyConnectorRule.metadataTablesConfigSupplier().get(),
|
||||
connector
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -201,6 +213,143 @@ public class SQLMetadataRuleManagerTest
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveRulesOlderThanWithNonExistenceDatasourceAndOlderThanTimestampShouldDelete()
|
||||
{
|
||||
List<Rule> rules = ImmutableList.of(
|
||||
new IntervalLoadRule(
|
||||
Intervals.of("2015-01-01/2015-02-01"), ImmutableMap.of(
|
||||
DruidServer.DEFAULT_TIER,
|
||||
DruidServer.DEFAULT_NUM_REPLICANTS
|
||||
)
|
||||
)
|
||||
);
|
||||
AuditInfo auditInfo = new AuditInfo("test_author", "test_comment", "127.0.0.1");
|
||||
ruleManager.overrideRule(
|
||||
"test_dataSource",
|
||||
rules,
|
||||
auditInfo
|
||||
);
|
||||
// Verify that rule was added
|
||||
ruleManager.poll();
|
||||
Map<String, List<Rule>> allRules = ruleManager.getAllRules();
|
||||
Assert.assertEquals(1, allRules.size());
|
||||
Assert.assertEquals(1, allRules.get("test_dataSource").size());
|
||||
|
||||
// Now delete rules
|
||||
ruleManager.removeRulesForEmptyDatasourcesOlderThan(System.currentTimeMillis());
|
||||
|
||||
// Verify that rule was deleted
|
||||
ruleManager.poll();
|
||||
allRules = ruleManager.getAllRules();
|
||||
Assert.assertEquals(0, allRules.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveRulesOlderThanWithNonExistenceDatasourceAndNewerThanTimestampShouldNotDelete()
|
||||
{
|
||||
List<Rule> rules = ImmutableList.of(
|
||||
new IntervalLoadRule(
|
||||
Intervals.of("2015-01-01/2015-02-01"), ImmutableMap.of(
|
||||
DruidServer.DEFAULT_TIER,
|
||||
DruidServer.DEFAULT_NUM_REPLICANTS
|
||||
)
|
||||
)
|
||||
);
|
||||
AuditInfo auditInfo = new AuditInfo("test_author", "test_comment", "127.0.0.1");
|
||||
ruleManager.overrideRule(
|
||||
"test_dataSource",
|
||||
rules,
|
||||
auditInfo
|
||||
);
|
||||
// Verify that rule was added
|
||||
ruleManager.poll();
|
||||
Map<String, List<Rule>> allRules = ruleManager.getAllRules();
|
||||
Assert.assertEquals(1, allRules.size());
|
||||
Assert.assertEquals(1, allRules.get("test_dataSource").size());
|
||||
|
||||
// This will not delete the rule as the rule was created just now so it will have the created timestamp later than
|
||||
// the timestamp 2012-01-01T00:00:00Z
|
||||
ruleManager.removeRulesForEmptyDatasourcesOlderThan(DateTimes.of("2012-01-01T00:00:00Z").getMillis());
|
||||
|
||||
// Verify that rule was not deleted
|
||||
ruleManager.poll();
|
||||
allRules = ruleManager.getAllRules();
|
||||
Assert.assertEquals(1, allRules.size());
|
||||
Assert.assertEquals(1, allRules.get("test_dataSource").size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveRulesOlderThanWithActiveDatasourceShouldNotDelete() throws Exception
|
||||
{
|
||||
List<Rule> rules = ImmutableList.of(
|
||||
new IntervalLoadRule(
|
||||
Intervals.of("2015-01-01/2015-02-01"), ImmutableMap.of(
|
||||
DruidServer.DEFAULT_TIER,
|
||||
DruidServer.DEFAULT_NUM_REPLICANTS
|
||||
)
|
||||
)
|
||||
);
|
||||
AuditInfo auditInfo = new AuditInfo("test_author", "test_comment", "127.0.0.1");
|
||||
ruleManager.overrideRule(
|
||||
"test_dataSource",
|
||||
rules,
|
||||
auditInfo
|
||||
);
|
||||
|
||||
// Verify that rule was added
|
||||
ruleManager.poll();
|
||||
Map<String, List<Rule>> allRules = ruleManager.getAllRules();
|
||||
Assert.assertEquals(1, allRules.size());
|
||||
Assert.assertEquals(1, allRules.get("test_dataSource").size());
|
||||
|
||||
// Add segment metadata to segment table so that the datasource is considered active
|
||||
DataSegment dataSegment = new DataSegment(
|
||||
"test_dataSource",
|
||||
Intervals.of("2015-01-01/2015-02-01"),
|
||||
"1",
|
||||
ImmutableMap.of(
|
||||
"type", "s3_zip",
|
||||
"bucket", "test",
|
||||
"key", "test_dataSource/xxx"
|
||||
),
|
||||
ImmutableList.of("dim1", "dim2", "dim3"),
|
||||
ImmutableList.of("count", "value"),
|
||||
NoneShardSpec.instance(),
|
||||
1,
|
||||
1234L
|
||||
);
|
||||
publisher.publishSegment(dataSegment);
|
||||
|
||||
// This will not delete the rule as the datasource has segment in the segment metadata table
|
||||
ruleManager.removeRulesForEmptyDatasourcesOlderThan(System.currentTimeMillis());
|
||||
|
||||
// Verify that rule was not deleted
|
||||
ruleManager.poll();
|
||||
allRules = ruleManager.getAllRules();
|
||||
Assert.assertEquals(1, allRules.size());
|
||||
Assert.assertEquals(1, allRules.get("test_dataSource").size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRemoveRulesOlderThanShouldNotDeleteDefault()
|
||||
{
|
||||
// Create the default rule
|
||||
ruleManager.start();
|
||||
// Verify the default rule
|
||||
ruleManager.poll();
|
||||
Map<String, List<Rule>> allRules = ruleManager.getAllRules();
|
||||
Assert.assertEquals(1, allRules.size());
|
||||
Assert.assertEquals(1, allRules.get("_default").size());
|
||||
// Delete everything
|
||||
ruleManager.removeRulesForEmptyDatasourcesOlderThan(System.currentTimeMillis());
|
||||
// Verify the default rule was not deleted
|
||||
ruleManager.poll();
|
||||
allRules = ruleManager.getAllRules();
|
||||
Assert.assertEquals(1, allRules.size());
|
||||
Assert.assertEquals(1, allRules.get("_default").size());
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanup()
|
||||
{
|
||||
|
|
|
@ -174,6 +174,8 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT0s")
|
||||
);
|
||||
|
|
|
@ -146,6 +146,8 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT0s")
|
||||
);
|
||||
|
|
|
@ -83,6 +83,8 @@ public class HttpLoadQueuePeonTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
Duration.ZERO
|
||||
)
|
||||
|
|
|
@ -98,6 +98,8 @@ public class LoadQueuePeonTest extends CuratorTestBase
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
Duration.millis(0)
|
||||
)
|
||||
|
@ -296,6 +298,8 @@ public class LoadQueuePeonTest extends CuratorTestBase
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT1s")
|
||||
)
|
||||
|
@ -351,6 +355,8 @@ public class LoadQueuePeonTest extends CuratorTestBase
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT1s")
|
||||
)
|
||||
|
|
|
@ -47,6 +47,8 @@ public class LoadQueuePeonTester extends CuratorLoadQueuePeon
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT1s")
|
||||
)
|
||||
|
|
|
@ -33,6 +33,8 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
|||
private final Duration coordinatorKillDurationToRetain;
|
||||
private final Duration coordinatorAuditKillPeriod;
|
||||
private final Duration coordinatorAuditKillDurationToRetain;
|
||||
private final Duration coordinatorRuleKillPeriod;
|
||||
private final Duration coordinatorRuleKillDurationToRetain;
|
||||
private final Duration getLoadQueuePeonRepeatDelay;
|
||||
private final int coordinatorKillMaxSegments;
|
||||
|
||||
|
@ -46,6 +48,8 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
|||
Duration coordinatorKillDurationToRetain,
|
||||
Duration coordinatorAuditKillPeriod,
|
||||
Duration coordinatorAuditKillDurationToRetain,
|
||||
Duration coordinatorRuleKillPeriod,
|
||||
Duration coordinatorRuleKillDurationToRetain,
|
||||
int coordinatorKillMaxSegments,
|
||||
Duration getLoadQueuePeonRepeatDelay
|
||||
)
|
||||
|
@ -59,6 +63,8 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
|||
this.coordinatorKillDurationToRetain = coordinatorKillDurationToRetain;
|
||||
this.coordinatorAuditKillPeriod = coordinatorAuditKillPeriod;
|
||||
this.coordinatorAuditKillDurationToRetain = coordinatorAuditKillDurationToRetain;
|
||||
this.coordinatorRuleKillPeriod = coordinatorRuleKillPeriod;
|
||||
this.coordinatorRuleKillDurationToRetain = coordinatorRuleKillDurationToRetain;
|
||||
this.coordinatorKillMaxSegments = coordinatorKillMaxSegments;
|
||||
this.getLoadQueuePeonRepeatDelay = getLoadQueuePeonRepeatDelay;
|
||||
}
|
||||
|
@ -111,6 +117,18 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
|||
return coordinatorAuditKillDurationToRetain;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Duration getCoordinatorRuleKillPeriod()
|
||||
{
|
||||
return coordinatorRuleKillPeriod;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Duration getCoordinatorRuleKillDurationToRetain()
|
||||
{
|
||||
return coordinatorRuleKillDurationToRetain;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCoordinatorKillMaxSegments()
|
||||
{
|
||||
|
|
|
@ -64,6 +64,8 @@ public class KillAuditLogTest
|
|||
null,
|
||||
new Duration(Long.MAX_VALUE),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
@ -86,6 +88,8 @@ public class KillAuditLogTest
|
|||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
@ -108,6 +112,8 @@ public class KillAuditLogTest
|
|||
null,
|
||||
new Duration("PT3S"),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
@ -129,6 +135,8 @@ public class KillAuditLogTest
|
|||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT-1S"),
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
|
|
@ -0,0 +1,154 @@
|
|||
/*
|
||||
* 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.MetadataRuleManager;
|
||||
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
|
||||
import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig;
|
||||
import org.joda.time.Duration;
|
||||
import org.junit.Before;
|
||||
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 KillRulesTest
|
||||
{
|
||||
@Mock
|
||||
private MetadataRuleManager mockRuleManager;
|
||||
|
||||
@Mock
|
||||
private DruidCoordinatorRuntimeParams mockDruidCoordinatorRuntimeParams;
|
||||
|
||||
@Mock
|
||||
private ServiceEmitter mockServiceEmitter;
|
||||
|
||||
@Rule
|
||||
public ExpectedException exception = ExpectedException.none();
|
||||
|
||||
private KillRules killRules;
|
||||
|
||||
@Before
|
||||
public void setup()
|
||||
{
|
||||
Mockito.when(mockDruidCoordinatorRuntimeParams.getDatabaseRuleManager()).thenReturn(mockRuleManager);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunSkipIfLastRunLessThanPeriod()
|
||||
{
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration(Long.MAX_VALUE),
|
||||
new Duration("PT1S"),
|
||||
10,
|
||||
null
|
||||
);
|
||||
killRules = new KillRules(druidCoordinatorConfig);
|
||||
killRules.run(mockDruidCoordinatorRuntimeParams);
|
||||
Mockito.verifyZeroInteractions(mockRuleManager);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunNotSkipIfLastRunMoreThanPeriod()
|
||||
{
|
||||
Mockito.when(mockDruidCoordinatorRuntimeParams.getEmitter()).thenReturn(mockServiceEmitter);
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT1S"),
|
||||
10,
|
||||
null
|
||||
);
|
||||
killRules = new KillRules(druidCoordinatorConfig);
|
||||
killRules.run(mockDruidCoordinatorRuntimeParams);
|
||||
Mockito.verify(mockRuleManager).removeRulesForEmptyDatasourcesOlderThan(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,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT3S"),
|
||||
new Duration("PT1S"),
|
||||
10,
|
||||
null
|
||||
);
|
||||
exception.expect(IllegalArgumentException.class);
|
||||
exception.expectMessage("coordinator rule kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod");
|
||||
killRules = new KillRules(druidCoordinatorConfig);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstructorFailIfInvalidRetainDuration()
|
||||
{
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT-1S"),
|
||||
10,
|
||||
null
|
||||
);
|
||||
exception.expect(IllegalArgumentException.class);
|
||||
exception.expectMessage("coordinator rule kill retainDuration must be >= 0");
|
||||
killRules = new KillRules(druidCoordinatorConfig);
|
||||
}
|
||||
}
|
|
@ -111,6 +111,8 @@ public class KillUnusedSegmentsTest
|
|||
Duration.parse("PT86400S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
1000,
|
||||
Duration.ZERO
|
||||
)
|
||||
|
|
|
@ -73,6 +73,7 @@ import org.apache.druid.server.coordinator.KillStalePendingSegments;
|
|||
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.KillUnusedSegments;
|
||||
import org.apache.druid.server.http.ClusterResource;
|
||||
import org.apache.druid.server.http.CompactionResource;
|
||||
|
@ -256,6 +257,11 @@ public class CliCoordinator extends ServerRunnable
|
|||
Predicates.equalTo("true"),
|
||||
KillAuditLog.class
|
||||
);
|
||||
conditionalMetadataStoreManagementDutyMultibind.addConditionBinding(
|
||||
"druid.coordinator.kill.rule.on",
|
||||
Predicates.equalTo("true"),
|
||||
KillRules.class
|
||||
);
|
||||
|
||||
bindNodeRoleAndAnnouncer(
|
||||
binder,
|
||||
|
|
Loading…
Reference in New Issue