You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/10/10 01:17:15 UTC

[1/3] git commit: ACCUMULO-3215 Ensure that ImportTable writes absolute paths, not relative.

Repository: accumulo
Updated Branches:
  refs/heads/1.6 3d7d762ac -> ff1ceeaf6
  refs/heads/master 837ce17c8 -> 556b17c63


ACCUMULO-3215 Ensure that ImportTable writes absolute paths, not relative.

ImportExportIT creates a table, exports it and then imports it to the
same instance to ensure that the imported copy is equivalent
to the original. Without the changes to ImportTable, the test failed
because the copy was unusable due to incorrect metadata table entries.


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

Branch: refs/heads/1.6
Commit: ff1ceeaf678e465a56d987755a6af971b2b9e3ce
Parents: 3d7d762
Author: Josh Elser <el...@apache.org>
Authored: Thu Oct 9 16:27:14 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu Oct 9 18:49:56 2014 -0400

----------------------------------------------------------------------
 .../accumulo/master/tableOps/ImportTable.java   |  23 ++-
 .../apache/accumulo/test/ImportExportIT.java    | 163 +++++++++++++++++++
 2 files changed, 180 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff1ceeaf/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
index 031ba9c..e57c689 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.io.Serializable;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -34,8 +35,8 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.impl.TableOperationsImpl;
 import org.apache.accumulo.core.client.impl.Namespaces;
+import org.apache.accumulo.core.client.impl.TableOperationsImpl;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
@@ -72,7 +73,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
 /**
- * 
+ *
  */
 class ImportedTableInfo implements Serializable {
 
@@ -218,7 +219,10 @@ class PopulateMetadataTable extends MasterRepo {
 
       Map<String,String> fileNameMappings = readMappingFile(fs, tableInfo);
 
-      String bulkDir = new Path(tableInfo.importDir).getName();
+      log.info("importDir is " + tableInfo.importDir);
+
+      // This is a directory already prefixed with proper volume information e.g. hdfs://localhost:8020/path/to/accumulo/tables/...
+      String bulkDir = tableInfo.importDir;
 
       ZipEntry zipEntry;
       while ((zipEntry = zis.getNextEntry()) != null) {
@@ -250,7 +254,7 @@ class PopulateMetadataTable extends MasterRepo {
                     "File " + oldName + " does not exist in import dir");
               }
 
-              cq = new Text("/" + bulkDir + "/" + newName);
+              cq = new Text(bulkDir + "/" + newName);
             } else {
               cq = key.getColumnQualifier();
             }
@@ -382,7 +386,7 @@ class MapImportFileNames extends MasterRepo {
 }
 
 class CreateImportDir extends MasterRepo {
-
+  private static final Logger log = Logger.getLogger(CreateImportDir.class);
   private static final long serialVersionUID = 1L;
 
   private ImportedTableInfo tableInfo;
@@ -396,13 +400,20 @@ class CreateImportDir extends MasterRepo {
 
     UniqueNameAllocator namer = UniqueNameAllocator.getInstance();
 
-    Path base = master.getFileSystem().matchingFileSystem(new Path(tableInfo.exportDir), ServerConstants.getTablesDirs());
+    Path exportDir = new Path(tableInfo.exportDir);
+    String[] tableDirs = ServerConstants.getTablesDirs();
+
+    log.info("Looking for matching filesystem for " + exportDir + " from options " + Arrays.toString(tableDirs));
+    Path base = master.getFileSystem().matchingFileSystem(exportDir, tableDirs);
+    log.info("Chose base table directory of " + base);
     Path directory = new Path(base, tableInfo.tableId);
 
     Path newBulkDir = new Path(directory, Constants.BULK_PREFIX + namer.getNextName());
 
     tableInfo.importDir = newBulkDir.toString();
 
+    log.info("Using import dir: " + tableInfo.importDir);
+
     return new MapImportFileNames(tableInfo);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff1ceeaf/test/src/test/java/org/apache/accumulo/test/ImportExportIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ImportExportIT.java b/test/src/test/java/org/apache/accumulo/test/ImportExportIT.java
new file mode 100644
index 0000000..d2de346
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/ImportExportIT.java
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.accumulo.test;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStreamReader;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.test.functional.SimpleMacIT;
+import org.apache.commons.io.FileUtils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Iterables;
+import com.google.common.io.Files;
+
+/**
+ * ImportTable didn't correctly place absolute paths in metadata. This resulted in the imported table only being usable when the actual HDFS directory for
+ * Accumulo was the same as {@link Property#INSTANCE_DFS_DIR}. If any other HDFS directory was used, any interactions with the table would fail because the
+ * relative path in the metadata table (created by the ImportTable process) would be converted to a non-existent absolute path.
+ * <p>
+ * ACCUMULO-3215
+ *
+ */
+public class ImportExportIT extends SimpleMacIT {
+
+  private static final Logger log = LoggerFactory.getLogger(ImportExportIT.class);
+
+  @Test
+  public void test() throws Exception {
+    Connector conn = getConnector();
+
+    String[] tableNames = getUniqueNames(2);
+    String srcTable = tableNames[0], destTable = tableNames[1];
+    conn.tableOperations().create(srcTable);
+
+    BatchWriter bw = conn.createBatchWriter(srcTable, new BatchWriterConfig());
+    for (int row = 0; row < 1000; row++) {
+      Mutation m = new Mutation(Integer.toString(row));
+      for (int col = 0; col < 100; col++) {
+        m.put(Integer.toString(col), "", Integer.toString(col * 2));
+      }
+      bw.addMutation(m);
+    }
+
+    bw.close();
+
+    conn.tableOperations().compact(srcTable, null, null, true, true);
+
+    // Make a directory we can use to throw the export and import directories
+    File baseDir = new File(System.getProperty("user.dir") + "/target/mini-tests", getClass().getName());
+    baseDir.mkdirs();
+    File exportDir = new File(baseDir, "export");
+    FileUtils.deleteQuietly(exportDir);
+    exportDir.mkdir();
+    File importDir = new File(baseDir, "import");
+    FileUtils.deleteQuietly(importDir);
+    importDir.mkdir();
+
+    log.info("Exporting table to {}", exportDir.getAbsolutePath());
+    log.info("Importing table from {}", importDir.getAbsolutePath());
+
+    // Offline the table
+    conn.tableOperations().offline(srcTable, true);
+    // Then export it
+    conn.tableOperations().exportTable(srcTable, exportDir.getAbsolutePath());
+
+    // Make sure the distcp.txt file that exporttable creates is available
+    File distcp = new File(exportDir, "distcp.txt");
+    Assert.assertTrue("Distcp file doesn't exist", distcp.exists());
+    FileInputStream fis = new FileInputStream(distcp);
+    BufferedReader reader = new BufferedReader(new InputStreamReader(fis));
+
+    // Copy each file that was exported to the import directory
+    String line;
+    while (null != (line = reader.readLine())) {
+      File f = new File(line.substring(5));
+      Assert.assertTrue("File doesn't exist: " + f, f.exists());
+      File dest = new File(importDir, f.getName());
+      Assert.assertFalse("Did not expect " + dest + " to exist", dest.exists());
+      Files.copy(f, dest);
+    }
+
+    reader.close();
+
+    log.info("Import dir: {}", Arrays.toString(importDir.list()));
+
+    // Import the exported data into a new table
+    conn.tableOperations().importTable(destTable, importDir.getAbsolutePath());
+
+    // Get the table ID for the table that the importtable command created
+    final String tableId = conn.tableOperations().tableIdMap().get(destTable);
+    Assert.assertNotNull(tableId);
+
+    // Get all `file` colfams from the metadata table for the new table
+    log.info("Imported into table with ID: {}", tableId);
+    Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    s.setRange(MetadataSchema.TabletsSection.getRange(tableId));
+    s.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME);
+
+    // Should find a single entry
+    Entry<Key,Value> fileEntry = Iterables.getOnlyElement(s);
+
+    // The file should be an absolute URI (file:///...), not a relative path (/b-000.../I000001.rf)
+    String fileUri = fileEntry.getKey().getColumnQualifier().toString();
+    Assert.assertFalse("Imported files should have absolute URIs, not relative: " + fileUri, looksLikeRelativePath(fileUri));
+
+    // Online the original table before we verify equivalence
+    conn.tableOperations().online(srcTable, true);
+
+    Iterator<Entry<Key,Value>> src = conn.createScanner(srcTable, Authorizations.EMPTY).iterator(), dest = conn.createScanner(destTable, Authorizations.EMPTY)
+        .iterator();
+    while (src.hasNext() && dest.hasNext()) {
+      Entry<Key,Value> orig = src.next(), copy = dest.next();
+      Assert.assertEquals(orig.getKey(), copy.getKey());
+      Assert.assertEquals(orig.getValue(), copy.getValue());
+    }
+    Assert.assertFalse("Source table had more data to read", src.hasNext());
+    Assert.assertFalse("Dest table had more data to read", dest.hasNext());
+  }
+
+  private boolean looksLikeRelativePath(String uri) {
+    if (uri.startsWith("/" + Constants.BULK_PREFIX)) {
+      if ('/' == uri.charAt(10)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+}


[3/3] git commit: Merge branch '1.6'

Posted by el...@apache.org.
Merge branch '1.6'

Conflicts:
	server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java


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

Branch: refs/heads/master
Commit: 556b17c635248e6d7c871761d281893228515f35
Parents: 837ce17 ff1ceea
Author: Josh Elser <el...@apache.org>
Authored: Thu Oct 9 19:17:02 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu Oct 9 19:17:02 2014 -0400

----------------------------------------------------------------------
 .../accumulo/master/tableOps/ImportTable.java   |  23 ++-
 .../apache/accumulo/test/ImportExportIT.java    | 163 +++++++++++++++++++
 2 files changed, 180 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/556b17c6/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
----------------------------------------------------------------------
diff --cc server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
index 9857137,e57c689..1129ede
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
@@@ -24,7 -24,7 +24,8 @@@ import java.io.IOException
  import java.io.InputStreamReader;
  import java.io.OutputStreamWriter;
  import java.io.Serializable;
 +import java.nio.charset.StandardCharsets;
+ import java.util.Arrays;
  import java.util.HashMap;
  import java.util.Map;
  import java.util.Map.Entry;


[2/3] git commit: ACCUMULO-3215 Ensure that ImportTable writes absolute paths, not relative.

Posted by el...@apache.org.
ACCUMULO-3215 Ensure that ImportTable writes absolute paths, not relative.

ImportExportIT creates a table, exports it and then imports it to the
same instance to ensure that the imported copy is equivalent
to the original. Without the changes to ImportTable, the test failed
because the copy was unusable due to incorrect metadata table entries.


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

Branch: refs/heads/master
Commit: ff1ceeaf678e465a56d987755a6af971b2b9e3ce
Parents: 3d7d762
Author: Josh Elser <el...@apache.org>
Authored: Thu Oct 9 16:27:14 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu Oct 9 18:49:56 2014 -0400

----------------------------------------------------------------------
 .../accumulo/master/tableOps/ImportTable.java   |  23 ++-
 .../apache/accumulo/test/ImportExportIT.java    | 163 +++++++++++++++++++
 2 files changed, 180 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff1ceeaf/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
index 031ba9c..e57c689 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.io.Serializable;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -34,8 +35,8 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.impl.TableOperationsImpl;
 import org.apache.accumulo.core.client.impl.Namespaces;
+import org.apache.accumulo.core.client.impl.TableOperationsImpl;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
@@ -72,7 +73,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
 /**
- * 
+ *
  */
 class ImportedTableInfo implements Serializable {
 
@@ -218,7 +219,10 @@ class PopulateMetadataTable extends MasterRepo {
 
       Map<String,String> fileNameMappings = readMappingFile(fs, tableInfo);
 
-      String bulkDir = new Path(tableInfo.importDir).getName();
+      log.info("importDir is " + tableInfo.importDir);
+
+      // This is a directory already prefixed with proper volume information e.g. hdfs://localhost:8020/path/to/accumulo/tables/...
+      String bulkDir = tableInfo.importDir;
 
       ZipEntry zipEntry;
       while ((zipEntry = zis.getNextEntry()) != null) {
@@ -250,7 +254,7 @@ class PopulateMetadataTable extends MasterRepo {
                     "File " + oldName + " does not exist in import dir");
               }
 
-              cq = new Text("/" + bulkDir + "/" + newName);
+              cq = new Text(bulkDir + "/" + newName);
             } else {
               cq = key.getColumnQualifier();
             }
@@ -382,7 +386,7 @@ class MapImportFileNames extends MasterRepo {
 }
 
 class CreateImportDir extends MasterRepo {
-
+  private static final Logger log = Logger.getLogger(CreateImportDir.class);
   private static final long serialVersionUID = 1L;
 
   private ImportedTableInfo tableInfo;
@@ -396,13 +400,20 @@ class CreateImportDir extends MasterRepo {
 
     UniqueNameAllocator namer = UniqueNameAllocator.getInstance();
 
-    Path base = master.getFileSystem().matchingFileSystem(new Path(tableInfo.exportDir), ServerConstants.getTablesDirs());
+    Path exportDir = new Path(tableInfo.exportDir);
+    String[] tableDirs = ServerConstants.getTablesDirs();
+
+    log.info("Looking for matching filesystem for " + exportDir + " from options " + Arrays.toString(tableDirs));
+    Path base = master.getFileSystem().matchingFileSystem(exportDir, tableDirs);
+    log.info("Chose base table directory of " + base);
     Path directory = new Path(base, tableInfo.tableId);
 
     Path newBulkDir = new Path(directory, Constants.BULK_PREFIX + namer.getNextName());
 
     tableInfo.importDir = newBulkDir.toString();
 
+    log.info("Using import dir: " + tableInfo.importDir);
+
     return new MapImportFileNames(tableInfo);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff1ceeaf/test/src/test/java/org/apache/accumulo/test/ImportExportIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ImportExportIT.java b/test/src/test/java/org/apache/accumulo/test/ImportExportIT.java
new file mode 100644
index 0000000..d2de346
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/ImportExportIT.java
@@ -0,0 +1,163 @@
+/*
+ * 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 org.apache.accumulo.test;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStreamReader;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.test.functional.SimpleMacIT;
+import org.apache.commons.io.FileUtils;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Iterables;
+import com.google.common.io.Files;
+
+/**
+ * ImportTable didn't correctly place absolute paths in metadata. This resulted in the imported table only being usable when the actual HDFS directory for
+ * Accumulo was the same as {@link Property#INSTANCE_DFS_DIR}. If any other HDFS directory was used, any interactions with the table would fail because the
+ * relative path in the metadata table (created by the ImportTable process) would be converted to a non-existent absolute path.
+ * <p>
+ * ACCUMULO-3215
+ *
+ */
+public class ImportExportIT extends SimpleMacIT {
+
+  private static final Logger log = LoggerFactory.getLogger(ImportExportIT.class);
+
+  @Test
+  public void test() throws Exception {
+    Connector conn = getConnector();
+
+    String[] tableNames = getUniqueNames(2);
+    String srcTable = tableNames[0], destTable = tableNames[1];
+    conn.tableOperations().create(srcTable);
+
+    BatchWriter bw = conn.createBatchWriter(srcTable, new BatchWriterConfig());
+    for (int row = 0; row < 1000; row++) {
+      Mutation m = new Mutation(Integer.toString(row));
+      for (int col = 0; col < 100; col++) {
+        m.put(Integer.toString(col), "", Integer.toString(col * 2));
+      }
+      bw.addMutation(m);
+    }
+
+    bw.close();
+
+    conn.tableOperations().compact(srcTable, null, null, true, true);
+
+    // Make a directory we can use to throw the export and import directories
+    File baseDir = new File(System.getProperty("user.dir") + "/target/mini-tests", getClass().getName());
+    baseDir.mkdirs();
+    File exportDir = new File(baseDir, "export");
+    FileUtils.deleteQuietly(exportDir);
+    exportDir.mkdir();
+    File importDir = new File(baseDir, "import");
+    FileUtils.deleteQuietly(importDir);
+    importDir.mkdir();
+
+    log.info("Exporting table to {}", exportDir.getAbsolutePath());
+    log.info("Importing table from {}", importDir.getAbsolutePath());
+
+    // Offline the table
+    conn.tableOperations().offline(srcTable, true);
+    // Then export it
+    conn.tableOperations().exportTable(srcTable, exportDir.getAbsolutePath());
+
+    // Make sure the distcp.txt file that exporttable creates is available
+    File distcp = new File(exportDir, "distcp.txt");
+    Assert.assertTrue("Distcp file doesn't exist", distcp.exists());
+    FileInputStream fis = new FileInputStream(distcp);
+    BufferedReader reader = new BufferedReader(new InputStreamReader(fis));
+
+    // Copy each file that was exported to the import directory
+    String line;
+    while (null != (line = reader.readLine())) {
+      File f = new File(line.substring(5));
+      Assert.assertTrue("File doesn't exist: " + f, f.exists());
+      File dest = new File(importDir, f.getName());
+      Assert.assertFalse("Did not expect " + dest + " to exist", dest.exists());
+      Files.copy(f, dest);
+    }
+
+    reader.close();
+
+    log.info("Import dir: {}", Arrays.toString(importDir.list()));
+
+    // Import the exported data into a new table
+    conn.tableOperations().importTable(destTable, importDir.getAbsolutePath());
+
+    // Get the table ID for the table that the importtable command created
+    final String tableId = conn.tableOperations().tableIdMap().get(destTable);
+    Assert.assertNotNull(tableId);
+
+    // Get all `file` colfams from the metadata table for the new table
+    log.info("Imported into table with ID: {}", tableId);
+    Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    s.setRange(MetadataSchema.TabletsSection.getRange(tableId));
+    s.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME);
+
+    // Should find a single entry
+    Entry<Key,Value> fileEntry = Iterables.getOnlyElement(s);
+
+    // The file should be an absolute URI (file:///...), not a relative path (/b-000.../I000001.rf)
+    String fileUri = fileEntry.getKey().getColumnQualifier().toString();
+    Assert.assertFalse("Imported files should have absolute URIs, not relative: " + fileUri, looksLikeRelativePath(fileUri));
+
+    // Online the original table before we verify equivalence
+    conn.tableOperations().online(srcTable, true);
+
+    Iterator<Entry<Key,Value>> src = conn.createScanner(srcTable, Authorizations.EMPTY).iterator(), dest = conn.createScanner(destTable, Authorizations.EMPTY)
+        .iterator();
+    while (src.hasNext() && dest.hasNext()) {
+      Entry<Key,Value> orig = src.next(), copy = dest.next();
+      Assert.assertEquals(orig.getKey(), copy.getKey());
+      Assert.assertEquals(orig.getValue(), copy.getValue());
+    }
+    Assert.assertFalse("Source table had more data to read", src.hasNext());
+    Assert.assertFalse("Dest table had more data to read", dest.hasNext());
+  }
+
+  private boolean looksLikeRelativePath(String uri) {
+    if (uri.startsWith("/" + Constants.BULK_PREFIX)) {
+      if ('/' == uri.charAt(10)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+}