|
18 | 18 |
|
19 | 19 | import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
|
20 | 20 |
|
| 21 | +import java.io.ByteArrayOutputStream; |
21 | 22 | import java.io.Closeable;
|
22 | 23 | import java.io.IOException;
|
23 | 24 | import java.io.PrintStream;
|
| 25 | +import java.nio.charset.StandardCharsets; |
24 | 26 | import java.nio.file.Path;
|
25 | 27 | import java.nio.file.Paths;
|
26 | 28 | import java.text.NumberFormat;
|
|
31 | 33 | import java.util.List;
|
32 | 34 | import java.util.Locale;
|
33 | 35 | import java.util.Map;
|
| 36 | +import java.util.concurrent.Callable; |
| 37 | +import java.util.concurrent.CompletableFuture; |
| 38 | +import java.util.concurrent.CompletionException; |
| 39 | +import java.util.concurrent.ExecutionException; |
34 | 40 | import java.util.concurrent.ExecutorService;
|
35 | 41 | import java.util.concurrent.Executors;
|
36 | 42 | import java.util.concurrent.TimeUnit;
|
| 43 | +import java.util.function.Supplier; |
37 | 44 | import org.apache.lucene.codecs.Codec;
|
38 | 45 | import org.apache.lucene.codecs.DocValuesProducer;
|
39 | 46 | import org.apache.lucene.codecs.NormsProducer;
|
|
80 | 87 | */
|
81 | 88 | public final class CheckIndex implements Closeable {
|
82 | 89 |
|
83 |
| - private static final int MAX_PER_SEGMENT_CONCURRENCY = 11; |
84 | 90 | private PrintStream infoStream;
|
85 | 91 | private Directory dir;
|
86 | 92 | private Lock writeLock;
|
@@ -521,22 +527,29 @@ public Status checkIndex() throws IOException {
|
521 | 527 | * quite a long time to run.
|
522 | 528 | */
|
523 | 529 | public Status checkIndex(List<String> onlySegments) throws IOException {
|
524 |
| - ExecutorService executorService = |
525 |
| - Executors.newFixedThreadPool(threadCount, new NamedThreadFactory("async-check-index")); |
| 530 | + ExecutorService executorService = null; |
| 531 | + |
| 532 | + if (threadCount > 0) { |
| 533 | + executorService = |
| 534 | + Executors.newFixedThreadPool(threadCount, new NamedThreadFactory("async-check-index")); |
| 535 | + } |
| 536 | + |
526 | 537 | msg(infoStream, "Checking index with async threadCount: " + threadCount);
|
527 | 538 | try {
|
528 | 539 | return checkIndex(onlySegments, executorService);
|
529 | 540 | } finally {
|
530 |
| - executorService.shutdown(); |
531 |
| - try { |
532 |
| - executorService.awaitTermination(5, TimeUnit.SECONDS); |
533 |
| - } catch (InterruptedException e) { |
534 |
| - msg( |
535 |
| - infoStream, |
536 |
| - "ERROR: Interrupted exception occurred when shutting down executor service"); |
537 |
| - if (infoStream != null) e.printStackTrace(infoStream); |
538 |
| - } finally { |
539 |
| - executorService.shutdownNow(); |
| 541 | + if (executorService != null) { |
| 542 | + executorService.shutdown(); |
| 543 | + try { |
| 544 | + executorService.awaitTermination(5, TimeUnit.SECONDS); |
| 545 | + } catch (InterruptedException e) { |
| 546 | + msg( |
| 547 | + infoStream, |
| 548 | + "ERROR: Interrupted exception occurred when shutting down executor service"); |
| 549 | + if (infoStream != null) e.printStackTrace(infoStream); |
| 550 | + } finally { |
| 551 | + executorService.shutdownNow(); |
| 552 | + } |
540 | 553 | }
|
541 | 554 | }
|
542 | 555 | }
|
@@ -667,34 +680,101 @@ public Status checkIndex(List<String> onlySegments, ExecutorService executorServ
|
667 | 680 | result.newSegments.clear();
|
668 | 681 | result.maxSegmentName = -1;
|
669 | 682 |
|
670 |
| - for (int i = 0; i < numSegments; i++) { |
671 |
| - final SegmentCommitInfo info = sis.info(i); |
672 |
| - long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX); |
673 |
| - if (segmentName > result.maxSegmentName) { |
674 |
| - result.maxSegmentName = segmentName; |
| 683 | + // checks segments sequentially |
| 684 | + if (executorService == null) { |
| 685 | + for (int i = 0; i < numSegments; i++) { |
| 686 | + final SegmentCommitInfo info = sis.info(i); |
| 687 | + updateMaxSegmentName(result, info); |
| 688 | + if (onlySegments != null && !onlySegments.contains(info.info.name)) { |
| 689 | + continue; |
| 690 | + } |
| 691 | + |
| 692 | + msg( |
| 693 | + infoStream, |
| 694 | + (1 + i) |
| 695 | + + " of " |
| 696 | + + numSegments |
| 697 | + + ": name=" |
| 698 | + + info.info.name |
| 699 | + + " maxDoc=" |
| 700 | + + info.info.maxDoc()); |
| 701 | + Status.SegmentInfoStatus segmentInfoStatus = testSegment(sis, info, infoStream); |
| 702 | + |
| 703 | + processSegmentInfoStatusResult(result, info, segmentInfoStatus); |
675 | 704 | }
|
676 |
| - if (onlySegments != null && !onlySegments.contains(info.info.name)) { |
677 |
| - continue; |
| 705 | + } else { |
| 706 | + List<ByteArrayOutputStream> outputs = new ArrayList<>(); |
| 707 | + List<CompletableFuture<Status.SegmentInfoStatus>> futures = new ArrayList<>(); |
| 708 | + |
| 709 | + // checks segments concurrently |
| 710 | + for (int i = 0; i < numSegments; i++) { |
| 711 | + final SegmentCommitInfo info = sis.info(i); |
| 712 | + updateMaxSegmentName(result, info); |
| 713 | + if (onlySegments != null && !onlySegments.contains(info.info.name)) { |
| 714 | + continue; |
| 715 | + } |
| 716 | + |
| 717 | + SegmentInfos finalSis = sis; |
| 718 | + |
| 719 | + ByteArrayOutputStream output = new ByteArrayOutputStream(); |
| 720 | + PrintStream stream; |
| 721 | + if (i > 0) { |
| 722 | + // buffer the messages for segment starting from the 2nd one so that they can later be |
| 723 | + // printed in order |
| 724 | + stream = new PrintStream(output, true, IOUtils.UTF_8); |
| 725 | + } else { |
| 726 | + // optimize for first segment to print real-time |
| 727 | + stream = infoStream; |
| 728 | + } |
| 729 | + msg( |
| 730 | + stream, |
| 731 | + (1 + i) |
| 732 | + + " of " |
| 733 | + + numSegments |
| 734 | + + ": name=" |
| 735 | + + info.info.name |
| 736 | + + " maxDoc=" |
| 737 | + + info.info.maxDoc()); |
| 738 | + |
| 739 | + outputs.add(output); |
| 740 | + futures.add( |
| 741 | + runAsyncSegmentCheck(() -> testSegment(finalSis, info, stream), executorService)); |
678 | 742 | }
|
679 |
| - msg( |
680 |
| - infoStream, |
681 |
| - (1 + i) |
682 |
| - + " of " |
683 |
| - + numSegments |
684 |
| - + ": name=" |
685 |
| - + info.info.name |
686 |
| - + " maxDoc=" |
687 |
| - + info.info.maxDoc()); |
688 |
| - |
689 |
| - Status.SegmentInfoStatus segmentInfoStatus = testSegment(sis, info, infoStream); |
690 |
| - |
691 |
| - result.segmentInfos.add(segmentInfoStatus); |
692 |
| - if (segmentInfoStatus.error != null) { |
693 |
| - result.totLoseDocCount += segmentInfoStatus.toLoseDocCount; |
694 |
| - result.numBadSegments++; |
695 |
| - } else { |
696 |
| - // Keeper |
697 |
| - result.newSegments.add(info.clone()); |
| 743 | + |
| 744 | + for (int i = 0; i < numSegments; i++) { |
| 745 | + SegmentCommitInfo info = sis.info(i); |
| 746 | + if (onlySegments != null && !onlySegments.contains(info.info.name)) { |
| 747 | + continue; |
| 748 | + } |
| 749 | + |
| 750 | + ByteArrayOutputStream output = outputs.get(i); |
| 751 | + |
| 752 | + // print segment results in order |
| 753 | + Status.SegmentInfoStatus segmentInfoStatus = null; |
| 754 | + try { |
| 755 | + segmentInfoStatus = futures.get(i).get(); |
| 756 | + } catch (InterruptedException e) { |
| 757 | + // the segment test output should come before interrupted exception message that follows, |
| 758 | + // hence it's not emitted from finally clause |
| 759 | + infoStream.println(output.toString(StandardCharsets.UTF_8).stripTrailing()); |
| 760 | + msg( |
| 761 | + infoStream, |
| 762 | + "ERROR: Interrupted exception occurred when getting segment check result for segment " |
| 763 | + + info.info.name); |
| 764 | + if (infoStream != null) e.printStackTrace(infoStream); |
| 765 | + } catch (ExecutionException e) { |
| 766 | + infoStream.println(output.toString(StandardCharsets.UTF_8).stripTrailing()); |
| 767 | + |
| 768 | + assert failFast; |
| 769 | + throw new CheckIndexException("Segment " + info.info.name + " check failed.", e); |
| 770 | + } |
| 771 | + |
| 772 | + if (i > 0) { |
| 773 | + // first segment output already printed by infoStream |
| 774 | + infoStream.println(output.toString(StandardCharsets.UTF_8).stripTrailing()); |
| 775 | + } |
| 776 | + |
| 777 | + processSegmentInfoStatusResult(result, info, segmentInfoStatus); |
698 | 778 | }
|
699 | 779 | }
|
700 | 780 |
|
@@ -731,6 +811,42 @@ public Status checkIndex(List<String> onlySegments, ExecutorService executorServ
|
731 | 811 | return result;
|
732 | 812 | }
|
733 | 813 |
|
| 814 | + private void updateMaxSegmentName(Status result, SegmentCommitInfo info) { |
| 815 | + long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX); |
| 816 | + if (segmentName > result.maxSegmentName) { |
| 817 | + result.maxSegmentName = segmentName; |
| 818 | + } |
| 819 | + } |
| 820 | + |
| 821 | + private void processSegmentInfoStatusResult( |
| 822 | + Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) { |
| 823 | + result.segmentInfos.add(segmentInfoStatus); |
| 824 | + if (segmentInfoStatus.error != null) { |
| 825 | + result.totLoseDocCount += segmentInfoStatus.toLoseDocCount; |
| 826 | + result.numBadSegments++; |
| 827 | + } else { |
| 828 | + // Keeper |
| 829 | + result.newSegments.add(info.clone()); |
| 830 | + } |
| 831 | + } |
| 832 | + |
| 833 | + private <R> CompletableFuture<R> runAsyncSegmentCheck( |
| 834 | + Callable<R> asyncCallable, ExecutorService executorService) { |
| 835 | + return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService); |
| 836 | + } |
| 837 | + |
| 838 | + private <T> Supplier<T> callableToSupplier(Callable<T> callable) { |
| 839 | + return () -> { |
| 840 | + try { |
| 841 | + return callable.call(); |
| 842 | + } catch (RuntimeException | Error e) { |
| 843 | + throw e; |
| 844 | + } catch (Throwable e) { |
| 845 | + throw new CompletionException(e); |
| 846 | + } |
| 847 | + }; |
| 848 | + } |
| 849 | + |
734 | 850 | private Status.SegmentInfoStatus testSegment(
|
735 | 851 | SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException {
|
736 | 852 | Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
|
@@ -923,8 +1039,6 @@ private Status.SegmentInfoStatus testSegment(
|
923 | 1039 | "Soft Deletes test failed", segInfoStat.softDeletesStatus.error);
|
924 | 1040 | }
|
925 | 1041 | }
|
926 |
| - |
927 |
| - msg(infoStream, ""); |
928 | 1042 | } catch (Throwable t) {
|
929 | 1043 | if (failFast) {
|
930 | 1044 | throw IOUtils.rethrowAlways(t);
|
@@ -3830,15 +3944,7 @@ public static Options parseOptions(String[] args) {
|
3830 | 3944 | throw new IllegalArgumentException("-threadCount requires a following number");
|
3831 | 3945 | }
|
3832 | 3946 | i++;
|
3833 |
| - int providedThreadCount = Integer.parseInt(args[i]); |
3834 |
| - // Current implementation supports up to 11 concurrent checks at any time, and no |
3835 |
| - // concurrency across segments. |
3836 |
| - // Capping the thread count to 11 to avoid unnecessary threads to be created. |
3837 |
| - if (providedThreadCount > MAX_PER_SEGMENT_CONCURRENCY) { |
3838 |
| - System.out.println( |
3839 |
| - "-threadCount currently only supports up to 11 threads. Value higher than that will be capped."); |
3840 |
| - } |
3841 |
| - opts.threadCount = Math.min(providedThreadCount, MAX_PER_SEGMENT_CONCURRENCY); |
| 3947 | + opts.threadCount = Integer.parseInt(args[i]); |
3842 | 3948 | } else {
|
3843 | 3949 | if (opts.indexPath != null) {
|
3844 | 3950 | throw new IllegalArgumentException("ERROR: unexpected extra argument '" + args[i] + "'");
|
@@ -3906,10 +4012,9 @@ public int doCheck(Options opts) throws IOException, InterruptedException {
|
3906 | 4012 | setDoSlowChecks(opts.doSlowChecks);
|
3907 | 4013 | setChecksumsOnly(opts.doChecksumsOnly);
|
3908 | 4014 | setInfoStream(opts.out, opts.verbose);
|
3909 |
| - // when threadCount was not provided via command line, don't overwrite the default |
3910 |
| - if (opts.threadCount > 0) { |
3911 |
| - setThreadCount(opts.threadCount); |
3912 |
| - } |
| 4015 | + // when threadCount was not provided via command line, override it with 0 to turn of concurrent |
| 4016 | + // check |
| 4017 | + setThreadCount(opts.threadCount); |
3913 | 4018 |
|
3914 | 4019 | Status result = checkIndex(opts.onlySegments);
|
3915 | 4020 |
|
|
0 commit comments