mirror of https://github.com/apache/druid.git
address code review
This commit is contained in:
parent
f9db3a1fc6
commit
10d45ab9cf
|
@ -19,8 +19,7 @@
|
||||||
|
|
||||||
package io.druid.server.coordinator.helper;
|
package io.druid.server.coordinator.helper;
|
||||||
|
|
||||||
import com.google.api.client.util.Maps;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.api.client.util.Sets;
|
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import io.druid.metadata.MetadataRuleManager;
|
import io.druid.metadata.MetadataRuleManager;
|
||||||
import io.druid.server.coordinator.CoordinatorStats;
|
import io.druid.server.coordinator.CoordinatorStats;
|
||||||
|
@ -33,14 +32,13 @@ import io.druid.timeline.DataSegment;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper
|
public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper
|
||||||
{
|
{
|
||||||
private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorRuleRunner.class);
|
private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorRuleRunner.class);
|
||||||
|
private static int MAX_MISSING_RULES = 10;
|
||||||
|
|
||||||
private final ReplicationThrottler replicatorThrottler;
|
private final ReplicationThrottler replicatorThrottler;
|
||||||
|
|
||||||
|
@ -92,7 +90,8 @@ public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper
|
||||||
DateTime now = new DateTime();
|
DateTime now = new DateTime();
|
||||||
MetadataRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager();
|
MetadataRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager();
|
||||||
|
|
||||||
final Map<String, Set<String>> missingRules = Maps.newHashMap();
|
final List<String> segmentsWithMissingRules = Lists.newArrayListWithCapacity(10);
|
||||||
|
int missingRules = 0;
|
||||||
for (DataSegment segment : paramsWithReplicationManager.getAvailableSegments()) {
|
for (DataSegment segment : paramsWithReplicationManager.getAvailableSegments()) {
|
||||||
List<Rule> rules = databaseRuleManager.getRulesWithDefault(segment.getDataSource());
|
List<Rule> rules = databaseRuleManager.getRulesWithDefault(segment.getDataSource());
|
||||||
boolean foundMatchingRule = false;
|
boolean foundMatchingRule = false;
|
||||||
|
@ -105,18 +104,17 @@ public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!foundMatchingRule) {
|
if (!foundMatchingRule) {
|
||||||
Set<String> missingSegments = missingRules.get(segment.getDataSource());
|
if (segmentsWithMissingRules.size() < MAX_MISSING_RULES) {
|
||||||
if (missingSegments == null) {
|
segmentsWithMissingRules.add(segment.getIdentifier());
|
||||||
missingSegments = Sets.newHashSet();
|
|
||||||
missingRules.put(segment.getDataSource(), missingSegments);
|
|
||||||
}
|
}
|
||||||
missingSegments.add(segment.getIdentifier());
|
missingRules++;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!missingRules.isEmpty()) {
|
if (!segmentsWithMissingRules.isEmpty()) {
|
||||||
log.makeAlert("Unable to find a matching rules!")
|
log.makeAlert("Unable to find matching rules!")
|
||||||
.addData("missingSegments", missingRules)
|
.addData("segmentsWithMissingRulesCount", missingRules)
|
||||||
|
.addData("segmentsWithMissingRules", segmentsWithMissingRules)
|
||||||
.emit();
|
.emit();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue