You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by up...@apache.org on 2016/06/01 00:11:25 UTC

incubator-geode git commit: GEODE-11: Adding a tool to dump the lucene index files

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 384d379ae -> 3f356594e


GEODE-11: Adding a tool to dump the lucene index files

Adding a function that will dump all of the files for the lucene index
to disk, for examination with external tools like Luke.


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

Branch: refs/heads/develop
Commit: 3f356594e63c8a601330b558bbc7023d6d7653c6
Parents: 384d379
Author: Dan Smith <up...@apache.org>
Authored: Fri May 27 11:55:26 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Tue May 31 17:05:49 2016 -0700

----------------------------------------------------------------------
 .../gemfire/test/junit/rules/DiskDirRule.java   |   8 ++
 .../lucene/internal/LuceneServiceImpl.java      |   2 +
 .../internal/directory/DumpDirectoryFiles.java  | 111 ++++++++++++++++
 .../cache/lucene/internal/filesystem/File.java  |  19 ++-
 .../lucene/internal/filesystem/FileSystem.java  |  17 +++
 .../internal/repository/IndexRepository.java    |  12 +-
 .../repository/IndexRepositoryImpl.java         |   5 +
 .../DumpDirectoryFilesIntegrationTest.java      |  87 +++++++++++++
 .../directory/DumpDirectoryFilesJUnitTest.java  | 127 +++++++++++++++++++
 .../filesystem/FileSystemJUnitTest.java         |  32 +++++
 10 files changed, 417 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3f356594/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java
----------------------------------------------------------------------
diff --git a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java
index 184619f..7ce8bbc 100644
--- a/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java
+++ b/geode-junit/src/main/java/com/gemstone/gemfire/test/junit/rules/DiskDirRule.java
@@ -24,6 +24,10 @@ import java.nio.file.Files;
 
 import org.junit.rules.ExternalResource;
 
+/**
+ * A rule that creates a temporary directory and
+ * cleans it up after the test.
+ */
 public class DiskDirRule extends ExternalResource {
   private File diskDir;
 
@@ -32,6 +36,10 @@ public class DiskDirRule extends ExternalResource {
   }
 
   @Override protected void after() {
+    if(!diskDir.exists()) {
+      return;
+    }
+
     try {
       Files.walk(diskDir.toPath()).forEach((path) -> {
         try {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3f356594/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
index ecade52..f9bb8ba 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImpl.java
@@ -33,6 +33,7 @@ import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
+import com.gemstone.gemfire.cache.lucene.internal.directory.DumpDirectoryFiles;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.EntryScore;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunctionContext;
@@ -80,6 +81,7 @@ public class LuceneServiceImpl implements InternalLuceneService {
     this.cache = gfc;
 
     FunctionService.registerFunction(new LuceneFunction());
+    FunctionService.registerFunction(new DumpDirectoryFiles());
     registerDataSerializables();
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3f356594/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFiles.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFiles.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFiles.java
new file mode 100644
index 0000000..0d7b859
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFiles.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding 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 the License.
+ */
+
+package com.gemstone.gemfire.cache.lucene.internal.directory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.execute.Function;
+import com.gemstone.gemfire.cache.execute.FunctionAdapter;
+import com.gemstone.gemfire.cache.execute.FunctionContext;
+import com.gemstone.gemfire.cache.execute.FunctionException;
+import com.gemstone.gemfire.cache.execute.RegionFunctionContext;
+import com.gemstone.gemfire.cache.execute.ResultSender;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
+import com.gemstone.gemfire.cache.lucene.LuceneService;
+import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
+import com.gemstone.gemfire.cache.lucene.internal.InternalLuceneIndex;
+import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.CollectorManager;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunctionContext;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollectorManager;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.FileSystem;
+import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
+import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepositoryImpl;
+import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector;
+import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
+import com.gemstone.gemfire.cache.query.QueryException;
+import com.gemstone.gemfire.internal.InternalEntity;
+import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
+import com.gemstone.gemfire.internal.logging.LogService;
+
+import org.apache.logging.log4j.Logger;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.Directory;
+
+public class DumpDirectoryFiles implements Function, InternalEntity {
+  private static final long serialVersionUID = 1L;
+
+  private static final Logger logger = LogService.getLogger();
+
+  @Override
+  public void execute(FunctionContext context) {
+    RegionFunctionContext ctx = (RegionFunctionContext) context;
+
+    if(!(context.getArguments() instanceof String[])) {
+      throw new IllegalArgumentException("Arguments should be a string array");
+    }
+    String[] args = (String[]) context.getArguments();
+    if(args.length != 2) {
+      throw new IllegalArgumentException("Expected 2 arguments: exportLocation, indexName");
+    }
+
+
+    String exportLocation =args[0];
+    String indexName =args[1];
+
+    final Region<Object, Object> region = ctx.getDataSet();
+    LuceneService service = LuceneServiceProvider.get(ctx.getDataSet().getCache());
+    InternalLuceneIndex index = (InternalLuceneIndex) service.getIndex(indexName, region.getFullPath());
+    if(index == null) {
+      throw new IllegalStateException("Index not found for region " + region + " index " + indexName);
+    }
+
+    final RepositoryManager repoManager = index.getRepositoryManager();
+    try {
+      final Collection<IndexRepository> repositories = repoManager.getRepositories(ctx);
+      repositories.stream().forEach(repo -> {
+        final IndexWriter writer = repo.getWriter();
+        RegionDirectory directory = (RegionDirectory) writer.getDirectory();
+        FileSystem fs = directory.getFileSystem();
+
+        String bucketName = index.getName() + "_" + repo.getRegion().getFullPath();
+        bucketName = bucketName.replace("/", "_");
+        File bucketDirectory = new File(exportLocation, bucketName);
+        bucketDirectory.mkdirs();
+        fs.export(bucketDirectory);
+      });
+      context.getResultSender().lastResult(null);
+    }
+    catch (BucketNotFoundException e) {
+      throw new FunctionException(e);
+    }
+  }
+
+  @Override
+  public boolean optimizeForWrite() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3f356594/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/File.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/File.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/File.java
index 2937af5..d27717e 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/File.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/File.java
@@ -25,9 +25,11 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Serializable;
+import java.nio.file.Files;
 import java.util.UUID;
 
 import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.InternalGemFireError;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.Version;
 
@@ -150,6 +152,19 @@ public class File implements DataSerializableFixedID {
     long low = in.readLong();
     id = new UUID(high, low);
   }
-  
-  
+
+
+  /**
+   * Export this to a {@link java.io.File}
+   */
+  public void export(final java.io.File exportLocation)
+  {
+    java.io.File targetFile = new java.io.File(exportLocation, getName());
+    try {
+      Files.copy(getInputStream(), targetFile.toPath());
+    }
+    catch (IOException e) {
+      throw new InternalGemFireError("Could not export file " + getName(), e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3f356594/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
index fdafcbe..5f5218a 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystem.java
@@ -167,4 +167,21 @@ public class FileSystem {
   public ConcurrentMap<ChunkKey, byte[]> getChunkRegion() {
     return chunkRegion;
   }
+
+  /**
+   * Export all of the files in the filesystem to the provided directory
+   */
+  public void export(final java.io.File exportLocation) {
+
+    listFileNames().stream().forEach(fileName-> {
+      try {
+        getFile(fileName).export(exportLocation);
+      }
+      catch (FileNotFoundException e) {
+        //ignore this, it was concurrently removed
+      }
+
+    });
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3f356594/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepository.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepository.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepository.java
index f1e63e0..fab2c2a 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepository.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepository.java
@@ -21,6 +21,9 @@ package com.gemstone.gemfire.cache.lucene.internal.repository;
 
 import java.io.IOException;
 
+import com.gemstone.gemfire.cache.Region;
+
+import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.search.Query;
 
 /**
@@ -64,11 +67,18 @@ public interface IndexRepository {
    * @throws IOException 
    */
   void commit() throws IOException;
-  
+
+  Region<?, ?> getRegion();
+
   /**
    * Check to see if this repository is closed due to
    * underlying resources being closed or destroyed
    * @return true if this repository is closed.
    */
   public boolean isClosed();
+
+  /**
+   * For debugging purposes, return the underlying IndexWriter
+   */
+  IndexWriter getWriter();
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3f356594/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
index 3dbbc94..47e43b5 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImpl.java
@@ -135,6 +135,11 @@ public class IndexRepositoryImpl implements IndexRepository {
     return writer;
   }
 
+  @Override
+  public Region<?, ?> getRegion() {
+    return region;
+  }
+
   public LuceneSerializer getSerializer() {
     return serializer;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3f356594/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFilesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFilesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFilesIntegrationTest.java
new file mode 100644
index 0000000..9f02f2b
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFilesIntegrationTest.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding 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 the License.
+ */
+package com.gemstone.gemfire.cache.lucene.internal.directory;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+
+import com.gemstone.gemfire.cache.PartitionAttributes;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.cache.execute.ResultCollector;
+import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.cache.lucene.LuceneIntegrationTest;
+import com.gemstone.gemfire.cache.lucene.LuceneQuery;
+import com.gemstone.gemfire.cache.lucene.test.TestObject;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.test.junit.rules.DiskDirRule;
+
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.FSDirectory;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(IntegrationTest.class)
+public class DumpDirectoryFilesIntegrationTest extends LuceneIntegrationTest {
+  @Rule
+  public DiskDirRule diskDirRule = new DiskDirRule();
+
+  @Test
+  public void shouldDumpReadableLuceneIndexFile() throws Exception {
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+
+    Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
+    region.put(0, new TestObject("title 1", "hello world"));
+    region.put(1 * 113, new TestObject("title 2", "this will not match"));
+    region.put(2 * 113, new TestObject("title 3", "hello world"));
+    region.put(3 * 113, new TestObject("hello world", "hello world"));
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+
+    index.waitUntilFlushed(60000);
+
+    ResultCollector resultCollector= FunctionService
+      .onRegion(region)
+      .withArgs(new String[] {diskDirRule.get().getAbsolutePath(), INDEX_NAME})
+      .execute(new DumpDirectoryFiles().getId());
+    resultCollector.getResult();
+
+    //Find the directory for the first bucket
+    File bucket0 = diskDirRule.get().listFiles(file -> file.getName().endsWith("_0"))[0];
+
+    //Test that we can read the lucene index from the dump
+    final FSDirectory directory = FSDirectory.open(bucket0.toPath());
+    IndexReader reader = DirectoryReader.open(directory);
+    IndexSearcher searcher = new IndexSearcher(reader);
+    final TopDocs results = searcher.search(new MatchAllDocsQuery(), 1000);
+    assertEquals(4, results.totalHits);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3f356594/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFilesJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFilesJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFilesJUnitTest.java
new file mode 100644
index 0000000..f6f91bf
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/directory/DumpDirectoryFilesJUnitTest.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding 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 the License.
+ */
+package com.gemstone.gemfire.cache.lucene.internal.directory;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.util.Collections;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.execute.RegionFunctionContext;
+import com.gemstone.gemfire.cache.execute.ResultSender;
+import com.gemstone.gemfire.cache.lucene.LuceneService;
+import com.gemstone.gemfire.cache.lucene.internal.InternalLuceneIndex;
+import com.gemstone.gemfire.cache.lucene.internal.InternalLuceneService;
+import com.gemstone.gemfire.cache.lucene.internal.filesystem.FileSystem;
+import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
+import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.test.fake.Fakes;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import org.apache.lucene.index.IndexWriter;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+
+@Category(UnitTest.class)
+public class DumpDirectoryFilesJUnitTest {
+
+  private RegionFunctionContext context;
+  private String indexName = "index";
+  private String directoryName = "directory";
+  private String bucketName = "bucket";
+  private FileSystem fileSystem;
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  @Before
+  public void createMocks() throws BucketNotFoundException {
+
+    GemFireCacheImpl cache = Fakes.cache();
+    context = mock(RegionFunctionContext.class);
+    ResultSender sender = mock(ResultSender.class);
+    Region region = mock(Region.class);
+    InternalLuceneService service = mock(InternalLuceneService.class);
+    InternalLuceneIndex index = mock(InternalLuceneIndex.class);
+    RepositoryManager repoManager = mock(RepositoryManager.class);
+    IndexRepository repo = mock(IndexRepository.class);
+    IndexWriter writer = mock(IndexWriter.class);
+    RegionDirectory directory = mock(RegionDirectory.class);
+    fileSystem = mock(FileSystem.class);
+    Region bucket = mock(Region.class);
+    when(bucket.getFullPath()).thenReturn(bucketName);
+
+    when(context.getArguments()).thenReturn(new String[] {directoryName, indexName});
+    when(context.getResultSender()).thenReturn(sender);
+    when(context.getDataSet()).thenReturn(region);
+    when(region.getCache()).thenReturn(cache);
+    when(cache.getService(any())).thenReturn(service);
+    when(repoManager.getRepositories(eq(context))).thenReturn(Collections.singleton(repo));
+    when(index.getRepositoryManager()).thenReturn(repoManager);
+    when(index.getName()).thenReturn(indexName);
+    when(service.getIndex(eq(indexName), any())).thenReturn(index);
+    when(directory.getFileSystem()).thenReturn(fileSystem);
+    when(writer.getDirectory()).thenReturn(directory);
+    when(repo.getWriter()).thenReturn(writer);
+    when(repo.getRegion()).thenReturn(bucket);
+  }
+
+  @Test
+  public void shouldInvokeExportOnBuckets() throws BucketNotFoundException {
+    DumpDirectoryFiles dump = new DumpDirectoryFiles();
+    dump.execute(context);
+
+    File expectedDir = new File(directoryName, indexName + "_" + bucketName);
+    verify(fileSystem).export(eq(expectedDir));
+  }
+
+  @Test
+  public void shouldThrowIllegalStateWhenMissingIndex() throws BucketNotFoundException {
+    DumpDirectoryFiles dump = new DumpDirectoryFiles();
+    when(context.getArguments()).thenReturn(new String[] {"badDirectory", "badIndex"});
+    expectedException.expect(IllegalStateException.class);
+    dump.execute(context);
+  }
+
+  @Test
+  public void shouldThrowIllegalArgumentWhenGivenBadArguments() throws BucketNotFoundException {
+    DumpDirectoryFiles dump = new DumpDirectoryFiles();
+    when(context.getArguments()).thenReturn(new Object());
+    expectedException.expect(IllegalArgumentException.class);
+    dump.execute(context);
+  }
+
+  @Test
+  public void shouldThrowIllegalArgumentWhenMissingArgument() throws BucketNotFoundException {
+    DumpDirectoryFiles dump = new DumpDirectoryFiles();
+    when(context.getArguments()).thenReturn(new String[] {"not enough args"});
+    expectedException.expect(IllegalArgumentException.class);
+    dump.execute(context);
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3f356594/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
index 1d936ca..78bf1de 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/filesystem/FileSystemJUnitTest.java
@@ -20,18 +20,23 @@ package com.gemstone.gemfire.cache.lucene.internal.filesystem;
 
 import static org.junit.Assert.*;
 
+import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.lang.reflect.Method;
+import java.nio.file.Files;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Random;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
@@ -40,6 +45,7 @@ import org.mockito.stubbing.Answer;
 
 import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.rules.DiskDirRule;
 
 @Category(UnitTest.class)
 public class FileSystemJUnitTest {
@@ -52,6 +58,9 @@ public class FileSystemJUnitTest {
   private ConcurrentHashMap<String, File> fileRegion;
   private ConcurrentHashMap<ChunkKey, byte[]> chunkRegion;
 
+  @Rule
+  public DiskDirRule dirRule = new DiskDirRule();
+
   @Before
   public void setUp() {
     fileRegion = new ConcurrentHashMap<String, File>();
@@ -465,6 +474,29 @@ public class FileSystemJUnitTest {
     }
   }
 
+  @Test
+  public void testExport() throws IOException {
+    String name1 = "testFile1";
+    File file1= system.createFile(name1);
+    byte[] file1Data = writeRandomBytes(file1);
+
+    String name2 = "testFile2";
+    File file2= system.createFile(name2);
+    byte[] file2Data = writeRandomBytes(file2);
+
+    java.io.File parentDir = dirRule.get();
+    system.export(dirRule.get());
+    assertArrayEquals(new String[] {"testFile1", "testFile2"}, parentDir.list());
+
+    assertExportedFileContents(file1Data, new java.io.File(parentDir, "testFile1"));
+    assertExportedFileContents(file2Data, new java.io.File(parentDir, "testFile2"));
+  }
+
+  private void assertExportedFileContents(final byte[] expected, final java.io.File exportedFile) throws IOException {
+    byte[] actual = Files.readAllBytes(exportedFile.toPath());
+    assertArrayEquals(expected, actual);
+  }
+
   private void assertContents(byte[] data, File file) throws IOException {
     assertEquals(data.length, file.getLength());