You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2013/12/19 21:51:19 UTC

git commit: Revert "ACCUMULO-2057 track lastLocation through a split"

Updated Branches:
  refs/heads/1.6.0-SNAPSHOT 9cae04863 -> e11b3338a


Revert "ACCUMULO-2057 track lastLocation through a split"

checked in changes for other tickets... need to redo

This reverts commit 9cae04863ae46bcd47b09ec883d39dc676574a53.


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: e11b3338a8719bbc702337038ca6188e9ee9d224
Parents: 9cae048
Author: Eric Newton <er...@gmail.com>
Authored: Thu Dec 19 15:50:32 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Dec 19 15:50:32 2013 -0500

----------------------------------------------------------------------
 .../client/mapreduce/AbstractInputFormat.java   | 254 +++++++++----------
 .../client/mapreduce/AccumuloOutputFormat.java  |   5 +-
 .../mapreduce/lib/util/InputConfigurator.java   |  12 +-
 .../server/master/state/TServerInstance.java    |  16 +-
 .../org/apache/accumulo/tserver/Tablet.java     |  17 +-
 .../apache/accumulo/tserver/TabletServer.java   |   6 +-
 .../test/randomwalk/concurrent/CreateTable.java |   3 -
 7 files changed, 141 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e11b3338/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index 9f30563..2b7e958 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -525,148 +525,144 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
     Level logLevel = getLogLevel(context);
     log.setLevel(logLevel);
     validateOptions(context);
-    
+
     LinkedList<InputSplit> splits = new LinkedList<InputSplit>();
     Map<String,InputTableConfig> tableConfigs = getInputTableConfigs(context);
-    Instance instance = getInstance(context);
-    try {
-      for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
-        
-        String tableName = tableConfigEntry.getKey();
-        InputTableConfig tableConfig = tableConfigEntry.getValue();
-        
-        boolean mockInstance;
-        String tableId;
-        // resolve table name to id once, and use id from this point forward
-        if (instance instanceof MockInstance) {
-          tableId = "";
-          mockInstance = true;
-        } else {
-          try {
-            tableId = Tables.getTableId(instance, tableName);
-          } catch (TableNotFoundException e) {
-            throw new IOException(e);
-          }
-          mockInstance = false;
-        }
-        
-        Authorizations auths = getScanAuthorizations(context);
-        String principal = getPrincipal(context);
-        AuthenticationToken token = getAuthenticationToken(context);
-        
-        boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
-        List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
-        if (ranges.isEmpty()) {
-          ranges = new ArrayList<Range>(1);
-          ranges.add(new Range());
-        }
-        
-        // get the metadata information for these ranges
-        Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
-        TabletLocator tl;
+    for (Map.Entry<String,InputTableConfig> tableConfigEntry : tableConfigs.entrySet()) {
+
+      String tableName = tableConfigEntry.getKey();
+      InputTableConfig tableConfig = tableConfigEntry.getValue();
+      
+      Instance instance = getInstance(context);
+      boolean mockInstance;
+      String tableId;
+      // resolve table name to id once, and use id from this point forward
+      if (instance instanceof MockInstance) {
+        tableId = "";
+        mockInstance = true;
+      } else {
         try {
-          if (tableConfig.isOfflineScan()) {
+          tableId = Tables.getTableId(instance, tableName);
+        } catch (TableNotFoundException e) {
+          throw new IOException(e);
+        }
+        mockInstance = false;
+      }
+      
+      Authorizations auths = getScanAuthorizations(context);
+      String principal = getPrincipal(context);
+      AuthenticationToken token = getAuthenticationToken(context);
+
+      boolean autoAdjust = tableConfig.shouldAutoAdjustRanges();
+      List<Range> ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges();
+      if (ranges.isEmpty()) {
+        ranges = new ArrayList<Range>(1);
+        ranges.add(new Range());
+      }
+
+      // get the metadata information for these ranges
+      Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
+      TabletLocator tl;
+      try {
+        if (tableConfig.isOfflineScan()) {
+          binnedRanges = binOfflineTable(context, tableId, ranges);
+          while (binnedRanges == null) {
+            // Some tablets were still online, try again
+            UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
             binnedRanges = binOfflineTable(context, tableId, ranges);
-            while (binnedRanges == null) {
-              // Some tablets were still online, try again
-              UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
-              binnedRanges = binOfflineTable(context, tableId, ranges);
-              
+
+          }
+        } else {
+          tl = getTabletLocator(context, tableId);
+          // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
+          tl.invalidateCache();
+          Credentials creds = new Credentials(getPrincipal(context), getAuthenticationToken(context));
+
+          while (!tl.binRanges(creds, ranges, binnedRanges).isEmpty()) {
+            if (!(instance instanceof MockInstance)) {
+              if (!Tables.exists(instance, tableId))
+                throw new TableDeletedException(tableId);
+              if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
+                throw new TableOfflineException(instance, tableId);
             }
-          } else {
-            tl = getTabletLocator(context, tableId);
-            // its possible that the cache could contain complete, but old information about a tables tablets... so clear it
+            binnedRanges.clear();
+            log.warn("Unable to locate bins for specified ranges. Retrying.");
+            UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
             tl.invalidateCache();
-            Credentials creds = new Credentials(getPrincipal(context), getAuthenticationToken(context));
-            
-            while (!tl.binRanges(creds, ranges, binnedRanges).isEmpty()) {
-              if (!(instance instanceof MockInstance)) {
-                if (!Tables.exists(instance, tableId))
-                  throw new TableDeletedException(tableId);
-                if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
-                  throw new TableOfflineException(instance, tableId);
-              }
-              binnedRanges.clear();
-              log.warn("Unable to locate bins for specified ranges. Retrying.");
-              UtilWaitThread.sleep(100 + (int) (Math.random() * 100)); // sleep randomly between 100 and 200 ms
-              tl.invalidateCache();
-            }
           }
-        } catch (Exception e) {
-          throw new IOException(e);
         }
-        
-        HashMap<Range,ArrayList<String>> splitsToAdd = null;
-        
-        if (!autoAdjust)
-          splitsToAdd = new HashMap<Range,ArrayList<String>>();
-        
-        HashMap<String,String> hostNameCache = new HashMap<String,String>();
-        for (Map.Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
-          String ip = tserverBin.getKey().split(":", 2)[0];
-          String location = hostNameCache.get(ip);
-          if (location == null) {
-            InetAddress inetAddress = InetAddress.getByName(ip);
-            location = inetAddress.getHostName();
-            hostNameCache.put(ip, location);
-          }
-          for (Map.Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
-            Range ke = extentRanges.getKey().toDataRange();
-            for (Range r : extentRanges.getValue()) {
-              if (autoAdjust) {
-                // divide ranges into smaller ranges, based on the tablets
-                RangeInputSplit split = new RangeInputSplit(tableName, tableId, ke.clip(r), new String[] {location});
-                
-                split.setOffline(tableConfig.isOfflineScan());
-                split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-                split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-                split.setMockInstance(mockInstance);
-                split.setFetchedColumns(tableConfig.getFetchedColumns());
-                split.setPrincipal(principal);
-                split.setToken(token);
-                split.setInstanceName(instance.getInstanceName());
-                split.setZooKeepers(instance.getZooKeepers());
-                split.setAuths(auths);
-                split.setIterators(tableConfig.getIterators());
-                split.setLogLevel(logLevel);
-                
-                splits.add(split);
-              } else {
-                // don't divide ranges
-                ArrayList<String> locations = splitsToAdd.get(r);
-                if (locations == null)
-                  locations = new ArrayList<String>(1);
-                locations.add(location);
-                splitsToAdd.put(r, locations);
-              }
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+
+      HashMap<Range,ArrayList<String>> splitsToAdd = null;
+
+      if (!autoAdjust)
+        splitsToAdd = new HashMap<Range,ArrayList<String>>();
+
+      HashMap<String,String> hostNameCache = new HashMap<String,String>();
+      for (Map.Entry<String,Map<KeyExtent,List<Range>>> tserverBin : binnedRanges.entrySet()) {
+        String ip = tserverBin.getKey().split(":", 2)[0];
+        String location = hostNameCache.get(ip);
+        if (location == null) {
+          InetAddress inetAddress = InetAddress.getByName(ip);
+          location = inetAddress.getHostName();
+          hostNameCache.put(ip, location);
+        }
+        for (Map.Entry<KeyExtent,List<Range>> extentRanges : tserverBin.getValue().entrySet()) {
+          Range ke = extentRanges.getKey().toDataRange();
+          for (Range r : extentRanges.getValue()) {
+            if (autoAdjust) {
+              // divide ranges into smaller ranges, based on the tablets
+              RangeInputSplit split = new RangeInputSplit(tableName, tableId, ke.clip(r), new String[] {location});
+              
+              split.setOffline(tableConfig.isOfflineScan());
+              split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
+              split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
+              split.setMockInstance(mockInstance);
+              split.setFetchedColumns(tableConfig.getFetchedColumns());
+              split.setPrincipal(principal);
+              split.setToken(token);
+              split.setInstanceName(instance.getInstanceName());
+              split.setZooKeepers(instance.getZooKeepers());
+              split.setAuths(auths);
+              split.setIterators(tableConfig.getIterators());
+              split.setLogLevel(logLevel);
+              
+              splits.add(split);
+            } else {
+              // don't divide ranges
+              ArrayList<String> locations = splitsToAdd.get(r);
+              if (locations == null)
+                locations = new ArrayList<String>(1);
+              locations.add(location);
+              splitsToAdd.put(r, locations);
             }
           }
         }
-        
-        if (!autoAdjust)
-          for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
-            RangeInputSplit split = new RangeInputSplit(tableName, tableId, entry.getKey(), entry.getValue().toArray(new String[0]));
-            
-            split.setOffline(tableConfig.isOfflineScan());
-            split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
-            split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
-            split.setMockInstance(mockInstance);
-            split.setFetchedColumns(tableConfig.getFetchedColumns());
-            split.setPrincipal(principal);
-            split.setToken(token);
-            split.setInstanceName(instance.getInstanceName());
-            split.setZooKeepers(instance.getZooKeepers());
-            split.setAuths(auths);
-            split.setIterators(tableConfig.getIterators());
-            split.setLogLevel(logLevel);
-            
-            splits.add(split);
-          }
       }
-      return splits;
-    } finally {
-      instance.close();
+
+      if (!autoAdjust)
+        for (Map.Entry<Range,ArrayList<String>> entry : splitsToAdd.entrySet()) {
+          RangeInputSplit split = new RangeInputSplit(tableName, tableId, entry.getKey(), entry.getValue().toArray(new String[0]));
+
+          split.setOffline(tableConfig.isOfflineScan());
+          split.setIsolatedScan(tableConfig.shouldUseIsolatedScanners());
+          split.setUsesLocalIterators(tableConfig.shouldUseLocalIterators());
+          split.setMockInstance(mockInstance);
+          split.setFetchedColumns(tableConfig.getFetchedColumns());
+          split.setPrincipal(principal);
+          split.setToken(token);
+          split.setInstanceName(instance.getInstanceName());
+          split.setZooKeepers(instance.getZooKeepers());
+          split.setAuths(auths);
+          split.setIterators(tableConfig.getIterators());
+          split.setLogLevel(logLevel);
+          
+          splits.add(split);
+        }
     }
+    return splits;
   }
 
   // use reflection to pull the Configuration out of the JobContext for Hadoop 1 and Hadoop 2 compatibility

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e11b3338/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
index b816d43..0c924b1 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
@@ -533,20 +533,17 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
   public void checkOutputSpecs(JobContext job) throws IOException {
     if (!isConnectorInfoSet(job))
       throw new IOException("Connector info has not been set.");
-    Instance instance = getInstance(job);
     try {
       // if the instance isn't configured, it will complain here
       String principal = getPrincipal(job);
       AuthenticationToken token = getAuthenticationToken(job);
-      Connector c = instance.getConnector(principal, token);
+      Connector c = getInstance(job).getConnector(principal, token);
       if (!c.securityOperations().authenticateUser(principal, token))
         throw new IOException("Unable to authenticate user");
     } catch (AccumuloException e) {
       throw new IOException(e);
     } catch (AccumuloSecurityException e) {
       throw new IOException(e);
-    } finally {
-      instance.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e11b3338/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
index 7419d9b..7b17d11 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java
@@ -601,11 +601,7 @@ public class InputConfigurator extends ConfiguratorBase {
     if ("MockInstance".equals(instanceType))
       return new MockTabletLocator();
     Instance instance = getInstance(implementingClass, conf);
-    try {
-      return TabletLocator.getLocator(instance, new Text(tableId));
-    } finally {
-      instance.close();
-    }
+    return TabletLocator.getLocator(instance, new Text(tableId));
   }
 
   // InputFormat doesn't have the equivalent of OutputFormat's checkOutputSpecs(JobContext job)
@@ -629,12 +625,10 @@ public class InputConfigurator extends ConfiguratorBase {
     if (!"MockInstance".equals(instanceKey) && !"ZooKeeperInstance".equals(instanceKey))
       throw new IOException("Instance info has not been set.");
     // validate that we can connect as configured
-    Instance inst = getInstance(implementingClass, conf);
     try {
       String principal = getPrincipal(implementingClass, conf);
       AuthenticationToken token = getAuthenticationToken(implementingClass, conf);
-
-      Connector c = inst.getConnector(principal, token);
+      Connector c = getInstance(implementingClass, conf).getConnector(principal, token);
       if (!c.securityOperations().authenticateUser(principal, token))
         throw new IOException("Unable to authenticate user");
 
@@ -662,8 +656,6 @@ public class InputConfigurator extends ConfiguratorBase {
       throw new IOException(e);
     } catch (TableNotFoundException e) {
       throw new IOException(e);
-    } finally {
-      inst.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e11b3338/server/base/src/main/java/org/apache/accumulo/server/master/state/TServerInstance.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TServerInstance.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TServerInstance.java
index e7dce67..54d7e71 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TServerInstance.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TServerInstance.java
@@ -16,9 +16,6 @@
  */
 package org.apache.accumulo.server.master.state;
 
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
 import java.io.Serializable;
 
 import org.apache.accumulo.core.data.Mutation;
@@ -38,8 +35,7 @@ public class TServerInstance implements Comparable<TServerInstance>, Serializabl
   
   private static final long serialVersionUID = 1L;
   
-  // HostAndPort is not Serializable
-  private transient HostAndPort location;
+  private HostAndPort location;
   private String session;
   private String cachedStringRepresentation;
   
@@ -129,14 +125,4 @@ public class TServerInstance implements Comparable<TServerInstance>, Serializabl
   public String getSession() {
     return session;
   }
-  
-  private void writeObject(ObjectOutputStream out) throws IOException {
-    out.defaultWriteObject();
-    out.writeObject(location.toString());
-  }
-  
-  private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
-    in.defaultReadObject();
-    location = HostAndPort.fromString(in.readObject().toString());
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e11b3338/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
index d904037..7daa21a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
@@ -1080,8 +1080,9 @@ public class Tablet {
     splitCreationTime = 0;
   }
 
-  public Tablet(KeyExtent extent, TabletServer tabletServer, TabletResourceManager trm, SplitInfo info) throws IOException {
-    this(tabletServer, new Text(info.dir), extent, trm, CachedConfiguration.getInstance(), info.datafiles, info.time, info.initFlushID, info.initCompactID, info.lastLocation);
+  public Tablet(TabletServer tabletServer, Text location, KeyExtent extent, TabletResourceManager trm, SortedMap<FileRef,DataFileValue> datafiles, String time,
+      long initFlushID, long initCompactID) throws IOException {
+    this(tabletServer, location, extent, trm, CachedConfiguration.getInstance(), datafiles, time, initFlushID, initCompactID);
     splitCreationTime = System.currentTimeMillis();
   }
 
@@ -1093,8 +1094,8 @@ public class Tablet {
   static private final List<LogEntry> EMPTY = Collections.emptyList();
 
   private Tablet(TabletServer tabletServer, Text location, KeyExtent extent, TabletResourceManager trm, Configuration conf,
-      SortedMap<FileRef,DataFileValue> datafiles, String time, long initFlushID, long initCompactID, TServerInstance last) throws IOException {
-    this(tabletServer, location, extent, trm, conf, VolumeManagerImpl.get(), EMPTY, datafiles, time, last, new HashSet<FileRef>(), initFlushID, initCompactID);
+      SortedMap<FileRef,DataFileValue> datafiles, String time, long initFlushID, long initCompactID) throws IOException {
+    this(tabletServer, location, extent, trm, conf, VolumeManagerImpl.get(), EMPTY, datafiles, time, null, new HashSet<FileRef>(), initFlushID, initCompactID);
   }
 
   private static String lookupTime(AccumuloConfiguration conf, KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
@@ -3476,15 +3477,13 @@ public class Tablet {
     String time;
     long initFlushID;
     long initCompactID;
-    TServerInstance lastLocation;
 
-    SplitInfo(String d, SortedMap<FileRef,DataFileValue> dfv, String time, long initFlushID, long initCompactID, TServerInstance lastLocation) {
+    SplitInfo(String d, SortedMap<FileRef,DataFileValue> dfv, String time, long initFlushID, long initCompactID) {
       this.dir = d;
       this.datafiles = dfv;
       this.time = time;
       this.initFlushID = initFlushID;
       this.initCompactID = initCompactID;
-      this.lastLocation = lastLocation;
     }
 
   }
@@ -3573,8 +3572,8 @@ public class Tablet {
 
       log.log(TLevel.TABLET_HIST, extent + " split " + low + " " + high);
 
-      newTablets.put(high, new SplitInfo(tabletDirectory, highDatafileSizes, time, lastFlushID, lastCompactID, this.lastLocation));
-      newTablets.put(low, new SplitInfo(lowDirectory, lowDatafileSizes, time, lastFlushID, lastCompactID, this.lastLocation));
+      newTablets.put(high, new SplitInfo(tabletDirectory, highDatafileSizes, time, lastFlushID, lastCompactID));
+      newTablets.put(low, new SplitInfo(lowDirectory, lowDatafileSizes, time, lastFlushID, lastCompactID));
 
       long t2 = System.currentTimeMillis();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e11b3338/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 9dabee7..1a4f5eb 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -2658,10 +2658,12 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     Tablet[] newTablets = new Tablet[2];
 
     Entry<KeyExtent,SplitInfo> first = tabletInfo.firstEntry();
-    newTablets[0] = new Tablet(first.getKey(), TabletServer.this, resourceManager.createTabletResourceManager(), first.getValue());
+    newTablets[0] = new Tablet(TabletServer.this, new Text(first.getValue().dir), first.getKey(), resourceManager.createTabletResourceManager(),
+        first.getValue().datafiles, first.getValue().time, first.getValue().initFlushID, first.getValue().initCompactID);
 
     Entry<KeyExtent,SplitInfo> last = tabletInfo.lastEntry();
-    newTablets[1] = new Tablet(first.getKey(), TabletServer.this, resourceManager.createTabletResourceManager(), last.getValue());
+    newTablets[1] = new Tablet(TabletServer.this, new Text(last.getValue().dir), last.getKey(), resourceManager.createTabletResourceManager(),
+        last.getValue().datafiles, last.getValue().time, last.getValue().initFlushID, last.getValue().initCompactID);
 
     // roll tablet stats over into tablet server's statsKeeper object as
     // historical data

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e11b3338/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTable.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTable.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTable.java
index 5a695bc..b65fbaa 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTable.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/CreateTable.java
@@ -20,7 +20,6 @@ import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.test.randomwalk.State;
@@ -46,8 +45,6 @@ public class CreateTable extends Test {
       log.debug("Create " + tableName + " failed, it exists");
     } catch (IllegalArgumentException e) {
       log.debug("Create: " + e.toString());
-    } catch (AccumuloSecurityException e) {
-      log.debug("Could not create table: " + e);
     }
   }
 }