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/05/06 06:15:47 UTC

[GitHub] [lucene] zacharymorn opened a new pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

zacharymorn opened a new pull request #128:
URL: https://github.com/apache/lucene/pull/128


   # Description
   **This PR is currently WIP**
   
   Parallelize CheckIndex code
   
   commit 1: parallelize checking each index part within each segment 
   
   # Solution
   
   Please provide a short description of the approach taken to implement your solution.
   
   # Tests
   
   Passed ` ./gradlew clean; ./gradlew check -Ptests.nightly=true -Pvalidation.git.failOnModified=false` without nocommit comment
   
   


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r694930138



##########
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:
       > I've implemented the above by sorting segments by the file size they contain in increasing order.
   
   Hmm, shouldn't we sort them by decreasing size, so that the biggest segments get the most time to check concurrently?  I.e. the biggest segments will take the longest to check so I think we want them to kick off first?




-- 
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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-907952441


   > Thanks @zacharymorn -- the changes look awesome! I'm looking forward to faster `CheckIndex`!
   
   Thanks @mikemccand again for the review and approval, and I look forward to the speedup as well! I'll merge in a few days just in case if anyone else may have further comment.


-- 
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


[GitHub] [lucene] rmuir commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-850340290


   I'm gonna throw out the crazy idea to make `-fast` the new default. The previous `-slow` could be moved to `-slower` and the previous current behavior could be activated by `-slow`.
   
   I think the tool's defaults are unnecessarily slow just for historical reasons? (not having checksums originally)


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851538612


   > To get the best speedup, even at -slow, we should do concurrency both ways, and then sort those tasks by decreasing expected cost. 
   
   This fine-grained concurrency (one task for "part X segment") would then mean we can get the postings check of the big segments all going, early, concurrently across all segments, and likely the biggest segments postings check would be the long pole, and {{CheckIndex}} would finish in (about) that much time.
   
   Versus "thread per segment" concurrency, where the long pole will be one thread checking the biggest segment.
   
   The really fun part of concurrency across *and* within segments will be figure out the rough heuristic of how to assign estimated cost to each task :)
   
   But this all can come later!   "thread per segment" is already a big step forward!


-- 
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.

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


[GitHub] [lucene] mikemccand commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r634640332



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -216,6 +225,9 @@
 
       /** Status of vectors */
       public VectorValuesStatus vectorValuesStatus;
+
+      /** Status of soft deletes */
+      public SoftDeletsStatus softDeletesStatus;

Review comment:
       Whoa, were we failing to `CheckIndex` soft deletes before this?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -926,17 +1100,19 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
    * @lucene.experimental
    */
   public static Status.LiveDocStatus testLiveDocs(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+      CodecReader reader, PrintStream infoStream, String segmentId) {
     long startNS = System.nanoTime();
+    String segmentPartId = segmentId + "[LiveDocs]";
     final Status.LiveDocStatus status = new Status.LiveDocStatus();
 
     try {
-      if (infoStream != null) infoStream.print("    test: check live docs.....");
+      if (infoStream != null) infoStream.print(segmentPartId + "    test: check live docs.....");
       final int numDocs = reader.numDocs();
       if (reader.hasDeletions()) {
         Bits liveDocs = reader.getLiveDocs();
         if (liveDocs == null) {
-          throw new RuntimeException("segment should have deletions, but liveDocs is null");
+          throw new RuntimeException(

Review comment:
       Should we maybe make a new `RuntimeException` subclass, that takes this `segmentPartId` as its own `String` parameter, and the exception message, and maybe a `Throwable cause`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -468,6 +495,10 @@ private static void msg(PrintStream out, String msg) {
     if (out != null) out.println(msg);
   }
 
+  private static void msg(PrintStream out, String id, String msg) {
+    if (out != null) out.println(id + " " + msg);

Review comment:
       Could you break this into separate lines and add `{ ... }`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -926,17 +1100,19 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
    * @lucene.experimental
    */
   public static Status.LiveDocStatus testLiveDocs(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+      CodecReader reader, PrintStream infoStream, String segmentId) {
     long startNS = System.nanoTime();
+    String segmentPartId = segmentId + "[LiveDocs]";
     final Status.LiveDocStatus status = new Status.LiveDocStatus();
 
     try {
-      if (infoStream != null) infoStream.print("    test: check live docs.....");
+      if (infoStream != null) infoStream.print(segmentPartId + "    test: check live docs.....");

Review comment:
       Sorry about not answering the `// nocommit` question before.
   
   Ideally, all `infoStream.print` for a given "part" of the index checking would first append to a per-part log, and then (under lock) print to console/main infoStream as a single "block" of output?  (So that we don't see confusing interleaved across segments/parts checks)?
   
   But I think it is OK to make this (cosmetic) improvement as a followon PR ... this change is already awesome enough.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -372,6 +384,14 @@ private FieldNormStatus() {}
       /** Exception thrown during term index test (null on success) */
       public Throwable error = null;
     }
+
+    /** Status from testing soft deletes */
+    public static final class SoftDeletsStatus {
+      SoftDeletsStatus() {}
+
+      /** Exception thrown during soft deletes test (null on success) */
+      public Throwable error = null;

Review comment:
       Hmm we don't need the `= null` -- it is already java's default.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -2106,16 +2286,6 @@ static void checkImpacts(Impacts impacts, int lastTarget) {
     }
   }
 
-  /**
-   * Test the term index.
-   *
-   * @lucene.experimental
-   */
-  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream)

Review comment:
       Hmm, did this just move elsewhere?  This is a helpful API to test just postings ...

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -2737,13 +2910,14 @@ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
    * @lucene.experimental
    */
   public static Status.StoredFieldStatus testStoredFields(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+      CodecReader reader, PrintStream infoStream, String segmentId) {
     long startNS = System.nanoTime();
+    String segmentPartId = segmentId + "[StoredFields]";

Review comment:
       OK maybe the custom `RuntimeException` subclass could take `CheckIndexFailure(String segmentId, String indexPart, String message, Throwable rootCause) {...}`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -2795,12 +2972,14 @@ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
    * @lucene.experimental
    */
   public static Status.DocValuesStatus testDocValues(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {

Review comment:
       This is net/net an API break, but given that `CheckIndex` is internal/experimental, I think it is OK (to backport to Lucene 8.x too)?




-- 
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.

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


[GitHub] [lucene] rmuir commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-850335925


   > The python tool looks very cool and thanks for testing it! One issue though is that this bit flipping is causing checksum integrity check failures before the concurrent segment part checks kick in, so it may not test the changes here? I think we may actually need to write a semantically buggy segment file with good checksum verification to see the error still gets detected and propagated correctly?
   
   This is also why checkindex has its `-fast` option, to only verify file checksums, which is probably what most end-users actually want when trying to verify that their index is intact and correct. The current (historical) slow defaults are really only useful for debugging a bug in lucene itself. And there is even a `-slow` that can do even more of that.


-- 
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.

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


[GitHub] [lucene] mikemccand commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-847885348


   OK I ran `CheckIndex`, with this PR, on last night's English Wikipedia benchmark index:
   
   ```
   beast3:core[main]$ java -cp ../core/build/libs/lucene-core-9.0.0-SNAPSHOT.jar org.apache.lucene.index.CheckIndex /l/indices/trunk.nightly.index.prev/index
   
   NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
   
   Opening index @ /l/indices/trunk.nightly.index.prev/index
   
   Exception in thread "main" java.lang.IllegalArgumentException
           at java.base/java.util.concurrent.ThreadPoolExecutor.<init>(ThreadPoolExecutor.java:1295)
           at java.base/java.util.concurrent.ThreadPoolExecutor.<init>(ThreadPoolExecutor.java:1217)
           at java.base/java.util.concurrent.Executors.newFixedThreadPool(Executors.java:155)
           at org.apache.lucene.index.CheckIndex.checkIndex(CheckIndex.java:527)
           at org.apache.lucene.index.CheckIndex.doCheck(CheckIndex.java:4036)
           at org.apache.lucene.index.CheckIndex.doMain(CheckIndex.java:3920)
           at org.apache.lucene.index.CheckIndex.main(CheckIndex.java:3852)
   ```
   
   I think the issue is that `opts.threadCount` is `0` if you don't explicitly set the thread count.  Can we fix it to default to number of cores on the box, maybe capped at a maximum (4? 
    8?), when `CheckIndex` is invoked interactively from the command-line?


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r697140951



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -3622,6 +3860,7 @@ public static void main(String[] args) throws IOException, InterruptedException
     boolean doSlowChecks = false;

Review comment:
       I've created a spin-off issue available here https://issues.apache.org/jira/browse/LUCENE-10074.




-- 
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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r694932614



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -450,6 +480,14 @@ public void setChecksumsOnly(boolean v) {
 
   private boolean checksumsOnly;
 
+  /** Set threadCount used for parallelizing index integrity checking. */
+  public void setThreadCount(int tc) {
+    threadCount = tc;

Review comment:
       Maybe validate the argument?  It must be >= 1 at least?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -3787,8 +4032,12 @@ public int doCheck(Options opts) throws IOException, InterruptedException {
     setDoSlowChecks(opts.doSlowChecks);
     setChecksumsOnly(opts.doChecksumsOnly);
     setInfoStream(opts.out, opts.verbose);
+    // when threadCount was not provided via command line, override it with 0 to turn of concurrent

Review comment:
       s/`of`/`off`

##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
##########
@@ -321,6 +326,11 @@ public static void syncConcurrentMerges(MergeScheduler ms) {
       checker.setDoSlowChecks(doSlowChecks);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false);
+      if (concurrent) {
+        checker.setThreadCount(RandomizedTest.randomIntBetween(1, 5));
+      } else {
+        checker.setThreadCount(0);

Review comment:
       Hmm, shouldn't we use `1` instead of `0` to mean "check sequentially"?  And maybe `0` should not be allowed?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -3622,6 +3860,7 @@ public static void main(String[] args) throws IOException, InterruptedException
     boolean doSlowChecks = false;

Review comment:
       [Pre-existing] We could remove all these `= false` and `= null`.  Hmm maybe there is a static `ecj` checker for this.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -181,6 +193,9 @@
       /** True if we were able to open a CodecReader on this segment. */
       public boolean openReaderPassed;
 
+      /** doc count in this segment */

Review comment:
       Yeah let's just keep it the (inconsistent) way it was before for now.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -605,209 +681,115 @@ 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);
-        }
-
-        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();
+    // 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;
         }
 
-        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
+      List<SegmentCommitInfo> segmentCommitInfos = new ArrayList<>();
+      for (SegmentCommitInfo sci : sis) {
+        segmentCommitInfos.add(sci);
+      }
 
-        segInfoStat.openReaderPassed = true;
+      // sort segmentCommitInfos by segment size, as smaller segment tends to finish faster, and

Review comment:
       Ahhh OK I see -- this is a tricky tradeoff of seeing output sooner, versus finishing the overall `CheckIndex` sooner :)  Let's leave it as it is here (seeing output sooner)?

##########
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) {

Review comment:
       OK that is really sneaky deadlock!  Maybe `MDW#close` should not hold its monitor lock when it calls `CheckIndex`, but let's not try to solve that here.  We can refactor in the future.  This change is already a good step forwards -- progress not perfection!

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -488,8 +503,35 @@ public Status checkIndex() throws IOException {
    *     quite a long time to run.
    */
   public Status checkIndex(List<String> onlySegments) throws IOException {

Review comment:
       > Oh I didn't realize it would spawn multiple jvm processes,
   
   This is just how Lucene's test infrastructure runs tests -- it spawns multiple JVMs, each of which is running one Lucene test at a time, but that test may use (often uses?) its own threads, including here in `CheckIndex` if we make it concurrent by default.

##########
File path: lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
##########
@@ -895,7 +895,11 @@ public synchronized void close() throws IOException {
             System.out.println("\nNOTE: MockDirectoryWrapper: now run CheckIndex");
           }
 
-          TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose(), true, null);
+          // Methods in MockDirectoryWrapper hold locks on this, which will cause deadlock when
+          // TestUtil#checkIndex checks segment concurrently using another thread, but making
+          // call back to synchronized methods such as MockDirectoryWrapper#fileLength.
+          // Hence passing concurrent = false to this method to turn off concurrent checks.
+          TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose(), true, false, null);

Review comment:
       Maybe open a follow-on issue to fix this sync situation so that we could, randomly, sometimes use concurrency in `CheckIndex` from tests?  Maybe we could start by making some of the `TestUtil.checkIndex` use concurrency, just not the one that MDW invokes?

##########
File path: lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java
##########
@@ -54,4 +65,137 @@ public void testChecksumsOnlyVerbose() throws IOException {
   public void testObtainsLock() throws IOException {
     testObtainsLock(directory);
   }
+
+  @Test
+  public void testCheckIndexAllValid() throws Exception {

Review comment:
       Thank you for this nice unit test!  Confirming the textual output syntax from `CheckIndex`.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -843,6 +825,258 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     return result;
   }
 
+  private void updateMaxSegmentName(Status result, SegmentCommitInfo info) {
+    long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
+    if (segmentName > result.maxSegmentName) {
+      result.maxSegmentName = segmentName;
+    }
+  }
+
+  private void processSegmentInfoStatusResult(
+      Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) {
+    result.segmentInfos.add(segmentInfoStatus);
+    if (segmentInfoStatus.error != null) {
+      result.totLoseDocCount += segmentInfoStatus.toLoseDocCount;
+      result.numBadSegments++;
+    } else {
+      // Keeper
+      result.newSegments.add(info.clone());
+    }
+  }
+
+  private <R> CompletableFuture<R> runAsyncSegmentCheck(
+      Callable<R> asyncCallable, ExecutorService executorService) {
+    return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService);
+  }
+
+  private <T> Supplier<T> callableToSupplier(Callable<T> callable) {
+    return () -> {
+      try {
+        return callable.call();
+      } catch (RuntimeException | Error e) {
+        throw e;
+      } catch (Throwable e) {
+        throw new CompletionException(e);
+      }
+    };
+  }
+
+  private Status.SegmentInfoStatus testSegment(
+      SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException {
+    Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
+    segInfoStat.name = info.info.name;
+    segInfoStat.maxDoc = info.info.maxDoc();
+
+    final Version version = info.info.getVersion();
+    if (info.info.maxDoc() <= 0) {
+      throw new CheckIndexException(" 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.);
+      // nf#format is not thread-safe, and would generate random non valid results in concurrent
+      // setting
+      synchronized (nf) {
+        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);
+      }
+
+      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)));
+
+      segInfoStat.openReaderPassed = true;
+
+      long startIntegrityNS = System.nanoTime();
+      if (infoStream != null) infoStream.print("    test: check integrity.....");
+      reader.checkIntegrity();
+      msg(
+          infoStream,
+          String.format(
+              Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime() - startIntegrityNS)));
+
+      if (reader.maxDoc() != info.info.maxDoc()) {
+        throw new CheckIndexException(
+            "SegmentReader.maxDoc() "
+                + reader.maxDoc()
+                + " != SegmentInfo.maxDoc "
+                + info.info.maxDoc());
+      }
+
+      final int numDocs = reader.numDocs();
+      toLoseDocCount = numDocs;
+
+      if (reader.hasDeletions()) {
+        if (reader.numDocs() != info.info.maxDoc() - info.getDelCount()) {
+          throw new CheckIndexException(
+              "delete count mismatch: info="
+                  + (info.info.maxDoc() - info.getDelCount())
+                  + " vs reader="
+                  + reader.numDocs());
+        }
+        if ((info.info.maxDoc() - reader.numDocs()) > reader.maxDoc()) {
+          throw new CheckIndexException(
+              "too many deleted docs: maxDoc()="
+                  + reader.maxDoc()
+                  + " vs del count="
+                  + (info.info.maxDoc() - reader.numDocs()));
+        }
+        if (info.info.maxDoc() - reader.numDocs() != info.getDelCount()) {
+          throw new CheckIndexException(
+              "delete count mismatch: info="
+                  + info.getDelCount()
+                  + " vs reader="
+                  + (info.info.maxDoc() - reader.numDocs()));
+        }
+      } else {
+        if (info.getDelCount() != 0) {
+          throw new CheckIndexException(
+              "delete count mismatch: info="
+                  + info.getDelCount()
+                  + " vs reader="
+                  + (info.info.maxDoc() - reader.numDocs()));
+        }
+      }
+
+      if (checksumsOnly == false) {
+        // This redundant assignment is done to make compiler happy
+        SegmentReader finalReader = reader;

Review comment:
       Egads, why does compiler insist on that?  I don't see any anonymous classes / lambda bodies trying to reference `reader`/`finalReader`?  Is this a leftover?




-- 
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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-841609643


   > I am excited to see what happens to [`CheckIndex` time in Lucene's nightly benchmarks](https://home.apache.org/~mikemccand/lucenebench/checkIndexTime.html) after we push this! But I agree we must also not crush the more common case of machines that don't have tons of cores ...
   
   Thanks Michael for the comment! Just curious, do you have any suggestion to my nocommit questions in this PR with regard to how to parallelize some of the section of code that currently depends on ordering (message printing inside each segment / index part, error handling etc)? I could also give it a try first and see how folks feel about the solution. 


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r696246368



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
##########
@@ -321,6 +326,11 @@ public static void syncConcurrentMerges(MergeScheduler ms) {
       checker.setDoSlowChecks(doSlowChecks);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false);
+      if (concurrent) {
+        checker.setThreadCount(RandomizedTest.randomIntBetween(1, 5));
+      } else {
+        checker.setThreadCount(0);

Review comment:
       Currently in the code, it is using `0` to signal turning off concurrent checking (using the main thread to check index), and it is also used as default when users don't explicitly pass in `-threadCount` flag.
   
   In `CheckIndex#doCheck(Options)`, here's the logic to overwrite default thread count with user provided value (`0` when not specified)
   ```
   // when threadCount was not provided via command line, overrides it with 0 to turn off concurrent
   // check
   setThreadCount(opts.threadCount);
   ```
   
   and later in `CheckIndex#checkIndex(List)`, it is used as such
   
   ```
   public Status checkIndex(List<String> onlySegments) throws IOException {
       ExecutorService executorService = null;
   
       if (threadCount > 0) {
         executorService =
             Executors.newFixedThreadPool(threadCount, new NamedThreadFactory("async-check-index"));
       }
   
       try {
         return checkIndex(onlySegments, executorService);
       } finally {
         ...
       }
   }
   ```
   
   ---
   
   If we were to use `1`  to signal checking sequentially / turning off concurrency, then I would imagine users will need to use `-threadCount 2` when they meant "create 1 more thread in addition to main thread to check index", and in general N+1 when they were thinking about creating N additional threads right? I feel this might be a bit unintuitive? 
   




-- 
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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r642722424



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -843,6 +812,258 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     return result;
   }
 
+  private void updateMaxSegmentName(Status result, SegmentCommitInfo info) {
+    long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
+    if (segmentName > result.maxSegmentName) {
+      result.maxSegmentName = segmentName;
+    }
+  }
+
+  private void processSegmentInfoStatusResult(
+      Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) {
+    result.segmentInfos.add(segmentInfoStatus);
+    if (segmentInfoStatus.error != null) {
+      result.totLoseDocCount += segmentInfoStatus.toLoseDocCount;
+      result.numBadSegments++;
+    } else {
+      // Keeper
+      result.newSegments.add(info.clone());
+    }
+  }
+
+  private <R> CompletableFuture<R> runAsyncSegmentCheck(
+      Callable<R> asyncCallable, ExecutorService executorService) {
+    return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService);
+  }
+
+  private <T> Supplier<T> callableToSupplier(Callable<T> callable) {
+    return () -> {
+      try {
+        return callable.call();
+      } catch (RuntimeException | Error e) {
+        throw e;
+      } catch (Throwable e) {
+        throw new CompletionException(e);
+      }
+    };
+  }
+
+  private Status.SegmentInfoStatus testSegment(
+      SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException {
+    Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
+    segInfoStat.name = info.info.name;
+    segInfoStat.maxDoc = info.info.maxDoc();
+
+    final Version version = info.info.getVersion();
+    if (info.info.maxDoc() <= 0) {
+      throw new CheckIndexException(" illegal number of documents: maxDoc=" + info.info.maxDoc());
+    }
+
+    int toLoseDocCount = info.info.maxDoc();

Review comment:
       I took a closer look at this code. I think the reason of inconsistency here is that, between `toLoseDocCount = info.info.maxDoc()` and `toLoseDocCount = reader.numDocs()`, there are a few places where exception can be thrown (e.g. `reader = new SegmentReader` may throw IOException), but we still want to catch the exception and add the value into result via `segInfoStat.toLoseDocCount = toLoseDocCount` (using maxDoc count as the upper bound). So I guess this inconsistency is intended?




-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r627960571



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -605,6 +610,15 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     result.newSegments.clear();
     result.maxSegmentName = -1;
 
+    // nocommit number of threads should be set dynamically

Review comment:
       Great suggestions! I've updated it to work for both `checkIndex` invoked from `main`, as well as invoked from other classes / methods. 




-- 
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.

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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-850170387


   >  the lines come as each check finishes, so you can see what is fast/slow. It seems postings is slowest, preceded by doc values, and everything else is super fast.
   > Can't wait to see the next CheckIndex time in nightly benchmarks after we push this :)
   
   I was finally able to rebuild my local index with `wikibigall` generating 15 segments, and performed 2 test runs with different threadCount.  With 11 threadCount, it took 359.293 sec in total to finish, and with 1 threadCount it took 378.583 sec in total, so about 5% time saving. I feel faster machine will have better time saving, but in general the speed up seems to be limited given the skewed distribution of checking speed of different segment parts  (e.g. posting check can account for around 85% ~ 90%  of the total time spent in the first segment check). 
   
   > Good news first! I wrote a simple little Python tool to randomly flip a random bit in a random file in a provided directory.
   > It looks like corruption is indeed still detected with this PR, wonderful!
   
   The python tool looks very cool and thanks for testing it! One issue though is that this bit flipping is causing checksum integrity check failures *before* the concurrent segment part checks kick in, so it may not test the changes here? I think we may actually need to write a semantically buggy segment file with good checksum verification to see the error still gets detected and propagated correctly? 
   
   With the above two, I feel maybe I should also look into parallelizing across segments and keeping it single-threaded / simple within each segment (much of the learnings here can be applied there anyway)? The other approach to get better speed up could be to split up posting check, so that it can be handled by multiple threads as well. But I'm not sure now if it's easily parallelize-able and also need to look into it further. 
   
   I'm also good with merging this in and see how it performs.


-- 
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.

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


[GitHub] [lucene] dweiss commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
dweiss commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r638657529



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
##########
@@ -321,6 +321,7 @@ public static void syncConcurrentMerges(MergeScheduler ms) {
       checker.setDoSlowChecks(doSlowChecks);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false);
+      checker.setThreadCount(RandomNumbers.randomIntBetween(new Random(), 1, 5));

Review comment:
       You should use randomized testing's context random, otherwise tests are not reproducible. Use this static method from RandomizedTest: https://github.com/randomizedtesting/randomizedtesting/blob/master/randomized-runner/src/main/java/com/carrotsearch/randomizedtesting/RandomizedTest.java#L161.




-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r639455850



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -926,17 +1100,19 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
    * @lucene.experimental
    */
   public static Status.LiveDocStatus testLiveDocs(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+      CodecReader reader, PrintStream infoStream, String segmentId) {
     long startNS = System.nanoTime();
+    String segmentPartId = segmentId + "[LiveDocs]";
     final Status.LiveDocStatus status = new Status.LiveDocStatus();
 
     try {
-      if (infoStream != null) infoStream.print("    test: check live docs.....");
+      if (infoStream != null) infoStream.print(segmentPartId + "    test: check live docs.....");

Review comment:
       I've implemented it here https://github.com/apache/lucene/pull/128/commits/57f542f48ee08ec2bd63520c43deb0734455bd28 . The per part messages should be printed as soon as each concurrent check finishes, and without locking since the shared `PrintStream` object already locks internally.




-- 
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.

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


[GitHub] [lucene] mikemccand commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r638784407



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -926,17 +1100,19 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
    * @lucene.experimental
    */
   public static Status.LiveDocStatus testLiveDocs(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+      CodecReader reader, PrintStream infoStream, String segmentId) {
     long startNS = System.nanoTime();
+    String segmentPartId = segmentId + "[LiveDocs]";
     final Status.LiveDocStatus status = new Status.LiveDocStatus();
 
     try {
-      if (infoStream != null) infoStream.print("    test: check live docs.....");
+      if (infoStream != null) infoStream.print(segmentPartId + "    test: check live docs.....");

Review comment:
       > > Ideally, all infoStream.print for a given "part" of the index checking would first append to a per-part log, and then (under lock) print to console/main infoStream as a single "block" of output? (So that we don't see confusing interleaved across segments/parts checks)?
   > 
   > Oh I see, haven't thought about this approach before, and it sounds interesting! I assume by "per-part log" you meant an array of in-memory, per part buffers that accumulate messages over concurrent check right? If we were to combine these buffers at the end of / after the concurrent index check, we should be ok to just print them out to main InfoStream without locking?
   
   Yes, exactly!  So we won't see the logged output coming out in real-time as the checks happen, like `CheckIndex` does today, but rather all things are running concurrently, and then, once you've joined all those concurrent checker threads back to main thread, the main thread prints all per-part output messages to the console.  So then the user would still see the same (coherent looking) output, just with some delay since we wait for all concurrent checks to finish.
   
   Or, alternatively, once any concurrent check finishes, you immediately acquire "console printing lock", and print its full output.  This is a bit better because you see the output as each part finishes, and the long-pole slow checker parts won't delay the output of the fast parts.  Less nail-biting for the user ...




-- 
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.

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


[GitHub] [lucene] rmuir commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
rmuir commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r627567282



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -605,6 +610,15 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     result.newSegments.clear();
     result.maxSegmentName = -1;
 
+    // nocommit number of threads should be set dynamically

Review comment:
       can the checkIndex() method just take ExecutorService as a parameter? This way, e.g. the calling `main` method could configure the ExecutorService (possibly based on commandline options), unit tests can use a single thread, etc.




-- 
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.

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


[GitHub] [lucene] mikemccand commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r638777677



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
##########
@@ -321,6 +321,7 @@ public static void syncConcurrentMerges(MergeScheduler ms) {
       checker.setDoSlowChecks(doSlowChecks);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false);
+      checker.setThreadCount(RandomNumbers.randomIntBetween(new Random(), 1, 5));

Review comment:
       @dweiss also opened a good follow-on issue to [add `new Random()` to forbidden APIs for test](https://issues.apache.org/jira/browse/LUCENE-9973).  Ahh, it was already there, but something else was wrong in how `test-framework` invokes forbidden APIs.  Anyway it looks like @dweiss is fixing this, thanks ;)




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r642639399



##########
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) {

Review comment:
       This is actually done to avoid deadlock during test (I put a comment in `MockDirectoryWrapper#close` on passing a flag to have null `executorService`, but it might not be immediately obvious here). 
   
   Essentially, the deadlock can be formed in test as such even with a single threaded executor:
   1. At the end of the tests that use directory, `MockDirectoryWrapper#close` was called, which would hold directory's monitor as the method is `synchronized`
   2. `MockDirectoryWrapper#close` would call `TestUtil#checkIndex` and passed in itself for directory reference as 1st argument
   3. With concurrent execution across segments in `TestUtil#checkIndex`, another thread checking segment would be making call back to methods from directory, such as `MockDirectoryWrapper#fileLength`, which again require directory's monitor access as they are also `synchronized`
   4. Deadlock occurred as another thread is waiting for directory's monitor, which is held by the main thread waiting for the other thread to complete

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -843,6 +812,258 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     return result;
   }
 
+  private void updateMaxSegmentName(Status result, SegmentCommitInfo info) {
+    long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
+    if (segmentName > result.maxSegmentName) {
+      result.maxSegmentName = segmentName;
+    }
+  }
+
+  private void processSegmentInfoStatusResult(
+      Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) {
+    result.segmentInfos.add(segmentInfoStatus);
+    if (segmentInfoStatus.error != null) {
+      result.totLoseDocCount += segmentInfoStatus.toLoseDocCount;
+      result.numBadSegments++;
+    } else {
+      // Keeper
+      result.newSegments.add(info.clone());
+    }
+  }
+
+  private <R> CompletableFuture<R> runAsyncSegmentCheck(
+      Callable<R> asyncCallable, ExecutorService executorService) {
+    return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService);
+  }
+
+  private <T> Supplier<T> callableToSupplier(Callable<T> callable) {
+    return () -> {
+      try {
+        return callable.call();
+      } catch (RuntimeException | Error e) {
+        throw e;
+      } catch (Throwable e) {
+        throw new CompletionException(e);
+      }
+    };
+  }
+
+  private Status.SegmentInfoStatus testSegment(
+      SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException {
+    Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
+    segInfoStat.name = info.info.name;
+    segInfoStat.maxDoc = info.info.maxDoc();
+
+    final Version version = info.info.getVersion();
+    if (info.info.maxDoc() <= 0) {
+      throw new CheckIndexException(" illegal number of documents: maxDoc=" + info.info.maxDoc());
+    }
+
+    int toLoseDocCount = info.info.maxDoc();

Review comment:
       This part was actually copied / extracted into method from existing logic:
   
   https://github.com/apache/lucene/blob/c46bcf75cc6592cc492fabd42e1ec41dbe96304d/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java#L637-L699
   
   but I can update it as well to make it consistent.




-- 
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.

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


[GitHub] [lucene] rmuir commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-850330270


   > I was finally able to rebuild my local index with wikibigall generating 15 segments, and performed 2 test runs with different threadCount. With 11 threadCount, it took 359.293 sec in total to finish, and with 1 threadCount it took 378.583 sec in total, so about 5% time saving. I feel faster machine will have better time saving, but in general the speed up seems to be limited given the skewed distribution of checking speed of different segment parts (e.g. posting check can account for around 85% ~ 90% of the total time spent in the first segment check).
   
   I really don't think we should add this complexity to checkindex if it only improves it 5%. Let's take a step back and do this issue differently (e.g. just check whole segments concurrently with different threads). This should also require less complexity and be less invasive to the checkindex tool.


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r639460784



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -1058,16 +1261,14 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
 
       msg(
           infoStream,
+          segmentId + partId,
           String.format(
               Locale.ROOT,
               "OK [%d fields] [took %.3f sec]",
               status.totFields,
               nsToSec(System.nanoTime() - startNS)));
     } catch (Throwable e) {
-      if (failFast) {
-        throw IOUtils.rethrowAlways(e);
-      }
-      msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
+      msg(infoStream, segmentId + partId, "ERROR [" + String.valueOf(e.getMessage()) + "]");

Review comment:
       I've removed the use of `segmentId` and `partId` as part of 57f542f.




-- 
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.

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


[GitHub] [lucene] muse-dev[bot] commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
muse-dev[bot] commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r638542309



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
##########
@@ -321,6 +321,7 @@ public static void syncConcurrentMerges(MergeScheduler ms) {
       checker.setDoSlowChecks(doSlowChecks);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false);
+      checker.setThreadCount(RandomNumbers.randomIntBetween(new Random(), 1, 5));

Review comment:
       *PREDICTABLE_RANDOM:*  This random generator (java.util.Random) is predictable [(details)](https://find-sec-bugs.github.io/bugs.htm#PREDICTABLE_RANDOM)
   (at-me [in a reply](https://docs.muse.dev/docs/talk-to-muse/) with `help` or `ignore`)




-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r634953443



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -488,8 +519,35 @@ public Status checkIndex() throws IOException {
    *     quite a long time to run.
    */
   public Status checkIndex(List<String> onlySegments) throws IOException {
+    ExecutorService executorService =
+        Executors.newFixedThreadPool(threadCount, new NamedThreadFactory("async-check-index"));
+    try {
+      return checkIndex(onlySegments, executorService);
+    } finally {
+      executorService.shutdown();
+      try {
+        executorService.awaitTermination(5, TimeUnit.SECONDS);
+      } catch (
+          @SuppressWarnings("unused")

Review comment:
       Yeah agreed. Updated. 
   

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -701,104 +765,196 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         if (reader.hasDeletions()) {
           if (reader.numDocs() != info.info.maxDoc() - info.getDelCount()) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + (info.info.maxDoc() - info.getDelCount())
                     + " vs reader="
                     + reader.numDocs());
           }
           if ((info.info.maxDoc() - reader.numDocs()) > reader.maxDoc()) {
             throw new RuntimeException(
-                "too many deleted docs: maxDoc()="
+                segmentId
+                    + "too many deleted docs: maxDoc()="
                     + reader.maxDoc()
                     + " vs del count="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
           if (info.info.maxDoc() - reader.numDocs() != info.getDelCount()) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + info.getDelCount()
                     + " vs reader="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
         } else {
           if (info.getDelCount() != 0) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + info.getDelCount()
                     + " vs reader="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              runAysncSegmentPartCheck(

Review comment:
       Oops. Fixed. 




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-906051631


   > Sorry for taking so long to respond @zacharymorn! This change looks great -- I left a bunch of minor comments. Thank you for getting the output to match what we see today, and adding a test case confirming that :) Too bad about the MDW deadlock, but we can defer solving that one.
   
   No worry there, and as usual appreciate your time and effort for reviewing my changes, and providing great suggestions and feedback!


-- 
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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r634956106



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -468,6 +495,10 @@ private static void msg(PrintStream out, String msg) {
     if (out != null) out.println(msg);
   }
 
+  private static void msg(PrintStream out, String id, String msg) {
+    if (out != null) out.println(id + " " + msg);

Review comment:
       Done. 

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -372,6 +384,14 @@ private FieldNormStatus() {}
       /** Exception thrown during term index test (null on success) */
       public Throwable error = null;
     }
+
+    /** Status from testing soft deletes */
+    public static final class SoftDeletsStatus {
+      SoftDeletsStatus() {}
+
+      /** Exception thrown during soft deletes test (null on success) */
+      public Throwable error = null;

Review comment:
       Removed, as well as the same ones used in other status classes. 




-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r639459734



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -731,74 +810,173 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testLiveDocs(finalReader, infoStream, segmentId),
+                  liveDocStatus -> segInfoStat.liveDocStatus = liveDocStatus);
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldInfos(finalReader, infoStream, segmentId),
+                  fieldInfoStatus -> segInfoStat.fieldInfoStatus = fieldInfoStatus);
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldNorms(finalReader, infoStream, segmentId),
+                  fieldNormStatus -> segInfoStat.fieldNormStatus = fieldNormStatus);
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPostings(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termIndexStatus -> segInfoStat.termIndexStatus = termIndexStatus);
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testStoredFields(finalReader, infoStream, segmentId),
+                  storedFieldStatus -> segInfoStat.storedFieldStatus = storedFieldStatus);
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testTermVectors(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termVectorStatus -> segInfoStat.termVectorStatus = termVectorStatus);
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testDocValues(finalReader, infoStream, segmentId),
+                  docValuesStatus -> segInfoStat.docValuesStatus = docValuesStatus);
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPoints(finalReader, infoStream, segmentId),
+                  pointsStatus -> segInfoStat.pointsStatus = pointsStatus);
 
           // Test VectorValues
-          segInfoStat.vectorValuesStatus = testVectors(reader, infoStream, failFast);
+          CompletableFuture<Void> testVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testVectors(finalReader, infoStream, segmentId),
+                  vectorValuesStatus -> segInfoStat.vectorValuesStatus = vectorValuesStatus);
 
           // Test index sort
-          segInfoStat.indexSortStatus = testSort(reader, indexSort, infoStream, failFast);
+          CompletableFuture<Void> testSort =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testSort(finalReader, indexSort, infoStream, segmentId),
+                  indexSortStatus -> segInfoStat.indexSortStatus = indexSortStatus);
+
+          CompletableFuture<Void> testSoftDeletes = null;
+          final String softDeletesField = reader.getFieldInfos().getSoftDeletesField();
+          if (softDeletesField != null) {
+            testSoftDeletes =
+                runAsyncSegmentPartCheck(
+                    executorService,
+                    () ->
+                        checkSoftDeletes(
+                            softDeletesField, info, finalReader, infoStream, segmentId),
+                    softDeletesStatus -> segInfoStat.softDeletesStatus = softDeletesStatus);
+          }
 
           // Rethrow the first exception we encountered
           //  This will cause stats for failed segments to be incremented properly
+          testliveDocs.join();
           if (segInfoStat.liveDocStatus.error != null) {
-            throw new RuntimeException("Live docs test failed");
-          } else if (segInfoStat.fieldInfoStatus.error != null) {
-            throw new RuntimeException("Field Info test failed");
-          } else if (segInfoStat.fieldNormStatus.error != null) {
-            throw new RuntimeException("Field Norm test failed");
-          } else if (segInfoStat.termIndexStatus.error != null) {
-            throw new RuntimeException("Term Index test failed");
-          } else if (segInfoStat.storedFieldStatus.error != null) {
-            throw new RuntimeException("Stored Field test failed");
-          } else if (segInfoStat.termVectorStatus.error != null) {
-            throw new RuntimeException("Term Vector test failed");
-          } else if (segInfoStat.docValuesStatus.error != null) {
-            throw new RuntimeException("DocValues test failed");
-          } else if (segInfoStat.pointsStatus.error != null) {
-            throw new RuntimeException("Points test failed");
+            throw new CheckIndexException(
+                segmentId, "", "Live docs test failed", segInfoStat.liveDocStatus.error);
+          }
+
+          testFieldInfos.join();

Review comment:
       I've handled it inside each of the future blocks in https://github.com/apache/lucene/pull/128/commits/57f542f48ee08ec2bd63520c43deb0734455bd28, so that the per part messages can be printed out as soon as they are available, and don't need to wait for `join` of other futures. 




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r642550067



##########
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) {

Review comment:
       Maybe we should make a single threaded executor so we don't have to bifurcate the code?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -843,6 +812,258 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     return result;
   }
 
+  private void updateMaxSegmentName(Status result, SegmentCommitInfo info) {
+    long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
+    if (segmentName > result.maxSegmentName) {
+      result.maxSegmentName = segmentName;
+    }
+  }
+
+  private void processSegmentInfoStatusResult(
+      Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) {
+    result.segmentInfos.add(segmentInfoStatus);
+    if (segmentInfoStatus.error != null) {
+      result.totLoseDocCount += segmentInfoStatus.toLoseDocCount;
+      result.numBadSegments++;
+    } else {
+      // Keeper
+      result.newSegments.add(info.clone());
+    }
+  }
+
+  private <R> CompletableFuture<R> runAsyncSegmentCheck(
+      Callable<R> asyncCallable, ExecutorService executorService) {
+    return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService);
+  }
+
+  private <T> Supplier<T> callableToSupplier(Callable<T> callable) {
+    return () -> {
+      try {
+        return callable.call();
+      } catch (RuntimeException | Error e) {
+        throw e;
+      } catch (Throwable e) {
+        throw new CompletionException(e);
+      }
+    };
+  }
+
+  private Status.SegmentInfoStatus testSegment(
+      SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException {
+    Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
+    segInfoStat.name = info.info.name;
+    segInfoStat.maxDoc = info.info.maxDoc();
+
+    final Version version = info.info.getVersion();
+    if (info.info.maxDoc() <= 0) {
+      throw new CheckIndexException(" illegal number of documents: maxDoc=" + info.info.maxDoc());
+    }
+
+    int toLoseDocCount = info.info.maxDoc();

Review comment:
       Hmm, sometimes this seems to be `maxDoc` (here) and other times `numDocs` (accounting for deleted documents properly) -- let's try to be consistent with what it was before (I think `numDocs`)?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -450,6 +479,14 @@ public void setChecksumsOnly(boolean v) {
 
   private boolean checksumsOnly;
 
+  /** Set threadCount used for parallelizing index integrity checking. */
+  public void setThreadCount(int tc) {
+    threadCount = tc;
+  }
+
+  // capped threadCount at 4
+  private int threadCount = Math.min(Runtime.getRuntime().availableProcessors(), 4);

Review comment:
       Whoa, why `4` :)  Could we maybe use java's `Runtime.getRuntime().availableProcessorts()`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -843,6 +812,258 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     return result;
   }
 
+  private void updateMaxSegmentName(Status result, SegmentCommitInfo info) {
+    long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
+    if (segmentName > result.maxSegmentName) {
+      result.maxSegmentName = segmentName;
+    }
+  }
+
+  private void processSegmentInfoStatusResult(
+      Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) {
+    result.segmentInfos.add(segmentInfoStatus);
+    if (segmentInfoStatus.error != null) {
+      result.totLoseDocCount += segmentInfoStatus.toLoseDocCount;
+      result.numBadSegments++;
+    } else {
+      // Keeper
+      result.newSegments.add(info.clone());
+    }
+  }
+
+  private <R> CompletableFuture<R> runAsyncSegmentCheck(
+      Callable<R> asyncCallable, ExecutorService executorService) {
+    return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService);
+  }
+
+  private <T> Supplier<T> callableToSupplier(Callable<T> callable) {
+    return () -> {
+      try {
+        return callable.call();
+      } catch (RuntimeException | Error e) {
+        throw e;
+      } catch (Throwable e) {
+        throw new CompletionException(e);
+      }
+    };
+  }
+
+  private Status.SegmentInfoStatus testSegment(
+      SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException {
+    Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
+    segInfoStat.name = info.info.name;
+    segInfoStat.maxDoc = info.info.maxDoc();
+
+    final Version version = info.info.getVersion();
+    if (info.info.maxDoc() <= 0) {
+      throw new CheckIndexException(" 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.);
+      // nf#format is not thread-safe, and would generate random non valid results in concurrent
+      // setting
+      synchronized (nf) {
+        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);
+      }
+
+      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)));
+
+      segInfoStat.openReaderPassed = true;
+
+      long startIntegrityNS = System.nanoTime();
+      if (infoStream != null) infoStream.print("    test: check integrity.....");
+      reader.checkIntegrity();
+      msg(
+          infoStream,
+          String.format(
+              Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime() - startIntegrityNS)));
+
+      if (reader.maxDoc() != info.info.maxDoc()) {
+        throw new CheckIndexException(
+            "SegmentReader.maxDoc() "
+                + reader.maxDoc()
+                + " != SegmentInfo.maxDoc "
+                + info.info.maxDoc());
+      }
+
+      final int numDocs = reader.numDocs();
+      toLoseDocCount = numDocs;

Review comment:
       Here is it `numDocs` (taking deletions into account).

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -181,6 +193,9 @@
       /** True if we were able to open a CodecReader on this segment. */
       public boolean openReaderPassed;
 
+      /** doc count in this segment */

Review comment:
       Once we decide whether this is `maxDoc` or `docCount` (taking deletions into account) let's update this javadoc?

##########
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:
       I think it would be better to 1) buffer all segment's output, and 2) print each segment's full output once it is done.  This way the tiny segments which finish quickly would produce their output, and the large segments would be the long poles, finally finishing and printing theirs.




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851528281


   > One issue though is that this bit flipping is causing checksum integrity check failures before the concurrent segment part checks kick in, so it may not test the changes here? I think we may actually need to write a semantically buggy segment file with good checksum verification to see the error still gets detected and propagated correctly?
   
   You're right -- it is able to do that.  It's random, so if it flips a "really important" bit, such that the `segments_N` file cannot be loaded, then `CheckIndex` will fail, quickly, without checking the segments.  If you watch its output and re-run it until you get a single segment corruption event, then it should showcase the gains in this PR.


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851527406


   Thank you for all the awesome iterations here @zacharymorn!
   
   To get the best speedup, even at `-slow`, we should do concurrency both ways, and then sort those tasks by decreasing expected cost.  This way the work queue would first output all postings checks (across all segments), one per thread, followed by doc values, etc.  We could even get a bit crazy, e.g. checking postings for a tiny segment is surely expected to be faster than checking doc values for a massive segment.
   
   But we can add such complexity later -- the PR now ("thread per segment") is surely a great step forward too :)
   
   And +1 to spinoff a separate issue to change `CheckIndex` to default to `-fast` -- this is really long overdue since we added end-to-end checksums to Lucene!


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r639457972



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -700,29 +771,37 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
 
         if (reader.hasDeletions()) {
           if (reader.numDocs() != info.info.maxDoc() - info.getDelCount()) {
-            throw new RuntimeException(
+            throw new CheckIndexException(
+                segmentId,
+                "",

Review comment:
       I've removed the use of `segmentId` and `partId` as part of https://github.com/apache/lucene/pull/128/commits/57f542f48ee08ec2bd63520c43deb0734455bd28. 




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r642719647



##########
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:
       I've implemented the above by sorting segments by the file size they contain in increasing order. Here are the test results:
   
   ### Full check on good index
   ```
   5:12:03 PM: Executing task 'CheckIndex.main()'...
   
   > Task :buildSrc:compileJava UP-TO-DATE
   > Task :buildSrc:compileGroovy NO-SOURCE
   > Task :buildSrc:processResources NO-SOURCE
   > Task :buildSrc:classes UP-TO-DATE
   > Task :buildSrc:jar UP-TO-DATE
   > Task :buildSrc:assemble UP-TO-DATE
   > Task :buildSrc:compileTestJava NO-SOURCE
   > Task :buildSrc:compileTestGroovy NO-SOURCE
   > Task :buildSrc:processTestResources NO-SOURCE
   > Task :buildSrc:testClasses UP-TO-DATE
   > Task :buildSrc:test NO-SOURCE
   > Task :buildSrc:check UP-TO-DATE
   > Task :buildSrc:build UP-TO-DATE
   
   > Configure project :
   IntelliJ Idea IDE detected.
   
   > Task :errorProneSkipped
   WARNING: errorprone disabled (skipped on non-nightly runs)
   
   > Task :lucene:core:processResources UP-TO-DATE
   > Task :lucene:core:compileJava
   > Task :lucene:core:classes
   
   > Task :lucene:core:CheckIndex.main()
   
   NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
   
   Opening index @ /Users/xichen/IdeaProjects/benchmarks/indices/wikibigall.lucene_baseline.facets.taxonomy:Date.taxonomy:Month.taxonomy:DayOfYear.sortedset:Month.sortedset:DayOfYear.Lucene90.Lucene90.nd6.64758M/index
   
   Checking index with async threadCount: 12
   0.00% total deletions; 6647577 documents; 0 deletions
   Segments file=segments_2 numSegments=15 version=9.0.0 id=59c6he3dhebad46x7proh30nq userData={userData=multi}
   1 of 15: name=_h2 maxDoc=11248
       version=9.0.0
       id=59c6he3dhebad46x7proh30nm
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=10.617
       diagnostics = {os.version=10.15.5, java.runtime.version=11.0.9+11, os.arch=x86_64, source=flush, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, timestamp=1622102791291, os=Mac OS X, java.vendor=AdoptOpenJDK}
       no deletions
       test: open reader.........OK [took 0.167 sec]
       test: check integrity.....OK [took 0.097 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.007 sec]
       test: terms, freq, prox...OK [253387 terms; 1570705 terms/docs pairs; 3390075 tokens] [took 0.771 sec]
       test: stored fields.......OK [33744 total field count; avg 3.0 fields per doc] [took 0.057 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.111 sec]
       test: points..............OK [2 fields, 22496 points] [took 0.016 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   2 of 15: name=_h1 maxDoc=11979
       version=9.0.0
       id=59c6he3dhebad46x7proh30nj
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=12.824
       diagnostics = {os.version=10.15.5, java.runtime.version=11.0.9+11, os.arch=x86_64, source=flush, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, timestamp=1622102788648, os=Mac OS X, java.vendor=AdoptOpenJDK}
       no deletions
       test: open reader.........OK [took 0.166 sec]
       test: check integrity.....OK [took 0.108 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.004 sec]
       test: terms, freq, prox...OK [290488 terms; 1843478 terms/docs pairs; 4383419 tokens] [took 0.880 sec]
       test: stored fields.......OK [35937 total field count; avg 3.0 fields per doc] [took 0.021 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.083 sec]
       test: points..............OK [2 fields, 23958 points] [took 0.005 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   ...
   ...
   
   14 of 15: name=_65 maxDoc=1197893
       version=9.0.0
       id=59c6he3dhebad46x7proh2zqv
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=1,539.981
       diagnostics = {os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11, os=Mac OS X, timestamp=1622100810971, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1}
       no deletions
       test: open reader.........OK [took 0.018 sec]
       test: check integrity.....OK [took 14.172 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.045 sec]
       test: terms, freq, prox...OK [15042354 terms; 274837439 terms/docs pairs; 686566591 tokens] [took 74.763 sec]
       test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 0.910 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 2.224 sec]
       test: points..............OK [2 fields, 2395786 points] [took 0.183 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   15 of 15: name=_32 maxDoc=1197893
       version=9.0.0
       id=59c6he3dhebad46x7proh2zhm
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=2,531.843
       diagnostics = {os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11, os=Mac OS X, timestamp=1622100146526, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1}
       no deletions
       test: open reader.........OK [took 0.020 sec]
       test: check integrity.....OK [took 21.075 sec]
       test: check live docs.....OK [took 0.014 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.041 sec]
       test: terms, freq, prox...OK [20065511 terms; 450728331 terms/docs pairs; 1175837878 tokens] [took 111.604 sec]
       test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 1.112 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 2.648 sec]
       test: points..............OK [2 fields, 2395786 points] [took 0.207 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   No problems were detected with this index.
   
   Took 138.332 sec total.
   
   
   BUILD SUCCESSFUL in 2m 22s
   4 actionable tasks: 3 executed, 1 up-to-date
   5:14:26 PM: Task execution finished 'CheckIndex.main()'.
   
   ```
   
   ### Full check on bad index
   ```
   > Task :lucene:core:CheckIndex.main()
   
   NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
   
   Opening index @ /Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/
   
   Checking index with async threadCount: 12
   0.00% total deletions; 6647577 documents; 0 deletions
   Segments file=segments_2 numSegments=15 version=9.0.0 id=59c6he3dhebad46x7proh30nq userData={userData=multi}
   1 of 15: name=_h2 maxDoc=11248
       version=9.0.0
       id=59c6he3dhebad46x7proh30nm
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=10.617
       diagnostics = {os.arch=x86_64, source=flush, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, timestamp=1622102791291, os=Mac OS X, java.vendor=AdoptOpenJDK, os.version=10.15.5, java.runtime.version=11.0.9+11}
       no deletions
       test: open reader.........OK [took 0.101 sec]
       test: check integrity.....OK [took 0.047 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.005 sec]
       test: terms, freq, prox...OK [253387 terms; 1570705 terms/docs pairs; 3390075 tokens] [took 0.863 sec]
       test: stored fields.......OK [33744 total field count; avg 3.0 fields per doc] [took 0.059 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.133 sec]
       test: points..............OK [2 fields, 22496 points] [took 0.015 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   ...
   ...
   
   6 of 15: name=_gb maxDoc=119789
       version=9.0.0
       id=59c6he3dhebad46x7proh30ld
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=125.605
       diagnostics = {os=Mac OS X, timestamp=1622102690942, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11}
       no deletions
       test: open reader.........OK [took 0.101 sec]
       test: check integrity.....OK [took 0.678 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.007 sec]
       test: terms, freq, prox...OK [1773712 terms; 20129621 terms/docs pairs; 51648295 tokens] [took 5.681 sec]
       test: stored fields.......OK [359367 total field count; avg 3.0 fields per doc] [took 0.168 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.856 sec]
       test: points..............OK [2 fields, 239578 points] [took 0.037 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   7 of 15: name=_gx maxDoc=119789
       version=9.0.0
       id=59c6he3dhebad46x7proh30n7
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=129.046
       diagnostics = {os=Mac OS X, timestamp=1622102767300, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11}
       no deletions
       test: open reader.........OK [took 0.101 sec]
       test: check integrity.....FAILED
       WARNING: exorciseIndex() would remove reference to this segment; full exception:
   org.apache.lucene.index.CorruptIndexException: checksum failed (hardware problem?) : expected=87e2aa4 actual=7b3afcbd (resource=BufferedChecksumIndexInput(MMapIndexInput(path="/Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/_gx_Lucene90_0.dvd")))
   	at org.apache.lucene.codecs.CodecUtil.checkFooter(CodecUtil.java:440)
   	at org.apache.lucene.codecs.CodecUtil.checksumEntireFile(CodecUtil.java:614)
   	at org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer.checkIntegrity(Lucene90DocValuesProducer.java:1656)
   	at org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat$FieldsReader.checkIntegrity(PerFieldDocValuesFormat.java:364)
   	at org.apache.lucene.index.CodecReader.checkIntegrity(CodecReader.java:252)
   	at org.apache.lucene.index.SegmentReader.checkIntegrity(SegmentReader.java:391)
   	at org.apache.lucene.index.CheckIndex.testSegment(CheckIndex.java:925)
   	at org.apache.lucene.index.CheckIndex.lambda$checkIndex$1(CheckIndex.java:756)
   	at org.apache.lucene.index.CheckIndex.lambda$callableToSupplier$2(CheckIndex.java:854)
   	at java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1700)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at java.base/java.lang.Thread.run(Thread.java:834)
   
   14 of 15: name=_65 maxDoc=1197893
       version=9.0.0
       id=59c6he3dhebad46x7proh2zqv
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=1,539.981
       diagnostics = {os=Mac OS X, timestamp=1622100810971, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11}
       no deletions
       test: open reader.........OK [took 0.011 sec]
       test: check integrity.....OK [took 11.858 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.044 sec]
       test: terms, freq, prox...OK [15042354 terms; 274837439 terms/docs pairs; 686566591 tokens] [took 83.292 sec]
       test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 0.903 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 2.599 sec]
       test: points..............OK [2 fields, 2395786 points] [took 0.210 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   15 of 15: name=_32 maxDoc=1197893
       version=9.0.0
       id=59c6he3dhebad46x7proh2zhm
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=2,531.843
       diagnostics = {os=Mac OS X, timestamp=1622100146526, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11}
       no deletions
       test: open reader.........OK [took 0.016 sec]
       test: check integrity.....OK [took 19.048 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.047 sec]
       test: terms, freq, prox...OK [20065511 terms; 450728331 terms/docs pairs; 1175837878 tokens] [took 118.554 sec]
       test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 1.099 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 2.914 sec]
       test: points..............OK [2 fields, 2395786 points] [took 0.214 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   WARNING: 1 broken segments (containing 119789 documents) detected
   Took 143.432 sec total.
   WARNING: would write new segments file, and 119789 documents would be lost, if -exorcise were specified
   
   
   
   > Task :lucene:core:CheckIndex.main() FAILED
   
   Execution failed for task ':lucene:core:CheckIndex.main()'.
   > Process 'command '/Library/Java/JavaVirtualMachines/adoptopenjdk-11.jdk/Contents/Home/bin/java'' finished with non-zero exit value 1
   
   
   
   ```
   
   ### Check with `-segment _gx` flag
   ```
   > Task :lucene:core:CheckIndex.main() FAILED
   
   NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
   
   Opening index @ /Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/
   
   Checking index with async threadCount: 12
   0.00% total deletions; 6647577 documents; 0 deletions
   Segments file=segments_2 numSegments=15 version=9.0.0 id=59c6he3dhebad46x7proh30nq userData={userData=multi}
   
   Checking only these segments: _gx:
   7 of 15: name=_gx maxDoc=119789
       version=9.0.0
       id=59c6he3dhebad46x7proh30n7
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=129.046
       diagnostics = {os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11, os=Mac OS X, timestamp=1622102767300, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge}
       no deletions
       test: open reader.........OK [took 0.058 sec]
       test: check integrity.....FAILED
       WARNING: exorciseIndex() would remove reference to this segment; full exception:
   org.apache.lucene.index.CorruptIndexException: checksum failed (hardware problem?) : expected=87e2aa4 actual=7b3afcbd (resource=BufferedChecksumIndexInput(MMapIndexInput(path="/Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/_gx_Lucene90_0.dvd")))
   	at org.apache.lucene.codecs.CodecUtil.checkFooter(CodecUtil.java:440)
   	at org.apache.lucene.codecs.CodecUtil.checksumEntireFile(CodecUtil.java:614)
   	at org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer.checkIntegrity(Lucene90DocValuesProducer.java:1656)
   	at org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat$FieldsReader.checkIntegrity(PerFieldDocValuesFormat.java:364)
   	at org.apache.lucene.index.CodecReader.checkIntegrity(CodecReader.java:252)
   	at org.apache.lucene.index.SegmentReader.checkIntegrity(SegmentReader.java:391)
   	at org.apache.lucene.index.CheckIndex.testSegment(CheckIndex.java:925)
   	at org.apache.lucene.index.CheckIndex.lambda$checkIndex$1(CheckIndex.java:756)
   	at org.apache.lucene.index.CheckIndex.lambda$callableToSupplier$2(CheckIndex.java:854)
   	at java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1700)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at java.base/java.lang.Thread.run(Thread.java:834)
   
   WARNING: 1 broken segments (containing 119789 documents) detected
   Took 0.347 sec total.
   WARNING: would write new segments file, and 119789 documents would be lost, if -exorcise were specified
   
   
   
   Execution failed for task ':lucene:core:CheckIndex.main()'.
   > Process 'command '/Library/Java/JavaVirtualMachines/adoptopenjdk-11.jdk/Contents/Home/bin/java'' finished with non-zero exit value 1
   
   
   
   ```




-- 
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.

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


[GitHub] [lucene] mikemccand commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r638785020



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -216,6 +225,9 @@
 
       /** Status of vectors */
       public VectorValuesStatus vectorValuesStatus;
+
+      /** Status of soft deletes */
+      public SoftDeletsStatus softDeletesStatus;

Review comment:
       Ahh, that's great!  Thanks.




-- 
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.

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


[GitHub] [lucene] dweiss commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-849741666


   bq. LOL!! Now what am I going to do with my 117 remaining cores!!
   
   I think this is very appropriate:
   https://en.wikipedia.org/wiki/Core_War


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-856394004


   Hi @mikemccand, I have addressed the comment above with some additional changes and posted some updated results. Could you please let me know if they look ready?


-- 
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.

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


[GitHub] [lucene] dweiss commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-849356353


   There is a typo in "deleteions"?


-- 
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.

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


[GitHub] [lucene] rmuir commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
rmuir commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r628122895



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -488,8 +503,35 @@ public Status checkIndex() throws IOException {
    *     quite a long time to run.
    */
   public Status checkIndex(List<String> onlySegments) throws IOException {

Review comment:
       please don't overload it here.  force callers to pass executor.
   
   I am still worried that tests are using multiple threads here which must not happen: on an 8 core machine we don't want to use 8 jvms * 8 threads each. 
   
   I am also concerned about newly-created synchronization issues here (e.g. with output). If checkindex fails it is really important that we can read this output.
   
   All these changes leave my concerned that this is the right way to go. At the minimal, if we are going to make these changes to CheckIndex, then it needs to be done in a more cautious/defensive way (e.g. default threadcount to 1 instead of numcpus). 




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r696247145



##########
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) {

Review comment:
       Sounds good! I've created a new issue for following up on this https://issues.apache.org/jira/browse/LUCENE-10071 




-- 
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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r698012354



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
##########
@@ -321,6 +326,11 @@ public static void syncConcurrentMerges(MergeScheduler ms) {
       checker.setDoSlowChecks(doSlowChecks);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false);
+      if (concurrent) {
+        checker.setThreadCount(RandomizedTest.randomIntBetween(1, 5));
+      } else {
+        checker.setThreadCount(0);

Review comment:
       Great thanks @zacharymorn  -- that sounds like a good approach.




-- 
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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r663317429



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -843,6 +812,258 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     return result;
   }
 
+  private void updateMaxSegmentName(Status result, SegmentCommitInfo info) {
+    long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
+    if (segmentName > result.maxSegmentName) {
+      result.maxSegmentName = segmentName;
+    }
+  }
+
+  private void processSegmentInfoStatusResult(
+      Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) {
+    result.segmentInfos.add(segmentInfoStatus);
+    if (segmentInfoStatus.error != null) {
+      result.totLoseDocCount += segmentInfoStatus.toLoseDocCount;
+      result.numBadSegments++;
+    } else {
+      // Keeper
+      result.newSegments.add(info.clone());
+    }
+  }
+
+  private <R> CompletableFuture<R> runAsyncSegmentCheck(
+      Callable<R> asyncCallable, ExecutorService executorService) {
+    return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService);
+  }
+
+  private <T> Supplier<T> callableToSupplier(Callable<T> callable) {
+    return () -> {
+      try {
+        return callable.call();
+      } catch (RuntimeException | Error e) {
+        throw e;
+      } catch (Throwable e) {
+        throw new CompletionException(e);
+      }
+    };
+  }
+
+  private Status.SegmentInfoStatus testSegment(
+      SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException {
+    Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
+    segInfoStat.name = info.info.name;
+    segInfoStat.maxDoc = info.info.maxDoc();
+
+    final Version version = info.info.getVersion();
+    if (info.info.maxDoc() <= 0) {
+      throw new CheckIndexException(" illegal number of documents: maxDoc=" + info.info.maxDoc());
+    }
+
+    int toLoseDocCount = info.info.maxDoc();

Review comment:
       Hi @mikemccand, just want to check back on this and see if you have any further concern?




-- 
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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r696251777



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -3787,8 +4032,12 @@ public int doCheck(Options opts) throws IOException, InterruptedException {
     setDoSlowChecks(opts.doSlowChecks);
     setChecksumsOnly(opts.doChecksumsOnly);
     setInfoStream(opts.out, opts.verbose);
+    // when threadCount was not provided via command line, override it with 0 to turn of concurrent

Review comment:
       Love the sed command reference! :D Fixed. 
   
   




-- 
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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851028668


   I've gone ahead and reverted the changes to parallelize within segment, and then added the code that used many of the same ideas to parallelize across segments - with 11 threads the total runtime has been cut down to 130+ seconds, around 65% reduction! I also removed the restriction of using up to 11 threads, as large index may well contain more than 11 segments, so idle cores can be utilized as well!
   
   Please let me know how this looks to you. @mikemccand @rmuir @dweiss 
   
   > I'm gonna throw out the crazy idea to make -fast the new default. The previous -slow could be moved to -slower and the previous current behavior could be activated by -slow.
   > I think the tool's defaults are unnecessarily slow just for historical reasons? (not having checksums originally)
   
   This also makes sense and seems to be an easy change to switch the default? Is there anything I need to add specifically so that users can be made aware of this change when they upgrade lucene version (e.g. extra log to indicate the switch) ?


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r627959173



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -731,40 +747,188 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testLiveDocs(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      liveDocStatus -> {
+                        segInfoStat.liveDocStatus = liveDocStatus;
+                      });
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testFieldInfos(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      fieldInfoStatus -> {
+                        segInfoStat.fieldInfoStatus = fieldInfoStatus;
+                      });
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testFieldNorms(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      fieldNormStatus -> {
+                        segInfoStat.fieldNormStatus = fieldNormStatus;
+                      });
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testPostings(
+                              finalReader, infoStream, verbose, doSlowChecks, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      termIndexStatus -> {
+                        segInfoStat.termIndexStatus = termIndexStatus;
+                      });
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testStoredFields(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      storedFieldStatus -> {
+                        segInfoStat.storedFieldStatus = storedFieldStatus;
+                      });
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testTermVectors(
+                              finalReader, infoStream, verbose, doSlowChecks, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      termVectorStatus -> {
+                        segInfoStat.termVectorStatus = termVectorStatus;
+                      });
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testDocValues(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      docValuesStatus -> {
+                        segInfoStat.docValuesStatus = docValuesStatus;
+                      });
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              CompletableFuture.supplyAsync(
+                      () -> {

Review comment:
       I also feel this is quite repetitive and cumbersome to look at.  I gave it some more tries but it seems to be a bit difficult actually. The main issue here is the `testXXX` methods all throw checked `IOException`, and thus any callable I define that wrap around the method will need to handle the IOException _there_ as well, instead of handling it inside the utility method, or the compiler may not be happy:
   
   ```
   utilityMethod(() -> {
         try {
              testXXX
         } catch (IOException) {
              handle IOE
         })
   ```
   
   Maybe some refactoring in those `testXXX` methods can work around the issue?
   
   For the code in `thenAccept`, we should be able to simplify with generics and creating a new `CheckIndexStatus` that all status classes extend (right now they don't share a super class).




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851551676


   Also, note that the "first segment" might not necessarily always be the largest!  The way our default merge policy works, that tends to be the case, but users can customize the merge policy, and we recently added a feature to let `IndexWriter` customize how it sorts the segments.  We should maybe sort the segments by descending `docCount` and work on them (concurrently) in that order?


-- 
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.

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


[GitHub] [lucene] mikemccand commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-849611238


   > -threadCount currently only supports up to 11 threads. Value higher than that will be capped.
   
   LOL!!  Now what am I going to do with my 117 remaining cores!!


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r647068377



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -843,6 +812,258 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     return result;
   }
 
+  private void updateMaxSegmentName(Status result, SegmentCommitInfo info) {
+    long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
+    if (segmentName > result.maxSegmentName) {
+      result.maxSegmentName = segmentName;
+    }
+  }
+
+  private void processSegmentInfoStatusResult(
+      Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) {
+    result.segmentInfos.add(segmentInfoStatus);
+    if (segmentInfoStatus.error != null) {
+      result.totLoseDocCount += segmentInfoStatus.toLoseDocCount;
+      result.numBadSegments++;
+    } else {
+      // Keeper
+      result.newSegments.add(info.clone());
+    }
+  }
+
+  private <R> CompletableFuture<R> runAsyncSegmentCheck(
+      Callable<R> asyncCallable, ExecutorService executorService) {
+    return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService);
+  }
+
+  private <T> Supplier<T> callableToSupplier(Callable<T> callable) {
+    return () -> {
+      try {
+        return callable.call();
+      } catch (RuntimeException | Error e) {
+        throw e;
+      } catch (Throwable e) {
+        throw new CompletionException(e);
+      }
+    };
+  }
+
+  private Status.SegmentInfoStatus testSegment(
+      SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException {
+    Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
+    segInfoStat.name = info.info.name;
+    segInfoStat.maxDoc = info.info.maxDoc();
+
+    final Version version = info.info.getVersion();
+    if (info.info.maxDoc() <= 0) {
+      throw new CheckIndexException(" 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.);
+      // nf#format is not thread-safe, and would generate random non valid results in concurrent
+      // setting
+      synchronized (nf) {
+        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);
+      }
+
+      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)));
+
+      segInfoStat.openReaderPassed = true;
+
+      long startIntegrityNS = System.nanoTime();
+      if (infoStream != null) infoStream.print("    test: check integrity.....");
+      reader.checkIntegrity();
+      msg(
+          infoStream,
+          String.format(
+              Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime() - startIntegrityNS)));
+
+      if (reader.maxDoc() != info.info.maxDoc()) {
+        throw new CheckIndexException(
+            "SegmentReader.maxDoc() "
+                + reader.maxDoc()
+                + " != SegmentInfo.maxDoc "
+                + info.info.maxDoc());
+      }
+
+      final int numDocs = reader.numDocs();
+      toLoseDocCount = numDocs;

Review comment:
       Same as above https://github.com/apache/lucene/pull/128#discussion_r642722424, this might be intended behavior?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -181,6 +193,9 @@
       /** True if we were able to open a CodecReader on this segment. */
       public boolean openReaderPassed;
 
+      /** doc count in this segment */

Review comment:
       Same as above https://github.com/apache/lucene/pull/128#discussion_r642722424, this might be intended behavior?




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r696246368



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
##########
@@ -321,6 +326,11 @@ public static void syncConcurrentMerges(MergeScheduler ms) {
       checker.setDoSlowChecks(doSlowChecks);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false);
+      if (concurrent) {
+        checker.setThreadCount(RandomizedTest.randomIntBetween(1, 5));
+      } else {
+        checker.setThreadCount(0);

Review comment:
       Currently in the code, it is using `0` to signal turning off concurrent checking (using the main thread to check index), and it is also used as default when users don't explicitly pass in `-threadCount` flag.
   
   In `CheckIndex#doCheck(Options)`, here's the logic to overwrite default thread count with user provided value (`0` when not specified)
   ```
   // when threadCount was not provided via command line, override it with 0 to turn of concurrent
   // check
   setThreadCount(opts.threadCount);
   ```
   
   and later in `CheckIndex#checkIndex(List)`, it is used as such
   
   ```
   public Status checkIndex(List<String> onlySegments) throws IOException {
       ExecutorService executorService = null;
   
       if (threadCount > 0) {
         executorService =
             Executors.newFixedThreadPool(threadCount, new NamedThreadFactory("async-check-index"));
       }
   
       try {
         return checkIndex(onlySegments, executorService);
       } finally {
         ...
       }
   }
   ```
   
   ---
   
   If we were to use `1`  to signal checking sequentially / turning off concurrency, then I would imagine users will need to use `-threadCount 2` when they meant "create 1 more thread in addition to main thread to check index", and in general N+1 when they were thinking about creating N additional threads right? I feel this might be a bit unintuitive? 
   




-- 
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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r642646093



##########
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:
       > Hmm, could we instead just output the full segment's output as each finishes? So we would presumably see small segments finish first, then all checks for that small segment are printed?
   
   > Also, note that the "first segment" might not necessarily always be the largest! The way our default merge policy works, that tends to be the case, but users can customize the merge policy, and we recently added a feature to let IndexWriter customize how it sorts the segments. We should maybe sort the segments by descending docCount and work on them (concurrently) in that order?
   
   
   Hmm ok, and good to know about the custom segment sorting feature! The current implementation actually ensures the output to follow the same segment order as if it were executed sequentially, hence the little trick there to print first segment output early. But yeah I guess this order requirement is not really needed, especially under concurrent setting. Let me update that to give better output (and maybe even fail faster if smaller segment encounters error early!).




-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r634957781



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -2795,12 +2972,14 @@ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
    * @lucene.experimental
    */
   public static Status.DocValuesStatus testDocValues(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {

Review comment:
       I also think it should be ok to backport. The only thing I would like to mention is that, in addition to the API change, the more subtle change is that these methods would now no longer throw unchecked RuntimeException when the check find index integrity error, and `failFast` set to `true`. For any application that (should not have) relied on this behavior, it may now find the check would continue processing instead of aborting with exception when the check finds an error. 




-- 
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.

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


[GitHub] [lucene] dweiss commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
dweiss commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r627177953



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -810,6 +976,8 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
       result.newSegments.add(info.clone());
     }
 
+    executor.shutdown();

Review comment:
       Add await; shutdown is non-blocking.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -605,6 +610,15 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     result.newSegments.clear();
     result.maxSegmentName = -1;
 
+    // nocommit number of threads should be set dynamically

Review comment:
       I'd say: set dynamically based on CPU count, but provide an option (command line) to override?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -731,40 +747,188 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testLiveDocs(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      liveDocStatus -> {
+                        segInfoStat.liveDocStatus = liveDocStatus;
+                      });
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testFieldInfos(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      fieldInfoStatus -> {
+                        segInfoStat.fieldInfoStatus = fieldInfoStatus;
+                      });
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testFieldNorms(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      fieldNormStatus -> {
+                        segInfoStat.fieldNormStatus = fieldNormStatus;
+                      });
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testPostings(
+                              finalReader, infoStream, verbose, doSlowChecks, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      termIndexStatus -> {
+                        segInfoStat.termIndexStatus = termIndexStatus;
+                      });
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testStoredFields(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      storedFieldStatus -> {
+                        segInfoStat.storedFieldStatus = storedFieldStatus;
+                      });
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testTermVectors(
+                              finalReader, infoStream, verbose, doSlowChecks, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      termVectorStatus -> {
+                        segInfoStat.termVectorStatus = termVectorStatus;
+                      });
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testDocValues(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      docValuesStatus -> {
+                        segInfoStat.docValuesStatus = docValuesStatus;
+                      });
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              CompletableFuture.supplyAsync(
+                      () -> {

Review comment:
       maybe add a utility method that accepts a callable and wraps in completion exception? This is a very repetitive block.




-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r627960267



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -810,6 +976,8 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
       result.newSegments.add(info.clone());
     }
 
+    executor.shutdown();

Review comment:
       Oops sorry. Thought it would be ok with the `join` call before `shutdown`, but yeah even the idle threads may take time to shutdown. Added.




-- 
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.

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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-843795302


   > I love this change -- `CheckIndex` is often run in the "heat of the moment" so time is of the essence to recovering your Lucene index and getting things back online. Yet it is [very slow today, even on highly concurrent boxes](https://home.apache.org/~mikemccand/lucenebench/checkIndexTime.html).
   > 
   > I left some small comments that I think are fine to do in a followon PR. This change is already massive enough (GitHub was at first not willing to even render the `CheckIndex` diffs!) and impactful enough that we can do the improvements after.
   
   Thanks Michael for the review and feedback. I think as far as the original scope of the jira ticket goes, there's also the parallelization across segments that has not been implemented yet. But agree that this PR is already big and should already provide a good speed up on powerful concurrent boxes (up to 11 concurrent checks for each segment), so we can probably let it run for a while and see if parallelization across segments is still needed, which from my quick in-mind coding will definitely require much more changes for concurrency control to get it right.
   
   One thing I'm still researching is that, it seems there's limited direct test coverage for this `CheckIndex`  class? I see there's `TestCheckIndex`, but it only has 4 tests, and the majority of its functionalities seems to be put under tests by other index testing utilities and test cases. Shall I still add a few more tests for these changes (and should I put them in `TestCheckIndex`)? On the other hand, I've been running `./gradlew check -Ptests.nightly=true -Pvalidation.git.failOnModified=false` the whole time and all tests have been passing.


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r628696357



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -731,40 +747,188 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testLiveDocs(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      liveDocStatus -> {
+                        segInfoStat.liveDocStatus = liveDocStatus;
+                      });
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testFieldInfos(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      fieldInfoStatus -> {
+                        segInfoStat.fieldInfoStatus = fieldInfoStatus;
+                      });
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testFieldNorms(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      fieldNormStatus -> {
+                        segInfoStat.fieldNormStatus = fieldNormStatus;
+                      });
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testPostings(
+                              finalReader, infoStream, verbose, doSlowChecks, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      termIndexStatus -> {
+                        segInfoStat.termIndexStatus = termIndexStatus;
+                      });
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testStoredFields(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      storedFieldStatus -> {
+                        segInfoStat.storedFieldStatus = storedFieldStatus;
+                      });
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testTermVectors(
+                              finalReader, infoStream, verbose, doSlowChecks, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      termVectorStatus -> {
+                        segInfoStat.termVectorStatus = termVectorStatus;
+                      });
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testDocValues(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      docValuesStatus -> {
+                        segInfoStat.docValuesStatus = docValuesStatus;
+                      });
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              CompletableFuture.supplyAsync(
+                      () -> {

Review comment:
       Ah ok I see. I think I encountered the issue earlier when I tried to use `Supplier` directly instead of `Callable` there. I've updated it according to your suggestion (I used a `Function` for the follow-up lambda as it takes one argument)




-- 
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.

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


[GitHub] [lucene] mikemccand commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r638788325



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -305,7 +317,7 @@ private FieldNormStatus() {}
       public long totVectors = 0;
 
       /** Exception thrown during term vector test (null on success) */
-      public Throwable error = null;

Review comment:
       Thank you for fixing all of these!

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -731,74 +810,173 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testLiveDocs(finalReader, infoStream, segmentId),
+                  liveDocStatus -> segInfoStat.liveDocStatus = liveDocStatus);
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldInfos(finalReader, infoStream, segmentId),
+                  fieldInfoStatus -> segInfoStat.fieldInfoStatus = fieldInfoStatus);
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldNorms(finalReader, infoStream, segmentId),
+                  fieldNormStatus -> segInfoStat.fieldNormStatus = fieldNormStatus);
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPostings(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termIndexStatus -> segInfoStat.termIndexStatus = termIndexStatus);
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testStoredFields(finalReader, infoStream, segmentId),
+                  storedFieldStatus -> segInfoStat.storedFieldStatus = storedFieldStatus);
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testTermVectors(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termVectorStatus -> segInfoStat.termVectorStatus = termVectorStatus);
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testDocValues(finalReader, infoStream, segmentId),
+                  docValuesStatus -> segInfoStat.docValuesStatus = docValuesStatus);
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPoints(finalReader, infoStream, segmentId),
+                  pointsStatus -> segInfoStat.pointsStatus = pointsStatus);
 
           // Test VectorValues
-          segInfoStat.vectorValuesStatus = testVectors(reader, infoStream, failFast);
+          CompletableFuture<Void> testVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testVectors(finalReader, infoStream, segmentId),
+                  vectorValuesStatus -> segInfoStat.vectorValuesStatus = vectorValuesStatus);
 
           // Test index sort
-          segInfoStat.indexSortStatus = testSort(reader, indexSort, infoStream, failFast);
+          CompletableFuture<Void> testSort =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testSort(finalReader, indexSort, infoStream, segmentId),
+                  indexSortStatus -> segInfoStat.indexSortStatus = indexSortStatus);
+
+          CompletableFuture<Void> testSoftDeletes = null;
+          final String softDeletesField = reader.getFieldInfos().getSoftDeletesField();
+          if (softDeletesField != null) {
+            testSoftDeletes =
+                runAsyncSegmentPartCheck(
+                    executorService,
+                    () ->
+                        checkSoftDeletes(
+                            softDeletesField, info, finalReader, infoStream, segmentId),
+                    softDeletesStatus -> segInfoStat.softDeletesStatus = softDeletesStatus);
+          }
 
           // Rethrow the first exception we encountered
           //  This will cause stats for failed segments to be incremented properly
+          testliveDocs.join();
           if (segInfoStat.liveDocStatus.error != null) {
-            throw new RuntimeException("Live docs test failed");
-          } else if (segInfoStat.fieldInfoStatus.error != null) {
-            throw new RuntimeException("Field Info test failed");
-          } else if (segInfoStat.fieldNormStatus.error != null) {
-            throw new RuntimeException("Field Norm test failed");
-          } else if (segInfoStat.termIndexStatus.error != null) {
-            throw new RuntimeException("Term Index test failed");
-          } else if (segInfoStat.storedFieldStatus.error != null) {
-            throw new RuntimeException("Stored Field test failed");
-          } else if (segInfoStat.termVectorStatus.error != null) {
-            throw new RuntimeException("Term Vector test failed");
-          } else if (segInfoStat.docValuesStatus.error != null) {
-            throw new RuntimeException("DocValues test failed");
-          } else if (segInfoStat.pointsStatus.error != null) {
-            throw new RuntimeException("Points test failed");
+            throw new CheckIndexException(
+                segmentId, "", "Live docs test failed", segInfoStat.liveDocStatus.error);
+          }
+
+          testFieldInfos.join();
+          if (segInfoStat.fieldInfoStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Field Info test failed", segInfoStat.fieldInfoStatus.error);
+          }
+
+          testFieldNorms.join();
+          if (segInfoStat.fieldNormStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Field Norm test failed", segInfoStat.fieldNormStatus.error);
+          }
+
+          testTermIndex.join();
+          if (segInfoStat.termIndexStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Term Index test failed", segInfoStat.termIndexStatus.error);
+          }
+
+          testStoredFields.join();
+          if (segInfoStat.storedFieldStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Stored Field test failed", segInfoStat.storedFieldStatus.error);
+          }
+
+          testTermVectors.join();
+          if (segInfoStat.termVectorStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Term Vector test failed", segInfoStat.termVectorStatus.error);

Review comment:
       I think instead of empty string for `partId` param we can pass e.g. `"field infos"`, "term vectors"`, etc.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -1058,16 +1261,14 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
 
       msg(
           infoStream,
+          segmentId + partId,
           String.format(
               Locale.ROOT,
               "OK [%d fields] [took %.3f sec]",
               status.totFields,
               nsToSec(System.nanoTime() - startNS)));
     } catch (Throwable e) {
-      if (failFast) {
-        throw IOUtils.rethrowAlways(e);
-      }
-      msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
+      msg(infoStream, segmentId + partId, "ERROR [" + String.valueOf(e.getMessage()) + "]");

Review comment:
       Hmm if `partId` is a simple string like `"term vectors"` but then sometimes empty string, maybe we need to make a helper method to carefully join them?  E.g. if `partId` is `null` or `""` then we leave it off, but if it is not then we maybe print something like `_7 [term vectors]: ...`?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -700,29 +771,37 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
 
         if (reader.hasDeletions()) {
           if (reader.numDocs() != info.info.maxDoc() - info.getDelCount()) {
-            throw new RuntimeException(
+            throw new CheckIndexException(
+                segmentId,
+                "",

Review comment:
       You could maybe pass `"live docs"` as the `partId` here?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -731,74 +810,173 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testLiveDocs(finalReader, infoStream, segmentId),
+                  liveDocStatus -> segInfoStat.liveDocStatus = liveDocStatus);
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldInfos(finalReader, infoStream, segmentId),
+                  fieldInfoStatus -> segInfoStat.fieldInfoStatus = fieldInfoStatus);
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldNorms(finalReader, infoStream, segmentId),
+                  fieldNormStatus -> segInfoStat.fieldNormStatus = fieldNormStatus);
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPostings(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termIndexStatus -> segInfoStat.termIndexStatus = termIndexStatus);
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testStoredFields(finalReader, infoStream, segmentId),
+                  storedFieldStatus -> segInfoStat.storedFieldStatus = storedFieldStatus);
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testTermVectors(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termVectorStatus -> segInfoStat.termVectorStatus = termVectorStatus);
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testDocValues(finalReader, infoStream, segmentId),
+                  docValuesStatus -> segInfoStat.docValuesStatus = docValuesStatus);
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPoints(finalReader, infoStream, segmentId),
+                  pointsStatus -> segInfoStat.pointsStatus = pointsStatus);
 
           // Test VectorValues
-          segInfoStat.vectorValuesStatus = testVectors(reader, infoStream, failFast);
+          CompletableFuture<Void> testVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testVectors(finalReader, infoStream, segmentId),
+                  vectorValuesStatus -> segInfoStat.vectorValuesStatus = vectorValuesStatus);
 
           // Test index sort
-          segInfoStat.indexSortStatus = testSort(reader, indexSort, infoStream, failFast);
+          CompletableFuture<Void> testSort =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testSort(finalReader, indexSort, infoStream, segmentId),
+                  indexSortStatus -> segInfoStat.indexSortStatus = indexSortStatus);
+
+          CompletableFuture<Void> testSoftDeletes = null;
+          final String softDeletesField = reader.getFieldInfos().getSoftDeletesField();
+          if (softDeletesField != null) {
+            testSoftDeletes =
+                runAsyncSegmentPartCheck(
+                    executorService,
+                    () ->
+                        checkSoftDeletes(
+                            softDeletesField, info, finalReader, infoStream, segmentId),
+                    softDeletesStatus -> segInfoStat.softDeletesStatus = softDeletesStatus);
+          }
 
           // Rethrow the first exception we encountered
           //  This will cause stats for failed segments to be incremented properly
+          testliveDocs.join();
           if (segInfoStat.liveDocStatus.error != null) {
-            throw new RuntimeException("Live docs test failed");
-          } else if (segInfoStat.fieldInfoStatus.error != null) {
-            throw new RuntimeException("Field Info test failed");
-          } else if (segInfoStat.fieldNormStatus.error != null) {
-            throw new RuntimeException("Field Norm test failed");
-          } else if (segInfoStat.termIndexStatus.error != null) {
-            throw new RuntimeException("Term Index test failed");
-          } else if (segInfoStat.storedFieldStatus.error != null) {
-            throw new RuntimeException("Stored Field test failed");
-          } else if (segInfoStat.termVectorStatus.error != null) {
-            throw new RuntimeException("Term Vector test failed");
-          } else if (segInfoStat.docValuesStatus.error != null) {
-            throw new RuntimeException("DocValues test failed");
-          } else if (segInfoStat.pointsStatus.error != null) {
-            throw new RuntimeException("Points test failed");
+            throw new CheckIndexException(
+                segmentId, "", "Live docs test failed", segInfoStat.liveDocStatus.error);
+          }
+
+          testFieldInfos.join();
+          if (segInfoStat.fieldInfoStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Field Info test failed", segInfoStat.fieldInfoStatus.error);
+          }
+
+          testFieldNorms.join();
+          if (segInfoStat.fieldNormStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Field Norm test failed", segInfoStat.fieldNormStatus.error);
+          }
+
+          testTermIndex.join();
+          if (segInfoStat.termIndexStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Term Index test failed", segInfoStat.termIndexStatus.error);
+          }
+
+          testStoredFields.join();
+          if (segInfoStat.storedFieldStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Stored Field test failed", segInfoStat.storedFieldStatus.error);
+          }
+
+          testTermVectors.join();
+          if (segInfoStat.termVectorStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Term Vector test failed", segInfoStat.termVectorStatus.error);
+          }
+
+          testDocValues.join();
+          if (segInfoStat.docValuesStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "DocValues test failed", segInfoStat.docValuesStatus.error);
+          }
+
+          testPointvalues.join();
+          if (segInfoStat.pointsStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Points test failed", segInfoStat.pointsStatus.error);
+          }
+
+          testVectors.join();
+          if (segInfoStat.vectorValuesStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Vectors test failed", segInfoStat.vectorValuesStatus.error);

Review comment:
       This `partId` can be `"vectors"`.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -488,8 +503,35 @@ public Status checkIndex() throws IOException {
    *     quite a long time to run.
    */
   public Status checkIndex(List<String> onlySegments) throws IOException {

Review comment:
       Maybe post the `CheckIndex` output on a real-ish (multi-segment) index as an example?  Actually I will do that on the nightly benchmark (English Wikipedia) index!

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -731,74 +810,173 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testLiveDocs(finalReader, infoStream, segmentId),
+                  liveDocStatus -> segInfoStat.liveDocStatus = liveDocStatus);
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldInfos(finalReader, infoStream, segmentId),
+                  fieldInfoStatus -> segInfoStat.fieldInfoStatus = fieldInfoStatus);
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldNorms(finalReader, infoStream, segmentId),
+                  fieldNormStatus -> segInfoStat.fieldNormStatus = fieldNormStatus);
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPostings(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termIndexStatus -> segInfoStat.termIndexStatus = termIndexStatus);
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testStoredFields(finalReader, infoStream, segmentId),
+                  storedFieldStatus -> segInfoStat.storedFieldStatus = storedFieldStatus);
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testTermVectors(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termVectorStatus -> segInfoStat.termVectorStatus = termVectorStatus);
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testDocValues(finalReader, infoStream, segmentId),
+                  docValuesStatus -> segInfoStat.docValuesStatus = docValuesStatus);
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPoints(finalReader, infoStream, segmentId),
+                  pointsStatus -> segInfoStat.pointsStatus = pointsStatus);
 
           // Test VectorValues
-          segInfoStat.vectorValuesStatus = testVectors(reader, infoStream, failFast);
+          CompletableFuture<Void> testVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testVectors(finalReader, infoStream, segmentId),
+                  vectorValuesStatus -> segInfoStat.vectorValuesStatus = vectorValuesStatus);
 
           // Test index sort
-          segInfoStat.indexSortStatus = testSort(reader, indexSort, infoStream, failFast);
+          CompletableFuture<Void> testSort =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testSort(finalReader, indexSort, infoStream, segmentId),
+                  indexSortStatus -> segInfoStat.indexSortStatus = indexSortStatus);
+
+          CompletableFuture<Void> testSoftDeletes = null;
+          final String softDeletesField = reader.getFieldInfos().getSoftDeletesField();
+          if (softDeletesField != null) {
+            testSoftDeletes =
+                runAsyncSegmentPartCheck(
+                    executorService,
+                    () ->
+                        checkSoftDeletes(
+                            softDeletesField, info, finalReader, infoStream, segmentId),
+                    softDeletesStatus -> segInfoStat.softDeletesStatus = softDeletesStatus);
+          }
 
           // Rethrow the first exception we encountered
           //  This will cause stats for failed segments to be incremented properly
+          testliveDocs.join();
           if (segInfoStat.liveDocStatus.error != null) {
-            throw new RuntimeException("Live docs test failed");
-          } else if (segInfoStat.fieldInfoStatus.error != null) {
-            throw new RuntimeException("Field Info test failed");
-          } else if (segInfoStat.fieldNormStatus.error != null) {
-            throw new RuntimeException("Field Norm test failed");
-          } else if (segInfoStat.termIndexStatus.error != null) {
-            throw new RuntimeException("Term Index test failed");
-          } else if (segInfoStat.storedFieldStatus.error != null) {
-            throw new RuntimeException("Stored Field test failed");
-          } else if (segInfoStat.termVectorStatus.error != null) {
-            throw new RuntimeException("Term Vector test failed");
-          } else if (segInfoStat.docValuesStatus.error != null) {
-            throw new RuntimeException("DocValues test failed");
-          } else if (segInfoStat.pointsStatus.error != null) {
-            throw new RuntimeException("Points test failed");
+            throw new CheckIndexException(
+                segmentId, "", "Live docs test failed", segInfoStat.liveDocStatus.error);
+          }
+
+          testFieldInfos.join();

Review comment:
       OK, here is where we could always e.g.`System.out.print(segInfoState.fieldInfoMessages.toString())` or so?




-- 
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.

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


[GitHub] [lucene] mikemccand commented on a change in pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r640598779



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -3720,6 +3957,20 @@ public static Options parseOptions(String[] args) {
         }
         i++;
         opts.dirImpl = args[i];
+      } else if ("-threadCount".equals(arg)) {
+        if (i == args.length - 1) {
+          throw new IllegalArgumentException("-threadCount requires a following number");
+        }
+        i++;
+        int providedThreadCount = Integer.parseInt(args[i]);
+        // Current implementation supports up to 11 concurrent checks at any time, and no
+        // concurrency across segments.
+        // Capping the thread count to 11 to avoid unnecessary threads to be created.
+        if (providedThreadCount > 11) {

Review comment:
       Over time, as we add new index / codec formats, this would increase right?  But I don't think we have any simple programmatic way to "count how many *CodecFormats exist"?  Could we maybe just pull this `11` into a `static final int MAX_PER_SEGMENT_CONCURRENCY = 11;` constant at the top of the class?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -731,74 +810,173 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testLiveDocs(finalReader, infoStream, segmentId),
+                  liveDocStatus -> segInfoStat.liveDocStatus = liveDocStatus);
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldInfos(finalReader, infoStream, segmentId),
+                  fieldInfoStatus -> segInfoStat.fieldInfoStatus = fieldInfoStatus);
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldNorms(finalReader, infoStream, segmentId),
+                  fieldNormStatus -> segInfoStat.fieldNormStatus = fieldNormStatus);
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPostings(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termIndexStatus -> segInfoStat.termIndexStatus = termIndexStatus);
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testStoredFields(finalReader, infoStream, segmentId),
+                  storedFieldStatus -> segInfoStat.storedFieldStatus = storedFieldStatus);
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testTermVectors(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termVectorStatus -> segInfoStat.termVectorStatus = termVectorStatus);
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testDocValues(finalReader, infoStream, segmentId),
+                  docValuesStatus -> segInfoStat.docValuesStatus = docValuesStatus);
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPoints(finalReader, infoStream, segmentId),
+                  pointsStatus -> segInfoStat.pointsStatus = pointsStatus);
 
           // Test VectorValues
-          segInfoStat.vectorValuesStatus = testVectors(reader, infoStream, failFast);
+          CompletableFuture<Void> testVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testVectors(finalReader, infoStream, segmentId),
+                  vectorValuesStatus -> segInfoStat.vectorValuesStatus = vectorValuesStatus);
 
           // Test index sort
-          segInfoStat.indexSortStatus = testSort(reader, indexSort, infoStream, failFast);
+          CompletableFuture<Void> testSort =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testSort(finalReader, indexSort, infoStream, segmentId),
+                  indexSortStatus -> segInfoStat.indexSortStatus = indexSortStatus);
+
+          CompletableFuture<Void> testSoftDeletes = null;
+          final String softDeletesField = reader.getFieldInfos().getSoftDeletesField();
+          if (softDeletesField != null) {
+            testSoftDeletes =
+                runAsyncSegmentPartCheck(
+                    executorService,
+                    () ->
+                        checkSoftDeletes(
+                            softDeletesField, info, finalReader, infoStream, segmentId),
+                    softDeletesStatus -> segInfoStat.softDeletesStatus = softDeletesStatus);
+          }
 
           // Rethrow the first exception we encountered
           //  This will cause stats for failed segments to be incremented properly
+          testliveDocs.join();
           if (segInfoStat.liveDocStatus.error != null) {
-            throw new RuntimeException("Live docs test failed");
-          } else if (segInfoStat.fieldInfoStatus.error != null) {
-            throw new RuntimeException("Field Info test failed");
-          } else if (segInfoStat.fieldNormStatus.error != null) {
-            throw new RuntimeException("Field Norm test failed");
-          } else if (segInfoStat.termIndexStatus.error != null) {
-            throw new RuntimeException("Term Index test failed");
-          } else if (segInfoStat.storedFieldStatus.error != null) {
-            throw new RuntimeException("Stored Field test failed");
-          } else if (segInfoStat.termVectorStatus.error != null) {
-            throw new RuntimeException("Term Vector test failed");
-          } else if (segInfoStat.docValuesStatus.error != null) {
-            throw new RuntimeException("DocValues test failed");
-          } else if (segInfoStat.pointsStatus.error != null) {
-            throw new RuntimeException("Points test failed");
+            throw new CheckIndexException(
+                segmentId, "", "Live docs test failed", segInfoStat.liveDocStatus.error);
+          }
+
+          testFieldInfos.join();

Review comment:
       Woot!




-- 
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.

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


[GitHub] [lucene] mikemccand commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-841300744


   I am excited to see what happens to [`CheckIndex` time in Lucene's nightly benchmarks](https://home.apache.org/~mikemccand/lucenebench/checkIndexTime.html) after we push this!  But I agree we must also not crush the more common case of machines that don't have tons of cores ...


-- 
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.

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


[GitHub] [lucene] dweiss commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
dweiss commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r628015890



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -731,40 +747,188 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testLiveDocs(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      liveDocStatus -> {
+                        segInfoStat.liveDocStatus = liveDocStatus;
+                      });
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testFieldInfos(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      fieldInfoStatus -> {
+                        segInfoStat.fieldInfoStatus = fieldInfoStatus;
+                      });
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testFieldNorms(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      fieldNormStatus -> {
+                        segInfoStat.fieldNormStatus = fieldNormStatus;
+                      });
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testPostings(
+                              finalReader, infoStream, verbose, doSlowChecks, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      termIndexStatus -> {
+                        segInfoStat.termIndexStatus = termIndexStatus;
+                      });
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testStoredFields(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      storedFieldStatus -> {
+                        segInfoStat.storedFieldStatus = storedFieldStatus;
+                      });
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testTermVectors(
+                              finalReader, infoStream, verbose, doSlowChecks, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      termVectorStatus -> {
+                        segInfoStat.termVectorStatus = termVectorStatus;
+                      });
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              CompletableFuture.supplyAsync(
+                      () -> {
+                        try {
+                          return testDocValues(finalReader, infoStream, failFast);
+                        } catch (IOException e) {
+                          throw new CompletionException(e);
+                        }
+                      },
+                      executor)
+                  .thenAccept(
+                      docValuesStatus -> {
+                        segInfoStat.docValuesStatus = docValuesStatus;
+                      });
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              CompletableFuture.supplyAsync(
+                      () -> {

Review comment:
       I added a commit that shows what I meant (and replaces three calls). Sorry for not replacing all of them. Maybe it can be made even less verbose if you fold all those blocks into a single function that accepts executor, first callable, follow-up callable and just returns completable future from the composition of these.




-- 
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.

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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-849375766


   > There is a typo in "deleteions"?
   
   Fixed.


-- 
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.

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


[GitHub] [lucene] msokolov commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-849694198


   Not bitcoin mining I hope
   
   On Thu, May 27, 2021 at 8:56 AM Michael McCandless ***@***.***>
   wrote:
   
   > -threadCount currently only supports up to 11 threads. Value higher than
   > that will be capped.
   >
   > LOL!! Now what am I going to do with my 117 remaining cores!!
   >
   > —
   > You are receiving this because you are subscribed to this thread.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/lucene/pull/128#issuecomment-849611238>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/AAHHUQNLVSCOXD754HTBBLDTPY6PHANCNFSM44GLV3QQ>
   > .
   >
   


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-843836186


   > > ./gradlew check -Ptests.nightly=true -Pvalidation.git.failOnModified=false
   > 
   > 
   > 
   > Hi Zach. You can also "git add -A ." (stage your changes for commit); or just commit them in. Then there's  no need for the fail-on-modified flag to be turned off. :)
   
   Ha yes I came to realize that also some time ago, but kinda formed the habit of passing it in by default now (mostly from past command search) so that I don't need to worry about where my changes are. But yeah that's good tip. Thanks Dawid!


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r696243823



##########
File path: lucene/core/src/test/org/apache/lucene/index/TestCheckIndex.java
##########
@@ -54,4 +65,137 @@ public void testChecksumsOnlyVerbose() throws IOException {
   public void testObtainsLock() throws IOException {
     testObtainsLock(directory);
   }
+
+  @Test
+  public void testCheckIndexAllValid() throws Exception {

Review comment:
       No problem! As I iterated through the changes, I see the need to create a test to guard against any unintentional bug, hence creating one here.




-- 
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


[GitHub] [lucene] zacharymorn edited a comment on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn edited a comment on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-848482687


   Thanks @mikemccand for the feedback comment, as well as running the index check and posting the results here! I was planning to do that next after adding some more tests, but you beat me to it.
   
   > More tests are always welcome! But I don't think that should block this change -- we can open a follow-on issue to get better direct unit tests for CheckIndex? Those would be fun tests to write: make a healthy index, then make a random single bit change to one of its files, and then see if CheckIndex catches it. Hmm I think we have such a test somewhere :) But not apparently in BaseTestCheckIndex...
   
   Sounds good. Yes I think I came across them before, but didn't recall now exactly where they are now...but will work on them in a follow-up PR.
   
   
   > I think the issue is that opts.threadCount is 0 if you don't explicitly set the thread count. Can we fix it to default to number of cores on the box, maybe capped at a maximum (4?
   8?), when CheckIndex is invoked interactively from the command-line?
   
   Ah sorry about this (embarrassing) bug! There was a default 1 for threadCount set in the code, but when it was not provided via the command -line, the default was overwritten by 0, causing this exception to be thrown. I'll fix it and cap at 4.
   
   
   > I think we should try to remove the duplicate segment/partId (e.g.[Segment 614][StoredFields]) in some lines? 
   
   > But the output is jumbled, I think because we are missing newlines somewhere, or maybe necessary locking?
   
   Yes these repeated segment / part ids are due to concurrent threads printing messages without newlines:
   https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java#L934. I also took a look at the implementation of `PrintStream#println` and it is synchronized on `this`, so it shouldn't require additional locking (assuming its sub classes are also similar). 
   
   I think this is indeed a problem with this approach to use segment / part id to organize messages, as it still requires certain way of printing the messages. I'll switch over to the other approach then to use per part buffer. 
   
   > Hmm, also why are we calling it Segment 614 when its name is _h2? Hmm, is that the decimal translation of the base36 value?
   
   This was due to `segmentName` was used for id there, but it should actually use `info.info.name` . I'll fix that.
   
   
   
   > Finally I ran CheckIndex -threadCount 128
   
   > It went a bit faster! (203 down to 176 seconds).
   
   Glad to see it actually improved the speed there :D ! I think 128 might be too big of a threadCount though for the current implementation, as it only parallelize (up to 11) part checking in each segment at a time. I'll cap the threadCount to 11 and print out a message to alert the user if big value was passed in as well 
   
   
   


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r696248395



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -843,6 +825,258 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     return result;
   }
 
+  private void updateMaxSegmentName(Status result, SegmentCommitInfo info) {
+    long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
+    if (segmentName > result.maxSegmentName) {
+      result.maxSegmentName = segmentName;
+    }
+  }
+
+  private void processSegmentInfoStatusResult(
+      Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) {
+    result.segmentInfos.add(segmentInfoStatus);
+    if (segmentInfoStatus.error != null) {
+      result.totLoseDocCount += segmentInfoStatus.toLoseDocCount;
+      result.numBadSegments++;
+    } else {
+      // Keeper
+      result.newSegments.add(info.clone());
+    }
+  }
+
+  private <R> CompletableFuture<R> runAsyncSegmentCheck(
+      Callable<R> asyncCallable, ExecutorService executorService) {
+    return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService);
+  }
+
+  private <T> Supplier<T> callableToSupplier(Callable<T> callable) {
+    return () -> {
+      try {
+        return callable.call();
+      } catch (RuntimeException | Error e) {
+        throw e;
+      } catch (Throwable e) {
+        throw new CompletionException(e);
+      }
+    };
+  }
+
+  private Status.SegmentInfoStatus testSegment(
+      SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException {
+    Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
+    segInfoStat.name = info.info.name;
+    segInfoStat.maxDoc = info.info.maxDoc();
+
+    final Version version = info.info.getVersion();
+    if (info.info.maxDoc() <= 0) {
+      throw new CheckIndexException(" 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.);
+      // nf#format is not thread-safe, and would generate random non valid results in concurrent
+      // setting
+      synchronized (nf) {
+        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);
+      }
+
+      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)));
+
+      segInfoStat.openReaderPassed = true;
+
+      long startIntegrityNS = System.nanoTime();
+      if (infoStream != null) infoStream.print("    test: check integrity.....");
+      reader.checkIntegrity();
+      msg(
+          infoStream,
+          String.format(
+              Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime() - startIntegrityNS)));
+
+      if (reader.maxDoc() != info.info.maxDoc()) {
+        throw new CheckIndexException(
+            "SegmentReader.maxDoc() "
+                + reader.maxDoc()
+                + " != SegmentInfo.maxDoc "
+                + info.info.maxDoc());
+      }
+
+      final int numDocs = reader.numDocs();
+      toLoseDocCount = numDocs;
+
+      if (reader.hasDeletions()) {
+        if (reader.numDocs() != info.info.maxDoc() - info.getDelCount()) {
+          throw new CheckIndexException(
+              "delete count mismatch: info="
+                  + (info.info.maxDoc() - info.getDelCount())
+                  + " vs reader="
+                  + reader.numDocs());
+        }
+        if ((info.info.maxDoc() - reader.numDocs()) > reader.maxDoc()) {
+          throw new CheckIndexException(
+              "too many deleted docs: maxDoc()="
+                  + reader.maxDoc()
+                  + " vs del count="
+                  + (info.info.maxDoc() - reader.numDocs()));
+        }
+        if (info.info.maxDoc() - reader.numDocs() != info.getDelCount()) {
+          throw new CheckIndexException(
+              "delete count mismatch: info="
+                  + info.getDelCount()
+                  + " vs reader="
+                  + (info.info.maxDoc() - reader.numDocs()));
+        }
+      } else {
+        if (info.getDelCount() != 0) {
+          throw new CheckIndexException(
+              "delete count mismatch: info="
+                  + info.getDelCount()
+                  + " vs reader="
+                  + (info.info.maxDoc() - reader.numDocs()));
+        }
+      }
+
+      if (checksumsOnly == false) {
+        // This redundant assignment is done to make compiler happy
+        SegmentReader finalReader = reader;

Review comment:
       Ah yes this is left over when I was previously using lambda for each test. I've removed this. 




-- 
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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r634956303



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -216,6 +225,9 @@
 
       /** Status of vectors */
       public VectorValuesStatus vectorValuesStatus;
+
+      /** Status of soft deletes */
+      public SoftDeletsStatus softDeletesStatus;

Review comment:
       It was checked before, but was done in a way that's different (not using status class for example) from the rest https://github.com/apache/lucene/blob/65820e5170ed15e91cc3349e6dd4da90689ecd5d/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java#L786-L789, so I went ahead and updated it to follow the same convention. 




-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r634953939



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -701,104 +765,196 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         if (reader.hasDeletions()) {
           if (reader.numDocs() != info.info.maxDoc() - info.getDelCount()) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + (info.info.maxDoc() - info.getDelCount())
                     + " vs reader="
                     + reader.numDocs());
           }
           if ((info.info.maxDoc() - reader.numDocs()) > reader.maxDoc()) {
             throw new RuntimeException(
-                "too many deleted docs: maxDoc()="
+                segmentId
+                    + "too many deleted docs: maxDoc()="
                     + reader.maxDoc()
                     + " vs del count="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
           if (info.info.maxDoc() - reader.numDocs() != info.getDelCount()) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + info.getDelCount()
                     + " vs reader="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
         } else {
           if (info.getDelCount() != 0) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + info.getDelCount()
                     + " vs reader="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testLiveDocs(finalReader, infoStream, segmentId),
+                  liveDocStatus -> segInfoStat.liveDocStatus = liveDocStatus);
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldInfos(finalReader, infoStream, segmentId),
+                  fieldInfoStatus -> segInfoStat.fieldInfoStatus = fieldInfoStatus);
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldNorms(finalReader, infoStream, segmentId),
+                  fieldNormStatus -> segInfoStat.fieldNormStatus = fieldNormStatus);
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testPostings(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termIndexStatus -> segInfoStat.termIndexStatus = termIndexStatus);
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testStoredFields(finalReader, infoStream, segmentId),
+                  storedFieldStatus -> segInfoStat.storedFieldStatus = storedFieldStatus);
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testTermVectors(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termVectorStatus -> segInfoStat.termVectorStatus = termVectorStatus);
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testDocValues(finalReader, infoStream, segmentId),
+                  docValuesStatus -> segInfoStat.docValuesStatus = docValuesStatus);
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testPoints(finalReader, infoStream, segmentId),
+                  pointsStatus -> segInfoStat.pointsStatus = pointsStatus);
 
           // Test VectorValues
-          segInfoStat.vectorValuesStatus = testVectors(reader, infoStream, failFast);
+          CompletableFuture<Void> testVectors =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testVectors(finalReader, infoStream, segmentId),
+                  vectorValuesStatus -> segInfoStat.vectorValuesStatus = vectorValuesStatus);
 
           // Test index sort
-          segInfoStat.indexSortStatus = testSort(reader, indexSort, infoStream, failFast);
+          CompletableFuture<Void> testSort =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testSort(finalReader, indexSort, infoStream, segmentId),
+                  indexSortStatus -> segInfoStat.indexSortStatus = indexSortStatus);
+
+          CompletableFuture<Void> testSoftDeletes = null;
+          final String softDeletesField = reader.getFieldInfos().getSoftDeletesField();
+          if (softDeletesField != null) {
+            testSoftDeletes =
+                runAysncSegmentPartCheck(
+                    executorService,
+                    () ->
+                        checkSoftDeletes(
+                            softDeletesField, info, finalReader, infoStream, segmentId),
+                    softDeletesStatus -> segInfoStat.softDeletesStatus = softDeletesStatus);
+          }
 
           // Rethrow the first exception we encountered
           //  This will cause stats for failed segments to be incremented properly
+          testliveDocs.join();
           if (segInfoStat.liveDocStatus.error != null) {
-            throw new RuntimeException("Live docs test failed");
-          } else if (segInfoStat.fieldInfoStatus.error != null) {
-            throw new RuntimeException("Field Info test failed");
-          } else if (segInfoStat.fieldNormStatus.error != null) {
-            throw new RuntimeException("Field Norm test failed");
-          } else if (segInfoStat.termIndexStatus.error != null) {
-            throw new RuntimeException("Term Index test failed");
-          } else if (segInfoStat.storedFieldStatus.error != null) {
-            throw new RuntimeException("Stored Field test failed");
-          } else if (segInfoStat.termVectorStatus.error != null) {
-            throw new RuntimeException("Term Vector test failed");
-          } else if (segInfoStat.docValuesStatus.error != null) {
-            throw new RuntimeException("DocValues test failed");
-          } else if (segInfoStat.pointsStatus.error != null) {
-            throw new RuntimeException("Points test failed");
+            throw new RuntimeException(segmentId + "Live docs test failed");
+          }
+
+          testFieldInfos.join();
+          if (segInfoStat.fieldInfoStatus.error != null) {
+            throw new RuntimeException(segmentId + "Field Info test failed");

Review comment:
       Ah good catch. Updated. 




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r641274199



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -3720,6 +3957,20 @@ public static Options parseOptions(String[] args) {
         }
         i++;
         opts.dirImpl = args[i];
+      } else if ("-threadCount".equals(arg)) {
+        if (i == args.length - 1) {
+          throw new IllegalArgumentException("-threadCount requires a following number");
+        }
+        i++;
+        int providedThreadCount = Integer.parseInt(args[i]);
+        // Current implementation supports up to 11 concurrent checks at any time, and no
+        // concurrency across segments.
+        // Capping the thread count to 11 to avoid unnecessary threads to be created.
+        if (providedThreadCount > 11) {

Review comment:
       > as we add new index / codec formats, this would increase right?
   
   I think since currently most of the checks finish super fast, this max threadCount should be good for a while actually until we also support currency across segments.
   
   > Could we maybe just pull this 11 into a static final int MAX_PER_SEGMENT_CONCURRENCY = 11; constant at the top of the class?
   
   Ah yes shouldn't have left a magic number there...updated!
   




-- 
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.

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


[GitHub] [lucene] mikemccand commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r638781499



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -2795,12 +2972,14 @@ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
    * @lucene.experimental
    */
   public static Status.DocValuesStatus testDocValues(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {

Review comment:
       OK, but on finding a problem in the index, with this change, and with `failFast=true`, `CheckIndex` will still throw an exception, right?  It may just take a bit longer since it waits for all concurrent checking to finish for that one segment?




-- 
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.

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


[GitHub] [lucene] mikemccand commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-849627976


   OK I re-ran with the latest PR and I LOVE the output -- the lines come as each check finishes, so you can see what is fast/slow.  It seems postings is slowest, preceded by doc values, and everything else is super fast.
   
   It's curious to me that checking vectors seems to be super fast -- is our vectors check also checking integrity the full HNSW graph, not just the float vectors?
   
   I need a tool that randomly flips a single bit somewhere in the index, to see how `CheckIndex` now behaves on a corrupt index!  Presumably there is some compact UNIX one-liner to do this?  Maybe a perfect job for a quick Python script!


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r697971217



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
##########
@@ -321,6 +326,11 @@ public static void syncConcurrentMerges(MergeScheduler ms) {
       checker.setDoSlowChecks(doSlowChecks);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false);
+      if (concurrent) {
+        checker.setThreadCount(RandomizedTest.randomIntBetween(1, 5));
+      } else {
+        checker.setThreadCount(0);

Review comment:
       Ok after some more thoughts I do think that makes sense, particularly around the idea of this (`main` vs. `executor`) is really an implementation detail that users shouldn't need to worry about, and so `-threadCount 1` to represent single-threaded execution would be the most intuitive approach here. I've pushed a new commit to update this accordingly.
   
   Also, in the latest commit I've made it the default that when users don't specify `-threadCount` value via command line, concurrent index checking will be used on machines with more than 1 core, but the number of threads will be capped at 4 via `Math.min(Runtime.getRuntime().availableProcessors(), 4)`.  I think this default behavior was discussed in multiple places in this PR, but would like to double confirm that this is the preferred default setting we would like to have (versus sequential index checking)?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -450,6 +480,14 @@ public void setChecksumsOnly(boolean v) {
 
   private boolean checksumsOnly;
 
+  /** Set threadCount used for parallelizing index integrity checking. */
+  public void setThreadCount(int tc) {
+    threadCount = tc;

Review comment:
       Done.




-- 
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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851783171


   I've created a spinoff issue to track the default setting change https://issues.apache.org/jira/browse/LUCENE-9984


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r634957136



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -926,17 +1100,19 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
    * @lucene.experimental
    */
   public static Status.LiveDocStatus testLiveDocs(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+      CodecReader reader, PrintStream infoStream, String segmentId) {
     long startNS = System.nanoTime();
+    String segmentPartId = segmentId + "[LiveDocs]";
     final Status.LiveDocStatus status = new Status.LiveDocStatus();
 
     try {
-      if (infoStream != null) infoStream.print("    test: check live docs.....");
+      if (infoStream != null) infoStream.print(segmentPartId + "    test: check live docs.....");

Review comment:
       > Sorry about not answering the // nocommit question before.
   
   No problem, and thanks again for the review and feedback!
   
   > Ideally, all infoStream.print for a given "part" of the index checking would first append to a per-part log, and then (under lock) print to console/main infoStream as a single "block" of output? (So that we don't see confusing interleaved across segments/parts checks)?
   
   Oh I see, haven't thought about this approach before, and it sounds interesting! I assume by "per-part log" you meant an array of in-memory, per part buffers that accumulate messages over concurrent check right? If we were to combine these buffers at the end of / after the concurrent index check, we should be ok to just print them out to main InfoStream without locking?

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -926,17 +1100,19 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
    * @lucene.experimental
    */
   public static Status.LiveDocStatus testLiveDocs(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+      CodecReader reader, PrintStream infoStream, String segmentId) {
     long startNS = System.nanoTime();
+    String segmentPartId = segmentId + "[LiveDocs]";
     final Status.LiveDocStatus status = new Status.LiveDocStatus();
 
     try {
-      if (infoStream != null) infoStream.print("    test: check live docs.....");
+      if (infoStream != null) infoStream.print(segmentPartId + "    test: check live docs.....");
       final int numDocs = reader.numDocs();
       if (reader.hasDeletions()) {
         Bits liveDocs = reader.getLiveDocs();
         if (liveDocs == null) {
-          throw new RuntimeException("segment should have deletions, but liveDocs is null");
+          throw new RuntimeException(

Review comment:
       Done.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -2106,16 +2286,6 @@ static void checkImpacts(Impacts impacts, int lastTarget) {
     }
   }
 
-  /**
-   * Test the term index.
-   *
-   * @lucene.experimental
-   */
-  public static Status.TermIndexStatus testPostings(CodecReader reader, PrintStream infoStream)

Review comment:
       I think I accidentally removed it...I've restored it as well as another one.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -2737,13 +2910,14 @@ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
    * @lucene.experimental
    */
   public static Status.StoredFieldStatus testStoredFields(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+      CodecReader reader, PrintStream infoStream, String segmentId) {
     long startNS = System.nanoTime();
+    String segmentPartId = segmentId + "[StoredFields]";

Review comment:
       Done (I used `CheckIndexException` instead of `CheckIndexFailure` for naming consistency). I also replaced all `RuntimeException` in `CheckIndex` with this new exception class.




-- 
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.

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


[GitHub] [lucene] mikemccand commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-849681533


   Phew, aha!  I caught my mistake!  When I ran `CheckIndex`, I failed to pass the `.broken` index -- I was just checking the original (presumably NOT corrupt) index.  I reran with the right argument (`.broken` version), and good news!:
   
   ```
   beast3:core[main]$ java -cp ../core/build/libs/lucene-core-9.0.0-SNAPSHOT.jar org.apache.lucene.index.CheckIndex /l/indices/trunk.nightly.index.prev.broken/index -segment _gm
   
   NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
   
   Opening index @ /l/indices/trunk.nightly.index.prev.broken/index
   
   Checking index with async threadCount: 4
   0.00% total deletions; 27625038 documents; 0 deletions
   Segments file=segments_2 numSegments=15 version=9.0.0 id=4tts2a3tdztvmyrx4eavuuvca userData={userData=multi}
   
   Checking only these segments: _gm:
   9 of 15: name=_gm maxDoc=497803
       version=9.0.0
       id=4tts2a3tdztvmyrx4eavuuvau
       codec=Lucene90
       compound=false
       numFiles=20
       size (MB)=402.17
       diagnostics = {os.arch=amd64, mergeFactor=10, java.runtime.version=15.0.1+9-18, os=Linux, timestamp=1622011837396, lucene.version=9.0.0, java.vm.version=15.0.1+9-18, java.version=15.0.\
   1, mergeMaxNumSegments=-1, os.version=5.11.2-arch1-1, java.vendor=Oracle Corporation, source=merge}
       no deletions
       test: open reader.........OK [took 0.054 sec]
       test: check integrity.....FAILED
       WARNING: exorciseIndex() would remove reference to this segment; full exception:
   org.apache.lucene.index.CorruptIndexException: checksum failed (hardware problem?) : expected=681c742f actual=5f6ae61b (resource=BufferedChecksumIndexInput(MMapIndexInput(path="/l/indices/\
   trunk.nightly.index.prev.broken/index/_gm.kdi")))
           at org.apache.lucene.codecs.CodecUtil.checkFooter(CodecUtil.java:440)
           at org.apache.lucene.codecs.CodecUtil.checksumEntireFile(CodecUtil.java:614)
           at org.apache.lucene.codecs.lucene90.Lucene90PointsReader.checkIntegrity(Lucene90PointsReader.java:143)
           at org.apache.lucene.index.CodecReader.checkIntegrity(CodecReader.java:267)
           at org.apache.lucene.index.SegmentReader.checkIntegrity(SegmentReader.java:391)
           at org.apache.lucene.index.CheckIndex.checkIndex(CheckIndex.java:745)
           at org.apache.lucene.index.CheckIndex.checkIndex(CheckIndex.java:527)
           at org.apache.lucene.index.CheckIndex.doCheck(CheckIndex.java:4046)
           at org.apache.lucene.index.CheckIndex.doMain(CheckIndex.java:3919)
           at org.apache.lucene.index.CheckIndex.main(CheckIndex.java:3851)
   
   WARNING: 1 broken segments (containing 497803 documents) detected
   Took 0.176 sec total.
   WARNING: would write new segments file, and 497803 documents would be lost, if -exorcise were specified
   ```
   
   OK all is good!  It looks like corruption is indeed still detected with this PR, wonderful!


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-909872219


   Hi @mikemccand, I've merged this PR and will wait for an update on the [nightly check index time page](https://home.apache.org/~mikemccand/lucenebench/checkIndexTime.html). Once the result there looks good, I believe we should backport this change to 8x as well?


-- 
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


[GitHub] [lucene] zacharymorn merged pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments

Posted by GitBox <gi...@apache.org>.
zacharymorn merged pull request #128:
URL: https://github.com/apache/lucene/pull/128


   


-- 
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


[GitHub] [lucene] mikemccand commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-847864069


   > One thing I'm still researching is that, it seems there's limited direct test coverage for this `CheckIndex` class? I see there's `TestCheckIndex`, but it only has 4 tests, and the majority of its functionalities seems to be put under tests by other index testing utilities and test cases. Shall I still add a few more tests for these changes (and should I put them in `TestCheckIndex`)? On the other hand, I've been running `./gradlew check -Ptests.nightly=true -Pvalidation.git.failOnModified=false` the whole time and all tests have been passing.
   
   More tests are always welcome!  But I don't think that should block this change -- we can open a follow-on issue to get better direct unit tests for `CheckIndex`?  Those would be fun tests to write: make a healthy index, then make a random single bit change to one of its files, and then see if `CheckIndex` catches it.  Hmm I think we have such a test somewhere :)  But not apparently in `BaseTestCheckIndex`...


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r639455850



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -926,17 +1100,19 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
    * @lucene.experimental
    */
   public static Status.LiveDocStatus testLiveDocs(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {
+      CodecReader reader, PrintStream infoStream, String segmentId) {
     long startNS = System.nanoTime();
+    String segmentPartId = segmentId + "[LiveDocs]";
     final Status.LiveDocStatus status = new Status.LiveDocStatus();
 
     try {
-      if (infoStream != null) infoStream.print("    test: check live docs.....");
+      if (infoStream != null) infoStream.print(segmentPartId + "    test: check live docs.....");

Review comment:
       I've implemented it here https://github.com/apache/lucene/pull/128/commits/57f542f48ee08ec2bd63520c43deb0734455bd28 . The per part messages should be printed as soon as each concurrent check finishes, and without locking since the shared `PrintStream` object already handles locking internally.




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-911200983


   > Hi @mikemccand, I've merged this PR and will wait for an update on the [nightly check index time page](https://home.apache.org/~mikemccand/lucenebench/checkIndexTime.html). Once the result there looks good, I believe we should backport this change to 8x as well?
   
   From the [nightly check index time page](https://home.apache.org/~mikemccand/lucenebench/checkIndexTime.html), looks like the changes reduced the index checking time by a bit more than half! Woo hoo!


-- 
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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851739934


   > > To get the best speedup, even at -slow, we should do concurrency both ways, and then sort those tasks by decreasing expected cost.
   > 
   > This fine-grained concurrency (one task for "part X segment") would then mean we can get the postings check of the big segments all going, early, concurrently across all segments, and likely the biggest segments postings check would be the long pole, and {{CheckIndex}} would finish in (about) that much time.
   > 
   > Versus "thread per segment" concurrency, where the long pole will be one thread checking the biggest segment.
   > 
   > The really fun part of concurrency across _and_ within segments will be figure out the rough heuristic of how to assign estimated cost to each task :)
   > 
   > But this all can come later! "thread per segment" is already a big step forward!
   
   Yup this indeed will be both fun and challenging to implement! I think maybe it's also worth looking into breaking posting check into parallelize-able parts, as they alone can often take up 80% - 90% of the entire segment check time?


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851030385


   Oh one more thing. As the log output was buffered during parallel execution and printed later in sequential fashion to maintain order, to help out those who might be eager to see the output, for the first segment (which consumes the most of time during check) I have used the "global" `infoStream`  to print log as they are available - this gives the "weird" printing behavior that the first segment check prints slowly while it progress, and once the first segment finishes then all the subsequent segment outputs got printed at once.  Not sure if this behavior is ok as it may be perceived as buggy by the user?


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r639402016



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
##########
@@ -321,6 +321,7 @@ public static void syncConcurrentMerges(MergeScheduler ms) {
       checker.setDoSlowChecks(doSlowChecks);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false);
+      checker.setThreadCount(RandomNumbers.randomIntBetween(new Random(), 1, 5));

Review comment:
       Ah thanks @dweiss and @mikemccand for the context and solution! I used this one instead https://github.com/randomizedtesting/randomizedtesting/blob/8be8a09e3b3bf1a57947d5367a1afe68322ac4dc/randomized-runner/src/main/java/com/carrotsearch/randomizedtesting/RandomizedTest.java#L178 since the other one was marked as deprecated.




-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r627960267



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -810,6 +976,8 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
       result.newSegments.add(info.clone());
     }
 
+    executor.shutdown();

Review comment:
       Oops sorry. Thought it should be ok with the `join` call before `shutdown`, but yeah even the idle threads may take time to shutdown. Added.




-- 
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.

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


[GitHub] [lucene] mikemccand commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-847906215


   Finally I ran `CheckIndex -threadCount 128`:
   
   ```
   beast3:core[main]$ java -cp ../core/build/libs/lucene-core-9.0.0-SNAPSHOT.jar org.apache.lucene.index.CheckIndex /l/indices/trunk.nightly.index.prev/index -threadCount 128
   
   NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
   
   Opening index @ /l/indices/trunk.nightly.index.prev/index
   
   0.00% total deletions; 27625038 documents; 0 deleteions
   Segments file=segments_2 numSegments=15 version=9.0.0 id=5kpcv7rf0icb64q1ern1xs4bp userData={userData=multi}
   [Segment 110]   1 of 15: name=_32 maxDoc=4978031
   [Segment 110]     version=9.0.0
   [Segment 110]     id=5kpcv7rf0icb64q1ern1xs35l
   [Segment 110]     codec=Lucene90
   [Segment 110]     compound=false
   [Segment 110]     numFiles=20
   [Segment 110]     size (MB)=3,890.496
   [Segment 110]     diagnostics = {os.arch=amd64, mergeFactor=10, java.runtime.version=15.0.1+9-18, os=Linux, timestamp=1621919422678, lucene.version=9.0.0, java.vm.version=15.0.1+9-18, java.version=15.0.1, mergeMaxNumSegments=-1, os.version=5.11.2-arch1-1, java.vendor=Oracle Corporation, source=merge}
   [Segment 110]     no deletions
   [Segment 110]    test: open reader.........[Segment 110] OK [took 0.161 sec]
   [Segment 110]    test: check integrity.....[Segment 110] OK [took 0.617 sec]
   [Segment 110][LiveDocs]    test: check live docs.....[Segment 110][LiveDocs] OK [took 0.001 sec]
   [Segment 110][FieldInfos]    test: field infos.........[Segment 110][FieldNorms]    test: field norms.........[Segment 110][FieldInfos] OK [23 fields] [took 0.001 sec]
   [Segment 110][Postings]    test: terms, freq, prox...[Segment 110][StoredFields]    test: stored fields.......[Segment 110][TermVectors]    test: term vectors........[Segment 110][DocValues]    test: docvalues...........[Segment 110][TermVectors] OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.001 sec]
   [Segment 110][Points]    test: points..............[Segment 110][Vectors]    test: vectors..............[Segment 110][FieldNorms] OK [2 fields] [took 0.032 sec]
   [Segment 110][Vectors] OK [1 fields, 4978031 vectors] [took 0.190 sec]
   [Segment 110][Points] OK [4 fields, 19912124 points] [took 0.675 sec]
   [Segment 110][StoredFields] OK [14934093 total field count; avg 3.0 fields per doc] [took 1.229 sec]
   [Segment 110][DocValues] OK [15 docvalues fields; 4 BINARY; 2 NUMERIC; 7 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 3.357 sec]
   OK [16412141 terms; 525829996 terms/docs pairs; 765284459 tokens] [took 33.251 sec]
   [Segment 110]
   [Segment 221]   2 of 15: name=_65 maxDoc=4978032
   [Segment 221]     version=9.0.0
   [Segment 221]     id=5kpcv7rf0icb64q1ern1xs3eu
   [Segment 221]     codec=Lucene90
   [Segment 221]     compound=false
   [Segment 221]     numFiles=20
   [Segment 221]     size (MB)=3,873.589
   [Segment 221]     diagnostics = {os.arch=amd64, mergeFactor=10, java.runtime.version=15.0.1+9-18, os=Linux, timestamp=1621920782361, lucene.version=9.0.0, java.vm.version=15.0.1+9-18, java.version=15.0.1, mergeMaxNumSegments=-1, os.version=5.11.2-arch1-1, java.vendor=Oracle Corporation, source=merge}
   [Segment 221]     no deletions
   [Segment 221]    test: open reader.........[Segment 221] OK [took 0.166 sec]
   [Segment 221]    test: check integrity.....[Segment 221] OK [took 0.533 sec]
   [Segment 221][LiveDocs]    test: check live docs.....[Segment 221][FieldInfos]    test: field infos.........[Segment 221][LiveDocs] OK [took 0.000 sec]
   [Segment 221][FieldNorms]    test: field norms.........[Segment 221][Postings]    test: terms, freq, prox...[Segment 221][FieldInfos] OK [23 fields] [took 0.000 sec]
   [Segment 221][StoredFields]    test: stored fields.......[Segment 221][TermVectors]    test: term vectors........[Segment 221][DocValues]    test: docvalues...........[Segment 221][Points]    test: points..............[Segment 221][Vectors]    test: vectors..............[Segment 221][TermVectors] OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [t\
   ook 0.000 sec]
   [Segment 221][FieldNorms] OK [2 fields] [took 0.012 sec]
   [Segment 221][Vectors] OK [1 fields, 4978032 vectors] [took 0.163 sec]
   [Segment 221][Points] OK [4 fields, 19912128 points] [took 0.646 sec]
   [Segment 221][StoredFields] OK [14934096 total field count; avg 3.0 fields per doc] [took 1.029 sec]
   [Segment 221][DocValues] OK [15 docvalues fields; 4 BINARY; 2 NUMERIC; 7 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 3.073 sec]
   OK [15879702 terms; 497899770 terms/docs pairs; 726182953 tokens] [took 31.948 sec]
   [Segment 221]
   [Segment 332]   3 of 15: name=_98 maxDoc=4978032
   [Segment 332]     version=9.0.0
   [Segment 332]     id=5kpcv7rf0icb64q1ern1xs3o3
   [Segment 332]     codec=Lucene90
   [Segment 332]     compound=false
   [Segment 332]     numFiles=20
   [Segment 332]     size (MB)=3,837.561
   [Segment 332]     diagnostics = {os.arch=amd64, mergeFactor=10, java.runtime.version=15.0.1+9-18, os=Linux, timestamp=1621922132326, lucene.version=9.0.0, java.vm.version=15.0.1+9-18, java.version=15.0.1, mergeMaxNumSegments=-1, os.version=5.11.2-arch1-1, java.vendor=Oracle Corporation, source=merge}
   [Segment 332]     no deletions
   [Segment 332]    test: open reader.........[Segment 332] OK [took 0.136 sec]
   [Segment 332]    test: check integrity.....[Segment 332] OK [took 0.554 sec]
   [Segment 332][LiveDocs]    test: check live docs.....[Segment 332][FieldInfos]    test: field infos.........[Segment 332][LiveDocs] OK [took 0.000 sec]
   [Segment 332][FieldNorms]    test: field norms.........[Segment 332][Postings]    test: terms, freq, prox...[Segment 332][FieldInfos] OK [23 fields] [took 0.000 sec]
   [Segment 332][StoredFields]    test: stored fields.......[Segment 332][TermVectors]    test: term vectors........[Segment 332][DocValues]    test: docvalues...........[Segment 332][Points]    test: points..............[Segment 332][TermVectors] OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
   [Segment 332][Vectors]    test: vectors..............[Segment 332][FieldNorms] OK [2 fields] [took 0.014 sec]
   [Segment 332][Vectors] OK [1 fields, 4978032 vectors] [took 0.157 sec]
   [Segment 332][Points] OK [4 fields, 19912128 points] [took 0.639 sec]
   [Segment 332][StoredFields] OK [14934096 total field count; avg 3.0 fields per doc] [took 1.032 sec]
   [Segment 332][DocValues] OK [15 docvalues fields; 4 BINARY; 2 NUMERIC; 7 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 3.124 sec]
   OK [16460318 terms; 475335907 terms/docs pairs; 694693997 tokens] [took 30.290 sec]
   [Segment 332]
   
   ...
   
   [Segment 614]   15 of 15: name=_h2 maxDoc=46742
   [Segment 614]     version=9.0.0
   [Segment 614]     id=5kpcv7rf0icb64q1ern1xs4bl
   [Segment 614]     codec=Lucene90
   [Segment 614]     compound=false
   [Segment 614]     numFiles=20
   [Segment 614]     size (MB)=39.37
   [Segment 614]     diagnostics = {java.vm.version=15.0.1+9-18, java.version=15.0.1, timestamp=1621925578914, os=Linux, java.vendor=Oracle Corporation, os.version=5.11.2-arch1-1, java.runtime.version=15.0.1+9-18, os.arch=amd64, source=flush, lucene.version=9.0.0}
   [Segment 614]     no deletions
   [Segment 614]    test: open reader.........[Segment 614] OK [took 0.004 sec]
   [Segment 614]    test: check integrity.....[Segment 614] OK [took 0.011 sec]
   [Segment 614][LiveDocs]    test: check live docs.....[Segment 614][FieldInfos]    test: field infos.........[Segment 614][FieldNorms]    test: field norms.........[Segment 614][Postings]    test: terms, freq, prox...[Segment 614][LiveDocs] OK [took 0.000 sec]
   [Segment 614][StoredFields]    test: stored fields.......[Segment 614][TermVectors]    test: term vectors........[Segment 614][DocValues]    test: docvalues...........[Segment 614][Points]    test: points..............[Segment 614][Vectors]    test: vectors..............[Segment 614][FieldInfos] OK [23 fields] [took 0.000 sec]
   [Segment 614][TermVectors] OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
   [Segment 614][FieldNorms] OK [2 fields] [took 0.002 sec]
   [Segment 614][Vectors] OK [1 fields, 46742 vectors] [took 0.002 sec]
   [Segment 614][Points] OK [4 fields, 186968 points] [took 0.006 sec]
   [Segment 614][StoredFields] OK [140226 total field count; avg 3.0 fields per doc] [took 0.012 sec]
   [Segment 614][DocValues] OK [15 docvalues fields; 4 BINARY; 2 NUMERIC; 7 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.079 sec]
   OK [344318 terms; 3965374 terms/docs pairs; 6341319 tokens] [took 0.331 sec]
   [Segment 614]
   No problems were detected with this index.
   
   Took 176.318 sec total.
   ```
   
   It went a bit faster!  (203 down to 176 seconds).
   
   But the output is jumbled, I think because we are missing newlines somewhere, or maybe necessary locking?


-- 
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.

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


[GitHub] [lucene] mikemccand commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-849677698


   OK, I have good news and bad news.
   
   Good news first!  I wrote a [simple little Python tool](https://github.com/mikemccand/luceneutil/commit/77ef7e6708ccaed7077ef83e009da8e3b91f45ad) to randomly flip a random bit in a random file in a provided directory.
   
   Bad news!  I ran the tool, confirmed it seems to flip just the one bit, then ran the `CheckIndex` here, and no corruption was detected!!  Then I also ran `CheckIndex` from a clean `main` checkout, and we still fail to detect the corruption.  WTF?  Surely the bit flip would alter the checksum and we should have detected that in `CheckIndex`?  Or is it possible `CheckIndex` does not actually fully `checkIntegrity` too?
   
   For the record, this is how I ran the new bit-flipper tool:
   
   ```
    python3 -u /l/util/src/python/flip_random_bit.py /l/indices/trunk.nightly.index.prev.broken/index -seed 7 -real
   ```
   
   and this is its output:
   
   ```
   python3 -u /l/util.nightly/src/python/flip_random_bit.py /l/indices/trunk.nightly.index.prev.broken/index -seed 7 -real
   
   RANDOM SEED: 0x7
   
   Directory has 302 files:
     _32.fdm
     _32.fdt
     _32.fdx
     _32.fnm
     _32.kdd
     _32.kdi
     _32.kdm
     _32.nvd
     ...
       _h2.fdm
     _h2.fdt
     _h2.fdx
     _h2.fnm
     _h2.kdd
     _h2.kdi
     _h2.kdm
     _h2.nvd
     _h2.nvm
     _h2.si
     _h2_Lucene90HnswVectorFormat_0.vec
     _h2_Lucene90HnswVectorFormat_0.vem
     _h2_Lucene90HnswVectorFormat_0.vex
     _h2_Lucene90_0.doc
     _h2_Lucene90_0.dvd
     _h2_Lucene90_0.dvm
     _h2_Lucene90_0.pos
     _h2_Lucene90_0.tim
     _h2_Lucene90_0.tip
     _h2_Lucene90_0.tmd
     segments_2
     write.lock
   
   **WARNING**: this tool will soon corrupt bit 39544 (of 152368 bits) in /l/indices/trunk.nightly.index.prev.broken/index/_gm.kdi!!!
   
   Be really certain this is what you want... you have 5 seconds to change your mind!
   
   5...
   4...
   3...
   2...
   1...
   
   **BOOOOOOOM**
   ```
   
   And then `cmp` and `diff` confirm the file is indeed changed, yet `CheckIndex` (with or without this PR) doesn't catch it.  I'll try a few more bit flips.


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r697141222



##########
File path: lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
##########
@@ -895,7 +895,11 @@ public synchronized void close() throws IOException {
             System.out.println("\nNOTE: MockDirectoryWrapper: now run CheckIndex");
           }
 
-          TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose(), true, null);
+          // Methods in MockDirectoryWrapper hold locks on this, which will cause deadlock when
+          // TestUtil#checkIndex checks segment concurrently using another thread, but making
+          // call back to synchronized methods such as MockDirectoryWrapper#fileLength.
+          // Hence passing concurrent = false to this method to turn off concurrent checks.
+          TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose(), true, false, null);

Review comment:
       > Maybe open a follow-on issue to fix this sync situation so that we could, randomly, sometimes use concurrency in CheckIndex from tests? 
   
   Sounds good. I've created this issue https://issues.apache.org/jira/browse/LUCENE-10071 for following up on this.
   
   > Maybe we could start by making some of the TestUtil.checkIndex use concurrency, just not the one that MDW invokes?
   
   Yes this is already done in https://github.com/apache/lucene/pull/128/commits/138b72e9f2512df257c9acf01516bd071c9fb1d4, where `CheckIndex#checkIndex` got a new parameter `concurrent`, and most of the invocations except the one in MDW would pass in `true` to enable concurrency. 




-- 
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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851169333


   Some test results with corrupted index (`_gx_Lucene90_0.dvd`):
   
   ### Full check
   ```
   > Task :lucene:core:CheckIndex.main()
   
   NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
   
   Opening index @ /Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/
   
   Checking index with async threadCount: 12
   0.00% total deletions; 6647577 documents; 0 deletions
   Segments file=segments_2 numSegments=15 version=9.0.0 id=59c6he3dhebad46x7proh30nq userData={userData=multi}
   1 of 15: name=_32 maxDoc=1197893
       version=9.0.0
       id=59c6he3dhebad46x7proh2zhm
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=2,531.843
       diagnostics = {timestamp=1622100146526, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11, os=Mac OS X}
       no deletions
       test: open reader.........OK [took 0.125 sec]
       test: check integrity.....OK [took 20.451 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.044 sec]
       test: terms, freq, prox...OK [20065511 terms; 450728331 terms/docs pairs; 1175837878 tokens] [took 109.702 sec]
       test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 0.967 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 2.575 sec]
       test: points..............OK [2 fields, 2395786 points] [took 0.204 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   2 of 15: name=_65 maxDoc=1197893
       version=9.0.0
       id=59c6he3dhebad46x7proh2zqv
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=1,539.981
       diagnostics = {timestamp=1622100810971, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11, os=Mac OS X}
       no deletions
       test: open reader.........OK [took 0.124 sec]
       test: check integrity.....OK [took 13.612 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.042 sec]
       test: terms, freq, prox...OK [15042354 terms; 274837439 terms/docs pairs; 686566591 tokens] [took 76.072 sec]
       test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 0.982 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 2.351 sec]
       test: points..............OK [2 fields, 2395786 points] [took 0.194 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   ...
   ...
   
   10 of 15: name=_gx maxDoc=119789
       version=9.0.0
       id=59c6he3dhebad46x7proh30n7
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=129.046
       diagnostics = {timestamp=1622102767300, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11, os=Mac OS X}
       no deletions
       test: open reader.........OK [took 0.125 sec]
       test: check integrity.....FAILED
       WARNING: exorciseIndex() would remove reference to this segment; full exception:
   org.apache.lucene.index.CorruptIndexException: checksum failed (hardware problem?) : expected=87e2aa4 actual=7b3afcbd (resource=BufferedChecksumIndexInput(MMapIndexInput(path="/Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/_gx_Lucene90_0.dvd")))
   	at org.apache.lucene.codecs.CodecUtil.checkFooter(CodecUtil.java:440)
   	at org.apache.lucene.codecs.CodecUtil.checksumEntireFile(CodecUtil.java:614)
   	at org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer.checkIntegrity(Lucene90DocValuesProducer.java:1656)
   	at org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat$FieldsReader.checkIntegrity(PerFieldDocValuesFormat.java:364)
   	at org.apache.lucene.index.CodecReader.checkIntegrity(CodecReader.java:252)
   	at org.apache.lucene.index.SegmentReader.checkIntegrity(SegmentReader.java:391)
   	at org.apache.lucene.index.CheckIndex.testSegment(CheckIndex.java:913)
   	at org.apache.lucene.index.CheckIndex.lambda$checkIndex$0(CheckIndex.java:742)
   	at org.apache.lucene.index.CheckIndex.lambda$callableToSupplier$1(CheckIndex.java:842)
   	at java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1700)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at java.base/java.lang.Thread.run(Thread.java:834)
   
   11 of 15: name=_gy maxDoc=11979
       version=9.0.0
       id=59c6he3dhebad46x7proh30na
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=13.473
       diagnostics = {os.version=10.15.5, java.runtime.version=11.0.9+11, os.arch=x86_64, source=flush, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, timestamp=1622102778710, os=Mac OS X, java.vendor=AdoptOpenJDK}
       no deletions
       test: open reader.........OK [took 0.124 sec]
       test: check integrity.....OK [took 0.065 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.005 sec]
       test: terms, freq, prox...OK [295343 terms; 1930448 terms/docs pairs; 4662478 tokens] [took 1.269 sec]
       test: stored fields.......OK [35937 total field count; avg 3.0 fields per doc] [took 0.077 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.130 sec]
       test: points..............OK [2 fields, 23958 points] [took 0.014 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   ...
   ...
   
   15 of 15: name=_h2 maxDoc=11248
       version=9.0.0
       id=59c6he3dhebad46x7proh30nm
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=10.617
       diagnostics = {os.version=10.15.5, java.runtime.version=11.0.9+11, os.arch=x86_64, source=flush, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, timestamp=1622102791291, os=Mac OS X, java.vendor=AdoptOpenJDK}
       no deletions
       test: open reader.........OK [took 0.014 sec]
       test: check integrity.....OK [took 0.155 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.001 sec]
       test: terms, freq, prox...OK [253387 terms; 1570705 terms/docs pairs; 3390075 tokens] [took 0.703 sec]
       test: stored fields.......OK [33744 total field count; avg 3.0 fields per doc] [took 0.015 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.074 sec]
       test: points..............OK [2 fields, 22496 points] [took 0.004 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   WARNING: 1 broken segments (containing 119789 documents) detected
   Took 134.361 sec total.
   WARNING: would write new segments file, and 119789 documents would be lost, if -exorcise were specified
   
   
   
   > Task :lucene:core:CheckIndex.main() FAILED
   
   Execution failed for task ':lucene:core:CheckIndex.main()'.
   > Process 'command '/Library/Java/JavaVirtualMachines/adoptopenjdk-11.jdk/Contents/Home/bin/java'' finished with non-zero exit value 1
   ```
   
   ### Check with `-segment _gx` flag
   ```
   > Task :lucene:core:CheckIndex.main() FAILED
   
   NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
   
   Opening index @ /Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/
   
   Checking index with async threadCount: 12
   0.00% total deletions; 6647577 documents; 0 deletions
   Segments file=segments_2 numSegments=15 version=9.0.0 id=59c6he3dhebad46x7proh30nq userData={userData=multi}
   
   Checking only these segments: _gx:
   10 of 15: name=_gx maxDoc=119789
       version=9.0.0
       id=59c6he3dhebad46x7proh30n7
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=129.046
       diagnostics = {timestamp=1622102767300, lucene.version=9.0.0, java.vm.version=11.0.9+11, java.version=11.0.9, mergeMaxNumSegments=-1, os.version=10.15.5, java.vendor=AdoptOpenJDK, source=merge, os.arch=x86_64, mergeFactor=10, java.runtime.version=11.0.9+11, os=Mac OS X}
       no deletions
       test: open reader.........OK [took 0.086 sec]
       test: check integrity.....FAILED
       WARNING: exorciseIndex() would remove reference to this segment; full exception:
   org.apache.lucene.index.CorruptIndexException: checksum failed (hardware problem?) : expected=87e2aa4 actual=7b3afcbd (resource=BufferedChecksumIndexInput(MMapIndexInput(path="/Users/xichen/IdeaProjects/benchmarks/indices/corrupted/index/_gx_Lucene90_0.dvd")))
       at org.apache.lucene.codecs.CodecUtil.checkFooter(CodecUtil.java:440)
       at org.apache.lucene.codecs.CodecUtil.checksumEntireFile(CodecUtil.java:614)
       at org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer.checkIntegrity(Lucene90DocValuesProducer.java:1656)
       at org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat$FieldsReader.checkIntegrity(PerFieldDocValuesFormat.java:364)
       at org.apache.lucene.index.CodecReader.checkIntegrity(CodecReader.java:252)
       at org.apache.lucene.index.SegmentReader.checkIntegrity(SegmentReader.java:391)
       at org.apache.lucene.index.CheckIndex.testSegment(CheckIndex.java:913)
       at org.apache.lucene.index.CheckIndex.lambda$checkIndex$0(CheckIndex.java:742)
       at org.apache.lucene.index.CheckIndex.lambda$callableToSupplier$1(CheckIndex.java:842)
       at java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1700)
       at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
       at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
       at java.base/java.lang.Thread.run(Thread.java:834)
   
   WARNING: 1 broken segments (containing 119789 documents) detected
   Took 0.734 sec total.
   WARNING: would write new segments file, and 119789 documents would be lost, if -exorcise were specified
   
   
   
   Execution failed for task ':lucene:core:CheckIndex.main()'.
   > Process 'command '/Library/Java/JavaVirtualMachines/adoptopenjdk-11.jdk/Contents/Home/bin/java'' finished with non-zero exit value 1
   
   
   
   ```


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
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


[GitHub] [lucene] mikemccand commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-847869442


   Our nightly benchmarks also exercise `CheckIndex` each night on a "real" index, and sometimes that finds interesting surprises ;)


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r697884682



##########
File path: lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
##########
@@ -321,6 +326,11 @@ public static void syncConcurrentMerges(MergeScheduler ms) {
       checker.setDoSlowChecks(doSlowChecks);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(output, false, IOUtils.UTF_8), false);
+      if (concurrent) {
+        checker.setThreadCount(RandomizedTest.randomIntBetween(1, 5));
+      } else {
+        checker.setThreadCount(0);

Review comment:
       Hmm I still don't think we should special-case `0` to mean "use a single thread" -- I think that's kinda counter-intuitive.
   
   In fact I think `0` should not even be allowed (user should get an `IllegalArgumentException`) -- how can one make progress with no threads :)
   
   I think future users would indeed understand that `-threadCount 1` means single-threaded, and that that most closely matches how `CheckIndex` functions before this awesome addition of optional concurrency.
   
   I suppose we could debate whether `-threadCount 1` means "use the incoming (main) thread to do the checking" or it means "make an executor with one worker thread", but I think that's really an implementation detail.  From the uer's standpoint, only one thread is making progress on checking.




-- 
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


[GitHub] [lucene] mikemccand commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-847899963


   Next I ran `CheckIndex` with `-threadCount 1`:
   
   ```
   beast3:core[main]$ java -cp ../core/build/libs/lucene-core-9.0.0-SNAPSHOT.jar org.apache.lucene.index.CheckIndex /l/indices/trunk.nightly.index.prev/index -threadCount 1
   
   NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
   
   Opening index @ /l/indices/trunk.nightly.index.prev/index
   
   0.00% total deletions; 27625038 documents; 0 deleteions
   Segments file=segments_2 numSegments=15 version=9.0.0 id=5kpcv7rf0icb64q1ern1xs4bp userData={userData=multi}
   [Segment 110]   1 of 15: name=_32 maxDoc=4978031
   [Segment 110]     version=9.0.0
   [Segment 110]     id=5kpcv7rf0icb64q1ern1xs35l
   [Segment 110]     codec=Lucene90
   [Segment 110]     compound=false
   [Segment 110]     numFiles=20
   [Segment 110]     size (MB)=3,890.496
   [Segment 110]     diagnostics = {java.version=15.0.1, java.vm.version=15.0.1+9-18, lucene.version=9.0.0, timestamp=1621919422678, os=Linux, java.runtime.version=15.0.1+9-18, mergeFactor=10\
   , os.arch=amd64, source=merge, java.vendor=Oracle Corporation, os.version=5.11.2-arch1-1, mergeMaxNumSegments=-1}
   [Segment 110]     no deletions
   [Segment 110]    test: open reader.........[Segment 110] OK [took 0.166 sec]
   [Segment 110]    test: check integrity.....[Segment 110] OK [took 0.580 sec]
   [Segment 110][LiveDocs]    test: check live docs.....[Segment 110][LiveDocs] OK [took 0.001 sec]
   [Segment 110][FieldInfos]    test: field infos.........[Segment 110][FieldInfos] OK [23 fields] [took 0.000 sec]
   [Segment 110][FieldNorms]    test: field norms.........[Segment 110][FieldNorms] OK [2 fields] [took 0.031 sec]
   [Segment 110][Postings]    test: terms, freq, prox...OK [16412141 terms; 525829996 terms/docs pairs; 765284459 tokens] [took 35.202 sec]
   [Segment 110][StoredFields]    test: stored fields.......[Segment 110][StoredFields] OK [14934093 total field count; avg 3.0 fields per doc] [took 0.836 sec]
   [Segment 110][TermVectors]    test: term vectors........[Segment 110][TermVectors] OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
   [Segment 110][DocValues]    test: docvalues...........[Segment 110][DocValues] OK [15 docvalues fields; 4 BINARY; 2 NUMERIC; 7 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 3.205 sec]
   [Segment 110][Points]    test: points..............[Segment 110][Points] OK [4 fields, 19912124 points] [took 0.636 sec]
   [Segment 110][Vectors]    test: vectors..............[Segment 110][Vectors] OK [1 fields, 4978031 vectors] [took 0.169 sec]
   [Segment 110]
   [Segment 221]   2 of 15: name=_65 maxDoc=4978032
   [Segment 221]     version=9.0.0
   [Segment 221]     id=5kpcv7rf0icb64q1ern1xs3eu
   [Segment 221]     codec=Lucene90
   [Segment 221]     compound=false
   [Segment 221]     numFiles=20
   [Segment 221]     size (MB)=3,873.589
   [Segment 221]     diagnostics = {java.version=15.0.1, java.vm.version=15.0.1+9-18, lucene.version=9.0.0, timestamp=1621920782361, os=Linux, java.runtime.version=15.0.1+9-18, mergeFactor=10\
   , os.arch=amd64, source=merge, java.vendor=Oracle Corporation, os.version=5.11.2-arch1-1, mergeMaxNumSegments=-1}
   [Segment 221]     no deletions
   [Segment 221]    test: open reader.........[Segment 221] OK [took 0.172 sec]
   [Segment 221]    test: check integrity.....[Segment 221] OK [took 0.562 sec]
   [Segment 221][LiveDocs]    test: check live docs.....[Segment 221][LiveDocs] OK [took 0.000 sec]
   [Segment 221][FieldInfos]    test: field infos.........[Segment 221][FieldInfos] OK [23 fields] [took 0.000 sec]
   [Segment 221][FieldNorms]    test: field norms.........[Segment 221][FieldNorms] OK [2 fields] [took 0.009 sec]
   [Segment 221][Postings]    test: terms, freq, prox...OK [15879702 terms; 497899770 terms/docs pairs; 726182953 tokens] [took 31.904 sec]
   [Segment 221][StoredFields]    test: stored fields.......[Segment 221][StoredFields] OK [14934096 total field count; avg 3.0 fields per doc] [took 0.869 sec]
   [Segment 221][TermVectors]    test: term vectors........[Segment 221][TermVectors] OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
   [Segment 221][DocValues]    test: docvalues...........[Segment 221][DocValues] OK [15 docvalues fields; 4 BINARY; 2 NUMERIC; 7 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 3.234 sec]
   [Segment 221][Points]    test: points..............[Segment 221][Points] OK [4 fields, 19912128 points] [took 0.618 sec]
   [Segment 221][Vectors]    test: vectors..............[Segment 221][Vectors] OK [1 fields, 4978032 vectors] [took 0.153 sec]
   [Segment 221]
   [Segment 332]   3 of 15: name=_98 maxDoc=4978032
   [Segment 332]     version=9.0.0
   [Segment 332]     id=5kpcv7rf0icb64q1ern1xs3o3
   [Segment 332]     codec=Lucene90
   [Segment 332]     compound=false
   [Segment 332]     numFiles=20
   [Segment 332]     size (MB)=3,837.561
   [Segment 332]     diagnostics = {java.version=15.0.1, java.vm.version=15.0.1+9-18, lucene.version=9.0.0, timestamp=1621922132326, os=Linux, java.runtime.version=15.0.1+9-18, mergeFactor=10\
   , os.arch=amd64, source=merge, java.vendor=Oracle Corporation, os.version=5.11.2-arch1-1, mergeMaxNumSegments=-1}
   [Segment 332]     no deletions
   [Segment 332]    test: open reader.........[Segment 332] OK [took 0.137 sec]
   [Segment 332]    test: check integrity.....[Segment 332] OK [took 0.516 sec]
   [Segment 332][LiveDocs]    test: check live docs.....[Segment 332][LiveDocs] OK [took 0.000 sec]
   [Segment 332][FieldInfos]    test: field infos.........[Segment 332][FieldInfos] OK [23 fields] [took 0.000 sec]
   [Segment 332][FieldNorms]    test: field norms.........[Segment 332][FieldNorms] OK [2 fields] [took 0.012 sec]
   [Segment 332][Postings]    test: terms, freq, prox...OK [16460318 terms; 475335907 terms/docs pairs; 694693997 tokens] [took 28.982 sec]
   [Segment 332][StoredFields]    test: stored fields.......[Segment 332][StoredFields] OK [14934096 total field count; avg 3.0 fields per doc] [took 0.870 sec]
   [Segment 332][TermVectors]    test: term vectors........[Segment 332][TermVectors] OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
   [Segment 332][DocValues]    test: docvalues...........[Segment 332][DocValues] OK [15 docvalues fields; 4 BINARY; 2 NUMERIC; 7 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 3.323 sec]
   [Segment 332][Points]    test: points..............[Segment 332][Points] OK [4 fields, 19912128 points] [took 0.623 sec]
   [Segment 332][Vectors]    test: vectors..............[Segment 332][Vectors] OK [1 fields, 4978032 vectors] [took 0.151 sec]
   [Segment 332]
   
   ...
   
   [Segment 614]   15 of 15: name=_h2 maxDoc=46742
   [Segment 614]     version=9.0.0
   [Segment 614]     id=5kpcv7rf0icb64q1ern1xs4bl
   [Segment 614]     codec=Lucene90
   [Segment 614]     compound=false
   [Segment 614]     numFiles=20
   [Segment 614]     size (MB)=39.37
   [Segment 614]     diagnostics = {java.vendor=Oracle Corporation, os=Linux, timestamp=1621925578914, java.version=15.0.1, java.vm.version=15.0.1+9-18, lucene.version=9.0.0, source=flush, os\
   .arch=amd64, java.runtime.version=15.0.1+9-18, os.version=5.11.2-arch1-1}
   [Segment 614]     no deletions
   [Segment 614]    test: open reader.........[Segment 614] OK [took 0.004 sec]
   [Segment 614]    test: check integrity.....[Segment 614] OK [took 0.009 sec]
   [Segment 614][LiveDocs]    test: check live docs.....[Segment 614][LiveDocs] OK [took 0.000 sec]
   [Segment 614][FieldInfos]    test: field infos.........[Segment 614][FieldInfos] OK [23 fields] [took 0.000 sec]
   [Segment 614][FieldNorms]    test: field norms.........[Segment 614][FieldNorms] OK [2 fields] [took 0.001 sec]
   [Segment 614][Postings]    test: terms, freq, prox...OK [344318 terms; 3965374 terms/docs pairs; 6341319 tokens] [took 0.342 sec]
   [Segment 614][StoredFields]    test: stored fields.......[Segment 614][StoredFields] OK [140226 total field count; avg 3.0 fields per doc] [took 0.009 sec]
   [Segment 614][TermVectors]    test: term vectors........[Segment 614][TermVectors] OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
   [Segment 614][DocValues]    test: docvalues...........[Segment 614][DocValues] OK [15 docvalues fields; 4 BINARY; 2 NUMERIC; 7 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.075 sec]
   [Segment 614][Points]    test: points..............[Segment 614][Points] OK [4 fields, 186968 points] [took 0.006 sec]
   [Segment 614][Vectors]    test: vectors..............[Segment 614][Vectors] OK [1 fields, 46742 vectors] [took 0.002 sec]
   [Segment 614]
   No problems were detected with this index.
   
   Took 202.570 sec total.
   ```
   
   I think we should try to remove the duplicate segment/partId (e.g.`[Segment 614][StoredFields]`) in some lines?  And in fact since each segment is still done "at once" (we don't do concurrency across segments, yet), we shouldn't need the e.g. `[Segment 614]` prefix on each line?
   
   Hmm, also why are we calling it `Segment 614` when its name is `_h2`?  Hmm, is that the decimal translation of the base36 value?


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r697140979



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -605,209 +681,115 @@ 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);
-        }
-
-        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();
+    // 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;
         }
 
-        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
+      List<SegmentCommitInfo> segmentCommitInfos = new ArrayList<>();
+      for (SegmentCommitInfo sci : sis) {
+        segmentCommitInfos.add(sci);
+      }
 
-        segInfoStat.openReaderPassed = true;
+      // sort segmentCommitInfos by segment size, as smaller segment tends to finish faster, and

Review comment:
       Sounds good!




-- 
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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-909872219


   Hi @mikemccand, I've merged this PR and will wait for an update on the [nightly check index time page](https://home.apache.org/~mikemccand/lucenebench/checkIndexTime.html). Once the result there looks good, I believe we should backport this change to 8x as well?


-- 
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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r639460089



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -731,74 +810,173 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testLiveDocs(finalReader, infoStream, segmentId),
+                  liveDocStatus -> segInfoStat.liveDocStatus = liveDocStatus);
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldInfos(finalReader, infoStream, segmentId),
+                  fieldInfoStatus -> segInfoStat.fieldInfoStatus = fieldInfoStatus);
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldNorms(finalReader, infoStream, segmentId),
+                  fieldNormStatus -> segInfoStat.fieldNormStatus = fieldNormStatus);
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPostings(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termIndexStatus -> segInfoStat.termIndexStatus = termIndexStatus);
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testStoredFields(finalReader, infoStream, segmentId),
+                  storedFieldStatus -> segInfoStat.storedFieldStatus = storedFieldStatus);
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testTermVectors(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termVectorStatus -> segInfoStat.termVectorStatus = termVectorStatus);
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testDocValues(finalReader, infoStream, segmentId),
+                  docValuesStatus -> segInfoStat.docValuesStatus = docValuesStatus);
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testPoints(finalReader, infoStream, segmentId),
+                  pointsStatus -> segInfoStat.pointsStatus = pointsStatus);
 
           // Test VectorValues
-          segInfoStat.vectorValuesStatus = testVectors(reader, infoStream, failFast);
+          CompletableFuture<Void> testVectors =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testVectors(finalReader, infoStream, segmentId),
+                  vectorValuesStatus -> segInfoStat.vectorValuesStatus = vectorValuesStatus);
 
           // Test index sort
-          segInfoStat.indexSortStatus = testSort(reader, indexSort, infoStream, failFast);
+          CompletableFuture<Void> testSort =
+              runAsyncSegmentPartCheck(
+                  executorService,
+                  () -> testSort(finalReader, indexSort, infoStream, segmentId),
+                  indexSortStatus -> segInfoStat.indexSortStatus = indexSortStatus);
+
+          CompletableFuture<Void> testSoftDeletes = null;
+          final String softDeletesField = reader.getFieldInfos().getSoftDeletesField();
+          if (softDeletesField != null) {
+            testSoftDeletes =
+                runAsyncSegmentPartCheck(
+                    executorService,
+                    () ->
+                        checkSoftDeletes(
+                            softDeletesField, info, finalReader, infoStream, segmentId),
+                    softDeletesStatus -> segInfoStat.softDeletesStatus = softDeletesStatus);
+          }
 
           // Rethrow the first exception we encountered
           //  This will cause stats for failed segments to be incremented properly
+          testliveDocs.join();
           if (segInfoStat.liveDocStatus.error != null) {
-            throw new RuntimeException("Live docs test failed");
-          } else if (segInfoStat.fieldInfoStatus.error != null) {
-            throw new RuntimeException("Field Info test failed");
-          } else if (segInfoStat.fieldNormStatus.error != null) {
-            throw new RuntimeException("Field Norm test failed");
-          } else if (segInfoStat.termIndexStatus.error != null) {
-            throw new RuntimeException("Term Index test failed");
-          } else if (segInfoStat.storedFieldStatus.error != null) {
-            throw new RuntimeException("Stored Field test failed");
-          } else if (segInfoStat.termVectorStatus.error != null) {
-            throw new RuntimeException("Term Vector test failed");
-          } else if (segInfoStat.docValuesStatus.error != null) {
-            throw new RuntimeException("DocValues test failed");
-          } else if (segInfoStat.pointsStatus.error != null) {
-            throw new RuntimeException("Points test failed");
+            throw new CheckIndexException(
+                segmentId, "", "Live docs test failed", segInfoStat.liveDocStatus.error);
+          }
+
+          testFieldInfos.join();
+          if (segInfoStat.fieldInfoStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Field Info test failed", segInfoStat.fieldInfoStatus.error);
+          }
+
+          testFieldNorms.join();
+          if (segInfoStat.fieldNormStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Field Norm test failed", segInfoStat.fieldNormStatus.error);
+          }
+
+          testTermIndex.join();
+          if (segInfoStat.termIndexStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Term Index test failed", segInfoStat.termIndexStatus.error);
+          }
+
+          testStoredFields.join();
+          if (segInfoStat.storedFieldStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Stored Field test failed", segInfoStat.storedFieldStatus.error);
+          }
+
+          testTermVectors.join();
+          if (segInfoStat.termVectorStatus.error != null) {
+            throw new CheckIndexException(
+                segmentId, "", "Term Vector test failed", segInfoStat.termVectorStatus.error);

Review comment:
       I've removed the use of `segmentId` and `partId` as part of 57f542f.




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
rmuir commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851476434


   > This also makes sense and seems to be an easy change to switch the default? Is there anything I need to add specifically so that users can be made aware of this change when they upgrade lucene version (e.g. extra log to indicate the switch) ?
   
   Maybe we should just open a separate issue for that. The change could be for 9.0-only, have its own issue, CHANGES.txt, entry MIGRATE.txt entry, etc.


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-842879141


   For now I've gone ahead and implemented the idea to add segment / part ids to logging messages so that they are still usable (with some log parsing) in concurrent setting. I also removed `failFast` support in each individual index test methods, as they seems to be primarily used for skipping (using exception) during sequential execution, and removing it can simplify exception handling in concurrent code a bit.
   
   What do you think about these changes? @rmuir @dweiss @mikemccand 


-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r694928200



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -843,6 +812,258 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
     return result;
   }
 
+  private void updateMaxSegmentName(Status result, SegmentCommitInfo info) {
+    long segmentName = Long.parseLong(info.info.name.substring(1), Character.MAX_RADIX);
+    if (segmentName > result.maxSegmentName) {
+      result.maxSegmentName = segmentName;
+    }
+  }
+
+  private void processSegmentInfoStatusResult(
+      Status result, SegmentCommitInfo info, Status.SegmentInfoStatus segmentInfoStatus) {
+    result.segmentInfos.add(segmentInfoStatus);
+    if (segmentInfoStatus.error != null) {
+      result.totLoseDocCount += segmentInfoStatus.toLoseDocCount;
+      result.numBadSegments++;
+    } else {
+      // Keeper
+      result.newSegments.add(info.clone());
+    }
+  }
+
+  private <R> CompletableFuture<R> runAsyncSegmentCheck(
+      Callable<R> asyncCallable, ExecutorService executorService) {
+    return CompletableFuture.supplyAsync(callableToSupplier(asyncCallable), executorService);
+  }
+
+  private <T> Supplier<T> callableToSupplier(Callable<T> callable) {
+    return () -> {
+      try {
+        return callable.call();
+      } catch (RuntimeException | Error e) {
+        throw e;
+      } catch (Throwable e) {
+        throw new CompletionException(e);
+      }
+    };
+  }
+
+  private Status.SegmentInfoStatus testSegment(
+      SegmentInfos sis, SegmentCommitInfo info, PrintStream infoStream) throws IOException {
+    Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
+    segInfoStat.name = info.info.name;
+    segInfoStat.maxDoc = info.info.maxDoc();
+
+    final Version version = info.info.getVersion();
+    if (info.info.maxDoc() <= 0) {
+      throw new CheckIndexException(" illegal number of documents: maxDoc=" + info.info.maxDoc());
+    }
+
+    int toLoseDocCount = info.info.maxDoc();

Review comment:
       Oh hello, sorry this nearly fell past the event horizon of my TODO list!
   
   Since these inconsistencies of the "to lost doc count" are pre-existing, let's not fret about it here/now.  This change is already big/exciting enough!




-- 
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


[GitHub] [lucene] zacharymorn merged pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index check across segments

Posted by GitBox <gi...@apache.org>.
zacharymorn merged pull request #128:
URL: https://github.com/apache/lucene/pull/128


   


-- 
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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851080738


   Latest ok output from 12 threads (my machine actually only has 6 physical cores though) : 
   
   ```
   4:39:26 PM: Executing task 'CheckIndex.main()'...
   
   > Task :buildSrc:compileJava UP-TO-DATE
   > Task :buildSrc:compileGroovy NO-SOURCE
   > Task :buildSrc:processResources NO-SOURCE
   > Task :buildSrc:classes UP-TO-DATE
   > Task :buildSrc:jar UP-TO-DATE
   > Task :buildSrc:assemble UP-TO-DATE
   > Task :buildSrc:compileTestJava NO-SOURCE
   > Task :buildSrc:compileTestGroovy NO-SOURCE
   > Task :buildSrc:processTestResources NO-SOURCE
   > Task :buildSrc:testClasses UP-TO-DATE
   > Task :buildSrc:test NO-SOURCE
   > Task :buildSrc:check UP-TO-DATE
   > Task :buildSrc:build UP-TO-DATE
   
   > Configure project :
   IntelliJ Idea IDE detected.
   
   > Task :errorProneSkipped
   WARNING: errorprone disabled (skipped on non-nightly runs)
   
   > Task :lucene:core:processResources UP-TO-DATE
   > Task :lucene:core:compileJava
   > Task :lucene:core:classes
   
   > Task :lucene:core:CheckIndex.main()
   
   NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
   
   Opening index @ /Users/xichen/IdeaProjects/benchmarks/indices/wikibigall.lucene_baseline.facets.taxonomy:Date.taxonomy:Month.taxonomy:DayOfYear.sortedset:Month.sortedset:DayOfYear.Lucene90.Lucene90.nd6.64758M/index
   
   Checking index with async threadCount: 12
   0.00% total deletions; 6647577 documents; 0 deletions
   Segments file=segments_2 numSegments=15 version=9.0.0 id=59c6he3dhebad46x7proh30nq userData={userData=multi}
   1 of 15: name=_32 maxDoc=1197893
       version=9.0.0
       id=59c6he3dhebad46x7proh2zhm
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=2,531.843
       diagnostics = {os.version=10.15.5, mergeMaxNumSegments=-1, java.version=11.0.9, java.vm.version=11.0.9+11, lucene.version=9.0.0, timestamp=1622100146526, os=Mac OS X, java.runtime.version=11.0.9+11, mergeFactor=10, os.arch=x86_64, source=merge, java.vendor=AdoptOpenJDK}
       no deletions
       test: open reader.........OK [took 0.100 sec]
       test: check integrity.....OK [took 18.376 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.053 sec]
       test: terms, freq, prox...OK [20065511 terms; 450728331 terms/docs pairs; 1175837878 tokens] [took 107.241 sec]
       test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 0.955 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 3.127 sec]
       test: points..............OK [2 fields, 2395786 points] [took 0.263 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   2 of 15: name=_65 maxDoc=1197893
       version=9.0.0
       id=59c6he3dhebad46x7proh2zqv
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=1,539.981
       diagnostics = {os.version=10.15.5, mergeMaxNumSegments=-1, java.version=11.0.9, java.vm.version=11.0.9+11, lucene.version=9.0.0, timestamp=1622100810971, os=Mac OS X, java.runtime.version=11.0.9+11, mergeFactor=10, os.arch=x86_64, source=merge, java.vendor=AdoptOpenJDK}
       no deletions
       test: open reader.........OK [took 0.100 sec]
       test: check integrity.....OK [took 12.488 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.057 sec]
       test: terms, freq, prox...OK [15042354 terms; 274837439 terms/docs pairs; 686566591 tokens] [took 74.407 sec]
       test: stored fields.......OK [3593679 total field count; avg 3.0 fields per doc] [took 1.087 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 2.439 sec]
       test: points..............OK [2 fields, 2395786 points] [took 0.185 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   ...
   ...
   
   14 of 15: name=_h1 maxDoc=11979
       version=9.0.0
       id=59c6he3dhebad46x7proh30nj
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=12.824
       diagnostics = {source=flush, os.arch=x86_64, java.runtime.version=11.0.9+11, os.version=10.15.5, java.vendor=AdoptOpenJDK, os=Mac OS X, timestamp=1622102788648, java.version=11.0.9, java.vm.version=11.0.9+11, lucene.version=9.0.0}
       no deletions
       test: open reader.........OK [took 0.014 sec]
       test: check integrity.....OK [took 0.172 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.002 sec]
       test: terms, freq, prox...OK [290488 terms; 1843478 terms/docs pairs; 4383419 tokens] [took 0.788 sec]
       test: stored fields.......OK [35937 total field count; avg 3.0 fields per doc] [took 0.014 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.072 sec]
       test: points..............OK [2 fields, 23958 points] [took 0.003 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   15 of 15: name=_h2 maxDoc=11248
       version=9.0.0
       id=59c6he3dhebad46x7proh30nm
       codec=Lucene90
       compound=false
       numFiles=17
       size (MB)=10.617
       diagnostics = {source=flush, os.arch=x86_64, java.runtime.version=11.0.9+11, os.version=10.15.5, java.vendor=AdoptOpenJDK, os=Mac OS X, timestamp=1622102791291, java.version=11.0.9, java.vm.version=11.0.9+11, lucene.version=9.0.0}
       no deletions
       test: open reader.........OK [took 0.011 sec]
       test: check integrity.....OK [took 0.124 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: field infos.........OK [17 fields] [took 0.000 sec]
       test: field norms.........OK [2 fields] [took 0.001 sec]
       test: terms, freq, prox...OK [253387 terms; 1570705 terms/docs pairs; 3390075 tokens] [took 0.589 sec]
       test: stored fields.......OK [33744 total field count; avg 3.0 fields per doc] [took 0.013 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [10 docvalues fields; 3 BINARY; 0 NUMERIC; 5 SORTED; 0 SORTED_NUMERIC; 2 SORTED_SET] [took 0.066 sec]
       test: points..............OK [2 fields, 22496 points] [took 0.004 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
   
   No problems were detected with this index.
   
   Took 130.404 sec total.
   
   
   BUILD SUCCESSFUL in 2m 12s
   4 actionable tasks: 3 executed, 1 up-to-date
   4:41:39 PM: Task execution finished 'CheckIndex.main()'.
   ```


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-848533196


   I just tried to run `CheckIndex#checkIndex` via command line with index built for other task, but it failed from `Version` check (even without my latest changes):
   ```
   if (major > 255 || major < 0) {
       throw new IllegalArgumentException("Illegal major version: " + major);
   }
   ```
   I'll dig into that more tomorrow. 
   
   From the unit test `TestCheckIndex#testCheckIndexAllValid`, I'm able to get this log output:
   
   ```
   0.00% total deletions; 5 documents; 0 deleteions
   Segments file=segments_1 numSegments=1 version=9.0.0 id=bqdz36uoui8ymz6pd2tp9wmet
   1 of 1: name=_0 maxDoc=5
       version=9.0.0
       id=bqdz36uoui8ymz6pd2tp9wmeq
       codec=Asserting(Lucene90)
       compound=false
       numFiles=26
       sort=<int: "sort_field">!
       size (MB)=0.004
       diagnostics = {source=flush, os.arch=x86_64, java.runtime.version=11.0.9+11, os.version=10.15.5, java.vendor=AdoptOpenJDK, os=Mac OS X, timestamp=1622013701421, java.version=11.0.9, java.vm.version=11.0.9+11, lucene.version=9.0.0}
       no deletions
       test: open reader.........OK [took 0.004 sec]
       test: check integrity.....OK [took 0.001 sec]
       test: check live docs.....OK [took 0.000 sec]
   
       test: field infos.........OK [8 fields] [took 0.000 sec]
   
       test: field norms.........OK [1 fields] [took 0.000 sec]
   
       test: stored fields.......OK [8 total field count; avg 1.6 fields per doc] [took 0.001 sec]
   
       test: terms, freq, prox...OK [5 terms; 8 terms/docs pairs; 12 tokens] [took 0.004 sec]
   
       test: docvalues...........OK [2 docvalues fields; 0 BINARY; 2 NUMERIC; 0 SORTED; 0 SORTED_NUMERIC; 0 SORTED_SET] [took 0.002 sec]
   
       test: term vectors........OK [4 total term vector count; avg 1.0 term/freq vector fields per doc] [took 0.006 sec]
   
       test: vectors..............OK [2 fields, 8 vectors] [took 0.001 sec]
   
       test: index sort..........OK [took 0.002 sec]
   
       test: check soft deletes.....
       test: points..............OK [1 fields, 4 points] [took 0.005 sec]
   
   
   No problems were detected with this index.
   
   Took 0.046 sec total.
   
   ```
   
   


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: CheckIndex should be concurrent - parallelizing index parts check within each segment

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-849250243


   Here's the latest console output I got from running against a local small index
   
   ```
   > Task :lucene:core:CheckIndex.main()
   -threadCount currently only supports up to 11 threads. Value higher than that will be capped.
   
   NOTE: testing will be more thorough if you run java with '-ea:org.apache.lucene...', so assertions are enabled
   
   Opening index @ /Users/xichen/IdeaProjects/benchmarks/data/tmp/index-msmacropassages/
   
   Checking index with async threadCount: 11
   0.00% total deletions; 8841823 documents; 0 deleteions
   Segments file=segments_2 numSegments=1 version=9.0.0 id=arx9mawlpijzmvgvi7ehv0ret
   1 of 1: name=_a maxDoc=8841823
       version=9.0.0
       id=arx9mawlpijzmvgvi7ehv0req
       codec=Lucene90
       compound=false
       numFiles=12
       size (MB)=2,658.426
       diagnostics = {source=merge, java.vendor=AdoptOpenJDK, os.version=10.15.5, mergeMaxNumSegments=1, java.version=11.0.9, java.vm.version=11.0.9+11, lucene.version=9.0.0, timestamp=1622049007198, os=Mac OS X, java.runtime.version=11.0.9+11, mergeFactor=10, os.arch=x86_64}
       no deletions
       test: open reader.........OK [took 0.051 sec]
       test: check integrity.....OK [took 12.599 sec]
       test: field infos.........OK [2 fields] [took 0.000 sec]
       test: check live docs.....OK [took 0.000 sec]
       test: term vectors........OK [0 total term vector count; avg 0.0 term/freq vector fields per doc] [took 0.000 sec]
       test: docvalues...........OK [0 docvalues fields; 0 BINARY; 0 NUMERIC; 0 SORTED; 0 SORTED_NUMERIC; 0 SORTED_SET] [took 0.000 sec]
       test: points..............OK [0 fields, 0 points] [took 0.000 sec]
       test: vectors.............OK [0 fields, 0 vectors] [took 0.000 sec]
       test: field norms.........OK [1 fields] [took 0.382 sec]
       test: stored fields.......OK [17683646 total field count; avg 2.0 fields per doc] [took 25.261 sec]
       test: terms, freq, prox...OK [11795964 terms; 363490228 terms/docs pairs; 514795921 tokens] [took 82.775 sec]
   
   No problems were detected with this index.
   
   Took 95.669 sec total.
   ```
   
   I'll search for the nightly benchmark index to test next.


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-848482687


   Thanks @mikemccand for the feedback comment, as well as running the index check and posting the results here! I was planning to do that next after adding some more tests, but you beat me to it.
   
   > More tests are always welcome! But I don't think that should block this change -- we can open a follow-on issue to get better direct unit tests for CheckIndex? Those would be fun tests to write: make a healthy index, then make a random single bit change to one of its files, and then see if CheckIndex catches it. Hmm I think we have such a test somewhere :) But not apparently in BaseTestCheckIndex...
   
   Sounds good. Yes I think I came across them before, but didn't recall now exactly where they are now...but will work on them in a follow-up PR.
   
   
   > I think the issue is that opts.threadCount is 0 if you don't explicitly set the thread count. Can we fix it to default to number of cores on the box, maybe capped at a maximum (4?
   8?), when CheckIndex is invoked interactively from the command-line?
   
   Ah sorry about this (embarrassing) bu ! There was a default 1 for threadCount set in the code, but when it was not provided via the command -line, the default was overwritten by 0, causing this exception to be thrown. I'll fix it and cap at 4.
   
   
   > I think we should try to remove the duplicate segment/partId (e.g.[Segment 614][StoredFields]) in some lines? 
   
   > But the output is jumbled, I think because we are missing newlines somewhere, or maybe necessary locking?
   
   Yes these repeated segment / part ids are due to concurrent threads printing messages without newlines:
   https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java#L934. I also took a look at the implementation of `PrintStream#println` and it is synchronized on `this`, so it shouldn't require additional locking (assuming its sub classes are also similar). 
   
   I think this is indeed a problem with this approach to use segment / part id to organize messages, as it still requires certain way of printing the messages. I'll switch over to the other approach then to use per part buffer. 
   
   > Hmm, also why are we calling it Segment 614 when its name is _h2? Hmm, is that the decimal translation of the base36 value?
   
   This was due to `segmentName` was used for id there, but it should actually use `info.info.name` . I'll fix that.
   
   
   
   > Finally I ran CheckIndex -threadCount 128
   
   > It went a bit faster! (203 down to 176 seconds).
   
   Glad to see it actually improved the speed there :D ! I think 128 might be too big of a threadCount though for the current implementation, as it only parallelize (up to 11) part checking in each segment at a time. I'll cap the threadCount to 11 and print out a message to alert the user if big value was passed in as well 
   
   
   


-- 
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.

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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r639417993



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -2795,12 +2972,14 @@ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue) {
    * @lucene.experimental
    */
   public static Status.DocValuesStatus testDocValues(
-      CodecReader reader, PrintStream infoStream, boolean failFast) throws IOException {

Review comment:
       Yes. If it's invoked via `CheckIndex#checkIndex`, and `failFast = true`, error from segment part check will be stored inside each `Status` object, and then after `join` `CheckIndexException` will be thrown via these code
   ```
   if (segInfoStat.liveDocStatus.error != null) {
       throw new CheckIndexException(segmentId, "", "Live docs test failed", segInfoStat.liveDocStatus.error);
   }
   ``` 
   
   which will be caught in the outside `catch` block and rethrown via 
   ```
   if (failFast) {
       throw IOUtils.rethrowAlways(t);
   }
   ```




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r696247199



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -181,6 +193,9 @@
       /** True if we were able to open a CodecReader on this segment. */
       public boolean openReaderPassed;
 
+      /** doc count in this segment */

Review comment:
       Sounds good.




-- 
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


[GitHub] [lucene] zacharymorn commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r628697141



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -488,8 +503,35 @@ public Status checkIndex() throws IOException {
    *     quite a long time to run.
    */
   public Status checkIndex(List<String> onlySegments) throws IOException {

Review comment:
       > please don't overload it here. force callers to pass executor.
   
   I took a quick look at these methods' current usage. `checkIndex()` has 4 references in tests and 1 in luke module, and `checkIndex(onlySegments)` has 6 references in tests and 1 in `CheckIndex` main, so it should be straightforward to remove these convenience methods and have all callers to invoke `checkIndex(onlySegments, executorService)`. Having said that, it does seems to be a burden to put on each caller to handle executor creation and shutdown each time this method gets called though. Maybe we could keep these methods and use the 1 default threadcount executor like you suggested if one is not explicitly passed in (I also updated the default threadCount to be 1 in my latest commit) ?
   
   > I am still worried that tests are using multiple threads here which must not happen: on an 8 core machine we don't want to use 8 jvms * 8 threads each.
   
   Oh I didn't realize it would spawn multiple jvm processes, or are you suggesting that if this gets run in multiple jvm processes for some reasons then the overall overhead will be high (which I agree!) ? Shall we do something like `cpu / constant` to reduce it a bit, or any other mechanism we can use to better control the number of threads based on whole system's overhead?
   
   > I am also concerned about newly-created synchronization issues here (e.g. with output). If checkindex fails it is really important that we can read this output.
   
   Yeah definitely agree with this. For the output messages synchronization issue, I put in some nocommit comment earlier and proposed to also include segment / part id in these messages as a cheap way to allow manual sorting / grep-ing after the messages were written, to essentially work around this issue. Not sure if this will be good enough? 




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
zacharymorn commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r642641460



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -450,6 +479,14 @@ public void setChecksumsOnly(boolean v) {
 
   private boolean checksumsOnly;
 
+  /** Set threadCount used for parallelizing index integrity checking. */
+  public void setThreadCount(int tc) {
+    threadCount = tc;
+  }
+
+  // capped threadCount at 4
+  private int threadCount = Math.min(Runtime.getRuntime().availableProcessors(), 4);

Review comment:
       Ah I thought we would like to cap it to 4 (or 8) per earlier comment. But I think in latest changes I actually removed away the usage of this default, as when the user does not pass in `-threadCount`, the execution will use 0 for `threadCount` and fallback to sequential check. Should we give user this flexibility to not use concurrent check?




-- 
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.

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


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

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-851532048


   > Oh one more thing. As the log output was buffered during parallel execution and printed later in sequential fashion to maintain order, to help out those who might be eager to see the output, for the first segment (which consumes the most of time during check) I have used the "global" infoStream to print log as they are available - this gives the "weird" printing behavior that the first segment check prints slowly while it progress, and once the first segment finishes then all the subsequent segment outputs got printed at once. Not sure if this behavior is ok as it may be perceived as buggy by the user?
   
   Hmm, could we instead just output the full segment's output as each finishes?  So we would presumably see small segments finish first, then all checks for that small segment are printed?


-- 
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.

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


[GitHub] [lucene] dweiss commented on pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
dweiss commented on pull request #128:
URL: https://github.com/apache/lucene/pull/128#issuecomment-843813330


   > ./gradlew check -Ptests.nightly=true -Pvalidation.git.failOnModified=false
   
   Hi Zach. You can also "git add -A ." (stage your changes for commit); or just commit them in. Then there's  no need for the fail-on-modified flag to be turned off. :)


-- 
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.

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


[GitHub] [lucene] dweiss commented on a change in pull request #128: LUCENE-9662: [WIP] CheckIndex should be concurrent

Posted by GitBox <gi...@apache.org>.
dweiss commented on a change in pull request #128:
URL: https://github.com/apache/lucene/pull/128#discussion_r634091331



##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -701,104 +765,196 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         if (reader.hasDeletions()) {
           if (reader.numDocs() != info.info.maxDoc() - info.getDelCount()) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + (info.info.maxDoc() - info.getDelCount())
                     + " vs reader="
                     + reader.numDocs());
           }
           if ((info.info.maxDoc() - reader.numDocs()) > reader.maxDoc()) {
             throw new RuntimeException(
-                "too many deleted docs: maxDoc()="
+                segmentId
+                    + "too many deleted docs: maxDoc()="
                     + reader.maxDoc()
                     + " vs del count="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
           if (info.info.maxDoc() - reader.numDocs() != info.getDelCount()) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + info.getDelCount()
                     + " vs reader="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
         } else {
           if (info.getDelCount() != 0) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + info.getDelCount()
                     + " vs reader="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testLiveDocs(finalReader, infoStream, segmentId),
+                  liveDocStatus -> segInfoStat.liveDocStatus = liveDocStatus);
 
           // Test Fieldinfos
-          segInfoStat.fieldInfoStatus = testFieldInfos(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldInfos =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldInfos(finalReader, infoStream, segmentId),
+                  fieldInfoStatus -> segInfoStat.fieldInfoStatus = fieldInfoStatus);
 
           // Test Field Norms
-          segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream, failFast);
+          CompletableFuture<Void> testFieldNorms =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testFieldNorms(finalReader, infoStream, segmentId),
+                  fieldNormStatus -> segInfoStat.fieldNormStatus = fieldNormStatus);
 
           // Test the Term Index
-          segInfoStat.termIndexStatus =
-              testPostings(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermIndex =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testPostings(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termIndexStatus -> segInfoStat.termIndexStatus = termIndexStatus);
 
           // Test Stored Fields
-          segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream, failFast);
+          CompletableFuture<Void> testStoredFields =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testStoredFields(finalReader, infoStream, segmentId),
+                  storedFieldStatus -> segInfoStat.storedFieldStatus = storedFieldStatus);
 
           // Test Term Vectors
-          segInfoStat.termVectorStatus =
-              testTermVectors(reader, infoStream, verbose, doSlowChecks, failFast);
+          CompletableFuture<Void> testTermVectors =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testTermVectors(finalReader, infoStream, segmentId, verbose, doSlowChecks),
+                  termVectorStatus -> segInfoStat.termVectorStatus = termVectorStatus);
 
           // Test Docvalues
-          segInfoStat.docValuesStatus = testDocValues(reader, infoStream, failFast);
+          CompletableFuture<Void> testDocValues =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testDocValues(finalReader, infoStream, segmentId),
+                  docValuesStatus -> segInfoStat.docValuesStatus = docValuesStatus);
 
           // Test PointValues
-          segInfoStat.pointsStatus = testPoints(reader, infoStream, failFast);
+          CompletableFuture<Void> testPointvalues =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testPoints(finalReader, infoStream, segmentId),
+                  pointsStatus -> segInfoStat.pointsStatus = pointsStatus);
 
           // Test VectorValues
-          segInfoStat.vectorValuesStatus = testVectors(reader, infoStream, failFast);
+          CompletableFuture<Void> testVectors =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testVectors(finalReader, infoStream, segmentId),
+                  vectorValuesStatus -> segInfoStat.vectorValuesStatus = vectorValuesStatus);
 
           // Test index sort
-          segInfoStat.indexSortStatus = testSort(reader, indexSort, infoStream, failFast);
+          CompletableFuture<Void> testSort =
+              runAysncSegmentPartCheck(
+                  executorService,
+                  () -> testSort(finalReader, indexSort, infoStream, segmentId),
+                  indexSortStatus -> segInfoStat.indexSortStatus = indexSortStatus);
+
+          CompletableFuture<Void> testSoftDeletes = null;
+          final String softDeletesField = reader.getFieldInfos().getSoftDeletesField();
+          if (softDeletesField != null) {
+            testSoftDeletes =
+                runAysncSegmentPartCheck(
+                    executorService,
+                    () ->
+                        checkSoftDeletes(
+                            softDeletesField, info, finalReader, infoStream, segmentId),
+                    softDeletesStatus -> segInfoStat.softDeletesStatus = softDeletesStatus);
+          }
 
           // Rethrow the first exception we encountered
           //  This will cause stats for failed segments to be incremented properly
+          testliveDocs.join();
           if (segInfoStat.liveDocStatus.error != null) {
-            throw new RuntimeException("Live docs test failed");
-          } else if (segInfoStat.fieldInfoStatus.error != null) {
-            throw new RuntimeException("Field Info test failed");
-          } else if (segInfoStat.fieldNormStatus.error != null) {
-            throw new RuntimeException("Field Norm test failed");
-          } else if (segInfoStat.termIndexStatus.error != null) {
-            throw new RuntimeException("Term Index test failed");
-          } else if (segInfoStat.storedFieldStatus.error != null) {
-            throw new RuntimeException("Stored Field test failed");
-          } else if (segInfoStat.termVectorStatus.error != null) {
-            throw new RuntimeException("Term Vector test failed");
-          } else if (segInfoStat.docValuesStatus.error != null) {
-            throw new RuntimeException("DocValues test failed");
-          } else if (segInfoStat.pointsStatus.error != null) {
-            throw new RuntimeException("Points test failed");
+            throw new RuntimeException(segmentId + "Live docs test failed");
+          }
+
+          testFieldInfos.join();
+          if (segInfoStat.fieldInfoStatus.error != null) {
+            throw new RuntimeException(segmentId + "Field Info test failed");

Review comment:
       I think the cause should be added to those runtime exceptions.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -701,104 +765,196 @@ public Status checkIndex(List<String> onlySegments) throws IOException {
         if (reader.hasDeletions()) {
           if (reader.numDocs() != info.info.maxDoc() - info.getDelCount()) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + (info.info.maxDoc() - info.getDelCount())
                     + " vs reader="
                     + reader.numDocs());
           }
           if ((info.info.maxDoc() - reader.numDocs()) > reader.maxDoc()) {
             throw new RuntimeException(
-                "too many deleted docs: maxDoc()="
+                segmentId
+                    + "too many deleted docs: maxDoc()="
                     + reader.maxDoc()
                     + " vs del count="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
           if (info.info.maxDoc() - reader.numDocs() != info.getDelCount()) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + info.getDelCount()
                     + " vs reader="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
         } else {
           if (info.getDelCount() != 0) {
             throw new RuntimeException(
-                "delete count mismatch: info="
+                segmentId
+                    + "delete count mismatch: info="
                     + info.getDelCount()
                     + " vs reader="
                     + (info.info.maxDoc() - reader.numDocs()));
           }
         }
 
         if (checksumsOnly == false) {
+          // This redundant assignment is done to make compiler happy
+          SegmentReader finalReader = reader;
+
           // Test Livedocs
-          segInfoStat.liveDocStatus = testLiveDocs(reader, infoStream, failFast);
+          CompletableFuture<Void> testliveDocs =
+              runAysncSegmentPartCheck(

Review comment:
       Typo in aysnc.

##########
File path: lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
##########
@@ -488,8 +519,35 @@ public Status checkIndex() throws IOException {
    *     quite a long time to run.
    */
   public Status checkIndex(List<String> onlySegments) throws IOException {
+    ExecutorService executorService =
+        Executors.newFixedThreadPool(threadCount, new NamedThreadFactory("async-check-index"));
+    try {
+      return checkIndex(onlySegments, executorService);
+    } finally {
+      executorService.shutdown();
+      try {
+        executorService.awaitTermination(5, TimeUnit.SECONDS);
+      } catch (
+          @SuppressWarnings("unused")

Review comment:
       Why ignore? Seems like something went wrong there - log it at least?




-- 
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.

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