mirror of https://github.com/apache/druid.git
Add feature to automatically remove compaction configurations for inactive datasources (#11232)
* add auto cleanup * add auto cleanup * add auto cleanup * add tests * add tests * use retryutils * use retryutils * use retryutils * address comments
This commit is contained in:
parent
d11be88c4b
commit
3455352241
|
@ -0,0 +1,39 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.java.util;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.java.util.common.RetryUtils;
|
||||
|
||||
/**
|
||||
* This Exception class can be use with {@link RetryUtils}.
|
||||
* The method {@link RetryUtils#retry(RetryUtils.Task, Predicate, int)} retry condition (Predicate argument)
|
||||
* requires an exception to be thrown and applying the predicate to the thrown exception.
|
||||
* For cases where the task method does not throw an exception but still needs retrying,
|
||||
* the method can throw this RetryableException so that the RetryUtils can then retry the task
|
||||
*/
|
||||
public class RetryableException extends Exception
|
||||
{
|
||||
public RetryableException(Throwable t)
|
||||
{
|
||||
super(t);
|
||||
}
|
||||
|
||||
}
|
|
@ -22,6 +22,7 @@ package org.apache.druid.java.util.common;
|
|||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import org.apache.commons.lang3.mutable.MutableInt;
|
||||
import org.apache.druid.java.util.RetryableException;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.hamcrest.CoreMatchers;
|
||||
import org.junit.Assert;
|
||||
|
@ -190,4 +191,22 @@ public class RetryUtilsTest
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExceptionPredicateForRetryableException() throws Exception
|
||||
{
|
||||
final AtomicInteger count = new AtomicInteger();
|
||||
String result = RetryUtils.retry(
|
||||
() -> {
|
||||
if (count.incrementAndGet() >= 2) {
|
||||
return "hey";
|
||||
} else {
|
||||
throw new RetryableException(new RuntimeException("uhh"));
|
||||
}
|
||||
},
|
||||
e -> e instanceof RetryableException,
|
||||
3
|
||||
);
|
||||
Assert.assertEquals(result, "hey");
|
||||
Assert.assertEquals("count", 2, count.get());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -753,6 +753,8 @@ 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 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.compaction.on`| Boolean value for whether to enable automatic deletion of compaction configurations. If set to true, Coordinator will periodically remove compaction configuration of inactive datasource (datasource with no used and unused segments) from the config table in metadata storage. | No | False|
|
||||
|`druid.coordinator.kill.compaction.period`| How often to do automatic deletion of compaction configurations 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.compaction.on` is set to "True".| No| `P1D`|
|
||||
|`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 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|
|
||||
|
|
|
@ -258,6 +258,7 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina
|
|||
|`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/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/compaction/count`|Total number of compaction configurations that were automatically deleted from metadata store per each Coordinator kill compaction configuration duty run. Note that this metric is only emitted when `druid.coordinator.kill.compaction.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.|
|
||||
|`metadata/kill/datasource/count`|Total number of datasource metadata that were automatically deleted from metadata store per each Coordinator kill datasource duty run (Note: datasource metadata only exists for datasource created from supervisor). This metric can help adjust `druid.coordinator.kill.datasource.durationToRetain` configuration based on whether more or less datasource metadata need to be deleted per cycle. Note that this metric is only emitted when `druid.coordinator.kill.datasource.on` is set to true.| |Varies.|
|
||||
|
||||
|
|
|
@ -71,6 +71,10 @@ public abstract class DruidCoordinatorConfig
|
|||
@Default("PT-1s")
|
||||
public abstract Duration getCoordinatorAuditKillDurationToRetain();
|
||||
|
||||
@Config("druid.coordinator.kill.compaction.period")
|
||||
@Default("P1D")
|
||||
public abstract Duration getCoordinatorCompactionKillPeriod();
|
||||
|
||||
@Config("druid.coordinator.kill.rule.period")
|
||||
@Default("P1D")
|
||||
public abstract Duration getCoordinatorRuleKillPeriod();
|
||||
|
|
|
@ -0,0 +1,162 @@
|
|||
/*
|
||||
* 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.common.collect.ImmutableList;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.audit.AuditInfo;
|
||||
import org.apache.druid.common.config.ConfigManager;
|
||||
import org.apache.druid.common.config.JacksonConfigManager;
|
||||
import org.apache.druid.java.util.RetryableException;
|
||||
import org.apache.druid.java.util.common.RetryUtils;
|
||||
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.SqlSegmentsMetadataManager;
|
||||
import org.apache.druid.server.coordinator.CoordinatorCompactionConfig;
|
||||
import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
|
||||
import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
|
||||
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* CoordinatorDuty for automatic deletion of compaction configurations from the config table in metadata storage.
|
||||
* Note that this will delete compaction configuration for inactive datasources
|
||||
* (datasource with no used and unused segments) immediately.
|
||||
*/
|
||||
public class KillCompactionConfig implements CoordinatorDuty
|
||||
{
|
||||
private static final Logger log = new Logger(KillCompactionConfig.class);
|
||||
private static final int UPDATE_NUM_RETRY = 5;
|
||||
|
||||
static final String COUNT_METRIC = "metadata/kill/compaction/count";
|
||||
|
||||
private final long period;
|
||||
private long lastKillTime = 0;
|
||||
|
||||
private final JacksonConfigManager jacksonConfigManager;
|
||||
private final SqlSegmentsMetadataManager sqlSegmentsMetadataManager;
|
||||
|
||||
@Inject
|
||||
public KillCompactionConfig(
|
||||
DruidCoordinatorConfig config,
|
||||
SqlSegmentsMetadataManager sqlSegmentsMetadataManager,
|
||||
JacksonConfigManager jacksonConfigManager
|
||||
)
|
||||
{
|
||||
this.sqlSegmentsMetadataManager = sqlSegmentsMetadataManager;
|
||||
this.jacksonConfigManager = jacksonConfigManager;
|
||||
this.period = config.getCoordinatorCompactionKillPeriod().getMillis();
|
||||
Preconditions.checkArgument(
|
||||
this.period >= config.getCoordinatorMetadataStoreManagementPeriod().getMillis(),
|
||||
"Coordinator compaction configuration kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod"
|
||||
);
|
||||
log.debug(
|
||||
"Compaction Configuration Kill Task scheduling enabled with period [%s]",
|
||||
this.period
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
|
||||
{
|
||||
long currentTimeMillis = System.currentTimeMillis();
|
||||
if ((lastKillTime + period) < currentTimeMillis) {
|
||||
lastKillTime = currentTimeMillis;
|
||||
try {
|
||||
RetryUtils.retry(
|
||||
() -> {
|
||||
CoordinatorCompactionConfig current = CoordinatorCompactionConfig.current(jacksonConfigManager);
|
||||
// If current compaction config is empty then there is nothing to do
|
||||
if (CoordinatorCompactionConfig.empty().equals(current)) {
|
||||
log.info(
|
||||
"Finished running KillCompactionConfig duty. Nothing to do as compaction config is already empty.");
|
||||
emitMetric(params.getEmitter(), 0);
|
||||
return ConfigManager.SetResult.ok();
|
||||
}
|
||||
|
||||
// Get all active datasources
|
||||
// Note that we get all active datasources after getting compaction config to prevent race condition if new
|
||||
// datasource and config are added.
|
||||
Set<String> activeDatasources = sqlSegmentsMetadataManager.retrieveAllDataSourceNames();
|
||||
final Map<String, DataSourceCompactionConfig> updated = current
|
||||
.getCompactionConfigs()
|
||||
.stream()
|
||||
.filter(dataSourceCompactionConfig -> activeDatasources.contains(dataSourceCompactionConfig.getDataSource()))
|
||||
.collect(Collectors.toMap(DataSourceCompactionConfig::getDataSource, Function.identity()));
|
||||
|
||||
// Calculate number of compaction configs to remove for logging
|
||||
int compactionConfigRemoved = current.getCompactionConfigs().size() - updated.size();
|
||||
|
||||
ConfigManager.SetResult result = jacksonConfigManager.set(
|
||||
CoordinatorCompactionConfig.CONFIG_KEY,
|
||||
// Do database insert without swap if the current config is empty as this means the config may be null in the database
|
||||
current,
|
||||
CoordinatorCompactionConfig.from(current, ImmutableList.copyOf(updated.values())),
|
||||
new AuditInfo(
|
||||
"KillCompactionConfig",
|
||||
"CoordinatorDuty for automatic deletion of compaction config",
|
||||
""
|
||||
)
|
||||
);
|
||||
if (result.isOk()) {
|
||||
log.info(
|
||||
"Finished running KillCompactionConfig duty. Removed %,d compaction configs",
|
||||
compactionConfigRemoved
|
||||
);
|
||||
emitMetric(params.getEmitter(), compactionConfigRemoved);
|
||||
} else if (result.isRetryable()) {
|
||||
// Failed but is retryable
|
||||
log.debug("Retrying KillCompactionConfig duty");
|
||||
throw new RetryableException(result.getException());
|
||||
} else {
|
||||
// Failed and not retryable
|
||||
log.error(result.getException(), "Failed to kill compaction configurations");
|
||||
emitMetric(params.getEmitter(), 0);
|
||||
}
|
||||
return result;
|
||||
},
|
||||
e -> e instanceof RetryableException,
|
||||
UPDATE_NUM_RETRY
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(e, "Failed to kill compaction configurations");
|
||||
emitMetric(params.getEmitter(), 0);
|
||||
}
|
||||
}
|
||||
return params;
|
||||
}
|
||||
|
||||
private void emitMetric(ServiceEmitter emitter, int compactionConfigRemoved)
|
||||
{
|
||||
emitter.emit(
|
||||
new ServiceMetricEvent.Builder().build(
|
||||
COUNT_METRIC,
|
||||
compactionConfigRemoved
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -180,6 +180,7 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT0s")
|
||||
);
|
||||
|
|
|
@ -152,6 +152,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT0s")
|
||||
);
|
||||
|
|
|
@ -89,6 +89,7 @@ public class HttpLoadQueuePeonTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
Duration.ZERO
|
||||
)
|
||||
|
|
|
@ -104,6 +104,7 @@ public class LoadQueuePeonTest extends CuratorTestBase
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
Duration.millis(0)
|
||||
)
|
||||
|
@ -308,6 +309,7 @@ public class LoadQueuePeonTest extends CuratorTestBase
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT1s")
|
||||
)
|
||||
|
@ -369,6 +371,7 @@ public class LoadQueuePeonTest extends CuratorTestBase
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT1s")
|
||||
)
|
||||
|
|
|
@ -53,6 +53,7 @@ public class LoadQueuePeonTester extends CuratorLoadQueuePeon
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
new Duration("PT1s")
|
||||
)
|
||||
|
|
|
@ -35,6 +35,7 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
|||
private final Duration coordinatorSupervisorKillDurationToRetain;
|
||||
private final Duration coordinatorAuditKillPeriod;
|
||||
private final Duration coordinatorAuditKillDurationToRetain;
|
||||
private final Duration coordinatorCompactionKillPeriod;
|
||||
private final Duration coordinatorRuleKillPeriod;
|
||||
private final Duration coordinatorRuleKillDurationToRetain;
|
||||
private final Duration coordinatorDatasourceKillPeriod;
|
||||
|
@ -54,6 +55,7 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
|||
Duration coordinatorSupervisorKillDurationToRetain,
|
||||
Duration coordinatorAuditKillPeriod,
|
||||
Duration coordinatorAuditKillDurationToRetain,
|
||||
Duration coordinatorCompactionKillPeriod,
|
||||
Duration coordinatorRuleKillPeriod,
|
||||
Duration coordinatorRuleKillDurationToRetain,
|
||||
Duration coordinatorDatasourceKillPeriod,
|
||||
|
@ -73,6 +75,7 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
|||
this.coordinatorSupervisorKillDurationToRetain = coordinatorSupervisorKillDurationToRetain;
|
||||
this.coordinatorAuditKillPeriod = coordinatorAuditKillPeriod;
|
||||
this.coordinatorAuditKillDurationToRetain = coordinatorAuditKillDurationToRetain;
|
||||
this.coordinatorCompactionKillPeriod = coordinatorCompactionKillPeriod;
|
||||
this.coordinatorRuleKillPeriod = coordinatorRuleKillPeriod;
|
||||
this.coordinatorRuleKillDurationToRetain = coordinatorRuleKillDurationToRetain;
|
||||
this.coordinatorDatasourceKillPeriod = coordinatorDatasourceKillPeriod;
|
||||
|
@ -141,6 +144,12 @@ public class TestDruidCoordinatorConfig extends DruidCoordinatorConfig
|
|||
return coordinatorAuditKillDurationToRetain;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Duration getCoordinatorCompactionKillPeriod()
|
||||
{
|
||||
return coordinatorCompactionKillPeriod;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Duration getCoordinatorRuleKillPeriod()
|
||||
{
|
||||
|
|
|
@ -70,6 +70,7 @@ public class KillAuditLogTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
@ -98,6 +99,7 @@ public class KillAuditLogTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
@ -126,6 +128,7 @@ public class KillAuditLogTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
@ -153,6 +156,7 @@ public class KillAuditLogTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
|
|
@ -0,0 +1,368 @@
|
|||
/*
|
||||
* 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.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.common.config.ConfigManager;
|
||||
import org.apache.druid.common.config.JacksonConfigManager;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEventBuilder;
|
||||
import org.apache.druid.metadata.SqlSegmentsMetadataManager;
|
||||
import org.apache.druid.server.coordinator.CoordinatorCompactionConfig;
|
||||
import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
|
||||
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
|
||||
import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig;
|
||||
import org.apache.druid.server.coordinator.UserCompactionTaskGranularityConfig;
|
||||
import org.joda.time.Duration;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.mockito.ArgumentCaptor;
|
||||
import org.mockito.ArgumentMatchers;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.junit.MockitoJUnitRunner;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
@RunWith(MockitoJUnitRunner.class)
|
||||
public class KillCompactionConfigTest
|
||||
{
|
||||
@Mock
|
||||
private DruidCoordinatorRuntimeParams mockDruidCoordinatorRuntimeParams;
|
||||
|
||||
@Mock
|
||||
private ServiceEmitter mockServiceEmitter;
|
||||
|
||||
@Mock
|
||||
private SqlSegmentsMetadataManager mockSqlSegmentsMetadataManager;
|
||||
|
||||
@Mock
|
||||
private JacksonConfigManager mockJacksonConfigManager;
|
||||
|
||||
@Rule
|
||||
public ExpectedException exception = ExpectedException.none();
|
||||
|
||||
private KillCompactionConfig killCompactionConfig;
|
||||
|
||||
@Test
|
||||
public void testRunSkipIfLastRunLessThanPeriod()
|
||||
{
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration(Long.MAX_VALUE),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
killCompactionConfig = new KillCompactionConfig(druidCoordinatorConfig, mockSqlSegmentsMetadataManager, mockJacksonConfigManager);
|
||||
killCompactionConfig.run(mockDruidCoordinatorRuntimeParams);
|
||||
Mockito.verifyZeroInteractions(mockSqlSegmentsMetadataManager);
|
||||
Mockito.verifyZeroInteractions(mockJacksonConfigManager);
|
||||
Mockito.verifyZeroInteractions(mockServiceEmitter);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstructorFailIfInvalidPeriod()
|
||||
{
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT3S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
exception.expect(IllegalArgumentException.class);
|
||||
exception.expectMessage("Coordinator compaction configuration kill period must be >= druid.coordinator.period.metadataStoreManagementPeriod");
|
||||
killCompactionConfig = new KillCompactionConfig(druidCoordinatorConfig, mockSqlSegmentsMetadataManager, mockJacksonConfigManager);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testRunDoNothingIfCurrentConfigIsEmpty()
|
||||
{
|
||||
Mockito.when(mockDruidCoordinatorRuntimeParams.getEmitter()).thenReturn(mockServiceEmitter);
|
||||
// Set current compaction config to an empty compaction config
|
||||
Mockito.when(mockJacksonConfigManager.watch(
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()))
|
||||
).thenReturn(new AtomicReference<>(CoordinatorCompactionConfig.empty()));
|
||||
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
killCompactionConfig = new KillCompactionConfig(druidCoordinatorConfig, mockSqlSegmentsMetadataManager, mockJacksonConfigManager);
|
||||
killCompactionConfig.run(mockDruidCoordinatorRuntimeParams);
|
||||
Mockito.verifyZeroInteractions(mockSqlSegmentsMetadataManager);
|
||||
final ArgumentCaptor<ServiceEventBuilder> emittedEventCaptor = ArgumentCaptor.forClass(ServiceEventBuilder.class);
|
||||
Mockito.verify(mockServiceEmitter).emit(emittedEventCaptor.capture());
|
||||
Assert.assertEquals(KillCompactionConfig.COUNT_METRIC, emittedEventCaptor.getValue().build(ImmutableMap.of()).toMap().get("metric"));
|
||||
Assert.assertEquals(0, emittedEventCaptor.getValue().build(ImmutableMap.of()).toMap().get("value"));
|
||||
Mockito.verify(mockJacksonConfigManager).watch(
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.empty())
|
||||
);
|
||||
Mockito.verifyNoMoreInteractions(mockJacksonConfigManager);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunRemoveInactiveDatasourceCompactionConfig()
|
||||
{
|
||||
String inactiveDatasourceName = "inactive_datasource";
|
||||
String activeDatasourceName = "active_datasource";
|
||||
DataSourceCompactionConfig inactiveDatasourceConfig = new DataSourceCompactionConfig(
|
||||
inactiveDatasourceName,
|
||||
null,
|
||||
500L,
|
||||
null,
|
||||
new Period(3600),
|
||||
null,
|
||||
new UserCompactionTaskGranularityConfig(Granularities.HOUR, null),
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
|
||||
DataSourceCompactionConfig activeDatasourceConfig = new DataSourceCompactionConfig(
|
||||
activeDatasourceName,
|
||||
null,
|
||||
500L,
|
||||
null,
|
||||
new Period(3600),
|
||||
null,
|
||||
new UserCompactionTaskGranularityConfig(Granularities.HOUR, null),
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig, activeDatasourceConfig));
|
||||
Mockito.when(mockDruidCoordinatorRuntimeParams.getEmitter()).thenReturn(mockServiceEmitter);
|
||||
Mockito.when(mockJacksonConfigManager.watch(
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()))
|
||||
).thenReturn(new AtomicReference<>(originalCurrentConfig));
|
||||
Mockito.when(mockSqlSegmentsMetadataManager.retrieveAllDataSourceNames()).thenReturn(ImmutableSet.of(activeDatasourceName));
|
||||
final ArgumentCaptor<CoordinatorCompactionConfig> oldConfigCaptor = ArgumentCaptor.forClass(CoordinatorCompactionConfig.class);
|
||||
final ArgumentCaptor<CoordinatorCompactionConfig> newConfigCaptor = ArgumentCaptor.forClass(CoordinatorCompactionConfig.class);
|
||||
Mockito.when(mockJacksonConfigManager.set(
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY),
|
||||
oldConfigCaptor.capture(),
|
||||
newConfigCaptor.capture(),
|
||||
ArgumentMatchers.any())
|
||||
).thenReturn(ConfigManager.SetResult.ok());
|
||||
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
killCompactionConfig = new KillCompactionConfig(druidCoordinatorConfig, mockSqlSegmentsMetadataManager, mockJacksonConfigManager);
|
||||
killCompactionConfig.run(mockDruidCoordinatorRuntimeParams);
|
||||
|
||||
// Verify and Assert
|
||||
Assert.assertNotNull(oldConfigCaptor.getValue());
|
||||
Assert.assertEquals(oldConfigCaptor.getValue(), originalCurrentConfig);
|
||||
Assert.assertNotNull(newConfigCaptor.getValue());
|
||||
// The updated config should only contains one compaction config for the active datasource
|
||||
Assert.assertEquals(1, newConfigCaptor.getValue().getCompactionConfigs().size());
|
||||
|
||||
Assert.assertEquals(activeDatasourceConfig, newConfigCaptor.getValue().getCompactionConfigs().get(0));
|
||||
final ArgumentCaptor<ServiceEventBuilder> emittedEventCaptor = ArgumentCaptor.forClass(ServiceEventBuilder.class);
|
||||
Mockito.verify(mockServiceEmitter).emit(emittedEventCaptor.capture());
|
||||
Assert.assertEquals(KillCompactionConfig.COUNT_METRIC, emittedEventCaptor.getValue().build(ImmutableMap.of()).toMap().get("metric"));
|
||||
// Should delete 1 config
|
||||
Assert.assertEquals(1, emittedEventCaptor.getValue().build(ImmutableMap.of()).toMap().get("value"));
|
||||
|
||||
Mockito.verify(mockJacksonConfigManager).watch(
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.empty())
|
||||
);
|
||||
Mockito.verify(mockJacksonConfigManager).set(
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY),
|
||||
ArgumentMatchers.any(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.any(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.any()
|
||||
);
|
||||
Mockito.verifyNoMoreInteractions(mockJacksonConfigManager);
|
||||
Mockito.verify(mockSqlSegmentsMetadataManager).retrieveAllDataSourceNames();
|
||||
Mockito.verifyNoMoreInteractions(mockSqlSegmentsMetadataManager);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRunRetryForRetryableException()
|
||||
{
|
||||
String inactiveDatasourceName = "inactive_datasource";
|
||||
DataSourceCompactionConfig inactiveDatasourceConfig = new DataSourceCompactionConfig(
|
||||
inactiveDatasourceName,
|
||||
null,
|
||||
500L,
|
||||
null,
|
||||
new Period(3600),
|
||||
null,
|
||||
new UserCompactionTaskGranularityConfig(Granularities.HOUR, null),
|
||||
null,
|
||||
ImmutableMap.of("key", "val")
|
||||
);
|
||||
|
||||
CoordinatorCompactionConfig originalCurrentConfig = CoordinatorCompactionConfig.from(ImmutableList.of(inactiveDatasourceConfig));
|
||||
Mockito.when(mockDruidCoordinatorRuntimeParams.getEmitter()).thenReturn(mockServiceEmitter);
|
||||
Mockito.when(mockJacksonConfigManager.watch(
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.empty()))
|
||||
).thenReturn(new AtomicReference<>(originalCurrentConfig));
|
||||
Mockito.when(mockSqlSegmentsMetadataManager.retrieveAllDataSourceNames()).thenReturn(ImmutableSet.of());
|
||||
Mockito.when(mockJacksonConfigManager.set(
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY),
|
||||
ArgumentMatchers.any(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.any(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.any())
|
||||
).thenAnswer(new Answer() {
|
||||
private int count = 0;
|
||||
@Override
|
||||
public Object answer(InvocationOnMock invocation)
|
||||
{
|
||||
if (count++ < 3) {
|
||||
// Return fail result with RetryableException the first three call to updated set
|
||||
return ConfigManager.SetResult.fail(new Exception(), true);
|
||||
} else {
|
||||
// Return success ok on the fourth call to set updated config
|
||||
return ConfigManager.SetResult.ok();
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
TestDruidCoordinatorConfig druidCoordinatorConfig = new TestDruidCoordinatorConfig(
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT5S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
killCompactionConfig = new KillCompactionConfig(druidCoordinatorConfig, mockSqlSegmentsMetadataManager, mockJacksonConfigManager);
|
||||
killCompactionConfig.run(mockDruidCoordinatorRuntimeParams);
|
||||
|
||||
// Verify and Assert
|
||||
final ArgumentCaptor<ServiceEventBuilder> emittedEventCaptor = ArgumentCaptor.forClass(ServiceEventBuilder.class);
|
||||
Mockito.verify(mockServiceEmitter).emit(emittedEventCaptor.capture());
|
||||
Assert.assertEquals(KillCompactionConfig.COUNT_METRIC, emittedEventCaptor.getValue().build(ImmutableMap.of()).toMap().get("metric"));
|
||||
// Should delete 1 config
|
||||
Assert.assertEquals(1, emittedEventCaptor.getValue().build(ImmutableMap.of()).toMap().get("value"));
|
||||
|
||||
// Should call watch (to refresh current compaction config) four times due to RetryableException when failed
|
||||
Mockito.verify(mockJacksonConfigManager, Mockito.times(4)).watch(
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.empty())
|
||||
);
|
||||
// Should call set (to try set new updated compaction config) four times due to RetryableException when failed
|
||||
Mockito.verify(mockJacksonConfigManager, Mockito.times(4)).set(
|
||||
ArgumentMatchers.eq(CoordinatorCompactionConfig.CONFIG_KEY),
|
||||
ArgumentMatchers.any(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.any(CoordinatorCompactionConfig.class),
|
||||
ArgumentMatchers.any()
|
||||
);
|
||||
Mockito.verifyNoMoreInteractions(mockJacksonConfigManager);
|
||||
// Should call retrieveAllDataSourceNames four times due to RetryableException when failed
|
||||
Mockito.verify(mockSqlSegmentsMetadataManager, Mockito.times(4)).retrieveAllDataSourceNames();
|
||||
Mockito.verifyNoMoreInteractions(mockSqlSegmentsMetadataManager);
|
||||
}
|
||||
}
|
|
@ -77,6 +77,7 @@ public class KillDatasourceMetadataTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration(Long.MAX_VALUE),
|
||||
new Duration("PT1S"),
|
||||
10,
|
||||
|
@ -107,6 +108,7 @@ public class KillDatasourceMetadataTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT1S"),
|
||||
10,
|
||||
|
@ -135,6 +137,7 @@ public class KillDatasourceMetadataTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT3S"),
|
||||
new Duration("PT1S"),
|
||||
10,
|
||||
|
@ -162,6 +165,7 @@ public class KillDatasourceMetadataTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT-1S"),
|
||||
10,
|
||||
|
@ -191,6 +195,7 @@ public class KillDatasourceMetadataTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT1S"),
|
||||
10,
|
||||
|
|
|
@ -73,6 +73,7 @@ public class KillRulesTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration(Long.MAX_VALUE),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
|
@ -101,6 +102,7 @@ public class KillRulesTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
|
@ -129,6 +131,7 @@ public class KillRulesTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT3S"),
|
||||
new Duration("PT1S"),
|
||||
null,
|
||||
|
@ -156,6 +159,7 @@ public class KillRulesTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
new Duration("PT6S"),
|
||||
new Duration("PT-1S"),
|
||||
null,
|
||||
|
|
|
@ -70,6 +70,7 @@ public class KillSupervisorsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
@ -98,6 +99,7 @@ public class KillSupervisorsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
@ -126,6 +128,7 @@ public class KillSupervisorsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
@ -153,6 +156,7 @@ public class KillSupervisorsTest
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
10,
|
||||
null
|
||||
);
|
||||
|
|
|
@ -117,6 +117,7 @@ public class KillUnusedSegmentsTest
|
|||
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.KillCompactionConfig;
|
||||
import org.apache.druid.server.coordinator.duty.KillDatasourceMetadata;
|
||||
import org.apache.druid.server.coordinator.duty.KillRules;
|
||||
import org.apache.druid.server.coordinator.duty.KillSupervisors;
|
||||
|
@ -274,6 +275,11 @@ public class CliCoordinator extends ServerRunnable
|
|||
Predicates.equalTo("true"),
|
||||
KillDatasourceMetadata.class
|
||||
);
|
||||
conditionalMetadataStoreManagementDutyMultibind.addConditionBinding(
|
||||
"druid.coordinator.kill.compaction.on",
|
||||
Predicates.equalTo("true"),
|
||||
KillCompactionConfig.class
|
||||
);
|
||||
|
||||
bindNodeRoleAndAnnouncer(
|
||||
binder,
|
||||
|
|
Loading…
Reference in New Issue