mirror of https://github.com/apache/lucene.git
SOLR-6685: ConcurrentModificationException in Overseer Status API
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1636875 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
5567f629fb
commit
ba0c5980fe
|
@ -267,6 +267,8 @@ Bug Fixes
|
||||||
* SOLR-2927: Solr does not unregister all mbeans upon exception in constructor
|
* SOLR-2927: Solr does not unregister all mbeans upon exception in constructor
|
||||||
causing memory leaks. (tom liu, Sharath Babu, Cyrille Roy, shalin)
|
causing memory leaks. (tom liu, Sharath Babu, Cyrille Roy, shalin)
|
||||||
|
|
||||||
|
* SOLR-6685: ConcurrentModificationException in Overseer Status API. (shalin)
|
||||||
|
|
||||||
Optimizations
|
Optimizations
|
||||||
----------------------
|
----------------------
|
||||||
|
|
||||||
|
|
|
@ -40,6 +40,7 @@ import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
import java.util.Map.Entry;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
|
@ -1962,7 +1963,7 @@ public class Overseer implements Closeable {
|
||||||
public static class Stats {
|
public static class Stats {
|
||||||
static final int MAX_STORED_FAILURES = 10;
|
static final int MAX_STORED_FAILURES = 10;
|
||||||
|
|
||||||
final Map<String, Stat> stats = Collections.synchronizedMap(new HashMap<String, Stat>());
|
final Map<String, Stat> stats = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
public Map<String, Stat> getStats() {
|
public Map<String, Stat> getStats() {
|
||||||
return stats;
|
return stats;
|
||||||
|
@ -1980,19 +1981,16 @@ public class Overseer implements Closeable {
|
||||||
|
|
||||||
public void success(String operation) {
|
public void success(String operation) {
|
||||||
String op = operation.toLowerCase(Locale.ROOT);
|
String op = operation.toLowerCase(Locale.ROOT);
|
||||||
synchronized (stats) {
|
Stat stat = stats.get(op);
|
||||||
Stat stat = stats.get(op);
|
if (stat == null) {
|
||||||
if (stat == null) {
|
stat = new Stat();
|
||||||
stat = new Stat();
|
stats.put(op, stat);
|
||||||
stats.put(op, stat);
|
|
||||||
}
|
|
||||||
stat.success.incrementAndGet();
|
|
||||||
}
|
}
|
||||||
|
stat.success.incrementAndGet();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void error(String operation) {
|
public void error(String operation) {
|
||||||
String op = operation.toLowerCase(Locale.ROOT);
|
String op = operation.toLowerCase(Locale.ROOT);
|
||||||
synchronized (stats) {
|
|
||||||
Stat stat = stats.get(op);
|
Stat stat = stats.get(op);
|
||||||
if (stat == null) {
|
if (stat == null) {
|
||||||
stat = new Stat();
|
stat = new Stat();
|
||||||
|
@ -2000,26 +1998,20 @@ public class Overseer implements Closeable {
|
||||||
}
|
}
|
||||||
stat.errors.incrementAndGet();
|
stat.errors.incrementAndGet();
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
public TimerContext time(String operation) {
|
public TimerContext time(String operation) {
|
||||||
String op = operation.toLowerCase(Locale.ROOT);
|
String op = operation.toLowerCase(Locale.ROOT);
|
||||||
Stat stat;
|
Stat stat = stats.get(op);
|
||||||
synchronized (stats) {
|
|
||||||
stat = stats.get(op);
|
|
||||||
if (stat == null) {
|
if (stat == null) {
|
||||||
stat = new Stat();
|
stat = new Stat();
|
||||||
stats.put(op, stat);
|
stats.put(op, stat);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
return stat.requestTime.time();
|
return stat.requestTime.time();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void storeFailureDetails(String operation, ZkNodeProps request, SolrResponse resp) {
|
public void storeFailureDetails(String operation, ZkNodeProps request, SolrResponse resp) {
|
||||||
String op = operation.toLowerCase(Locale.ROOT);
|
String op = operation.toLowerCase(Locale.ROOT);
|
||||||
Stat stat ;
|
Stat stat = stats.get(op);
|
||||||
synchronized (stats) {
|
|
||||||
stat = stats.get(op);
|
|
||||||
if (stat == null) {
|
if (stat == null) {
|
||||||
stat = new Stat();
|
stat = new Stat();
|
||||||
stats.put(op, stat);
|
stats.put(op, stat);
|
||||||
|
@ -2032,7 +2024,6 @@ public class Overseer implements Closeable {
|
||||||
failedOps.addLast(new FailedOp(request, resp));
|
failedOps.addLast(new FailedOp(request, resp));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
public List<FailedOp> getFailureDetails(String operation) {
|
public List<FailedOp> getFailureDetails(String operation) {
|
||||||
Stat stat = stats.get(operation.toLowerCase(Locale.ROOT));
|
Stat stat = stats.get(operation.toLowerCase(Locale.ROOT));
|
||||||
|
|
Loading…
Reference in New Issue