-
Notifications
You must be signed in to change notification settings - Fork 608
/
ExportWriter.java
840 lines (762 loc) · 29.6 KB
/
ExportWriter.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.solr.handler.export;
import static java.util.Collections.singletonList;
import static java.util.Collections.singletonMap;
import java.io.Closeable;
import java.io.IOException;
import java.io.OutputStream;
import java.io.OutputStreamWriter;
import java.io.PrintWriter;
import java.lang.invoke.MethodHandles;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Map;
import java.util.TreeSet;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.BitSetIterator;
import org.apache.lucene.util.FixedBitSet;
import org.apache.solr.client.solrj.impl.BinaryResponseParser;
import org.apache.solr.client.solrj.io.Tuple;
import org.apache.solr.client.solrj.io.stream.StreamContext;
import org.apache.solr.client.solrj.io.stream.TupleStream;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParameter;
import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
import org.apache.solr.common.IteratorWriter;
import org.apache.solr.common.MapWriter;
import org.apache.solr.common.MapWriter.EntryWriter;
import org.apache.solr.common.PushWriter;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.common.params.StreamParams;
import org.apache.solr.common.util.JavaBinCodec;
import org.apache.solr.core.SolrCore;
import org.apache.solr.metrics.SolrMetricsContext;
import org.apache.solr.request.SolrQueryRequest;
import org.apache.solr.request.SolrRequestInfo;
import org.apache.solr.response.BinaryResponseWriter;
import org.apache.solr.response.JSONResponseWriter;
import org.apache.solr.response.QueryResponseWriter;
import org.apache.solr.response.SolrQueryResponse;
import org.apache.solr.schema.BoolField;
import org.apache.solr.schema.DateValueFieldType;
import org.apache.solr.schema.DoubleValueFieldType;
import org.apache.solr.schema.FieldType;
import org.apache.solr.schema.FloatValueFieldType;
import org.apache.solr.schema.IndexSchema;
import org.apache.solr.schema.IntValueFieldType;
import org.apache.solr.schema.LongValueFieldType;
import org.apache.solr.schema.SchemaField;
import org.apache.solr.schema.SortableTextField;
import org.apache.solr.schema.StrField;
import org.apache.solr.search.SolrIndexSearcher;
import org.apache.solr.search.SortSpec;
import org.apache.solr.search.SyntaxError;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Prepares and writes the documents requested by /export requests
*
* <p>{@link ExportWriter} gathers and sorts the documents for a core using "stream sorting".
*
* <p>Stream sorting works by repeatedly processing and modifying a bitmap of matching documents.
* Each pass over the bitmap identifies the smallest docs (default is {@link #DEFAULT_BATCH_SIZE})
* that haven't been sent yet and stores them in a Priority Queue. They are then exported (written
* across the wire) and marked as sent (unset in the bitmap). This process repeats until all
* matching documents have been sent.
*/
public class ExportWriter implements SolrCore.RawWriter, Closeable {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String BATCH_SIZE_PARAM = "batchSize";
public static final String QUEUE_SIZE_PARAM = "queueSize";
public static final int DEFAULT_BATCH_SIZE = 30000;
public static final int DEFAULT_QUEUE_SIZE = 150000;
private OutputStreamWriter respWriter;
final SolrQueryRequest req;
final SolrQueryResponse res;
final StreamContext initialStreamContext;
final SolrMetricsContext solrMetricsContext;
final String metricsPath;
// The batch size for the output writer thread.
final int batchSize;
// The max combined size of the segment level priority queues.
private int priorityQueueSize;
StreamExpression streamExpression;
StreamContext streamContext;
FieldWriter[] fieldWriters;
int totalHits = 0;
FixedBitSet[] sets = null;
PushWriter writer;
private String wt;
public ExportWriter(
SolrQueryRequest req,
SolrQueryResponse res,
String wt,
StreamContext initialStreamContext,
SolrMetricsContext solrMetricsContext,
String metricsPath) {
this.req = req;
this.res = res;
this.wt = wt;
this.initialStreamContext = initialStreamContext;
this.solrMetricsContext = solrMetricsContext;
this.metricsPath = metricsPath;
this.priorityQueueSize = req.getParams().getInt(QUEUE_SIZE_PARAM, DEFAULT_QUEUE_SIZE);
this.batchSize = DEFAULT_BATCH_SIZE;
}
@Override
public String getContentType() {
if ("javabin".equals(wt)) {
return BinaryResponseParser.BINARY_CONTENT_TYPE;
} else return "json";
}
@Override
public void close() throws IOException {
if (writer != null) {
try {
writer.close();
} catch (Throwable t) {
// We're going to sit on this.
}
}
if (respWriter != null) {
try {
respWriter.flush();
respWriter.close();
} catch (Throwable t) {
}
}
}
protected void writeException(Exception e, PushWriter w, boolean logException)
throws IOException {
w.writeMap(
mw -> {
mw.put("responseHeader", singletonMap("status", 400))
.put(
"response",
Map.of(
"numFound",
0,
"docs",
singletonList(singletonMap("EXCEPTION", e.getMessage()))));
});
if (logException) {
SolrException.log(log, e);
}
}
public void write(OutputStream os) throws IOException {
try {
_write(os);
} finally {
}
}
private void _write(OutputStream os) throws IOException {
QueryResponseWriter rw = req.getCore().getResponseWriters().get(wt);
if (rw instanceof BinaryResponseWriter) {
// todo add support for other writers after testing
writer = new JavaBinCodec(os, null);
} else {
respWriter = new OutputStreamWriter(os, StandardCharsets.UTF_8);
writer = JSONResponseWriter.getPushWriter(respWriter, req, res);
}
Exception exception = res.getException();
if (exception != null) {
if (!(exception instanceof IgnoreException)) {
writeException(exception, writer, false);
}
return;
}
SolrRequestInfo info = SolrRequestInfo.getRequestInfo();
SortSpec sortSpec = info.getResponseBuilder().getSortSpec();
if (sortSpec == null) {
writeException(
(new IOException(new SyntaxError("No sort criteria was provided."))), writer, true);
return;
}
SolrIndexSearcher searcher = req.getSearcher();
Sort sort = searcher.weightSort(sortSpec.getSort());
if (sort == null) {
writeException(
(new IOException(new SyntaxError("No sort criteria was provided."))), writer, true);
return;
}
if (sort != null && sort.needsScores()) {
writeException(
(new IOException(new SyntaxError("Scoring is not currently supported with xsort."))),
writer,
true);
return;
}
// There is a bailout in SolrIndexSearcher.getDocListNC when there are _no_ docs in the index at
// all.
// if (lastDocRequested <= 0) {
// That causes the totalHits and export entries in the context to _not_ get set.
// The only time that really matters is when we search against an _empty_ set. That's too
// obscure a condition to handle as part of this patch, if someone wants to pursue it can be
// reproduced with:
// ant test -Dtestcase=StreamingTest -Dtests.method=testAllValidExportTypes
// -Dtests.seed=10F13879D0D1D6AD -Dtests.locale=es-PA
// -Dtests.timezone=America/Bahia_Banderas -Dtests.asserts=true -Dtests.file.encoding=ISO-8859-1
// You'll have to uncomment the if below to hit the null pointer exception.
// This is such an unusual case (i.e. an empty index) that catching this concdition here is
// probably OK.
// This came to light in the very artifical case of indexing a single doc to Cloud.
if (req.getContext().get("totalHits") != null) {
totalHits = ((Integer) req.getContext().get("totalHits")).intValue();
sets = (FixedBitSet[]) req.getContext().get("export");
if (sets == null) {
writeException(
(new IOException(
new SyntaxError("xport RankQuery is required for xsort: rq={!xport}"))),
writer,
true);
return;
}
}
SolrParams params = req.getParams();
String fl = params.get("fl");
String[] fields = null;
if (fl == null) {
writeException(
(new IOException(new SyntaxError("export field list (fl) must be specified."))),
writer,
true);
return;
} else {
fields = fl.split(",");
for (int i = 0; i < fields.length; i++) {
fields[i] = fields[i].trim();
if (fields[i].equals("score")) {
writeException(
(new IOException(new SyntaxError("Scoring is not currently supported with xsort."))),
writer,
true);
return;
}
}
}
try {
fieldWriters = getFieldWriters(fields, req.getSearcher());
} catch (Exception e) {
writeException(e, writer, true);
return;
}
String expr = params.get(StreamParams.EXPR);
if (expr != null) {
StreamFactory streamFactory = initialStreamContext.getStreamFactory();
streamFactory.withDefaultSort(params.get(CommonParams.SORT));
try {
StreamExpression expression = StreamExpressionParser.parse(expr);
if (streamFactory.isEvaluator(expression)) {
streamExpression = new StreamExpression(StreamParams.TUPLE);
streamExpression.addParameter(
new StreamExpressionNamedParameter(StreamParams.RETURN_VALUE, expression));
} else {
streamExpression = expression;
}
} catch (Exception e) {
writeException(e, writer, true);
return;
}
streamContext = new StreamContext();
streamContext.setRequestParams(params);
streamContext.setLocal(true);
streamContext.workerID = 0;
streamContext.numWorkers = 1;
streamContext.setSolrClientCache(initialStreamContext.getSolrClientCache());
streamContext.setModelCache(initialStreamContext.getModelCache());
streamContext.setObjectCache(initialStreamContext.getObjectCache());
streamContext.put("core", req.getCore().getName());
streamContext.put("solr-core", req.getCore());
streamContext.put(CommonParams.SORT, params.get(CommonParams.SORT));
}
try {
writer.writeMap(
m -> {
m.put("responseHeader", singletonMap("status", 0));
m.put(
"response",
(MapWriter)
mw -> {
mw.put("numFound", totalHits);
mw.put("docs", (IteratorWriter) iw -> writeDocs(req, os, iw, sort));
});
});
} catch (java.io.EOFException e) {
log.info("Caught Eof likely caused by early client disconnect");
}
if (streamContext != null) {
streamContext = null;
}
}
private TupleStream createTupleStream() throws IOException {
StreamFactory streamFactory = (StreamFactory) initialStreamContext.getStreamFactory().clone();
// Set the sort in the stream factory so it can be used during initialization.
streamFactory.withDefaultSort(((String) streamContext.get(CommonParams.SORT)));
TupleStream tupleStream = streamFactory.constructStream(streamExpression);
tupleStream.setStreamContext(streamContext);
return tupleStream;
}
private void transferBatchToBufferForOutput(
MergeIterator mergeIterator, ExportBuffers.Buffer destination) throws IOException {
try {
int outDocsIndex = -1;
for (int i = 0; i < batchSize; i++) {
SortDoc sortDoc = mergeIterator.next();
if (sortDoc != null) {
destination.outDocs[++outDocsIndex].setValues(sortDoc);
} else {
break;
}
}
destination.outDocsIndex = outDocsIndex;
} catch (Throwable t) {
log.error("transfer", t);
if (t instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
throw t;
} finally {
}
}
protected void writeDocs(
SolrQueryRequest req, OutputStream os, IteratorWriter.ItemWriter writer, Sort sort)
throws IOException {
List<LeafReaderContext> leaves = req.getSearcher().getTopReaderContext().leaves();
final int queueSize = Math.min(batchSize, totalHits);
ExportBuffers buffers =
new ExportBuffers(
this, leaves, req.getSearcher(), os, writer, sort, queueSize, totalHits, sets);
if (streamExpression != null) {
streamContext.put(ExportBuffers.EXPORT_BUFFERS_KEY, buffers);
final TupleStream tupleStream;
try {
tupleStream = createTupleStream();
tupleStream.open();
} catch (Exception e) {
buffers.getWriter().add((MapWriter) ew -> Tuple.EXCEPTION(e, true).writeMap(ew));
return;
}
buffers.run(
() -> {
for (; ; ) {
if (Thread.currentThread().isInterrupted()) {
break;
}
final Tuple t;
try {
t = tupleStream.read();
} catch (final Exception e) {
buffers.getWriter().add((MapWriter) ew -> Tuple.EXCEPTION(e, true).writeMap(ew));
break;
}
if (t == null) {
break;
}
if (t.EOF && !t.EXCEPTION) {
break;
}
// use decorated writer to monitor the number of output writes
// and flush the output quickly in case of very few (reduced) output items
buffers.getWriter().add((MapWriter) ew -> t.writeMap(ew));
if (t.EXCEPTION && t.EOF) {
break;
}
}
return true;
});
tupleStream.close();
} else {
buffers.run(
() -> {
// get the initial buffer
log.debug("--- writer init exchanging from empty");
buffers.exchangeBuffers();
ExportBuffers.Buffer buffer = buffers.getOutputBuffer();
log.debug("--- writer init got {}", buffer);
while (buffer.outDocsIndex != ExportBuffers.Buffer.NO_MORE_DOCS) {
if (Thread.currentThread().isInterrupted()) {
log.debug("--- writer interrupted");
break;
}
try {
for (int i = 0; i <= buffer.outDocsIndex; ++i) {
// we're using the raw writer here because there's no potential
// reduction in the number of output items, unlike when using
// streaming expressions
final SortDoc currentDoc = buffer.outDocs[i];
writer.add((MapWriter) ew -> writeDoc(currentDoc, leaves, ew, fieldWriters));
}
} finally {
}
// log.debug("--- writer exchanging from {}", buffer);
try {
long startExchangeBuffers = System.nanoTime();
buffers.exchangeBuffers();
long endExchangeBuffers = System.nanoTime();
if (log.isDebugEnabled()) {
log.debug(
"Waited for reader thread {}:",
Long.toString(((endExchangeBuffers - startExchangeBuffers) / 1000000)));
}
} finally {
}
buffer = buffers.getOutputBuffer();
// log.debug("--- writer got {}", buffer);
}
return true;
});
}
}
void fillOutDocs(MergeIterator mergeIterator, ExportBuffers.Buffer buffer) throws IOException {
transferBatchToBufferForOutput(mergeIterator, buffer);
}
void writeDoc(
SortDoc sortDoc, List<LeafReaderContext> leaves, EntryWriter ew, FieldWriter[] writers)
throws IOException {
int ord = sortDoc.ord;
LeafReaderContext context = leaves.get(ord);
int fieldIndex = 0;
for (FieldWriter fieldWriter : writers) {
if (fieldWriter.write(sortDoc, context, ew, fieldIndex)) {
++fieldIndex;
}
}
}
public FieldWriter[] getFieldWriters(String[] fields, SolrIndexSearcher searcher)
throws IOException {
IndexSchema schema = searcher.getSchema();
FieldWriter[] writers = new FieldWriter[fields.length];
for (int i = 0; i < fields.length; i++) {
String field = fields[i];
SchemaField schemaField = null;
try {
schemaField = schema.getField(field);
} catch (Exception e) {
throw new IOException(e);
}
if (!schemaField.hasDocValues()) {
throw new IOException(schemaField + " must have DocValues to use this feature.");
}
boolean multiValued = schemaField.multiValued();
FieldType fieldType = schemaField.getType();
if (fieldType instanceof SortableTextField && schemaField.useDocValuesAsStored() == false) {
throw new IOException(
schemaField + " Must have useDocValuesAsStored='true' to be used with export writer");
}
if (fieldType instanceof IntValueFieldType) {
if (multiValued) {
writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
} else {
writers[i] = new IntFieldWriter(field);
}
} else if (fieldType instanceof LongValueFieldType) {
if (multiValued) {
writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
} else {
writers[i] = new LongFieldWriter(field);
}
} else if (fieldType instanceof FloatValueFieldType) {
if (multiValued) {
writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
} else {
writers[i] = new FloatFieldWriter(field);
}
} else if (fieldType instanceof DoubleValueFieldType) {
if (multiValued) {
writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
} else {
writers[i] = new DoubleFieldWriter(field);
}
} else if (fieldType instanceof StrField || fieldType instanceof SortableTextField) {
if (multiValued) {
writers[i] = new MultiFieldWriter(field, fieldType, schemaField, false);
} else {
writers[i] = new StringFieldWriter(field, fieldType);
}
} else if (fieldType instanceof DateValueFieldType) {
if (multiValued) {
writers[i] = new MultiFieldWriter(field, fieldType, schemaField, false);
} else {
writers[i] = new DateFieldWriter(field);
}
} else if (fieldType instanceof BoolField) {
if (multiValued) {
writers[i] = new MultiFieldWriter(field, fieldType, schemaField, true);
} else {
writers[i] = new BoolFieldWriter(field, fieldType);
}
} else {
throw new IOException(
"Export fields must be one of the following types: int,float,long,double,string,date,boolean,SortableText");
}
}
return writers;
}
SortDoc getSortDoc(SolrIndexSearcher searcher, SortField[] sortFields) throws IOException {
SortValue[] sortValues = new SortValue[sortFields.length];
IndexSchema schema = searcher.getSchema();
for (int i = 0; i < sortFields.length; ++i) {
SortField sf = sortFields[i];
String field = sf.getField();
boolean reverse = sf.getReverse();
SchemaField schemaField = schema.getField(field);
FieldType ft = schemaField.getType();
if (!schemaField.hasDocValues()) {
throw new IOException(field + " must have DocValues to use this feature.");
}
if (ft instanceof SortableTextField && schemaField.useDocValuesAsStored() == false) {
throw new IOException(
schemaField + " Must have useDocValuesAsStored='true' to be used with export writer");
}
if (ft instanceof IntValueFieldType) {
if (reverse) {
sortValues[i] = new IntValue(field, new IntComp.IntDesc());
} else {
sortValues[i] = new IntValue(field, new IntComp.IntAsc());
}
} else if (ft instanceof FloatValueFieldType) {
if (reverse) {
sortValues[i] = new FloatValue(field, new FloatComp.FloatDesc());
} else {
sortValues[i] = new FloatValue(field, new FloatComp.FloatAsc());
}
} else if (ft instanceof DoubleValueFieldType) {
if (reverse) {
sortValues[i] = new DoubleValue(field, new DoubleComp.DoubleDesc());
} else {
sortValues[i] = new DoubleValue(field, new DoubleComp.DoubleAsc());
}
} else if (ft instanceof LongValueFieldType) {
if (reverse) {
sortValues[i] = new LongValue(field, new LongComp.LongDesc());
} else {
sortValues[i] = new LongValue(field, new LongComp.LongAsc());
}
} else if (ft instanceof StrField || ft instanceof SortableTextField) {
LeafReader reader = searcher.getSlowAtomicReader();
SortedDocValues vals = reader.getSortedDocValues(field);
if (reverse) {
sortValues[i] = new StringValue(vals, field, new IntComp.IntDesc());
} else {
sortValues[i] = new StringValue(vals, field, new IntComp.IntAsc());
}
} else if (ft instanceof DateValueFieldType) {
if (reverse) {
sortValues[i] = new LongValue(field, new LongComp.LongDesc());
} else {
sortValues[i] = new LongValue(field, new LongComp.LongAsc());
}
} else if (ft instanceof BoolField) {
// This is a bit of a hack, but since the boolean field stores ByteRefs, just like Strings
// _and_ since "F" happens to sort before "T" (thus false sorts "less" than true)
// we can just use the existing StringValue here.
LeafReader reader = searcher.getSlowAtomicReader();
SortedDocValues vals = reader.getSortedDocValues(field);
if (reverse) {
sortValues[i] = new StringValue(vals, field, new IntComp.IntDesc());
} else {
sortValues[i] = new StringValue(vals, field, new IntComp.IntAsc());
}
} else {
throw new IOException(
"Sort fields must be one of the following types: int,float,long,double,string,date,boolean,SortableText");
}
}
// SingleValueSortDoc etc are specialized classes which don't have array lookups. On
// benchmarking large datasets, this is faster than the using an array in SortDoc. So upto 4
// sort fields we still want to keep specialized classes. SOLR-12616 has more details
if (sortValues.length == 1) {
return new SingleValueSortDoc(sortValues[0]);
} else if (sortValues.length == 2) {
return new DoubleValueSortDoc(sortValues[0], sortValues[1]);
} else if (sortValues.length == 3) {
return new TripleValueSortDoc(sortValues[0], sortValues[1], sortValues[2]);
} else if (sortValues.length == 4) {
return new QuadValueSortDoc(sortValues[0], sortValues[1], sortValues[2], sortValues[3]);
}
return new SortDoc(sortValues);
}
static class MergeIterator {
private TreeSet<SortDoc> set = new TreeSet<>();
private SegmentIterator[] segmentIterators;
private SortDoc outDoc;
public MergeIterator(SegmentIterator[] segmentIterators, SortDoc proto) throws IOException {
outDoc = proto.copy();
this.segmentIterators = segmentIterators;
for (int i = 0; i < segmentIterators.length; i++) {
try {
SortDoc sortDoc = segmentIterators[i].next();
if (sortDoc != null) {
set.add(sortDoc);
}
} catch (IOException e) {
log.error("Error in MergeIterator: ", e);
throw e;
}
}
}
/*
* Merge sorts the SortDocs from Segment Iterators
* Returns null when all docs are iterated.
*/
public SortDoc next() throws IOException {
SortDoc sortDoc = set.pollLast();
// We've exhausted all documents
if (sortDoc == null) {
return null;
} else {
outDoc.setValues(sortDoc);
}
SortDoc nextDoc = segmentIterators[sortDoc.ord].next();
if (nextDoc != null) {
// The entire expense of the operation is here
set.add(nextDoc);
}
return outDoc;
}
}
public MergeIterator getMergeIterator(
List<LeafReaderContext> leaves, FixedBitSet[] bits, SortDoc sortDoc) throws IOException {
try {
long totalDocs = 0;
for (int i = 0; i < leaves.size(); i++) {
totalDocs += leaves.get(i).reader().maxDoc();
}
// Resize the priorityQueueSize down for small result sets.
this.priorityQueueSize = Math.min(this.priorityQueueSize, (int) (this.totalHits * 1.2));
if (log.isDebugEnabled()) {
log.debug("Total priority queue size {}:", this.priorityQueueSize);
}
int[] sizes = new int[leaves.size()];
int combineQueueSize = 0;
for (int i = 0; i < leaves.size(); i++) {
long maxDoc = leaves.get(i).reader().maxDoc();
int sortQueueSize =
Math.min(
(int) (((double) maxDoc / (double) totalDocs) * this.priorityQueueSize), batchSize);
// Protect against too small a queue size as well
if (sortQueueSize < 10) {
sortQueueSize = 10;
}
if (log.isDebugEnabled()) {
log.debug("Segment priority queue size {}:", sortQueueSize);
}
sizes[i] = sortQueueSize;
combineQueueSize += sortQueueSize;
}
if (log.isDebugEnabled()) {
log.debug("Combined priority queue size {}:", combineQueueSize);
}
SegmentIterator[] segmentIterators = new SegmentIterator[leaves.size()];
for (int i = 0; i < segmentIterators.length; i++) {
SortQueue sortQueue = new SortQueue(sizes[i], sortDoc.copy());
segmentIterators[i] =
new SegmentIterator(bits[i], leaves.get(i), sortQueue, sortDoc.copy());
}
return new MergeIterator(segmentIterators, sortDoc);
} finally {
}
}
private static class SegmentIterator {
private final FixedBitSet bits;
private final SortQueue queue;
private final SortDoc sortDoc;
private final LeafReaderContext context;
private final SortDoc[] outDocs;
private SortDoc nextDoc;
private int index;
public SegmentIterator(
FixedBitSet bits, LeafReaderContext context, SortQueue sortQueue, SortDoc sortDoc)
throws IOException {
this.bits = bits;
this.queue = sortQueue;
this.sortDoc = sortDoc;
this.nextDoc = sortDoc.copy();
this.context = context;
this.outDocs = new SortDoc[sortQueue.maxSize];
topDocs();
}
public SortDoc next() throws IOException {
SortDoc _sortDoc = null;
if (index > -1) {
_sortDoc = outDocs[index--];
} else {
topDocs();
if (index > -1) {
_sortDoc = outDocs[index--];
}
}
if (_sortDoc != null) {
// Clear the bit so it's not loaded again.
bits.clear(_sortDoc.docId);
// Load the global ordinal (only matters for strings)
_sortDoc.setGlobalValues(nextDoc);
nextDoc.setValues(_sortDoc);
// We are now done with this doc.
_sortDoc.reset();
} else {
nextDoc = null;
}
return nextDoc;
}
private void topDocs() throws IOException {
try {
queue.reset();
SortDoc top = queue.top();
this.sortDoc.setNextReader(context);
DocIdSetIterator it = new BitSetIterator(bits, 0); // cost is not useful here
int docId;
while ((docId = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
this.sortDoc.setValues(docId);
if (top.lessThan(this.sortDoc)) {
top.setValues(this.sortDoc);
top = queue.updateTop();
}
}
// Pop the queue and load up the array.
index = -1;
SortDoc _sortDoc;
while ((_sortDoc = queue.pop()) != null) {
if (_sortDoc.docId > -1) {
outDocs[++index] = _sortDoc;
}
}
} catch (Exception e) {
log.error("Segment Iterator Error:", e);
throw new IOException(e);
} finally {
}
}
}
public static class IgnoreException extends IOException {
public void printStackTrace(PrintWriter pw) {
pw.print("Early Client Disconnect");
}
public String getMessage() {
return "Early Client Disconnect";
}
}
}