You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/08/25 03:54:20 UTC

[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments

zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r695371765



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -605,209 +680,103 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     result.newSegments.clear();
     result.maxSegmentName = -1;
 
-    for (int i = 0; i < numSegments; i++) {
-      final SegmentCommitInfo info = sis.info(i);
-      long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
-      if (segmentName > result.maxSegmentName) {
-        result.maxSegmentName = segmentName;
-      }
-      if (onlySegments != null && !onlySegments.contains(info.info.name)) {
-        continue;
-      }
-      Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
-      result.segmentInfos.add(segInfoStat);
-      msg(
-          infoStream,
-          "  "
-              + (1 + i)
-              + " of "
-              + numSegments
-              + ": name="
-              + info.info.name
-              + " maxDoc="
-              + info.info.maxDoc());
-      segInfoStat.name = info.info.name;
-      segInfoStat.maxDoc = info.info.maxDoc();
-
-      final Version version = info.info.getVersion();
-      if (info.info.maxDoc() <= 0) {
-        throw new RuntimeException("illegal number of documents: maxDoc=" + info.info.maxDoc());
-      }
-
-      int toLoseDocCount = info.info.maxDoc();
-
-      SegmentReader reader = null;
-
-      try {
-        msg(infoStream, "    version=" + (version == null ? "3.0" : version));
-        msg(infoStream, "    id=" + StringHelper.idToString(info.info.getId()));
-        final Codec codec = info.info.getCodec();
-        msg(infoStream, "    codec=" + codec);
-        segInfoStat.codec = codec;
-        msg(infoStream, "    compound=" + info.info.getUseCompoundFile());
-        segInfoStat.compound = info.info.getUseCompoundFile();
-        msg(infoStream, "    numFiles=" + info.files().size());
-        Sort indexSort = info.info.getIndexSort();
-        if (indexSort != null) {
-          msg(infoStream, "    sort=" + indexSort);
-        }
-        segInfoStat.numFiles = info.files().size();
-        segInfoStat.sizeMB = info.sizeInBytes() / (1024. * 1024.);
-        msg(infoStream, "    size (MB)=" + nf.format(segInfoStat.sizeMB));
-        Map<String, String> diagnostics = info.info.getDiagnostics();
-        segInfoStat.diagnostics = diagnostics;
-        if (diagnostics.size() > 0) {
-          msg(infoStream, "    diagnostics = " + diagnostics);
+    // checks segments sequentially
+    if (executorService == null) {
+      for (int i = 0; i < numSegments; i++) {
+        final SegmentCommitInfo info = sis.info(i);
+        updateMaxSegmentName(result, info);
+        if (onlySegments != null && !onlySegments.contains(info.info.name)) {
+          continue;
         }
 
-        if (!info.hasDeletions()) {
-          msg(infoStream, "    no deletions");
-          segInfoStat.hasDeletions = false;
-        } else {
-          msg(infoStream, "    has deletions [delGen=" + info.getDelGen() + "]");
-          segInfoStat.hasDeletions = true;
-          segInfoStat.deletionsGen = info.getDelGen();
-        }
-
-        long startOpenReaderNS = System.nanoTime();
-        if (infoStream != null) infoStream.print("    test: open reader.........");
-        reader = new SegmentReader(info, sis.getIndexCreatedVersionMajor(), IOContext.DEFAULT);
         msg(
             infoStream,
-            String.format(
-                Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime() - startOpenReaderNS)));
+            (1 + i)
+                + " of "
+                + numSegments
+                + ": name="
+                + info.info.name
+                + " maxDoc="
+                + info.info.maxDoc());
+        Status.SegmentInfoStatus segmentInfoStatus = testSegment(sis, info, infoStream);
+
+        processSegmentInfoStatusResult(result, info, segmentInfoStatus);
+      }
+    } else {
+      ByteArrayOutputStream[] outputs = new ByteArrayOutputStream[numSegments];
+      @SuppressWarnings({"unchecked", "rawtypes"})
+      CompletableFuture<Status.SegmentInfoStatus>[] futures = new CompletableFuture[numSegments];
+
+      // checks segments concurrently
+      for (int i = 0; i < numSegments; i++) {
+        final SegmentCommitInfo info = sis.info(i);
+        updateMaxSegmentName(result, info);
+        if (onlySegments != null && !onlySegments.contains(info.info.name)) {
+          continue;
+        }
 
-        segInfoStat.openReaderPassed = true;
+        SegmentInfos finalSis = sis;
 
-        long startIntegrityNS = System.nanoTime();
-        if (infoStream != null) infoStream.print("    test: check integrity.....");
-        reader.checkIntegrity();
+        ByteArrayOutputStream output = new ByteArrayOutputStream();
+        PrintStream stream;
+        if (i > 0) {
+          // buffer the messages for segment starting from the 2nd one so that they can later be
+          // printed in order
+          stream = new PrintStream(output, true, IOUtils.UTF_8);
+        } else {
+          // optimize for first segment to print real-time

Review comment:
       Oh sorry my comment above might be a bit misleading. What I meant was that the segments were sorted in increasing order by file size for print out purpose (so that the smaller ones finishing faster should be printed out earlier) https://github.com/apache/lucene/pull/128/commits/70dc71c41bfcc279c5a22dee61ab902202f3933d, but the larger segments will be kicked off earlier https://github.com/apache/lucene/pull/128/commits/817c050408449a4fdeae256ba37ebf3799d4cfa6 .




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org