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/27 13:04:09 UTC

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

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