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 {
|
||||
boolean terms;
|
||||
boolean postings;
|
||||
boolean termFrequencies;
|
||||
boolean positions;
|
||||
boolean offsets;
|
||||
boolean docValues;
|
||||
boolean storedFields;
|
||||
boolean norms;
|
||||
boolean payloads;
|
||||
boolean termVectors;
|
||||
boolean points;
|
||||
// while these fields are currently only sequentially accessed, we expect concurrent access by future usages (and custom plugins)
|
||||
volatile boolean terms;
|
||||
volatile boolean postings;
|
||||
volatile boolean termFrequencies;
|
||||
volatile boolean positions;
|
||||
volatile boolean offsets;
|
||||
volatile boolean docValues;
|
||||
volatile boolean storedFields;
|
||||
volatile boolean norms;
|
||||
volatile boolean payloads;
|
||||
volatile boolean termVectors;
|
||||
volatile boolean points;
|
||||
}
|
||||
|
||||
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
|
||||
public void close() {
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue