mirror of https://github.com/apache/druid.git
Not use ConcurrentHashMap in CoordinatorRuleManager.rules (#9302)
This commit is contained in:
parent
54bc4dc71b
commit
98cefc61fa
|
@ -41,10 +41,9 @@ import org.jboss.netty.handler.codec.http.HttpResponseStatus;
|
|||
import org.joda.time.Duration;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
|
@ -58,7 +57,7 @@ public class CoordinatorRuleManager
|
|||
private final ObjectMapper jsonMapper;
|
||||
private final Supplier<TieredBrokerConfig> config;
|
||||
|
||||
private final AtomicReference<ConcurrentHashMap<String, List<Rule>>> rules;
|
||||
private final AtomicReference<Map<String, List<Rule>>> rules;
|
||||
|
||||
private final DruidLeaderClient druidLeaderClient;
|
||||
|
||||
|
@ -80,7 +79,7 @@ public class CoordinatorRuleManager
|
|||
this.druidLeaderClient = druidLeaderClient;
|
||||
|
||||
this.rules = new AtomicReference<>(
|
||||
new ConcurrentHashMap<String, List<Rule>>()
|
||||
Collections.emptyMap()
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -120,7 +119,7 @@ public class CoordinatorRuleManager
|
|||
return;
|
||||
}
|
||||
|
||||
rules.set(new ConcurrentHashMap<String, List<Rule>>());
|
||||
rules.set(Collections.emptyMap());
|
||||
|
||||
started = false;
|
||||
exec.shutdownNow();
|
||||
|
@ -149,17 +148,13 @@ public class CoordinatorRuleManager
|
|||
);
|
||||
}
|
||||
|
||||
ConcurrentHashMap<String, List<Rule>> newRules = new ConcurrentHashMap<>(
|
||||
(Map<String, List<Rule>>) jsonMapper.readValue(
|
||||
rules.set(
|
||||
Collections.unmodifiableMap(jsonMapper.readValue(
|
||||
response.getContent(), new TypeReference<Map<String, List<Rule>>>()
|
||||
{
|
||||
}
|
||||
)
|
||||
))
|
||||
);
|
||||
|
||||
log.debug("Got [%,d] rules", newRules.size());
|
||||
|
||||
rules.set(newRules);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(e, "Exception while polling for rules");
|
||||
|
@ -169,7 +164,7 @@ public class CoordinatorRuleManager
|
|||
public List<Rule> getRulesWithDefault(final String dataSource)
|
||||
{
|
||||
List<Rule> rulesWithDefault = new ArrayList<>();
|
||||
ConcurrentMap<String, List<Rule>> theRules = rules.get();
|
||||
Map<String, List<Rule>> theRules = rules.get();
|
||||
List<Rule> dataSourceRules = theRules.get(dataSource);
|
||||
if (dataSourceRules != null) {
|
||||
rulesWithDefault.addAll(dataSourceRules);
|
||||
|
|
Loading…
Reference in New Issue