From 10d45ab9cf4925fefb7b03ffee76714c42db8af1 Mon Sep 17 00:00:00 2001 From: fjy Date: Fri, 21 Nov 2014 16:48:45 -0800 Subject: [PATCH] address code review --- .../helper/DruidCoordinatorRuleRunner.java | 24 +++++++++---------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java index dfe62334eeb..387ce89d09b 100644 --- a/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java +++ b/server/src/main/java/io/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java @@ -19,8 +19,7 @@ package io.druid.server.coordinator.helper; -import com.google.api.client.util.Maps; -import com.google.api.client.util.Sets; +import com.google.common.collect.Lists; import com.metamx.emitter.EmittingLogger; import io.druid.metadata.MetadataRuleManager; import io.druid.server.coordinator.CoordinatorStats; @@ -33,14 +32,13 @@ import io.druid.timeline.DataSegment; import org.joda.time.DateTime; import java.util.List; -import java.util.Map; -import java.util.Set; /** */ public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper { private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorRuleRunner.class); + private static int MAX_MISSING_RULES = 10; private final ReplicationThrottler replicatorThrottler; @@ -92,7 +90,8 @@ public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper DateTime now = new DateTime(); MetadataRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager(); - final Map> missingRules = Maps.newHashMap(); + final List segmentsWithMissingRules = Lists.newArrayListWithCapacity(10); + int missingRules = 0; for (DataSegment segment : paramsWithReplicationManager.getAvailableSegments()) { List rules = databaseRuleManager.getRulesWithDefault(segment.getDataSource()); boolean foundMatchingRule = false; @@ -105,18 +104,17 @@ public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper } if (!foundMatchingRule) { - Set missingSegments = missingRules.get(segment.getDataSource()); - if (missingSegments == null) { - missingSegments = Sets.newHashSet(); - missingRules.put(segment.getDataSource(), missingSegments); + if (segmentsWithMissingRules.size() < MAX_MISSING_RULES) { + segmentsWithMissingRules.add(segment.getIdentifier()); } - missingSegments.add(segment.getIdentifier()); + missingRules++; } } - if (!missingRules.isEmpty()) { - log.makeAlert("Unable to find a matching rules!") - .addData("missingSegments", missingRules) + if (!segmentsWithMissingRules.isEmpty()) { + log.makeAlert("Unable to find matching rules!") + .addData("segmentsWithMissingRulesCount", missingRules) + .addData("segmentsWithMissingRules", segmentsWithMissingRules) .emit(); }