From 24ac5fdd13c84d58dbb15214e16ce67a22359fc5 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Tue, 5 Jun 2012 16:53:38 +0000 Subject: [PATCH] diagnostics for merged segments weren't being; add test coverage for that and for segment warming git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1346465 13f79535-47bb-0310-9956-ffa450edef68 --- .../org/apache/lucene/index/IndexWriter.java | 11 ++++++----- .../ThreadedIndexingAndSearchingTestCase.java | 16 ++++++++++++++++ 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java index ba4e53b302a..7b1b27c0654 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -3288,20 +3288,21 @@ public class IndexWriter implements Closeable, TwoPhaseCommit { checkpoint(); } - Map details = new HashMap(); - details.put("mergeMaxNumSegments", ""+merge.maxNumSegments); - details.put("mergeFactor", Integer.toString(merge.segments.size())); - // Bind a new segment name here so even with // ConcurrentMergePolicy we keep deterministic segment // names. final String mergeSegmentName = newSegmentName(); - SegmentInfo si = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergeSegmentName, -1, false, codec, details, null); + SegmentInfo si = new SegmentInfo(directory, Constants.LUCENE_MAIN_VERSION, mergeSegmentName, -1, false, codec, null, null); merge.info = new SegmentInfoPerCommit(si, 0, -1L); // Lock order: IW -> BD bufferedDeletesStream.prune(segmentInfos); + Map details = new HashMap(); + details.put("mergeMaxNumSegments", ""+merge.maxNumSegments); + details.put("mergeFactor", Integer.toString(merge.segments.size())); + setDiagnostics(si, "merge", details); + if (infoStream.isEnabled("IW")) { infoStream.message("IW", "merge seg=" + merge.info.info.name); } diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java index 3f361dcbef1..cc52cd32edb 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java @@ -334,6 +334,19 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas try { final IndexSearcher s = getCurrentSearcher(); try { + // Verify 1) IW is correctly setting + // diagnostics, and 2) segment warming for + // merged segments is actually happening: + for(AtomicReader sub : ((DirectoryReader) s.getIndexReader()).getSequentialSubReaders()) { + SegmentReader segReader = (SegmentReader) sub; + Map diagnostics = segReader.getSegmentInfo().info.getDiagnostics(); + assertNotNull(diagnostics); + String source = diagnostics.get("source"); + assertNotNull(source); + if (source.equals("merge")) { + assertTrue("sub reader " + sub + " wasn't warmed", warmed.containsKey(((SegmentReader) sub).core)); + } + } if (s.getIndexReader().numDocs() > 0) { smokeTestSearcher(s); Fields fields = MultiFields.getFields(s.getIndexReader()); @@ -405,6 +418,8 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas protected void doClose() throws Exception { } + private final Map warmed = new WeakHashMap(); + public void runTest(String testName) throws Exception { failed.set(false); @@ -442,6 +457,7 @@ public abstract class ThreadedIndexingAndSearchingTestCase extends LuceneTestCas if (VERBOSE) { System.out.println("TEST: now warm merged reader=" + reader); } + warmed.put(((SegmentReader) reader).core, Boolean.TRUE); final int maxDoc = reader.maxDoc(); final Bits liveDocs = reader.getLiveDocs(); int sum = 0;