You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2013/10/30 02:09:12 UTC

git commit: ACCUMULO-1766 store root tablet dir in zookeeper

Updated Branches:
  refs/heads/master 51fdb90ae -> 5f330f082


ACCUMULO-1766 store root tablet dir in zookeeper


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

Branch: refs/heads/master
Commit: 5f330f0829a893e14d604b1e0228eb90ef48e421
Parents: 51fdb90
Author: Keith Turner <kt...@apache.org>
Authored: Tue Oct 29 17:18:08 2013 -0400
Committer: Keith Turner <kt...@apache.org>
Committed: Tue Oct 29 20:58:12 2013 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/core/Constants.java     |  2 +-
 .../accumulo/core/metadata/RootTable.java       |  1 +
 .../apache/accumulo/server/ServerConstants.java | 11 ++--
 .../apache/accumulo/server/master/Master.java   | 55 ++++++++++++-----
 .../accumulo/server/tabletserver/Tablet.java    |  4 +-
 .../server/tabletserver/TabletServer.java       |  6 +-
 .../apache/accumulo/server/util/Initialize.java | 16 ++---
 .../accumulo/server/util/MetadataTableUtil.java | 18 ++++--
 .../server/util/OfflineMetadataScanner.java     |  3 +-
 test/system/upgrade_test.sh                     | 62 ++++++++++----------
 10 files changed, 109 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f330f08/core/src/main/java/org/apache/accumulo/core/Constants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java
index 3b4c1e6..9db0c40 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -36,7 +36,7 @@ public class Constants {
   public static final String ZTABLE_FLUSH_ID = "/flush-id";
   public static final String ZTABLE_COMPACT_ID = "/compact-id";
   public static final String ZTABLE_COMPACT_CANCEL_ID = "/compact-cancel-id";
-  
+
   public static final String ZMASTERS = "/masters";
   public static final String ZMASTER_LOCK = ZMASTERS + "/lock";
   public static final String ZMASTER_GOAL_STATE = ZMASTERS + "/goal_state";

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f330f08/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
index a47f183..c3ba6b2 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
@@ -40,6 +40,7 @@ public class RootTable {
   public static final String ZROOT_TABLET_FUTURE_LOCATION = ZROOT_TABLET + "/future_location";
   public static final String ZROOT_TABLET_LAST_LOCATION = ZROOT_TABLET + "/lastlocation";
   public static final String ZROOT_TABLET_WALOGS = ZROOT_TABLET + "/walogs";
+  public static final String ZROOT_TABLET_PATH = ZROOT_TABLET + "/dir";
   
   public static final KeyExtent EXTENT = new KeyExtent(new Text(ID), null, null);
   public static final KeyExtent OLD_EXTENT = new KeyExtent(new Text(MetadataTable.ID), KeyExtent.getMetadataEntry(new Text(MetadataTable.ID), null), null);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f330f08/server/src/main/java/org/apache/accumulo/server/ServerConstants.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/ServerConstants.java b/server/src/main/java/org/apache/accumulo/server/ServerConstants.java
index 55b8114..9e0ac39 100644
--- a/server/src/main/java/org/apache/accumulo/server/ServerConstants.java
+++ b/server/src/main/java/org/apache/accumulo/server/ServerConstants.java
@@ -55,6 +55,8 @@ public class ServerConstants {
           throw new RuntimeException(e);
         }
       } else {
+        if (!dfsUri.contains(":"))
+          throw new IllegalArgumentException("Expected fully qualified URI for " + Property.INSTANCE_DFS_URI.getKey() + " got " + dfsUri);
         baseDir = dfsUri + singleNamespace;
       }
       
@@ -75,6 +77,11 @@ public class ServerConstants {
         baseDirs = new String[] {getDefaultBaseDir()};
       } else {
         String namespaces[] = ns.split(",");
+        for (String namespace : namespaces) {
+          if (!namespace.contains(":")) {
+            throw new IllegalArgumentException("Expected fully qualified URI for " + Property.INSTANCE_VOLUMES.getKey() + " got " + namespace);
+          }
+        }
         baseDirs = prefix(namespaces, singleNamespace);
       }
     }
@@ -127,8 +134,4 @@ public class ServerConstants {
   public static String[] getMetadataTableDirs() {
     return prefix(getTablesDirs(), MetadataTable.ID);
   }
-  
-  public static String getRootTabletDir() {
-    return prefix(getRootTableDirs(), RootTable.ROOT_TABLET_LOCATION)[0];
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f330f08/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index 28da13d..6a962da 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -78,7 +78,6 @@ import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.SecurityUtil;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.ByteBufferUtil;
-import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
@@ -149,8 +148,6 @@ import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.trace.instrument.thrift.TraceWrap;
 import org.apache.accumulo.trace.thrift.TInfo;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -264,6 +261,42 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     }
   }
   
+  private void moveRootTabletToRootTable(IZooReaderWriter zoo) throws Exception {
+    String dirZPath = ZooUtil.getRoot(instance) + RootTable.ZROOT_TABLET_PATH;
+
+    if (!zoo.exists(dirZPath)) {
+      Path oldPath = fs.getFullPath(FileType.TABLE, "/!0/root_tablet");
+      if (fs.exists(oldPath)) {
+        String newPath = fs.choose(ServerConstants.getTablesDirs()) + "/" + RootTable.ID;
+        fs.mkdirs(new Path(newPath));
+        if (!fs.rename(oldPath, new Path(newPath))) {
+          throw new IOException("Failed to move root tablet from " + oldPath + " to " + newPath);
+        }
+
+        log.info("Upgrade renamed " + oldPath + " to " + newPath);
+      }
+
+      Path location = null;
+
+      for (String basePath : ServerConstants.getTablesDirs()) {
+        Path path = new Path(basePath + "/" + RootTable.ID + RootTable.ROOT_TABLET_LOCATION);
+        if (fs.exists(path)) {
+          if (location != null) {
+            throw new IllegalStateException("Root table at multiple locations " + location + " " + path);
+          }
+
+          location = path;
+        }
+      }
+
+      if (location == null)
+        throw new IllegalStateException("Failed to find root tablet");
+
+      log.info("Upgrade setting root table location in zookeeper " + location);
+      zoo.putPersistentData(dirZPath, location.toString().getBytes(), NodeExistsPolicy.FAIL);
+    }
+  }
+
   private void upgradeZookeeper() {
     if (Accumulo.getAccumuloPersistentVersion(fs) == ServerConstants.PREV_DATA_VERSION) {
       try {
@@ -272,10 +305,12 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
         IZooReaderWriter zoo = ZooReaderWriter.getInstance();
         
         if (!Tables.exists(instance, RootTable.ID)) {
-          TableManager.prepareNewTableState(instance.getInstanceID(), RootTable.ID, RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.FAIL);
+          TableManager.prepareNewTableState(instance.getInstanceID(), RootTable.ID, RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.SKIP);
           Initialize.initMetadataConfig(RootTable.ID);
         }
         
+        moveRootTabletToRootTable(zoo);
+
         zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/loggers", NodeMissingPolicy.SKIP);
         zoo.recursiveDelete(ZooUtil.getRoot(instance) + "/dead/loggers", NodeMissingPolicy.SKIP);
         zoo.putPersistentData(ZooUtil.getRoot(instance) + Constants.ZRECOVERY, new byte[] {'0'}, NodeExistsPolicy.SKIP);
@@ -303,18 +338,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           @Override
           public void run() {
             try {
-              Path oldDir = fs.getFullPath(FileType.TABLE, ServerConstants.getDefaultBaseDir() + "/tables/!0/root_tablet");
-              for (FileStatus file : fs.listStatus(oldDir)) {
-                if (fs.isFile(file.getPath())) {
-                  Path newFile = new Path(ServerConstants.getRootTabletDir(), file.getPath().getName());
-                  FileUtil.copy(
-                      fs.getFileSystemByPath(file.getPath()), file.getPath(),
-                      fs.getFileSystemByPath(newFile), newFile,
-                      false,
-                      true,
-                      CachedConfiguration.getInstance());
-                }
-              }
               MetadataTableUtil.moveMetaDeleteMarkers(instance, SystemCredentials.get());
               Accumulo.updateAccumuloVersion(fs);
               

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f330f08/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
index 7719bac..1be1c74 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
@@ -1123,9 +1123,7 @@ public class Tablet {
     TreeMap<FileRef,DataFileValue> datafiles = new TreeMap<FileRef,DataFileValue>();
     
     if (extent.isRootTablet()) { // the meta0 tablet
-      Path location = new Path(ServerConstants.getRootTabletDir());
-      FileSystem defaultVolume = fs.getDefaultVolume();
-      location = location.makeQualified(defaultVolume.getUri(), defaultVolume.getWorkingDirectory());
+      Path location = new Path(MetadataTableUtil.getRootTabletDir());
       // cleanUpFiles() has special handling for delete. files
       FileStatus[] files = fs.listStatus(location);
       Collection<String> goodPaths = cleanUpFiles(fs, files, true);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f330f08/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
index b58df33..f3680a6 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
@@ -3275,7 +3275,11 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       throw new AccumuloException("Root tablet already has a location set");
     }
     
-    return new Pair<Text,KeyExtent>(new Text(RootTable.ROOT_TABLET_LOCATION), null);
+    try {
+      return new Pair<Text,KeyExtent>(new Text(MetadataTableUtil.getRootTabletDir()), null);
+    } catch (IOException e) {
+      throw new AccumuloException(e);
+    }
   }
   
   public static Pair<Text,KeyExtent> verifyTabletInformation(KeyExtent extent, TServerInstance instance, SortedMap<Key,Value> tabletsKeyValues,

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f330f08/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/util/Initialize.java b/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
index 7c93d5c..db84cee 100644
--- a/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
+++ b/server/src/main/java/org/apache/accumulo/server/util/Initialize.java
@@ -170,15 +170,17 @@ public class Initialize {
   public static boolean initialize(Opts opts, String instanceNamePath, VolumeManager fs) {
     
     UUID uuid = UUID.randomUUID();
+    // the actual disk locations of the root table and tablets
+    final Path rootTablet = new Path(fs.choose(ServerConstants.getTablesDirs()) + "/" + RootTable.ID + RootTable.ROOT_TABLET_LOCATION);
     try {
-      initZooKeeper(opts, uuid.toString(), instanceNamePath);
+      initZooKeeper(opts, uuid.toString(), instanceNamePath, rootTablet);
     } catch (Exception e) {
       log.fatal("Failed to initialize zookeeper", e);
       return false;
     }
     
     try {
-      initFileSystem(opts, fs, uuid);
+      initFileSystem(opts, fs, uuid, rootTablet);
     } catch (Exception e) {
       log.fatal("Failed to initialize filesystem", e);
       return false;
@@ -214,12 +216,9 @@ public class Initialize {
   
   //TODO Remove deprecation warning suppression when Hadoop1 support is dropped
   @SuppressWarnings("deprecation")
-  private static void initFileSystem(Opts opts, VolumeManager fs, UUID uuid) throws IOException {
+  private static void initFileSystem(Opts opts, VolumeManager fs, UUID uuid, Path rootTablet) throws IOException {
     FileStatus fstat;
-    
-    // the actual disk locations of the root table and tablets
-    final Path rootTablet = new Path(ServerConstants.getRootTabletDir());
-    
+
     // the actual disk locations of the metadata table and tablets
     final Path[] metadataTableDirs = paths(ServerConstants.getMetadataTableDirs());
     
@@ -343,7 +342,7 @@ public class Initialize {
     }
   }
   
-  private static void initZooKeeper(Opts opts, String uuid, String instanceNamePath) throws KeeperException, InterruptedException {
+  private static void initZooKeeper(Opts opts, String uuid, String instanceNamePath, Path rootTablet) throws KeeperException, InterruptedException {
     // setup basic data in zookeeper
     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     ZooUtil.putPersistentData(zoo.getZooKeeper(), Constants.ZROOT, new byte[0], -1, NodeExistsPolicy.SKIP, Ids.OPEN_ACL_UNSAFE);
@@ -364,6 +363,7 @@ public class Initialize {
     zoo.putPersistentData(zkInstanceRoot + Constants.ZPROBLEMS, new byte[0], NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET, new byte[0], NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET_WALOGS, new byte[0], NodeExistsPolicy.FAIL);
+    zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET_PATH, rootTablet.toString().getBytes(), NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTRACERS, new byte[0], NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZMASTERS, new byte[0], NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZMASTER_LOCK, new byte[0], NodeExistsPolicy.FAIL);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f330f08/server/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java b/server/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index ab8ffce..8a8b1d5 100644
--- a/server/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@ -33,6 +33,7 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -85,7 +86,6 @@ import org.apache.accumulo.server.security.SystemCredentials;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -800,6 +800,18 @@ public class MetadataTableUtil {
     return e;
   }
   
+  public static String getRootTabletDir() throws IOException {
+    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_PATH;
+    try {
+      return new String(zoo.getData(zpath, null), Constants.UTF8);
+    } catch (KeeperException e) {
+      throw new IOException(e);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+
   public static Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>> getFileAndLogEntries(Credentials credentials, KeyExtent extent) throws KeeperException,
       InterruptedException, IOException {
     ArrayList<LogEntry> result = new ArrayList<LogEntry>();
@@ -808,9 +820,7 @@ public class MetadataTableUtil {
     VolumeManager fs = VolumeManagerImpl.get();
     if (extent.isRootTablet()) {
       getRootLogEntries(result);
-      Path rootDir = new Path(ServerConstants.getRootTabletDir());
-      FileSystem defaultVolume = fs.getDefaultVolume();
-      rootDir = rootDir.makeQualified(defaultVolume.getUri(), defaultVolume.getWorkingDirectory());
+      Path rootDir = new Path(getRootTabletDir());
       FileStatus[] files = fs.listStatus(rootDir);
       for (FileStatus fileStatus : files) {
         if (fileStatus.getPath().toString().endsWith("_tmp")) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f330f08/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java b/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java
index 3fee062..cefec22 100644
--- a/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java
+++ b/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java
@@ -51,7 +51,6 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Lo
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.core.util.TextUtil;
-import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -138,7 +137,7 @@ public class OfflineMetadataScanner extends ScannerOptions implements Scanner {
       throw new RuntimeException("Root tablet has write ahead logs, can not scan offline");
     }
     
-    FileStatus[] rootFiles = fs.listStatus(new Path(ServerConstants.getRootTabletDir()));
+    FileStatus[] rootFiles = fs.listStatus(new Path(MetadataTableUtil.getRootTabletDir()));
     
     for (FileStatus rootFile : rootFiles) {
       allFiles.add(rootFile.getPath().toString());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5f330f08/test/system/upgrade_test.sh
----------------------------------------------------------------------
diff --git a/test/system/upgrade_test.sh b/test/system/upgrade_test.sh
index c5dee53..6259e1c 100755
--- a/test/system/upgrade_test.sh
+++ b/test/system/upgrade_test.sh
@@ -22,53 +22,55 @@ then
   exit -1
 fi
 
-#this script test upgrade from 1.4 to 1.5.  This script is not self verifying, its output must be inspected for correctness.
+#this script test upgrade.   This script is not self verifying, its output must be inspected for correctness.
 
-#set the following to point to configured 1.4 and 1.5 accumulo dirs.  Ensure both point to the same walogs
+#set DIR  to point to configured accumulo dirs. 
 
 #TODO could support multinode configs, this script assumes single node config
 
-ONE_FOUR_DIR=../../../accumulo-1.4
-ONE_FIVE_DIR=../../
+PREV=../../../../accumulo-1.5.0
+CURR=../../
+DIR=/accumulo
+BULK=/tmp/upt
 
 pkill -f accumulo.start
-hadoop fs -rmr /accumulo-1.4
-hadoop fs -rmr /testmf
-hadoop fs -rmr /testmfFail
-
-echo -e "uptest\nsecret\nsecret" | $ONE_FOUR_DIR/bin/accumulo init --clear-instance-name
-$ONE_FOUR_DIR/bin/start-all.sh
-$ONE_FOUR_DIR/bin/accumulo 'org.apache.accumulo.server.test.TestIngest$CreateTable' 0 200000 10 root secret
-$ONE_FOUR_DIR/bin/accumulo org.apache.accumulo.server.test.TestIngest -timestamp 1 -size 50 -random 56 100000 0 1
-$ONE_FOUR_DIR/bin/accumulo org.apache.accumulo.server.test.TestIngest -rfile /testmf/mf01 -timestamp 1 -size 50 -random 56 100000 100000 1
-$ONE_FOUR_DIR/bin/accumulo org.apache.accumulo.server.test.BulkImportDirectory root secret test_ingest /testmf /testmfFail
+hadoop fs -rmr $DIR
+hadoop fs -rmr $BULK
+hadoop fs -mkdir $BULK/fail
+
+$PREV/bin/accumulo init --clear-instance-name --instance-name testUp --password secret
+$PREV/bin/start-all.sh
+
+$PREV/bin/accumulo org.apache.accumulo.test.TestIngest -u root -p secret --timestamp 1 --size 50 --random 56 --rows 200000 --start 0 --cols 1  --createTable --splits 10
+$PREV/bin/accumulo org.apache.accumulo.test.TestIngest --rfile $BULK/bulk/test --timestamp 1 --size 50 --random 56 --rows 200000 --start 200000 --cols 1
+
+echo -e "table test_ingest\nimportdirectory $BULK/bulk $BULK/fail false" | $PREV/bin/accumulo shell -u root -p secret
 if [ $1 == "dirty" ]; then
 	pkill -9 -f accumulo.start
 else 
-	$ONE_FOUR_DIR/bin/stop-all.sh
+	$PREV/bin/stop-all.sh
 fi
 
-echo "==== Starting 1.5 ==="
+echo "==== Starting Current ==="
 
+$CURR/bin/start-all.sh
+$CURR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 1 --random 56 --rows 400000 --start 0 --cols 1 -u root -p secret
+echo "compact -t test_ingest -w" | $CURR/bin/accumulo shell -u root -p secret
+$CURR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 1 --random 56 --rows 400000 --start 0 --cols 1 -u root -p secret
 
-$ONE_FIVE_DIR/bin/start-all.sh
-$ONE_FIVE_DIR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 1 --random 56 --rows 200000 --start 0 --cols 1 -u root -p secret
-echo "compact -t test_ingest -w" | $ONE_FIVE_DIR/bin/accumulo shell -u root -p secret
-$ONE_FIVE_DIR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 1 --random 56 --rows 200000 --start 0 --cols 1 -u root -p secret
 
-#test overwriting data writting in 1.4
-$ONE_FIVE_DIR/bin/accumulo org.apache.accumulo.test.TestIngest --timestamp 2 --size 50 --random 57 --rows 300000 --start 0 --cols 1 -cv "L1&L2&G1&GROUP2" -u root -p secret
-$ONE_FIVE_DIR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 300000 --start 0 --cols 1 -u root -p secret
-echo "compact -t test_ingest -w" | $ONE_FIVE_DIR/bin/accumulo shell -u root -p secret
-$ONE_FIVE_DIR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 300000 --start 0 --cols 1 -u root -p secret
+$CURR/bin/accumulo org.apache.accumulo.test.TestIngest --timestamp 2 --size 50 --random 57 --rows 500000 --start 0 --cols 1 -u root -p secret
+$CURR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 500000 --start 0 --cols 1 -u root -p secret
+echo "compact -t test_ingest -w" | $CURR/bin/accumulo shell -u root -p secret
+$CURR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 500000 --start 0 --cols 1 -u root -p secret
 
-$ONE_FIVE_DIR/bin/stop-all.sh
-$ONE_FIVE_DIR/bin/start-all.sh
+$CURR/bin/stop-all.sh
+$CURR/bin/start-all.sh
 
-$ONE_FIVE_DIR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 300000 --start 0 --cols 1 -u root -p secret
+$CURR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 500000 --start 0 --cols 1 -u root -p secret
 
 pkill -9 -f accumulo.start
-$ONE_FIVE_DIR/bin/start-all.sh
+$CURR/bin/start-all.sh
 
-$ONE_FIVE_DIR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 300000 --start 0 --cols 1 -u root -p secret
+$CURR/bin/accumulo  org.apache.accumulo.test.VerifyIngest --size 50 --timestamp 2 --random 57 --rows 500000 --start 0 --cols 1 -u root -p secret