You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/11/11 17:21:17 UTC

svn commit: r1638204 [2/3] - in /lucene/dev/branches/lucene6005: ./ dev-tools/ dev-tools/scripts/ lucene/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/ lucene/core/ lucene/core/src/java/org/apache/lucene/analysis/tokenattri...

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java Tue Nov 11 16:21:15 2014
@@ -20,7 +20,6 @@ package org.apache.lucene.store;
 import java.io.IOException;
 import java.nio.file.Path;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.List;
 import java.util.Random;
 
@@ -265,18 +264,14 @@ public class TestBufferedIndexInput exte
       }
     }
 
-    private static class MockFSDirectory extends BaseDirectory {
+    private static class MockFSDirectory extends FilterDirectory {
 
-      List<IndexInput> allIndexInputs = new ArrayList<>();
-
-      Random rand;
-
-      private Directory dir;
+      final List<IndexInput> allIndexInputs = new ArrayList<>();
+      final Random rand;
 
       public MockFSDirectory(Path path, Random rand) throws IOException {
+        super(new SimpleFSDirectory(path));
         this.rand = rand;
-        lockFactory = NoLockFactory.getNoLockFactory();
-        dir = new SimpleFSDirectory(path, null);
       }
 
       public void tweakBufferSizes() {
@@ -294,46 +289,9 @@ public class TestBufferedIndexInput exte
       public IndexInput openInput(String name, IOContext context) throws IOException {
         // Make random changes to buffer size
         //bufferSize = 1+Math.abs(rand.nextInt() % 10);
-        IndexInput f = dir.openInput(name, context);
+        IndexInput f = super.openInput(name, context);
         allIndexInputs.add(f);
         return f;
       }
-
-      @Override
-      public IndexOutput createOutput(String name, IOContext context) throws IOException {
-        return dir.createOutput(name, context);
-      }
-
-      @Override
-      public void close() throws IOException {
-        dir.close();
-      }
-
-      @Override
-      public void deleteFile(String name)
-        throws IOException
-      {
-        dir.deleteFile(name);
-      }
-      @Override
-      public String[] listAll()
-        throws IOException
-      {
-        return dir.listAll();
-      }
-      @Override
-      public void sync(Collection<String> names) throws IOException {
-        dir.sync(names);
-      }
-      
-      @Override
-      public void renameFile(String source, String dest) throws IOException {
-        dir.renameFile(source, dest);
-      }
-
-      @Override
-      public long fileLength(String name) throws IOException {
-        return dir.fileLength(name);
-      }
     }
 }

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestDirectory.java Tue Nov 11 16:21:15 2014
@@ -17,17 +17,12 @@ package org.apache.lucene.store;
  * limitations under the License.
  */
 
-import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.file.Files;
-import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
 import java.util.Arrays;
-import java.util.Collections;
 
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.TestUtil;
 
 public class TestDirectory extends BaseDirectoryTestCase {
 
@@ -63,9 +58,9 @@ public class TestDirectory extends BaseD
     }
 
     final FSDirectory[] dirs = new FSDirectory[] {
-      new SimpleFSDirectory(path, null),
-      new NIOFSDirectory(path, null),
-      new MMapDirectory(path, null)
+      new SimpleFSDirectory(path),
+      new NIOFSDirectory(path),
+      new MMapDirectory(path)
     };
 
     for (int i=0; i<dirs.length; i++) {
@@ -142,11 +137,12 @@ public class TestDirectory extends BaseD
   }
 
   // LUCENE-1468
+  @SuppressWarnings("resource")
   public void testCopySubdir() throws Throwable {
     Path path = createTempDir("testsubdir");
     try {
       Files.createDirectory(path.resolve("subdir"));
-      Directory fsDir = new SimpleFSDirectory(path, null);
+      Directory fsDir = new SimpleFSDirectory(path);
       assertEquals(0, new RAMDirectory(fsDir, newIOContext(random())).listAll().length);
     } finally {
       IOUtils.rm(path);
@@ -156,13 +152,13 @@ public class TestDirectory extends BaseD
   // LUCENE-1468
   public void testNotDirectory() throws Throwable {
     Path path = createTempDir("testnotdir");
-    Directory fsDir = new SimpleFSDirectory(path, null);
+    Directory fsDir = new SimpleFSDirectory(path);
     try {
       IndexOutput out = fsDir.createOutput("afile", newIOContext(random()));
       out.close();
       assertTrue(slowFileExists(fsDir, "afile"));
       try {
-        new SimpleFSDirectory(path.resolve("afile"), null);
+        new SimpleFSDirectory(path.resolve("afile"));
         fail("did not hit expected exception");
       } catch (IOException nsde) {
         // Expected

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestLockFactory.java Tue Nov 11 16:21:15 2014
@@ -44,12 +44,8 @@ public class TestLockFactory extends Luc
     // methods are called at the right time, locks are created, etc.
 
     public void testCustomLockFactory() throws IOException {
-        Directory dir = new MockDirectoryWrapper(random(), new RAMDirectory());
         MockLockFactory lf = new MockLockFactory();
-        dir.setLockFactory(lf);
-
-        // Lock prefix should have been set:
-        assertTrue("lock prefix was not set by the RAMDirectory", lf.lockPrefixSet);
+        Directory dir = new MockDirectoryWrapper(random(), new RAMDirectory(lf));
 
         IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
 
@@ -77,11 +73,8 @@ public class TestLockFactory extends Luc
     // exceptions raised:
     // Verify: NoLockFactory allows two IndexWriters
     public void testRAMDirectoryNoLocking() throws IOException {
-        MockDirectoryWrapper dir = new MockDirectoryWrapper(random(), new RAMDirectory());
-        dir.setLockFactory(NoLockFactory.getNoLockFactory());
-        dir.setWrapLockFactory(false); // we are gonna explicitly test we get this back
-        assertTrue("RAMDirectory.setLockFactory did not take",
-                   NoLockFactory.class.isInstance(dir.getLockFactory()));
+        MockDirectoryWrapper dir = new MockDirectoryWrapper(random(), new RAMDirectory(NoLockFactory.INSTANCE));
+        dir.setAssertLocks(false); // we are gonna explicitly test we get this back
 
         IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
         writer.commit(); // required so the second open succeed 
@@ -104,10 +97,10 @@ public class TestLockFactory extends Luc
     // Verify: SingleInstanceLockFactory is the default lock for RAMDirectory
     // Verify: RAMDirectory does basic locking correctly (can't create two IndexWriters)
     public void testDefaultRAMDirectory() throws IOException {
-        Directory dir = new RAMDirectory();
+        RAMDirectory dir = new RAMDirectory();
 
-        assertTrue("RAMDirectory did not use correct LockFactory: got " + dir.getLockFactory(),
-                   SingleInstanceLockFactory.class.isInstance(dir.getLockFactory()));
+        assertTrue("RAMDirectory did not use correct LockFactory: got " + dir.lockFactory,
+                   dir.lockFactory instanceof SingleInstanceLockFactory);
 
         IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(new MockAnalyzer(random())));
 
@@ -129,8 +122,8 @@ public class TestLockFactory extends Luc
     // IndexWriters over & over in 2 threads and making sure
     // no unexpected exceptions are raised:
     @Nightly
-    public void testStressLocks() throws Exception {
-      _testStressLocks(null, createTempDir("index.TestLockFactory6"));
+    public void testStressLocksSimpleFSLockFactory() throws Exception {
+      _testStressLocks(SimpleFSLockFactory.INSTANCE, createTempDir("index.TestLockFactory6"));
     }
 
     // Verify: do stress test, by opening IndexReaders and
@@ -140,7 +133,7 @@ public class TestLockFactory extends Luc
     @Nightly
     public void testStressLocksNativeFSLockFactory() throws Exception {
       Path dir = createTempDir("index.TestLockFactory7");
-      _testStressLocks(new NativeFSLockFactory(dir), dir);
+      _testStressLocks(NativeFSLockFactory.INSTANCE, dir);
     }
 
     public void _testStressLocks(LockFactory lockFactory, Path indexDir) throws Exception {
@@ -170,11 +163,10 @@ public class TestLockFactory extends Luc
 
     // Verify: NativeFSLockFactory works correctly
     public void testNativeFSLockFactory() throws IOException {
-      NativeFSLockFactory f = new NativeFSLockFactory(createTempDir(LuceneTestCase.getTestClass().getSimpleName()));
+      Directory dir = FSDirectory.open(createTempDir(LuceneTestCase.getTestClass().getSimpleName()), NativeFSLockFactory.INSTANCE);
 
-      f.setLockPrefix("test");
-      Lock l = f.makeLock("commit");
-      Lock l2 = f.makeLock("commit");
+      Lock l = dir.makeLock("commit");
+      Lock l2 = dir.makeLock("commit");
 
       assertTrue("failed to obtain lock", l.obtain());
       assertTrue("succeeded in obtaining lock twice", !l2.obtain());
@@ -199,55 +191,14 @@ public class TestLockFactory extends Luc
       Path lockFile = tempDir.resolve("test.lock");
       Files.createFile(lockFile);
       
-      Lock l = new NativeFSLockFactory(tempDir).makeLock("test.lock");
+      Directory dir = FSDirectory.open(tempDir, NativeFSLockFactory.INSTANCE);
+      Lock l = dir.makeLock("test.lock");
       assertTrue("failed to obtain lock", l.obtain());
       l.close();
       assertFalse("failed to release lock", l.isLocked());
       Files.deleteIfExists(lockFile);
     }
 
-    // Verify: NativeFSLockFactory assigns null as lockPrefix if the lockDir is inside directory
-    public void testNativeFSLockFactoryPrefix() throws IOException {
-
-      Path fdir1 = createTempDir("TestLockFactory.8");
-      Path fdir2 = createTempDir("TestLockFactory.8.Lockdir");
-      Directory dir1 = newFSDirectory(fdir1, new NativeFSLockFactory(fdir1));
-      // same directory, but locks are stored somewhere else. The prefix of the lock factory should != null
-      Directory dir2 = newFSDirectory(fdir1, new NativeFSLockFactory(fdir2));
-
-      String prefix1 = dir1.getLockFactory().getLockPrefix();
-      assertNull("Lock prefix for lockDir same as directory should be null", prefix1);
-      
-      String prefix2 = dir2.getLockFactory().getLockPrefix();
-      assertNotNull("Lock prefix for lockDir outside of directory should be not null", prefix2);
-
-      dir1.close();
-      dir2.close();
-      IOUtils.rm(fdir1, fdir2);
-    }
-
-    // Verify: default LockFactory has no prefix (ie
-    // write.lock is stored in index):
-    public void testDefaultFSLockFactoryPrefix() throws IOException {
-
-      // Make sure we get null prefix, which wont happen if setLockFactory is ever called.
-      Path dirName = createTempDir("TestLockFactory.10");
-
-      Directory dir = new SimpleFSDirectory(dirName);
-      assertNull("Default lock prefix should be null", dir.getLockFactory().getLockPrefix());
-      dir.close();
-      
-      dir = new MMapDirectory(dirName);
-      assertNull("Default lock prefix should be null", dir.getLockFactory().getLockPrefix());
-      dir.close();
-      
-      dir = new NIOFSDirectory(dirName);
-      assertNull("Default lock prefix should be null", dir.getLockFactory().getLockPrefix());
-      dir.close();
- 
-      IOUtils.rm(dirName);
-    }
-
     private class WriterThread extends Thread { 
         private Directory dir;
         private int numIteration;
@@ -348,29 +299,19 @@ public class TestLockFactory extends Luc
         }
     }
 
-    public class MockLockFactory extends LockFactory {
+    class MockLockFactory extends LockFactory {
 
-        public boolean lockPrefixSet;
         public Map<String,Lock> locksCreated = Collections.synchronizedMap(new HashMap<String,Lock>());
         public int makeLockCount = 0;
 
         @Override
-        public void setLockPrefix(String lockPrefix) {    
-            super.setLockPrefix(lockPrefix);
-            lockPrefixSet = true;
-        }
-
-        @Override
-        synchronized public Lock makeLock(String lockName) {
+        public synchronized Lock makeLock(Directory dir, String lockName) {
             Lock lock = new MockLock();
             locksCreated.put(lockName, lock);
             makeLockCount++;
             return lock;
         }
 
-        @Override
-        public void clearLock(String specificLockName) {}
-
         public class MockLock extends Lock {
             public int lockAttempts;
 

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMockDirectoryWrapper.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMockDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMockDirectoryWrapper.java Tue Nov 11 16:21:15 2014
@@ -39,8 +39,7 @@ public class TestMockDirectoryWrapper ex
   }
   
   public void testFailIfIndexWriterNotClosedChangeLockFactory() throws IOException {
-    MockDirectoryWrapper dir = newMockDirectory();
-    dir.setLockFactory(new SingleInstanceLockFactory());
+    MockDirectoryWrapper dir = newMockDirectory(random(), new SingleInstanceLockFactory());
     IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null));
     try {
       dir.close();

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestMultiMMap.java Tue Nov 11 16:21:15 2014
@@ -39,7 +39,7 @@ public class TestMultiMMap extends BaseD
 
   @Override
   protected Directory getDirectory(Path path) throws IOException {
-    return new MMapDirectory(path, null, 1<<TestUtil.nextInt(random(), 10, 28));
+    return new MMapDirectory(path, 1<<TestUtil.nextInt(random(), 10, 28));
   }
   
   @Override
@@ -177,7 +177,7 @@ public class TestMultiMMap extends BaseD
 
   public void testSeekZero() throws Exception {
     for (int i = 0; i < 31; i++) {
-      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSeekZero"), null, 1<<i);
+      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSeekZero"), 1<<i);
       IndexOutput io = mmapDir.createOutput("zeroBytes", newIOContext(random()));
       io.close();
       IndexInput ii = mmapDir.openInput("zeroBytes", newIOContext(random()));
@@ -189,7 +189,7 @@ public class TestMultiMMap extends BaseD
   
   public void testSeekSliceZero() throws Exception {
     for (int i = 0; i < 31; i++) {
-      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSeekSliceZero"), null, 1<<i);
+      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSeekSliceZero"), 1<<i);
       IndexOutput io = mmapDir.createOutput("zeroBytes", newIOContext(random()));
       io.close();
       IndexInput slicer = mmapDir.openInput("zeroBytes", newIOContext(random()));
@@ -203,7 +203,7 @@ public class TestMultiMMap extends BaseD
   
   public void testSeekEnd() throws Exception {
     for (int i = 0; i < 17; i++) {
-      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSeekEnd"), null, 1<<i);
+      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSeekEnd"), 1<<i);
       IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
       byte bytes[] = new byte[1<<i];
       random().nextBytes(bytes);
@@ -221,7 +221,7 @@ public class TestMultiMMap extends BaseD
   
   public void testSeekSliceEnd() throws Exception {
     for (int i = 0; i < 17; i++) {
-      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSeekSliceEnd"), null, 1<<i);
+      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSeekSliceEnd"), 1<<i);
       IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
       byte bytes[] = new byte[1<<i];
       random().nextBytes(bytes);
@@ -241,7 +241,7 @@ public class TestMultiMMap extends BaseD
   
   public void testSeeking() throws Exception {
     for (int i = 0; i < 10; i++) {
-      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSeeking"), null, 1<<i);
+      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSeeking"), 1<<i);
       IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
       byte bytes[] = new byte[1<<(i+1)]; // make sure we switch buffers
       random().nextBytes(bytes);
@@ -268,7 +268,7 @@ public class TestMultiMMap extends BaseD
   // the various offset+length and just does readBytes.
   public void testSlicedSeeking() throws Exception {
     for (int i = 0; i < 10; i++) {
-      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSlicedSeeking"), null, 1<<i);
+      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSlicedSeeking"), 1<<i);
       IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
       byte bytes[] = new byte[1<<(i+1)]; // make sure we switch buffers
       random().nextBytes(bytes);
@@ -292,7 +292,7 @@ public class TestMultiMMap extends BaseD
 
   public void testSliceOfSlice() throws Exception {
     for (int i = 0; i < 10; i++) {
-      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSliceOfSlice"), null, 1<<i);
+      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testSliceOfSlice"), 1<<i);
       IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
       byte bytes[] = new byte[1<<(i+1)]; // make sure we switch buffers
       random().nextBytes(bytes);
@@ -328,13 +328,14 @@ public class TestMultiMMap extends BaseD
   
   public void testRandomChunkSizes() throws Exception {
     int num = atLeast(10);
-    for (int i = 0; i < num; i++)
+    for (int i = 0; i < num; i++) {
       assertChunking(random(), TestUtil.nextInt(random(), 20, 100));
+    }
   }
   
   private void assertChunking(Random random, int chunkSize) throws Exception {
     Path path = createTempDir("mmap" + chunkSize);
-    MMapDirectory mmapDir = new MMapDirectory(path, null, chunkSize);
+    MMapDirectory mmapDir = new MMapDirectory(path, chunkSize);
     // we will map a lot, try to turn on the unmap hack
     if (MMapDirectory.UNMAP_SUPPORTED)
       mmapDir.setUseUnmap(true);
@@ -363,7 +364,7 @@ public class TestMultiMMap extends BaseD
   public void testImplementations() throws Exception {
     for (int i = 2; i < 12; i++) {
       final int chunkSize = 1<<i;
-      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testImplementations"), null, chunkSize);
+      MMapDirectory mmapDir = new MMapDirectory(createTempDir("testImplementations"), chunkSize);
       IndexOutput io = mmapDir.createOutput("bytes", newIOContext(random()));
       int size = random().nextInt(chunkSize * 2) + 3; // add some buffer of 3 for slice tests
       byte bytes[] = new byte[size];

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/store/TestWindowsMMap.java Tue Nov 11 16:21:15 2014
@@ -63,7 +63,7 @@ public class TestWindowsMMap extends Luc
     // may take some time until the files are finally dereferenced. So clean the
     // directory up front, or otherwise new IndexWriter will fail.
     Path dirPath = createTempDir("testLuceneMmap");
-    MMapDirectory dir = new MMapDirectory(dirPath, null);
+    MMapDirectory dir = new MMapDirectory(dirPath);
     
     // plan to add a set of useful stopwords, consider changing some of the
     // interior filters.

Modified: lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java (original)
+++ lucene/dev/branches/lucene6005/lucene/core/src/test/org/apache/lucene/util/TestAttributeSource.java Tue Nov 11 16:21:15 2014
@@ -159,4 +159,21 @@ public class TestAttributeSource extends
     assertTrue("The hashCode is identical, so the captured state was preserved.", hash1 != src1.hashCode());
     assertEquals(src2.hashCode(), src1.hashCode());
   }
+  
+  public void testClonePayloadAttribute() throws Exception {
+    // LUCENE-6055: verify that PayloadAttribute.clone() does deep cloning.
+    PayloadAttributeImpl src = new PayloadAttributeImpl(new BytesRef(new byte[] { 1, 2, 3 }));
+    
+    // test clone()
+    PayloadAttributeImpl clone = src.clone();
+    clone.getPayload().bytes[0] = 10; // modify one byte, srcBytes shouldn't change
+    assertEquals("clone() wasn't deep", 1, src.getPayload().bytes[0]);
+    
+    // test copyTo()
+    clone = new PayloadAttributeImpl();
+    src.copyTo(clone);
+    clone.getPayload().bytes[0] = 10; // modify one byte, srcBytes shouldn't change
+    assertEquals("clone() wasn't deep", 1, src.getPayload().bytes[0]);
+  }
+  
 }

Modified: lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java (original)
+++ lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java Tue Nov 11 16:21:15 2014
@@ -90,6 +90,7 @@ public class NativeUnixDirectory extends
   /** Create a new NIOFSDirectory for the named location.
    * 
    * @param path the path of the directory
+   * @param lockFactory to use
    * @param mergeBufferSize Size of buffer to use for
    *    merging.  See {@link #DEFAULT_MERGE_BUFFER_SIZE}.
    * @param minBytesDirect Merges, or files to be opened for
@@ -99,8 +100,8 @@ public class NativeUnixDirectory extends
    * @param delegate fallback Directory for non-merges
    * @throws IOException If there is a low-level I/O error
    */
-  public NativeUnixDirectory(Path path, int mergeBufferSize, long minBytesDirect, Directory delegate) throws IOException {
-    super(path, delegate.getLockFactory());
+  public NativeUnixDirectory(Path path, int mergeBufferSize, long minBytesDirect, LockFactory lockFactory, Directory delegate) throws IOException {
+    super(path, lockFactory);
     if ((mergeBufferSize & ALIGN) != 0) {
       throw new IllegalArgumentException("mergeBufferSize must be 0 mod " + ALIGN + " (got: " + mergeBufferSize + ")");
     }
@@ -112,11 +113,22 @@ public class NativeUnixDirectory extends
   /** Create a new NIOFSDirectory for the named location.
    * 
    * @param path the path of the directory
+   * @param lockFactory the lock factory to use
+   * @param delegate fallback Directory for non-merges
+   * @throws IOException If there is a low-level I/O error
+   */
+  public NativeUnixDirectory(Path path, LockFactory lockFactory, Directory delegate) throws IOException {
+    this(path, DEFAULT_MERGE_BUFFER_SIZE, DEFAULT_MIN_BYTES_DIRECT, lockFactory, delegate);
+  }  
+
+  /** Create a new NIOFSDirectory for the named location with {@link FSLockFactory#getDefault()}.
+   * 
+   * @param path the path of the directory
    * @param delegate fallback Directory for non-merges
    * @throws IOException If there is a low-level I/O error
    */
   public NativeUnixDirectory(Path path, Directory delegate) throws IOException {
-    this(path, DEFAULT_MERGE_BUFFER_SIZE, DEFAULT_MIN_BYTES_DIRECT, delegate);
+    this(path, DEFAULT_MERGE_BUFFER_SIZE, DEFAULT_MIN_BYTES_DIRECT, FSLockFactory.getDefault(), delegate);
   }  
 
   @Override

Modified: lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/RAFDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/RAFDirectory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/RAFDirectory.java (original)
+++ lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/RAFDirectory.java Tue Nov 11 16:21:15 2014
@@ -40,8 +40,7 @@ public class RAFDirectory extends FSDire
   /** Create a new RAFDirectory for the named location.
    *
    * @param path the path of the directory
-   * @param lockFactory the lock factory to use, or null for the default
-   * ({@link NativeFSLockFactory});
+   * @param lockFactory the lock factory to use
    * @throws IOException if there is a low-level I/O error
    */
   public RAFDirectory(Path path, LockFactory lockFactory) throws IOException {
@@ -49,14 +48,13 @@ public class RAFDirectory extends FSDire
     path.toFile(); // throw exception if we can't get a File
   }
   
-  /** Create a new SimpleFSDirectory for the named location and {@link NativeFSLockFactory}.
+  /** Create a new SimpleFSDirectory for the named location and {@link FSLockFactory#getDefault()}.
    *
    * @param path the path of the directory
    * @throws IOException if there is a low-level I/O error
    */
   public RAFDirectory(Path path) throws IOException {
-    super(path, null);
-    path.toFile(); // throw exception if we can't get a File
+    this(path, FSLockFactory.getDefault());
   }
 
   /** Creates an IndexInput for the file with the given name. */

Modified: lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/WindowsDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/WindowsDirectory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/WindowsDirectory.java (original)
+++ lucene/dev/branches/lucene6005/lucene/misc/src/java/org/apache/lucene/store/WindowsDirectory.java Tue Nov 11 16:21:15 2014
@@ -22,7 +22,6 @@ import java.io.EOFException;
 import java.nio.file.Path;
 
 import org.apache.lucene.store.Directory; // javadoc
-import org.apache.lucene.store.NativeFSLockFactory; // javadoc
 
 /**
  * Native {@link Directory} implementation for Microsoft Windows.
@@ -52,21 +51,20 @@ public class WindowsDirectory extends FS
   /** Create a new WindowsDirectory for the named location.
    * 
    * @param path the path of the directory
-   * @param lockFactory the lock factory to use, or null for the default
-   * ({@link NativeFSLockFactory});
+   * @param lockFactory the lock factory to use
    * @throws IOException If there is a low-level I/O error
    */
   public WindowsDirectory(Path path, LockFactory lockFactory) throws IOException {
     super(path, lockFactory);
   }
 
-  /** Create a new WindowsDirectory for the named location and {@link NativeFSLockFactory}.
+  /** Create a new WindowsDirectory for the named location and {@link FSLockFactory#getDefault()}.
    *
    * @param path the path of the directory
    * @throws IOException If there is a low-level I/O error
    */
   public WindowsDirectory(Path path) throws IOException {
-    super(path, null);
+    this(path, FSLockFactory.getDefault());
   }
 
   @Override

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java Tue Nov 11 16:21:15 2014
@@ -317,28 +317,6 @@ public abstract class BaseCompoundFormat
     dir.close();
   }
   
-  // test that cfs reader is read-only
-  public void testClearLockDisabled() throws IOException {
-    final String testfile = "_123.test";
-
-    Directory dir = newDirectory();
-    IndexOutput out = dir.createOutput(testfile, IOContext.DEFAULT);
-    out.writeInt(3);
-    out.close();
- 
-    SegmentInfo si = newSegmentInfo(dir, "_123");
-    si.getCodec().compoundFormat().write(dir, si, Collections.<String>emptyList(), MergeState.CheckAbort.NONE, IOContext.DEFAULT);
-    Directory cfs = si.getCodec().compoundFormat().getCompoundReader(dir, si, IOContext.DEFAULT);
-    try {
-      cfs.clearLock("foobar");
-      fail("didn't get expected exception");
-    } catch (UnsupportedOperationException expected) {
-      // expected UOE
-    }
-    cfs.close();
-    dir.close();
-  }
-  
   /** 
    * This test creates a compound file based on a large number of files of
    * various length. The file content is generated randomly. The sizes range

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java Tue Nov 11 16:21:15 2014
@@ -74,7 +74,7 @@ public class MockDirectoryWrapper extend
   boolean assertNoDeleteOpenFile = false;
   boolean preventDoubleWrite = true;
   boolean trackDiskUsage = false;
-  boolean wrapLockFactory = true;
+  boolean wrapLocking = true;
   boolean useSlowOpenClosers = true;
   boolean enableVirusScanner = true;
   boolean allowRandomFileNotFoundException = true;
@@ -86,7 +86,6 @@ public class MockDirectoryWrapper extend
   volatile boolean crashed;
   private ThrottledIndexOutput throttledOutput;
   private Throttling throttling = Throttling.SOMETIMES;
-  protected LockFactory lockFactory;
 
   final AtomicInteger inputCloneCount = new AtomicInteger();
 
@@ -129,8 +128,6 @@ public class MockDirectoryWrapper extend
     this.randomState = new Random(random.nextInt());
     this.throttledOutput = new ThrottledIndexOutput(ThrottledIndexOutput
         .mBitsToBytes(40 + randomState.nextInt(10)), 5 + randomState.nextInt(5), null);
-    // force wrapping of lockfactory
-    this.lockFactory = new MockLockFactoryWrapper(this, delegate.getLockFactory());
     init();
   }
 
@@ -709,16 +706,16 @@ public class MockDirectoryWrapper extend
   }
   
   /**
-   * Set to false if you want to return the pure lockfactory
-   * and not wrap it with MockLockFactoryWrapper.
+   * Set to false if you want to return the pure {@link LockFactory} and not
+   * wrap all lock with {@code AssertingLock}.
    * <p>
-   * Be careful if you turn this off: MockDirectoryWrapper might
-   * no longer be able to detect if you forget to close an IndexWriter,
+   * Be careful if you turn this off: {@code MockDirectoryWrapper} might
+   * no longer be able to detect if you forget to close an {@link IndexWriter},
    * and spit out horribly scary confusing exceptions instead of
    * simply telling you that.
    */
-  public void setWrapLockFactory(boolean v) {
-    this.wrapLockFactory = v;
+  public void setAssertLocks(boolean v) {
+    this.wrapLocking = v;
   }
 
   @Override
@@ -987,39 +984,43 @@ public class MockDirectoryWrapper extend
   @Override
   public synchronized Lock makeLock(String name) {
     maybeYield();
-    return getLockFactory().makeLock(name);
-  }
-
-  @Override
-  public synchronized void clearLock(String name) throws IOException {
-    maybeYield();
-    getLockFactory().clearLock(name);
+    if (wrapLocking) {
+      return new AssertingLock(super.makeLock(name), name);
+    } else {
+      return super.makeLock(name);
+    }
   }
+  
+  private final class AssertingLock extends Lock {
+    private final Lock delegateLock;
+    private final String name;
+    
+    AssertingLock(Lock delegate, String name) {
+      this.delegateLock = delegate;
+      this.name = name;
+    }
 
-  @Override
-  public synchronized void setLockFactory(LockFactory lockFactory) throws IOException {
-    maybeYield();
-    // sneaky: we must pass the original this way to the dir, because
-    // some impls (e.g. FSDir) do instanceof here.
-    in.setLockFactory(lockFactory);
-    // now set our wrapped factory here
-    this.lockFactory = new MockLockFactoryWrapper(this, lockFactory);
-  }
+    @Override
+    public boolean obtain() throws IOException {
+      if (delegateLock.obtain()) {
+        assert delegateLock == NoLockFactory.SINGLETON_LOCK || !openLocks.containsKey(name);
+        openLocks.put(name, new RuntimeException("lock \"" + name + "\" was not released"));
+        return true;
+      } else {
+        return false;
+      }
+    }
 
-  @Override
-  public synchronized LockFactory getLockFactory() {
-    maybeYield();
-    if (wrapLockFactory) {
-      return lockFactory;
-    } else {
-      return in.getLockFactory();
+    @Override
+    public void close() throws IOException {
+      delegateLock.close();
+      openLocks.remove(name);
     }
-  }
 
-  @Override
-  public synchronized String getLockID() {
-    maybeYield();
-    return in.getLockID();
+    @Override
+    public boolean isLocked() throws IOException {
+      return delegateLock.isLocked();
+    }
   }
 
   @Override

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Tue Nov 11 16:21:15 2014
@@ -115,6 +115,7 @@ import org.apache.lucene.search.QueryUti
 import org.apache.lucene.store.BaseDirectoryWrapper;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.FSLockFactory;
 import org.apache.lucene.store.FlushInfo;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IOContext.Context;
@@ -1214,6 +1215,14 @@ public abstract class LuceneTestCase ext
     return wrapDirectory(r, newDirectoryImpl(r, TEST_DIRECTORY), rarely(r));
   }
 
+  /**
+   * Returns a new Directory instance, using the specified random.
+   * See {@link #newDirectory()} for more information.
+   */
+  public static BaseDirectoryWrapper newDirectory(Random r, LockFactory lf) {
+    return wrapDirectory(r, newDirectoryImpl(r, TEST_DIRECTORY, lf), rarely(r));
+  }
+
   public static MockDirectoryWrapper newMockDirectory() {
     return newMockDirectory(random());
   }
@@ -1222,8 +1231,16 @@ public abstract class LuceneTestCase ext
     return (MockDirectoryWrapper) wrapDirectory(r, newDirectoryImpl(r, TEST_DIRECTORY), false);
   }
 
+  public static MockDirectoryWrapper newMockDirectory(Random r, LockFactory lf) {
+    return (MockDirectoryWrapper) wrapDirectory(r, newDirectoryImpl(r, TEST_DIRECTORY, lf), false);
+  }
+
   public static MockDirectoryWrapper newMockFSDirectory(Path f) {
-    return (MockDirectoryWrapper) newFSDirectory(f, null, false);
+    return (MockDirectoryWrapper) newFSDirectory(f, FSLockFactory.getDefault(), false);
+  }
+
+  public static MockDirectoryWrapper newMockFSDirectory(Path f, LockFactory lf) {
+    return (MockDirectoryWrapper) newFSDirectory(f, lf, false);
   }
 
   /**
@@ -1237,7 +1254,7 @@ public abstract class LuceneTestCase ext
 
   /** Returns a new FSDirectory instance over the given file, which must be a folder. */
   public static BaseDirectoryWrapper newFSDirectory(Path f) {
-    return newFSDirectory(f, null);
+    return newFSDirectory(f, FSLockFactory.getDefault());
   }
 
   /** Returns a new FSDirectory instance over the given file, which must be a folder. */
@@ -1261,11 +1278,8 @@ public abstract class LuceneTestCase ext
         clazz = CommandLineUtil.loadFSDirectoryClass(fsdirClass);
       }
 
-      Directory fsdir = newFSDirectoryImpl(clazz, f);
+      Directory fsdir = newFSDirectoryImpl(clazz, f, lf);
       BaseDirectoryWrapper wrapped = wrapDirectory(random(), fsdir, bare);
-      if (lf != null) {
-        wrapped.setLockFactory(lf);
-      }
       return wrapped;
     } catch (Exception e) {
       Rethrow.rethrow(e);
@@ -1462,17 +1476,21 @@ public abstract class LuceneTestCase ext
     }
   }
 
-  private static Directory newFSDirectoryImpl(Class<? extends FSDirectory> clazz, Path path) throws IOException {
+  private static Directory newFSDirectoryImpl(Class<? extends FSDirectory> clazz, Path path, LockFactory lf) throws IOException {
     FSDirectory d = null;
     try {
-      d = CommandLineUtil.newFSDirectory(clazz, path);
-    } catch (NoSuchMethodException | InstantiationException | IllegalAccessException | InvocationTargetException e) {
+      d = CommandLineUtil.newFSDirectory(clazz, path, lf);
+    } catch (ReflectiveOperationException e) {
       Rethrow.rethrow(e);
     }
     return d;
   }
 
   static Directory newDirectoryImpl(Random random, String clazzName) {
+    return newDirectoryImpl(random, clazzName, FSLockFactory.getDefault());
+  }
+  
+  static Directory newDirectoryImpl(Random random, String clazzName, LockFactory lf) {
     if (clazzName.equals("random")) {
       if (rarely(random)) {
         clazzName = RandomPicks.randomFrom(random, CORE_DIRECTORIES);
@@ -1486,21 +1504,27 @@ public abstract class LuceneTestCase ext
       // If it is a FSDirectory type, try its ctor(Path)
       if (FSDirectory.class.isAssignableFrom(clazz)) {
         final Path dir = createTempDir("index-" + clazzName);
-        return newFSDirectoryImpl(clazz.asSubclass(FSDirectory.class), dir);
+        return newFSDirectoryImpl(clazz.asSubclass(FSDirectory.class), dir, lf);
       }
 
-      // See if it has a Path ctor even though it's not an
+      // See if it has a Path/LockFactory ctor even though it's not an
       // FSDir subclass:
-      Constructor<? extends Directory> pathCtor = null;
       try {
-        pathCtor = clazz.getConstructor(Path.class);
+        Constructor<? extends Directory> pathCtor = clazz.getConstructor(Path.class, LockFactory.class);
+        final Path dir = createTempDir("index");
+        return pathCtor.newInstance(dir, lf);
       } catch (NoSuchMethodException nsme) {
         // Ignore
       }
-
-      if (pathCtor != null) {
-        final Path dir = createTempDir("index");
-        return pathCtor.newInstance(dir);
+      
+      // the remaining dirs are no longer filesystem based, so we must check that the passedLockFactory is not file based:
+      if (!(lf instanceof FSLockFactory)) {
+        // try ctor with only LockFactory (e.g. RAMDirectory)
+        try {
+          return clazz.getConstructor(LockFactory.class).newInstance(lf);
+        } catch (NoSuchMethodException nsme) {
+          // Ignore
+        }
       }
 
       // try empty ctor

Modified: lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java (original)
+++ lucene/dev/branches/lucene6005/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java Tue Nov 11 16:21:15 2014
@@ -215,9 +215,9 @@ public final class TestUtil {
    *  look for any other corruption.  */
   public static CheckIndex.Status checkIndex(Directory dir, boolean crossCheckTermVectors, boolean failFast) throws IOException {
     ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
-    // TODO: actually use the dir's lock factory, unless test uses a special method?
+    // TODO: actually use the dir's locking, unless test uses a special method?
     // some tests e.g. exception tests become much more complicated if they have to close the writer
-    try (CheckIndex checker = new CheckIndex(dir, NoLockFactory.getNoLockFactory().makeLock("bogus"))) {
+    try (CheckIndex checker = new CheckIndex(dir, NoLockFactory.INSTANCE.makeLock(dir, "bogus"))) {
       checker.setCrossCheckTermVectors(crossCheckTermVectors);
       checker.setFailFast(failFast);
       checker.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8), false);

Modified: lucene/dev/branches/lucene6005/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/CHANGES.txt?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/CHANGES.txt (original)
+++ lucene/dev/branches/lucene6005/solr/CHANGES.txt Tue Nov 11 16:21:15 2014
@@ -272,6 +272,16 @@ Bug Fixes
 * SOLR-6706: /update/json/docs throws RuntimeException if a nested structure
   contains a non-leaf float field (Noble Paul, shalin)
 
+* SOLR-6579:SnapPuller Replication blocks clean shutdown of tomcat
+  (Philip Black-Knight via Noble Paul)
+
+* SOLR-6721: ZkController.ensureReplicaInLeaderInitiatedRecovery puts replica
+  in local map before writing to ZK. (shalin)
+
+* SOLR-6679: Remove suggester component from stock solrconfig.xml since
+  it caused long startup times on lage indexes even when it wasn't used.
+  (yonik)
+
 Optimizations
 ----------------------
 

Modified: lucene/dev/branches/lucene6005/solr/bin/solr
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/bin/solr?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/bin/solr (original)
+++ lucene/dev/branches/lucene6005/solr/bin/solr Tue Nov 11 16:21:15 2014
@@ -988,7 +988,18 @@ function launch_solr() {
   
   SOLR_ADDL_ARGS="$2"
 
-  # commented out debugging info
+  # deal with Java version specific GC and other flags
+  JAVA_VERSION=`echo "$(java -version 2>&1)" | grep "java version" | awk '{ print substr($3, 2, length($3)-2); }'`
+  if [ "${JAVA_VERSION:0:3}" == "1.7" ]; then
+    # Specific Java version hacking
+    GC_TUNE="$GC_TUNE -XX:CMSFullGCsBeforeCompaction=1 -XX:CMSTriggerPermRatio=80"
+    JAVA_MINOR_VERSION=${JAVA_VERSION:(-2)}
+    if [[ $JAVA_MINOR_VERSION -ge 40 && $JAVA_MINOR_VERSION -le 51 ]]; then
+      GC_TUNE="$GC_TUNE -XX:-UseSuperWord"
+      echo -e "\nWARNING: Java version $JAVA_VERSION has known bugs with Lucene and requires the -XX:-UseSuperWord flag. Please consider upgrading your JVM.\n"
+    fi
+  fi
+
   if $verbose ; then
     echo -e "\nStarting Solr using the following settings:"
     echo -e "    SOLR_SERVER_DIR = $SOLR_SERVER_DIR"
@@ -1028,7 +1039,9 @@ $SOLR_HOST_ARG -Djetty.port=$SOLR_PORT \
   if [ "$SOLR_MODE" == "solrcloud" ]; then
     IN_CLOUD_MODE=" in SolrCloud mode"
   fi
-  
+
+  mkdir -p $SOLR_SERVER_DIR/logs
+
   if [ "$run_in_foreground" == "true" ]; then
     echo -e "\nStarting Solr$IN_CLOUD_MODE on port $SOLR_PORT from $SOLR_SERVER_DIR\n"
     $JAVA $SOLR_START_OPTS $SOLR_ADDL_ARGS -XX:OnOutOfMemoryError="$SOLR_TIP/bin/oom_solr.sh $SOLR_PORT" -jar start.jar
@@ -1174,7 +1187,8 @@ else
     echo $CLOUD_REPFACT
 
     USER_INPUT=
-    read -e -p "Please choose a configuration for the $CLOUD_COLLECTION collection, available options are:\nbasic_configs, data_driven_schema_configs, or sample_techproducts_configs [data_driven_schema_configs] " USER_INPUT
+    echo "Please choose a configuration for the $CLOUD_COLLECTION collection, available options are:"
+    read -e -p "basic_configs, data_driven_schema_configs, or sample_techproducts_configs [data_driven_schema_configs] " USER_INPUT
     # trim whitespace out of the user input
     CLOUD_CONFIG=`echo $USER_INPUT | tr -d ' '`
 

Modified: lucene/dev/branches/lucene6005/solr/bin/solr.in.sh
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/bin/solr.in.sh?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/bin/solr.in.sh (original)
+++ lucene/dev/branches/lucene6005/solr/bin/solr.in.sh Tue Nov 11 16:21:15 2014
@@ -19,15 +19,14 @@
 #SOLR_JAVA_HOME=""
 
 # Increase Java Min/Max Heap as needed to support your indexing / query needs
-SOLR_JAVA_MEM="-Xms512m -Xmx512m -XX:MaxPermSize=256m -XX:PermSize=256m"
+SOLR_JAVA_MEM="-Xms512m -Xmx512m"
 
 # Enable verbose GC logging
 GC_LOG_OPTS="-verbose:gc -XX:+PrintHeapAtGC -XX:+PrintGCDetails \
 -XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps -XX:+PrintTenuringDistribution -XX:+PrintGCApplicationStoppedTime"
 
 # These GC settings have shown to work well for a number of common Solr workloads
-GC_TUNE="-XX:-UseSuperWord \
--XX:NewRatio=3 \
+GC_TUNE="-XX:NewRatio=3 \
 -XX:SurvivorRatio=4 \
 -XX:TargetSurvivorRatio=90 \
 -XX:MaxTenuringThreshold=8 \
@@ -36,23 +35,11 @@ GC_TUNE="-XX:-UseSuperWord \
 -XX:ConcGCThreads=4 -XX:ParallelGCThreads=4 \
 -XX:+CMSScavengeBeforeRemark \
 -XX:PretenureSizeThreshold=64m \
--XX:CMSFullGCsBeforeCompaction=1 \
 -XX:+UseCMSInitiatingOccupancyOnly \
 -XX:CMSInitiatingOccupancyFraction=50 \
--XX:CMSTriggerPermRatio=80 \
 -XX:CMSMaxAbortablePrecleanTime=6000 \
 -XX:+CMSParallelRemarkEnabled \
--XX:+ParallelRefProcEnabled \
--XX:+AggressiveOpts"
-
-# Mac OSX and Cygwin don't seem to like the UseLargePages flag
-thisOs=`uname -s`
-# for now, we don't support running this script from cygwin due to problems
-# like not having lsof, ps waux, curl, and awkward directory handling
-if [[ "$thisOs" != "Darwin" && "${thisOs:0:6}" != "CYGWIN" ]]; then
-  # UseLargePages flag causes JVM crash on Mac OSX
-  GC_TUNE="$GC_TUNE -XX:+UseLargePages"
-fi
+-XX:+ParallelRefProcEnabled"
 
 # Set the ZooKeeper connection string if using an external ZooKeeper ensemble
 # e.g. host1:2181,host2:2181/chroot

Modified: lucene/dev/branches/lucene6005/solr/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/common-build.xml?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/common-build.xml (original)
+++ lucene/dev/branches/lucene6005/solr/common-build.xml Tue Nov 11 16:21:15 2014
@@ -141,10 +141,16 @@
     </sequential>
   </macrodef>
 
-  <!-- we don't want to run HDFS tests on Windows, because they require Cygwin.
-    If you have Cygwin, you can override this property on command line: -->
+  <!-- 
+    - We don't test HDFS on Java 7 because it causes permgen errors. Java 8 no longer has permgen.
+    - We don't want to run HDFS tests on Windows, because they require Cygwin.
+    If you have Cygwin or manually raised permgen, you can override this property on command line:
+  -->
   <condition property="tests.disableHdfs" value="true">
-    <os family="windows"/>
+    <or>
+      <equals arg1="${build.java.runtime}" arg2="1.7"/>
+      <os family="windows"/>
+    </or>
   </condition>
 
   <target name="validate" depends="compile-tools">

Modified: lucene/dev/branches/lucene6005/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java (original)
+++ lucene/dev/branches/lucene6005/solr/contrib/map-reduce/src/java/org/apache/solr/hadoop/TreeMergeOutputFormat.java Tue Nov 11 16:21:15 2014
@@ -94,7 +94,7 @@ public class TreeMergeOutputFormat exten
       writeShardNumberFile(context);      
       heartBeater.needHeartBeat();
       try {
-        Directory mergedIndex = new HdfsDirectory(workDir, NoLockFactory.getNoLockFactory(), context.getConfiguration());
+        Directory mergedIndex = new HdfsDirectory(workDir, NoLockFactory.INSTANCE, context.getConfiguration());
         
         // TODO: shouldn't we pull the Version from the solrconfig.xml?
         IndexWriterConfig writerConfig = new IndexWriterConfig(null)
@@ -128,7 +128,7 @@ public class TreeMergeOutputFormat exten
         
         Directory[] indexes = new Directory[shards.size()];
         for (int i = 0; i < shards.size(); i++) {
-          indexes[i] = new HdfsDirectory(shards.get(i), NoLockFactory.getNoLockFactory(), context.getConfiguration());
+          indexes[i] = new HdfsDirectory(shards.get(i), NoLockFactory.INSTANCE, context.getConfiguration());
         }
 
         context.setStatus("Logically merging " + shards.size() + " shards into one shard");

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/cloud/ZkController.java Tue Nov 11 16:21:15 2014
@@ -190,7 +190,7 @@ public final class ZkController {
   private volatile boolean isClosed;
   
   // keeps track of replicas that have been asked to recover by leaders running on this node
-  private Map<String,String> replicasInLeaderInitiatedRecovery = new HashMap<String,String>();
+  private final Map<String,String> replicasInLeaderInitiatedRecovery = new HashMap<String,String>();
   
   // This is an expert and unsupported development mode that does not create
   // an Overseer or register a /live node. This let's you monitor the cluster
@@ -1925,10 +1925,10 @@ public final class ZkController {
       // if the replica's state is not DOWN right now, make it so ...
       // we only really need to try to send the recovery command if the node itself is "live"
       if (getZkStateReader().getClusterState().liveNodesContain(replicaNodeName)) {
-        replicasInLeaderInitiatedRecovery.put(replicaUrl, 
-            getLeaderInitiatedRecoveryZnodePath(collection, shardId, replicaCoreNodeName));
         // create a znode that requires the replica needs to "ack" to verify it knows it was out-of-sync
         updateLeaderInitiatedRecoveryState(collection, shardId, replicaCoreNodeName, ZkStateReader.DOWN);
+        replicasInLeaderInitiatedRecovery.put(replicaUrl,
+            getLeaderInitiatedRecoveryZnodePath(collection, shardId, replicaCoreNodeName));
         log.info("Put replica core={} coreNodeName={} on "+
           replicaNodeName+" into leader-initiated recovery.", replicaCoreProps.getCoreName(), replicaCoreNodeName);
         publishDownState = true;        
@@ -1957,8 +1957,8 @@ public final class ZkController {
     }
     
     return nodeIsLive;
-  }  
-  
+  }
+
   public boolean isReplicaInRecoveryHandling(String replicaUrl) {
     boolean exists = false;
     synchronized (replicasInLeaderInitiatedRecovery) {

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java Tue Nov 11 16:21:15 2014
@@ -347,7 +347,7 @@ public abstract class CachingDirectoryFa
       }
       
       if (directory == null) {
-        directory = create(fullPath, createLockFactory(fullPath, rawLockType), dirContext);
+        directory = create(fullPath, createLockFactory(rawLockType), dirContext);
         boolean success = false;
         try {
           directory = rateLimit(directory);

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java Tue Nov 11 16:21:15 2014
@@ -81,11 +81,10 @@ public abstract class DirectoryFactory i
   
   /**
    * Creates a new LockFactory for a given path.
-   * @param lockPath the path of the index directory
    * @param rawLockType A string value as passed in config. Every factory should at least support 'none' to disable locking.
    * @throws IOException If there is a low-level I/O error.
    */
-  protected abstract LockFactory createLockFactory(String lockPath, String rawLockType) throws IOException;
+  protected abstract LockFactory createLockFactory(String rawLockType) throws IOException;
   
   /**
    * Returns true if a Directory exists for a given path.

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java Tue Nov 11 16:21:15 2014
@@ -17,25 +17,21 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
-import java.io.File;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+
 import java.io.IOException;
 import java.net.URI;
 import java.net.URLEncoder;
 import java.util.Locale;
 
 import org.apache.hadoop.conf.Configuration;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.LockFactory;
 import org.apache.lucene.store.NRTCachingDirectory;
-import org.apache.lucene.store.NativeFSLockFactory;
 import org.apache.lucene.store.NoLockFactory;
-import org.apache.lucene.store.SimpleFSLockFactory;
 import org.apache.lucene.store.SingleInstanceLockFactory;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
@@ -108,7 +104,7 @@ public class HdfsDirectoryFactory extend
   }
   
   @Override
-  protected LockFactory createLockFactory(String lockPath, String rawLockType) throws IOException {
+  protected LockFactory createLockFactory(String rawLockType) throws IOException {
     if (null == rawLockType) {
       LOG.warn("No lockType configured, assuming 'hdfs'.");
       rawLockType = "hdfs";
@@ -116,11 +112,11 @@ public class HdfsDirectoryFactory extend
     final String lockType = rawLockType.toLowerCase(Locale.ROOT).trim();
     switch (lockType) {
       case "hdfs":
-        return new HdfsLockFactory(new Path(lockPath), getConf());
+        return HdfsLockFactory.INSTANCE;
       case "single":
         return new SingleInstanceLockFactory();
       case "none":
-        return NoLockFactory.getNoLockFactory();
+        return NoLockFactory.INSTANCE;
       default:
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
             "Unrecognized lockType: " + rawLockType);

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/MMapDirectoryFactory.java Tue Nov 11 16:21:15 2014
@@ -49,7 +49,7 @@ public class MMapDirectoryFactory extend
   public void init(NamedList args) {
     super.init(args);
     SolrParams params = SolrParams.toSolrParams( args );
-    maxChunk = params.getInt("maxChunkSize", MMapDirectory.DEFAULT_MAX_BUFF);
+    maxChunk = params.getInt("maxChunkSize", MMapDirectory.DEFAULT_MAX_CHUNK_SIZE);
     if (maxChunk <= 0){
       throw new IllegalArgumentException("maxChunk must be greater than 0");
     }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/RAMDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/RAMDirectoryFactory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/RAMDirectoryFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/RAMDirectoryFactory.java Tue Nov 11 16:21:15 2014
@@ -36,7 +36,7 @@ public class RAMDirectoryFactory extends
   public static Logger LOG = LoggerFactory.getLogger(RAMDirectoryFactory.class);
   
   @Override
-  protected LockFactory createLockFactory(String lockPath, String rawLockType) throws IOException {
+  protected LockFactory createLockFactory(String rawLockType) throws IOException {
     if (!(rawLockType == null || "single".equalsIgnoreCase(rawLockType.trim()))) {
       throw new SolrException(ErrorCode.FORBIDDEN,
           "RAMDirectory can only be used with the 'single' lock factory type.");
@@ -46,9 +46,7 @@ public class RAMDirectoryFactory extends
 
   @Override
   protected Directory create(String path, LockFactory lockFactory, DirContext dirContext) throws IOException {
-    final Directory dir = new RAMDirectory();
-    dir.setLockFactory(lockFactory); // more or less a no-op, just for completeness
-    return dir;
+    return new RAMDirectory(lockFactory);
   }
 
 }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/core/StandardDirectoryFactory.java Tue Nov 11 16:21:15 2014
@@ -32,7 +32,6 @@ import org.apache.lucene.store.RateLimit
 import org.apache.lucene.store.SimpleFSLockFactory;
 import org.apache.lucene.store.SingleInstanceLockFactory;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -55,7 +54,7 @@ public class StandardDirectoryFactory ex
   }
   
   @Override
-  protected LockFactory createLockFactory(String lockPath, String rawLockType) throws IOException {
+  protected LockFactory createLockFactory(String rawLockType) throws IOException {
     if (null == rawLockType) {
       // we default to "native"
       log.warn("No lockType configured, assuming 'native'.");
@@ -64,13 +63,13 @@ public class StandardDirectoryFactory ex
     final String lockType = rawLockType.toLowerCase(Locale.ROOT).trim();
     switch (lockType) {
       case "simple":
-        return new SimpleFSLockFactory(new File(lockPath).toPath());
+        return SimpleFSLockFactory.INSTANCE;
       case "native":
-        return new NativeFSLockFactory(new File(lockPath).toPath());
+        return NativeFSLockFactory.INSTANCE;
       case "single":
         return new SingleInstanceLockFactory();
       case "none":
-        return NoLockFactory.getNoLockFactory();
+        return NoLockFactory.INSTANCE;
       default:
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
             "Unrecognized lockType: " + rawLockType);

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java Tue Nov 11 16:21:15 2014
@@ -228,6 +228,7 @@ public class ReplicationHandler extends 
           doFetch(paramsCopy, false);
         }
       };
+      puller.setDaemon(false);
       puller.start();
       if (solrParams.getBool(WAIT, false)) {
         puller.join();

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/SnapPuller.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/SnapPuller.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/SnapPuller.java Tue Nov 11 16:21:15 2014
@@ -88,7 +88,6 @@ import org.apache.solr.util.FileUtils;
 import org.apache.solr.util.PropertiesInputStream;
 import org.apache.solr.util.PropertiesOutputStream;
 import org.apache.solr.util.RefCounted;
-import org.eclipse.jetty.util.log.Log;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -509,7 +508,11 @@ public class SnapPuller {
     } finally {
       try {
         if (!successfulInstall) {
-          logReplicationTimeAndConfFiles(null, successfulInstall);
+          try {
+            logReplicationTimeAndConfFiles(null, successfulInstall);
+          } catch(Exception e) {
+            LOG.error("caught", e);
+          }
         }
         filesToDownload = filesDownloaded = confFilesDownloaded = confFilesToDownload = null;
         replicationStartTime = 0;

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/SnapShooter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/SnapShooter.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/SnapShooter.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/handler/SnapShooter.java Tue Nov 11 16:21:15 2014
@@ -31,8 +31,8 @@ import java.util.regex.Pattern;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
-import org.apache.lucene.store.Lock;
-import org.apache.lucene.store.SimpleFSLockFactory;
+import org.apache.lucene.store.NoLockFactory;
+import org.apache.lucene.store.SimpleFSDirectory;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.DirectoryFactory;
@@ -52,11 +52,9 @@ public class SnapShooter {
   private static final Logger LOG = LoggerFactory.getLogger(SnapShooter.class.getName());
   private String snapDir = null;
   private SolrCore solrCore;
-  private SimpleFSLockFactory lockFactory;
   private String snapshotName = null;
   private String directoryName = null;
   private File snapShotDir = null;
-  private Lock lock = null;
 
   public SnapShooter(SolrCore core, String location, String snapshotName) {
     solrCore = core;
@@ -67,11 +65,6 @@ public class SnapShooter {
       File dir = new File(snapDir);
       if (!dir.exists())  dir.mkdirs();
     }
-    try {
-      lockFactory = new SimpleFSLockFactory(new File(snapDir).toPath());
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
     this.snapshotName = snapshotName;
 
     if(snapshotName != null) {
@@ -122,12 +115,7 @@ public class SnapShooter {
   }
 
   void validateCreateSnapshot() throws IOException {
-    Lock lock = lockFactory.makeLock(directoryName + ".lock");
     snapShotDir = new File(snapDir, directoryName);
-    if (lock.isLocked()) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Unable to acquire lock for snapshot directory: " + snapShotDir.getAbsolutePath());
-    }
     if (snapShotDir.exists()) {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
           "Snapshot directory already exists: " + snapShotDir.getAbsolutePath());
@@ -142,15 +130,12 @@ public class SnapShooter {
     LOG.info("Creating backup snapshot...");
     NamedList<Object> details = new NamedList<>();
     details.add("startTime", new Date().toString());
-    String directoryName = null;
-
     try {
       Collection<String> files = indexCommit.getFileNames();
-      FileCopier fileCopier = new FileCopier();
 
       Directory dir = solrCore.getDirectoryFactory().get(solrCore.getIndexDir(), DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
       try {
-        fileCopier.copyFiles(dir, files, snapShotDir);
+        copyFiles(dir, files, snapShotDir);
       } finally {
         solrCore.getDirectoryFactory().release(dir);
       }
@@ -167,13 +152,6 @@ public class SnapShooter {
     } finally {
       replicationHandler.core.getDeletionPolicy().releaseCommitPoint(indexCommit.getGeneration());
       replicationHandler.snapShootDetails = details;
-      if (lock != null) {
-        try {
-          lock.close();
-        } catch (IOException e) {
-          LOG.error("Unable to release snapshoot lock: " + directoryName + ".lock");
-        }
-      }
     }
   }
 
@@ -245,35 +223,12 @@ public class SnapShooter {
   public static final String DATE_FMT = "yyyyMMddHHmmssSSS";
   
 
-  private class FileCopier {
-    
-    public void copyFiles(Directory sourceDir, Collection<String> files,
-        File destDir) throws IOException {
-      // does destinations directory exist ?
-      if (destDir != null && !destDir.exists()) {
-        destDir.mkdirs();
-      }
-      
-      FSDirectory dir = FSDirectory.open(destDir.toPath());
-      try {
-        for (String indexFile : files) {
-          copyFile(sourceDir, indexFile, new File(destDir, indexFile), dir);
-        }
-      } finally {
-        dir.close();
-      }
-    }
-    
-    public void copyFile(Directory sourceDir, String indexFile, File destination, Directory destDir)
-      throws IOException {
-
-      // make sure we can write to destination
-      if (destination.exists() && !destination.canWrite()) {
-        String message = "Unable to open file " + destination + " for writing.";
-        throw new IOException(message);
+  private static void copyFiles(Directory sourceDir, Collection<String> files, File destDir) throws IOException {
+    try (FSDirectory dir = new SimpleFSDirectory(destDir.toPath(), NoLockFactory.INSTANCE)) {
+      for (String indexFile : files) {
+        sourceDir.copy(dir, indexFile, indexFile, DirectoryFactory.IOCONTEXT_NO_CACHE);
       }
-
-      sourceDir.copy(destDir, indexFile, indexFile, DirectoryFactory.IOCONTEXT_NO_CACHE);
     }
   }
+    
 }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectory.java Tue Nov 11 16:21:15 2014
@@ -20,17 +20,15 @@ package org.apache.solr.store.blockcache
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.Collection;
 import java.util.Set;
 
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.FilterDirectory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.Lock;
-import org.apache.lucene.store.LockFactory;
 import org.apache.solr.store.hdfs.HdfsDirectory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,7 +36,7 @@ import org.slf4j.LoggerFactory;
 /**
  * @lucene.experimental
  */
-public class BlockDirectory extends Directory {
+public class BlockDirectory extends FilterDirectory {
   public static Logger LOG = LoggerFactory.getLogger(BlockDirectory.class);
   
   public static final long BLOCK_SHIFT = 13; // 2^13 = 8,192 bytes per block
@@ -86,7 +84,6 @@ public class BlockDirectory extends Dire
     public void releaseResources() {}
   };
   
-  private final Directory directory;
   private final int blockSize;
   private final String dirName;
   private final Cache cache;
@@ -97,8 +94,8 @@ public class BlockDirectory extends Dire
   public BlockDirectory(String dirName, Directory directory, Cache cache,
       Set<String> blockCacheFileTypes, boolean blockCacheReadEnabled,
       boolean blockCacheWriteEnabled) throws IOException {
+    super(directory);
     this.dirName = dirName;
-    this.directory = directory;
     blockSize = BLOCK_SIZE;
     this.cache = cache;
     if (blockCacheFileTypes == null || blockCacheFileTypes.isEmpty()) {
@@ -114,14 +111,11 @@ public class BlockDirectory extends Dire
     if (!blockCacheWriteEnabled) {
       LOG.info("Block cache on write is disabled");
     }
-    if (directory.getLockFactory() != null) {
-      setLockFactory(directory.getLockFactory());
-    }
   }
   
   private IndexInput openInput(String name, int bufferSize, IOContext context)
       throws IOException {
-    final IndexInput source = directory.openInput(name, context);
+    final IndexInput source = super.openInput(name, context);
     if (useReadCache(name, context)) {
       return new CachedIndexInput(source, blockSize, name,
           getFileCacheName(name), cache, bufferSize);
@@ -241,7 +235,7 @@ public class BlockDirectory extends Dire
     } catch (FileNotFoundException e) {
       // the local file system folder may be gone
     } finally {
-      directory.close();
+      super.close();
       cache.releaseResources();
     }
   }
@@ -251,24 +245,20 @@ public class BlockDirectory extends Dire
   }
   
   private long getFileModified(String name) throws IOException {
-    if (directory instanceof FSDirectory) {
-      File directory = ((FSDirectory) this.directory).getDirectory().toFile();
+    if (in instanceof FSDirectory) {
+      File directory = ((FSDirectory) in).getDirectory().toFile();
       File file = new File(directory, name);
       if (!file.exists()) {
         throw new FileNotFoundException("File [" + name + "] not found");
       }
       return file.lastModified();
-    } else if (directory instanceof HdfsDirectory) {
-      return ((HdfsDirectory) directory).fileModified(name);
+    } else if (in instanceof HdfsDirectory) {
+      return ((HdfsDirectory) in).fileModified(name);
     } else {
-      throw new RuntimeException("Not supported");
+      throw new UnsupportedOperationException();
     }
   }
   
-  public void clearLock(String name) throws IOException {
-    directory.clearLock(name);
-  }
-  
   String getFileCacheLocation(String name) {
     return dirName + "/" + name;
   }
@@ -282,42 +272,6 @@ public class BlockDirectory extends Dire
     return cache;
   }
   
-  @Override
-  public void copy(Directory to, String src, String dest, IOContext context)
-      throws IOException {
-    directory.copy(to, src, dest, context);
-  }
-  
-  public LockFactory getLockFactory() {
-    return directory.getLockFactory();
-  }
-  
-  public String getLockID() {
-    return directory.getLockID();
-  }
-  
-  public Lock makeLock(String name) {
-    return directory.makeLock(name);
-  }
-  
-  public void setLockFactory(LockFactory lockFactory) throws IOException {
-    directory.setLockFactory(lockFactory);
-  }
-  
-  @Override
-  public void sync(Collection<String> names) throws IOException {
-    directory.sync(names);
-  }
-  
-  // @SuppressWarnings("deprecation")
-  // public void sync(String name) throws IOException {
-  // _directory.sync(name);
-  // }
-  
-  public String toString() {
-    return directory.toString();
-  }
-  
   /**
    * Determine whether read caching should be used for a particular
    * file/context.
@@ -363,47 +317,18 @@ public class BlockDirectory extends Dire
   @Override
   public IndexOutput createOutput(String name, IOContext context)
       throws IOException {
-    IndexOutput dest = directory.createOutput(name, context);
+    final IndexOutput dest = super.createOutput(name, context);
     if (useWriteCache(name, context)) {
-      return new CachedIndexOutput(this, dest, blockSize, name, cache,
-          blockSize);
+      return new CachedIndexOutput(this, dest, blockSize, name, cache, blockSize);
     }
     return dest;
   }
   
   public void deleteFile(String name) throws IOException {
     cache.delete(getFileCacheName(name));
-    directory.deleteFile(name);
+    super.deleteFile(name);
   }
-  
-  @Override
-  public void renameFile(String source, String dest) throws IOException {
-    directory.renameFile(source, dest);
-  }
-
-  public long fileLength(String name) throws IOException {
-    return directory.fileLength(name);
-  }
-  
-  // @SuppressWarnings("deprecation")
-  // public long fileModified(String name) throws IOException {
-  // return _directory.fileModified(name);
-  // }
-  
-  public String[] listAll() throws IOException {
-    return directory.listAll();
-  }
-  
-  // @SuppressWarnings("deprecation")
-  // public void touchFile(String name) throws IOException {
-  // _directory.touchFile(name);
-  // }
-  
-  public Directory getDirectory() {
-    return directory;
-  }
-  
-  
+    
   public boolean isBlockCacheReadEnabled() {
     return blockCacheReadEnabled;
   }

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/hdfs/HdfsDirectory.java Tue Nov 11 16:21:15 2014
@@ -45,15 +45,19 @@ public class HdfsDirectory extends BaseD
   public static final int BUFFER_SIZE = 8192;
   
   private static final String LF_EXT = ".lf";
-  protected Path hdfsDirPath;
-  protected Configuration configuration;
+  protected final Path hdfsDirPath;
+  protected final Configuration configuration;
   
   private final FileSystem fileSystem;
   private final FileContext fileContext;
   
+  public HdfsDirectory(Path hdfsDirPath, Configuration configuration) throws IOException {
+    this(hdfsDirPath, HdfsLockFactory.INSTANCE, configuration);
+  }
+
   public HdfsDirectory(Path hdfsDirPath, LockFactory lockFactory, Configuration configuration)
       throws IOException {
-    setLockFactory(lockFactory);
+    super(lockFactory);
     this.hdfsDirPath = hdfsDirPath;
     this.configuration = configuration;
     fileSystem = FileSystem.newInstance(hdfsDirPath.toUri(), configuration);

Modified: lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/hdfs/HdfsLockFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/hdfs/HdfsLockFactory.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/hdfs/HdfsLockFactory.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/java/org/apache/solr/store/hdfs/HdfsLockFactory.java Tue Nov 11 16:21:15 2014
@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.FileAlreadyE
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockFactory;
 import org.apache.lucene.store.LockReleaseFailedException;
@@ -35,76 +36,24 @@ import org.slf4j.LoggerFactory;
 public class HdfsLockFactory extends LockFactory {
   public static Logger log = LoggerFactory.getLogger(HdfsLockFactory.class);
   
-  private Path lockPath;
-  private Configuration configuration;
+  public static final HdfsLockFactory INSTANCE = new HdfsLockFactory();
   
-  public HdfsLockFactory(Path lockPath, Configuration configuration) {
-    this.lockPath = lockPath;
-    this.configuration = configuration;
-  }
+  private HdfsLockFactory() {}
   
   @Override
-  public Lock makeLock(String lockName) {
-    
-    if (lockPrefix != null) {
-      lockName = lockPrefix + "-" + lockName;
+  public Lock makeLock(Directory dir, String lockName) {
+    if (!(dir instanceof HdfsDirectory)) {
+      throw new UnsupportedOperationException("HdfsLockFactory can only be used with HdfsDirectory subclasses, got: " + dir);
     }
-    
-    HdfsLock lock = new HdfsLock(lockPath, lockName, configuration);
-    
-    return lock;
-  }
-  
-  @Override
-  public void clearLock(String lockName) throws IOException {
-    FileSystem fs = null;
-    try {
-      fs = FileSystem.newInstance(lockPath.toUri(), configuration);
-      while (true) {
-        if (fs.exists(lockPath)) {
-          if (lockPrefix != null) {
-            lockName = lockPrefix + "-" + lockName;
-          }
-          
-          Path lockFile = new Path(lockPath, lockName);
-          try {
-            if (fs.exists(lockFile) && !fs.delete(lockFile, false)) {
-              throw new IOException("Cannot delete " + lockFile);
-            }
-          } catch (RemoteException e) {
-            if (e.getClassName().equals(
-                "org.apache.hadoop.hdfs.server.namenode.SafeModeException")) {
-              log.warn("The NameNode is in SafeMode - Solr will wait 5 seconds and try again.");
-              try {
-                Thread.sleep(5000);
-              } catch (InterruptedException e1) {
-                Thread.interrupted();
-              }
-              continue;
-            }
-            throw e;
-          }
-          break;
-        }
-      }
-    } finally {
-      IOUtils.closeQuietly(fs);
-    }
-  }
-  
-  public Path getLockPath() {
-    return lockPath;
-  }
-  
-  public void setLockPath(Path lockPath) {
-    this.lockPath = lockPath;
+    final HdfsDirectory hdfsDir = (HdfsDirectory) dir;
+    return new HdfsLock(hdfsDir.getHdfsDirPath(), lockName, hdfsDir.getConfiguration());
   }
   
   static class HdfsLock extends Lock {
     
-    private Path lockPath;
-    private String lockName;
-    private Configuration conf;
+    private final Path lockPath;
+    private final String lockName;
+    private final Configuration conf;
     
     public HdfsLock(Path lockPath, String lockName, Configuration conf) {
       this.lockPath = lockPath;

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java Tue Nov 11 16:21:15 2014
@@ -48,7 +48,7 @@ public class SolrCoreCheckLockOnStartupT
   @Test
   public void testSimpleLockErrorOnStartup() throws Exception {
 
-    Directory directory = newFSDirectory(new File(initCoreDataDir, "index").toPath(), new SimpleFSLockFactory());
+    Directory directory = newFSDirectory(new File(initCoreDataDir, "index").toPath(), SimpleFSLockFactory.INSTANCE);
     //creates a new IndexWriter without releasing the lock yet
     IndexWriter indexWriter = new IndexWriter(directory, new IndexWriterConfig(null));
 
@@ -74,7 +74,7 @@ public class SolrCoreCheckLockOnStartupT
 
     File indexDir = new File(initCoreDataDir, "index");
     log.info("Acquiring lock on {}", indexDir.getAbsolutePath());
-    Directory directory = newFSDirectory(indexDir.toPath(), new NativeFSLockFactory());
+    Directory directory = newFSDirectory(indexDir.toPath(), NativeFSLockFactory.INSTANCE);
     //creates a new IndexWriter without releasing the lock yet
     IndexWriter indexWriter = new IndexWriter(directory, new IndexWriterConfig(null));
 

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/store/hdfs/HdfsDirectoryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/store/hdfs/HdfsDirectoryTest.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/store/hdfs/HdfsDirectoryTest.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/store/hdfs/HdfsDirectoryTest.java Tue Nov 11 16:21:15 2014
@@ -74,7 +74,7 @@ public class HdfsDirectoryTest extends S
     Configuration conf = new Configuration();
     conf.set("dfs.permissions.enabled", "false");
     
-    directory = new HdfsDirectory(new Path(dfsCluster.getURI().toString() + createTempDir().toFile().getAbsolutePath() + "/hdfs"), NoLockFactory.getNoLockFactory(), conf);
+    directory = new HdfsDirectory(new Path(dfsCluster.getURI().toString() + createTempDir().toFile().getAbsolutePath() + "/hdfs"), NoLockFactory.INSTANCE, conf);
     
     random = random();
   }

Modified: lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/store/hdfs/HdfsLockFactoryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/store/hdfs/HdfsLockFactoryTest.java?rev=1638204&r1=1638203&r2=1638204&view=diff
==============================================================================
--- lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/store/hdfs/HdfsLockFactoryTest.java (original)
+++ lucene/dev/branches/lucene6005/solr/core/src/test/org/apache/solr/store/hdfs/HdfsLockFactoryTest.java Tue Nov 11 16:21:15 2014
@@ -65,8 +65,9 @@ public class HdfsLockFactoryTest extends
   public void testBasic() throws IOException {
     URI uri = dfsCluster.getURI();
     Path lockPath = new Path(uri.toString(), "/basedir/lock");
-    HdfsLockFactory lockFactory = new HdfsLockFactory(lockPath, new Configuration());
-    Lock lock = lockFactory.makeLock("testlock");
+    Configuration conf = new Configuration();
+    HdfsDirectory dir = new HdfsDirectory(lockPath, conf);
+    Lock lock = dir.makeLock("testlock");
     boolean success = lock.obtain();
     assertTrue("We could not get the lock when it should be available", success);
     success = lock.obtain();
@@ -76,6 +77,7 @@ public class HdfsLockFactoryTest extends
     assertTrue("We could not get the lock when it should be available", success);
     success = lock.obtain();
     assertFalse("We got the lock but it should be unavailble", success);
+    dir.close();
   }