You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2014/12/01 18:02:51 UTC

svn commit: r1642713 [3/3] - in /lucene/dev/branches/lucene2878/lucene: ./ analysis/ analysis/common/ analysis/common/src/test/org/apache/lucene/analysis/core/ analysis/common/src/test/org/apache/lucene/analysis/miscellaneous/ analysis/common/src/test/...

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java Mon Dec  1 17:02:49 2014
@@ -211,6 +211,7 @@ abstract class BaseIndexFileFormatTestCa
   }
 
   /** Test the accuracy of the ramBytesUsed estimations. */
+  @Slow
   public void testRamBytesUsed() throws IOException {
     if (Codec.getDefault() instanceof RandomCodec) {
       // this test relies on the fact that two segments will be written with

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java Mon Dec  1 17:02:49 2014
@@ -185,6 +185,30 @@ public abstract class BaseNormsFormatTes
   }
   
   public void testNCommon() throws Exception {
+    final Random r = random();
+    final int N = TestUtil.nextInt(r, 2, 15);
+    final long[] commonValues = new long[N];
+    for (int j = 0; j < N; ++j) {
+      commonValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+    }
+    final int numOtherValues = TestUtil.nextInt(r, 2, 256 - N);
+    final long[] otherValues = new long[numOtherValues];
+    for (int j = 0; j < numOtherValues; ++j) {
+      otherValues[j] = TestUtil.nextLong(r, Byte.MIN_VALUE, Byte.MAX_VALUE);
+    }
+    doTestNormsVersusStoredFields(new LongProducer() {
+      @Override
+      long next() {
+        return r.nextInt(100) == 0 ? otherValues[r.nextInt(numOtherValues - 1)] : commonValues[r.nextInt(N - 1)];
+      }
+    });
+  }
+  
+  /**
+   * a more thorough n-common that tests all low bpv
+   */
+  @Nightly
+  public void testNCommonBig() throws Exception {
     final int iterations = atLeast(1);
     final Random r = random();
     for (int i = 0; i < iterations; ++i) {

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Mon Dec  1 17:02:49 2014
@@ -1589,12 +1589,9 @@ public abstract class BasePostingsFormat
                       while (docs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
                         docFreq++;
                         totalTermFreq += docs.freq();
-                        if (docs instanceof DocsEnum) {
-                          DocsEnum posEnum = (DocsEnum) docs;
-                          int limit = TestUtil.nextInt(random(), 1, docs.freq());
-                          for(int i=0;i<limit;i++) {
-                            posEnum.nextPosition();
-                          }
+                        int limit = TestUtil.nextInt(random(), 1, docs.freq());
+                        for(int i=0;i<limit;i++) {
+                          docs.nextPosition();
                         }
                       }
 
@@ -1638,12 +1635,9 @@ public abstract class BasePostingsFormat
                         while (docs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
                           docFreq++;
                           totalTermFreq += docs.freq();
-                          if (docs instanceof DocsEnum) {
-                            DocsEnum posEnum = (DocsEnum) docs;
-                            int limit = TestUtil.nextInt(random(), 1, docs.freq());
-                            for(int i=0;i<limit;i++) {
-                              posEnum.nextPosition();
-                            }
+                          int limit = TestUtil.nextInt(random(), 1, docs.freq());
+                          for(int i=0;i<limit;i++) {
+                            docs.nextPosition();
                           }
                         }
 

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/PerThreadPKLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/PerThreadPKLookup.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/PerThreadPKLookup.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/PerThreadPKLookup.java Mon Dec  1 17:02:49 2014
@@ -60,17 +60,14 @@ public class PerThreadPKLookup {
     int numSegs = 0;
     boolean hasDeletions = false;
     for(int i=0;i<leaves.size();i++) {
-      Fields fields = leaves.get(i).reader().fields();
-      if (fields != null) {
-        Terms terms = fields.terms(idFieldName);
-        if (terms != null) {
-          termsEnums[numSegs] = terms.iterator(null);
-          assert termsEnums[numSegs] != null;
-          docBases[numSegs] = leaves.get(i).docBase;
-          liveDocs[numSegs] = leaves.get(i).reader().getLiveDocs();
-          hasDeletions |= leaves.get(i).reader().hasDeletions();
-          numSegs++;
-        }
+      Terms terms = leaves.get(i).reader().terms(idFieldName);
+      if (terms != null) {
+        termsEnums[numSegs] = terms.iterator(null);
+        assert termsEnums[numSegs] != null;
+        docBases[numSegs] = leaves.get(i).docBase;
+        liveDocs[numSegs] = leaves.get(i).reader().getLiveDocs();
+        hasDeletions |= leaves.get(i).reader().hasDeletions();
+        numSegs++;
       }
     }
     this.numSegs = numSegs;

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java Mon Dec  1 17:02:49 2014
@@ -353,9 +353,6 @@ public abstract class ThreadedIndexingAn
                   if (s.getIndexReader().numDocs() > 0) {
                     smokeTestSearcher(s);
                     Fields fields = MultiFields.getFields(s.getIndexReader());
-                    if (fields == null) {
-                      continue;
-                    }
                     Terms terms = fields.terms("body");
                     if (terms == null) {
                       continue;
@@ -663,8 +660,8 @@ public abstract class ThreadedIndexingAn
 
   private int runQuery(IndexSearcher s, Query q) throws Exception {
     s.search(q, 10);
-    int hitCount = s.search(q, null, 10, new Sort(new SortField("title", SortField.Type.STRING))).totalHits;
-    final Sort dvSort = new Sort(new SortField("title", SortField.Type.STRING));
+    int hitCount = s.search(q, null, 10, new Sort(new SortField("titleDV", SortField.Type.STRING))).totalHits;
+    final Sort dvSort = new Sort(new SortField("titleDV", SortField.Type.STRING));
     int hitCount2 = s.search(q, null, 10, dvSort).totalHits;
     assertEquals(hitCount, hitCount2);
     return hitCount;

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java Mon Dec  1 17:02:49 2014
@@ -406,7 +406,7 @@ public abstract class BaseDirectoryTestC
       
       @Override
       public void run() {
-        for (int i = 0; i < 3000; i++) {
+        for (int i = 0; i < 1000; i++) {
           String fileName = this.name + i;
           try {
             //System.out.println("create:" + fileName);

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Mon Dec  1 17:02:49 2014
@@ -127,7 +127,7 @@ public class MockDirectoryWrapper extend
     // not be reproducible from the original seed
     this.randomState = new Random(random.nextInt());
     this.throttledOutput = new ThrottledIndexOutput(ThrottledIndexOutput
-        .mBitsToBytes(40 + randomState.nextInt(10)), 5 + randomState.nextInt(5), null);
+        .mBitsToBytes(40 + randomState.nextInt(10)), 1 + randomState.nextInt(5), null);
     init();
   }
 
@@ -180,7 +180,7 @@ public class MockDirectoryWrapper extend
   public static enum Throttling {
     /** always emulate a slow hard disk. could be very slow! */
     ALWAYS,
-    /** sometimes (2% of the time) emulate a slow hard disk. */
+    /** sometimes (0.5% of the time) emulate a slow hard disk. */
     SOMETIMES,
     /** never throttle output */
     NEVER

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java Mon Dec  1 17:02:49 2014
@@ -68,7 +68,7 @@ public class MockIndexOutputWrapper exte
         if (b != null) {
           delegate.writeBytes(b, offset, (int) freeSpace);
         } else {
-          delegate.copyBytes(in, len);
+          delegate.copyBytes(in, (int) freeSpace);
         }
       }
       if (realUsage > dir.maxUsedSize) {

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/LineFileDocs.java Mon Dec  1 17:02:49 2014
@@ -38,6 +38,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.IntField;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
@@ -162,6 +163,7 @@ public class LineFileDocs implements Clo
     final Field body;
     final Field id;
     final Field idNum;
+    final Field idNumDV;
     final Field date;
 
     public DocState(boolean useDocValues) {
@@ -193,9 +195,12 @@ public class LineFileDocs implements Clo
 
       if (useDocValues) {
         titleDV = new SortedDocValuesField("titleDV", new BytesRef());
+        idNumDV = new NumericDocValuesField("docid_intDV", 0);
         doc.add(titleDV);
+        doc.add(idNumDV);
       } else {
         titleDV = null;
+        idNumDV = null;
       }
     }
   }
@@ -244,6 +249,9 @@ public class LineFileDocs implements Clo
     final int i = id.getAndIncrement();
     docState.id.setStringValue(Integer.toString(i));
     docState.idNum.setIntValue(i);
+    if (docState.idNumDV != null) {
+      docState.idNumDV.setLongValue(i);
+    }
     return docState.doc;
   }
 }

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Mon Dec  1 17:02:49 2014
@@ -82,6 +82,7 @@ import com.carrotsearch.randomizedtestin
 import com.carrotsearch.randomizedtesting.rules.StaticFieldsInvariantRule;
 import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule;
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.FieldType;
@@ -819,6 +820,11 @@ public abstract class LuceneTestCase ext
   }
 
   /** create a new index writer config with random defaults */
+  public static IndexWriterConfig newIndexWriterConfig() {
+    return newIndexWriterConfig(new MockAnalyzer(random()));
+  }
+
+  /** create a new index writer config with random defaults */
   public static IndexWriterConfig newIndexWriterConfig(Analyzer a) {
     return newIndexWriterConfig(random(), a);
   }
@@ -842,7 +848,16 @@ public abstract class LuceneTestCase ext
     } else if (rarely(r)) {
       int maxThreadCount = TestUtil.nextInt(r, 1, 4);
       int maxMergeCount = TestUtil.nextInt(r, maxThreadCount, maxThreadCount + 4);
-      ConcurrentMergeScheduler cms = new ConcurrentMergeScheduler();
+      ConcurrentMergeScheduler cms;
+      if (r.nextBoolean()) {
+        cms = new ConcurrentMergeScheduler();
+      } else {
+        cms = new ConcurrentMergeScheduler() {
+            @Override
+            protected synchronized void maybeStall() {
+            }
+          };
+      }
       cms.setMaxMergesAndThreads(maxMergeCount, maxThreadCount);
       c.setMergeScheduler(cms);
     }
@@ -1256,7 +1271,7 @@ public abstract class LuceneTestCase ext
     }
     
     if (rarely(random) && !bare) { 
-      final double maxMBPerSec = 10 + 5*(random.nextDouble()-0.5);
+      final double maxMBPerSec = TestUtil.nextInt(random, 20, 40);
       if (LuceneTestCase.VERBOSE) {
         System.out.println("LuceneTestCase: will rate limit output IndexOutput to " + maxMBPerSec + " MB/sec");
       }

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleTemporaryFilesCleanup.java Mon Dec  1 17:02:49 2014
@@ -1,14 +1,22 @@
 package org.apache.lucene.util;
 
 import java.io.IOException;
+import java.net.URI;
+import java.nio.file.FileSystem;
+import java.nio.file.FileSystems;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
+import java.util.Random;
 
+import org.apache.lucene.mockfile.DisableFsyncFS;
+import org.apache.lucene.mockfile.HandleLimitFS;
+import org.apache.lucene.mockfile.LeakFS;
+import org.apache.lucene.mockfile.VerboseFS;
+import org.apache.lucene.mockfile.WindowsFS;
 import org.apache.lucene.util.LuceneTestCase.SuppressTempFileChecks;
 
 import com.carrotsearch.randomizedtesting.RandomizedContext;
@@ -52,6 +60,11 @@ final class TestRuleTemporaryFilesCleanu
    * Per-test class temporary folder.
    */
   private Path tempDirBase;
+  
+  /**
+   * Per-test filesystem
+   */
+  private FileSystem fileSystem;
 
   /**
    * Suite failure marker.
@@ -90,11 +103,39 @@ final class TestRuleTemporaryFilesCleanu
     super.before();
 
     assert tempDirBase == null;
+    fileSystem = initializeFileSystem();
     javaTempDir = initializeJavaTempDir();
   }
+  
+  // os/config-independent limit for too many open files
+  // TODO: can we make this lower?
+  private static final int MAX_OPEN_FILES = 2048;
+  
+  private FileSystem initializeFileSystem() {
+    FileSystem fs = FileSystems.getDefault();
+    if (LuceneTestCase.VERBOSE) {
+      fs = new VerboseFS(fs, new TestRuleSetupAndRestoreClassEnv.ThreadNameFixingPrintStreamInfoStream(System.out)).getFileSystem(null);
+    }
+    Random random = RandomizedContext.current().getRandom();
+    // sometimes just use a bare filesystem
+    if (random.nextInt(10) > 0) {
+      fs = new DisableFsyncFS(fs).getFileSystem(null);
+      fs = new LeakFS(fs).getFileSystem(null);
+      fs = new HandleLimitFS(fs, MAX_OPEN_FILES).getFileSystem(null);
+      // windows is currently slow
+      if (random.nextInt(10) == 0) {
+        fs = new WindowsFS(fs).getFileSystem(null);
+      }
+    }
+    if (LuceneTestCase.VERBOSE) {
+      System.out.println("filesystem: " + fs.provider());
+    }
+    return fs.provider().getFileSystem(URI.create("file:///"));
+  }
 
   private Path initializeJavaTempDir() throws IOException {
-    Path javaTempDir = Paths.get(System.getProperty("tempDir", System.getProperty("java.io.tmpdir")));
+    Path javaTempDir = fileSystem.getPath(System.getProperty("tempDir", System.getProperty("java.io.tmpdir")));
+    
     Files.createDirectories(javaTempDir);
 
     assert Files.isDirectory(javaTempDir) &&
@@ -134,6 +175,9 @@ final class TestRuleTemporaryFilesCleanu
         }
         throw e;
       }
+      if (fileSystem != FileSystems.getDefault()) {
+        fileSystem.close();
+      }
     } else {
       if (tempDirBasePath != null) {
         System.err.println("NOTE: leaving temporary files on disk at: " + tempDirBasePath);
@@ -171,7 +215,7 @@ final class TestRuleTemporaryFilesCleanu
     }
     return tempDirBase;
   }
-
+  
   /**
    * @see LuceneTestCase#createTempDir()
    */

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Mon Dec  1 17:02:49 2014
@@ -19,6 +19,32 @@ package org.apache.lucene.util;
 
 import com.carrotsearch.randomizedtesting.generators.RandomInts;
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import java.io.BufferedInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.CharBuffer;
+import java.nio.file.FileSystem;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.PostingsFormat;
@@ -58,38 +84,19 @@ import org.apache.lucene.index.SegmentRe
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.TieredMergePolicy;
+import org.apache.lucene.mockfile.FilterFileSystem;
+import org.apache.lucene.mockfile.WindowsFS;
 import org.apache.lucene.search.FieldDoc;
 import org.apache.lucene.search.FilteredQuery;
 import org.apache.lucene.search.FilteredQuery.FilterStrategy;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.FilterDirectory;
 import org.apache.lucene.store.NoLockFactory;
 import org.junit.Assert;
 
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.CharBuffer;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipInputStream;
 
 /**
  * General utility methods for Lucene unit tests. 
@@ -105,6 +112,7 @@ public final class TestUtil {
    * Closes the given InputStream after extracting! 
    */
   public static void unzip(InputStream in, Path destDir) throws IOException {
+    in = new BufferedInputStream(in);
     IOUtils.rm(destDir);
     Files.createDirectory(destDir);
 
@@ -1141,6 +1149,35 @@ public final class TestUtil {
       return sb.toString();
     }
   }
+
+  /** Returns true if this is an FSDirectory backed by {@link WindowsFS}. */
+  public static boolean isWindowsFS(Directory dir) {
+    // First unwrap directory to see if there is an FSDir:
+    while (true) {
+      if (dir instanceof FSDirectory) {
+        return isWindowsFS(((FSDirectory) dir).getDirectory());
+      } else if (dir instanceof FilterDirectory) {
+        dir = ((FilterDirectory) dir).getDelegate();
+      } else {
+        return false;
+      }
+    }
+  }
+
+  /** Returns true if this Path is backed by {@link WindowsFS}. */
+  public static boolean isWindowsFS(Path path) {
+    FileSystem fs = path.getFileSystem();
+    while (true) {
+      if (fs instanceof FilterFileSystem) {
+        if (((FilterFileSystem) fs).getParent() instanceof WindowsFS) {
+          return true;
+        }
+        fs = ((FilterFileSystem) fs).getDelegate();
+      } else {
+        return false;
+      }
+    }
+  }
   
   /** List of characters that match {@link Character#isWhitespace} */
   public static final char[] WHITESPACE_CHARACTERS = new char[] {

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/util/ThrottledIndexOutput.java Mon Dec  1 17:02:49 2014
@@ -53,7 +53,7 @@ public class ThrottledIndexOutput extend
   }
 
   public static final int mBitsToBytes(int mbits) {
-    return mbits * 125000;
+    return mbits * 125000000;
   }
 
   public ThrottledIndexOutput(int bytesPerSecond, long flushDelayMillis,

Modified: lucene/dev/branches/lucene2878/lucene/tools/javadoc/ecj.javadocs.prefs
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/tools/javadoc/ecj.javadocs.prefs?rev=1642713&r1=1642712&r2=1642713&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/tools/javadoc/ecj.javadocs.prefs (original)
+++ lucene/dev/branches/lucene2878/lucene/tools/javadoc/ecj.javadocs.prefs Mon Dec  1 17:02:49 2014
@@ -4,9 +4,9 @@ org.eclipse.jdt.core.compiler.annotation
 org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault
 org.eclipse.jdt.core.compiler.annotation.nonnullisdefault=disabled
 org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable
-org.eclipse.jdt.core.compiler.annotation.nullanalysis=enabled
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7
-org.eclipse.jdt.core.compiler.compliance=1.7
+org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.8
+org.eclipse.jdt.core.compiler.compliance=1.8
 org.eclipse.jdt.core.compiler.doc.comment.support=enabled
 org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=ignore
 org.eclipse.jdt.core.compiler.problem.assertIdentifier=error
@@ -93,4 +93,4 @@ org.eclipse.jdt.core.compiler.problem.un
 org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=ignore
 org.eclipse.jdt.core.compiler.problem.unusedWarningToken=ignore
 org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=ignore
-org.eclipse.jdt.core.compiler.source=1.7
+org.eclipse.jdt.core.compiler.source=1.8