mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-06-28 09:28:55 -04:00
Allow field usage tracker to be concurrent accessed (#79088)
While Lucene readers are currently only sequentially accessed, we expect future usages (and custom plugins) to access this concurrently. Closes #78899
This commit is contained in:
parent
b50b81e974
commit
d29a8d3ec1
1 changed files with 14 additions and 12 deletions
|
@ -75,22 +75,24 @@ public class ShardFieldUsageTracker {
|
||||||
}
|
}
|
||||||
|
|
||||||
static class PerField {
|
static class PerField {
|
||||||
boolean terms;
|
// while these fields are currently only sequentially accessed, we expect concurrent access by future usages (and custom plugins)
|
||||||
boolean postings;
|
volatile boolean terms;
|
||||||
boolean termFrequencies;
|
volatile boolean postings;
|
||||||
boolean positions;
|
volatile boolean termFrequencies;
|
||||||
boolean offsets;
|
volatile boolean positions;
|
||||||
boolean docValues;
|
volatile boolean offsets;
|
||||||
boolean storedFields;
|
volatile boolean docValues;
|
||||||
boolean norms;
|
volatile boolean storedFields;
|
||||||
boolean payloads;
|
volatile boolean norms;
|
||||||
boolean termVectors;
|
volatile boolean payloads;
|
||||||
boolean points;
|
volatile boolean termVectors;
|
||||||
|
volatile boolean points;
|
||||||
}
|
}
|
||||||
|
|
||||||
public class FieldUsageStatsTrackingSession implements FieldUsageNotifier, Releasable {
|
public class FieldUsageStatsTrackingSession implements FieldUsageNotifier, Releasable {
|
||||||
|
|
||||||
private final Map<String, PerField> usages = new HashMap<>();
|
// while this map is currently only sequentially accessed, we expect future usages (and custom plugins) to access this concurrently
|
||||||
|
private final Map<String, PerField> usages = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() {
|
public void close() {
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue