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/07 06:26:45 UTC

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

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