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

svn commit: r1619427 - in /lucene/dev/trunk/solr: ./ contrib/map-reduce/src/test/org/apache/solr/hadoop/ contrib/morphlines-core/src/test-files/solr/minimr/conf/ contrib/morphlines-core/src/test-files/solr/solrcloud/conf/ core/src/java/org/apache/solr/...

Author: markrmiller
Date: Thu Aug 21 15:03:29 2014
New Revision: 1619427

URL: http://svn.apache.org/r1619427
Log:
SOLR-6089: When using the HDFS block cache, when a file is deleted, it's underlying data entries in the block cache are not removed, which is a problem with the global block cache option. 

Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java
    lucene/dev/trunk/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/solrconfig.xml
    lucene/dev/trunk/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectory.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectoryCache.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/Cache.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsTestUtil.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/store/blockcache/BlockDirectoryTest.java
    lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Thu Aug 21 15:03:29 2014
@@ -302,6 +302,10 @@ Bug Fixes
 * SOLR-6268: HdfsUpdateLog has a race condition that can expose a closed HDFS FileSystem instance and should 
   close it's FileSystem instance if either inherited close method is called. (Mark Miller)
 
+* SOLR-6089: When using the HDFS block cache, when a file is deleted, it's underlying data entries in the 
+  block cache are not removed, which is a problem with the global block cache option. 
+  (Mark Miller, Patrick Hunt)
+
 Optimizations
 ---------------------
 

Modified: lucene/dev/trunk/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java (original)
+++ lucene/dev/trunk/solr/contrib/map-reduce/src/test/org/apache/solr/hadoop/MorphlineGoLiveMiniMRTest.java Thu Aug 21 15:03:29 2014
@@ -44,6 +44,7 @@ import org.apache.hadoop.util.JarFinder;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
@@ -65,6 +66,7 @@ import org.apache.solr.common.cloud.Slic
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.hadoop.hack.MiniMRClientCluster;
@@ -128,6 +130,10 @@ public class MorphlineGoLiveMiniMRTest e
   
   @BeforeClass
   public static void setupClass() throws Exception {
+    System.setProperty("solr.hdfs.blockcache.global", Boolean.toString(LuceneTestCase.random().nextBoolean()));
+    System.setProperty("solr.hdfs.blockcache.enabled", Boolean.toString(LuceneTestCase.random().nextBoolean()));
+    System.setProperty("solr.hdfs.blockcache.blocksperbank", "2048");
+    
     solrHomeDirectory = createTempDir();
     assumeTrue(
             "Currently this test can only be run without the lucene test security policy in place",
@@ -156,8 +162,6 @@ public class MorphlineGoLiveMiniMRTest e
     int taskTrackers = 2;
     int dataNodes = 2;
     
-    System.setProperty("solr.hdfs.blockcache.enabled", "false");
-    
     JobConf conf = new JobConf();
     conf.set("dfs.block.access.token.enable", "false");
     conf.set("dfs.permissions", "true");
@@ -218,6 +222,8 @@ public class MorphlineGoLiveMiniMRTest e
   
   @AfterClass
   public static void teardownClass() throws Exception {
+    System.clearProperty("solr.hdfs.blockcache.global");
+    System.clearProperty("solr.hdfs.blockcache.blocksperbank");
     System.clearProperty("solr.hdfs.blockcache.enabled");
     System.clearProperty("hadoop.log.dir");
     System.clearProperty("test.build.dir");
@@ -627,8 +633,65 @@ public class MorphlineGoLiveMiniMRTest e
       checkConsistency(replicatedCollection);
       
       assertEquals(RECORD_COUNT, executeSolrQuery(cloudClient, "*:*").size());
-    }  
+    }
+    
+    // delete collection
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.DELETE.toString());
+    params.set(CoreAdminParams.DELETE_INSTANCE_DIR, true);
+    params.set(CoreAdminParams.DELETE_DATA_DIR, true);
+    params.set(CoreAdminParams.DELETE_INDEX, true);
+    params.set("name", replicatedCollection);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+    cloudClient.request(request);
+
+    
+    long timeout = System.currentTimeMillis() + 10000;
+    while (cloudClient.getZkStateReader().getClusterState().hasCollection(replicatedCollection)) {
+      if (System.currentTimeMillis() > timeout) {
+        throw new AssertionError("Timeout waiting to see removed collection leave clusterstate");
+      }
+      
+      Thread.sleep(200);
+      cloudClient.getZkStateReader().updateClusterState(true);
+    }
+    
+    if (TEST_NIGHTLY) {
+      createCollection(replicatedCollection, 11, 3, 11);
+    } else {
+      createCollection(replicatedCollection, 2, 3, 2);
+    }
+    
+    waitForRecoveriesToFinish(replicatedCollection, false);
+    printLayout();
+    assertEquals(0, executeSolrQuery(cloudClient, "*:*").getNumFound());
+    
+    
+    args = new String[] {
+        "--solr-home-dir=" + MINIMR_CONF_DIR.getAbsolutePath(),
+        "--output-dir=" + outDir.toString(),
+        "--shards", "2",
+        "--mappers=3",
+        "--verbose",
+        "--go-live", 
+        "--go-live-threads", Integer.toString(random().nextInt(15) + 1),  dataDir.toString()
+    };
+    args = prependInitialArgs(args);
+
+    argList = new ArrayList<>();
+    getShardUrlArgs(argList, replicatedCollection);
+    args = concat(args, argList.toArray(new String[0]));
+    
+    tool = new MapReduceIndexerTool();
+    res = ToolRunner.run(jobConf, tool, args);
+    assertEquals(0, res);
+    assertTrue(tool.job.isComplete());
+    assertTrue(tool.job.isSuccessful());
+    
+    checkConsistency(replicatedCollection);
     
+    assertEquals(RECORD_COUNT, executeSolrQuery(cloudClient, "*:*").size());
   }
 
   private void getShardUrlArgs(List<String> args) {

Modified: lucene/dev/trunk/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/solrconfig.xml?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/solrconfig.xml (original)
+++ lucene/dev/trunk/solr/contrib/morphlines-core/src/test-files/solr/minimr/conf/solrconfig.xml Thu Aug 21 15:03:29 2014
@@ -130,6 +130,7 @@
     <bool name="solr.hdfs.nrtcachingdirectory.enable">${solr.hdfs.nrtcachingdirectory.enable:true}</bool>
     <int name="solr.hdfs.nrtcachingdirectory.maxmergesizemb">${solr.hdfs.nrtcachingdirectory.maxmergesizemb:16}</int>
     <int name="solr.hdfs.nrtcachingdirectory.maxcachedmb">${solr.hdfs.nrtcachingdirectory.maxcachedmb:192}</int>
+    <str name="solr.hdfs.blockcache.global">${solr.hdfs.blockcache.global:false}</str>
   </directoryFactory> 
 
   <!-- The CodecFactory for defining the format of the inverted index.

Modified: lucene/dev/trunk/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml (original)
+++ lucene/dev/trunk/solr/contrib/morphlines-core/src/test-files/solr/solrcloud/conf/solrconfig.xml Thu Aug 21 15:03:29 2014
@@ -133,6 +133,7 @@
     <bool name="solr.hdfs.nrtcachingdirectory.enable">${solr.hdfs.nrtcachingdirectory.enable:true}</bool>
     <int name="solr.hdfs.nrtcachingdirectory.maxmergesizemb">${solr.hdfs.nrtcachingdirectory.maxmergesizemb:16}</int>
     <int name="solr.hdfs.nrtcachingdirectory.maxcachedmb">${solr.hdfs.nrtcachingdirectory.maxcachedmb:192}</int>
+    <str name="solr.hdfs.blockcache.global">${solr.hdfs.blockcache.global:false}</str>
   </directoryFactory> 
 
   <!-- The CodecFactory for defining the format of the inverted index.

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/HdfsDirectoryFactory.java Thu Aug 21 15:03:29 2014
@@ -139,11 +139,11 @@ public class HdfsDirectoryFactory extend
       int bufferSize = params.getInt("solr.hdfs.blockcache.bufferstore.buffersize", 128);
       int bufferCount = params.getInt("solr.hdfs.blockcache.bufferstore.buffercount", 128 * 128);
       
-      BlockCache blockCache = getBlockDirectoryCache(path, numberOfBlocksPerBank,
+      BlockCache blockCache = getBlockDirectoryCache(numberOfBlocksPerBank,
           blockSize, bankCount, directAllocation, slabSize,
           bufferSize, bufferCount, blockCacheGlobal);
       
-      Cache cache = new BlockDirectoryCache(blockCache, path, metrics);
+      Cache cache = new BlockDirectoryCache(blockCache, path, metrics, blockCacheGlobal);
       HdfsDirectory hdfsDirectory = new HdfsDirectory(new Path(path), conf);
       dir = new BlockDirectory(path, hdfsDirectory, cache, null,
           blockCacheReadEnabled, blockCacheWriteEnabled);
@@ -164,17 +164,16 @@ public class HdfsDirectoryFactory extend
     return dir;
   }
 
-  private BlockCache getBlockDirectoryCache(String path,
-      int numberOfBlocksPerBank, int blockSize, int bankCount,
+  private BlockCache getBlockDirectoryCache(int numberOfBlocksPerBank, int blockSize, int bankCount,
       boolean directAllocation, int slabSize, int bufferSize, int bufferCount, boolean staticBlockCache) {
     if (!staticBlockCache) {
       LOG.info("Creating new single instance HDFS BlockCache");
       return createBlockCache(numberOfBlocksPerBank, blockSize, bankCount, directAllocation, slabSize, bufferSize, bufferCount);
     }
-    LOG.info("Creating new global HDFS BlockCache");
     synchronized (HdfsDirectoryFactory.class) {
       
       if (globalBlockCache == null) {
+        LOG.info("Creating new global HDFS BlockCache");
         globalBlockCache = createBlockCache(numberOfBlocksPerBank, blockSize, bankCount,
             directAllocation, slabSize, bufferSize, bufferCount);
       }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java Thu Aug 21 15:03:29 2014
@@ -80,6 +80,10 @@ public class BlockCache {
     this.blockSize = blockSize;
   }
   
+  public void release(BlockCacheKey key) {
+    releaseLocation(cache.get(key));
+  }
+  
   private void releaseLocation(BlockCacheLocation location) {
     if (location == null) {
       return;

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectory.java?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectory.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectory.java Thu Aug 21 15:03:29 2014
@@ -80,19 +80,27 @@ public class BlockDirectory extends Dire
     
     @Override
     public void renameCacheFile(String source, String dest) {}
+
+    @Override
+    public void releaseResources() {}
   };
   
-  private Directory directory;
-  private int blockSize;
-  private String dirName;
+  private final Directory directory;
+  private final int blockSize;
+  private final String dirName;
   private final Cache cache;
-  private Set<String> blockCacheFileTypes;
+  private final Set<String> blockCacheFileTypes;
   private final boolean blockCacheReadEnabled;
   private final boolean blockCacheWriteEnabled;
 
   public BlockDirectory(String dirName, Directory directory, Cache cache,
       Set<String> blockCacheFileTypes, boolean blockCacheReadEnabled,
       boolean blockCacheWriteEnabled) throws IOException {
+    this(dirName, directory, cache, blockCacheFileTypes, blockCacheReadEnabled, blockCacheWriteEnabled, false);
+  }
+  public BlockDirectory(String dirName, Directory directory, Cache cache,
+      Set<String> blockCacheFileTypes, boolean blockCacheReadEnabled,
+      boolean blockCacheWriteEnabled, boolean releaseBlocksOnClose) throws IOException {
     this.dirName = dirName;
     this.directory = directory;
     blockSize = BLOCK_SIZE;
@@ -233,7 +241,10 @@ public class BlockDirectory extends Dire
       for (String file : files) {
         cache.delete(getFileCacheName(file));
       }
+      // segments.gen won't be removed above
+      cache.delete(dirName + "/" + "segments.gen");
       
+      cache.releaseResources();
     } catch (FileNotFoundException e) {
       // the local file system folder may be gone
     } finally {

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectoryCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectoryCache.java?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectoryCache.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectoryCache.java Thu Aug 21 15:03:29 2014
@@ -17,7 +17,10 @@ package org.apache.solr.store.blockcache
  * limitations under the License.
  */
 
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -26,15 +29,23 @@ import java.util.concurrent.atomic.Atomi
  */
 public class BlockDirectoryCache implements Cache {
   private final BlockCache blockCache;
-  private AtomicInteger counter = new AtomicInteger();
-  private Map<String,Integer> names = new ConcurrentHashMap<>();
-  private String path;
-  private Metrics metrics;
+  private final AtomicInteger counter = new AtomicInteger();
+  private final Map<String,Integer> names = new ConcurrentHashMap<>();
+  private Set<BlockCacheKey> keys;
+  private final String path;
+  private final Metrics metrics;
   
   public BlockDirectoryCache(BlockCache blockCache, String path, Metrics metrics) {
+    this(blockCache, path, metrics, false);
+  }
+  
+  public BlockDirectoryCache(BlockCache blockCache, String path, Metrics metrics, boolean releaseBlocks) {
     this.blockCache = blockCache;
     this.path = path;
     this.metrics = metrics;
+    if (releaseBlocks) {
+      keys = Collections.synchronizedSet(new HashSet<BlockCacheKey>());
+    }
   }
   
   /**
@@ -64,6 +75,9 @@ public class BlockDirectoryCache impleme
     blockCacheKey.setBlock(blockId);
     blockCacheKey.setFile(file);
     blockCache.store(blockCacheKey, blockOffset, buffer, offset, length);
+    if (keys != null) {
+      keys.add(blockCacheKey);
+    }
   }
   
   @Override
@@ -100,4 +114,13 @@ public class BlockDirectoryCache impleme
       names.put(dest, file);
     }
   }
+
+  @Override
+  public void releaseResources() {
+    if (keys != null) {
+      for (BlockCacheKey key : keys) {
+        blockCache.release(key);
+      }
+    }
+  }
 }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/Cache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/Cache.java?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/Cache.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/store/blockcache/Cache.java Thu Aug 21 15:03:29 2014
@@ -61,5 +61,10 @@ public interface Cache {
    *          final name
    */
   void renameCacheFile(String source, String dest);
+
+  /**
+   * Release any resources associated with the cache.
+   */
+  void releaseResources();
   
 }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeySafeLeaderTest.java Thu Aug 21 15:03:29 2014
@@ -172,14 +172,6 @@ public class ChaosMonkeySafeLeaderTest e
     numShardsNumReplicas.add(1);
     checkForCollection("testcollection",numShardsNumReplicas, null);
   }
-
-  private void randomlyEnableAutoSoftCommit() {
-    if (r.nextBoolean()) {
-      enableAutoSoftCommit(1000);
-    } else {
-      log.info("Not turning on auto soft commit");
-    }
-  }
   
   // skip the randoms - they can deadlock...
   @Override

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsTestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsTestUtil.java?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsTestUtil.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/HdfsTestUtil.java Thu Aug 21 15:03:29 2014
@@ -13,7 +13,6 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
 import org.apache.solr.SolrTestCaseJ4;
 
 /*

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/hdfs/StressHdfsTest.java Thu Aug 21 15:03:29 2014
@@ -90,6 +90,8 @@ public class StressHdfsTest extends Basi
   
   @Override
   public void doTest() throws Exception {
+    randomlyEnableAutoSoftCommit();
+    
     int cnt = random().nextInt(2) + 1;
     for (int i = 0; i < cnt; i++) {
       createAndDeleteCollection();
@@ -161,19 +163,19 @@ public class StressHdfsTest extends Basi
     
     int i = 0;
     for (SolrServer client : clients) {
-      HttpSolrServer c = new HttpSolrServer(getBaseUrl(client)
-          + "/delete_data_dir");
+      HttpSolrServer c = new HttpSolrServer(getBaseUrl(client) + "/" + DELETE_DATA_DIR_COLLECTION);
       try {
-        c.add(getDoc("id", i++));
-        if (random().nextBoolean()) c.add(getDoc("id", i++));
-        if (random().nextBoolean()) c.add(getDoc("id", i++));
+        int docCnt = random().nextInt(1000) + 1;
+        for (int j = 0; j < docCnt; j++) {
+          c.add(getDoc("id", i++, "txt_t", "just some random text for a doc"));
+        }
+
         if (random().nextBoolean()) {
           c.commit();
         } else {
           c.commit(true, true, true);
         }
         
-        c.query(new SolrQuery("id:" + i));
         c.setConnectionTimeout(30000);
         NamedList<Object> response = c.query(
             new SolrQuery().setRequestHandler("/admin/system")).getResponse();
@@ -192,6 +194,10 @@ public class StressHdfsTest extends Basi
       assertEquals(0, cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound());
     }
     
+    cloudClient.commit();
+    cloudClient.query(new SolrQuery("*:*"));
+    
+    // delete collection
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set("action", CollectionAction.DELETE.toString());
     params.set("name", DELETE_DATA_DIR_COLLECTION);

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/store/blockcache/BlockDirectoryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/store/blockcache/BlockDirectoryTest.java?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/store/blockcache/BlockDirectoryTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/store/blockcache/BlockDirectoryTest.java Thu Aug 21 15:03:29 2014
@@ -22,17 +22,13 @@ import java.io.IOException;
 import java.util.Map;
 import java.util.Random;
 
-import org.apache.commons.io.FileUtils;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.MergeInfo;
-import org.apache.lucene.util.LuceneTestCase;
-
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.store.hdfs.HdfsDirectory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -89,6 +85,9 @@ public class BlockDirectoryTest extends 
     @Override
     public void renameCacheFile(String source, String dest) {
     }
+
+    @Override
+    public void releaseResources() {}
   }
 
   private static final int MAX_NUMBER_OF_WRITES = 10000;

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java?rev=1619427&r1=1619426&r2=1619427&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java Thu Aug 21 15:03:29 2014
@@ -1092,6 +1092,14 @@ public abstract class AbstractFullDistri
     }
   }
   
+  protected void randomlyEnableAutoSoftCommit() {
+    if (r.nextBoolean()) {
+      enableAutoSoftCommit(1000);
+    } else {
+      log.info("Not turning on auto soft commit");
+    }
+  }
+  
   protected void enableAutoSoftCommit(int time) {
     log.info("Turning on auto soft commit: " + time);
     for (List<CloudJettyRunner> jettyList : shardToJetty.values()) {