You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2017/03/15 18:15:45 UTC

[3/9] geode git commit: GEODE-2643: Combine chunk and file region into a single region

GEODE-2643: Combine chunk and file region into a single region

* removed file and chunk count from stat
* removed tests that were doing checks against chunk bucketRegions


Project: http://git-wip-us.apache.org/repos/asf/geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/geode/commit/d8a89730
Tree: http://git-wip-us.apache.org/repos/asf/geode/tree/d8a89730
Diff: http://git-wip-us.apache.org/repos/asf/geode/diff/d8a89730

Branch: refs/heads/feature/GEODE-2648
Commit: d8a897305c1fe9954f2511a6af0c8a633ef152d0
Parents: 3c212fb
Author: Jason Huynh <hu...@gmail.com>
Authored: Mon Mar 13 16:35:56 2017 -0700
Committer: Jason Huynh <hu...@gmail.com>
Committed: Tue Mar 14 11:47:54 2017 -0700

----------------------------------------------------------------------
 .../lucene/internal/IndexRepositoryFactory.java |  28 +++--
 .../LuceneIndexForPartitionedRegion.java        |  76 ++++---------
 .../internal/directory/RegionDirectory.java     |   5 +-
 .../lucene/internal/filesystem/FileSystem.java  |  70 +++++-------
 .../internal/filesystem/FileSystemStats.java    |  20 ----
 .../lucene/LuceneIndexDestroyDUnitTest.java     |   5 -
 .../LuceneIndexMaintenanceIntegrationTest.java  |   2 -
 ...LuceneQueriesPersistenceIntegrationTest.java |   2 -
 .../LuceneIndexForPartitionedRegionTest.java    | 102 ++---------------
 .../LuceneIndexRecoveryHAIntegrationTest.java   |   6 +-
 .../PartitionedRepositoryManagerJUnitTest.java  |  56 ++++------
 .../directory/RegionDirectoryJUnitTest.java     |   3 +-
 .../DistributedScoringJUnitTest.java            |   5 +-
 .../filesystem/FileSystemJUnitTest.java         | 112 ++++---------------
 .../IndexRepositoryImplJUnitTest.java           |   5 +-
 .../IndexRepositoryImplPerformanceTest.java     |  13 +--
 .../cache/lucene/test/LuceneTestUtilities.java  |   3 -
 17 files changed, 131 insertions(+), 382 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/IndexRepositoryFactory.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/IndexRepositoryFactory.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/IndexRepositoryFactory.java
index 7e685b7..475311d 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/IndexRepositoryFactory.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/IndexRepositoryFactory.java
@@ -43,20 +43,18 @@ public class IndexRepositoryFactory {
       LuceneIndexImpl index, PartitionedRegion userRegion, final IndexRepository oldRepository)
       throws IOException {
     LuceneIndexForPartitionedRegion indexForPR = (LuceneIndexForPartitionedRegion) index;
-    final PartitionedRegion fileRegion = indexForPR.getFileRegion();
-    final PartitionedRegion chunkRegion = indexForPR.getChunkRegion();
+    final PartitionedRegion fileRegion = indexForPR.getFileAndChunkRegion();
 
-    BucketRegion fileBucket = getMatchingBucket(fileRegion, bucketId);
-    BucketRegion chunkBucket = getMatchingBucket(chunkRegion, bucketId);
+    BucketRegion fileAndChunkBucket = getMatchingBucket(fileRegion, bucketId);
     BucketRegion dataBucket = getMatchingBucket(userRegion, bucketId);
     boolean success = false;
-    if (fileBucket == null || chunkBucket == null) {
+    if (fileAndChunkBucket == null) {
       if (oldRepository != null) {
         oldRepository.cleanup();
       }
       return null;
     }
-    if (!chunkBucket.getBucketAdvisor().isPrimary()) {
+    if (!fileAndChunkBucket.getBucketAdvisor().isPrimary()) {
       if (oldRepository != null) {
         oldRepository.cleanup();
       }
@@ -71,26 +69,26 @@ public class IndexRepositoryFactory {
       oldRepository.cleanup();
     }
     DistributedLockService lockService = getLockService();
-    String lockName = getLockName(bucketId, fileBucket);
+    String lockName = getLockName(fileAndChunkBucket);
     while (!lockService.lock(lockName, 100, -1)) {
-      if (!chunkBucket.getBucketAdvisor().isPrimary()) {
+      if (!fileAndChunkBucket.getBucketAdvisor().isPrimary()) {
         return null;
       }
     }
 
     final IndexRepository repo;
     try {
-      RegionDirectory dir = new RegionDirectory(getBucketTargetingMap(fileBucket, bucketId),
-          getBucketTargetingMap(chunkBucket, bucketId), indexForPR.getFileSystemStats());
+      RegionDirectory dir = new RegionDirectory(getBucketTargetingMap(fileAndChunkBucket, bucketId),
+          indexForPR.getFileSystemStats());
       IndexWriterConfig config = new IndexWriterConfig(indexForPR.getAnalyzer());
       IndexWriter writer = new IndexWriter(dir, config);
-      repo = new IndexRepositoryImpl(fileBucket, writer, serializer, indexForPR.getIndexStats(),
-          dataBucket, lockService, lockName);
+      repo = new IndexRepositoryImpl(fileAndChunkBucket, writer, serializer,
+          indexForPR.getIndexStats(), dataBucket, lockService, lockName);
       success = true;
       return repo;
     } catch (IOException e) {
       logger.info("Exception thrown while constructing Lucene Index for bucket:" + bucketId
-          + " for file region:" + fileBucket.getFullPath());
+          + " for file region:" + fileAndChunkBucket.getFullPath());
       throw e;
     } finally {
       if (!success) {
@@ -104,8 +102,8 @@ public class IndexRepositoryFactory {
     return new BucketTargetingMap(region, bucketId);
   }
 
-  private String getLockName(final Integer bucketId, final BucketRegion fileBucket) {
-    return FILE_REGION_LOCK_FOR_BUCKET_ID + fileBucket.getFullPath();
+  private String getLockName(final BucketRegion fileAndChunkBucket) {
+    return FILE_REGION_LOCK_FOR_BUCKET_ID + fileAndChunkBucket.getFullPath();
   }
 
   private DistributedLockService getLockService() {

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegion.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
index 7274d6a..80e0c44 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegion.java
@@ -29,7 +29,6 @@ import org.apache.geode.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.cache.lucene.internal.directory.DumpDirectoryFiles;
-import org.apache.geode.cache.lucene.internal.filesystem.ChunkKey;
 import org.apache.geode.cache.lucene.internal.filesystem.File;
 import org.apache.geode.cache.lucene.internal.filesystem.FileSystemStats;
 import org.apache.geode.cache.lucene.internal.partition.BucketTargetingFixedResolver;
@@ -48,12 +47,10 @@ import java.util.Set;
 
 /* wrapper of IndexWriter */
 public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
-  protected Region<String, File> fileRegion;
-  protected Region<ChunkKey, byte[]> chunkRegion;
+  protected Region fileAndChunkRegion;
   protected final FileSystemStats fileSystemStats;
 
   public static final String FILES_REGION_SUFFIX = ".files";
-  public static final String CHUNKS_REGION_SUFFIX = ".chunks";
 
   public LuceneIndexForPartitionedRegion(String indexName, String regionPath, Cache cache) {
     super(indexName, regionPath, cache);
@@ -81,16 +78,12 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
       regionShortCut = RegionShortcut.PARTITION;
     }
 
-    // create PR fileRegion, but not to create its buckets for now
+    // create PR fileAndChunkRegion, but not to create its buckets for now
     final String fileRegionName = createFileRegionName();
     PartitionAttributes partitionAttributes = dataRegion.getPartitionAttributes();
-    if (!fileRegionExists(fileRegionName)) {
-      fileRegion =
-          createFileRegion(regionShortCut, fileRegionName, partitionAttributes, regionAttributes);
-    }
+
 
     // create PR chunkRegion, but not to create its buckets for now
-    final String chunkRegionName = createChunkRegionName();
 
     // we will create RegionDirectories on the fly when data comes in
     HeterogeneousLuceneSerializer mapper = new HeterogeneousLuceneSerializer(getFieldNames());
@@ -99,23 +92,20 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     DM dm = ((GemFireCacheImpl) getCache()).getDistributedSystem().getDistributionManager();
     LuceneBucketListener lucenePrimaryBucketListener =
         new LuceneBucketListener(partitionedRepositoryManager, dm);
-    if (!chunkRegionExists(chunkRegionName)) {
-      chunkRegion = createChunkRegion(regionShortCut, fileRegionName, partitionAttributes,
-          chunkRegionName, regionAttributes, lucenePrimaryBucketListener);
+
+    if (!fileRegionExists(fileRegionName)) {
+      fileAndChunkRegion = createFileRegion(regionShortCut, fileRegionName, partitionAttributes,
+          regionAttributes, lucenePrimaryBucketListener);
     }
-    fileSystemStats.setFileSupplier(() -> (int) getFileRegion().getLocalSize());
-    fileSystemStats.setChunkSupplier(() -> (int) getChunkRegion().getLocalSize());
-    fileSystemStats.setBytesSupplier(() -> getChunkRegion().getPrStats().getDataStoreBytesInUse());
 
-    return partitionedRepositoryManager;
-  }
+    fileSystemStats
+        .setBytesSupplier(() -> getFileAndChunkRegion().getPrStats().getDataStoreBytesInUse());
 
-  public PartitionedRegion getFileRegion() {
-    return (PartitionedRegion) fileRegion;
+    return partitionedRepositoryManager;
   }
 
-  public PartitionedRegion getChunkRegion() {
-    return (PartitionedRegion) chunkRegion;
+  public PartitionedRegion getFileAndChunkRegion() {
+    return (PartitionedRegion) fileAndChunkRegion;
   }
 
   public FileSystemStats getFileSystemStats() {
@@ -123,35 +113,20 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
   }
 
   boolean fileRegionExists(String fileRegionName) {
-    return cache.<String, File>getRegion(fileRegionName) != null;
+    return cache.getRegion(fileRegionName) != null;
   }
 
   Region createFileRegion(final RegionShortcut regionShortCut, final String fileRegionName,
-      final PartitionAttributes partitionAttributes, final RegionAttributes regionAttributes) {
+      final PartitionAttributes partitionAttributes, final RegionAttributes regionAttributes,
+      PartitionListener listener) {
     return createRegion(fileRegionName, regionShortCut, this.regionPath, partitionAttributes,
-        regionAttributes, null);
+        regionAttributes, listener);
   }
 
   public String createFileRegionName() {
     return LuceneServiceImpl.getUniqueIndexRegionName(indexName, regionPath, FILES_REGION_SUFFIX);
   }
 
-  boolean chunkRegionExists(String chunkRegionName) {
-    return cache.<ChunkKey, byte[]>getRegion(chunkRegionName) != null;
-  }
-
-  Region<ChunkKey, byte[]> createChunkRegion(final RegionShortcut regionShortCut,
-      final String fileRegionName, final PartitionAttributes partitionAttributes,
-      final String chunkRegionName, final RegionAttributes regionAttributes,
-      final PartitionListener lucenePrimaryBucketListener) {
-    return createRegion(chunkRegionName, regionShortCut, fileRegionName, partitionAttributes,
-        regionAttributes, lucenePrimaryBucketListener);
-  }
-
-  public String createChunkRegionName() {
-    return LuceneServiceImpl.getUniqueIndexRegionName(indexName, regionPath, CHUNKS_REGION_SUFFIX);
-  }
-
   private PartitionAttributesFactory configureLuceneRegionAttributesFactory(
       PartitionAttributesFactory attributesFactory,
       PartitionAttributes<?, ?> dataRegionAttributes) {
@@ -175,8 +150,7 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
       final RegionShortcut regionShortCut, final String colocatedWithRegionName,
       final PartitionAttributes partitionAttributes, final RegionAttributes regionAttributes,
       PartitionListener lucenePrimaryBucketListener) {
-    PartitionAttributesFactory partitionAttributesFactory =
-        new PartitionAttributesFactory<String, File>();
+    PartitionAttributesFactory partitionAttributesFactory = new PartitionAttributesFactory();
     if (lucenePrimaryBucketListener != null) {
       partitionAttributesFactory.addPartitionListener(lucenePrimaryBucketListener);
     }
@@ -217,23 +191,13 @@ public class LuceneIndexForPartitionedRegion extends LuceneIndexImpl {
     // Invoke super destroy to remove the extension
     super.destroy(initiator);
 
-    // Destroy the chunk region (colocated with the file region)
-    // localDestroyRegion can't be used because locally destroying regions is not supported on
-    // colocated regions
-    if (!chunkRegion.isDestroyed()) {
-      chunkRegion.destroyRegion();
-      if (logger.isDebugEnabled()) {
-        logger.debug("Destroyed chunkRegion=" + chunkRegion.getName());
-      }
-    }
-
     // Destroy the file region (colocated with the application region)
     // localDestroyRegion can't be used because locally destroying regions is not supported on
     // colocated regions
-    if (!fileRegion.isDestroyed()) {
-      fileRegion.destroyRegion();
+    if (!fileAndChunkRegion.isDestroyed()) {
+      fileAndChunkRegion.destroyRegion();
       if (logger.isDebugEnabled()) {
-        logger.debug("Destroyed fileRegion=" + fileRegion.getName());
+        logger.debug("Destroyed fileAndChunkRegion=" + fileAndChunkRegion.getName());
       }
     }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/directory/RegionDirectory.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/directory/RegionDirectory.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/directory/RegionDirectory.java
index 18428ec..9cdf94b 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/directory/RegionDirectory.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/directory/RegionDirectory.java
@@ -47,10 +47,9 @@ public class RegionDirectory extends BaseDirectory {
    * Create a region directory with a given file and chunk region. These regions may be bucket
    * regions or they may be replicated regions.
    */
-  public RegionDirectory(Map<String, File> fileRegion, Map<ChunkKey, byte[]> chunkRegion,
-      FileSystemStats stats) {
+  public RegionDirectory(Map fileAndChunkRegion, FileSystemStats stats) {
     super(new SingleInstanceLockFactory());
-    fs = new FileSystem(fileRegion, chunkRegion, stats);
+    fs = new FileSystem(fileAndChunkRegion, stats);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystem.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystem.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystem.java
index 660816d..164955f 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystem.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystem.java
@@ -4,9 +4,9 @@
  * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance with the License. You may obtain a
  * copy of the License at
- * 
+ *
  * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ *
  * Unless required by applicable law or agreed to in writing, software distributed under the License
  * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
  * or implied. See the License for the specific language governing permissions and limitations under
@@ -15,7 +15,6 @@
 
 package org.apache.geode.cache.lucene.internal.filesystem;
 
-import org.apache.geode.cache.Region;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.logging.log4j.Logger;
 
@@ -23,15 +22,15 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Map;
-import java.util.concurrent.ConcurrentMap;
+import java.util.stream.Collectors;
 
 /**
  * A Filesystem like interface that stores file data in geode regions.
- * 
+ *
  * This filesystem is safe for use with multiple threads if the threads are not modifying the same
  * files. A single file is not safe to modify by multiple threads, even between different members of
  * the distributed system.
- * 
+ *
  * Changes to a file may not be visible to other members of the system until the FileOutputStream is
  * closed.
  *
@@ -39,37 +38,34 @@ import java.util.concurrent.ConcurrentMap;
 public class FileSystem {
   private static final Logger logger = LogService.getLogger();
 
-  private final Map<String, File> fileRegion;
-  private final Map<ChunkKey, byte[]> chunkRegion;
+  private final Map fileAndChunkRegion;
 
   static final int CHUNK_SIZE = 1024 * 1024; // 1 MB
   private final FileSystemStats stats;
 
   /**
-   * Create filesystem that will store data in the two provided regions. The fileRegion contains
-   * metadata about the files, and the chunkRegion contains the actual data. If data from either
-   * region is missing or inconsistent, no guarantees are made about what this class will do, so
-   * it's best if these regions are colocated and in the same disk store to ensure the data remains
-   * together.
-   * 
-   * @param fileRegion the region to store metadata about the files
-   * @param chunkRegion the region to store actual file data.
+   * Create filesystem that will store data in the two provided regions. The fileAndChunkRegion
+   * contains metadata about the files, and the chunkRegion contains the actual data. If data from
+   * either region is missing or inconsistent, no guarantees are made about what this class will do,
+   * so it's best if these regions are colocated and in the same disk store to ensure the data
+   * remains together.
+   *
+   * @param fileAndChunkRegion the region to store metadata about the files
    */
-  public FileSystem(Map<String, File> fileRegion, Map<ChunkKey, byte[]> chunkRegion,
-      FileSystemStats stats) {
-    this.fileRegion = fileRegion;
-    this.chunkRegion = chunkRegion;
+  public FileSystem(Map fileAndChunkRegion, FileSystemStats stats) {
+    this.fileAndChunkRegion = fileAndChunkRegion;
     this.stats = stats;
   }
 
   public Collection<String> listFileNames() {
-    return fileRegion.keySet();
+    return (Collection<String>) fileAndChunkRegion.keySet().stream()
+        .filter(entry -> (entry instanceof String)).collect(Collectors.toList());
   }
 
   public File createFile(final String name) throws IOException {
     // TODO lock region ?
     final File file = new File(this, name);
-    if (null != fileRegion.putIfAbsent(name, file)) {
+    if (null != fileAndChunkRegion.putIfAbsent(name, file)) {
       throw new IOException("File exists.");
     }
     stats.incFileCreates(1);
@@ -79,7 +75,7 @@ public class FileSystem {
 
   public File putIfAbsentFile(String name, File file) throws IOException {
     // TODO lock region ?
-    if (null != fileRegion.putIfAbsent(name, file)) {
+    if (null != fileAndChunkRegion.putIfAbsent(name, file)) {
       throw new IOException("File exists.");
     }
     stats.incFileCreates(1);
@@ -94,7 +90,7 @@ public class FileSystem {
   }
 
   public File getFile(final String name) throws FileNotFoundException {
-    final File file = fileRegion.get(name);
+    final File file = (File) fileAndChunkRegion.get(name);
 
     if (null == file) {
       throw new FileNotFoundException(name);
@@ -111,7 +107,7 @@ public class FileSystem {
     // things crash in the middle of removing this file?
     // Seems like a file will be left with some
     // dangling chunks at the end of the file
-    File file = fileRegion.remove(name);
+    File file = (File) fileAndChunkRegion.remove(name);
     if (file == null) {
       throw new FileNotFoundException(name);
     }
@@ -121,7 +117,7 @@ public class FileSystem {
       final ChunkKey key = new ChunkKey(file.id, 0);
       while (true) {
         // TODO consider mutable ChunkKey
-        if (null == chunkRegion.remove(key)) {
+        if (null == fileAndChunkRegion.remove(key)) {
           // no more chunks
           break;
         }
@@ -133,7 +129,7 @@ public class FileSystem {
 
   public void renameFile(String source, String dest) throws IOException {
 
-    final File sourceFile = fileRegion.get(source);
+    final File sourceFile = (File) fileAndChunkRegion.get(source);
     if (null == sourceFile) {
       throw new FileNotFoundException(source);
     }
@@ -152,7 +148,7 @@ public class FileSystem {
     updateFile(sourceFile);
     putIfAbsentFile(dest, destFile);
 
-    fileRegion.remove(source);
+    fileAndChunkRegion.remove(source);
     stats.incFileRenames(1);
   }
 
@@ -162,14 +158,14 @@ public class FileSystem {
     // The file's metadata indicates that this chunk shouldn't
     // exist. Purge all of the chunks that are larger than the file metadata
     if (id >= file.chunks) {
-      while (chunkRegion.containsKey(key)) {
-        chunkRegion.remove(key);
+      while (fileAndChunkRegion.containsKey(key)) {
+        fileAndChunkRegion.remove(key);
         key.chunkId++;
       }
       return null;
     }
 
-    final byte[] chunk = chunkRegion.get(key);
+    final byte[] chunk = (byte[]) fileAndChunkRegion.get(key);
     if (chunk != null) {
       stats.incReadBytes(chunk.length);
     } else {
@@ -181,21 +177,18 @@ public class FileSystem {
 
   public void putChunk(final File file, final int id, final byte[] chunk) {
     final ChunkKey key = new ChunkKey(file.id, id);
-    chunkRegion.put(key, chunk);
+    fileAndChunkRegion.put(key, chunk);
     stats.incWrittenBytes(chunk.length);
   }
 
   void updateFile(File file) {
-    fileRegion.put(file.getName(), file);
+    fileAndChunkRegion.put(file.getName(), file);
   }
 
-  public Map<String, File> getFileRegion() {
-    return fileRegion;
+  public Map getFileAndChunkRegion() {
+    return fileAndChunkRegion;
   }
 
-  public Map<ChunkKey, byte[]> getChunkRegion() {
-    return chunkRegion;
-  }
 
   /**
    * Export all of the files in the filesystem to the provided directory
@@ -208,7 +201,6 @@ public class FileSystem {
       } catch (FileNotFoundException e) {
         // ignore this, it was concurrently removed
       }
-
     });
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystemStats.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystemStats.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystemStats.java
index 85ae6d7..09bb989 100644
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystemStats.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystemStats.java
@@ -38,8 +38,6 @@ public class FileSystemStats {
   private static final int temporaryFileCreatesId;
   private static final int fileDeletesId;
   private static final int fileRenamesId;
-  private static final int filesId;
-  private static final int chunksId;
   private static final int bytesId;
 
   static {
@@ -63,8 +61,6 @@ public class FileSystemStats {
     temporaryFileCreatesId = statsType.nameToId("temporaryFileCreates");
     fileDeletesId = statsType.nameToId("fileDeletes");
     fileRenamesId = statsType.nameToId("fileRenames");
-    filesId = statsType.nameToId("files");
-    chunksId = statsType.nameToId("chunks");
     bytesId = statsType.nameToId("bytes");
   }
 
@@ -96,22 +92,6 @@ public class FileSystemStats {
     stats.incInt(fileRenamesId, delta);
   }
 
-  public void setFileSupplier(IntSupplier supplier) {
-    stats.setIntSupplier(filesId, supplier);
-  }
-
-  public int getFiles() {
-    return stats.getInt(filesId);
-  }
-
-  public void setChunkSupplier(IntSupplier supplier) {
-    stats.setIntSupplier(chunksId, supplier);
-  }
-
-  public int getChunks() {
-    return stats.getInt(chunksId);
-  }
-
   public void setBytesSupplier(LongSupplier supplier) {
     stats.setLongSupplier(bytesId, supplier);
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
index 037c99f..c878fb7 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexDestroyDUnitTest.java
@@ -439,11 +439,6 @@ public class LuceneIndexDestroyDUnitTest extends LuceneDUnitTest {
         LuceneIndexForPartitionedRegion.FILES_REGION_SUFFIX);
     assertNull(getCache().getRegion(filesRegionName));
 
-    // Verify the underlying chunks region no longer exists
-    String chunksRegionName = LuceneServiceImpl.getUniqueIndexRegionName(indexName, REGION_NAME,
-        LuceneIndexForPartitionedRegion.CHUNKS_REGION_SUFFIX);
-    assertNull(getCache().getRegion(chunksRegionName));
-
     // Verify the underlying AsyncEventQueue no longer exists
     String aeqId = LuceneServiceImpl.getUniqueIndexName(indexName, REGION_NAME);
     assertNull(getCache().getAsyncEventQueue(aeqId));

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
index 00abb99..f5bd83f 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
@@ -143,8 +143,6 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
     FileSystemStats fileSystemStats = index.getFileSystemStats();
     LuceneIndexStats indexStats = index.getIndexStats();
     await(() -> assertEquals(4, indexStats.getDocuments()));
-    await(() -> assertTrue(fileSystemStats.getFiles() > 0));
-    await(() -> assertTrue(fileSystemStats.getChunks() > 0));
     await(() -> assertTrue(fileSystemStats.getBytes() > 0));
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
index f7bb5ab..65b250c 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
@@ -88,8 +88,6 @@ public class LuceneQueriesPersistenceIntegrationTest extends LuceneIntegrationTe
 
     PartitionedRegion fileRegion = (PartitionedRegion) cache.getRegion(aeqId + ".files");
     assertNotNull(fileRegion);
-    PartitionedRegion chunkRegion = (PartitionedRegion) cache.getRegion(aeqId + ".chunks");
-    assertNotNull(chunkRegion);
     Assert.assertTrue(0 < userRegion.getDiskRegionStats().getNumOverflowOnDisk());
 
     LuceneQuery<Integer, Type1> query = service.createLuceneQueryFactory().create(INDEX_NAME,

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
index 93cc0a8..f2bfdd4 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexForPartitionedRegionTest.java
@@ -99,33 +99,6 @@ public class LuceneIndexForPartitionedRegionTest {
   }
 
   @Test
-  public void chunkRegionExistsWhenChunkRegionExistsShouldReturnTrue() {
-    String name = "indexName";
-    String regionPath = "regionName";
-    Cache cache = Fakes.cache();
-    PartitionedRegion region = mock(PartitionedRegion.class);
-    LuceneIndexForPartitionedRegion index =
-        new LuceneIndexForPartitionedRegion(name, regionPath, cache);
-    String chunkRegionName = index.createChunkRegionName();
-    when(cache.getRegion(chunkRegionName)).thenReturn(region);
-
-    assertTrue(index.chunkRegionExists(chunkRegionName));
-  }
-
-  @Test
-  public void chunkRegionExistsWhenChunkRegionDoesNotExistShouldReturnFalse() {
-    String name = "indexName";
-    String regionPath = "regionName";
-    Cache cache = Fakes.cache();
-    LuceneIndexForPartitionedRegion index =
-        new LuceneIndexForPartitionedRegion(name, regionPath, cache);
-    String chunkRegionName = index.createChunkRegionName();
-    when(cache.getRegion(chunkRegionName)).thenReturn(null);
-
-    assertFalse(index.chunkRegionExists(chunkRegionName));
-  }
-
-  @Test
   public void createAEQWithPersistenceCallsCreateOnAEQFactory() {
     String name = "indexName";
     String regionPath = "regionName";
@@ -244,30 +217,13 @@ public class LuceneIndexForPartitionedRegionTest {
       final LuceneIndexForPartitionedRegion index) {
     index.setSearchableFields(new String[] {"field"});
     LuceneIndexForPartitionedRegion spy = spy(index);
-    doReturn(null).when(spy).createFileRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any(), any(), any());
+    doReturn(null).when(spy).createFileRegion(any(), any(), any(), any(), any());
     doReturn(null).when(spy).createAEQ(eq(region));
     spy.initialize();
     return spy;
   }
 
   @Test
-  public void initializeShouldCreatePartitionChunkRegion() {
-    boolean withPersistence = false;
-    String name = "indexName";
-    String regionPath = "regionName";
-    Cache cache = Fakes.cache();
-    Region region = initializeScenario(withPersistence, regionPath, cache);
-
-    LuceneIndexForPartitionedRegion index =
-        new LuceneIndexForPartitionedRegion(name, regionPath, cache);
-    LuceneIndexForPartitionedRegion spy = setupSpy(region, index);
-
-    verify(spy).createChunkRegion(eq(RegionShortcut.PARTITION), eq(index.createFileRegionName()),
-        any(), eq(index.createChunkRegionName()), any(), any());
-  }
-
-  @Test
   public void initializeShouldCreatePartitionFileRegion() {
     boolean withPersistence = false;
     String name = "indexName";
@@ -280,7 +236,7 @@ public class LuceneIndexForPartitionedRegionTest {
     LuceneIndexForPartitionedRegion spy = setupSpy(region, index);
 
     verify(spy).createFileRegion(eq(RegionShortcut.PARTITION), eq(index.createFileRegionName()),
-        any(), any());
+        any(), any(), any());
   }
 
   @Test
@@ -296,7 +252,7 @@ public class LuceneIndexForPartitionedRegionTest {
         new LuceneIndexForPartitionedRegion(name, regionPath, cache);
     LuceneIndexForPartitionedRegion indexSpy = spy(index);
     indexSpy.createFileRegion(RegionShortcut.PARTITION, index.createFileRegionName(),
-        partitionAttributes, regionAttributes);
+        partitionAttributes, regionAttributes, null);
     String fileRegionName = index.createFileRegionName();
     verify(indexSpy).createRegion(fileRegionName, RegionShortcut.PARTITION, regionPath,
         partitionAttributes, regionAttributes, null);
@@ -304,48 +260,6 @@ public class LuceneIndexForPartitionedRegionTest {
   }
 
   @Test
-  public void createChunkRegionWithPartitionShortcutCreatesRegionUsingCreateVMRegion()
-      throws Exception {
-    String name = "indexName";
-    String regionPath = "regionName";
-    GemFireCacheImpl cache = Fakes.cache();
-    PartitionAttributes partitionAttributes = initializeAttributes(cache);
-    RegionAttributes regionAttributes = mock(RegionAttributes.class);
-    when(regionAttributes.getDataPolicy()).thenReturn(DataPolicy.PARTITION);
-    LuceneIndexForPartitionedRegion index =
-        new LuceneIndexForPartitionedRegion(name, regionPath, cache);
-    LuceneIndexForPartitionedRegion indexSpy = spy(index);
-    String chunkRegionName = index.createChunkRegionName();
-    String fileRegionName = index.createFileRegionName();
-    indexSpy.createChunkRegion(RegionShortcut.PARTITION, fileRegionName, partitionAttributes,
-        chunkRegionName, regionAttributes, null);
-    verify(indexSpy).createRegion(chunkRegionName, RegionShortcut.PARTITION, fileRegionName,
-        partitionAttributes, regionAttributes, null);
-    verify(cache).createVMRegion(eq(chunkRegionName), any(), any());
-  }
-
-  @Test
-  public void initializeShouldCreatePartitionPersistentChunkRegion() {
-    boolean withPersistence = true;
-    String name = "indexName";
-    String regionPath = "regionName";
-    Cache cache = Fakes.cache();
-    initializeScenario(withPersistence, regionPath, cache);
-
-    LuceneIndexForPartitionedRegion index =
-        new LuceneIndexForPartitionedRegion(name, regionPath, cache);
-    index.setSearchableFields(new String[] {"field"});
-    LuceneIndexForPartitionedRegion spy = spy(index);
-    doReturn(null).when(spy).createFileRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any(), any(), any());
-    doReturn(null).when(spy).createAEQ(any());
-    spy.initialize();
-
-    verify(spy).createChunkRegion(eq(RegionShortcut.PARTITION_PERSISTENT),
-        eq(index.createFileRegionName()), any(), eq(index.createChunkRegionName()), any(), any());
-  }
-
-  @Test
   public void initializeShouldCreatePartitionPersistentFileRegion() {
     boolean withPersistence = true;
     String name = "indexName";
@@ -357,13 +271,12 @@ public class LuceneIndexForPartitionedRegionTest {
         new LuceneIndexForPartitionedRegion(name, regionPath, cache);
     index.setSearchableFields(new String[] {"field"});
     LuceneIndexForPartitionedRegion spy = spy(index);
-    doReturn(null).when(spy).createFileRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any(), any(), any());
+    doReturn(null).when(spy).createFileRegion(any(), any(), any(), any(), any());
     doReturn(null).when(spy).createAEQ(any());
     spy.initialize();
 
     verify(spy).createFileRegion(eq(RegionShortcut.PARTITION_PERSISTENT),
-        eq(index.createFileRegionName()), any(), any());
+        eq(index.createFileRegionName()), any(), any(), any());
   }
 
   @Test
@@ -378,14 +291,13 @@ public class LuceneIndexForPartitionedRegionTest {
         new LuceneIndexForPartitionedRegion(name, regionPath, cache);
     index.setSearchableFields(new String[] {"field"});
     LuceneIndexForPartitionedRegion spy = spy(index);
-    doReturn(null).when(spy).createFileRegion(any(), any(), any(), any());
-    doReturn(null).when(spy).createChunkRegion(any(), any(), any(), any(), any(), any());
+    doReturn(null).when(spy).createFileRegion(any(), any(), any(), any(), any());
     doReturn(null).when(spy).createAEQ(any());
     spy.initialize();
     spy.initialize();
 
     verify(spy).createFileRegion(eq(RegionShortcut.PARTITION_PERSISTENT),
-        eq(index.createFileRegionName()), any(), any());
+        eq(index.createFileRegionName()), any(), any(), any());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
index f853baf..b6b1d23 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/LuceneIndexRecoveryHAIntegrationTest.java
@@ -69,8 +69,8 @@ public class LuceneIndexRecoveryHAIntegrationTest {
   }
 
   /**
-   * On rebalance, new repository manager will be created. It will try to read fileRegion and
-   * construct index. This test simulates the same.
+   * On rebalance, new repository manager will be created. It will try to read fileAndChunkRegion
+   * and construct index. This test simulates the same.
    */
   // @Test
   public void recoverRepoInANewNode()
@@ -99,7 +99,7 @@ public class LuceneIndexRecoveryHAIntegrationTest {
 
     // close the region to simulate bucket movement. New node will create repo using data persisted
     // by old region
-    // ((PartitionedRegion)index.fileRegion).close();
+    // ((PartitionedRegion)index.fileAndChunkRegion).close();
     // ((PartitionedRegion)index.chunkRegion).close();
     userRegion.close();
 

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
index 9c603c7..87317cc 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/PartitionedRepositoryManagerJUnitTest.java
@@ -29,7 +29,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.geode.cache.lucene.internal.partition.BucketTargetingMap;
-import org.apache.geode.distributed.DistributedLockService;
 import org.apache.geode.distributed.internal.locks.DLockService;
 import org.apache.geode.internal.cache.BucketAdvisor;
 import org.apache.geode.internal.cache.PartitionedRegionHelper;
@@ -63,15 +62,12 @@ import org.apache.geode.test.junit.categories.UnitTest;
 public class PartitionedRepositoryManagerJUnitTest {
 
   protected PartitionedRegion userRegion;
-  protected PartitionedRegion fileRegion;
-  protected PartitionedRegion chunkRegion;
+  protected PartitionedRegion fileAndChunkRegion;
   protected LuceneSerializer serializer;
   protected PartitionedRegionDataStore userDataStore;
   protected PartitionedRegionDataStore fileDataStore;
-  protected PartitionedRegionDataStore chunkDataStore;
 
-  protected Map<Integer, BucketRegion> fileBuckets = new HashMap<Integer, BucketRegion>();
-  protected Map<Integer, BucketRegion> chunkBuckets = new HashMap<Integer, BucketRegion>();
+  protected Map<Integer, BucketRegion> fileAndChunkBuckets = new HashMap<Integer, BucketRegion>();
   protected Map<Integer, BucketRegion> dataBuckets = new HashMap<Integer, BucketRegion>();
   protected LuceneIndexStats indexStats;
   protected FileSystemStats fileSystemStats;
@@ -100,20 +96,16 @@ public class PartitionedRepositoryManagerJUnitTest {
   }
 
   protected void createIndexAndRepoManager() {
-    fileRegion = Mockito.mock(PartitionedRegion.class);
+    fileAndChunkRegion = Mockito.mock(PartitionedRegion.class);
     fileDataStore = Mockito.mock(PartitionedRegionDataStore.class);
-    when(fileRegion.getDataStore()).thenReturn(fileDataStore);
-    when(fileRegion.getTotalNumberOfBuckets()).thenReturn(113);
-    when(fileRegion.getFullPath()).thenReturn("FileRegion");
-    chunkRegion = Mockito.mock(PartitionedRegion.class);
-    chunkDataStore = Mockito.mock(PartitionedRegionDataStore.class);
-    when(chunkRegion.getFullPath()).thenReturn("ChunkRegion");
-    when(chunkRegion.getDataStore()).thenReturn(chunkDataStore);
+    when(fileAndChunkRegion.getDataStore()).thenReturn(fileDataStore);
+    when(fileAndChunkRegion.getTotalNumberOfBuckets()).thenReturn(113);
+    when(fileAndChunkRegion.getFullPath()).thenReturn("FileRegion");
     indexStats = Mockito.mock(LuceneIndexStats.class);
     fileSystemStats = Mockito.mock(FileSystemStats.class);
     indexForPR = Mockito.mock(LuceneIndexForPartitionedRegion.class);
-    when(((LuceneIndexForPartitionedRegion) indexForPR).getFileRegion()).thenReturn(fileRegion);
-    when(((LuceneIndexForPartitionedRegion) indexForPR).getChunkRegion()).thenReturn(chunkRegion);
+    when(((LuceneIndexForPartitionedRegion) indexForPR).getFileAndChunkRegion())
+        .thenReturn(fileAndChunkRegion);
     when(((LuceneIndexForPartitionedRegion) indexForPR).getFileSystemStats())
         .thenReturn(fileSystemStats);
     when(indexForPR.getIndexStats()).thenReturn(indexStats);
@@ -159,7 +151,7 @@ public class PartitionedRepositoryManagerJUnitTest {
     assertNotNull(repo0);
     checkRepository(repo0, 0);
 
-    BucketRegion fileBucket0 = fileBuckets.get(0);
+    BucketRegion fileBucket0 = fileAndChunkBuckets.get(0);
     BucketRegion dataBucket0 = dataBuckets.get(0);
 
     // Simulate rebalancing of a bucket by marking the old bucket is destroyed
@@ -189,11 +181,11 @@ public class PartitionedRepositoryManagerJUnitTest {
 
     when(fileDataStore.getLocalBucketById(eq(0))).thenReturn(null);
 
-    when(fileRegion.getOrCreateNodeForBucketWrite(eq(0), (RetryTimeKeeper) any()))
+    when(fileAndChunkRegion.getOrCreateNodeForBucketWrite(eq(0), (RetryTimeKeeper) any()))
         .then(new Answer() {
           @Override
           public Object answer(InvocationOnMock invocation) throws Throwable {
-            when(fileDataStore.getLocalBucketById(eq(0))).thenReturn(fileBuckets.get(0));
+            when(fileDataStore.getLocalBucketById(eq(0))).thenReturn(fileAndChunkBuckets.get(0));
             return null;
           }
         });
@@ -241,36 +233,30 @@ public class PartitionedRepositoryManagerJUnitTest {
   protected void checkRepository(IndexRepositoryImpl repo0, int bucketId) {
     IndexWriter writer0 = repo0.getWriter();
     RegionDirectory dir0 = (RegionDirectory) writer0.getDirectory();
-    assertEquals(new BucketTargetingMap(fileBuckets.get(bucketId), bucketId),
-        dir0.getFileSystem().getFileRegion());
-    assertEquals(new BucketTargetingMap(chunkBuckets.get(bucketId), bucketId),
-        dir0.getFileSystem().getChunkRegion());
+    assertEquals(new BucketTargetingMap(fileAndChunkBuckets.get(bucketId), bucketId),
+        dir0.getFileSystem().getFileAndChunkRegion());
     assertEquals(serializer, repo0.getSerializer());
   }
 
   protected BucketRegion setUpMockBucket(int id) throws BucketNotFoundException {
     BucketRegion mockBucket = Mockito.mock(BucketRegion.class);
-    BucketRegion fileBucket = Mockito.mock(BucketRegion.class);
-    // Allowing the fileBucket to behave like a map so that the IndexWriter operations don't fail
-    Fakes.addMapBehavior(fileBucket);
-    when(fileBucket.getFullPath()).thenReturn("File" + id);
-    BucketRegion chunkBucket = Mockito.mock(BucketRegion.class);
-    when(chunkBucket.getFullPath()).thenReturn("Chunk" + id);
+    BucketRegion fileAndChunkBucket = Mockito.mock(BucketRegion.class);
+    // Allowing the fileAndChunkBucket to behave like a map so that the IndexWriter operations don't
+    // fail
+    Fakes.addMapBehavior(fileAndChunkBucket);
+    when(fileAndChunkBucket.getFullPath()).thenReturn("File" + id);
     when(mockBucket.getId()).thenReturn(id);
     when(userRegion.getBucketRegion(eq(id), eq(null))).thenReturn(mockBucket);
     when(userDataStore.getLocalBucketById(eq(id))).thenReturn(mockBucket);
     when(userRegion.getBucketRegion(eq(id + 113), eq(null))).thenReturn(mockBucket);
     when(userDataStore.getLocalBucketById(eq(id + 113))).thenReturn(mockBucket);
-    when(fileDataStore.getLocalBucketById(eq(id))).thenReturn(fileBucket);
-    when(chunkDataStore.getLocalBucketById(eq(id))).thenReturn(chunkBucket);
+    when(fileDataStore.getLocalBucketById(eq(id))).thenReturn(fileAndChunkBucket);
 
-    fileBuckets.put(id, fileBucket);
-    chunkBuckets.put(id, chunkBucket);
+    fileAndChunkBuckets.put(id, fileAndChunkBucket);
     dataBuckets.put(id, mockBucket);
 
     BucketAdvisor mockBucketAdvisor = Mockito.mock(BucketAdvisor.class);
-    when(fileBucket.getBucketAdvisor()).thenReturn(mockBucketAdvisor);
-    when(chunkBucket.getBucketAdvisor()).thenReturn(mockBucketAdvisor);
+    when(fileAndChunkBucket.getBucketAdvisor()).thenReturn(mockBucketAdvisor);
     when(mockBucketAdvisor.isPrimary()).thenReturn(true);
     return mockBucket;
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
index 32249e4..3f468f4 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/directory/RegionDirectoryJUnitTest.java
@@ -58,7 +58,6 @@ public class RegionDirectoryJUnitTest extends BaseDirectoryTestCase {
     // test asserts that no system properties have changed. Unfortunately, there is no
     // way to control the order of rules, so we can't clear this property with a rule
     // or @After method. Instead, do it in the close method of the directory.
-    return new RegionDirectory(new ConcurrentHashMap<String, File>(),
-        new ConcurrentHashMap<ChunkKey, byte[]>(), stats);
+    return new RegionDirectory(new ConcurrentHashMap(), stats);
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
index 6062904..41d90ef 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
@@ -132,9 +132,8 @@ public class DistributedScoringJUnitTest {
   }
 
   private IndexRepositoryImpl createIndexRepo() throws IOException {
-    ConcurrentHashMap<String, File> fileRegion = new ConcurrentHashMap<String, File>();
-    ConcurrentHashMap<ChunkKey, byte[]> chunkRegion = new ConcurrentHashMap<ChunkKey, byte[]>();
-    RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion, fileSystemStats);
+    ConcurrentHashMap fileAndChunkRegion = new ConcurrentHashMap();
+    RegionDirectory dir = new RegionDirectory(fileAndChunkRegion, fileSystemStats);
 
     IndexWriterConfig config = new IndexWriterConfig(analyzer);
     IndexWriter writer = new IndexWriter(dir, config);

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystemJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
index ee41e40..2008458 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
@@ -23,9 +23,11 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.lang.reflect.Method;
 import java.nio.file.Files;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -35,7 +37,6 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.ArgumentCaptor;
 import org.mockito.Mockito;
-import org.mockito.Spy;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -51,8 +52,7 @@ public class FileSystemJUnitTest {
 
   private FileSystem system;
   private Random rand = new Random();
-  private ConcurrentHashMap<String, File> fileRegion;
-  private ConcurrentHashMap<ChunkKey, byte[]> chunkRegion;
+  private ConcurrentHashMap fileAndChunkRegion;
 
   @Rule
   public DiskDirRule dirRule = new DiskDirRule();
@@ -60,10 +60,9 @@ public class FileSystemJUnitTest {
 
   @Before
   public void setUp() {
-    fileRegion = new ConcurrentHashMap<String, File>();
-    chunkRegion = new ConcurrentHashMap<ChunkKey, byte[]>();
+    fileAndChunkRegion = new ConcurrentHashMap();
     fileSystemStats = mock(FileSystemStats.class);
-    system = new FileSystem(fileRegion, chunkRegion, fileSystemStats);
+    system = new FileSystem(fileAndChunkRegion, fileSystemStats);
   }
 
   /**
@@ -201,7 +200,7 @@ public class FileSystemJUnitTest {
 
     // Assert that there are only 2 chunks in the system, since we wrote just
     // past the end of the first chunk.
-    assertEquals(2, chunkRegion.size());
+    assertEquals(2, numberOfChunks(fileAndChunkRegion));
 
     SeekableInputStream in = file.getInputStream();
 
@@ -254,7 +253,7 @@ public class FileSystemJUnitTest {
     file1 = system.getFile(name1);
     file2 = system.getFile(name2);
 
-    assertEquals(new HashSet(Arrays.asList(name1, name2)), system.listFileNames());
+    assertEquals(Arrays.asList(name1, name2), system.listFileNames());
     assertContents(file1Data, file1);
     assertContents(file2Data, file2);
 
@@ -265,7 +264,7 @@ public class FileSystemJUnitTest {
     } catch (IOException expected) {
 
     }
-    assertEquals(new HashSet(Arrays.asList(name1, name2)), system.listFileNames());
+    assertEquals(Arrays.asList(name1, name2), system.listFileNames());
     assertContents(file1Data, file1);
     assertContents(file2Data, file2);
 
@@ -275,17 +274,17 @@ public class FileSystemJUnitTest {
 
     File file3 = system.getFile(name3);
 
-    assertEquals(new HashSet(Arrays.asList(name3, name2)), system.listFileNames());
+    assertEquals(Arrays.asList(name3, name2), system.listFileNames());
     assertContents(file1Data, file3);
     assertContents(file2Data, file2);
 
     system.deleteFile(name2);
 
-    assertEquals(new HashSet(Arrays.asList(name3)), system.listFileNames());
+    assertEquals(Arrays.asList(name3), system.listFileNames());
 
     system.renameFile(name3, name2);
 
-    assertEquals(new HashSet(Arrays.asList(name2)), system.listFileNames());
+    assertEquals(Arrays.asList(name2), system.listFileNames());
 
     file2 = system.getFile(name2);
     assertContents(file1Data, file2);
@@ -313,7 +312,7 @@ public class FileSystemJUnitTest {
     byte[] bytes = getRandomBytes(size);
     file1.getOutputStream().write(bytes);
 
-    FileSystem system2 = new FileSystem(fileRegion, chunkRegion, fileSystemStats);
+    FileSystem system2 = new FileSystem(fileAndChunkRegion, fileSystemStats);
     File file = system2.getFile(name1);
 
     assertTrue(file.getLength() <= bytes.length);
@@ -325,7 +324,7 @@ public class FileSystemJUnitTest {
 
     if (length == 0) {
       assertEquals(-1, file.getInputStream().read(results));
-      assertTrue(chunkRegion.isEmpty());
+      assertEquals(0, numberOfChunks(fileAndChunkRegion));
     } else {
       // Make sure the amount of data we can read matches the length
       assertEquals(length, file.getInputStream().read(results));
@@ -349,12 +348,11 @@ public class FileSystemJUnitTest {
     // Create a couple of mock regions where we count the operations
     // that happen to them. We will then use this to abort the rename
     // in the middle.
-    ConcurrentHashMap<String, File> spyFileRegion =
-        mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, fileRegion));
-    ConcurrentHashMap<ChunkKey, byte[]> spyChunkRegion =
-        mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, chunkRegion));
+    ConcurrentHashMap spyFileAndChunkRegion =
+        mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, fileAndChunkRegion));
 
-    system = new FileSystem(spyFileRegion, spyChunkRegion, fileSystemStats);
+
+    system = new FileSystem(spyFileAndChunkRegion, fileSystemStats);
 
     String name = "file";
     File file = system.createFile(name);
@@ -394,7 +392,7 @@ public class FileSystemJUnitTest {
 
     }
 
-    system = new FileSystem(fileRegion, chunkRegion, fileSystemStats);
+    system = new FileSystem(fileAndChunkRegion, fileSystemStats);
 
     // This is not the ideal behavior. We are left
     // with two duplicate files. However, we will still
@@ -409,72 +407,6 @@ public class FileSystemJUnitTest {
     assertContents(expectedBytes, destFile);
   }
 
-  /**
-   * Test what happens a file delete is aborted in the middle due to the a cache closed exception.
-   * The next member that uses those files should be able to clean up after the partial rename.
-   */
-  @Test
-  public void testPartialDelete() throws Exception {
-
-    final CountOperations countOperations = new CountOperations();
-    // Create a couple of mock regions where we count the operations
-    // that happen to them. We will then use this to abort the rename
-    // in the middle.
-    ConcurrentHashMap<String, File> spyFileRegion =
-        mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, fileRegion));
-    ConcurrentHashMap<ChunkKey, byte[]> spyChunkRegion =
-        mock(ConcurrentHashMap.class, new SpyWrapper(countOperations, chunkRegion));
-
-    system = new FileSystem(spyFileRegion, spyChunkRegion, fileSystemStats);
-
-    String name1 = "file1";
-    String name2 = "file2";
-    File file1 = system.createFile(name1);
-    File file2 = system.createFile(name2);
-
-    ByteArrayOutputStream expected = new ByteArrayOutputStream();
-
-    // Make sure the file has a lot of chunks
-    for (int i = 0; i < 10; i++) {
-      byte[] bytes = writeRandomBytes(file1);
-      writeBytes(file2, bytes);
-      expected.write(bytes);
-    }
-
-    countOperations.reset();
-
-    system.deleteFile(name1);
-
-    assertTrue(2 <= countOperations.count);
-
-    countOperations.after(countOperations.count / 2, new Runnable() {
-
-      @Override
-      public void run() {
-        throw new CacheClosedException();
-      }
-    });
-    countOperations.reset();
-
-    try {
-      system.deleteFile(name2);
-      fail("should have seen an error");
-    } catch (CacheClosedException expectedException) {
-    }
-
-    system = new FileSystem(fileRegion, chunkRegion, fileSystemStats);
-
-    if (system.listFileNames().size() == 0) {
-      // File was deleted, but shouldn't have any dangling chunks at this point
-      assertEquals(Collections.EMPTY_SET, fileRegion.keySet());
-      // TODO - need to purge chunks of deleted files somehow.
-      // assertIndexDetailsEquals(Collections.EMPTY_SET, chunkRegion.keySet());
-    } else {
-      file2 = system.getFile(name2);
-      assertContents(expected.toByteArray(), file2);
-    }
-  }
-
   @Test
   public void testExport() throws IOException {
     String name1 = "testFile1";
@@ -544,8 +476,8 @@ public class FileSystemJUnitTest {
 
   @Test
   public void testDeletePossiblyRenamedFileDoesNotDestroyChunks() throws Exception {
-    ConcurrentHashMap<String, File> spyFileRegion = Mockito.spy(fileRegion);
-    system = new FileSystem(spyFileRegion, chunkRegion, fileSystemStats);
+    ConcurrentHashMap spyFileRegion = Mockito.spy(fileAndChunkRegion);
+    system = new FileSystem(spyFileRegion, fileSystemStats);
 
     String sourceFileName = "sourceFile";
     File file1 = system.createFile(sourceFileName);
@@ -610,6 +542,10 @@ public class FileSystemJUnitTest {
     return data;
   }
 
+  private long numberOfChunks(Map map) {
+    return map.keySet().parallelStream().filter(k -> (k instanceof ChunkKey)).count();
+  }
+
   /**
    * A wrapper around an object that will also invoke a callback before applying an operation.
    *

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
index 42cc2bc..78fc657 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplJUnitTest.java
@@ -69,10 +69,9 @@ public class IndexRepositoryImplJUnitTest {
 
   @Before
   public void setUp() throws IOException {
-    ConcurrentHashMap<String, File> fileRegion = new ConcurrentHashMap<String, File>();
-    ConcurrentHashMap<ChunkKey, byte[]> chunkRegion = new ConcurrentHashMap<ChunkKey, byte[]>();
+    ConcurrentHashMap fileAndChunkRegion = new ConcurrentHashMap();
     fileSystemStats = mock(FileSystemStats.class);
-    RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion, fileSystemStats);
+    RegionDirectory dir = new RegionDirectory(fileAndChunkRegion, fileSystemStats);
     IndexWriterConfig config = new IndexWriterConfig(analyzer);
     writer = new IndexWriter(dir, config);
     String[] indexedFields = new String[] {"s", "i", "l", "d", "f", "s2", "missing"};

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
index 9067376..46425b9 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
@@ -93,12 +93,10 @@ public class IndexRepositoryImplPerformanceTest {
       @Override
       public void init() throws Exception {
         cache = new CacheFactory().set(MCAST_PORT, "0").set(LOG_LEVEL, "error").create();
-        Region<String, File> fileRegion =
-            cache.<String, File>createRegionFactory(RegionShortcut.REPLICATE).create("files");
-        Region<ChunkKey, byte[]> chunkRegion =
-            cache.<ChunkKey, byte[]>createRegionFactory(RegionShortcut.REPLICATE).create("chunks");
+        Region fileAndChunkRegion =
+            cache.createRegionFactory(RegionShortcut.REPLICATE).create("files");
 
-        RegionDirectory dir = new RegionDirectory(fileRegion, chunkRegion,
+        RegionDirectory dir = new RegionDirectory(fileAndChunkRegion,
             new FileSystemStats(cache.getDistributedSystem(), "region-index"));
         final LuceneIndexStats stats =
             new LuceneIndexStats(cache.getDistributedSystem(), "region-index");
@@ -108,7 +106,7 @@ public class IndexRepositoryImplPerformanceTest {
         writer = new IndexWriter(dir, config);
         String[] indexedFields = new String[] {"text"};
         HeterogeneousLuceneSerializer mapper = new HeterogeneousLuceneSerializer(indexedFields);
-        repo = new IndexRepositoryImpl(fileRegion, writer, mapper, stats, null);
+        repo = new IndexRepositoryImpl(fileAndChunkRegion, writer, mapper, stats, null);
       }
 
       @Override
@@ -213,8 +211,7 @@ public class IndexRepositoryImplPerformanceTest {
       public void init() throws Exception {
         cache = new CacheFactory().set(MCAST_PORT, "0").set(LOG_LEVEL, "warning").create();
         final FileSystemStats stats = new FileSystemStats(cache.getDistributedSystem(), "stats");
-        RegionDirectory dir = new RegionDirectory(new ConcurrentHashMap<String, File>(),
-            new ConcurrentHashMap<ChunkKey, byte[]>(), stats);
+        RegionDirectory dir = new RegionDirectory(new ConcurrentHashMap(), stats);
         IndexWriterConfig config = new IndexWriterConfig(analyzer);
         writer = new IndexWriter(dir, config);
         searcherManager = new SearcherManager(writer, true, true, null);

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a89730/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
index e2d3abc..670e6a8 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/LuceneTestUtilities.java
@@ -168,10 +168,7 @@ public class LuceneTestUtilities {
     LuceneIndexForPartitionedRegion index =
         (LuceneIndexForPartitionedRegion) luceneService.getIndex(INDEX_NAME, REGION_NAME);
 
-    // Verify the meta regions exist and are internal
-    LocalRegion chunkRegion = (LocalRegion) cache.getRegion(index.createChunkRegionName());
     LocalRegion fileRegion = (LocalRegion) cache.getRegion(index.createFileRegionName());
-    verify.accept(chunkRegion);
     verify.accept(fileRegion);
   }