22
22
import org .apache .logging .log4j .LogManager ;
23
23
import org .apache .logging .log4j .Logger ;
24
24
import org .apache .logging .log4j .message .ParameterizedMessage ;
25
- import org .apache . lucene .index .DirectoryReader ;
25
+ import org .elasticsearch . common . lucene .index .ElasticsearchDirectoryReader ;
26
26
import org .elasticsearch .common .unit .TimeValue ;
27
- import org .elasticsearch .index .engine .Engine ;
28
27
import org .elasticsearch .index .fielddata .IndexFieldData ;
29
28
import org .elasticsearch .index .fielddata .IndexFieldDataService ;
30
29
import org .elasticsearch .index .mapper .MappedFieldType ;
@@ -58,22 +57,22 @@ public final class IndexWarmer {
58
57
this .listeners = Collections .unmodifiableList (list );
59
58
}
60
59
61
- void warm (Engine . Searcher searcher , IndexShard shard , IndexSettings settings ) {
60
+ void warm (ElasticsearchDirectoryReader reader , IndexShard shard , IndexSettings settings ) {
62
61
if (shard .state () == IndexShardState .CLOSED ) {
63
62
return ;
64
63
}
65
64
if (settings .isWarmerEnabled () == false ) {
66
65
return ;
67
66
}
68
67
if (logger .isTraceEnabled ()) {
69
- logger .trace ("{} top warming [{}]" , shard .shardId (), searcher . reader () );
68
+ logger .trace ("{} top warming [{}]" , shard .shardId (), reader );
70
69
}
71
70
shard .warmerService ().onPreWarm ();
72
71
long time = System .nanoTime ();
73
72
final List <TerminationHandle > terminationHandles = new ArrayList <>();
74
73
// get a handle on pending tasks
75
74
for (final Listener listener : listeners ) {
76
- terminationHandles .add (listener .warmReader (shard , searcher ));
75
+ terminationHandles .add (listener .warmReader (shard , reader ));
77
76
}
78
77
// wait for termination
79
78
for (TerminationHandle terminationHandle : terminationHandles ) {
@@ -103,7 +102,7 @@ public interface TerminationHandle {
103
102
public interface Listener {
104
103
/** Queue tasks to warm-up the given segments and return handles that allow to wait for termination of the
105
104
* execution of those tasks. */
106
- TerminationHandle warmReader (IndexShard indexShard , Engine . Searcher searcher );
105
+ TerminationHandle warmReader (IndexShard indexShard , ElasticsearchDirectoryReader reader );
107
106
}
108
107
109
108
private static class FieldDataWarmer implements IndexWarmer .Listener {
@@ -117,7 +116,7 @@ private static class FieldDataWarmer implements IndexWarmer.Listener {
117
116
}
118
117
119
118
@ Override
120
- public TerminationHandle warmReader (final IndexShard indexShard , final Engine . Searcher searcher ) {
119
+ public TerminationHandle warmReader (final IndexShard indexShard , final ElasticsearchDirectoryReader reader ) {
121
120
final MapperService mapperService = indexShard .mapperService ();
122
121
final Map <String , MappedFieldType > warmUpGlobalOrdinals = new HashMap <>();
123
122
for (MappedFieldType fieldType : mapperService .fieldTypes ()) {
@@ -133,7 +132,6 @@ public TerminationHandle warmReader(final IndexShard indexShard, final Engine.Se
133
132
try {
134
133
final long start = System .nanoTime ();
135
134
IndexFieldData .Global ifd = indexFieldDataService .getForField (fieldType );
136
- DirectoryReader reader = searcher .getDirectoryReader ();
137
135
IndexFieldData <?> global = ifd .loadGlobal (reader );
138
136
if (reader .leaves ().isEmpty () == false ) {
139
137
global .load (reader .leaves ().get (0 ));
0 commit comments