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/06/05 06:42:43 UTC

[01/35] git commit: ACCUMULO-2635 Add ZooCacheFactory and integrate across code

Repository: accumulo
Updated Branches:
  refs/heads/ACCUMULO-378 73d34ec71 -> 856f23590


ACCUMULO-2635 Add ZooCacheFactory and integrate across code

This commit introduces a ZooCacheFactory class to centralize and maximize reuse of
ZooCache instances and to facilitate testing. The caching mechanism formerly in ZooCache
was moved to the factory; also, the separate caching mechanism in ServerClient was removed
in favor of the factory.


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

Branch: refs/heads/ACCUMULO-378
Commit: a61e242e39b526fc61e1b5f757191d7a8b8fb7c8
Parents: 7c17eb1
Author: Bill Havanki <bh...@cloudera.com>
Authored: Fri Apr 4 18:15:28 2014 -0400
Committer: Bill Havanki <bh...@cloudera.com>
Committed: Mon Jun 2 10:24:22 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/ZooKeeperInstance.java |   7 +-
 .../core/client/impl/ConditionalWriterImpl.java |   4 +-
 .../client/impl/InstanceOperationsImpl.java     |   3 +-
 .../accumulo/core/client/impl/Namespaces.java   |   3 +-
 .../core/client/impl/RootTabletLocator.java     |  14 +-
 .../accumulo/core/client/impl/ServerClient.java |  15 +-
 .../accumulo/core/client/impl/Tables.java       |   3 +-
 .../core/client/impl/ZookeeperLockChecker.java  |  10 +-
 .../core/client/ZooKeeperInstanceTest.java      | 143 +++++++++++++++++++
 .../core/client/impl/RootTabletLocatorTest.java |  61 ++++++++
 .../client/impl/ZookeeperLockCheckerTest.java   |  58 ++++++++
 .../accumulo/fate/zookeeper/ZooCache.java       |  15 +-
 .../fate/zookeeper/ZooCacheFactory.java         |  78 ++++++++++
 .../apache/accumulo/fate/zookeeper/ZooLock.java |   2 +-
 .../fate/zookeeper/ZooCacheFactoryTest.java     |  87 +++++++++++
 .../accumulo/server/client/HdfsZooInstance.java |   3 +-
 .../server/conf/NamespaceConfiguration.java     |   3 +-
 .../server/conf/TableConfiguration.java         |   3 +-
 .../accumulo/server/conf/ZooConfiguration.java  |   5 +-
 19 files changed, 476 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index 43b315d..6d35757 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -37,6 +37,7 @@ import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.commons.configuration.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
@@ -132,8 +133,10 @@ public class ZooKeeperInstance implements Instance {
    *          See {@link ClientConfiguration} which extends Configuration with convenience methods specific to Accumulo.
    * @since 1.6.0
    */
-
   public ZooKeeperInstance(Configuration config) {
+    this(config, new ZooCacheFactory());
+  }
+  ZooKeeperInstance(Configuration config, ZooCacheFactory zcf) {
     ArgumentChecker.notNull(config);
     if (config instanceof ClientConfiguration) {
       this.clientConf = (ClientConfiguration)config;
@@ -146,7 +149,7 @@ public class ZooKeeperInstance implements Instance {
       throw new IllegalArgumentException("Expected exactly one of instanceName and instanceId to be set");
     this.zooKeepers = clientConf.get(ClientProperty.INSTANCE_ZK_HOST);
     this.zooKeepersSessionTimeOut = (int) AccumuloConfiguration.getTimeInMillis(clientConf.get(ClientProperty.INSTANCE_ZK_TIMEOUT));
-    zooCache = ZooCache.getInstance(zooKeepers, zooKeepersSessionTimeOut);
+    zooCache = zcf.getZooCache(zooKeepers, zooKeepersSessionTimeOut);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
index 01e4b95..f5e6dd2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
@@ -73,6 +73,7 @@ import org.apache.accumulo.core.util.LoggingRunnable;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.LockID;
 import org.apache.accumulo.trace.instrument.Trace;
@@ -662,8 +663,9 @@ class ConditionalWriterImpl implements ConditionalWriter {
     
     LockID lid = new LockID(ZooUtil.getRoot(instance) + Constants.ZTSERVERS, sessionId.lockId);
     
+    ZooCacheFactory zcf = new ZooCacheFactory();
     while (true) {
-      if (!ZooLock.isLockHeld(ServerClient.getZooCache(instance), lid)) {
+      if (!ZooLock.isLockHeld(zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut()), lid)) {
         // ACCUMULO-1152 added a tserver lock check to the tablet location cache, so this invalidation prevents future attempts to contact the
         // tserver even its gone zombie and is still running w/o a lock
         locator.invalidateCache(location);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
index 12e4912..63e3497 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
@@ -41,6 +41,7 @@ import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransport;
@@ -109,7 +110,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
 
   @Override
   public List<String> getTabletServers() {
-    ZooCache cache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    ZooCache cache = new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
     List<String> results = new ArrayList<String>();
     for (String candidate : cache.getChildren(path)) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
index c134c04..30f0266 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 
 public class Namespaces {
   public static final String VALID_NAME_REGEX = "^\\w*$";
@@ -82,7 +83,7 @@ public class Namespaces {
     if (sm != null) {
       sm.checkPermission(TABLES_PERMISSION);
     }
-    return ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    return new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
 
   private static SortedMap<String,String> getMap(Instance instance, boolean nameAsKey) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
index 1224207..97d476b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
@@ -36,18 +36,24 @@ import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
 public class RootTabletLocator extends TabletLocator {
   
-  private Instance instance;
-  private TabletServerLockChecker lockChecker;
+  private final Instance instance;
+  private final TabletServerLockChecker lockChecker;
+  private final ZooCacheFactory zcf;
   
   RootTabletLocator(Instance instance, TabletServerLockChecker lockChecker) {
+    this(instance, lockChecker, new ZooCacheFactory());
+  }
+  RootTabletLocator(Instance instance, TabletServerLockChecker lockChecker, ZooCacheFactory zcf) {
     this.instance = instance;
     this.lockChecker = lockChecker;
+    this.zcf = zcf;
   }
   
   @Override
@@ -87,7 +93,7 @@ public class RootTabletLocator extends TabletLocator {
   
   @Override
   public void invalidateCache(String server) {
-    ZooCache zooCache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    ZooCache zooCache = zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     String root = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
     zooCache.clear(root + "/" + server);
   }
@@ -97,7 +103,7 @@ public class RootTabletLocator extends TabletLocator {
   
   protected TabletLocation getRootTabletLocation() {
     String zRootLocPath = ZooUtil.getRoot(instance) + RootTable.ZROOT_TABLET_LOCATION;
-    ZooCache zooCache = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    ZooCache zooCache = zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     
     OpTimer opTimer = new OpTimer(Logger.getLogger(this.getClass()), Level.TRACE).start("Looking up root tablet location in zookeeper.");
     byte[] loc = zooCache.get(zRootLocPath);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
index bf3e1bc..8da530b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
@@ -17,8 +17,6 @@
 package org.apache.accumulo.core.client.impl;
 
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -38,22 +36,13 @@ import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.log4j.Logger;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
 
 public class ServerClient {
   private static final Logger log = Logger.getLogger(ServerClient.class);
-  private static final Map<String,ZooCache> zooCaches = new HashMap<String,ZooCache>();
-  
-  synchronized static ZooCache getZooCache(Instance instance) {
-    ZooCache result = zooCaches.get(instance.getZooKeepers());
-    if (result == null) {
-      result = new ZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), null);
-      zooCaches.put(instance.getZooKeepers(), result);
-    }
-    return result;
-  }
   
   public static <T> T execute(Instance instance, ClientExecReturn<T,ClientService.Client> exec) throws AccumuloException, AccumuloSecurityException {
     try {
@@ -135,7 +124,7 @@ public class ServerClient {
     ArrayList<ThriftTransportKey> servers = new ArrayList<ThriftTransportKey>();
     
     // add tservers
-    ZooCache zc = getZooCache(instance);
+    ZooCache zc = new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     for (String tserver : zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTSERVERS)) {
       String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + tserver;
       byte[] data = ZooUtil.getLockData(zc, path);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index 5988bda..32b6e28 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -36,6 +36,7 @@ import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.log4j.Logger;
 
 public class Tables {
@@ -117,7 +118,7 @@ public class Tables {
     if (sm != null) {
       sm.checkPermission(TABLES_PERMISSION);
     }
-    return ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    return new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
   }
 
   private static SortedMap<String,String> getMap(Instance instance, boolean nameAsKey) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
index 85bd171..be56ad4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ZookeeperLockChecker.java
@@ -21,6 +21,7 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletServerLockChecker;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.zookeeper.KeeperException;
 
@@ -29,11 +30,14 @@ import org.apache.zookeeper.KeeperException;
  */
 public class ZookeeperLockChecker implements TabletServerLockChecker {
   
-  private ZooCache zc;
-  private String root;
+  private final ZooCache zc;
+  private final String root;
 
   ZookeeperLockChecker(Instance instance) {
-    zc = ZooCache.getInstance(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
+    this(instance, new ZooCacheFactory());
+  }
+  ZookeeperLockChecker(Instance instance, ZooCacheFactory zcf) {
+    zc = zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
     this.root = ZooUtil.getRoot(instance) + Constants.ZTSERVERS;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java b/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
new file mode 100644
index 0000000..8d86d5a
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.core.client;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+
+public class ZooKeeperInstanceTest {
+  private static final UUID IID = UUID.randomUUID();
+  private static final String IID_STRING = IID.toString();
+  private ClientConfiguration config;
+  private ZooCacheFactory zcf;
+  private ZooCache zc;
+  private ZooKeeperInstance zki;
+
+  private void mockIdConstruction(ClientConfiguration config) {
+    expect(config.get(ClientProperty.INSTANCE_ID)).andReturn(IID_STRING);
+    expect(config.get(ClientProperty.INSTANCE_NAME)).andReturn(null);
+    expect(config.get(ClientProperty.INSTANCE_ZK_HOST)).andReturn("zk1");
+    expect(config.get(ClientProperty.INSTANCE_ZK_TIMEOUT)).andReturn("30");
+  }
+
+  private void mockNameConstruction(ClientConfiguration config) {
+    expect(config.get(ClientProperty.INSTANCE_ID)).andReturn(null);
+    expect(config.get(ClientProperty.INSTANCE_NAME)).andReturn("instance");
+    expect(config.get(ClientProperty.INSTANCE_ZK_HOST)).andReturn("zk1");
+    expect(config.get(ClientProperty.INSTANCE_ZK_TIMEOUT)).andReturn("30");
+  }
+
+  @Before
+  public void setUp() {
+    config = createMock(ClientConfiguration.class);
+    mockNameConstruction(config);
+    replay(config);
+    zcf = createMock(ZooCacheFactory.class);
+    zc = createMock(ZooCache.class);
+    expect(zcf.getZooCache("zk1", 30000)).andReturn(zc).anyTimes();
+    replay(zcf);
+    zki = new ZooKeeperInstance(config, zcf);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidConstruction() {
+    config = createMock(ClientConfiguration.class);
+    expect(config.get(ClientProperty.INSTANCE_ID)).andReturn(IID_STRING);
+    mockNameConstruction(config);
+    replay(config);
+    new ZooKeeperInstance(config);
+  }
+
+  @Test
+  public void testSimpleGetters() {
+    assertEquals("instance", zki.getInstanceName());
+    assertEquals("zk1", zki.getZooKeepers());
+    assertEquals(30000, zki.getZooKeepersSessionTimeOut());
+  }
+
+  @Test
+  public void testGetInstanceID_FromCache() {
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
+    replay(zc);
+    assertEquals(IID_STRING, zki.getInstanceID());
+  }
+
+  @Test
+  public void testGetInstanceID_Direct() {
+    config = createMock(ClientConfiguration.class);
+    mockIdConstruction(config);
+    replay(config);
+    zki = new ZooKeeperInstance(config, zcf);
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
+    replay(zc);
+    assertEquals(IID_STRING, zki.getInstanceID());
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGetInstanceID_NoMapping() {
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(null);
+    replay(zc);
+    zki.getInstanceID();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGetInstanceID_IDMissingForName() {
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn(null);
+    replay(zc);
+    zki.getInstanceID();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGetInstanceID_IDMissingForID() {
+    config = createMock(ClientConfiguration.class);
+    mockIdConstruction(config);
+    replay(config);
+    zki = new ZooKeeperInstance(config, zcf);
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn(null);
+    replay(zc);
+    zki.getInstanceID();
+  }
+
+  @Test
+  public void testGetInstanceName() {
+    config = createMock(ClientConfiguration.class);
+    mockIdConstruction(config);
+    replay(config);
+    zki = new ZooKeeperInstance(config, zcf);
+    expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
+    List<String> children = new java.util.ArrayList<String>();
+    children.add("child1");
+    children.add("child2");
+    expect(zc.getChildren(Constants.ZROOT + Constants.ZINSTANCES)).andReturn(children);
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child1")).andReturn(UUID.randomUUID().toString().getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child2")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    replay(zc);
+    assertEquals("child2", zki.getInstanceName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java
new file mode 100644
index 0000000..b7be982
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/RootTabletLocatorTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.core.client.impl;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletServerLockChecker;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
+import org.junit.Before;
+import org.junit.Test;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+public class RootTabletLocatorTest {
+  private Instance instance;
+  private TabletServerLockChecker lockChecker;
+  private ZooCacheFactory zcf;
+  private ZooCache zc;
+  private RootTabletLocator rtl;
+
+  @Before
+  public void setUp() {
+    instance = createMock(Instance.class);
+    expect(instance.getInstanceID()).andReturn("iid").anyTimes();
+    expect(instance.getZooKeepers()).andReturn("zk1").anyTimes();
+    expect(instance.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
+    replay(instance);
+    lockChecker = createMock(TabletServerLockChecker.class);
+    zcf = createMock(ZooCacheFactory.class);
+    zc = createMock(ZooCache.class);
+    rtl = new RootTabletLocator(instance, lockChecker, zcf);
+  }
+
+  @Test
+  public void testInvalidateCache_Server() {
+    expect(zcf.getZooCache("zk1", 30000)).andReturn(zc);
+    replay(zcf);
+    zc.clear(ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/server");
+    replay(zc);
+    rtl.invalidateCache("server");
+    verify(zc);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java
new file mode 100644
index 0000000..1749a4b
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/ZookeeperLockCheckerTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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.core.client.impl;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
+import org.junit.Before;
+import org.junit.Test;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+public class ZookeeperLockCheckerTest {
+  private Instance instance;
+  private ZooCacheFactory zcf;
+  private ZooCache zc;
+  private ZookeeperLockChecker zklc;
+
+  @Before
+  public void setUp() {
+    instance = createMock(Instance.class);
+    expect(instance.getInstanceID()).andReturn("iid").anyTimes();
+    expect(instance.getZooKeepers()).andReturn("zk1").anyTimes();
+    expect(instance.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
+    replay(instance);
+    zcf = createMock(ZooCacheFactory.class);
+    zc = createMock(ZooCache.class);
+    expect(zcf.getZooCache("zk1", 30000)).andReturn(zc);
+    replay(zcf);
+    zklc = new ZookeeperLockChecker(instance, zcf);
+  }
+
+  @Test
+  public void testInvalidateCache() {
+    zc.clear(ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/server");
+    replay(zc);
+    zklc.invalidateCache("server");
+    verify(zc);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
index e793a69..99ffd04 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
@@ -297,16 +297,13 @@ public class ZooCache {
     }
   }
 
-  private static Map<String,ZooCache> instances = new HashMap<String,ZooCache>();
+  private static ZooCacheFactory zcf = new ZooCacheFactory();
 
+  /**
+   * @deprecated Use {@link ZooCacheFactory} instead.
+   */
+  @Deprecated
   public static synchronized ZooCache getInstance(String zooKeepers, int sessionTimeout) {
-    String key = zooKeepers + ":" + sessionTimeout;
-    ZooCache zc = instances.get(key);
-    if (zc == null) {
-      zc = new ZooCache(zooKeepers, sessionTimeout);
-      instances.put(key, zc);
-    }
-
-    return zc;
+    return zcf.getZooCache(zooKeepers, sessionTimeout);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
new file mode 100644
index 0000000..3c59a00
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactory.java
@@ -0,0 +1,78 @@
+/*
+ * 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.fate.zookeeper;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.zookeeper.Watcher;
+
+/**
+ * A factory for {@link ZooCache} instances.
+ */
+public class ZooCacheFactory {
+  // TODO: make this better - LRU, soft references, ...
+  private static Map<String,ZooCache> instances = new HashMap<String,ZooCache>();
+
+  /**
+   * Gets a {@link ZooCache}. The same object may be returned for multiple calls with the same arguments.
+   *
+   * @param zooKeepers
+   *          comma-seprated list of ZooKeeper host[:port]s
+   * @param sessionTimeout
+   *          session timeout
+   * @return cache object
+   */
+  public ZooCache getZooCache(String zooKeepers, int sessionTimeout) {
+    String key = zooKeepers + ":" + sessionTimeout;
+    synchronized (instances) {
+      ZooCache zc = instances.get(key);
+      if (zc == null) {
+        zc = new ZooCache(zooKeepers, sessionTimeout);
+        instances.put(key, zc);
+      }
+      return zc;
+    }
+  }
+  /**
+   * Gets a watched {@link ZooCache}. If the watcher is null, then the same (unwatched)
+   * object may be returned for multiple calls with the same remaining arguments.
+   *
+   * @param zooKeepers
+   *          comma-seprated list of ZooKeeper host[:port]s
+   * @param sessionTimeout
+   *          session timeout
+   * @param watcher watcher (optional)
+   * @return cache object
+   */
+  public ZooCache getZooCache(String zooKeepers, int sessionTimeout, Watcher watcher) {
+    if (watcher == null) {
+      // reuse
+      return getZooCache(zooKeepers, sessionTimeout);
+    }
+    return new ZooCache(zooKeepers, sessionTimeout, watcher);
+  }
+
+  /**
+   * Resets the factory. All cached objects are flushed.
+   */
+  void reset() {
+    synchronized (instances) {
+      instances.clear();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
index 03e159f..38559a7 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java
@@ -68,7 +68,7 @@ public class ZooLock implements Watcher {
   private String asyncLock;
   
   public ZooLock(String zookeepers, int timeInMillis, String scheme, byte[] auth, String path) {
-    this(new ZooCache(zookeepers, timeInMillis), ZooReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path);
+    this(new ZooCacheFactory().getZooCache(zookeepers, timeInMillis), ZooReaderWriter.getInstance(zookeepers, timeInMillis, scheme, auth), path);
   }
   
   protected ZooLock(ZooCache zc, IZooReaderWriter zrw, String path) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.java
new file mode 100644
index 0000000..e7dffc1
--- /dev/null
+++ b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooCacheFactoryTest.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 org.apache.accumulo.fate.zookeeper;
+
+import org.apache.zookeeper.Watcher;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import static org.easymock.EasyMock.createMock;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+
+public class ZooCacheFactoryTest {
+  private ZooCacheFactory zcf;
+
+  @Before
+  public void setUp() {
+    zcf = new ZooCacheFactory();
+  }
+
+  @After
+  public void tearDown() {
+    zcf.reset();
+  }
+
+  @Test
+  public void testGetZooCache() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1);
+    ZooCache zc1a = zcf.getZooCache(zks1, timeout1);
+    assertSame(zc1, zc1a);
+
+    String zks2 = "zk2";
+    int timeout2 = 1000;
+    ZooCache zc2 = zcf.getZooCache(zks2, timeout2);
+    assertNotSame(zc1, zc2);
+
+    String zks3 = "zk1";
+    int timeout3 = 2000;
+    ZooCache zc3 = zcf.getZooCache(zks3, timeout3);
+    assertNotSame(zc1, zc3);
+  }
+
+  @Test
+  public void testGetZooCacheWatcher() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    Watcher watcher = createMock(Watcher.class);
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1, watcher);
+    assertNotNull(zc1);
+  }
+  @Test
+  public void testGetZooCacheWatcher_Null() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1, null);
+    assertNotNull(zc1);
+    ZooCache zc1a = zcf.getZooCache(zks1, timeout1);
+    assertSame(zc1, zc1a);
+  }
+
+  @Test
+  public void testReset() {
+    String zks1 = "zk1";
+    int timeout1 = 1000;
+    ZooCache zc1 = zcf.getZooCache(zks1, timeout1);
+    zcf.reset();
+    ZooCache zc1a = zcf.getZooCache(zks1, timeout1);
+    assertNotSame(zc1, zc1a);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index 620188c..27c289b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@ -41,6 +41,7 @@ import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -67,7 +68,7 @@ public class HdfsZooInstance implements Instance {
 
   private HdfsZooInstance() {
     AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
-    zooCache = new ZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
+    zooCache = new ZooCacheFactory().getZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
   }
 
   private static HdfsZooInstance cachedHdfsZooInstance = null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
index 99532ca..eab198e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.log4j.Logger;
 
@@ -83,7 +84,7 @@ public class NamespaceConfiguration extends AccumuloConfiguration {
   private void initializePropCache() {
     synchronized (lock) {
       if (propCache == null)
-        propCache = new ZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut(), new NamespaceConfWatcher(inst));
+        propCache = new ZooCacheFactory().getZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut(), new NamespaceConfWatcher(inst));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
index c134e31..909b450 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.conf.ConfigurationObserver;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.log4j.Logger;
 
@@ -63,7 +64,7 @@ public class TableConfiguration extends AccumuloConfiguration {
   private void initializeZooCache() {
     synchronized (initLock) {
       if (null == tablePropCache) {
-        tablePropCache = new ZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), new TableConfWatcher(instance));
+        tablePropCache = new ZooCacheFactory().getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut(), new TableConfWatcher(instance));
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a61e242e/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
index 0c03aac..2eafc50 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
+import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.client.HdfsZooInstance.AccumuloNotInitializedException;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -50,7 +51,7 @@ public class ZooConfiguration extends AccumuloConfiguration {
   
   synchronized public static ZooConfiguration getInstance(Instance inst, AccumuloConfiguration parent) {
     if (instance == null) {
-      propCache = new ZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
+      propCache = new ZooCacheFactory().getZooCache(inst.getZooKeepers(), inst.getZooKeepersSessionTimeOut());
       instance = new ZooConfiguration(parent);
       instanceId = inst.getInstanceID();
     }
@@ -59,7 +60,7 @@ public class ZooConfiguration extends AccumuloConfiguration {
   
   synchronized public static ZooConfiguration getInstance(AccumuloConfiguration parent) {
     if (instance == null) {
-      propCache = new ZooCache(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
+      propCache = new ZooCacheFactory().getZooCache(parent.get(Property.INSTANCE_ZK_HOST), (int) parent.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
       instance = new ZooConfiguration(parent);
       // InstanceID should be the same across all volumes, so just choose one
       VolumeManager fs;


[18/35] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'


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

Branch: refs/heads/ACCUMULO-378
Commit: 8be4a3d7462f7ca88805c00ec7b3db49319cb5e5
Parents: 9e770ca 60b5a1c
Author: Keith Turner <kt...@apache.org>
Authored: Tue Jun 3 13:07:58 2014 -0400
Committer: Keith Turner <kt...@apache.org>
Committed: Tue Jun 3 13:07:58 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/tserver/log/DfsLogger.java  | 60 +++++++++-----------
 1 file changed, 28 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/8be4a3d7/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index 5093ad1,5dabedc..245ba2b
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@@ -51,7 -51,9 +51,8 @@@ import org.apache.accumulo.core.securit
  import org.apache.accumulo.core.security.crypto.DefaultCryptoModule;
  import org.apache.accumulo.core.security.crypto.NoFlushOutputStream;
  import org.apache.accumulo.core.util.Daemon;
+ import org.apache.accumulo.core.util.LoggingRunnable;
  import org.apache.accumulo.core.util.Pair;
 -import org.apache.accumulo.core.util.StringUtil;
  import org.apache.accumulo.server.ServerConstants;
  import org.apache.accumulo.server.fs.VolumeManager;
  import org.apache.accumulo.server.master.state.TServerInstance;


[33/35] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'

Conflicts:
	server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java


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

Branch: refs/heads/ACCUMULO-378
Commit: 47d593311e6b57cf3462512214994a73b9d3deb9
Parents: a7884bb 05cf918
Author: Josh Elser <el...@apache.org>
Authored: Wed Jun 4 16:41:00 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Jun 4 16:41:00 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/tserver/log/DfsLogger.java | 17 +++++++++--------
 1 file changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/47d59331/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------


[04/35] git commit: ACCUMULO-2852: use Cwd, since we're using Cwd. also change usage text to match new executable name

Posted by el...@apache.org.
ACCUMULO-2852: use Cwd, since we're using Cwd. also change usage text to match new executable name

Signed-off-by: Josh Elser <el...@apache.org>


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

Branch: refs/heads/ACCUMULO-378
Commit: 05a64c13deb8205e478e43f43938a434e34d2009
Parents: 9fbc24a
Author: Michael Berman <mb...@sqrrl.com>
Authored: Mon Jun 2 17:39:24 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Jun 2 23:30:26 2014 -0400

----------------------------------------------------------------------
 test/system/continuous/master-agitator.pl | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/05a64c13/test/system/continuous/master-agitator.pl
----------------------------------------------------------------------
diff --git a/test/system/continuous/master-agitator.pl b/test/system/continuous/master-agitator.pl
index 0b7ff0d..329114e 100755
--- a/test/system/continuous/master-agitator.pl
+++ b/test/system/continuous/master-agitator.pl
@@ -17,9 +17,10 @@
 
 
 use POSIX qw(strftime);
+use Cwd qw();
 
 if(scalar(@ARGV) != 2){
-	print "Usage : magitator.pl <sleep before kill in minutes> <sleep before start-all in minutes>\n";
+	print "Usage : master-agitator.pl <sleep before kill in minutes> <sleep before start-all in minutes>\n";
 	exit(1);
 }
 


[15/35] git commit: ACCUMULO-2041 extract tablet classes to new files, move tablet-related code to o.a.a.tserver.tablet, make member variables private

Posted by el...@apache.org.
ACCUMULO-2041 extract tablet classes to new files, move tablet-related code to o.a.a.tserver.tablet, make member variables private


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

Branch: refs/heads/ACCUMULO-378
Commit: 7db2abf19c2e0585b2f3ea32068c3d62bd891590
Parents: 9e770ca
Author: Eric C. Newton <er...@gmail.com>
Authored: Mon Apr 21 13:12:13 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Jun 3 10:49:43 2014 -0400

----------------------------------------------------------------------
 .../accumulo/tserver/CompactionStats.java       |   59 -
 .../accumulo/tserver/CompactionWatcher.java     |  110 -
 .../org/apache/accumulo/tserver/Compactor.java  |  548 ---
 .../apache/accumulo/tserver/FileManager.java    |   12 +-
 .../apache/accumulo/tserver/InMemoryMap.java    |    2 +-
 .../accumulo/tserver/MinorCompactionReason.java |   21 +
 .../apache/accumulo/tserver/MinorCompactor.java |  146 -
 .../java/org/apache/accumulo/tserver/Rate.java  |   60 -
 .../org/apache/accumulo/tserver/RootFiles.java  |  133 -
 .../tserver/TConstraintViolationException.java  |   54 +
 .../org/apache/accumulo/tserver/Tablet.java     | 3810 ------------------
 .../tserver/TabletIteratorEnvironment.java      |    8 +-
 .../apache/accumulo/tserver/TabletServer.java   |   79 +-
 .../tserver/TabletServerResourceManager.java    |   67 +-
 .../accumulo/tserver/TabletStatsKeeper.java     |    5 +
 .../tserver/log/TabletServerLogger.java         |    4 +-
 .../apache/accumulo/tserver/tablet/Batch.java   |   35 +
 .../accumulo/tserver/tablet/CommitSession.java  |  121 +
 .../accumulo/tserver/tablet/CompactionInfo.java |  113 +
 .../tserver/tablet/CompactionRunner.java        |   76 +
 .../tserver/tablet/CompactionStats.java         |   59 +
 .../tserver/tablet/CompactionWatcher.java       |  109 +
 .../accumulo/tserver/tablet/Compactor.java      |  477 +++
 .../tserver/tablet/DatafileManager.java         |  581 +++
 .../apache/accumulo/tserver/tablet/KVEntry.java |   39 +
 .../tserver/tablet/MinorCompactionTask.java     |   96 +
 .../accumulo/tserver/tablet/MinorCompactor.java |  145 +
 .../apache/accumulo/tserver/tablet/Rate.java    |   60 +
 .../accumulo/tserver/tablet/RootFiles.java      |  133 +
 .../accumulo/tserver/tablet/ScanBatch.java      |   29 +
 .../accumulo/tserver/tablet/ScanDataSource.java |  222 +
 .../accumulo/tserver/tablet/ScanOptions.java    |   51 +
 .../apache/accumulo/tserver/tablet/Scanner.java |  135 +
 .../accumulo/tserver/tablet/SplitInfo.java      |   52 +
 .../accumulo/tserver/tablet/SplitRowSpec.java   |   29 +
 .../apache/accumulo/tserver/tablet/Tablet.java  | 2564 ++++++++++++
 .../tserver/tablet/TabletClosedException.java   |   29 +
 .../tserver/tablet/TabletCommitter.java         |   48 +
 .../accumulo/tserver/tablet/TabletMemory.java   |  190 +
 .../accumulo/tserver/CountingIteratorTest.java  |    2 +-
 .../apache/accumulo/tserver/RootFilesTest.java  |    1 +
 41 files changed, 5561 insertions(+), 4953 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/CompactionStats.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/CompactionStats.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/CompactionStats.java
deleted file mode 100644
index d359e95..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/CompactionStats.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.tserver;
-
-public class CompactionStats {
-  private long entriesRead;
-  private long entriesWritten;
-  private long fileSize;
-  
-  CompactionStats(long er, long ew) {
-    this.setEntriesRead(er);
-    this.setEntriesWritten(ew);
-  }
-  
-  public CompactionStats() {}
-  
-  private void setEntriesRead(long entriesRead) {
-    this.entriesRead = entriesRead;
-  }
-  
-  public long getEntriesRead() {
-    return entriesRead;
-  }
-  
-  private void setEntriesWritten(long entriesWritten) {
-    this.entriesWritten = entriesWritten;
-  }
-  
-  public long getEntriesWritten() {
-    return entriesWritten;
-  }
-  
-  public void add(CompactionStats mcs) {
-    this.entriesRead += mcs.entriesRead;
-    this.entriesWritten += mcs.entriesWritten;
-  }
-  
-  public void setFileSize(long fileSize) {
-    this.fileSize = fileSize;
-  }
-  
-  public long getFileSize() {
-    return this.fileSize;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/CompactionWatcher.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/CompactionWatcher.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/CompactionWatcher.java
deleted file mode 100644
index 2e4d7b7..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/CompactionWatcher.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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.tserver;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.server.util.time.SimpleTimer;
-import org.apache.accumulo.tserver.Compactor.CompactionInfo;
-import org.apache.log4j.Logger;
-
-/**
- * 
- */
-public class CompactionWatcher implements Runnable {
-  private Map<List<Long>,ObservedCompactionInfo> observedCompactions = new HashMap<List<Long>,ObservedCompactionInfo>();
-  private AccumuloConfiguration config;
-  private static boolean watching = false;
-  
-  private static class ObservedCompactionInfo {
-    CompactionInfo compactionInfo;
-    long firstSeen;
-    boolean loggedWarning;
-    
-    ObservedCompactionInfo(CompactionInfo ci, long time) {
-      this.compactionInfo = ci;
-      this.firstSeen = time;
-    }
-  }
-
-  public CompactionWatcher(AccumuloConfiguration config) {
-    this.config = config;
-  }
-
-  public void run() {
-    List<CompactionInfo> runningCompactions = Compactor.getRunningCompactions();
-    
-    Set<List<Long>> newKeys = new HashSet<List<Long>>();
-    
-    long time = System.currentTimeMillis();
-
-    for (CompactionInfo ci : runningCompactions) {
-      List<Long> compactionKey = Arrays.asList(ci.getID(), ci.getEntriesRead(), ci.getEntriesWritten());
-      newKeys.add(compactionKey);
-      
-      if (!observedCompactions.containsKey(compactionKey)) {
-        observedCompactions.put(compactionKey, new ObservedCompactionInfo(ci, time));
-      }
-    }
-    
-    // look for compactions that finished or made progress and logged a warning
-    HashMap<List<Long>,ObservedCompactionInfo> copy = new HashMap<List<Long>,ObservedCompactionInfo>(observedCompactions);
-    copy.keySet().removeAll(newKeys);
-    
-    for (ObservedCompactionInfo oci : copy.values()) {
-      if (oci.loggedWarning) {
-        Logger.getLogger(CompactionWatcher.class).info("Compaction of " + oci.compactionInfo.getExtent() + " is no longer stuck");
-      }
-    }
-
-    // remove any compaction that completed or made progress
-    observedCompactions.keySet().retainAll(newKeys);
-    
-    long warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
-
-    // check for stuck compactions
-    for (ObservedCompactionInfo oci : observedCompactions.values()) {
-      if (time - oci.firstSeen > warnTime && !oci.loggedWarning) {
-        Thread compactionThread = oci.compactionInfo.getThread();
-        if (compactionThread != null) {
-          StackTraceElement[] trace = compactionThread.getStackTrace();
-          Exception e = new Exception("Possible stack trace of compaction stuck on " + oci.compactionInfo.getExtent());
-          e.setStackTrace(trace);
-          Logger.getLogger(CompactionWatcher.class).warn(
-              "Compaction of " + oci.compactionInfo.getExtent() + " to " + oci.compactionInfo.getOutputFile() + " has not made progress for at least "
-                  + (time - oci.firstSeen) + "ms", e);
-          oci.loggedWarning = true;
-        }
-      }
-    }
-  }
-
-  public static synchronized void startWatching(AccumuloConfiguration config) {
-    if (!watching) {
-      SimpleTimer.getInstance(config).schedule(new CompactionWatcher(config), 10000, 10000);
-      watching = true;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/Compactor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Compactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Compactor.java
deleted file mode 100644
index 822171c..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Compactor.java
+++ /dev/null
@@ -1,548 +0,0 @@
-/*
- * 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.tserver;
-
-import java.io.IOException;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.data.thrift.IterInfo;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.IteratorUtil;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.WrappingIterator;
-import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
-import org.apache.accumulo.core.iterators.system.DeletingIterator;
-import org.apache.accumulo.core.iterators.system.MultiIterator;
-import org.apache.accumulo.core.iterators.system.TimeSettingIterator;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
-import org.apache.accumulo.core.tabletserver.thrift.CompactionReason;
-import org.apache.accumulo.core.tabletserver.thrift.CompactionType;
-import org.apache.accumulo.core.util.LocalityGroupUtil;
-import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
-import org.apache.accumulo.server.fs.FileRef;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.problems.ProblemReport;
-import org.apache.accumulo.server.problems.ProblemReportingIterator;
-import org.apache.accumulo.server.problems.ProblemReports;
-import org.apache.accumulo.server.problems.ProblemType;
-import org.apache.accumulo.trace.instrument.Span;
-import org.apache.accumulo.trace.instrument.Trace;
-import org.apache.accumulo.tserver.Tablet.MinorCompactionReason;
-import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.log4j.Logger;
-
-public class Compactor implements Callable<CompactionStats> {
-
-  public static class CountingIterator extends WrappingIterator {
-
-    private long count;
-    private ArrayList<CountingIterator> deepCopies;
-    private AtomicLong entriesRead;
-
-    @Override
-    public CountingIterator deepCopy(IteratorEnvironment env) {
-      return new CountingIterator(this, env);
-    }
-
-    private CountingIterator(CountingIterator other, IteratorEnvironment env) {
-      setSource(other.getSource().deepCopy(env));
-      count = 0;
-      this.deepCopies = other.deepCopies;
-      this.entriesRead = other.entriesRead;
-      deepCopies.add(this);
-    }
-
-    public CountingIterator(SortedKeyValueIterator<Key,Value> source, AtomicLong entriesRead) {
-      deepCopies = new ArrayList<Compactor.CountingIterator>();
-      this.setSource(source);
-      count = 0;
-      this.entriesRead = entriesRead;
-    }
-
-    @Override
-    public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void next() throws IOException {
-      super.next();
-      count++;
-      if (count % 1024 == 0) {
-        entriesRead.addAndGet(1024);
-      }
-    }
-
-    public long getCount() {
-      long sum = 0;
-      for (CountingIterator dc : deepCopies) {
-        sum += dc.count;
-      }
-
-      return count + sum;
-    }
-  }
-
-  private static final Logger log = Logger.getLogger(Compactor.class);
-
-  static class CompactionCanceledException extends Exception {
-    private static final long serialVersionUID = 1L;
-  }
-
-  interface CompactionEnv {
-    boolean isCompactionEnabled();
-
-    IteratorScope getIteratorScope();
-  }
-
-  private Map<FileRef,DataFileValue> filesToCompact;
-  private InMemoryMap imm;
-  private FileRef outputFile;
-  private boolean propogateDeletes;
-  private AccumuloConfiguration acuTableConf;
-  private CompactionEnv env;
-  private Configuration conf;
-  private VolumeManager fs;
-  protected KeyExtent extent;
-  private List<IteratorSetting> iterators;
-
-  // things to report
-  private String currentLocalityGroup = "";
-  private long startTime;
-
-  private MajorCompactionReason reason;
-  protected MinorCompactionReason mincReason;
-
-  private AtomicLong entriesRead = new AtomicLong(0);
-  private AtomicLong entriesWritten = new AtomicLong(0);
-  private DateFormat dateFormatter = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS");
-
-  private static AtomicLong nextCompactorID = new AtomicLong(0);
-
-  // a unique id to identify a compactor
-  private long compactorID = nextCompactorID.getAndIncrement();
-
-  protected volatile Thread thread;
-
-  private synchronized void setLocalityGroup(String name) {
-    this.currentLocalityGroup = name;
-  }
-
-  private void clearStats() {
-    entriesRead.set(0);
-    entriesWritten.set(0);
-  }
-
-  protected static final Set<Compactor> runningCompactions = Collections.synchronizedSet(new HashSet<Compactor>());
-
-  public static class CompactionInfo {
-
-    private Compactor compactor;
-    private String localityGroup;
-    private long entriesRead;
-    private long entriesWritten;
-
-    CompactionInfo(Compactor compactor) {
-      this.localityGroup = compactor.currentLocalityGroup;
-      this.entriesRead = compactor.entriesRead.get();
-      this.entriesWritten = compactor.entriesWritten.get();
-      this.compactor = compactor;
-    }
-
-    public long getID() {
-      return compactor.compactorID;
-    }
-
-    public KeyExtent getExtent() {
-      return compactor.getExtent();
-    }
-
-    public long getEntriesRead() {
-      return entriesRead;
-    }
-
-    public long getEntriesWritten() {
-      return entriesWritten;
-    }
-
-    public Thread getThread() {
-      return compactor.thread;
-    }
-
-    public String getOutputFile() {
-      return compactor.getOutputFile();
-    }
-
-    public ActiveCompaction toThrift() {
-
-      CompactionType type;
-
-      if (compactor.imm != null)
-        if (compactor.filesToCompact.size() > 0)
-          type = CompactionType.MERGE;
-        else
-          type = CompactionType.MINOR;
-      else if (!compactor.propogateDeletes)
-        type = CompactionType.FULL;
-      else
-        type = CompactionType.MAJOR;
-
-      CompactionReason reason;
-
-      if (compactor.imm != null)
-        switch (compactor.mincReason) {
-          case USER:
-            reason = CompactionReason.USER;
-            break;
-          case CLOSE:
-            reason = CompactionReason.CLOSE;
-            break;
-          case SYSTEM:
-          default:
-            reason = CompactionReason.SYSTEM;
-            break;
-        }
-      else
-        switch (compactor.reason) {
-          case USER:
-            reason = CompactionReason.USER;
-            break;
-          case CHOP:
-            reason = CompactionReason.CHOP;
-            break;
-          case IDLE:
-            reason = CompactionReason.IDLE;
-            break;
-          case NORMAL:
-          default:
-            reason = CompactionReason.SYSTEM;
-            break;
-        }
-
-      List<IterInfo> iiList = new ArrayList<IterInfo>();
-      Map<String,Map<String,String>> iterOptions = new HashMap<String,Map<String,String>>();
-
-      for (IteratorSetting iterSetting : compactor.iterators) {
-        iiList.add(new IterInfo(iterSetting.getPriority(), iterSetting.getIteratorClass(), iterSetting.getName()));
-        iterOptions.put(iterSetting.getName(), iterSetting.getOptions());
-      }
-      List<String> filesToCompact = new ArrayList<String>();
-      for (FileRef ref : compactor.filesToCompact.keySet())
-        filesToCompact.add(ref.toString());
-      return new ActiveCompaction(compactor.extent.toThrift(), System.currentTimeMillis() - compactor.startTime, filesToCompact,
-          compactor.outputFile.toString(), type, reason, localityGroup, entriesRead, entriesWritten, iiList, iterOptions);
-    }
-  }
-
-  public static List<CompactionInfo> getRunningCompactions() {
-    ArrayList<CompactionInfo> compactions = new ArrayList<Compactor.CompactionInfo>();
-
-    synchronized (runningCompactions) {
-      for (Compactor compactor : runningCompactions) {
-        compactions.add(new CompactionInfo(compactor));
-      }
-    }
-
-    return compactions;
-  }
-
-  Compactor(Configuration conf, VolumeManager fs, Map<FileRef,DataFileValue> files, InMemoryMap imm, FileRef outputFile, boolean propogateDeletes,
-      AccumuloConfiguration acuTableConf, KeyExtent extent, CompactionEnv env, List<IteratorSetting> iterators, MajorCompactionReason reason) {
-    this.extent = extent;
-    this.conf = conf;
-    this.fs = fs;
-    this.filesToCompact = files;
-    this.imm = imm;
-    this.outputFile = outputFile;
-    this.propogateDeletes = propogateDeletes;
-    this.acuTableConf = acuTableConf;
-    this.env = env;
-    this.iterators = iterators;
-    this.reason = reason;
-
-    startTime = System.currentTimeMillis();
-  }
-
-  Compactor(Configuration conf, VolumeManager fs, Map<FileRef,DataFileValue> files, InMemoryMap imm, FileRef outputFile, boolean propogateDeletes,
-      AccumuloConfiguration acuTableConf, KeyExtent extent, CompactionEnv env) {
-    this(conf, fs, files, imm, outputFile, propogateDeletes, acuTableConf, extent, env, new ArrayList<IteratorSetting>(), null);
-  }
-
-  public VolumeManager getFileSystem() {
-    return fs;
-  }
-
-  KeyExtent getExtent() {
-    return extent;
-  }
-
-  String getOutputFile() {
-    return outputFile.toString();
-  }
-
-  @Override
-  public CompactionStats call() throws IOException, CompactionCanceledException {
-
-    FileSKVWriter mfw = null;
-
-    CompactionStats majCStats = new CompactionStats();
-
-    boolean remove = runningCompactions.add(this);
-
-    clearStats();
-
-    String oldThreadName = Thread.currentThread().getName();
-    String newThreadName = "MajC compacting " + extent.toString() + " started " + dateFormatter.format(new Date()) + " file: " + outputFile;
-    Thread.currentThread().setName(newThreadName);
-    thread = Thread.currentThread();
-    try {
-      FileOperations fileFactory = FileOperations.getInstance();
-      FileSystem ns = this.fs.getVolumeByPath(outputFile.path()).getFileSystem();
-      mfw = fileFactory.openWriter(outputFile.path().toString(), ns, ns.getConf(), acuTableConf);
-
-      Map<String,Set<ByteSequence>> lGroups;
-      try {
-        lGroups = LocalityGroupUtil.getLocalityGroups(acuTableConf);
-      } catch (LocalityGroupConfigurationError e) {
-        throw new IOException(e);
-      }
-
-      long t1 = System.currentTimeMillis();
-
-      HashSet<ByteSequence> allColumnFamilies = new HashSet<ByteSequence>();
-
-      if (mfw.supportsLocalityGroups()) {
-        for (Entry<String,Set<ByteSequence>> entry : lGroups.entrySet()) {
-          setLocalityGroup(entry.getKey());
-          compactLocalityGroup(entry.getKey(), entry.getValue(), true, mfw, majCStats);
-          allColumnFamilies.addAll(entry.getValue());
-        }
-      }
-
-      setLocalityGroup("");
-      compactLocalityGroup(null, allColumnFamilies, false, mfw, majCStats);
-
-      long t2 = System.currentTimeMillis();
-
-      FileSKVWriter mfwTmp = mfw;
-      mfw = null; // set this to null so we do not try to close it again in finally if the close fails
-      mfwTmp.close(); // if the close fails it will cause the compaction to fail
-
-      // Verify the file, since hadoop 0.20.2 sometimes lies about the success of close()
-      try {
-        FileSKVIterator openReader = fileFactory.openReader(outputFile.path().toString(), false, ns, ns.getConf(), acuTableConf);
-        openReader.close();
-      } catch (IOException ex) {
-        log.error("Verification of successful compaction fails!!! " + extent + " " + outputFile, ex);
-        throw ex;
-      }
-
-      log.debug(String.format("Compaction %s %,d read | %,d written | %,6d entries/sec | %6.3f secs", extent, majCStats.getEntriesRead(),
-          majCStats.getEntriesWritten(), (int) (majCStats.getEntriesRead() / ((t2 - t1) / 1000.0)), (t2 - t1) / 1000.0));
-
-      majCStats.setFileSize(fileFactory.getFileSize(outputFile.path().toString(), ns, ns.getConf(), acuTableConf));
-      return majCStats;
-    } catch (IOException e) {
-      log.error(e, e);
-      throw e;
-    } catch (RuntimeException e) {
-      log.error(e, e);
-      throw e;
-    } finally {
-      Thread.currentThread().setName(oldThreadName);
-      if (remove) {
-        thread = null;
-        runningCompactions.remove(this);
-      }
-
-      try {
-        if (mfw != null) {
-          // compaction must not have finished successfully, so close its output file
-          try {
-            mfw.close();
-          } finally {
-            if (!fs.deleteRecursively(outputFile.path()))
-              if (fs.exists(outputFile.path()))
-                log.error("Unable to delete " + outputFile);
-          }
-        }
-      } catch (IOException e) {
-        log.warn(e, e);
-      } catch (RuntimeException exception) {
-        log.warn(exception, exception);
-      }
-    }
-  }
-
-  private List<SortedKeyValueIterator<Key,Value>> openMapDataFiles(String lgName, ArrayList<FileSKVIterator> readers) throws IOException {
-
-    List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<SortedKeyValueIterator<Key,Value>>(filesToCompact.size());
-
-    for (FileRef mapFile : filesToCompact.keySet()) {
-      try {
-
-        FileOperations fileFactory = FileOperations.getInstance();
-        FileSystem fs = this.fs.getVolumeByPath(mapFile.path()).getFileSystem();
-        FileSKVIterator reader;
-
-        reader = fileFactory.openReader(mapFile.path().toString(), false, fs, conf, acuTableConf);
-
-        readers.add(reader);
-
-        SortedKeyValueIterator<Key,Value> iter = new ProblemReportingIterator(extent.getTableId().toString(), mapFile.path().toString(), false, reader);
-
-        if (filesToCompact.get(mapFile).isTimeSet()) {
-          iter = new TimeSettingIterator(iter, filesToCompact.get(mapFile).getTime());
-        }
-
-        iters.add(iter);
-
-      } catch (Throwable e) {
-
-        ProblemReports.getInstance().report(new ProblemReport(extent.getTableId().toString(), ProblemType.FILE_READ, mapFile.path().toString(), e));
-
-        log.warn("Some problem opening map file " + mapFile + " " + e.getMessage(), e);
-        // failed to open some map file... close the ones that were opened
-        for (FileSKVIterator reader : readers) {
-          try {
-            reader.close();
-          } catch (Throwable e2) {
-            log.warn("Failed to close map file", e2);
-          }
-        }
-
-        readers.clear();
-
-        if (e instanceof IOException)
-          throw (IOException) e;
-        throw new IOException("Failed to open map data files", e);
-      }
-    }
-
-    return iters;
-  }
-
-  private void compactLocalityGroup(String lgName, Set<ByteSequence> columnFamilies, boolean inclusive, FileSKVWriter mfw, CompactionStats majCStats)
-      throws IOException, CompactionCanceledException {
-    ArrayList<FileSKVIterator> readers = new ArrayList<FileSKVIterator>(filesToCompact.size());
-    Span span = Trace.start("compact");
-    try {
-      long entriesCompacted = 0;
-      List<SortedKeyValueIterator<Key,Value>> iters = openMapDataFiles(lgName, readers);
-
-      if (imm != null) {
-        iters.add(imm.compactionIterator());
-      }
-
-      CountingIterator citr = new CountingIterator(new MultiIterator(iters, extent.toDataRange()), entriesRead);
-      DeletingIterator delIter = new DeletingIterator(citr, propogateDeletes);
-      ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
-
-      // if(env.getIteratorScope() )
-
-      TabletIteratorEnvironment iterEnv;
-      if (env.getIteratorScope() == IteratorScope.majc)
-        iterEnv = new TabletIteratorEnvironment(IteratorScope.majc, !propogateDeletes, acuTableConf);
-      else if (env.getIteratorScope() == IteratorScope.minc)
-        iterEnv = new TabletIteratorEnvironment(IteratorScope.minc, acuTableConf);
-      else
-        throw new IllegalArgumentException();
-
-      SortedKeyValueIterator<Key,Value> itr = iterEnv.getTopLevelIterator(IteratorUtil.loadIterators(env.getIteratorScope(), cfsi, extent, acuTableConf,
-          iterators, iterEnv));
-
-      itr.seek(extent.toDataRange(), columnFamilies, inclusive);
-
-      if (!inclusive) {
-        mfw.startDefaultLocalityGroup();
-      } else {
-        mfw.startNewLocalityGroup(lgName, columnFamilies);
-      }
-
-      Span write = Trace.start("write");
-      try {
-        while (itr.hasTop() && env.isCompactionEnabled()) {
-          mfw.append(itr.getTopKey(), itr.getTopValue());
-          itr.next();
-          entriesCompacted++;
-
-          if (entriesCompacted % 1024 == 0) {
-            // Periodically update stats, do not want to do this too often since its volatile
-            entriesWritten.addAndGet(1024);
-          }
-        }
-
-        if (itr.hasTop() && !env.isCompactionEnabled()) {
-          // cancel major compaction operation
-          try {
-            try {
-              mfw.close();
-            } catch (IOException e) {
-              log.error(e, e);
-            }
-            fs.deleteRecursively(outputFile.path());
-          } catch (Exception e) {
-            log.warn("Failed to delete Canceled compaction output file " + outputFile, e);
-          }
-          throw new CompactionCanceledException();
-        }
-
-      } finally {
-        CompactionStats lgMajcStats = new CompactionStats(citr.getCount(), entriesCompacted);
-        majCStats.add(lgMajcStats);
-        write.stop();
-      }
-
-    } finally {
-      // close sequence files opened
-      for (FileSKVIterator reader : readers) {
-        try {
-          reader.close();
-        } catch (Throwable e) {
-          log.warn("Failed to close map file", e);
-        }
-      }
-      span.stop();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
index e8958b1..017398e 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/FileManager.java
@@ -161,7 +161,7 @@ public class FileManager {
    * @param indexCache
    *          : underlying file can and should be able to handle a null cache
    */
-  FileManager(ServerConfiguration conf, VolumeManager fs, int maxOpen, BlockCache dataCache, BlockCache indexCache) {
+  public FileManager(ServerConfiguration conf, VolumeManager fs, int maxOpen, BlockCache dataCache, BlockCache indexCache) {
     
     if (maxOpen <= 0)
       throw new IllegalArgumentException("maxOpen <= 0");
@@ -481,7 +481,7 @@ public class FileManager {
       return newlyReservedReaders;
     }
     
-    synchronized List<InterruptibleIterator> openFiles(Map<FileRef,DataFileValue> files, boolean detachable) throws IOException {
+    public synchronized List<InterruptibleIterator> openFiles(Map<FileRef,DataFileValue> files, boolean detachable) throws IOException {
       
       List<FileSKVIterator> newlyReservedReaders = openFileRefs(files.keySet());
       
@@ -509,7 +509,7 @@ public class FileManager {
       return iters;
     }
     
-    synchronized void detach() {
+    public synchronized void detach() {
       
       releaseReaders(tabletReservedReaders, false);
       tabletReservedReaders.clear();
@@ -518,7 +518,7 @@ public class FileManager {
         fds.unsetIterator();
     }
     
-    synchronized void reattach() throws IOException {
+    public synchronized void reattach() throws IOException {
       if (tabletReservedReaders.size() != 0)
         throw new IllegalStateException();
       
@@ -545,13 +545,13 @@ public class FileManager {
       }
     }
     
-    synchronized void releaseOpenFiles(boolean sawIOException) {
+    public synchronized void releaseOpenFiles(boolean sawIOException) {
       releaseReaders(tabletReservedReaders, sawIOException);
       tabletReservedReaders.clear();
       dataSources.clear();
     }
     
-    synchronized int getNumOpenFiles() {
+    public synchronized int getNumOpenFiles() {
       return tabletReservedReaders.size();
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
index dc36718..3c9c32c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java
@@ -567,7 +567,7 @@ public class InMemoryMap {
     
   }
   
-  class MemoryIterator extends WrappingIterator implements InterruptibleIterator {
+  public class MemoryIterator extends WrappingIterator implements InterruptibleIterator {
     
     private AtomicBoolean closed;
     private SourceSwitchingIterator ssi;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactionReason.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactionReason.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactionReason.java
new file mode 100644
index 0000000..25cfd9b
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactionReason.java
@@ -0,0 +1,21 @@
+/*
+ * 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.tserver;
+
+public enum MinorCompactionReason {
+  USER, SYSTEM, CLOSE
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactor.java
deleted file mode 100644
index b2e84e5..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/MinorCompactor.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * 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.tserver;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Map;
-import java.util.Random;
-
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.master.state.tables.TableState;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.server.client.HdfsZooInstance;
-import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.server.fs.FileRef;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.problems.ProblemReport;
-import org.apache.accumulo.server.problems.ProblemReports;
-import org.apache.accumulo.server.problems.ProblemType;
-import org.apache.accumulo.tserver.Tablet.MinorCompactionReason;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.log4j.Logger;
-
-public class MinorCompactor extends Compactor {
-  
-  private static final Logger log = Logger.getLogger(MinorCompactor.class);
-  
-  private static final Map<FileRef,DataFileValue> EMPTY_MAP = Collections.emptyMap();
-  
-  private static Map<FileRef,DataFileValue> toFileMap(FileRef mergeFile, DataFileValue dfv) {
-    if (mergeFile == null)
-      return EMPTY_MAP;
-    
-    return Collections.singletonMap(mergeFile, dfv);
-  }
-  
-  MinorCompactor(Configuration conf, VolumeManager fs, InMemoryMap imm, FileRef mergeFile, DataFileValue dfv, FileRef outputFile, TableConfiguration acuTableConf,
-      KeyExtent extent, MinorCompactionReason mincReason) {
-    super(conf, fs, toFileMap(mergeFile, dfv), imm, outputFile, true, acuTableConf, extent, new CompactionEnv() {
-      
-      @Override
-      public boolean isCompactionEnabled() {
-        return true;
-      }
-      
-      @Override
-      public IteratorScope getIteratorScope() {
-        return IteratorScope.minc;
-      }
-    });
-    
-    super.mincReason = mincReason;
-  }
-  
-  private boolean isTableDeleting() {
-    try {
-      return Tables.getTableState(HdfsZooInstance.getInstance(), extent.getTableId().toString()) == TableState.DELETING;
-    } catch (Exception e) {
-      log.warn("Failed to determine if table " + extent.getTableId() + " was deleting ", e);
-      return false; // can not get positive confirmation that its deleting.
-    }
-  }
-  
-  @Override
-  public CompactionStats call() {
-    log.debug("Begin minor compaction " + getOutputFile() + " " + getExtent());
-    
-    // output to new MapFile with a temporary name
-    int sleepTime = 100;
-    double growthFactor = 4;
-    int maxSleepTime = 1000 * 60 * 3; // 3 minutes
-    boolean reportedProblem = false;
-    
-    runningCompactions.add(this);
-    try {
-      do {
-        try {
-          CompactionStats ret = super.call();
-          
-          // log.debug(String.format("MinC %,d recs in | %,d recs out | %,d recs/sec | %6.3f secs | %,d bytes ",map.size(), entriesCompacted,
-          // (int)(map.size()/((t2 - t1)/1000.0)), (t2 - t1)/1000.0, estimatedSizeInBytes()));
-          
-          if (reportedProblem) {
-            ProblemReports.getInstance().deleteProblemReport(getExtent().getTableId().toString(), ProblemType.FILE_WRITE, getOutputFile());
-          }
-          
-          return ret;
-        } catch (IOException e) {
-          log.warn("MinC failed (" + e.getMessage() + ") to create " + getOutputFile() + " retrying ...");
-          ProblemReports.getInstance().report(new ProblemReport(getExtent().getTableId().toString(), ProblemType.FILE_WRITE, getOutputFile(), e));
-          reportedProblem = true;
-        } catch (RuntimeException e) {
-          // if this is coming from a user iterator, it is possible that the user could change the iterator config and that the
-          // minor compaction would succeed
-          log.warn("MinC failed (" + e.getMessage() + ") to create " + getOutputFile() + " retrying ...", e);
-          ProblemReports.getInstance().report(new ProblemReport(getExtent().getTableId().toString(), ProblemType.FILE_WRITE, getOutputFile(), e));
-          reportedProblem = true;
-        } catch (CompactionCanceledException e) {
-          throw new IllegalStateException(e);
-        }
-        
-        Random random = new Random();
-        
-        int sleep = sleepTime + random.nextInt(sleepTime);
-        log.debug("MinC failed sleeping " + sleep + " ms before retrying");
-        UtilWaitThread.sleep(sleep);
-        sleepTime = (int) Math.round(Math.min(maxSleepTime, sleepTime * growthFactor));
-        
-        // clean up
-        try {
-          if (getFileSystem().exists(new Path(getOutputFile()))) {
-            getFileSystem().deleteRecursively(new Path(getOutputFile()));
-          }
-        } catch (IOException e) {
-          log.warn("Failed to delete failed MinC file " + getOutputFile() + " " + e.getMessage());
-        }
-        
-        if (isTableDeleting())
-          return new CompactionStats(0, 0);
-        
-      } while (true);
-    } finally {
-      thread = null;
-      runningCompactions.remove(this);
-    }
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/Rate.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Rate.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Rate.java
deleted file mode 100644
index b0ed9ee..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Rate.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.tserver;
-
-public class Rate {
-  private long lastCounter = -1;
-  private long lastTime = -1;
-  private double current = 0.0;
-  final double ratio;
-  
-  /**
-   * Turn a counter into an exponentially smoothed rate over time.
-   * 
-   * @param ratio
-   *          the rate at which each update influences the curve; must be (0., 1.0)
-   */
-  public Rate(double ratio) {
-    if (ratio <= 0. || ratio >= 1.0)
-      throw new IllegalArgumentException("ratio must be > 0. and < 1.0");
-    this.ratio = ratio;
-  }
-  
-  public double update(long counter) {
-    return update(System.currentTimeMillis(), counter);
-  }
-  
-  synchronized public double update(long when, long counter) {
-    if (lastCounter < 0) {
-      lastTime = when;
-      lastCounter = counter;
-      return current;
-    }
-    if (lastTime == when) {
-      throw new IllegalArgumentException("update time < last value");
-    }
-    double keep = 1. - ratio;
-    current = (keep * current + ratio * ((counter - lastCounter)) * 1000. / (when - lastTime));
-    lastTime = when;
-    lastCounter = counter;
-    return current;
-  }
-  
-  synchronized public double rate() {
-    return this.current;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/RootFiles.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/RootFiles.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/RootFiles.java
deleted file mode 100644
index f23c55d..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/RootFiles.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * 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.tserver;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Set;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.server.fs.FileRef;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.log4j.Logger;
-
-/**
- * 
- */
-public class RootFiles {
-
-  private static Logger log = Logger.getLogger(RootFiles.class);
-
-  static void prepareReplacement(VolumeManager fs, Path location, Set<FileRef> oldDatafiles, String compactName) throws IOException {
-    for (FileRef ref : oldDatafiles) {
-      Path path = ref.path();
-      Tablet.rename(fs, path, new Path(location + "/delete+" + compactName + "+" + path.getName()));
-    }
-  }
-
-  static void renameReplacement(VolumeManager fs, FileRef tmpDatafile, FileRef newDatafile) throws IOException {
-    if (fs.exists(newDatafile.path())) {
-      log.error("Target map file already exist " + newDatafile, new Exception());
-      throw new IllegalStateException("Target map file already exist " + newDatafile);
-    }
-
-    Tablet.rename(fs, tmpDatafile.path(), newDatafile.path());
-  }
-
-  static void finishReplacement(AccumuloConfiguration acuTableConf, VolumeManager fs, Path location, Set<FileRef> oldDatafiles, String compactName)
-      throws IOException {
-    // start deleting files, if we do not finish they will be cleaned
-    // up later
-    for (FileRef ref : oldDatafiles) {
-      Path path = ref.path();
-      Path deleteFile = new Path(location + "/delete+" + compactName + "+" + path.getName());
-      if (acuTableConf.getBoolean(Property.GC_TRASH_IGNORE) || !fs.moveToTrash(deleteFile))
-        fs.deleteRecursively(deleteFile);
-    }
-  }
-
-  public static void replaceFiles(AccumuloConfiguration acuTableConf, VolumeManager fs, Path location, Set<FileRef> oldDatafiles, FileRef tmpDatafile,
-      FileRef newDatafile) throws IOException {
-    String compactName = newDatafile.path().getName();
-
-    prepareReplacement(fs, location, oldDatafiles, compactName);
-    renameReplacement(fs, tmpDatafile, newDatafile);
-    finishReplacement(acuTableConf, fs, location, oldDatafiles, compactName);
-  }
-
-  public static Collection<String> cleanupReplacement(VolumeManager fs, FileStatus[] files, boolean deleteTmp) throws IOException {
-    /*
-     * called in constructor and before major compactions
-     */
-    Collection<String> goodFiles = new ArrayList<String>(files.length);
-
-    for (FileStatus file : files) {
-
-      String path = file.getPath().toString();
-      if (file.getPath().toUri().getScheme() == null) {
-        // depending on the behavior of HDFS, if list status does not return fully qualified volumes then could switch to the default volume
-        throw new IllegalArgumentException("Require fully qualified paths " + file.getPath());
-      }
-
-      String filename = file.getPath().getName();
-
-      // check for incomplete major compaction, this should only occur
-      // for root tablet
-      if (filename.startsWith("delete+")) {
-        String expectedCompactedFile = path.substring(0, path.lastIndexOf("/delete+")) + "/" + filename.split("\\+")[1];
-        if (fs.exists(new Path(expectedCompactedFile))) {
-          // compaction finished, but did not finish deleting compacted files.. so delete it
-          if (!fs.deleteRecursively(file.getPath()))
-            log.warn("Delete of file: " + file.getPath().toString() + " return false");
-          continue;
-        }
-        // compaction did not finish, so put files back
-
-        // reset path and filename for rest of loop
-        filename = filename.split("\\+", 3)[2];
-        path = path.substring(0, path.lastIndexOf("/delete+")) + "/" + filename;
-
-        Tablet.rename(fs, file.getPath(), new Path(path));
-      }
-
-      if (filename.endsWith("_tmp")) {
-        if (deleteTmp) {
-          log.warn("cleaning up old tmp file: " + path);
-          if (!fs.deleteRecursively(file.getPath()))
-            log.warn("Delete of tmp file: " + file.getPath().toString() + " return false");
-
-        }
-        continue;
-      }
-
-      if (!filename.startsWith(Constants.MAPFILE_EXTENSION + "_") && !FileOperations.getValidExtensions().contains(filename.split("\\.")[1])) {
-        log.error("unknown file in tablet" + path);
-        continue;
-      }
-
-      goodFiles.add(path);
-    }
-
-    return goodFiles;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/TConstraintViolationException.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TConstraintViolationException.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TConstraintViolationException.java
new file mode 100644
index 0000000..83fc43e
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TConstraintViolationException.java
@@ -0,0 +1,54 @@
+/*
+ * 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.tserver;
+
+import java.util.List;
+
+import org.apache.accumulo.core.constraints.Violations;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.tserver.tablet.CommitSession;
+
+public class TConstraintViolationException extends Exception {
+  private static final long serialVersionUID = 1L;
+  private final Violations violations;
+  private final List<Mutation> violators;
+  private final List<Mutation> nonViolators;
+  private final CommitSession commitSession;
+
+  public TConstraintViolationException(Violations violations, List<Mutation> violators, List<Mutation> nonViolators, CommitSession commitSession) {
+    this.violations = violations;
+    this.violators = violators;
+    this.nonViolators = nonViolators;
+    this.commitSession = commitSession;
+  }
+
+  Violations getViolations() {
+    return violations;
+  }
+
+  List<Mutation> getViolators() {
+    return violators;
+  }
+
+  List<Mutation> getNonViolators() {
+    return nonViolators;
+  }
+
+  CommitSession getCommitSession() {
+    return commitSession;
+  }
+}
\ No newline at end of file


[23/35] git commit: Merge remote-tracking branch 'origin/master' into ACCUMULO-378

Posted by el...@apache.org.
Merge remote-tracking branch 'origin/master' into ACCUMULO-378

Conflicts:
	server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
	server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
	server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java


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

Branch: refs/heads/ACCUMULO-378
Commit: e81eee7f7cd2641ffdace5af48a5027f7fcce620
Parents: 73d34ec f280e97
Author: Josh Elser <el...@apache.org>
Authored: Tue Jun 3 21:38:00 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jun 3 21:38:00 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/ZooKeeperInstance.java |    7 +-
 .../core/client/impl/ConditionalWriterImpl.java |    4 +-
 .../client/impl/InstanceOperationsImpl.java     |    3 +-
 .../accumulo/core/client/impl/Namespaces.java   |    3 +-
 .../core/client/impl/RootTabletLocator.java     |   14 +-
 .../accumulo/core/client/impl/ServerClient.java |   15 +-
 .../accumulo/core/client/impl/Tables.java       |    3 +-
 .../core/client/impl/ZookeeperLockChecker.java  |   10 +-
 .../org/apache/accumulo/core/conf/Property.java |    8 +-
 .../iterators/conf/ColumnToClassMapping.java    |    1 +
 .../accumulo/core/util/AsyncSocketAppender.java |    3 +-
 .../core/client/ZooKeeperInstanceTest.java      |  144 +
 .../core/client/impl/RootTabletLocatorTest.java |   61 +
 .../client/impl/ZookeeperLockCheckerTest.java   |   58 +
 .../core/util/AsyncSocketAppenderTest.java      |    8 +-
 docs/src/main/asciidoc/chapters/replication.txt |   21 +
 .../accumulo/fate/zookeeper/ZooCache.java       |   16 +-
 .../fate/zookeeper/ZooCacheFactory.java         |   78 +
 .../apache/accumulo/fate/zookeeper/ZooLock.java |    2 +-
 .../fate/zookeeper/ZooReaderWriter.java         |    4 -
 .../fate/zookeeper/ZooCacheFactoryTest.java     |   87 +
 .../accumulo/server/client/HdfsZooInstance.java |    3 +-
 .../server/conf/NamespaceConfiguration.java     |    3 +-
 .../server/conf/TableConfiguration.java         |    3 +-
 .../accumulo/server/conf/ZooConfiguration.java  |    5 +-
 .../accumulo/server/tablets/TabletTime.java     |    1 -
 .../zookeeper/ZooReaderWriterFactory.java       |    2 -
 .../apache/accumulo/server/AccumuloTest.java    |    1 -
 .../server/watcher/MonitorLog4jWatcherTest.java |    8 +-
 .../accumulo/tserver/CompactionStats.java       |   59 -
 .../accumulo/tserver/CompactionWatcher.java     |  110 -
 .../org/apache/accumulo/tserver/Compactor.java  |  548 ---
 .../apache/accumulo/tserver/FileManager.java    |   12 +-
 .../apache/accumulo/tserver/InMemoryMap.java    |    2 +-
 .../accumulo/tserver/MinorCompactionReason.java |   21 +
 .../apache/accumulo/tserver/MinorCompactor.java |  146 -
 .../java/org/apache/accumulo/tserver/Rate.java  |   60 -
 .../org/apache/accumulo/tserver/RootFiles.java  |  133 -
 .../tserver/TConstraintViolationException.java  |   54 +
 .../org/apache/accumulo/tserver/Tablet.java     | 3856 ------------------
 .../tserver/TabletIteratorEnvironment.java      |    8 +-
 .../apache/accumulo/tserver/TabletServer.java   |   83 +-
 .../tserver/TabletServerResourceManager.java    |   67 +-
 .../accumulo/tserver/TabletStatsKeeper.java     |    6 +
 .../apache/accumulo/tserver/log/DfsLogger.java  |   60 +-
 .../accumulo/tserver/log/LocalWALRecovery.java  |   14 +-
 .../tserver/log/TabletServerLogger.java         |    4 +-
 .../apache/accumulo/tserver/tablet/Batch.java   |   51 +
 .../accumulo/tserver/tablet/CommitSession.java  |  121 +
 .../accumulo/tserver/tablet/CompactionInfo.java |  129 +
 .../tserver/tablet/CompactionRunner.java        |   76 +
 .../tserver/tablet/CompactionStats.java         |   59 +
 .../tserver/tablet/CompactionWatcher.java       |  110 +
 .../accumulo/tserver/tablet/Compactor.java      |  424 ++
 .../tserver/tablet/CountingIterator.java        |   78 +
 .../tserver/tablet/DatafileManager.java         |  605 +++
 .../apache/accumulo/tserver/tablet/KVEntry.java |   39 +
 .../tserver/tablet/MinorCompactionTask.java     |   99 +
 .../accumulo/tserver/tablet/MinorCompactor.java |  142 +
 .../apache/accumulo/tserver/tablet/Rate.java    |   60 +
 .../accumulo/tserver/tablet/RootFiles.java      |  133 +
 .../accumulo/tserver/tablet/ScanBatch.java      |   37 +
 .../accumulo/tserver/tablet/ScanDataSource.java |  222 +
 .../accumulo/tserver/tablet/ScanOptions.java    |   82 +
 .../apache/accumulo/tserver/tablet/Scanner.java |  136 +
 .../accumulo/tserver/tablet/SplitInfo.java      |   76 +
 .../accumulo/tserver/tablet/SplitRowSpec.java   |   29 +
 .../apache/accumulo/tserver/tablet/Tablet.java  | 2581 ++++++++++++
 .../tserver/tablet/TabletClosedException.java   |   29 +
 .../tserver/tablet/TabletCommitter.java         |   51 +
 .../accumulo/tserver/tablet/TabletMemory.java   |  190 +
 .../accumulo/tserver/CountingIteratorTest.java  |    2 +-
 .../apache/accumulo/tserver/RootFilesTest.java  |  149 -
 .../accumulo/tserver/tablet/RootFilesTest.java  |  150 +
 .../test/functional/MonitorLoggingIT.java       |    1 -
 test/system/continuous/master-agitator.pl       |    3 +-
 76 files changed, 6430 insertions(+), 5253 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e81eee7f/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 8ad849b,1200fd1..59955f3
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@@ -454,26 -441,6 +454,26 @@@ public enum Property 
    GENERAL_MAVEN_PROJECT_BASEDIR(AccumuloClassLoader.MAVEN_PROJECT_BASEDIR_PROPERTY_NAME, AccumuloClassLoader.DEFAULT_MAVEN_PROJECT_BASEDIR_VALUE,
        PropertyType.ABSOLUTEPATH, "Set this to automatically add maven target/classes directories to your dynamic classpath"),
  
 +  // General properties for configuring replication
 +  REPLICATION_PREFIX("replication.", null, PropertyType.PREFIX, "Properties in this category affect the replication of data to other Accumulo instances."),
 +  REPLICATION_PEERS("replication.peer.", null, PropertyType.PREFIX, "Properties in this category control what systems data can be replicated to"),
 +  REPLICATION_PEER_USER("replication.peer.user.", null, PropertyType.PREFIX, "The username to provide when authenticating with the given peer"),
 +  @Sensitive
 +  REPLICATION_PEER_PASSWORD("replication.peer.password.", null, PropertyType.PREFIX, "The password to provide when authenticating with the given peer"),
 +  REPLICATION_NAME("replication.name", "", PropertyType.STRING, "Name of this cluster with respect to replication. Used to identify this instance from other peers"),
 +  REPLICATION_MAX_WORK_QUEUE("replication.max.work.queue", "1000", PropertyType.COUNT, "Upper bound of the number of files queued for replication"),
 +  REPLICATION_WORK_ASSIGNMENT_SLEEP("replication.work.assignment.sleep", "30s", PropertyType.TIMEDURATION, "Amount of time to sleep between replication work assignment"),
 +  REPLICATION_WORKER_THREADS("replication.worker.threads", "4", PropertyType.COUNT, "Size of the threadpool that each tabletserver devotes to replicating data"),
 +  REPLICATION_RECEIPT_SERVICE_PORT("replication.receipt.service.port", "10002", PropertyType.PORT, "Listen port used by thrift service in tserver listening for replication"),
 +  REPLICATION_WORK_ATTEMPTS("replication.work.attempts", "10", PropertyType.COUNT, "Number of attempts to try to replicate some data before giving up and letting it naturally be retried later"),
-   REPLICATION_MIN_THREADS("replication.receiver.min.threads", "1", PropertyType.COUNT, "Minimum number of threads for replciation"),
-   REPLICATION_THREADCHECK("replication.receiver.threadcheck.time", "5s", PropertyType.TIMEDURATION, "The time between adjustments of the replication thread pool."),
++  REPLICATION_MIN_THREADS("replication.receiver.min.threads", "1", PropertyType.COUNT, "Minimum number of threads for replication"),
++  REPLICATION_THREADCHECK("replication.receiver.threadcheck.time", "30s", PropertyType.TIMEDURATION, "The time between adjustments of the replication thread pool."),
 +  REPLICATION_MAX_UNIT_SIZE("replication.max.unit.size", "64M", PropertyType.MEMORY, "Maximum size of data to send in a replication message"),
 +  REPLICATION_WORK_ASSIGNER("replication.work.assigner", "org.apache.accumulo.master.replication.SequentialWorkAssigner", PropertyType.CLASSNAME,
 +      "Replication WorkAssigner implementation to use"),
-   REPLICATION_WORK_PROCESSOR_DELAY("replication.work.processor.delay", "0s", PropertyType.TIMEDURATION, "Amount of time to wait before first checking for replication work"),
-   REPLICATION_WORK_PROCESSOR_PERIOD("replication.work.processor.period", "0s", PropertyType.TIMEDURATION, "Amount of time to wait before re-checking for replication work"),
++  REPLICATION_WORK_PROCESSOR_DELAY("replication.work.processor.delay", "0s", PropertyType.TIMEDURATION, "Amount of time to wait before first checking for replication work, not useful outside of tests"),
++  REPLICATION_WORK_PROCESSOR_PERIOD("replication.work.processor.period", "0s", PropertyType.TIMEDURATION, "Amount of time to wait before re-checking for replication work, not useful outside of tests"),
 +
    ;
  
    private String key, defaultValue, description;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e81eee7f/docs/src/main/asciidoc/chapters/replication.txt
----------------------------------------------------------------------
diff --cc docs/src/main/asciidoc/chapters/replication.txt
index 9f367df,0000000..dc87b62
mode 100644,000000..100644
--- a/docs/src/main/asciidoc/chapters/replication.txt
+++ b/docs/src/main/asciidoc/chapters/replication.txt
@@@ -1,184 -1,0 +1,205 @@@
 +// 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.
 +
 +== Replication
 +
 +=== Overview
 +
 +Replication is a feature of Accumulo which provides a mechanism to automatically
 +copy data to other systems, typically for the purpose of disaster recovery,
 +high availability, or geographic locality. It is best to consider this feature
 +as a framework for automatic replication instead of the ability to copy data
 +from to another Accumulo instance as copying to another Accumulo cluster is
 +only an implementation detail. The local Accumulo cluster is hereby referred
 +to as the +primary+ while systems being replicated to are known as
 ++peers+.
 +
 +This replication framework makes two Accumulo instances, where one instance
 +replicates to another, eventually consistent between one another, as opposed
 +to the strong consistency that each single Accumulo instance still holds. That
 +is to say, attempts to read data from a table on a peer which has pending replication
 +from the primary will not wait for that data to be replicated before running the scan.
 +This is desirable for a number of reasons, the most important is that the replication
 +framework is not limited by network outages or offline peers, but only by the HDFS
 +space available on the primary system.
 +
 +Replication configurations can be considered as a directed graph which allows cycles.
 +The systems in which data was replicated from is maintained in each Mutation which
 +allow each system to determine if a peer has already has the data in which
 +the system wants to send.
 +
 +Data is replicated by using the Write-Ahead logs (WAL) that each TabletServer is
 +already maintaining. TabletServers records which WALs have data that need to be
 +replicated to the +accumulo.metadata+ table. The Master uses these records,
 +combined with the local Accumulo table that the WAL was used with, to create records
 +in the +replication+ table which track which peers the given WAL should be
 +replicated to. The Master latter uses these work entries to assign the actual
 +replication task to a local TabletServer using ZooKeeper. A TabletServer will get
 +a lock in ZooKeeper for the replication of this file to a peer, and proceed to
 +replicate to the peer, recording progress in the +replication+ table as
 +data is successfully replicated on the peer. Later, the Master and Garbage Collector
 +will remove records from the +accumulo.metadata+ and +replication+ tables
 +and files from HDFS, respectively, after replication to all peers is complete.
 +
 +=== Configuration
 +
 +Configuration of Accumulo to replicate data to another system can be categorized
 +into the following sections.
 +
 +==== Site Configuration
 +
 +Each system involved in replication (even the primary) needs a name that uniquely
 +identifies it across all peers in the replication graph. This should be considered
 +fixed for an instance, and set in +accumulo-site.xml+.
 +
 +----
 +<property>
 +    <name>replication.name</name>
 +    <value>primary</value>
 +    <description>Unique name for this system used by replication</description>
 +</property>
 +----
 +
 +==== Instance Configuration
 +
 +For each peer of this system, Accumulo needs to know the name of that peer,
 +the class used to replicate data to that system and some configuration information
 +to connect to this remote peer. In the case of Accumulo, this additional data
 +is the Accumulo instance name and ZooKeeper quorum; however, this varies on the
 +replication implementation for the peer.
 +
 +These can be set in the site configuration to ease deployments; however, as they may
 +change, it can be useful to set this information using the Accumulo shell.
 +
 +To configure a peer with the name +peer1+ which is an Accumulo system with an instance name of +accumulo_peer+
 +and a ZooKeeper quorum of +10.0.0.1,10.0.2.1,10.0.3.1+, invoke the following
 +command in the shell.
 +
 +----
 +root@accumulo_primary> config -s
 +replication.peer.peer1=org.apache.accumulo.tserver.replication.AccumuloReplicaSystem,accumulo_peer,10.0.0.1,10.0.2.1,10.0.3.1
 +----
 +
 +Since this is an Accumulo system, we also want to set a username and password
 +to use when authenticating with this peer. On our peer, we make a special user
 +which has permission to write to the tables we want to replicate data into, "replication"
 +with a password of "password". We then need to record this in the primary's configuration.
 +
 +----
 +root@accumulo_primary> config -s replication.peer.user.peer1=replication
 +root@accumulo_primary> config -s replication.peer.password.peer1=password
 +----
 +
 +==== Table Configuration
 +
 +Now, we presently have a peer defined, so we just need to configure which tables will
 +replicate to that peer. We also need to configure an identifier to determine where
 +this data will be replicated on the peer. Since we're replicating to another Accumulo
 +cluster, this is a table ID. In this example, we want to enable replication on
 ++my_table+ and configure our peer +accumulo_peer+ as a target, sending
 +the data to the table with an ID of +2+ in +accumulo_peer+.
 +
 +\begingroup\fontsize{8pt}{8pt}\selectfont\begin{verbatim}
 +root@accumulo_primary> config -t my_table -s table.replication=true
 +root@accumulo_primary> config -t my_table -s table.replication.target.acccumulo_peer=2
 +\end{verbatim}\endgroup
 +
 +To replicate a single table on the primary to multiple peers, the second command
 +in the above shell snippet can be issued, for each peer and remote identifier pair.
 +
 +=== Monitoring
 +
 +Basic information about replication status from a primary can be found on the Accumulo
 +Monitor server, using the +Replication+ link the sidebar.
 +
 +On this page, information is broken down into the following sections:
 +
 +1. Files pending replication by peer and target
 +2. Files queued for replication, with progress made
 +
 +=== Work Assignment
 +
 +Depending on the schema of a table, different implementations of the WorkAssigner used could
 +be configured. The implementation is controlled via the property +replication.work.assigner+
 +and the full class name for the implementation. This can be configured via the shell or
 ++accumulo-site.xml+.
 +
 +----
 +<property>
 +    <name>replication.work.assigner</name>
 +    <value>org.apache.accumulo.master.replication.SequentialWorkAssigner</value>
 +    <description>Implementation used to assign work for replication</description>
 +</property>
 +----
 +
 +----
 +root@accumulo_primary> config -t my_table -s replication.work.assigner=org.apache.accumulo.master.replication.SequentialWorkAssigner
 +----
 +
 +Two implementations are provided. By default, the +SequentialWorkAssigner+ is configured for an
 +instance. The SequentialWorkAssigner ensures that, per peer and each remote identifier, each WAL is
 +replicated in the order in which they were created. This is sufficient to ensure that updates to a table
 +will be replayed in the correct order on the peer. This implementation has the downside of only replicating
 +a single WAL at a time.
 +
 +The second implementation, the +UnorderedWorkAssigner+ can be used to overcome the limitation
 +of only a single WAL being replicated to a target and peer at any time. Depending on the table schema,
 +it's possible that multiple versions of the same Key with different values are infrequent or nonexistent.
 +In this case, parallel replication to a peer and target is possible without any downsides. In the case
 +where this implementation is used were column updates are frequent, it is possible that there will be
 +an inconsistency between the primary and the peer.
 +
 +=== ReplicaSystems
 +
 ++ReplicaSystem+ is the interface which allows abstraction of replication of data
 +to peers of various types. Presently, only an +AccumuloReplicaSystem+ is provided
 +which will replicate data to another Accumulo instance. A +ReplicaSystem+ implementation
 +is run inside of the TabletServer process, and can be configured as mentioned in the 
 ++Instance Configuration+ section of this document. Theoretically, an implementation
 +of this interface could send data to other filesystems, databases, etc.
 +
 +==== AccumuloReplicaSystem
 +
 +The +AccumuloReplicaSystem+ uses Thrift to communicate with a peer Accumulo instance
 +and replicate the necessary data. The TabletServer running on the primary will communicate
 +with the Master on the peer to request the address of a TabletServer on the peer which
 +this TabletServer will use to replicate the data.
 +
 +The TabletServer on the primary will then replicate data in batches of a configurable
 +size (+replication.max.unit.size+). The TabletServer on the peer will report how many
 +records were applied back to the primary, which will be used to record how many records
 +were successfully replicated. The TabletServer on the primary will continue to replicate
 +data in these batches until no more data can be read from the file.
++
++=== Other Configuration
++
++There are a number of configuration values that can be used to control how
++the implementation of various components operate.
++
++[width="75%",cols=">,^2,^2"]
++[options="header"]
++|====
++|Property | Description | Default
++|replication.max.work.queue | Maximum number of files queued for replication at one time | 1000
++|replication.work.assignment.sleep | Time between invocations of the WorkAssigner | 30s
++|replication.worker.threads | Size of threadpool used to replicate data to peers | 4
++|replication.receipt.service.port | Thrift service port to listen for replication requests, can use '0' for a random port | 10002
++|replication.work.attempts | Number of attempts to replicate to a peer before aborting the attempt | 10
++|replication.receiver.min.threads | Minimum number of idle threads for handling incoming replication | 1
++|replication.receiver.threadcheck.time | Time between attempting adjustments of thread pool for incoming replications | 30s
++|replication.max.unit.size | Maximum amount of data to be replicated in one RPC | 64M
++|replication.work.assigner | Work Assigner implementation | org.apache.accumulo.master.replication.SequentialWorkAssigner
++|tserver.replication.batchwriter.replayer.memory| Size of BatchWriter cache to use in applying replication requests | 50M
++|====

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e81eee7f/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index e4c7ef9,2a453a8..689557c
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@@ -223,8 -211,17 +213,19 @@@ import org.apache.accumulo.tserver.metr
  import org.apache.accumulo.tserver.metrics.TabletServerMinCMetrics;
  import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
  import org.apache.accumulo.tserver.metrics.TabletServerUpdateMetrics;
 +import org.apache.accumulo.tserver.replication.ReplicationServicerHandler;
 +import org.apache.accumulo.tserver.replication.ReplicationWorker;
+ import org.apache.accumulo.tserver.tablet.CommitSession;
+ import org.apache.accumulo.tserver.tablet.CompactionInfo;
+ import org.apache.accumulo.tserver.tablet.CompactionWatcher;
+ import org.apache.accumulo.tserver.tablet.Compactor;
+ import org.apache.accumulo.tserver.tablet.KVEntry;
+ import org.apache.accumulo.tserver.tablet.Tablet.LookupResult;
+ import org.apache.accumulo.tserver.tablet.ScanBatch;
+ import org.apache.accumulo.tserver.tablet.Scanner;
+ import org.apache.accumulo.tserver.tablet.SplitInfo;
+ import org.apache.accumulo.tserver.tablet.Tablet;
+ import org.apache.accumulo.tserver.tablet.TabletClosedException;
  import org.apache.commons.collections.map.LRUMap;
  import org.apache.hadoop.fs.FSError;
  import org.apache.hadoop.fs.FileSystem;
@@@ -3121,30 -3099,7 +3122,30 @@@ public class TabletServer extends Abstr
      return address;
    }
  
 +  private HostAndPort startReplicationService() throws UnknownHostException {
 +    ReplicationServicer.Iface repl = TraceWrap.service(new ReplicationServicerHandler(HdfsZooInstance.getInstance()));
 +    ReplicationServicer.Processor<ReplicationServicer.Iface> processor = new ReplicationServicer.Processor<ReplicationServicer.Iface>(repl);
 +    AccumuloConfiguration conf = getSystemConfiguration();
 +    Property maxMessageSizeProperty = (conf.get(Property.TSERV_MAX_MESSAGE_SIZE) != null ? Property.TSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
 +    ServerAddress sp = TServerUtils.startServer(conf, clientAddress.getHostText(), Property.REPLICATION_RECEIPT_SERVICE_PORT, processor,
 +        "ReplicationServicerHandler", "Replication Servicer", null, Property.REPLICATION_MIN_THREADS, Property.REPLICATION_THREADCHECK, maxMessageSizeProperty);
 +    this.replServer = sp.server;
 +    log.info("Started replication service on " + sp.address);
 +
 +    try {
 +      // The replication service is unique to the thrift service for a tserver, not just a host.
 +      // Advertise the host and port for replication service given the host and port for the tserver.
 +      ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(instance) + Constants.ZREPLICATION_TSERVERS + "/" + clientAddress.toString(),
 +          sp.address.toString().getBytes(StandardCharsets.UTF_8), NodeExistsPolicy.OVERWRITE);
 +    } catch (Exception e) {
 +      log.error("Could not advertise replication service port", e);
 +      throw new RuntimeException(e);
 +    }
 +
 +    return sp.address;
 +  }
 +
-   ZooLock getLock() {
+   public ZooLock getLock() {
      return tabletServerLock;
    }
  

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e81eee7f/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index b7b0aff,9fec437..b4f14ec
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@@ -41,9 -36,6 +41,8 @@@ import org.apache.accumulo.core.replica
  import org.apache.accumulo.core.util.UtilWaitThread;
  import org.apache.accumulo.server.conf.TableConfiguration;
  import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.security.SystemCredentials;
 +import org.apache.accumulo.server.util.ReplicationTableUtil;
- import org.apache.accumulo.tserver.Tablet.CommitSession;
  import org.apache.accumulo.tserver.TabletMutations;
  import org.apache.accumulo.tserver.TabletServer;
  import org.apache.accumulo.tserver.log.DfsLogger.LoggerOperation;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e81eee7f/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
index 0000000,2771db9..5b46b7b
mode 000000,100644..100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
@@@ -1,0 -1,581 +1,605 @@@
+ /*
+  * 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.tserver.tablet;
+ 
+ import java.io.IOException;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Map;
++import java.util.Map.Entry;
+ import java.util.Set;
+ import java.util.SortedMap;
+ import java.util.TreeMap;
+ import java.util.TreeSet;
 -import java.util.Map.Entry;
+ 
+ import org.apache.accumulo.core.client.Connector;
+ import org.apache.accumulo.core.conf.Property;
+ import org.apache.accumulo.core.data.KeyExtent;
+ import org.apache.accumulo.core.metadata.schema.DataFileValue;
++import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
++import org.apache.accumulo.core.replication.StatusUtil;
+ import org.apache.accumulo.core.security.Credentials;
+ import org.apache.accumulo.core.util.MapCounter;
+ import org.apache.accumulo.core.util.Pair;
+ import org.apache.accumulo.core.util.UtilWaitThread;
+ import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
+ import org.apache.accumulo.server.ServerConstants;
+ import org.apache.accumulo.server.client.HdfsZooInstance;
+ import org.apache.accumulo.server.fs.FileRef;
+ import org.apache.accumulo.server.fs.VolumeManager;
+ import org.apache.accumulo.server.master.state.TServerInstance;
+ import org.apache.accumulo.server.security.SystemCredentials;
+ import org.apache.accumulo.server.util.MasterMetadataUtil;
+ import org.apache.accumulo.server.util.MetadataTableUtil;
++import org.apache.accumulo.server.util.ReplicationTableUtil;
+ import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+ import org.apache.accumulo.trace.instrument.Span;
+ import org.apache.accumulo.trace.instrument.Trace;
+ import org.apache.accumulo.tserver.TLevel;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.log4j.Logger;
+ 
+ class DatafileManager {
+   private final Logger log = Logger.getLogger(DatafileManager.class);
+   // access to datafilesizes needs to be synchronized: see CompactionRunner#getNumFiles
+   private final Map<FileRef,DataFileValue> datafileSizes = Collections.synchronizedMap(new TreeMap<FileRef,DataFileValue>());
+   private final Tablet tablet;
+   
+   // ensure we only have one reader/writer of our bulk file notes at at time
+   private final Object bulkFileImportLock = new Object();
+ 
+   DatafileManager(Tablet tablet, SortedMap<FileRef,DataFileValue> datafileSizes) {
+     for (Entry<FileRef,DataFileValue> datafiles : datafileSizes.entrySet()) {
+       this.datafileSizes.put(datafiles.getKey(), datafiles.getValue());
+     }
+     this.tablet = tablet;
+   }
+ 
+   private FileRef mergingMinorCompactionFile = null;
+   private final Set<FileRef> filesToDeleteAfterScan = new HashSet<FileRef>();
+   private final Map<Long,Set<FileRef>> scanFileReservations = new HashMap<Long,Set<FileRef>>();
+   private final MapCounter<FileRef> fileScanReferenceCounts = new MapCounter<FileRef>();
+   private long nextScanReservationId = 0;
+   private boolean reservationsBlocked = false;
+ 
+   private final Set<FileRef> majorCompactingFiles = new HashSet<FileRef>();
+   
+   static void rename(VolumeManager fs, Path src, Path dst) throws IOException {
+     if (!fs.rename(src, dst)) {
+       throw new IOException("Rename " + src + " to " + dst + " returned false ");
+     }
+   }
+ 
+   Pair<Long,Map<FileRef,DataFileValue>> reserveFilesForScan() {
+     synchronized (tablet) {
+ 
+       while (reservationsBlocked) {
+         try {
+           tablet.wait(50);
+         } catch (InterruptedException e) {
+           log.warn(e, e);
+         }
+       }
+ 
+       Set<FileRef> absFilePaths = new HashSet<FileRef>(datafileSizes.keySet());
+ 
+       long rid = nextScanReservationId++;
+ 
+       scanFileReservations.put(rid, absFilePaths);
+ 
+       Map<FileRef,DataFileValue> ret = new HashMap<FileRef,DataFileValue>();
+ 
+       for (FileRef path : absFilePaths) {
+         fileScanReferenceCounts.increment(path, 1);
+         ret.put(path, datafileSizes.get(path));
+       }
+ 
+       return new Pair<Long,Map<FileRef,DataFileValue>>(rid, ret);
+     }
+   }
+ 
+   void returnFilesForScan(Long reservationId) {
+ 
+     final Set<FileRef> filesToDelete = new HashSet<FileRef>();
+ 
+     synchronized (tablet) {
+       Set<FileRef> absFilePaths = scanFileReservations.remove(reservationId);
+ 
+       if (absFilePaths == null)
+         throw new IllegalArgumentException("Unknown scan reservation id " + reservationId);
+ 
+       boolean notify = false;
+       for (FileRef path : absFilePaths) {
+         long refCount = fileScanReferenceCounts.decrement(path, 1);
+         if (refCount == 0) {
+           if (filesToDeleteAfterScan.remove(path))
+             filesToDelete.add(path);
+           notify = true;
+         } else if (refCount < 0)
+           throw new IllegalStateException("Scan ref count for " + path + " is " + refCount);
+       }
+ 
+       if (notify)
+         tablet.notifyAll();
+     }
+ 
+     if (filesToDelete.size() > 0) {
+       log.debug("Removing scan refs from metadata " + tablet.getExtent() + " " + filesToDelete);
+       MetadataTableUtil.removeScanFiles(tablet.getExtent(), filesToDelete, SystemCredentials.get(), tablet.getTabletServer().getLock());
+     }
+   }
+ 
+   void removeFilesAfterScan(Set<FileRef> scanFiles) {
+     if (scanFiles.size() == 0)
+       return;
+ 
+     Set<FileRef> filesToDelete = new HashSet<FileRef>();
+ 
+     synchronized (tablet) {
+       for (FileRef path : scanFiles) {
+         if (fileScanReferenceCounts.get(path) == 0)
+           filesToDelete.add(path);
+         else
+           filesToDeleteAfterScan.add(path);
+       }
+     }
+ 
+     if (filesToDelete.size() > 0) {
+       log.debug("Removing scan refs from metadata " + tablet.getExtent() + " " + filesToDelete);
+       MetadataTableUtil.removeScanFiles(tablet.getExtent(), filesToDelete, SystemCredentials.get(), tablet.getTabletServer().getLock());
+     }
+   }
+ 
+   private TreeSet<FileRef> waitForScansToFinish(Set<FileRef> pathsToWaitFor, boolean blockNewScans, long maxWaitTime) {
+     long startTime = System.currentTimeMillis();
+     TreeSet<FileRef> inUse = new TreeSet<FileRef>();
+ 
+     Span waitForScans = Trace.start("waitForScans");
+     try {
+       synchronized (tablet) {
+         if (blockNewScans) {
+           if (reservationsBlocked)
+             throw new IllegalStateException();
+ 
+           reservationsBlocked = true;
+         }
+ 
+         for (FileRef path : pathsToWaitFor) {
+           while (fileScanReferenceCounts.get(path) > 0 && System.currentTimeMillis() - startTime < maxWaitTime) {
+             try {
+               tablet.wait(100);
+             } catch (InterruptedException e) {
+               log.warn(e, e);
+             }
+           }
+         }
+ 
+         for (FileRef path : pathsToWaitFor) {
+           if (fileScanReferenceCounts.get(path) > 0)
+             inUse.add(path);
+         }
+ 
+         if (blockNewScans) {
+           reservationsBlocked = false;
+           tablet.notifyAll();
+         }
+ 
+       }
+     } finally {
+       waitForScans.stop();
+     }
+     return inUse;
+   }
+ 
+   public void importMapFiles(long tid, Map<FileRef,DataFileValue> pathsString, boolean setTime) throws IOException {
+ 
+     final KeyExtent extent = tablet.getExtent();
+     String bulkDir = null;
+ 
+     Map<FileRef,DataFileValue> paths = new HashMap<FileRef,DataFileValue>();
+     for (Entry<FileRef,DataFileValue> entry : pathsString.entrySet())
+       paths.put(entry.getKey(), entry.getValue());
+ 
+     for (FileRef tpath : paths.keySet()) {
+ 
+       boolean inTheRightDirectory = false;
+       Path parent = tpath.path().getParent().getParent();
+       for (String tablesDir : ServerConstants.getTablesDirs()) {
+         if (parent.equals(new Path(tablesDir, tablet.getExtent().getTableId().toString()))) {
+           inTheRightDirectory = true;
+           break;
+         }
+       }
+       if (!inTheRightDirectory) {
+         throw new IOException("Data file " + tpath + " not in table dirs");
+       }
+ 
+       if (bulkDir == null)
+         bulkDir = tpath.path().getParent().toString();
+       else if (!bulkDir.equals(tpath.path().getParent().toString()))
+         throw new IllegalArgumentException("bulk files in different dirs " + bulkDir + " " + tpath);
+ 
+     }
+ 
+     if (tablet.getExtent().isRootTablet()) {
+       throw new IllegalArgumentException("Can not import files to root tablet");
+     }
+ 
+     synchronized (bulkFileImportLock) {
+       Credentials creds = SystemCredentials.get();
+       Connector conn;
+       try {
+         conn = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), creds.getToken());
+       } catch (Exception ex) {
+         throw new IOException(ex);
+       }
+       // Remove any bulk files we've previously loaded and compacted away
+       List<FileRef> files = MetadataTableUtil.getBulkFilesLoaded(conn, extent, tid);
+ 
+       for (FileRef file : files)
+         if (paths.keySet().remove(file))
+           log.debug("Ignoring request to re-import a file already imported: " + extent + ": " + file);
+ 
+       if (paths.size() > 0) {
+         long bulkTime = Long.MIN_VALUE;
+         if (setTime) {
+           for (DataFileValue dfv : paths.values()) {
+             long nextTime = tablet.getAndUpdateTime();
+             if (nextTime < bulkTime)
+               throw new IllegalStateException("Time went backwards unexpectedly " + nextTime + " " + bulkTime);
+             bulkTime = nextTime;
+             dfv.setTime(bulkTime);
+           }
+         }
+         
+         tablet.updatePersistedTime(bulkTime, paths, tid);
+       }
+     }
+ 
+     synchronized (tablet) {
+       for (Entry<FileRef,DataFileValue> tpath : paths.entrySet()) {
+         if (datafileSizes.containsKey(tpath.getKey())) {
+           log.error("Adding file that is already in set " + tpath.getKey());
+         }
+         datafileSizes.put(tpath.getKey(), tpath.getValue());
+ 
+       }
+ 
+       tablet.getTabletResources().importedMapFiles();
+ 
+       tablet.computeNumEntries();
+     }
+ 
+     for (Entry<FileRef,DataFileValue> entry : paths.entrySet()) {
+       log.log(TLevel.TABLET_HIST, tablet.getExtent() + " import " + entry.getKey() + " " + entry.getValue());
+     }
+   }
+ 
+   FileRef reserveMergingMinorCompactionFile() {
+     if (mergingMinorCompactionFile != null)
+       throw new IllegalStateException("Tried to reserve merging minor compaction file when already reserved  : " + mergingMinorCompactionFile);
+ 
+     if (tablet.getExtent().isRootTablet())
+       return null;
+ 
+     int maxFiles = tablet.getTableConfiguration().getMaxFilesPerTablet();
+ 
+     // when a major compaction is running and we are at max files, write out
+     // one extra file... want to avoid the case where major compaction is
+     // compacting everything except for the largest file, and therefore the
+     // largest file is returned for merging.. the following check mostly
+     // avoids this case, except for the case where major compactions fail or
+     // are canceled
+     if (majorCompactingFiles.size() > 0 && datafileSizes.size() == maxFiles)
+       return null;
+ 
+     if (datafileSizes.size() >= maxFiles) {
+       // find the smallest file
+ 
+       long min = Long.MAX_VALUE;
+       FileRef minName = null;
+ 
+       for (Entry<FileRef,DataFileValue> entry : datafileSizes.entrySet()) {
+         if (entry.getValue().getSize() < min && !majorCompactingFiles.contains(entry.getKey())) {
+           min = entry.getValue().getSize();
+           minName = entry.getKey();
+         }
+       }
+ 
+       if (minName == null)
+         return null;
+ 
+       mergingMinorCompactionFile = minName;
+       return minName;
+     }
+ 
+     return null;
+   }
+ 
+   void unreserveMergingMinorCompactionFile(FileRef file) {
+     if ((file == null && mergingMinorCompactionFile != null) || (file != null && mergingMinorCompactionFile == null)
+         || (file != null && mergingMinorCompactionFile != null && !file.equals(mergingMinorCompactionFile)))
+       throw new IllegalStateException("Disagreement " + file + " " + mergingMinorCompactionFile);
+ 
+     mergingMinorCompactionFile = null;
+   }
+ 
+   void bringMinorCompactionOnline(FileRef tmpDatafile, FileRef newDatafile, FileRef absMergeFile, DataFileValue dfv, CommitSession commitSession, long flushId)
+       throws IOException {
+ 
+     IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
+     if (tablet.getExtent().isRootTablet()) {
+       try {
+         if (!zoo.isLockHeld(tablet.getTabletServer().getLock().getLockID())) {
+           throw new IllegalStateException();
+         }
+       } catch (Exception e) {
+         throw new IllegalStateException("Can not bring major compaction online, lock not held", e);
+       }
+     }
+ 
+     // rename before putting in metadata table, so files in metadata table should
+     // always exist
+     do {
+       try {
+         if (dfv.getNumEntries() == 0) {
+           tablet.getTabletServer().getFileSystem().deleteRecursively(tmpDatafile.path());
+         } else {
+           if (tablet.getTabletServer().getFileSystem().exists(newDatafile.path())) {
+             log.warn("Target map file already exist " + newDatafile);
+             tablet.getTabletServer().getFileSystem().deleteRecursively(newDatafile.path());
+           }
+ 
+           rename(tablet.getTabletServer().getFileSystem(), tmpDatafile.path(), newDatafile.path());
+         }
+         break;
+       } catch (IOException ioe) {
+         log.warn("Tablet " + tablet.getExtent() + " failed to rename " + newDatafile + " after MinC, will retry in 60 secs...", ioe);
+         UtilWaitThread.sleep(60 * 1000);
+       }
+     } while (true);
+ 
+     long t1, t2;
+ 
+     // the code below always assumes merged files are in use by scans... this must be done
+     // because the in memory list of files is not updated until after the metadata table
+     // therefore the file is available to scans until memory is updated, but want to ensure
+     // the file is not available for garbage collection... if memory were updated
+     // before this point (like major compactions do), then the following code could wait
+     // for scans to finish like major compactions do.... used to wait for scans to finish
+     // here, but that was incorrect because a scan could start after waiting but before
+     // memory was updated... assuming the file is always in use by scans leads to
+     // one uneeded metadata update when it was not actually in use
+     Set<FileRef> filesInUseByScans = Collections.emptySet();
+     if (absMergeFile != null)
+       filesInUseByScans = Collections.singleton(absMergeFile);
+ 
+     // very important to write delete entries outside of log lock, because
+     // this metadata write does not go up... it goes sideways or to itself
+     if (absMergeFile != null)
+       MetadataTableUtil.addDeleteEntries(tablet.getExtent(), Collections.singleton(absMergeFile), SystemCredentials.get());
+ 
+     Set<String> unusedWalLogs = tablet.beginClearingUnusedLogs();
++    boolean replicate = ReplicationConfigurationUtil.isEnabled(tablet.getExtent(), tablet.getTableConfiguration());
++    Set<String> logFileOnly = null;
++    if (replicate) {
++      // unusedWalLogs is of the form host/fileURI, need to strip off the host portion
++      logFileOnly = new HashSet<>();
++      for (String unusedWalLog : unusedWalLogs) {
++        int index = unusedWalLog.indexOf('/');
++        if (-1 == index) {
++          log.warn("Could not find host component to strip from DFSLogger representation of WAL");
++        } else {
++          unusedWalLog = unusedWalLog.substring(index + 1);
++        }
++        logFileOnly.add(unusedWalLog);
++      }
++    }
+     try {
+       // the order of writing to metadata and walog is important in the face of machine/process failures
+       // need to write to metadata before writing to walog, when things are done in the reverse order
+       // data could be lost... the minor compaction start even should be written before the following metadata
+       // write is made
+ 
+       tablet.updateTabletDataFile(commitSession.getMaxCommittedTime(), newDatafile, absMergeFile, dfv, unusedWalLogs, filesInUseByScans, flushId);
+ 
++      // Mark that we have data we want to replicate
++      // This WAL could still be in use by other Tablets *from the same table*, so we can only mark that there is data to replicate,
++      // but it is *not* closed
++      if (replicate) {
++        ReplicationTableUtil.updateFiles(SystemCredentials.get(), tablet.getExtent(), logFileOnly, StatusUtil.openWithUnknownLength());
++      }
+     } finally {
+       tablet.finishClearingUnusedLogs();
+     }
+ 
+     do {
+       try {
+         // the purpose of making this update use the new commit session, instead of the old one passed in,
+         // is because the new one will reference the logs used by current memory...
+         
+         tablet.getTabletServer().minorCompactionFinished(tablet.getTabletMemory().getCommitSession(), newDatafile.toString(), commitSession.getWALogSeq() + 2);
+         break;
+       } catch (IOException e) {
+         log.error("Failed to write to write-ahead log " + e.getMessage() + " will retry", e);
+         UtilWaitThread.sleep(1 * 1000);
+       }
+     } while (true);
+ 
+     synchronized (tablet) {
+       t1 = System.currentTimeMillis();
+ 
+       if (datafileSizes.containsKey(newDatafile)) {
+         log.error("Adding file that is already in set " + newDatafile);
+       }
+       
+       if (dfv.getNumEntries() > 0) {
+         datafileSizes.put(newDatafile, dfv);
+       }
+       
+       if (absMergeFile != null) {
+         datafileSizes.remove(absMergeFile);
+       }
+       
+       unreserveMergingMinorCompactionFile(absMergeFile);
+       
+       tablet.flushComplete(flushId);
+       
+       t2 = System.currentTimeMillis();
+     }
+ 
+     // must do this after list of files in memory is updated above
+     removeFilesAfterScan(filesInUseByScans);
+ 
+     if (absMergeFile != null)
+       log.log(TLevel.TABLET_HIST, tablet.getExtent() + " MinC [" + absMergeFile + ",memory] -> " + newDatafile);
+     else
+       log.log(TLevel.TABLET_HIST, tablet.getExtent() + " MinC [memory] -> " + newDatafile);
+     log.debug(String.format("MinC finish lock %.2f secs %s", (t2 - t1) / 1000.0, tablet.getExtent().toString()));
+     long splitSize = tablet.getTableConfiguration().getMemoryInBytes(Property.TABLE_SPLIT_THRESHOLD);
+     if (dfv.getSize() > splitSize) {
+       log.debug(String.format("Minor Compaction wrote out file larger than split threshold.  split threshold = %,d  file size = %,d", splitSize, dfv.getSize()));
+     }
+   }
+ 
+   public void reserveMajorCompactingFiles(Collection<FileRef> files) {
+     if (majorCompactingFiles.size() != 0)
+       throw new IllegalStateException("Major compacting files not empty " + majorCompactingFiles);
+ 
+     if (mergingMinorCompactionFile != null && files.contains(mergingMinorCompactionFile))
+       throw new IllegalStateException("Major compaction tried to resrve file in use by minor compaction " + mergingMinorCompactionFile);
+ 
+     majorCompactingFiles.addAll(files);
+   }
+ 
+   public void clearMajorCompactingFile() {
+     majorCompactingFiles.clear();
+   }
+ 
+   void bringMajorCompactionOnline(Set<FileRef> oldDatafiles, FileRef tmpDatafile, FileRef newDatafile, Long compactionId, DataFileValue dfv)
+       throws IOException {
+     final KeyExtent extent = tablet.getExtent();
+     long t1, t2;
+ 
+     if (!extent.isRootTablet()) {
+ 
+       if (tablet.getTabletServer().getFileSystem().exists(newDatafile.path())) {
+         log.error("Target map file already exist " + newDatafile, new Exception());
+         throw new IllegalStateException("Target map file already exist " + newDatafile);
+       }
+ 
+       // rename before putting in metadata table, so files in metadata table should
+       // always exist
+       rename(tablet.getTabletServer().getFileSystem(), tmpDatafile.path(), newDatafile.path());
+ 
+       if (dfv.getNumEntries() == 0) {
+         tablet.getTabletServer().getFileSystem().deleteRecursively(newDatafile.path());
+       }
+     }
+ 
+     TServerInstance lastLocation = null;
+     synchronized (tablet) {
+ 
+       t1 = System.currentTimeMillis();
+ 
+       IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
+ 
+       tablet.incrementDataSourceDeletions();
+ 
+       if (extent.isRootTablet()) {
+ 
+         waitForScansToFinish(oldDatafiles, true, Long.MAX_VALUE);
+ 
+         try {
+           if (!zoo.isLockHeld(tablet.getTabletServer().getLock().getLockID())) {
+             throw new IllegalStateException();
+           }
+         } catch (Exception e) {
+           throw new IllegalStateException("Can not bring major compaction online, lock not held", e);
+         }
+ 
+         // mark files as ready for deletion, but
+         // do not delete them until we successfully
+         // rename the compacted map file, in case
+         // the system goes down
+ 
+         RootFiles.replaceFiles(tablet.getTableConfiguration(), tablet.getTabletServer().getFileSystem(), tablet.getLocation(), oldDatafiles, tmpDatafile, newDatafile);
+       }
+ 
+       // atomically remove old files and add new file
+       for (FileRef oldDatafile : oldDatafiles) {
+         if (!datafileSizes.containsKey(oldDatafile)) {
+           log.error("file does not exist in set " + oldDatafile);
+         }
+         datafileSizes.remove(oldDatafile);
+         majorCompactingFiles.remove(oldDatafile);
+       }
+ 
+       if (datafileSizes.containsKey(newDatafile)) {
+         log.error("Adding file that is already in set " + newDatafile);
+       }
+ 
+       if (dfv.getNumEntries() > 0) {
+         datafileSizes.put(newDatafile, dfv);
+       }
+ 
+       // could be used by a follow on compaction in a multipass compaction
+       majorCompactingFiles.add(newDatafile);
+ 
+       tablet.computeNumEntries();
+ 
+       lastLocation = tablet.resetLastLocation();
+ 
+       tablet.setLastCompactionID(compactionId);
+       t2 = System.currentTimeMillis();
+     }
+ 
+     if (!extent.isRootTablet()) {
+       Set<FileRef> filesInUseByScans = waitForScansToFinish(oldDatafiles, false, 10000);
+       if (filesInUseByScans.size() > 0)
+         log.debug("Adding scan refs to metadata " + extent + " " + filesInUseByScans);
+       MasterMetadataUtil.replaceDatafiles(extent, oldDatafiles, filesInUseByScans, newDatafile, compactionId, dfv, SystemCredentials.get(),
+           tablet.getTabletServer().getClientAddressString(), lastLocation, tablet.getTabletServer().getLock());
+       removeFilesAfterScan(filesInUseByScans);
+     }
+ 
+     log.debug(String.format("MajC finish lock %.2f secs", (t2 - t1) / 1000.0));
+     log.log(TLevel.TABLET_HIST, extent + " MajC " + oldDatafiles + " --> " + newDatafile);
+   }
+ 
+   public SortedMap<FileRef,DataFileValue> getDatafileSizes() {
+     synchronized (tablet) {
+       TreeMap<FileRef,DataFileValue> copy = new TreeMap<FileRef,DataFileValue>(datafileSizes);
+       return Collections.unmodifiableSortedMap(copy);
+     }
+   }
+ 
+   public Set<FileRef> getFiles() {
+     synchronized (tablet) {
+       HashSet<FileRef> files = new HashSet<FileRef>(datafileSizes.keySet());
+       return Collections.unmodifiableSet(files);
+     }
+   }
+   
+   public int getNumFiles() {
+     return datafileSizes.size();
+   }
+ 
+ }


[17/35] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Posted by el...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Conflicts:
	server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java


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

Branch: refs/heads/ACCUMULO-378
Commit: 60b5a1c97f09dad8cb007c250046fcca7b68cd64
Parents: 3bcea0d 1e16159
Author: Keith Turner <kt...@apache.org>
Authored: Tue Jun 3 12:59:56 2014 -0400
Committer: Keith Turner <kt...@apache.org>
Committed: Tue Jun 3 12:59:56 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/tserver/log/DfsLogger.java  | 60 +++++++++-----------
 1 file changed, 28 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/60b5a1c9/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index 71dccc0,0000000..5dabedc
mode 100644,000000..100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@@ -1,559 -1,0 +1,555 @@@
 +/*
 + * 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.tserver.log;
 +
 +import static org.apache.accumulo.tserver.logger.LogEvents.COMPACTION_FINISH;
 +import static org.apache.accumulo.tserver.logger.LogEvents.COMPACTION_START;
 +import static org.apache.accumulo.tserver.logger.LogEvents.DEFINE_TABLET;
 +import static org.apache.accumulo.tserver.logger.LogEvents.MANY_MUTATIONS;
 +import static org.apache.accumulo.tserver.logger.LogEvents.OPEN;
 +
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.io.OutputStream;
 +import java.lang.reflect.InvocationTargetException;
 +import java.lang.reflect.Method;
 +import java.nio.channels.ClosedChannelException;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Set;
 +import java.util.UUID;
 +import java.util.concurrent.CountDownLatch;
 +import java.util.concurrent.LinkedBlockingQueue;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.security.crypto.CryptoModule;
 +import org.apache.accumulo.core.security.crypto.CryptoModuleFactory;
 +import org.apache.accumulo.core.security.crypto.CryptoModuleParameters;
 +import org.apache.accumulo.core.security.crypto.DefaultCryptoModule;
 +import org.apache.accumulo.core.security.crypto.NoFlushOutputStream;
 +import org.apache.accumulo.core.util.Daemon;
++import org.apache.accumulo.core.util.LoggingRunnable;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.StringUtil;
 +import org.apache.accumulo.server.ServerConstants;
 +import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.tserver.TabletMutations;
 +import org.apache.accumulo.tserver.logger.LogFileKey;
 +import org.apache.accumulo.tserver.logger.LogFileValue;
 +import org.apache.hadoop.fs.FSDataInputStream;
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * Wrap a connection to a logger.
 + * 
 + */
 +public class DfsLogger {
 +  // Package private so that LogSorter can find this
 +  static final String LOG_FILE_HEADER_V2 = "--- Log File Header (v2) ---";
 +  static final String LOG_FILE_HEADER_V3 = "--- Log File Header (v3) ---";
 +
 +  private static Logger log = Logger.getLogger(DfsLogger.class);
 +
 +  public static class LogClosedException extends IOException {
 +    private static final long serialVersionUID = 1L;
 +
 +    public LogClosedException() {
 +      super("LogClosed");
 +    }
 +  }
 +
 +  public static class DFSLoggerInputStreams {
 +
 +    private FSDataInputStream originalInput;
 +    private DataInputStream decryptingInputStream;
 +
 +    public DFSLoggerInputStreams(FSDataInputStream originalInput, DataInputStream decryptingInputStream) {
 +      this.originalInput = originalInput;
 +      this.decryptingInputStream = decryptingInputStream;
 +    }
 +
 +    public FSDataInputStream getOriginalInput() {
 +      return originalInput;
 +    }
 +
 +    public void setOriginalInput(FSDataInputStream originalInput) {
 +      this.originalInput = originalInput;
 +    }
 +
 +    public DataInputStream getDecryptingInputStream() {
 +      return decryptingInputStream;
 +    }
 +
 +    public void setDecryptingInputStream(DataInputStream decryptingInputStream) {
 +      this.decryptingInputStream = decryptingInputStream;
 +    }
 +  }
 +
 +  public interface ServerResources {
 +    AccumuloConfiguration getConfiguration();
 +
 +    VolumeManager getFileSystem();
 +
 +    Set<TServerInstance> getCurrentTServers();
 +  }
 +
 +  private final LinkedBlockingQueue<DfsLogger.LogWork> workQueue = new LinkedBlockingQueue<DfsLogger.LogWork>();
 +
 +  private final Object closeLock = new Object();
 +
 +  private static final DfsLogger.LogWork CLOSED_MARKER = new DfsLogger.LogWork(null);
 +
 +  private static final LogFileValue EMPTY = new LogFileValue();
 +
 +  private boolean closed = false;
 +
 +  private class LogSyncingTask implements Runnable {
 +
 +    @Override
 +    public void run() {
 +      ArrayList<DfsLogger.LogWork> work = new ArrayList<DfsLogger.LogWork>();
-       while (true) {
++      boolean sawClosedMarker = false;
++      while (!sawClosedMarker) {
 +        work.clear();
 +
 +        try {
 +          work.add(workQueue.take());
 +        } catch (InterruptedException ex) {
 +          continue;
 +        }
 +        workQueue.drainTo(work);
 +
-         synchronized (closeLock) {
-           if (!closed) {
-             try {
-               sync.invoke(logFile);
-             } catch (Exception ex) {
-               log.warn("Exception syncing " + ex);
-               for (DfsLogger.LogWork logWork : work) {
-                 logWork.exception = ex;
-               }
-             }
-           } else {
-             for (DfsLogger.LogWork logWork : work) {
-               logWork.exception = new LogClosedException();
-             }
++        try {
++          sync.invoke(logFile);
++        } catch (Exception ex) {
++          log.warn("Exception syncing " + ex);
++          for (DfsLogger.LogWork logWork : work) {
++            logWork.exception = ex;
 +          }
 +        }
 +
-         boolean sawClosedMarker = false;
 +        for (DfsLogger.LogWork logWork : work)
 +          if (logWork == CLOSED_MARKER)
 +            sawClosedMarker = true;
 +          else
 +            logWork.latch.countDown();
- 
-         if (sawClosedMarker) {
-           synchronized (closeLock) {
-             closeLock.notifyAll();
-           }
-           break;
-         }
 +      }
 +    }
 +  }
 +
 +  static class LogWork {
 +    CountDownLatch latch;
 +    volatile Exception exception;
 +
 +    public LogWork(CountDownLatch latch) {
 +      this.latch = latch;
 +    }
 +  }
 +
 +  public static class LoggerOperation {
 +    private final LogWork work;
 +
 +    public LoggerOperation(LogWork work) {
 +      this.work = work;
 +    }
 +
 +    public void await() throws IOException {
 +      try {
 +        work.latch.await();
 +      } catch (InterruptedException e) {
 +        throw new RuntimeException(e);
 +      }
 +
 +      if (work.exception != null) {
 +        if (work.exception instanceof IOException)
 +          throw (IOException) work.exception;
 +        else if (work.exception instanceof RuntimeException)
 +          throw (RuntimeException) work.exception;
 +        else
 +          throw new RuntimeException(work.exception);
 +      }
 +    }
 +  }
 +
 +  @Override
 +  public boolean equals(Object obj) {
 +    // filename is unique
 +    if (obj == null)
 +      return false;
 +    if (obj instanceof DfsLogger)
 +      return getFileName().equals(((DfsLogger) obj).getFileName());
 +    return false;
 +  }
 +
 +  @Override
 +  public int hashCode() {
 +    // filename is unique
 +    return getFileName().hashCode();
 +  }
 +
 +  private final ServerResources conf;
 +  private FSDataOutputStream logFile;
 +  private DataOutputStream encryptingLogFile = null;
 +  private Method sync;
 +  private String logPath;
++  private Daemon syncThread;
 +
 +  public DfsLogger(ServerResources conf) throws IOException {
 +    this.conf = conf;
 +  }
 +
 +  public DfsLogger(ServerResources conf, String filename) throws IOException {
 +    this.conf = conf;
 +    this.logPath = filename;
 +  }
 +
 +  public static DFSLoggerInputStreams readHeaderAndReturnStream(VolumeManager fs, Path path, AccumuloConfiguration conf) throws IOException {
 +    FSDataInputStream input = fs.open(path);
 +    DataInputStream decryptingInput = null;
 +
 +    byte[] magic = DfsLogger.LOG_FILE_HEADER_V3.getBytes();
 +    byte[] magicBuffer = new byte[magic.length];
 +    input.readFully(magicBuffer);
 +    if (Arrays.equals(magicBuffer, magic)) {
 +      // additional parameters it needs from the underlying stream.
 +      String cryptoModuleClassname = input.readUTF();
 +      CryptoModule cryptoModule = CryptoModuleFactory.getCryptoModule(cryptoModuleClassname);
 +
 +      // Create the parameters and set the input stream into those parameters
 +      CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
 +      params.setEncryptedInputStream(input);
 +
 +      // Create the plaintext input stream from the encrypted one
 +      params = cryptoModule.getDecryptingInputStream(params);
 +
 +      if (params.getPlaintextInputStream() instanceof DataInputStream) {
 +        decryptingInput = (DataInputStream) params.getPlaintextInputStream();
 +      } else {
 +        decryptingInput = new DataInputStream(params.getPlaintextInputStream());
 +      }
 +    } else {
 +      input.seek(0);
 +      byte[] magicV2 = DfsLogger.LOG_FILE_HEADER_V2.getBytes();
 +      byte[] magicBufferV2 = new byte[magicV2.length];
 +      input.readFully(magicBufferV2);
 +
 +      if (Arrays.equals(magicBufferV2, magicV2)) {
 +        // Log files from 1.5 dump their options in raw to the logger files. Since we don't know the class
 +        // that needs to read those files, we can make a couple of basic assumptions. Either it's going to be
 +        // the NullCryptoModule (no crypto) or the DefaultCryptoModule.
 +
 +        // If it's null, we won't have any parameters whatsoever. First, let's attempt to read
 +        // parameters
 +        Map<String,String> opts = new HashMap<String,String>();
 +        int count = input.readInt();
 +        for (int i = 0; i < count; i++) {
 +          String key = input.readUTF();
 +          String value = input.readUTF();
 +          opts.put(key, value);
 +        }
 +
 +        if (opts.size() == 0) {
 +          // NullCryptoModule, we're done
 +          decryptingInput = input;
 +        } else {
 +
 +          // The DefaultCryptoModule will want to read the parameters from the underlying file, so we will put the file back to that spot.
 +          org.apache.accumulo.core.security.crypto.CryptoModule cryptoModule = org.apache.accumulo.core.security.crypto.CryptoModuleFactory
 +              .getCryptoModule(DefaultCryptoModule.class.getName());
 +
 +          CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
 +
 +          input.seek(0);
 +          input.readFully(magicBufferV2);
 +          params.setEncryptedInputStream(input);
 +
 +          params = cryptoModule.getDecryptingInputStream(params);
 +          if (params.getPlaintextInputStream() instanceof DataInputStream) {
 +            decryptingInput = (DataInputStream) params.getPlaintextInputStream();
 +          } else {
 +            decryptingInput = new DataInputStream(params.getPlaintextInputStream());
 +          }
 +        }
 +
 +      } else {
 +
 +        input.seek(0);
 +        decryptingInput = input;
 +      }
 +
 +    }
 +    return new DFSLoggerInputStreams(input, decryptingInput);
 +  }
 +
 +  public synchronized void open(String address) throws IOException {
 +    String filename = UUID.randomUUID().toString();
 +    String logger = StringUtil.join(Arrays.asList(address.split(":")), "+");
 +
 +    log.debug("DfsLogger.open() begin");
 +    VolumeManager fs = conf.getFileSystem();
 +
 +    logPath = fs.choose(ServerConstants.getWalDirs()) + "/" + logger + "/" + filename;
 +    try {
 +      short replication = (short) conf.getConfiguration().getCount(Property.TSERV_WAL_REPLICATION);
 +      if (replication == 0)
 +        replication = fs.getDefaultReplication(new Path(logPath));
 +      long blockSize = conf.getConfiguration().getMemoryInBytes(Property.TSERV_WAL_BLOCKSIZE);
 +      if (blockSize == 0)
 +        blockSize = (long) (conf.getConfiguration().getMemoryInBytes(Property.TSERV_WALOG_MAX_SIZE) * 1.1);
 +      if (conf.getConfiguration().getBoolean(Property.TSERV_WAL_SYNC))
 +        logFile = fs.createSyncable(new Path(logPath), 0, replication, blockSize);
 +      else
 +        logFile = fs.create(new Path(logPath), true, 0, replication, blockSize);
 +
 +      String syncMethod = conf.getConfiguration().get(Property.TSERV_WAL_SYNC_METHOD);
 +      try {
 +        NoSuchMethodException e = null;
 +        try {
 +          // hsync: send data to datanodes and sync the data to disk
 +          sync = logFile.getClass().getMethod(syncMethod);
 +        } catch (NoSuchMethodException ex) {
 +          log.warn("Could not find configured " + syncMethod + " method, trying to fall back to old Hadoop sync method", ex);
 +          e = ex;
 +        }
 +        try {
 +          // sync: send data to datanodes
 +          sync = logFile.getClass().getMethod("sync");
 +          e = null;
 +        } catch (NoSuchMethodException ex) {}
 +        if (e != null)
 +          throw new RuntimeException(e);
 +      } catch (Exception e) {
 +        throw new RuntimeException(e);
 +      }
 +
 +      // Initialize the crypto operations.
 +      org.apache.accumulo.core.security.crypto.CryptoModule cryptoModule = org.apache.accumulo.core.security.crypto.CryptoModuleFactory.getCryptoModule(conf
 +          .getConfiguration().get(Property.CRYPTO_MODULE_CLASS));
 +
 +      // Initialize the log file with a header and the crypto params used to set up this log file.
 +      logFile.write(LOG_FILE_HEADER_V3.getBytes(Constants.UTF8));
 +
 +      CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf.getConfiguration());
 +
 +      NoFlushOutputStream nfos = new NoFlushOutputStream(logFile);
 +      params.setPlaintextOutputStream(nfos);
 +
 +      // In order to bootstrap the reading of this file later, we have to record the CryptoModule that was used to encipher it here,
 +      // so that that crypto module can re-read its own parameters.
 +
 +      logFile.writeUTF(conf.getConfiguration().get(Property.CRYPTO_MODULE_CLASS));
 +
 +      params = cryptoModule.getEncryptingOutputStream(params);
 +      OutputStream encipheringOutputStream = params.getEncryptedOutputStream();
 +
 +      // If the module just kicks back our original stream, then just use it, don't wrap it in
 +      // another data OutputStream.
 +      if (encipheringOutputStream == nfos) {
 +        log.debug("No enciphering, using raw output stream");
 +        encryptingLogFile = nfos;
 +      } else {
 +        log.debug("Enciphering found, wrapping in DataOutputStream");
 +        encryptingLogFile = new DataOutputStream(encipheringOutputStream);
 +      }
 +
 +      LogFileKey key = new LogFileKey();
 +      key.event = OPEN;
 +      key.tserverSession = filename;
 +      key.filename = filename;
 +      write(key, EMPTY);
 +      sync.invoke(logFile);
 +      log.debug("Got new write-ahead log: " + this);
 +    } catch (Exception ex) {
 +      if (logFile != null)
 +        logFile.close();
 +      logFile = null;
 +      encryptingLogFile = null;
 +      throw new IOException(ex);
 +    }
 +
-     Thread t = new Daemon(new LogSyncingTask());
-     t.setName("Accumulo WALog thread " + toString());
-     t.start();
++    syncThread = new Daemon(new LoggingRunnable(log, new LogSyncingTask()));
++    syncThread.setName("Accumulo WALog thread " + toString());
++    syncThread.start();
 +  }
 +
 +  @Override
 +  public String toString() {
 +    String fileName = getFileName();
 +    if (fileName.contains(":"))
 +      return getLogger() + "/" + getFileName();
 +    return fileName;
 +  }
 +
 +  public String getFileName() {
 +    return logPath.toString();
 +  }
 +
 +  public void close() throws IOException {
 +
 +    synchronized (closeLock) {
 +      if (closed)
 +        return;
 +      // after closed is set to true, nothing else should be added to the queue
 +      // CLOSED_MARKER should be the last thing on the queue, therefore when the
 +      // background thread sees the marker and exits there should be nothing else
 +      // to process... so nothing should be left waiting for the background
 +      // thread to do work
 +      closed = true;
 +      workQueue.add(CLOSED_MARKER);
-       while (!workQueue.isEmpty())
-         try {
-           closeLock.wait();
-         } catch (InterruptedException e) {
-           log.info("Interrupted");
-         }
 +    }
 +
++    // wait for background thread to finish before closing log file
++    if(syncThread != null){
++      try {
++        syncThread.join();
++      } catch (InterruptedException e) {
++        throw new RuntimeException(e);
++      }
++    }
++
++    //expect workq should be empty at this point
++    if(workQueue.size() != 0){
++      log.error("WAL work queue not empty after sync thread exited");
++      throw new IllegalStateException("WAL work queue not empty after sync thread exited");
++    }
++    
 +    if (encryptingLogFile != null)
 +      try {
 +        logFile.close();
 +      } catch (IOException ex) {
 +        log.error(ex);
 +        throw new LogClosedException();
 +      }
 +  }
 +
 +  public synchronized void defineTablet(int seq, int tid, KeyExtent tablet) throws IOException {
 +    // write this log to the METADATA table
 +    final LogFileKey key = new LogFileKey();
 +    key.event = DEFINE_TABLET;
 +    key.seq = seq;
 +    key.tid = tid;
 +    key.tablet = tablet;
 +    try {
 +      write(key, EMPTY);
 +      sync.invoke(logFile);
 +    } catch (IllegalArgumentException e) {
 +      log.error("Signature of sync method changed. Accumulo is likely incompatible with this version of Hadoop.");
 +      throw new RuntimeException(e);
 +    } catch (IllegalAccessException e) {
 +      log.error("Could not invoke sync method due to permission error.");
 +      throw new RuntimeException(e);
 +    } catch (InvocationTargetException e) {
 +      Throwable cause = e.getCause();
 +      if (cause instanceof IOException) {
 +        throw (IOException) cause;
 +      } else if (cause instanceof RuntimeException) {
 +        throw (RuntimeException) cause;
 +      } else if (cause instanceof Error) {
 +        throw (Error) cause;
 +      } else {
 +        // Cause is null, or some other checked exception that was added later.
 +        throw new RuntimeException(e);
 +      }
 +    }
 +  }
 +
 +  private synchronized void write(LogFileKey key, LogFileValue value) throws IOException {
 +    key.write(encryptingLogFile);
 +    value.write(encryptingLogFile);
 +    encryptingLogFile.flush();
 +  }
 +
 +  public LoggerOperation log(int seq, int tid, Mutation mutation) throws IOException {
 +    return logManyTablets(Collections.singletonList(new TabletMutations(tid, seq, Collections.singletonList(mutation))));
 +  }
 +
 +  private LoggerOperation logFileData(List<Pair<LogFileKey,LogFileValue>> keys) throws IOException {
 +    DfsLogger.LogWork work = new DfsLogger.LogWork(new CountDownLatch(1));
 +    synchronized (DfsLogger.this) {
 +      try {
 +        for (Pair<LogFileKey,LogFileValue> pair : keys) {
 +          write(pair.getFirst(), pair.getSecond());
 +        }
 +      } catch (ClosedChannelException ex) {
 +        throw new LogClosedException();
 +      } catch (Exception e) {
 +        log.error(e, e);
 +        work.exception = e;
 +      }
 +    }
 +
 +    synchronized (closeLock) {
 +      // use a different lock for close check so that adding to work queue does not need
 +      // to wait on walog I/O operations
 +
 +      if (closed)
 +        throw new LogClosedException();
 +      workQueue.add(work);
 +    }
 +
 +    return new LoggerOperation(work);
 +  }
 +
 +  public LoggerOperation logManyTablets(List<TabletMutations> mutations) throws IOException {
 +    List<Pair<LogFileKey,LogFileValue>> data = new ArrayList<Pair<LogFileKey,LogFileValue>>();
 +    for (TabletMutations tabletMutations : mutations) {
 +      LogFileKey key = new LogFileKey();
 +      key.event = MANY_MUTATIONS;
 +      key.seq = tabletMutations.getSeq();
 +      key.tid = tabletMutations.getTid();
 +      LogFileValue value = new LogFileValue();
 +      value.mutations = tabletMutations.getMutations();
 +      data.add(new Pair<LogFileKey,LogFileValue>(key, value));
 +    }
 +    return logFileData(data);
 +  }
 +
 +  public LoggerOperation minorCompactionFinished(int seq, int tid, String fqfn) throws IOException {
 +    LogFileKey key = new LogFileKey();
 +    key.event = COMPACTION_FINISH;
 +    key.seq = seq;
 +    key.tid = tid;
 +    return logFileData(Collections.singletonList(new Pair<LogFileKey,LogFileValue>(key, EMPTY)));
 +  }
 +
 +  public LoggerOperation minorCompactionStarted(int seq, int tid, String fqfn) throws IOException {
 +    LogFileKey key = new LogFileKey();
 +    key.event = COMPACTION_START;
 +    key.seq = seq;
 +    key.tid = tid;
 +    key.filename = fqfn;
 +    return logFileData(Collections.singletonList(new Pair<LogFileKey,LogFileValue>(key, EMPTY)));
 +  }
 +
 +  public String getLogger() {
 +    String parts[] = logPath.split("/");
 +    return StringUtil.join(Arrays.asList(parts[parts.length - 2].split("[+]")), ":");
 +  }
 +
 +}


[30/35] git commit: ACCUMULO-378 Duplicative logging

Posted by el...@apache.org.
ACCUMULO-378 Duplicative logging


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

Branch: refs/heads/ACCUMULO-378
Commit: 98eb56f4d1e2d4186172c4e8391a3ac4993f2212
Parents: 84e94a4
Author: Josh Elser <el...@apache.org>
Authored: Wed Jun 4 16:26:25 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Jun 4 16:26:25 2014 -0400

----------------------------------------------------------------------
 .../src/main/java/org/apache/accumulo/tserver/TabletServer.java    | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/98eb56f4/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 689557c..e6286ff 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
@@ -3239,8 +3239,6 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       throw new RuntimeException("Failed to start replication service", e);
     }
 
-    log.info("Started replication service at " + replicationAddress);
-
     // Start the pool to handle outgoing replications
     final ThreadPoolExecutor replicationThreadPool = new SimpleThreadPool(getSystemConfiguration().getCount(Property.REPLICATION_WORKER_THREADS), "replication task");
     replWorker.setExecutor(replicationThreadPool);


[07/35] git commit: ACCUMULO-1961 clean up warnings as listed by eclipse

Posted by el...@apache.org.
ACCUMULO-1961 clean up warnings as listed by eclipse


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

Branch: refs/heads/ACCUMULO-378
Commit: 9e770ca3d0ad2192057c35463277bf9048c0d491
Parents: 9d8d69a
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Jun 3 10:43:36 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Jun 3 10:43:36 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/iterators/conf/ColumnToClassMapping.java   | 1 +
 .../org/apache/accumulo/core/util/AsyncSocketAppender.java   | 3 +--
 .../apache/accumulo/core/util/AsyncSocketAppenderTest.java   | 8 ++++----
 .../java/org/apache/accumulo/fate/zookeeper/ZooCache.java    | 1 -
 .../org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java  | 4 ----
 .../accumulo/server/zookeeper/ZooReaderWriterFactory.java    | 2 --
 .../test/java/org/apache/accumulo/server/AccumuloTest.java   | 1 -
 .../accumulo/server/watcher/MonitorLog4jWatcherTest.java     | 8 ++++----
 .../org/apache/accumulo/tserver/log/LocalWALRecovery.java    | 6 ++----
 .../apache/accumulo/test/functional/MonitorLoggingIT.java    | 1 -
 10 files changed, 12 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/9e770ca3/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnToClassMapping.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnToClassMapping.java b/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnToClassMapping.java
index 97f242b..062b1ce 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnToClassMapping.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnToClassMapping.java
@@ -46,6 +46,7 @@ public class ColumnToClassMapping<K> {
 	  this(objectStrings, c, null);
   }
 
+  @SuppressWarnings("unchecked")
   public ColumnToClassMapping(Map<String,String> objectStrings, Class<? extends K> c, String context) throws InstantiationException, IllegalAccessException,
   ClassNotFoundException, IOException {
 	  this();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9e770ca3/core/src/main/java/org/apache/accumulo/core/util/AsyncSocketAppender.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/AsyncSocketAppender.java b/core/src/main/java/org/apache/accumulo/core/util/AsyncSocketAppender.java
index baae9ba..07c2b6c 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/AsyncSocketAppender.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/AsyncSocketAppender.java
@@ -17,9 +17,8 @@
 package org.apache.accumulo.core.util;
 
 import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.log4j.AsyncAppender;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
 import org.apache.log4j.net.SocketAppender;
 import org.apache.log4j.spi.LoggingEvent;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9e770ca3/core/src/test/java/org/apache/accumulo/core/util/AsyncSocketAppenderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/util/AsyncSocketAppenderTest.java b/core/src/test/java/org/apache/accumulo/core/util/AsyncSocketAppenderTest.java
index 414125a..7598ef0 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/AsyncSocketAppenderTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/AsyncSocketAppenderTest.java
@@ -16,14 +16,14 @@
  */
 package org.apache.accumulo.core.util;
 
+import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
-import org.apache.log4j.Priority;
 import org.apache.log4j.net.SocketAppender;
 import org.apache.log4j.spi.LoggingEvent;
 import org.junit.Before;
 import org.junit.Test;
+
 import static org.junit.Assert.*;
-import org.easymock.Capture;
 import static org.easymock.EasyMock.*;
 
 public class AsyncSocketAppenderTest {
@@ -63,8 +63,8 @@ public class AsyncSocketAppenderTest {
   public void testAppend() {
     asa = new AsyncSocketAppender(sa);
     assertFalse(asa.isAttached(sa));
-    LoggingEvent event1 = new LoggingEvent("java.lang.String", Logger.getRootLogger(), Priority.INFO, "event1", null);
-    LoggingEvent event2 = new LoggingEvent("java.lang.Integer", Logger.getRootLogger(), Priority.WARN, "event2", null);
+    LoggingEvent event1 = new LoggingEvent("java.lang.String", Logger.getRootLogger(), Level.INFO, "event1", null);
+    LoggingEvent event2 = new LoggingEvent("java.lang.Integer", Logger.getRootLogger(), Level.WARN, "event2", null);
     sa.activateOptions();
     sa.doAppend(event1);
     sa.doAppend(event2);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9e770ca3/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
index 99ffd04..fa1e76b 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java
@@ -27,7 +27,6 @@ import java.util.ConcurrentModificationException;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9e770ca3/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
index 16f7833..1f0ae14 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
@@ -17,17 +17,13 @@
 package org.apache.accumulo.fate.zookeeper;
 
 import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
 import java.security.SecurityPermission;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.accumulo.fate.util.UtilWaitThread;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.log4j.Logger;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.BadVersionException;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9e770ca3/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriterFactory.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriterFactory.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriterFactory.java
index 1682cda..0f22f07 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriterFactory.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriterFactory.java
@@ -19,10 +19,8 @@ package org.apache.accumulo.server.zookeeper;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Proxy;
 import java.nio.charset.Charset;
-import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.fate.util.UtilWaitThread;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.RetryingInvocationHandler;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9e770ca3/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
index a090b40..9c8d1e0 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
@@ -113,7 +113,6 @@ public class AccumuloTest {
     String confDirName = confDir.getAbsolutePath();
     assertTrue("Failed to make test configuration directory", confDir.mkdir());
     try {
-      File genericXml = new File(confDir, "generic_logger.xml");
       String genericXmlName = String.format("%s/generic_logger.xml", confDirName);
 
       assertEquals(genericXmlName, Accumulo.locateLogConfig(confDirName, "flogger"));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9e770ca3/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java b/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java
index 448e0ac..53c3c6e 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/watcher/MonitorLog4jWatcherTest.java
@@ -16,14 +16,14 @@
  */
 package org.apache.accumulo.server.watcher;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
-
 import org.junit.Before;
 import org.junit.Test;
-import static org.junit.Assert.*;
-import org.easymock.Capture;
-import static org.easymock.EasyMock.*;
 
 public class MonitorLog4jWatcherTest {
   private static final String INSTANCE = "instance";

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9e770ca3/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java
index 51c13d5..868188a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java
@@ -30,8 +30,8 @@ import org.apache.accumulo.core.security.SecurityUtil;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
-import org.apache.accumulo.server.logger.LogFileKey;
-import org.apache.accumulo.server.logger.LogFileValue;
+import org.apache.accumulo.tserver.logger.LogFileKey;
+import org.apache.accumulo.tserver.logger.LogFileValue;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -137,9 +137,7 @@ public class LocalWALRecovery implements Runnable {
           continue;
         }
 
-        @SuppressWarnings("deprecation")
         LogFileKey key = new LogFileKey();
-        @SuppressWarnings("deprecation")
         LogFileValue value = new LogFileValue();
 
         log.info("Openning local log " + file.getAbsolutePath());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9e770ca3/test/src/test/java/org/apache/accumulo/test/functional/MonitorLoggingIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/MonitorLoggingIT.java b/test/src/test/java/org/apache/accumulo/test/functional/MonitorLoggingIT.java
index c4a2e3d..331a546 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/MonitorLoggingIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/MonitorLoggingIT.java
@@ -30,7 +30,6 @@ import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.monitor.Monitor;
 import org.apache.accumulo.server.util.Admin;
 import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
 import org.apache.zookeeper.KeeperException;
 import org.junit.Test;


[14/35] ACCUMULO-2041 extract tablet classes to new files, move tablet-related code to o.a.a.tserver.tablet, make member variables private

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/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
deleted file mode 100644
index fadf4ed..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ /dev/null
@@ -1,3810 +0,0 @@
-/*
- * 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.tserver;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.PriorityQueue;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.impl.ScannerImpl;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.ConfigurationObserver;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.constraints.Violations;
-import org.apache.accumulo.core.data.ByteSequence;
-import org.apache.accumulo.core.data.Column;
-import org.apache.accumulo.core.data.ColumnUpdate;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
-import org.apache.accumulo.core.data.KeyValue;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.data.thrift.IterInfo;
-import org.apache.accumulo.core.data.thrift.MapFileInfo;
-import org.apache.accumulo.core.file.FileOperations;
-import org.apache.accumulo.core.file.FileSKVIterator;
-import org.apache.accumulo.core.iterators.IterationInterruptedException;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.IteratorUtil;
-import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
-import org.apache.accumulo.core.iterators.system.ColumnQualifierFilter;
-import org.apache.accumulo.core.iterators.system.DeletingIterator;
-import org.apache.accumulo.core.iterators.system.InterruptibleIterator;
-import org.apache.accumulo.core.iterators.system.MultiIterator;
-import org.apache.accumulo.core.iterators.system.SourceSwitchingIterator;
-import org.apache.accumulo.core.iterators.system.SourceSwitchingIterator.DataSource;
-import org.apache.accumulo.core.iterators.system.StatsIterator;
-import org.apache.accumulo.core.iterators.system.VisibilityFilter;
-import org.apache.accumulo.core.master.thrift.TabletLoadState;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.metadata.schema.DataFileValue;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.Credentials;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.accumulo.core.util.LocalityGroupUtil;
-import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
-import org.apache.accumulo.core.util.MapCounter;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.core.util.UtilWaitThread;
-import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
-import org.apache.accumulo.server.ServerConstants;
-import org.apache.accumulo.server.client.HdfsZooInstance;
-import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.server.fs.FileRef;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManager.FileType;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
-import org.apache.accumulo.server.fs.VolumeUtil;
-import org.apache.accumulo.server.fs.VolumeUtil.TabletFiles;
-import org.apache.accumulo.server.master.state.TServerInstance;
-import org.apache.accumulo.server.master.tableOps.CompactionIterators;
-import org.apache.accumulo.server.problems.ProblemReport;
-import org.apache.accumulo.server.problems.ProblemReports;
-import org.apache.accumulo.server.problems.ProblemType;
-import org.apache.accumulo.server.security.SystemCredentials;
-import org.apache.accumulo.server.tablets.TabletTime;
-import org.apache.accumulo.server.tablets.UniqueNameAllocator;
-import org.apache.accumulo.server.util.FileUtil;
-import org.apache.accumulo.server.util.MasterMetadataUtil;
-import org.apache.accumulo.server.util.MetadataTableUtil;
-import org.apache.accumulo.server.util.TabletOperations;
-import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
-import org.apache.accumulo.trace.instrument.Span;
-import org.apache.accumulo.trace.instrument.Trace;
-import org.apache.accumulo.tserver.Compactor.CompactionCanceledException;
-import org.apache.accumulo.tserver.Compactor.CompactionEnv;
-import org.apache.accumulo.tserver.FileManager.ScanFileManager;
-import org.apache.accumulo.tserver.InMemoryMap.MemoryIterator;
-import org.apache.accumulo.tserver.TabletServer.TservConstraintEnv;
-import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
-import org.apache.accumulo.tserver.TabletStatsKeeper.Operation;
-import org.apache.accumulo.tserver.compaction.CompactionPlan;
-import org.apache.accumulo.tserver.compaction.CompactionStrategy;
-import org.apache.accumulo.tserver.compaction.DefaultCompactionStrategy;
-import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
-import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
-import org.apache.accumulo.tserver.compaction.WriteParameters;
-import org.apache.accumulo.tserver.constraints.ConstraintChecker;
-import org.apache.accumulo.tserver.log.DfsLogger;
-import org.apache.accumulo.tserver.log.MutationReceiver;
-import org.apache.accumulo.tserver.mastermessage.TabletStatusMessage;
-import org.apache.accumulo.tserver.metrics.TabletServerMinCMetrics;
-import org.apache.commons.codec.DecoderException;
-import org.apache.commons.codec.binary.Hex;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.log4j.Logger;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.NoNodeException;
-
-/*
- * We need to be able to have the master tell a tabletServer to
- * close this file, and the tablet server to handle all pending client reads
- * before closing
- * 
- */
-
-/**
- * 
- * this class just provides an interface to read from a MapFile mostly takes care of reporting start and end keys
- * 
- * need this because a single row extent can have multiple columns this manages all the columns (each handled by a store) for a single row-extent
- * 
- * 
- */
-
-public class Tablet {
-
-  enum MinorCompactionReason {
-    USER, SYSTEM, CLOSE
-  }
-
-  public class CommitSession {
-
-    private int seq;
-    private InMemoryMap memTable;
-    private int commitsInProgress;
-    private long maxCommittedTime = Long.MIN_VALUE;
-
-    private CommitSession(int seq, InMemoryMap imm) {
-      this.seq = seq;
-      this.memTable = imm;
-      commitsInProgress = 0;
-    }
-
-    public int getWALogSeq() {
-      return seq;
-    }
-
-    private void decrementCommitsInProgress() {
-      if (commitsInProgress < 1)
-        throw new IllegalStateException("commitsInProgress = " + commitsInProgress);
-
-      commitsInProgress--;
-      if (commitsInProgress == 0)
-        Tablet.this.notifyAll();
-    }
-
-    private void incrementCommitsInProgress() {
-      if (commitsInProgress < 0)
-        throw new IllegalStateException("commitsInProgress = " + commitsInProgress);
-
-      commitsInProgress++;
-    }
-
-    private void waitForCommitsToFinish() {
-      while (commitsInProgress > 0) {
-        try {
-          Tablet.this.wait(50);
-        } catch (InterruptedException e) {
-          log.warn(e, e);
-        }
-      }
-    }
-
-    public void abortCommit(List<Mutation> value) {
-      Tablet.this.abortCommit(this, value);
-    }
-
-    public void commit(List<Mutation> mutations) {
-      Tablet.this.commit(this, mutations);
-    }
-
-    public Tablet getTablet() {
-      return Tablet.this;
-    }
-
-    public boolean beginUpdatingLogsUsed(ArrayList<DfsLogger> copy, boolean mincFinish) {
-      return Tablet.this.beginUpdatingLogsUsed(memTable, copy, mincFinish);
-    }
-
-    public void finishUpdatingLogsUsed() {
-      Tablet.this.finishUpdatingLogsUsed();
-    }
-
-    public int getLogId() {
-      return logId;
-    }
-
-    public KeyExtent getExtent() {
-      return extent;
-    }
-
-    private void updateMaxCommittedTime(long time) {
-      maxCommittedTime = Math.max(time, maxCommittedTime);
-    }
-
-    private long getMaxCommittedTime() {
-      if (maxCommittedTime == Long.MIN_VALUE)
-        throw new IllegalStateException("Tried to read max committed time when it was never set");
-      return maxCommittedTime;
-    }
-
-  }
-
-  private class TabletMemory {
-    private InMemoryMap memTable;
-    private InMemoryMap otherMemTable;
-    private InMemoryMap deletingMemTable;
-    private int nextSeq = 1;
-    private CommitSession commitSession;
-
-    TabletMemory() {
-      try {
-        memTable = new InMemoryMap(acuTableConf);
-      } catch (LocalityGroupConfigurationError e) {
-        throw new RuntimeException(e);
-      }
-      commitSession = new CommitSession(nextSeq, memTable);
-      nextSeq += 2;
-    }
-
-    InMemoryMap getMemTable() {
-      return memTable;
-    }
-
-    InMemoryMap getMinCMemTable() {
-      return otherMemTable;
-    }
-
-    CommitSession prepareForMinC() {
-      if (otherMemTable != null) {
-        throw new IllegalStateException();
-      }
-
-      if (deletingMemTable != null) {
-        throw new IllegalStateException();
-      }
-
-      otherMemTable = memTable;
-      try {
-        memTable = new InMemoryMap(acuTableConf);
-      } catch (LocalityGroupConfigurationError e) {
-        throw new RuntimeException(e);
-      }
-
-      CommitSession oldCommitSession = commitSession;
-      commitSession = new CommitSession(nextSeq, memTable);
-      nextSeq += 2;
-
-      tabletResources.updateMemoryUsageStats(Tablet.this, memTable.estimatedSizeInBytes(), otherMemTable.estimatedSizeInBytes());
-
-      return oldCommitSession;
-    }
-
-    void finishedMinC() {
-
-      if (otherMemTable == null) {
-        throw new IllegalStateException();
-      }
-
-      if (deletingMemTable != null) {
-        throw new IllegalStateException();
-      }
-
-      deletingMemTable = otherMemTable;
-
-      otherMemTable = null;
-      Tablet.this.notifyAll();
-    }
-
-    void finalizeMinC() {
-      try {
-        deletingMemTable.delete(15000);
-      } finally {
-        synchronized (Tablet.this) {
-          if (otherMemTable != null) {
-            throw new IllegalStateException();
-          }
-
-          if (deletingMemTable == null) {
-            throw new IllegalStateException();
-          }
-
-          deletingMemTable = null;
-
-          tabletResources.updateMemoryUsageStats(Tablet.this, memTable.estimatedSizeInBytes(), 0);
-        }
-      }
-    }
-
-    boolean memoryReservedForMinC() {
-      return otherMemTable != null || deletingMemTable != null;
-    }
-
-    void waitForMinC() {
-      while (otherMemTable != null || deletingMemTable != null) {
-        try {
-          Tablet.this.wait(50);
-        } catch (InterruptedException e) {
-          log.warn(e, e);
-        }
-      }
-    }
-
-    void mutate(CommitSession cm, List<Mutation> mutations) {
-      cm.memTable.mutate(mutations);
-    }
-
-    void updateMemoryUsageStats() {
-      long other = 0;
-      if (otherMemTable != null)
-        other = otherMemTable.estimatedSizeInBytes();
-      else if (deletingMemTable != null)
-        other = deletingMemTable.estimatedSizeInBytes();
-
-      tabletResources.updateMemoryUsageStats(Tablet.this, memTable.estimatedSizeInBytes(), other);
-    }
-
-    List<MemoryIterator> getIterators() {
-      List<MemoryIterator> toReturn = new ArrayList<MemoryIterator>(2);
-      toReturn.add(memTable.skvIterator());
-      if (otherMemTable != null)
-        toReturn.add(otherMemTable.skvIterator());
-      return toReturn;
-    }
-
-    void returnIterators(List<MemoryIterator> iters) {
-      for (MemoryIterator iter : iters) {
-        iter.close();
-      }
-    }
-
-    public long getNumEntries() {
-      if (otherMemTable != null)
-        return memTable.getNumEntries() + otherMemTable.getNumEntries();
-      return memTable.getNumEntries();
-    }
-
-    CommitSession getCommitSession() {
-      return commitSession;
-    }
-  }
-
-  private TabletMemory tabletMemory;
-
-  private final TabletTime tabletTime;
-  private long persistedTime;
-  private final Object timeLock = new Object();
-
-  private final Path location; // absolute path of this tablets dir
-  private TServerInstance lastLocation;
-
-  private Configuration conf;
-  private VolumeManager fs;
-
-  private final TableConfiguration acuTableConf;
-
-  private volatile boolean tableDirChecked = false;
-
-  private AtomicLong dataSourceDeletions = new AtomicLong(0);
-  private Set<ScanDataSource> activeScans = new HashSet<ScanDataSource>();
-
-  private volatile boolean closing = false;
-  private boolean closed = false;
-  private boolean closeComplete = false;
-
-  private long lastFlushID = -1;
-  private long lastCompactID = -1;
-
-  private KeyExtent extent;
-
-  private TabletResourceManager tabletResources;
-  final private DatafileManager datafileManager;
-  private volatile boolean majorCompactionInProgress = false;
-  private volatile boolean majorCompactionWaitingToStart = false;
-  private Set<MajorCompactionReason> majorCompactionQueued = Collections.synchronizedSet(EnumSet.noneOf(MajorCompactionReason.class));
-  private volatile boolean minorCompactionInProgress = false;
-  private volatile boolean minorCompactionWaitingToStart = false;
-
-  private boolean updatingFlushID = false;
-
-  private AtomicReference<ConstraintChecker> constraintChecker = new AtomicReference<ConstraintChecker>();
-
-  private final String tabletDirectory;
-
-  private int writesInProgress = 0;
-
-  private static final Logger log = Logger.getLogger(Tablet.class);
-  public TabletStatsKeeper timer;
-
-  private Rate queryRate = new Rate(0.2);
-  private long queryCount = 0;
-
-  private Rate queryByteRate = new Rate(0.2);
-  private long queryBytes = 0;
-
-  private Rate ingestRate = new Rate(0.2);
-  private long ingestCount = 0;
-
-  private Rate ingestByteRate = new Rate(0.2);
-  private long ingestBytes = 0;
-
-  private byte[] defaultSecurityLabel = new byte[0];
-
-  private long lastMinorCompactionFinishTime;
-  private long lastMapFileImportTime;
-
-  private volatile long numEntries;
-  private volatile long numEntriesInMemory;
-
-  // a count of the amount of data read by the iterators
-  private AtomicLong scannedCount = new AtomicLong(0);
-  private Rate scannedRate = new Rate(0.2);
-
-  private ConfigurationObserver configObserver;
-
-  private TabletServer tabletServer;
-
-  private final int logId;
-  // ensure we only have one reader/writer of our bulk file notes at at time
-  public final Object bulkFileImportLock = new Object();
-
-  public int getLogId() {
-    return logId;
-  }
-
-  public static class TabletClosedException extends RuntimeException {
-    public TabletClosedException(Exception e) {
-      super(e);
-    }
-
-    public TabletClosedException() {
-      super();
-    }
-
-    private static final long serialVersionUID = 1L;
-  }
-
-  FileRef getNextMapFilename(String prefix) throws IOException {
-    String extension = FileOperations.getNewFileExtension(tabletServer.getTableConfiguration(extent));
-    checkTabletDir();
-    return new FileRef(location.toString() + "/" + prefix + UniqueNameAllocator.getInstance().getNextName() + "." + extension);
-  }
-
-  private void checkTabletDir() throws IOException {
-    if (!tableDirChecked) {
-      checkTabletDir(this.location);
-      tableDirChecked = true;
-    }
-  }
-
-  private void checkTabletDir(Path tabletDir) throws IOException {
-
-    FileStatus[] files = null;
-    try {
-      files = fs.listStatus(tabletDir);
-    } catch (FileNotFoundException ex) {
-      // ignored
-    }
-
-    if (files == null) {
-      if (tabletDir.getName().startsWith("c-"))
-        log.debug("Tablet " + extent + " had no dir, creating " + tabletDir); // its a clone dir...
-      else
-        log.warn("Tablet " + extent + " had no dir, creating " + tabletDir);
-
-      fs.mkdirs(tabletDir);
-    }
-  }
-
-  class DatafileManager {
-    // access to datafilesizes needs to be synchronized: see CompactionRunner#getNumFiles
-    final private Map<FileRef,DataFileValue> datafileSizes = Collections.synchronizedMap(new TreeMap<FileRef,DataFileValue>());
-
-    DatafileManager(SortedMap<FileRef,DataFileValue> datafileSizes) {
-      for (Entry<FileRef,DataFileValue> datafiles : datafileSizes.entrySet())
-        this.datafileSizes.put(datafiles.getKey(), datafiles.getValue());
-    }
-
-    FileRef mergingMinorCompactionFile = null;
-    Set<FileRef> filesToDeleteAfterScan = new HashSet<FileRef>();
-    Map<Long,Set<FileRef>> scanFileReservations = new HashMap<Long,Set<FileRef>>();
-    MapCounter<FileRef> fileScanReferenceCounts = new MapCounter<FileRef>();
-    long nextScanReservationId = 0;
-    boolean reservationsBlocked = false;
-
-    Set<FileRef> majorCompactingFiles = new HashSet<FileRef>();
-
-    Pair<Long,Map<FileRef,DataFileValue>> reserveFilesForScan() {
-      synchronized (Tablet.this) {
-
-        while (reservationsBlocked) {
-          try {
-            Tablet.this.wait(50);
-          } catch (InterruptedException e) {
-            log.warn(e, e);
-          }
-        }
-
-        Set<FileRef> absFilePaths = new HashSet<FileRef>(datafileSizes.keySet());
-
-        long rid = nextScanReservationId++;
-
-        scanFileReservations.put(rid, absFilePaths);
-
-        Map<FileRef,DataFileValue> ret = new HashMap<FileRef,DataFileValue>();
-
-        for (FileRef path : absFilePaths) {
-          fileScanReferenceCounts.increment(path, 1);
-          ret.put(path, datafileSizes.get(path));
-        }
-
-        return new Pair<Long,Map<FileRef,DataFileValue>>(rid, ret);
-      }
-    }
-
-    void returnFilesForScan(Long reservationId) {
-
-      final Set<FileRef> filesToDelete = new HashSet<FileRef>();
-
-      synchronized (Tablet.this) {
-        Set<FileRef> absFilePaths = scanFileReservations.remove(reservationId);
-
-        if (absFilePaths == null)
-          throw new IllegalArgumentException("Unknown scan reservation id " + reservationId);
-
-        boolean notify = false;
-        for (FileRef path : absFilePaths) {
-          long refCount = fileScanReferenceCounts.decrement(path, 1);
-          if (refCount == 0) {
-            if (filesToDeleteAfterScan.remove(path))
-              filesToDelete.add(path);
-            notify = true;
-          } else if (refCount < 0)
-            throw new IllegalStateException("Scan ref count for " + path + " is " + refCount);
-        }
-
-        if (notify)
-          Tablet.this.notifyAll();
-      }
-
-      if (filesToDelete.size() > 0) {
-        log.debug("Removing scan refs from metadata " + extent + " " + filesToDelete);
-        MetadataTableUtil.removeScanFiles(extent, filesToDelete, SystemCredentials.get(), tabletServer.getLock());
-      }
-    }
-
-    private void removeFilesAfterScan(Set<FileRef> scanFiles) {
-      if (scanFiles.size() == 0)
-        return;
-
-      Set<FileRef> filesToDelete = new HashSet<FileRef>();
-
-      synchronized (Tablet.this) {
-        for (FileRef path : scanFiles) {
-          if (fileScanReferenceCounts.get(path) == 0)
-            filesToDelete.add(path);
-          else
-            filesToDeleteAfterScan.add(path);
-        }
-      }
-
-      if (filesToDelete.size() > 0) {
-        log.debug("Removing scan refs from metadata " + extent + " " + filesToDelete);
-        MetadataTableUtil.removeScanFiles(extent, filesToDelete, SystemCredentials.get(), tabletServer.getLock());
-      }
-    }
-
-    private TreeSet<FileRef> waitForScansToFinish(Set<FileRef> pathsToWaitFor, boolean blockNewScans, long maxWaitTime) {
-      long startTime = System.currentTimeMillis();
-      TreeSet<FileRef> inUse = new TreeSet<FileRef>();
-
-      Span waitForScans = Trace.start("waitForScans");
-      try {
-        synchronized (Tablet.this) {
-          if (blockNewScans) {
-            if (reservationsBlocked)
-              throw new IllegalStateException();
-
-            reservationsBlocked = true;
-          }
-
-          for (FileRef path : pathsToWaitFor) {
-            while (fileScanReferenceCounts.get(path) > 0 && System.currentTimeMillis() - startTime < maxWaitTime) {
-              try {
-                Tablet.this.wait(100);
-              } catch (InterruptedException e) {
-                log.warn(e, e);
-              }
-            }
-          }
-
-          for (FileRef path : pathsToWaitFor) {
-            if (fileScanReferenceCounts.get(path) > 0)
-              inUse.add(path);
-          }
-
-          if (blockNewScans) {
-            reservationsBlocked = false;
-            Tablet.this.notifyAll();
-          }
-
-        }
-      } finally {
-        waitForScans.stop();
-      }
-      return inUse;
-    }
-
-    public void importMapFiles(long tid, Map<FileRef,DataFileValue> pathsString, boolean setTime) throws IOException {
-
-      String bulkDir = null;
-
-      Map<FileRef,DataFileValue> paths = new HashMap<FileRef,DataFileValue>();
-      for (Entry<FileRef,DataFileValue> entry : pathsString.entrySet())
-        paths.put(entry.getKey(), entry.getValue());
-
-      for (FileRef tpath : paths.keySet()) {
-
-        boolean inTheRightDirectory = false;
-        Path parent = tpath.path().getParent().getParent();
-        for (String tablesDir : ServerConstants.getTablesDirs()) {
-          if (parent.equals(new Path(tablesDir, extent.getTableId().toString()))) {
-            inTheRightDirectory = true;
-            break;
-          }
-        }
-        if (!inTheRightDirectory) {
-          throw new IOException("Data file " + tpath + " not in table dirs");
-        }
-
-        if (bulkDir == null)
-          bulkDir = tpath.path().getParent().toString();
-        else if (!bulkDir.equals(tpath.path().getParent().toString()))
-          throw new IllegalArgumentException("bulk files in different dirs " + bulkDir + " " + tpath);
-
-      }
-
-      if (extent.isRootTablet()) {
-        throw new IllegalArgumentException("Can not import files to root tablet");
-      }
-
-      synchronized (bulkFileImportLock) {
-        Credentials creds = SystemCredentials.get();
-        Connector conn;
-        try {
-          conn = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), creds.getToken());
-        } catch (Exception ex) {
-          throw new IOException(ex);
-        }
-        // Remove any bulk files we've previously loaded and compacted away
-        List<FileRef> files = MetadataTableUtil.getBulkFilesLoaded(conn, extent, tid);
-
-        for (FileRef file : files)
-          if (paths.keySet().remove(file))
-            log.debug("Ignoring request to re-import a file already imported: " + extent + ": " + file);
-
-        if (paths.size() > 0) {
-          long bulkTime = Long.MIN_VALUE;
-          if (setTime) {
-            for (DataFileValue dfv : paths.values()) {
-              long nextTime = tabletTime.getAndUpdateTime();
-              if (nextTime < bulkTime)
-                throw new IllegalStateException("Time went backwards unexpectedly " + nextTime + " " + bulkTime);
-              bulkTime = nextTime;
-              dfv.setTime(bulkTime);
-            }
-          }
-
-          synchronized (timeLock) {
-            if (bulkTime > persistedTime)
-              persistedTime = bulkTime;
-
-            MetadataTableUtil.updateTabletDataFile(tid, extent, paths, tabletTime.getMetadataValue(persistedTime), creds, tabletServer.getLock());
-          }
-        }
-      }
-
-      synchronized (Tablet.this) {
-        for (Entry<FileRef,DataFileValue> tpath : paths.entrySet()) {
-          if (datafileSizes.containsKey(tpath.getKey())) {
-            log.error("Adding file that is already in set " + tpath.getKey());
-          }
-          datafileSizes.put(tpath.getKey(), tpath.getValue());
-
-        }
-
-        tabletResources.importedMapFiles();
-
-        computeNumEntries();
-      }
-
-      for (Entry<FileRef,DataFileValue> entry : paths.entrySet()) {
-        log.log(TLevel.TABLET_HIST, extent + " import " + entry.getKey() + " " + entry.getValue());
-      }
-    }
-
-    FileRef reserveMergingMinorCompactionFile() {
-      if (mergingMinorCompactionFile != null)
-        throw new IllegalStateException("Tried to reserve merging minor compaction file when already reserved  : " + mergingMinorCompactionFile);
-
-      if (extent.isRootTablet())
-        return null;
-
-      int maxFiles = acuTableConf.getMaxFilesPerTablet();
-
-      // when a major compaction is running and we are at max files, write out
-      // one extra file... want to avoid the case where major compaction is
-      // compacting everything except for the largest file, and therefore the
-      // largest file is returned for merging.. the following check mostly
-      // avoids this case, except for the case where major compactions fail or
-      // are canceled
-      if (majorCompactingFiles.size() > 0 && datafileSizes.size() == maxFiles)
-        return null;
-
-      if (datafileSizes.size() >= maxFiles) {
-        // find the smallest file
-
-        long min = Long.MAX_VALUE;
-        FileRef minName = null;
-
-        for (Entry<FileRef,DataFileValue> entry : datafileSizes.entrySet()) {
-          if (entry.getValue().getSize() < min && !majorCompactingFiles.contains(entry.getKey())) {
-            min = entry.getValue().getSize();
-            minName = entry.getKey();
-          }
-        }
-
-        if (minName == null)
-          return null;
-
-        mergingMinorCompactionFile = minName;
-        return minName;
-      }
-
-      return null;
-    }
-
-    void unreserveMergingMinorCompactionFile(FileRef file) {
-      if ((file == null && mergingMinorCompactionFile != null) || (file != null && mergingMinorCompactionFile == null)
-          || (file != null && mergingMinorCompactionFile != null && !file.equals(mergingMinorCompactionFile)))
-        throw new IllegalStateException("Disagreement " + file + " " + mergingMinorCompactionFile);
-
-      mergingMinorCompactionFile = null;
-    }
-
-    void bringMinorCompactionOnline(FileRef tmpDatafile, FileRef newDatafile, FileRef absMergeFile, DataFileValue dfv, CommitSession commitSession, long flushId)
-        throws IOException {
-
-      IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
-      if (extent.isRootTablet()) {
-        try {
-          if (!zoo.isLockHeld(tabletServer.getLock().getLockID())) {
-            throw new IllegalStateException();
-          }
-        } catch (Exception e) {
-          throw new IllegalStateException("Can not bring major compaction online, lock not held", e);
-        }
-      }
-
-      // rename before putting in metadata table, so files in metadata table should
-      // always exist
-      do {
-        try {
-          if (dfv.getNumEntries() == 0) {
-            fs.deleteRecursively(tmpDatafile.path());
-          } else {
-            if (fs.exists(newDatafile.path())) {
-              log.warn("Target map file already exist " + newDatafile);
-              fs.deleteRecursively(newDatafile.path());
-            }
-
-            rename(fs, tmpDatafile.path(), newDatafile.path());
-          }
-          break;
-        } catch (IOException ioe) {
-          log.warn("Tablet " + extent + " failed to rename " + newDatafile + " after MinC, will retry in 60 secs...", ioe);
-          UtilWaitThread.sleep(60 * 1000);
-        }
-      } while (true);
-
-      long t1, t2;
-
-      // the code below always assumes merged files are in use by scans... this must be done
-      // because the in memory list of files is not updated until after the metadata table
-      // therefore the file is available to scans until memory is updated, but want to ensure
-      // the file is not available for garbage collection... if memory were updated
-      // before this point (like major compactions do), then the following code could wait
-      // for scans to finish like major compactions do.... used to wait for scans to finish
-      // here, but that was incorrect because a scan could start after waiting but before
-      // memory was updated... assuming the file is always in use by scans leads to
-      // one uneeded metadata update when it was not actually in use
-      Set<FileRef> filesInUseByScans = Collections.emptySet();
-      if (absMergeFile != null)
-        filesInUseByScans = Collections.singleton(absMergeFile);
-
-      // very important to write delete entries outside of log lock, because
-      // this metadata write does not go up... it goes sideways or to itself
-      if (absMergeFile != null)
-        MetadataTableUtil.addDeleteEntries(extent, Collections.singleton(absMergeFile), SystemCredentials.get());
-
-      Set<String> unusedWalLogs = beginClearingUnusedLogs();
-      try {
-        // the order of writing to metadata and walog is important in the face of machine/process failures
-        // need to write to metadata before writing to walog, when things are done in the reverse order
-        // data could be lost... the minor compaction start even should be written before the following metadata
-        // write is made
-
-        synchronized (timeLock) {
-          if (commitSession.getMaxCommittedTime() > persistedTime)
-            persistedTime = commitSession.getMaxCommittedTime();
-
-          String time = tabletTime.getMetadataValue(persistedTime);
-          MasterMetadataUtil.updateTabletDataFile(extent, newDatafile, absMergeFile, dfv, time, SystemCredentials.get(), filesInUseByScans,
-              tabletServer.getClientAddressString(), tabletServer.getLock(), unusedWalLogs, lastLocation, flushId);
-        }
-
-      } finally {
-        finishClearingUnusedLogs();
-      }
-
-      do {
-        try {
-          // the purpose of making this update use the new commit session, instead of the old one passed in,
-          // is because the new one will reference the logs used by current memory...
-
-          tabletServer.minorCompactionFinished(tabletMemory.getCommitSession(), newDatafile.toString(), commitSession.getWALogSeq() + 2);
-          break;
-        } catch (IOException e) {
-          log.error("Failed to write to write-ahead log " + e.getMessage() + " will retry", e);
-          UtilWaitThread.sleep(1 * 1000);
-        }
-      } while (true);
-
-      synchronized (Tablet.this) {
-        lastLocation = null;
-
-        t1 = System.currentTimeMillis();
-        if (datafileSizes.containsKey(newDatafile)) {
-          log.error("Adding file that is already in set " + newDatafile);
-        }
-
-        if (dfv.getNumEntries() > 0) {
-          datafileSizes.put(newDatafile, dfv);
-        }
-
-        if (absMergeFile != null) {
-          datafileSizes.remove(absMergeFile);
-        }
-
-        unreserveMergingMinorCompactionFile(absMergeFile);
-
-        dataSourceDeletions.incrementAndGet();
-        tabletMemory.finishedMinC();
-
-        lastFlushID = flushId;
-
-        computeNumEntries();
-        t2 = System.currentTimeMillis();
-      }
-
-      // must do this after list of files in memory is updated above
-      removeFilesAfterScan(filesInUseByScans);
-
-      if (absMergeFile != null)
-        log.log(TLevel.TABLET_HIST, extent + " MinC [" + absMergeFile + ",memory] -> " + newDatafile);
-      else
-        log.log(TLevel.TABLET_HIST, extent + " MinC [memory] -> " + newDatafile);
-      log.debug(String.format("MinC finish lock %.2f secs %s", (t2 - t1) / 1000.0, getExtent().toString()));
-      if (dfv.getSize() > acuTableConf.getMemoryInBytes(Property.TABLE_SPLIT_THRESHOLD)) {
-        log.debug(String.format("Minor Compaction wrote out file larger than split threshold.  split threshold = %,d  file size = %,d",
-            acuTableConf.getMemoryInBytes(Property.TABLE_SPLIT_THRESHOLD), dfv.getSize()));
-      }
-
-    }
-
-    public void reserveMajorCompactingFiles(Collection<FileRef> files) {
-      if (majorCompactingFiles.size() != 0)
-        throw new IllegalStateException("Major compacting files not empty " + majorCompactingFiles);
-
-      if (mergingMinorCompactionFile != null && files.contains(mergingMinorCompactionFile))
-        throw new IllegalStateException("Major compaction tried to resrve file in use by minor compaction " + mergingMinorCompactionFile);
-
-      majorCompactingFiles.addAll(files);
-    }
-
-    public void clearMajorCompactingFile() {
-      majorCompactingFiles.clear();
-    }
-
-    void bringMajorCompactionOnline(Set<FileRef> oldDatafiles, FileRef tmpDatafile, FileRef newDatafile, Long compactionId, DataFileValue dfv)
-        throws IOException {
-      long t1, t2;
-
-      if (!extent.isRootTablet()) {
-
-        if (fs.exists(newDatafile.path())) {
-          log.error("Target map file already exist " + newDatafile, new Exception());
-          throw new IllegalStateException("Target map file already exist " + newDatafile);
-        }
-
-        // rename before putting in metadata table, so files in metadata table should
-        // always exist
-        rename(fs, tmpDatafile.path(), newDatafile.path());
-
-        if (dfv.getNumEntries() == 0) {
-          fs.deleteRecursively(newDatafile.path());
-        }
-      }
-
-      TServerInstance lastLocation = null;
-      synchronized (Tablet.this) {
-
-        t1 = System.currentTimeMillis();
-
-        IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
-
-        dataSourceDeletions.incrementAndGet();
-
-        if (extent.isRootTablet()) {
-
-          waitForScansToFinish(oldDatafiles, true, Long.MAX_VALUE);
-
-          try {
-            if (!zoo.isLockHeld(tabletServer.getLock().getLockID())) {
-              throw new IllegalStateException();
-            }
-          } catch (Exception e) {
-            throw new IllegalStateException("Can not bring major compaction online, lock not held", e);
-          }
-
-          // mark files as ready for deletion, but
-          // do not delete them until we successfully
-          // rename the compacted map file, in case
-          // the system goes down
-
-          RootFiles.replaceFiles(acuTableConf, fs, location, oldDatafiles, tmpDatafile, newDatafile);
-        }
-
-        // atomically remove old files and add new file
-        for (FileRef oldDatafile : oldDatafiles) {
-          if (!datafileSizes.containsKey(oldDatafile)) {
-            log.error("file does not exist in set " + oldDatafile);
-          }
-          datafileSizes.remove(oldDatafile);
-          majorCompactingFiles.remove(oldDatafile);
-        }
-
-        if (datafileSizes.containsKey(newDatafile)) {
-          log.error("Adding file that is already in set " + newDatafile);
-        }
-
-        if (dfv.getNumEntries() > 0) {
-          datafileSizes.put(newDatafile, dfv);
-        }
-
-        // could be used by a follow on compaction in a multipass compaction
-        majorCompactingFiles.add(newDatafile);
-
-        computeNumEntries();
-
-        lastLocation = Tablet.this.lastLocation;
-        Tablet.this.lastLocation = null;
-
-        if (compactionId != null)
-          lastCompactID = compactionId;
-
-        t2 = System.currentTimeMillis();
-      }
-
-      if (!extent.isRootTablet()) {
-        Set<FileRef> filesInUseByScans = waitForScansToFinish(oldDatafiles, false, 10000);
-        if (filesInUseByScans.size() > 0)
-          log.debug("Adding scan refs to metadata " + extent + " " + filesInUseByScans);
-        MasterMetadataUtil.replaceDatafiles(extent, oldDatafiles, filesInUseByScans, newDatafile, compactionId, dfv, SystemCredentials.get(),
-            tabletServer.getClientAddressString(), lastLocation, tabletServer.getLock());
-        removeFilesAfterScan(filesInUseByScans);
-      }
-
-      log.debug(String.format("MajC finish lock %.2f secs", (t2 - t1) / 1000.0));
-      log.log(TLevel.TABLET_HIST, extent + " MajC " + oldDatafiles + " --> " + newDatafile);
-    }
-
-    public SortedMap<FileRef,DataFileValue> getDatafileSizes() {
-      synchronized (Tablet.this) {
-        TreeMap<FileRef,DataFileValue> copy = new TreeMap<FileRef,DataFileValue>(datafileSizes);
-        return Collections.unmodifiableSortedMap(copy);
-      }
-    }
-
-    public Set<FileRef> getFiles() {
-      synchronized (Tablet.this) {
-        HashSet<FileRef> files = new HashSet<FileRef>(datafileSizes.keySet());
-        return Collections.unmodifiableSet(files);
-      }
-    }
-
-  }
-
-  public Tablet(TabletServer tabletServer, Text location, KeyExtent extent, TabletResourceManager trm, SortedMap<Key,Value> tabletsKeyValues)
-      throws IOException {
-    this(tabletServer, location, extent, trm, CachedConfiguration.getInstance(), tabletsKeyValues);
-    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);
-    splitCreationTime = System.currentTimeMillis();
-  }
-
-  private Tablet(TabletServer tabletServer, Text location, KeyExtent extent, TabletResourceManager trm, Configuration conf,
-      SortedMap<Key,Value> tabletsKeyValues) throws IOException {
-    this(tabletServer, location, extent, trm, conf, VolumeManagerImpl.get(), tabletsKeyValues);
-  }
-
-  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 lastLocation) throws IOException {
-    this(tabletServer, location, extent, trm, conf, VolumeManagerImpl.get(), EMPTY, datafiles, time, lastLocation, new HashSet<FileRef>(), initFlushID,
-        initCompactID);
-  }
-
-  private static String lookupTime(AccumuloConfiguration conf, KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
-    SortedMap<Key,Value> entries;
-
-    if (extent.isRootTablet()) {
-      return null;
-    } else {
-      entries = new TreeMap<Key,Value>();
-      Text rowName = extent.getMetadataEntry();
-      for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
-        if (entry.getKey().compareRow(rowName) == 0 && TabletsSection.ServerColumnFamily.TIME_COLUMN.hasColumns(entry.getKey())) {
-          entries.put(new Key(entry.getKey()), new Value(entry.getValue()));
-        }
-      }
-    }
-
-    // log.debug("extent : "+extent+"   entries : "+entries);
-
-    if (entries.size() == 1)
-      return entries.values().iterator().next().toString();
-    return null;
-  }
-
-  private static SortedMap<FileRef,DataFileValue> lookupDatafiles(AccumuloConfiguration conf, VolumeManager fs, KeyExtent extent,
-      SortedMap<Key,Value> tabletsKeyValues) throws IOException {
-
-    TreeMap<FileRef,DataFileValue> datafiles = new TreeMap<FileRef,DataFileValue>();
-
-    if (extent.isRootTablet()) { // the meta0 tablet
-      Path location = new Path(MetadataTableUtil.getRootTabletDir());
-
-      // cleanUpFiles() has special handling for delete. files
-      FileStatus[] files = fs.listStatus(location);
-      Collection<String> goodPaths = RootFiles.cleanupReplacement(fs, files, true);
-      for (String good : goodPaths) {
-        Path path = new Path(good);
-        String filename = path.getName();
-        FileRef ref = new FileRef(location.toString() + "/" + filename, path);
-        DataFileValue dfv = new DataFileValue(0, 0);
-        datafiles.put(ref, dfv);
-      }
-    } else {
-
-      Text rowName = extent.getMetadataEntry();
-
-      String tableId = extent.isMeta() ? RootTable.ID : MetadataTable.ID;
-      ScannerImpl mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), SystemCredentials.get(), tableId, Authorizations.EMPTY);
-
-      // Commented out because when no data file is present, each tablet will scan through metadata table and return nothing
-      // reduced batch size to improve performance
-      // changed here after endKeys were implemented from 10 to 1000
-      mdScanner.setBatchSize(1000);
-
-      // leave these in, again, now using endKey for safety
-      mdScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
-
-      mdScanner.setRange(new Range(rowName));
-
-      for (Entry<Key,Value> entry : mdScanner) {
-
-        if (entry.getKey().compareRow(rowName) != 0) {
-          break;
-        }
-
-        FileRef ref = new FileRef(fs, entry.getKey());
-        datafiles.put(ref, new DataFileValue(entry.getValue().get()));
-      }
-    }
-    return datafiles;
-  }
-
-  private static List<LogEntry> lookupLogEntries(KeyExtent ke, SortedMap<Key,Value> tabletsKeyValues) {
-    List<LogEntry> logEntries = new ArrayList<LogEntry>();
-
-    if (ke.isMeta()) {
-      try {
-        logEntries = MetadataTableUtil.getLogEntries(SystemCredentials.get(), ke);
-      } catch (Exception ex) {
-        throw new RuntimeException("Unable to read tablet log entries", ex);
-      }
-    } else {
-      log.debug("Looking at metadata " + tabletsKeyValues);
-      Text row = ke.getMetadataEntry();
-      for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
-        Key key = entry.getKey();
-        if (key.getRow().equals(row)) {
-          if (key.getColumnFamily().equals(LogColumnFamily.NAME)) {
-            logEntries.add(LogEntry.fromKeyValue(key, entry.getValue()));
-          }
-        }
-      }
-    }
-
-    log.debug("got " + logEntries + " for logs for " + ke);
-    return logEntries;
-  }
-
-  private static Set<FileRef> lookupScanFiles(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues, VolumeManager fs) throws IOException {
-    HashSet<FileRef> scanFiles = new HashSet<FileRef>();
-
-    Text row = extent.getMetadataEntry();
-    for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
-      Key key = entry.getKey();
-      if (key.getRow().equals(row) && key.getColumnFamily().equals(ScanFileColumnFamily.NAME)) {
-        scanFiles.add(new FileRef(fs, key));
-      }
-    }
-
-    return scanFiles;
-  }
-
-  private static long lookupFlushID(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
-    Text row = extent.getMetadataEntry();
-    for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
-      Key key = entry.getKey();
-      if (key.getRow().equals(row) && TabletsSection.ServerColumnFamily.FLUSH_COLUMN.equals(key.getColumnFamily(), key.getColumnQualifier()))
-        return Long.parseLong(entry.getValue().toString());
-    }
-
-    return -1;
-  }
-
-  private static long lookupCompactID(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
-    Text row = extent.getMetadataEntry();
-    for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
-      Key key = entry.getKey();
-      if (key.getRow().equals(row) && TabletsSection.ServerColumnFamily.COMPACT_COLUMN.equals(key.getColumnFamily(), key.getColumnQualifier()))
-        return Long.parseLong(entry.getValue().toString());
-    }
-
-    return -1;
-  }
-
-  private Tablet(TabletServer tabletServer, Text location, KeyExtent extent, TabletResourceManager trm, Configuration conf, VolumeManager fs,
-      SortedMap<Key,Value> tabletsKeyValues) throws IOException {
-    this(tabletServer, location, extent, trm, conf, fs, lookupLogEntries(extent, tabletsKeyValues), lookupDatafiles(tabletServer.getSystemConfiguration(), fs,
-        extent, tabletsKeyValues), lookupTime(tabletServer.getSystemConfiguration(), extent, tabletsKeyValues), lookupLastServer(extent, tabletsKeyValues),
-        lookupScanFiles(extent, tabletsKeyValues, fs), lookupFlushID(extent, tabletsKeyValues), lookupCompactID(extent, tabletsKeyValues));
-  }
-
-  private static TServerInstance lookupLastServer(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
-    for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
-      if (entry.getKey().getColumnFamily().compareTo(TabletsSection.LastLocationColumnFamily.NAME) == 0) {
-        return new TServerInstance(entry.getValue(), entry.getKey().getColumnQualifier());
-      }
-    }
-    return null;
-  }
-
-  /**
-   * yet another constructor - this one allows us to avoid costly lookups into the Metadata table if we already know the files we need - as at split time
-   */
-  private Tablet(final TabletServer tabletServer, final Text location, final KeyExtent extent, final TabletResourceManager trm, final Configuration conf,
-      final VolumeManager fs, final List<LogEntry> rawLogEntries, final SortedMap<FileRef,DataFileValue> rawDatafiles, String time,
-      final TServerInstance lastLocation, Set<FileRef> scanFiles, long initFlushID, long initCompactID) throws IOException {
-
-    TabletFiles tabletPaths = VolumeUtil.updateTabletVolumes(tabletServer.getLock(), fs, extent, new TabletFiles(location.toString(), rawLogEntries,
-        rawDatafiles));
-
-    Path locationPath;
-
-    if (tabletPaths.dir.contains(":")) {
-      locationPath = new Path(tabletPaths.dir.toString());
-    } else {
-      locationPath = fs.getFullPath(FileType.TABLE, extent.getTableId().toString() + tabletPaths.dir.toString());
-    }
-
-    final List<LogEntry> logEntries = tabletPaths.logEntries;
-    final SortedMap<FileRef,DataFileValue> datafiles = tabletPaths.datafiles;
-
-    this.location = locationPath;
-    this.lastLocation = lastLocation;
-    this.tabletDirectory = tabletPaths.dir;
-    this.conf = conf;
-    this.acuTableConf = tabletServer.getTableConfiguration(extent);
-
-    this.fs = fs;
-    this.extent = extent;
-    this.tabletResources = trm;
-
-    this.lastFlushID = initFlushID;
-    this.lastCompactID = initCompactID;
-
-    if (extent.isRootTablet()) {
-      long rtime = Long.MIN_VALUE;
-      for (FileRef ref : datafiles.keySet()) {
-        Path path = ref.path();
-        FileSystem ns = fs.getVolumeByPath(path).getFileSystem();
-        FileSKVIterator reader = FileOperations.getInstance().openReader(path.toString(), true, ns, ns.getConf(), tabletServer.getTableConfiguration(extent));
-        long maxTime = -1;
-        try {
-
-          while (reader.hasTop()) {
-            maxTime = Math.max(maxTime, reader.getTopKey().getTimestamp());
-            reader.next();
-          }
-
-        } finally {
-          reader.close();
-        }
-
-        if (maxTime > rtime) {
-          time = TabletTime.LOGICAL_TIME_ID + "" + maxTime;
-          rtime = maxTime;
-        }
-      }
-    }
-    if (time == null && datafiles.isEmpty() && extent.equals(RootTable.OLD_EXTENT)) {
-      // recovery... old root tablet has no data, so time doesn't matter:
-      time = TabletTime.LOGICAL_TIME_ID + "" + Long.MIN_VALUE;
-    }
-
-    this.tabletServer = tabletServer;
-    this.logId = tabletServer.createLogId(extent);
-
-    this.timer = new TabletStatsKeeper();
-
-    setupDefaultSecurityLabels(extent);
-
-    tabletMemory = new TabletMemory();
-    tabletTime = TabletTime.getInstance(time);
-    persistedTime = tabletTime.getTime();
-
-    acuTableConf.addObserver(configObserver = new ConfigurationObserver() {
-
-      private void reloadConstraints() {
-        constraintChecker.set(new ConstraintChecker(acuTableConf));
-      }
-
-      @Override
-      public void propertiesChanged() {
-        reloadConstraints();
-
-        try {
-          setupDefaultSecurityLabels(extent);
-        } catch (Exception e) {
-          log.error("Failed to reload default security labels for extent: " + extent.toString());
-        }
-      }
-
-      @Override
-      public void propertyChanged(String prop) {
-        if (prop.startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey()))
-          reloadConstraints();
-        else if (prop.equals(Property.TABLE_DEFAULT_SCANTIME_VISIBILITY.getKey())) {
-          try {
-            log.info("Default security labels changed for extent: " + extent.toString());
-            setupDefaultSecurityLabels(extent);
-          } catch (Exception e) {
-            log.error("Failed to reload default security labels for extent: " + extent.toString());
-          }
-        }
-
-      }
-
-      @Override
-      public void sessionExpired() {
-        log.debug("Session expired, no longer updating per table props...");
-      }
-
-    });
-
-    acuTableConf.getNamespaceConfiguration().addObserver(configObserver);
-
-    // Force a load of any per-table properties
-    configObserver.propertiesChanged();
-
-    if (!logEntries.isEmpty()) {
-      log.info("Starting Write-Ahead Log recovery for " + this.extent);
-      final long[] count = new long[2];
-      final CommitSession commitSession = tabletMemory.getCommitSession();
-      count[1] = Long.MIN_VALUE;
-      try {
-        Set<String> absPaths = new HashSet<String>();
-        for (FileRef ref : datafiles.keySet())
-          absPaths.add(ref.path().toString());
-
-        tabletServer.recover(this.tabletServer.getFileSystem(), extent, acuTableConf, logEntries, absPaths, new MutationReceiver() {
-          @Override
-          public void receive(Mutation m) {
-            // LogReader.printMutation(m);
-            Collection<ColumnUpdate> muts = m.getUpdates();
-            for (ColumnUpdate columnUpdate : muts) {
-              if (!columnUpdate.hasTimestamp()) {
-                // if it is not a user set timestamp, it must have been set
-                // by the system
-                count[1] = Math.max(count[1], columnUpdate.getTimestamp());
-              }
-            }
-            tabletMemory.mutate(commitSession, Collections.singletonList(m));
-            count[0]++;
-          }
-        });
-
-        if (count[1] != Long.MIN_VALUE) {
-          tabletTime.useMaxTimeFromWALog(count[1]);
-        }
-        commitSession.updateMaxCommittedTime(tabletTime.getTime());
-
-        if (count[0] == 0) {
-          MetadataTableUtil.removeUnusedWALEntries(extent, logEntries, tabletServer.getLock());
-          logEntries.clear();
-        }
-
-      } catch (Throwable t) {
-        if (acuTableConf.getBoolean(Property.TABLE_FAILURES_IGNORE)) {
-          log.warn("Error recovering from log files: ", t);
-        } else {
-          throw new RuntimeException(t);
-        }
-      }
-      // make some closed references that represent the recovered logs
-      currentLogs = new HashSet<DfsLogger>();
-      for (LogEntry logEntry : logEntries) {
-        for (String log : logEntry.logSet) {
-          currentLogs.add(new DfsLogger(tabletServer.getServerConfig(), log));
-        }
-      }
-
-      log.info("Write-Ahead Log recovery complete for " + this.extent + " (" + count[0] + " mutations applied, " + tabletMemory.getNumEntries()
-          + " entries created)");
-    }
-
-    String contextName = acuTableConf.get(Property.TABLE_CLASSPATH);
-    if (contextName != null && !contextName.equals("")) {
-      // initialize context classloader, instead of possibly waiting for it to initialize for a scan
-      // TODO this could hang, causing other tablets to fail to load - ACCUMULO-1292
-      AccumuloVFSClassLoader.getContextManager().getClassLoader(contextName);
-    }
-
-    // do this last after tablet is completely setup because it
-    // could cause major compaction to start
-    datafileManager = new DatafileManager(datafiles);
-
-    computeNumEntries();
-
-    datafileManager.removeFilesAfterScan(scanFiles);
-
-    // look for hints of a failure on the previous tablet server
-    if (!logEntries.isEmpty() || needsMajorCompaction(MajorCompactionReason.NORMAL)) {
-      // look for any temp files hanging around
-      removeOldTemporaryFiles();
-    }
-
-    log.log(TLevel.TABLET_HIST, extent + " opened");
-  }
-
-  private void removeOldTemporaryFiles() {
-    // remove any temporary files created by a previous tablet server
-    try {
-      for (FileStatus tmp : fs.globStatus(new Path(location, "*_tmp"))) {
-        try {
-          log.debug("Removing old temp file " + tmp.getPath());
-          fs.delete(tmp.getPath());
-        } catch (IOException ex) {
-          log.error("Unable to remove old temp file " + tmp.getPath() + ": " + ex);
-        }
-      }
-    } catch (IOException ex) {
-      log.error("Error scanning for old temp files in " + location);
-    }
-  }
-
-  private void setupDefaultSecurityLabels(KeyExtent extent) {
-    if (extent.isMeta()) {
-      defaultSecurityLabel = new byte[0];
-    } else {
-      try {
-        ColumnVisibility cv = new ColumnVisibility(acuTableConf.get(Property.TABLE_DEFAULT_SCANTIME_VISIBILITY));
-        this.defaultSecurityLabel = cv.getExpression();
-      } catch (Exception e) {
-        log.error(e, e);
-        this.defaultSecurityLabel = new byte[0];
-      }
-    }
-  }
-
-  public static class KVEntry extends KeyValue {
-    private static final long serialVersionUID = 1L;
-
-    public KVEntry(Key k, Value v) {
-      super(new Key(k), Arrays.copyOf(v.get(), v.get().length));
-    }
-
-    int numBytes() {
-      return getKey().getSize() + getValue().get().length;
-    }
-
-    int estimateMemoryUsed() {
-      return getKey().getSize() + getValue().get().length + (9 * 32); // overhead is 32 per object
-    }
-  }
-
-  private LookupResult lookup(SortedKeyValueIterator<Key,Value> mmfi, List<Range> ranges, HashSet<Column> columnSet, ArrayList<KVEntry> results,
-      long maxResultsSize) throws IOException {
-
-    LookupResult lookupResult = new LookupResult();
-
-    boolean exceededMemoryUsage = false;
-    boolean tabletClosed = false;
-
-    Set<ByteSequence> cfset = null;
-    if (columnSet.size() > 0)
-      cfset = LocalityGroupUtil.families(columnSet);
-
-    for (Range range : ranges) {
-
-      if (exceededMemoryUsage || tabletClosed) {
-        lookupResult.unfinishedRanges.add(range);
-        continue;
-      }
-
-      int entriesAdded = 0;
-
-      try {
-        if (cfset != null)
-          mmfi.seek(range, cfset, true);
-        else
-          mmfi.seek(range, LocalityGroupUtil.EMPTY_CF_SET, false);
-
-        while (mmfi.hasTop()) {
-          Key key = mmfi.getTopKey();
-
-          KVEntry kve = new KVEntry(key, mmfi.getTopValue());
-          results.add(kve);
-          entriesAdded++;
-          lookupResult.bytesAdded += kve.estimateMemoryUsed();
-          lookupResult.dataSize += kve.numBytes();
-
-          exceededMemoryUsage = lookupResult.bytesAdded > maxResultsSize;
-
-          if (exceededMemoryUsage) {
-            addUnfinishedRange(lookupResult, range, key, false);
-            break;
-          }
-
-          mmfi.next();
-        }
-
-      } catch (TooManyFilesException tmfe) {
-        // treat this as a closed tablet, and let the client retry
-        log.warn("Tablet " + getExtent() + " has too many files, batch lookup can not run");
-        handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
-        tabletClosed = true;
-      } catch (IOException ioe) {
-        if (shutdownInProgress()) {
-          // assume HDFS shutdown hook caused this exception
-          log.debug("IOException while shutdown in progress ", ioe);
-          handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
-          tabletClosed = true;
-        } else {
-          throw ioe;
-        }
-      } catch (IterationInterruptedException iie) {
-        if (isClosed()) {
-          handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
-          tabletClosed = true;
-        } else {
-          throw iie;
-        }
-      } catch (TabletClosedException tce) {
-        handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
-        tabletClosed = true;
-      }
-
-    }
-
-    return lookupResult;
-  }
-
-  private void handleTabletClosedDuringScan(ArrayList<KVEntry> results, LookupResult lookupResult, boolean exceededMemoryUsage, Range range, int entriesAdded) {
-    if (exceededMemoryUsage)
-      throw new IllegalStateException("tablet should not exceed memory usage or close, not both");
-
-    if (entriesAdded > 0)
-      addUnfinishedRange(lookupResult, range, results.get(results.size() - 1).getKey(), false);
-    else
-      lookupResult.unfinishedRanges.add(range);
-
-    lookupResult.closed = true;
-  }
-
-  private void addUnfinishedRange(LookupResult lookupResult, Range range, Key key, boolean inclusiveStartKey) {
-    if (range.getEndKey() == null || key.compareTo(range.getEndKey()) < 0) {
-      Range nlur = new Range(new Key(key), inclusiveStartKey, range.getEndKey(), range.isEndKeyInclusive());
-      lookupResult.unfinishedRanges.add(nlur);
-    }
-  }
-
-  public static interface KVReceiver {
-    void receive(List<KVEntry> matches) throws IOException;
-  }
-
-  class LookupResult {
-    List<Range> unfinishedRanges = new ArrayList<Range>();
-    long bytesAdded = 0;
-    long dataSize = 0;
-    boolean closed = false;
-  }
-
-  public LookupResult lookup(List<Range> ranges, HashSet<Column> columns, Authorizations authorizations, ArrayList<KVEntry> results, long maxResultSize,
-      List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, AtomicBoolean interruptFlag) throws IOException {
-
-    if (ranges.size() == 0) {
-      return new LookupResult();
-    }
-
-    ranges = Range.mergeOverlapping(ranges);
-    Collections.sort(ranges);
-
-    Range tabletRange = extent.toDataRange();
-    for (Range range : ranges) {
-      // do a test to see if this range falls within the tablet, if it does not
-      // then clip will throw an exception
-      tabletRange.clip(range);
-    }
-
-    ScanDataSource dataSource = new ScanDataSource(authorizations, this.defaultSecurityLabel, columns, ssiList, ssio, interruptFlag);
-
-    LookupResult result = null;
-
-    try {
-      SortedKeyValueIterator<Key,Value> iter = new SourceSwitchingIterator(dataSource);
-      result = lookup(iter, ranges, columns, results, maxResultSize);
-      return result;
-    } catch (IOException ioe) {
-      dataSource.close(true);
-      throw ioe;
-    } finally {
-      // code in finally block because always want
-      // to return mapfiles, even when exception is thrown
-      dataSource.close(false);
-
-      synchronized (this) {
-        queryCount += results.size();
-        if (result != null)
-          queryBytes += result.dataSize;
-      }
-    }
-  }
-
-  private Batch nextBatch(SortedKeyValueIterator<Key,Value> iter, Range range, int num, Set<Column> columns) throws IOException {
-
-    // log.info("In nextBatch..");
-
-    List<KVEntry> results = new ArrayList<KVEntry>();
-    Key key = null;
-
-    Value value;
-    long resultSize = 0L;
-    long resultBytes = 0L;
-
-    long maxResultsSize = acuTableConf.getMemoryInBytes(Property.TABLE_SCAN_MAXMEM);
-
-    if (columns.size() == 0) {
-      iter.seek(range, LocalityGroupUtil.EMPTY_CF_SET, false);
-    } else {
-      iter.seek(range, LocalityGroupUtil.families(columns), true);
-    }
-
-    Key continueKey = null;
-    boolean skipContinueKey = false;
-
-    boolean endOfTabletReached = false;
-    while (iter.hasTop()) {
-
-      value = iter.getTopValue();
-      key = iter.getTopKey();
-
-      KVEntry kvEntry = new KVEntry(key, value); // copies key and value
-      results.add(kvEntry);
-      resultSize += kvEntry.estimateMemoryUsed();
-      resultBytes += kvEntry.numBytes();
-
-      if (resultSize >= maxResultsSize || results.size() >= num) {
-        continueKey = new Key(key);
-        skipContinueKey = true;
-        break;
-      }
-
-      iter.next();
-    }
-
-    if (iter.hasTop() == false) {
-      endOfTabletReached = true;
-    }
-
-    Batch retBatch = new Batch();
-    retBatch.numBytes = resultBytes;
-
-    if (!endOfTabletReached) {
-      retBatch.continueKey = continueKey;
-      retBatch.skipContinueKey = skipContinueKey;
-    } else {
-      retBatch.continueKey = null;
-    }
-
-    if (endOfTabletReached && results.size() == 0)
-      retBatch.results = null;
-    else
-      retBatch.results = results;
-
-    return retBatch;
-  }
-
-  /**
-   * Determine if a JVM shutdown is in progress.
-   * 
-   */
-  private boolean shutdownInProgress() {
-    try {
-      Runtime.getRuntime().removeShutdownHook(new Thread(new Runnable() {
-        @Override
-        public void run() {}
-      }));
-    } catch (IllegalStateException ise) {
-      return true;
-    }
-
-    return false;
-  }
-
-  private class Batch {
-    public boolean skipContinueKey;
-    public List<KVEntry> results;
-    public Key continueKey;
-    public long numBytes;
-  }
-
-  Scanner createScanner(Range range, int num, Set<Column> columns, Authorizations authorizations, List<IterInfo> ssiList, Map<String,Map<String,String>> ssio,
-      boolean isolated, AtomicBoolean interruptFlag) {
-    // do a test to see if this range falls within the tablet, if it does not
-    // then clip will throw an exception
-    extent.toDataRange().clip(range);
-
-    ScanOptions opts = new ScanOptions(num, authorizations, this.defaultSecurityLabel, columns, ssiList, ssio, interruptFlag, isolated);
-    return new Scanner(range, opts);
-  }
-
-  class ScanBatch {
-    boolean more;
-    List<KVEntry> results;
-
-    ScanBatch(List<KVEntry> results, boolean more) {
-      this.results = results;
-      this.more = more;
-    }
-  }
-
-  class Scanner {
-
-    private ScanOptions options;
-    private Range range;
-    private SortedKeyValueIterator<Key,Value> isolatedIter;
-    private ScanDataSource isolatedDataSource;
-    private boolean sawException = false;
-    private boolean scanClosed = false;
-
-    Scanner(Range range, ScanOptions options) {
-      this.range = range;
-      this.options = options;
-    }
-
-    synchronized ScanBatch read() throws IOException, TabletClosedException {
-
-      if (sawException)
-        throw new IllegalStateException("Tried to use scanner after exception occurred.");
-
-      if (scanClosed)
-        throw new IllegalStateException("Tried to use scanner after it was closed.");
-
-      Batch results = null;
-
-      ScanDataSource dataSource;
-
-      if (options.isolated) {
-        if (isolatedDataSource == null)
-          isolatedDataSource = new ScanDataSource(options);
-        dataSource = isolatedDataSource;
-      } else {
-        dataSource = new ScanDataSource(options);
-      }
-
-      try {
-
-        SortedKeyValueIterator<Key,Value> iter;
-
-        if (options.isolated) {
-          if (isolatedIter == null)
-            isolatedIter = new SourceSwitchingIterator(dataSource, true);
-          else
-            isolatedDataSource.fileManager.reattach();
-          iter = isolatedIter;
-        } else {
-          iter = new SourceSwitchingIterator(dataSource, false);
-        }
-
-        results = nextBatch(iter, range, options.num, options.columnSet);
-
-        if (results.results == null) {
-          range = null;
-          return new ScanBatch(new ArrayList<Tablet.KVEntry>(), false);
-        } else if (results.continueKey == null) {
-          return new ScanBatch(results.results, false);
-        } else {
-          range = new Range(results.continueKey, !results.skipContinueKey, range.getEndKey(), range.isEndKeyInclusive());
-          return new ScanBatch(results.results, true);
-        }
-
-      } catch (IterationInterruptedException iie) {
-        sawException = true;
-        if (isClosed())
-          throw new TabletClosedException(iie);
-        else
-          throw iie;
-      } catch (IOException ioe) {
-        if (shutdownInProgress()) {
-          log.debug("IOException while shutdown in progress ", ioe);
-          throw new TabletClosedException(ioe); // assume IOException was caused by execution of HDFS shutdown hook
-        }
-
-        sawException = true;
-        dataSource.close(true);
-        throw ioe;
-      } catch (RuntimeException re) {
-        sawException = true;
-        throw re;
-      } finally {
-        // code in finally block because always want
-        // to return mapfiles, even when exception is thrown
-        if (!options.isolated)
-          dataSource.close(false);
-        else if (dataSource.fileManager != null)
-          dataSource.fileManager.detach();
-
-        synchronized (Tablet.this) {
-          if (results != null && results.results != null) {
-            long more = results.results.size();
-            queryCount += more;
-            queryBytes += results.numBytes;
-          }
-        }
-      }
-    }
-
-    // close and read are synchronized because can not call close on the data source while it is in use
-    // this cloud lead to the case where file iterators that are in use by a thread are returned
-    // to the pool... this would be bad
-    void close() {
-      options.interruptFlag.set(true);
-      synchronized (this) {
-        scanClosed = true;
-        if (isolatedDataSource != null)
-          isolatedDataSource.close(false);
-      }
-    }
-  }
-
-  static class ScanOptions {
-
-    // scan options
-    Authorizations authorizations;
-    byte[] defaultLabels;
-    Set<Column> columnSet;
-    List<IterInfo> ssiList;
-    Map<String,Map<String,String>> ssio;
-    AtomicBoolean interruptFlag;
-    int num;
-    boolean isolated;
-
-    ScanOptions(int num, Authorizations authorizations, byte[] defaultLabels, Set<Column> columnSet, List<IterInfo> ssiList,
-        Map<String,Map<String,String>> ssio, AtomicBoolean interruptFlag, boolean isolated) {
-      this.num = num;
-      this.authorizations = authorizations;
-      this.defaultLabels = defaultLabels;
-      this.columnSet = columnSet;
-      this.ssiList = ssiList;
-      this.ssio = ssio;
-      this.interruptFlag = interruptFlag;
-      this.isolated = isolated;
-    }
-
-  }
-
-  class ScanDataSource implements DataSource {
-
-    // data source state
-    private ScanFileManager fileManager;
-    private SortedKeyValueIterator<Key,Value> iter;
-    private long expectedDeletionCount;
-    private List<MemoryIterator> memIters = null;
-    private long fileReservationId;
-    private AtomicBoolean interruptFlag;
-    private StatsIterator statsIterator;
-
-    ScanOptions options;
-
-    ScanDataSource(Authorizations authorizations, byte[] defaultLabels, HashSet<Column> columnSet, List<IterInfo> ssiList, Map<String,Map<String,String>> ssio,
-        AtomicBoolean interruptFlag) {
-      expectedDeletionCount = dataSourceDeletions.get();
-      this.options = new ScanOptions(-1, authorizations, defaultLabels, columnSet, ssiList, ssio, interruptFlag, false);
-      this.interruptFlag = interruptFlag;
-    }
-
-    ScanDataSource(ScanOptions options) {
-      expectedDeletionCount = dataSourceDeletions.get();
-      this.options = options;
-      this.interruptFlag = options.interruptFlag;
-    }
-
-    @Override
-    public DataSource getNewDataSource() {
-      if (!isCurrent()) {
-        // log.debug("Switching data sources during a scan");
-        if (memIters != null) {
-          tabletMemory.returnIterators(memIters);
-          memIters = null;
-          datafileManager.returnFilesForScan(fileReservationId);
-          fileReservationId = -1;
-        }
-
-        if (fileManager != null)
-          fileManager.releaseOpenFiles(false);
-
-        expectedDeletionCount = dataSourceDeletions.get();
-        iter = null;
-
-        return this;
-      } else
-        return this;
-    }
-
-    @Override
-    public boolean isCurrent() {
-      return expectedDeletionCount == dataSourceDeletions.get();
-    }
-
-    @Override
-    public SortedKeyValueIterator<Key,Value> iterator() throws IOException {
-      if (iter == null)
-        iter = createIterator();
-      return iter;
-    }
-
-    private SortedKeyValueIterator<Key,Value> createIterator() throws IOException {
-
-      Map<FileRef,DataFileValue> files;
-
-      synchronized (Tablet.this) {
-
-        if (memIters != null)
-          throw new IllegalStateException("Tried to create new scan iterator w/o releasing memory");
-
-        if (Tablet.this.closed)
-          throw new TabletClosedException();
-
-        if (interruptFlag.get())
-          throw new IterationInterruptedException(extent.toString() + " " + interruptFlag.hashCode());
-
-        // only acquire the file manager when we know the tablet is open
-        if (fileManager == null) {
-          fileManager = tabletResources.newScanFileManager();
-          activeScans.add(this);
-        }
-
-        if (fileManager.getNumOpenFiles() != 0)
-          throw new IllegalStateException("Tried to create new scan iterator w/o releasing files");
-
-        // set this before trying to get iterators in case
-        // getIterators() throws an exception
-        expectedDeletionCount = dataSourceDeletions.get();
-
-        memIters = tabletMemory.getIterators();
-        Pair<Long,Map<FileRef,DataFileValue>> reservation = datafileManager.reserveFilesForScan();
-        fileReservationId = reservation.getFirst();
-        files = reservation.getSecond();
-      }
-
-      Collection<InterruptibleIterator> mapfiles = fileManager.openFiles(files, options.isolated);
-
-      List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<SortedKeyValueIterator<Key,Value>>(mapfiles.size() + memIters.size());
-
-      iters.addAll(mapfiles);
-      iters.addAll(memIters);
-
-      for (SortedKeyValueIterator<Key,Value> skvi : iters)
-        ((InterruptibleIterator) skvi).setInterruptFlag(interruptFlag);
-
-      MultiIterator multiIter = new MultiIterator(iters, extent);
-
-      TabletIteratorEnvironment iterEnv = new TabletIteratorEnvironment(IteratorScope.scan, acuTableConf, fileManager, files);
-
-      statsIterator = new StatsIterator(multiIter, TabletServer.seekCount, scannedCount);
-
-      DeletingIterator delIter = new DeletingIterator(statsIterator, false);
-
-      ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
-
-      ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, options.columnSet);
-
-      VisibilityFilter visFilter = new VisibilityFilter(colFilter, options.authorizations, options.defaultLabels);
-
-      return iterEnv.getTopLevelIterator(IteratorUtil
-          .loadIterators(IteratorScope.scan, visFilter, extent, acuTableConf, options.ssiList, options.ssio, iterEnv));
-    }
-
-    private void close(boolean sawErrors) {
-
-      if (memIters != null) {
-        tabletMemory.returnIterators(memIters);
-        memIters = null;
-        datafileManager.returnFilesForScan(fileReservationId);
-        fileReservationId = -1;
-      }
-
-      synchronized (Tablet.this) {
-        activeScans.remove(this);
-        if (activeScans.size() == 0)
-          Tablet.this.notifyAll();
-      }
-
-      if (fileManager != null) {
-        fileManager.releaseOpenFiles(sawErrors);
-        fileManager = null;
-      }
-
-      if (statsIterator != null) {
-        statsIterator.report();
-      }
-
-    }
-
-    public void interrupt() {
-      interruptFlag.set(true);
-    }
-
-    @Override
-    public DataSource getDeepCopyDataSource(IteratorEnvironment env) {
-      throw new UnsupportedOperationException();
-    }
-
-  }
-
-  private DataFileValue minorCompact(Configuration conf, VolumeManager fs, InMemoryMap memTable, FileRef tmpDatafile, FileRef newDatafile, FileRef mergeFile,
-      boolean hasQueueTime, long queued, CommitSession commitSession, long flushId, MinorCompactionReason mincReason) {
-    boolean failed = false;
-    long start = System.currentTimeMillis();
-    timer.incrementStatusMinor();
-
-    long count = 0;
-
-    try {
-      Span span = Trace.start("write");
-      CompactionStats stats;
-      try {
-        count = memTable.getNumEntries();
-
-        DataFileValue dfv = null;
-        if (mergeFile != null)
-          dfv = datafileManager.getDatafileSizes().get(mergeFile);
-
-        MinorCompactor compactor = new MinorCompactor(conf, fs, memTable, mergeFile, dfv, tmpDatafile, acuTableConf, extent, mincReason);
-        stats = compactor.call();
-      } finally {
-        span.stop();
-      }
-      span = Trace.start("bringOnline");
-      try {
-        datafileManager.bringMinorCompactionOnline(tmpDatafile, newDatafile, mergeFile, new DataFileValue(stats.getFileSize(), stats.getEntriesWritten()),
-            commitSession, flushId);
-      } finally {
-        span.stop();
-      }
-      return new DataFileValue(stats.getFileSize(), stats.getEntriesWritten());
-    } catch (Exception E) {
-      failed = true;
-      throw new RuntimeException(E);
-    } catch (Error E) {
-      // Weird errors like "OutOfMemoryError" when trying to create the thread for the compaction
-      failed = true;
-      throw new RuntimeException(E);
-    } finally {
-      try {
-        tabletMemory.finalizeMinC();
-      } catch (Throwable t) {
-        log.error("Failed to free tablet memory", t);
-      }
-
-      if (!failed) {
-        lastMinorCompactionFinishTime = System.currentTimeMillis();
-      }
-      if (tabletServer.mincMetrics.isEnabled())
-        tabletServer.mincMetrics.add(TabletServerMinCMetrics.minc, (lastMinorCompactionFinishTime - start));
-      if (hasQueueTime) {
-        timer.updateTime(Operation.MINOR, queued, start, count, failed);
-        if (tabletServer.mincMetrics.isEnabled())
-          tabletServer.mincMetrics.add(TabletServerMinCMetrics.queue, (start - queued));
-      } else
-        timer.updateTime(Operation.MINOR, start, count, failed);
-    }
-  }
-
-  private class MinorCompactionTask implements Runnable {
-
-    private long queued;
-    private CommitSession commitSession;
-    private DataFileValue stats;
-    private FileRef mergeFile;
-    private long flushId;
-    private MinorCompactionReason mincReason;
-
-    MinorCompactionTask(FileRef mergeFile, CommitSession commitSession, long flushId, MinorCompactionReason mincReason) {
-      queued = System.currentTimeMillis();
-      minorCompactionWaitingToStart = true;
-      this.commitSession = commitSession;
-      this.mergeFile = mergeFile;
-      this.flushId = flushId;
-      this.mincReason = mincReason;
-    }
-
-    @Override
-    public void run() {
-      minorCompactionWaitingToStart = false;
-      minorCompactionInProgress = true;
-      Span minorCompaction = Trace.on("minorCompaction");
-      try {
-        FileRef newMapfileLocation = getNextMapFilename(mergeFile == null ? "F" : "M");
-        FileRef tmpFileRef = new FileRef(newMapfileLocation.path() + "_tmp");
-        Span span = Trace.start("waitForCommits");
-        synchronized (Tablet.this) {
-          commitSession.waitForCommitsToFinish();
-        }
-        span.stop();
-        span = Trace.start("start");
-        while (true) {
-          try {
-            // the purpose of the minor compaction start event is to keep track of the filename... in the case
-            // where the metadata table write for the minor compaction finishes and the process dies before
-            // writing the minor compaction finish event, then the start event+filename in metadata table will
-            // prevent recovery of duplicate data... the minor compaction start event could be written at any time
-            // before the metadata write for the minor compaction
-            tabletServer.minorCompactionStarted(commitSession, commitSession.getWALogSeq() + 1, newMapfileLocation.path().toString());
-            break;
-          } catch (IOException e) {
-            log.warn("Failed to write to write ahead log " + e.getMessage(), e);
-          }
-        }
-        span.stop();
-        span = Trace.start("compact");
-        this.stats = minorCompact(conf, fs, tabletMemory.getMinCMemTable(), tmpFileRef, newMapfileLocation, mergeFile, true, queued, commitSession, flushId,
-            mincReason);
-        span.stop();
-
-        if (needsSplit()) {
-          tabletServer.executeSplit(Tablet.this);
-        } else {
-          initiateMajorCompaction(MajorCompactionReason.NORMAL);
-        }
-      } catch (Throwable t) {
-        log.error("Unknown error during minor compaction for extent: " + getExtent(), t);
-        throw new RuntimeException(t);
-      } finally {
-        minorCompactionInProgress = false;
-        minorCompaction.data("extent", extent.toString());
-        minorCompaction.data("numEntries", Long.toString(this.stats.getNumEntries()));
-        minorCompaction.data("size", Long.toString(this.stats.getSize()));
-        minorCompaction.stop();
-      }
-    }
-  }
-
-  private synchronized MinorCompactionTask prepareForMinC(long flushId, MinorCompactionReason mincReason) {
-    CommitSession oldCommitSession = tabletMemory.prepareForMinC();
-    otherLogs = currentLogs;
-    currentLogs = new HashSet<DfsLogger>();
-
-    FileRef mergeFile = datafileManager.reserveMergingMinorCompactionFile();
-
-    return new MinorCompactionTask(mergeFile, oldCommitSession, flushId, mincReason);
-
-  }
-
-  void flush(long tableFlushID) {
-    boolean updateMetadata = false;
-    boolean initiateMinor = false;
-
-    try {
-
-      synchronized (this) {
-
-        // only want one thing at a time to update flush ID to ensure that metadata table and tablet in memory state are consistent
-        if (updatingFlushID)
-          return;
-
-        if (lastFlushID >= tableFlushID)
-          return;
-
-        if (closing || closed || tabletMemory.memoryReservedForMinC())
-          return;
-
-        if (tabletMemory.getMemTable().getNumEntries() == 0) {
-          lastFlushID = tableFlushID;
-          updatingFlushID = true;
-          updateMetadata = true;
-        } else
-          initiateMinor = true;
-      }
-
-      if (updateMetadata) {
-        Credentials creds = SystemCredentials.get();
-        // if multiple threads were allowed to update this outside of a sync block, then it would be
-        // a race condition
-        MetadataTableUtil.updateTabletFlushID(extent, tableFlushID, creds, tabletServer.getLock());
-      } else if (initiateMinor)
-        initiateMinorCompaction(tableFlushID, MinorCompactionReason.USER);
-
-    } finally {
-      if (updateMetadata) {
-        synchronized (this) {
-          updatingFlushID = false;
-          this.notifyAll();
-        }
-      }
-    }
-
-  }
-
-  boolean initiateMinorCompaction(MinorCompactionReason mincReason) {
-    if (isClosed()) {
-      // don't bother trying to get flush id if closed... could be closed after this check but that is ok... just trying to cut down on uneeded log messages....
-      return false;
-    }
-
-    // get the flush id before the new memmap is made available for write
-    long flushId;
-    try {
-      flushId = getFlushID();
-    } catch (NoNodeException e) {
-      log.info("Asked to initiate MinC when there was no flush id " + getExtent() + " " + e.getMessage());
-      return false;
-    }
-    return initiateMinorCompaction(flushId, mincReason);
-  }
-
-  boolean minorCompactNow(MinorCompactionReason mincReason) {
-    long flushId;
-    try {
-      flushId = getFlushID();
-    } catch (NoNodeException e) {
-      log.info("Asked to initiate MinC when there was no flush id " + getExtent() + " " + e.getMessage());
-      return false;
-    }
-    MinorCompactionTask mct = createMinorCompactionTask(flushId, mincReason);
-    if (mct == null)
-      return false;
-    mct.run();
-    return true;
-  }
-
-  boolean initiateMinorCompaction(long flushId, MinorCompactionReason mincReason) {
-    MinorCompactionTask mct = createMinorCompactionTask(flushId, mincReason);
-    if (mct == null)
-      return false;
-    tabletResources.executeMinorCompaction(mct);
-    return true;
-  }
-
-  private MinorCompactionTask createMinorCompactionTask(long flushId, MinorCompactionReason mincReason) {
-    MinorCompactionTask mct;
-    long t1, t2;
-
-    StringBuilder logMessage = null;
-
-    try {
-      synchronized (this) {
-        t1 = System.currentTimeMillis();
-
-        if (closing || closed || majorCompactionWaitingToStart || tabletMemory.memoryReservedForMinC() || tabletMemory.getMemTable().getNumEntries() == 0
-            || updatingFlushID) {
-
-          logMessage = new StringBuilder();
-
-          logMessage.append(extent.toString());
-          logMessage.append(" closing " + closing);
-          logMessage.append(" closed " + closed);
-          logMessage.append(" majorCompactionWaitingToStart " + majorCompactionWaitingToStart);
-          if (tabletMemory != null)
-            logMessage.append(" tabletMemory.memoryReservedForMinC() " + tabletMemory.memoryReservedForMinC());
-          if (tabletMemory != null && tabletMemory.getMemTable() != null)
-            logMessage.append(" tabletMemory.getMemTable().getNumEntries() " + tabletMemory.getMemTable().getNumEntries());
-          logMessage.append(" updatingFlushID " + updatingFlushID);
-
-          return null;
-        }
-        // We're still recovering log entries
-        if (datafileManager == null) {
-          logMessage = new StringBuilder();
-          logMessage.append(extent.toString());
-          logMessage.append(" datafileManager " + datafileManager);
-          return null;
-        }
-
-        mct = prepareForMinC(flushId, mincReason);
-        t2 = System.currentTimeMillis();
-      }
-    } finally {
-      // log outside of sync block
-      if (logMessage != null && log.isDebugEnabled())
-        log.debug(logMessage);
-    }
-
-    log.debug(String.format("MinC initiate lock %.2f secs", (t2 - t1) / 1000.0));
-    return mct;
-  }
-
-  long getFlushID() throws NoNodeException {
-    try {
-      String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
-          + Constants.ZTABLE_FLUSH_ID;
-      return Long.parseLong(new String(ZooReaderWriter.getRetryingInstance().getData(zTablePath, null), StandardCharsets.UTF_8));
-    } catch (InterruptedException e) {
-      throw new RuntimeException(e);
-    } catch (NumberFormatException nfe) {
-      throw new RuntimeException(nfe);
-    } catch (KeeperException ke) {
-      if (ke instanceof NoNodeException) {
-        throw (NoNodeException) ke;
-      } else {
-        throw new RuntimeException(ke);
-      }
-    }
-  }
-
-  long getCompactionCancelID() {
-    String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
-        + Constants.ZTABLE_COMPACT_CANCEL_ID;
-
-    try {
-      return Long.parseLong(new String(ZooReaderWriter.getRetryingInstance().getData(zTablePath, null), StandardCharsets.UTF_8));
-    } catch (KeeperException e) {
-      throw new RuntimeException(e);
-    } catch (InterruptedException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  Pair<Long,List<IteratorSetting>> getCompactionID() throws NoNodeException {
-    try {
-      String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
-          + Constants.ZTABLE_COMPACT_ID;
-
-      String[] tokens = new String(ZooReaderWriter.getRetryingInstance().getData(zTablePath, null), StandardCharsets.UTF_8).split(",");
-      long compactID = Long.parseLong(tokens[0]);
-
-      CompactionIterators iters = new CompactionIterators();
-
-      if (tokens.length > 1) {
-        Hex hex = new Hex();
-        ByteArrayInputStream bais = new ByteArrayInputStream(hex.decode(tokens[1].split("=")[1].getBytes(StandardCharsets.UTF_8)));
-        DataInputStream dis = new DataInputStream(bais);
-
-        try {
-          iters.readFields(dis);
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-
-        KeyExtent ke = new KeyExtent(extent.getTableId(), iters.getEndRow(), iters.getStartRow());
-
-        if (!ke.overlaps(extent)) {
-          // only use iterators if compaction range overlaps
-          iters = new CompactionIterators();
-        }
-      }
-
-      return new Pair<Long,List<IteratorSetting>>(compactID, iters.getIterators());
-    } catch (InterruptedException e) {
-      throw new RuntimeException(e);
-    } catch (NumberFormatException nfe) {
-      throw new RuntimeException(nfe);
-    } catch (KeeperException ke) {
-      if (ke instanceof NoNodeException) {
-        throw (NoNodeException) ke;
-      } else {
-        throw new RuntimeException(ke);
-      }
-    } catch (DecoderException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  public synchronized void waitForMinC() {
-    tabletMemory.waitForMinC();
-  }
-
-  static class TConstraintViolationException extends Exception {
-    private static final long serialVersionUID = 1L;
-    private Violations violations;
-    private List<Mutation> violators;
-    private List<Mutation> nonViolators;
-    private CommitSession commitSession;
-
-    TConstraintViolationException(Violations violations, List<Mutation> violators, List<Mutation> nonViolators, CommitSession commitSession) {
-      this.violations = violations;
-      this.violators = violators;
-      this.nonViolators = nonViolators;
-      this.commitSession = commitSession;
-    }
-
-    Violations getViolations() {
-      return violations;
-    }
-
-    List<Mutation> getViolators() {
-      return violators;
-    }
-
-    List<Mutation> getNonViolators() {
-      return nonViolators;
-    }
-
-    CommitSession getCommitSession() {
-      return commitSession;
-    }
-  }
-
-  private synchronized CommitSession finishPreparingMutations(long time) {
-    if (writesInProgress < 0) {
-      throw new IllegalStateException("waitingForLogs < 0 " + writesInProgress);
-    }
-
-    if (closed || tabletMemory == null) {
-      // log.debug("tablet closed, can't commit");
-      return null;
-    }
-
-    writesInProgress++;
-    CommitSession commitSession = tabletMemory.getCommitSession();
-    commitSession.incrementCommitsInProgress();
-    commitSession.updateMaxCommittedTime(time);
-    return commitSession;


<TRUNCATED>

[28/35] git commit: ACCUMULO-2840 turn audit logging back on for the IT

Posted by el...@apache.org.
ACCUMULO-2840 turn audit logging back on for the IT


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

Branch: refs/heads/ACCUMULO-378
Commit: 66f3128030274b2134798fb61db5638dfef9b7f4
Parents: 60b5a1c
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Jun 4 16:23:45 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Jun 4 16:23:45 2014 -0400

----------------------------------------------------------------------
 test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/66f31280/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java b/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
index 6e01ffb..1730b14 100644
--- a/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
@@ -46,6 +46,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.test.functional.ConfigurableMacIT;
 import org.apache.commons.io.FileUtils;
@@ -69,6 +70,10 @@ public class AuditMessageIT extends ConfigurableMacIT {
   private static final String NEW_TEST_TABLE_NAME = "oranges";
   private static final String THIRD_TEST_TABLE_NAME = "pears";
   private static final Authorizations auths = new Authorizations("private", "public");
+  
+  public void beforeClusterStart(MiniAccumuloConfigImpl cfg) throws Exception {
+    new File(cfg.getConfDir(), "auditLog.xml").delete();
+  }
 
   // Must be static to survive Junit re-initialising the class every time.
   private static String lastAuditTimestamp;


[25/35] git commit: ACCUMULO-378 Core review fixes from bhavanki for replication

Posted by el...@apache.org.
ACCUMULO-378 Core review fixes from bhavanki for replication


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

Branch: refs/heads/ACCUMULO-378
Commit: 84e94a429bd92e469156642b1bfd69c422759e2d
Parents: 2f02d69
Author: Josh Elser <el...@apache.org>
Authored: Wed Jun 4 13:52:58 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Jun 4 13:52:58 2014 -0400

----------------------------------------------------------------------
 .../client/admin/ReplicationOperations.java     | 12 ++---
 .../core/client/impl/ReplicationClient.java     | 34 +++++++-----
 .../client/impl/ReplicationOperationsImpl.java  | 52 ++++++++----------
 .../replication/PeerNotFoundException.java      |  4 ++
 .../core/client/replication/ReplicaSystem.java  |  3 +-
 .../replication/ReplicaSystemFactory.java       |  6 ++-
 .../org/apache/accumulo/core/data/Mutation.java | 13 +++--
 .../master/replication/ReplicationDriver.java   | 13 +++--
 .../test/replication/CyclicReplicationIT.java   | 43 +++++++--------
 .../UnorderedWorkAssignerReplicationIT.java     | 57 ++++++++++----------
 10 files changed, 128 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/84e94a42/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
index 1d20f79..5873f73 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
@@ -32,14 +32,14 @@ import org.apache.accumulo.core.client.replication.ReplicaSystem;
 public interface ReplicationOperations {
 
   /**
-   * Define a cluster with the given name using the given {@link ReplicaSystem}
+   * Defines a cluster with the given name using the given {@link ReplicaSystem}.
    * @param name Name of the cluster, used for configuring replication on tables
    * @param system Type of system to be replicated to
    */
   public void addPeer(String name, ReplicaSystem system) throws AccumuloException, AccumuloSecurityException, PeerExistsException;
 
   /**
-   * Define a cluster with the given name and the given name system
+   * Defines a cluster with the given name and the given name system.
    * @param name Unique name for the cluster
    * @param replicaType {@link ReplicaSystem} class name to use to replicate the data
    * @throws PeerExistsException
@@ -47,14 +47,14 @@ public interface ReplicationOperations {
   public void addPeer(String name, String replicaType) throws AccumuloException, AccumuloSecurityException, PeerExistsException;
 
   /**
-   * Remove a cluster with the given name
+   * Removes a cluster with the given name.
    * @param name Name of the cluster to remove
    * @throws PeerNotFoundException
    */
   public void removePeer(String name) throws AccumuloException, AccumuloSecurityException, PeerNotFoundException;
 
   /**
-   * Wait for a table to be fully replicated
+   * Waits for a table to be fully replicated.
    * @param tableName The table to wait for
    * @throws AccumuloException
    * @throws AccumuloSecurityException
@@ -62,7 +62,7 @@ public interface ReplicationOperations {
   public void drain(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
 
   /**
-   * Wait for a table to be fully replicated as determined by the provided tables
+   * Waits for a table to be fully replicated as determined by the provided tables.
    * @param tableName The table to wait for
    * @throws AccumuloException
    * @throws AccumuloSecurityException
@@ -70,7 +70,7 @@ public interface ReplicationOperations {
   public void drain(String tableName, Set<String> files) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
 
   /**
-   * Get all of the referenced files for a table
+   * Gets all of the referenced files for a table.
    * @param tableName
    * @throws TableNotFoundException
    */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/84e94a42/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
index d7b12c7..13c027a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.core.client.impl;
 
-import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
 import java.net.UnknownHostException;
@@ -52,14 +51,19 @@ public class ReplicationClient {
    * @return Client to the ReplicationCoordinator service
    */
   public static ReplicationCoordinator.Client getCoordinatorConnectionWithRetry(Instance instance) throws AccumuloException {
-    checkArgument(instance != null, "instance is null");
+    checkNotNull(instance);
 
     for (int attempts = 1; attempts <= 10; attempts++) {
 
       ReplicationCoordinator.Client result = getCoordinatorConnection(instance);
       if (result != null)
         return result;
-      UtilWaitThread.sleep(attempts * 250);
+      log.debug("Could not get ReplicationCoordinator connection to {}, will retry", instance.getInstanceName());
+      try {
+        Thread.sleep(attempts * 250);
+      } catch (InterruptedException e) {
+        throw new AccumuloException(e);
+      }
     }
 
     throw new AccumuloException("Timed out trying to communicate with master from " + instance.getInstanceName());
@@ -69,14 +73,16 @@ public class ReplicationClient {
     List<String> locations = instance.getMasterLocations();
 
     if (locations.size() == 0) {
-      log.debug("No masters...");
+      log.debug("No masters for replication to instance {}", instance.getInstanceName());
       return null;
     }
 
     // This is the master thrift service, we just want the hostname, not the port
     String masterThriftService = locations.get(0);
-    if (masterThriftService.endsWith(":0"))
+    if (masterThriftService.endsWith(":0")) {
+      log.warn("Master found for {} did not have real location {}", instance.getInstanceName(), masterThriftService);
       return null;
+    }
 
 
     AccumuloConfiguration conf = ServerConfigurationUtil.getConfiguration(instance);
@@ -91,7 +97,7 @@ public class ReplicationClient {
       ZooReader reader = new ZooReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
       replCoordinatorAddr = new String(reader.getData(zkPath, null), StandardCharsets.UTF_8);
     } catch (KeeperException | InterruptedException e) {
-      log.error("Could not fetch remote coordinator port");
+      log.error("Could not fetch remote coordinator port", e);
       return null;
     }
 
@@ -106,11 +112,7 @@ public class ReplicationClient {
           conf);
       return client;
     } catch (TTransportException tte) {
-      if (tte.getCause().getClass().equals(UnknownHostException.class)) {
-        // do not expect to recover from this
-        throw new RuntimeException(tte);
-      }
-      log.debug("Failed to connect to master coordinator service ({}), will retry... ", coordinatorAddr.toString(), tte);
+      log.debug("Failed to connect to master coordinator service ({})", coordinatorAddr.toString(), tte);
       return null;
     }
   }
@@ -157,13 +159,17 @@ public class ReplicationClient {
   public static <T> T executeCoordinatorWithReturn(Instance instance, ClientExecReturn<T,ReplicationCoordinator.Client> exec) throws AccumuloException,
       AccumuloSecurityException {
     ReplicationCoordinator.Client client = null;
-    while (true) {
+    for (int i = 0; i < 10; i++) {
       try {
         client = getCoordinatorConnectionWithRetry(instance);
         return exec.execute(client);
       } catch (TTransportException tte) {
         log.debug("ReplicationClient coordinator request failed, retrying ... ", tte);
-        UtilWaitThread.sleep(100);
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException e) {
+          throw new AccumuloException(e);
+        }
       } catch (ThriftSecurityException e) {
         throw new AccumuloSecurityException(e.user, e.code, e);
       } catch (AccumuloException e) {
@@ -175,6 +181,8 @@ public class ReplicationClient {
           close(client);
       }
     }
+
+    throw new AccumuloException("Could not connect to ReplicationCoordinator at " + instance.getInstanceName());
   }
 
   public static void executeCoordinator(Instance instance, ClientExec<ReplicationCoordinator.Client> exec) throws AccumuloException, AccumuloSecurityException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/84e94a42/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
index 4355867..51a5367 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
@@ -66,10 +66,12 @@ import com.google.protobuf.InvalidProtocolBufferException;
 public class ReplicationOperationsImpl implements ReplicationOperations {
   private static final Logger log = LoggerFactory.getLogger(ReplicationOperationsImpl.class);
 
-  private Instance inst;
-  private Credentials creds;
+  private final Instance inst;
+  private final Credentials creds;
 
   public ReplicationOperationsImpl(Instance inst, Credentials creds) {
+    checkNotNull(inst);
+    checkNotNull(creds);
     this.inst = inst;
     this.creds = creds;
   }
@@ -125,32 +127,16 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
     checkNotNull(tableName);
 
     Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-    TableOperations tops = conn.tableOperations();
-    while (!tops.exists(ReplicationTable.NAME)) {
-      UtilWaitThread.sleep(200);
-    }
-
-    if (!conn.tableOperations().exists(tableName)) {
-      throw new TableNotFoundException(null, tableName, null);
-    }
-
-    String strTableId = null;
-    while (null == strTableId) {
-      strTableId = tops.tableIdMap().get(tableName);
-      if (null == strTableId) {
-        UtilWaitThread.sleep(200);
-      }
-    }
-
-    Text tableId = new Text(strTableId);
+    Text tableId = getTableId(conn, tableName);
 
     log.info("Waiting for {} to be replicated for {}", wals, tableId);
 
     log.info("Reading from metadata table");
     boolean allMetadataRefsReplicated = false;
+    final Set<Range> range = Collections.singleton(new Range(ReplicationSection.getRange()));
     while (!allMetadataRefsReplicated) {
       BatchScanner bs = conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
-      bs.setRanges(Collections.singleton(new Range(ReplicationSection.getRange())));
+      bs.setRanges(range);
       bs.fetchColumnFamily(ReplicationSection.COLF);
       try {
         allMetadataRefsReplicated = allReferencesReplicated(bs, tableId, wals);
@@ -228,13 +214,7 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
     return true;
   }
 
-  @Override
-  public Set<String> referencedFiles(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    checkNotNull(tableName);
-
-    log.debug("Collecting referenced files for replication of table {}", tableName);
-
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
+  protected Text getTableId(Connector conn, String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     TableOperations tops = conn.tableOperations();
     while (!tops.exists(ReplicationTable.NAME)) {
       UtilWaitThread.sleep(200);
@@ -252,13 +232,23 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
       }
     }
 
-    Text tableId = new Text(strTableId);
+    return new Text(strTableId);    
+  }
+
+  @Override
+  public Set<String> referencedFiles(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    checkNotNull(tableName);
+
+    log.debug("Collecting referenced files for replication of table {}", tableName);
+
+    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
+    Text tableId = getTableId(conn, tableName);
 
-    log.debug("Found id of {} for name {}", strTableId, tableName);
+    log.debug("Found id of {} for name {}", tableId, tableName);
 
     // Get the WALs currently referenced by the table
     BatchScanner metaBs = conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
-    metaBs.setRanges(Collections.singleton(MetadataSchema.TabletsSection.getRange(strTableId)));
+    metaBs.setRanges(Collections.singleton(MetadataSchema.TabletsSection.getRange(tableId.toString())));
     metaBs.fetchColumnFamily(LogColumnFamily.NAME);
     Set<String> wals = new HashSet<>();
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/84e94a42/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
index 1859c62..4e02218 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/replication/PeerNotFoundException.java
@@ -33,4 +33,8 @@ public class PeerNotFoundException extends Exception {
   public PeerNotFoundException(String message, Throwable cause) {
     super(message, cause);
   }
+
+  public PeerNotFoundException(String peer, String message, Throwable cause) {
+    super("Peer '" + peer + "' not found " + message, cause);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/84e94a42/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystem.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystem.java b/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystem.java
index e20d35f..cc51a11 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystem.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystem.java
@@ -31,6 +31,7 @@ public interface ReplicaSystem {
    * @param p Path to the resource we're reading from
    * @param status Information to replicate
    * @param target The peer
+   * @param helper Instance of ReplicaSystemHelper
    * @return A new Status for the progress that was made
    */
   public Status replicate(Path p, Status status, ReplicationTarget target, ReplicaSystemHelper helper);
@@ -39,7 +40,7 @@ public interface ReplicaSystem {
    * Configure the implementation with necessary information from the system configuration
    * <p>
    * For example, we only need one implementation for Accumulo, but, for each peer,
-   * we have a ZK quorom and instance name
+   * we have a ZK quorum and instance name
    * @param configuration
    */
   public void configure(String configuration);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/84e94a42/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystemFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystemFactory.java b/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystemFactory.java
index d1df97e..164512a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystemFactory.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/replication/ReplicaSystemFactory.java
@@ -27,6 +27,8 @@ import com.google.common.base.Preconditions;
 public class ReplicaSystemFactory {
   private static final Logger log = LoggerFactory.getLogger(ReplicaSystemFactory.class);
 
+  private ReplicaSystemFactory() {}
+
   /**
    * @param value
    *          {@link ReplicaSystem} implementation class name
@@ -53,10 +55,10 @@ public class ReplicaSystemFactory {
         return rs;
       }
 
-      throw new RuntimeException("Class is not assignable to ReplicaSystem: " + name);
+      throw new IllegalArgumentException("Class is not assignable to ReplicaSystem: " + name);
     } catch (ClassNotFoundException | InstantiationException | IllegalAccessException e) {
       log.error("Error creating ReplicaSystem object", e);
-      throw new RuntimeException(e);
+      throw new IllegalArgumentException(e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/84e94a42/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
index 619e522..a134ec8 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/Mutation.java
@@ -784,6 +784,9 @@ public class Mutation implements Writable {
    * @return An unmodifiable view of the replication sources
    */
   public Set<String> getReplicationSources() {
+    if (null == replicationSources) {
+      return EMPTY;
+    }
     return Collections.unmodifiableSet(replicationSources);
   }
   
@@ -926,9 +929,13 @@ public class Mutation implements Writable {
       }
     }
     if (0x02 == (0x02 & hasValues)) {
-      WritableUtils.writeVInt(out, replicationSources.size());
-      for (String source : replicationSources) {
-        WritableUtils.writeString(out, source);
+      if (null == replicationSources) {
+        WritableUtils.writeVInt(out, 0);
+      } else {
+        WritableUtils.writeVInt(out, replicationSources.size());
+        for (String source : replicationSources) {
+          WritableUtils.writeString(out, source);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/84e94a42/server/master/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java b/server/master/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java
index b340009..e98bc1d 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java
@@ -27,13 +27,14 @@ import org.apache.accumulo.master.Master;
 import org.apache.accumulo.trace.instrument.CountSampler;
 import org.apache.accumulo.trace.instrument.Sampler;
 import org.apache.accumulo.trace.instrument.Trace;
-import org.apache.log4j.Logger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Daemon wrapper around the {@link WorkMaker} that separates it from the Master
  */
 public class ReplicationDriver extends Daemon {
-  private static final Logger log = Logger.getLogger(ReplicationDriver.class);
+  private static final Logger log = LoggerFactory.getLogger(ReplicationDriver.class);
 
   private final Master master;
   private final AccumuloConfiguration conf;
@@ -95,7 +96,13 @@ public class ReplicationDriver extends Daemon {
       Trace.offNoFlush();
 
       // Sleep for a bit
-      UtilWaitThread.sleep(conf.getTimeInMillis(Property.MASTER_REPLICATION_SCAN_INTERVAL));
+      long sleepMillis = conf.getTimeInMillis(Property.MASTER_REPLICATION_SCAN_INTERVAL);
+      log.debug("Sleeping for {}ms before re-running", sleepMillis);
+      try {
+        Thread.sleep(sleepMillis);
+      } catch (InterruptedException e) {
+        log.error("Interrupted while sleeping", e);
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/84e94a42/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
index a03cfab..a75113b 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
@@ -121,6 +121,7 @@ public class CyclicReplicationIT {
 
       String master1UserName = "master1", master1Password = "foo";
       String master2UserName = "master2", master2Password = "bar";
+      String master1Table = master1Cluster.getInstanceName(), master2Table = master2Cluster.getInstanceName();
 
       connMaster1.securityOperations().createLocalUser(master1UserName, new PasswordToken(master1Password));
       connMaster2.securityOperations().createLocalUser(master2UserName, new PasswordToken(master2Password));
@@ -142,27 +143,27 @@ public class CyclicReplicationIT {
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
               AccumuloReplicaSystem.buildConfiguration(master1Cluster.getInstanceName(), master1Cluster.getZooKeepers())));
 
-      connMaster1.tableOperations().create(master1Cluster.getInstanceName(), false);
-      String master1TableId = connMaster1.tableOperations().tableIdMap().get(master1Cluster.getInstanceName());
+      connMaster1.tableOperations().create(master1Table, false);
+      String master1TableId = connMaster1.tableOperations().tableIdMap().get(master1Table);
       Assert.assertNotNull(master1TableId);
 
-      connMaster2.tableOperations().create(master2Cluster.getInstanceName(), false);
-      String master2TableId = connMaster2.tableOperations().tableIdMap().get(master2Cluster.getInstanceName());
+      connMaster2.tableOperations().create(master2Table, false);
+      String master2TableId = connMaster2.tableOperations().tableIdMap().get(master2Table);
       Assert.assertNotNull(master2TableId);
 
       // Replicate master1 in the master1 cluster to master2 in the master2 cluster
-      connMaster1.tableOperations().setProperty(master1Cluster.getInstanceName(), Property.TABLE_REPLICATION.getKey(), "true");
-      connMaster1.tableOperations().setProperty(master1Cluster.getInstanceName(),
+      connMaster1.tableOperations().setProperty(master1Table, Property.TABLE_REPLICATION.getKey(), "true");
+      connMaster1.tableOperations().setProperty(master1Table,
           Property.TABLE_REPLICATION_TARGETS.getKey() + master2Cluster.getInstanceName(), master2TableId);
 
       // Replicate master2 in the master2 cluster to master1 in the master2 cluster
-      connMaster2.tableOperations().setProperty(master2Cluster.getInstanceName(), Property.TABLE_REPLICATION.getKey(), "true");
-      connMaster2.tableOperations().setProperty(master2Cluster.getInstanceName(),
+      connMaster2.tableOperations().setProperty(master2Table, Property.TABLE_REPLICATION.getKey(), "true");
+      connMaster2.tableOperations().setProperty(master2Table,
           Property.TABLE_REPLICATION_TARGETS.getKey() + master1Cluster.getInstanceName(), master1TableId);
 
       // Give our replication user the ability to write to the respective table
-      connMaster1.securityOperations().grantTablePermission(master1UserName, master1Cluster.getInstanceName(), TablePermission.WRITE);
-      connMaster2.securityOperations().grantTablePermission(master2UserName, master2Cluster.getInstanceName(), TablePermission.WRITE);
+      connMaster1.securityOperations().grantTablePermission(master1UserName, master1Table, TablePermission.WRITE);
+      connMaster2.securityOperations().grantTablePermission(master2UserName, master2Table, TablePermission.WRITE);
 
       IteratorSetting summingCombiner = new IteratorSetting(50, SummingCombiner.class);
       SummingCombiner.setEncodingType(summingCombiner, Type.STRING);
@@ -170,17 +171,17 @@ public class CyclicReplicationIT {
 
       // Set a combiner on both instances that will sum multiple values
       // We can use this to verify that the mutation was not sent multiple times
-      connMaster1.tableOperations().attachIterator(master1Cluster.getInstanceName(), summingCombiner);
-      connMaster2.tableOperations().attachIterator(master2Cluster.getInstanceName(), summingCombiner);
+      connMaster1.tableOperations().attachIterator(master1Table, summingCombiner);
+      connMaster2.tableOperations().attachIterator(master2Table, summingCombiner);
 
       // Write a single entry
-      BatchWriter bw = connMaster1.createBatchWriter(master1Cluster.getInstanceName(), new BatchWriterConfig());
+      BatchWriter bw = connMaster1.createBatchWriter(master1Table, new BatchWriterConfig());
       Mutation m = new Mutation("row");
       m.put("count", "", "1");
       bw.addMutation(m);
       bw.close();
 
-      Set<String> files = connMaster1.replicationOperations().referencedFiles(master1Cluster.getInstanceName());
+      Set<String> files = connMaster1.replicationOperations().referencedFiles(master1Table);
 
       log.info("Found {} that need replication from master1", files);
 
@@ -194,22 +195,22 @@ public class CyclicReplicationIT {
       log.info("Restarted tserver on master1");
 
       // Sanity check that the element is there on master1
-      Scanner s = connMaster1.createScanner(master1Cluster.getInstanceName(), Authorizations.EMPTY);
+      Scanner s = connMaster1.createScanner(master1Table, Authorizations.EMPTY);
       Entry<Key,Value> entry = Iterables.getOnlyElement(s);
       Assert.assertEquals("1", entry.getValue().toString());
 
       // Wait for this table to replicate
-      connMaster1.replicationOperations().drain(master1Cluster.getInstanceName(), files);
+      connMaster1.replicationOperations().drain(master1Table, files);
 
       Thread.sleep(5000);
 
       // Check that the element made it to master2 only once
-      s = connMaster2.createScanner(master2Cluster.getInstanceName(), Authorizations.EMPTY);
+      s = connMaster2.createScanner(master2Table, Authorizations.EMPTY);
       entry = Iterables.getOnlyElement(s);
       Assert.assertEquals("1", entry.getValue().toString());
 
       // Wait for master2 to finish replicating it back
-      files = connMaster2.replicationOperations().referencedFiles(master2Cluster.getInstanceName());
+      files = connMaster2.replicationOperations().referencedFiles(master2Table);
 
       // Kill and restart the tserver to close the WAL on master2
       for (ProcessReference proc : master2Cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
@@ -219,16 +220,16 @@ public class CyclicReplicationIT {
       master2Cluster.exec(TabletServer.class);
 
       // Check that the element made it to master2 only once
-      s = connMaster2.createScanner(master2Cluster.getInstanceName(), Authorizations.EMPTY);
+      s = connMaster2.createScanner(master2Table, Authorizations.EMPTY);
       entry = Iterables.getOnlyElement(s);
       Assert.assertEquals("1", entry.getValue().toString());
 
-      connMaster2.replicationOperations().drain(master2Cluster.getInstanceName(), files);
+      connMaster2.replicationOperations().drain(master2Table, files);
 
       Thread.sleep(5000);
 
       // Verify that the entry wasn't sent back to master1
-      s = connMaster1.createScanner(master1Cluster.getInstanceName(), Authorizations.EMPTY);
+      s = connMaster1.createScanner(master1Table, Authorizations.EMPTY);
       entry = Iterables.getOnlyElement(s);
       Assert.assertEquals("1", entry.getValue().toString());
     } finally {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/84e94a42/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
index 6c21962..d561d2f 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
@@ -113,40 +113,40 @@ public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacIT {
     try {
       final Connector connMaster = getConnector();
       final Connector connPeer = peerCluster.getConnector("root", ROOT_PASSWORD);
-  
+
       ReplicationTable.create(connMaster);
 
       String peerUserName = "peer", peerPassword = "foo";
-  
+
       String peerClusterName = "peer";
 
       connPeer.securityOperations().createLocalUser(peerUserName, new PasswordToken(peerPassword));
-      
+
       connMaster.instanceOperations().setProperty(Property.REPLICATION_PEER_USER.getKey() + peerClusterName, peerUserName);
       connMaster.instanceOperations().setProperty(Property.REPLICATION_PEER_PASSWORD.getKey() + peerClusterName, peerPassword);
-  
+
       // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
       connMaster.instanceOperations().setProperty(
           Property.REPLICATION_PEERS.getKey() + peerClusterName,
           ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
               AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(), peerCluster.getZooKeepers())));
-  
+
       final String masterTable = "master", peerTable = "peer";
-  
+
       connMaster.tableOperations().create(masterTable);
       String masterTableId = connMaster.tableOperations().tableIdMap().get(masterTable);
       Assert.assertNotNull(masterTableId);
-  
+
       connPeer.tableOperations().create(peerTable);
       String peerTableId = connPeer.tableOperations().tableIdMap().get(peerTable);
       Assert.assertNotNull(peerTableId);
 
       connPeer.securityOperations().grantTablePermission(peerUserName, peerTable, TablePermission.WRITE);
-  
+
       // Replicate this table to the peerClusterName in a table with the peerTableId table id
       connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION.getKey(), "true");
       connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION_TARGETS.getKey() + peerClusterName, peerTableId);
-  
+
       // Write some data to table1
       BatchWriter bw = connMaster.createBatchWriter(masterTable, new BatchWriterConfig());
       for (int rows = 0; rows < 5000; rows++) {
@@ -157,23 +157,23 @@ public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacIT {
         }
         bw.addMutation(m);
       }
-  
+
       bw.close();
-  
+
       log.info("Wrote all data to master cluster");
-  
+
       final Set<String> filesNeedingReplication = connMaster.replicationOperations().referencedFiles(masterTable);
-  
+
       for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
         cluster.killProcess(ServerType.TABLET_SERVER, proc);
       }
       cluster.exec(TabletServer.class);
-  
+
       log.info("TabletServer restarted");
       for (@SuppressWarnings("unused")
       Entry<Key,Value> e : ReplicationTable.getScanner(connMaster)) {}
       log.info("TabletServer is online");
-  
+
       log.info("");
       log.info("Fetching metadata records:");
       for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
@@ -183,33 +183,33 @@ public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacIT {
           log.info(kv.getKey().toStringNoTruncate() + " " + kv.getValue());
         }
       }
-  
+
       log.info("");
       log.info("Fetching replication records:");
       for (Entry<Key,Value> kv : connMaster.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
         log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
       }
-  
+
       Future<Boolean> future = executor.submit(new Callable<Boolean>() {
-  
+
         @Override
         public Boolean call() throws Exception {
           connMaster.replicationOperations().drain(masterTable, filesNeedingReplication);
           log.info("Drain completed");
           return true;
         }
-  
+
       });
-  
+
       try {
         future.get(30, TimeUnit.SECONDS);
       } catch (TimeoutException e) {
         future.cancel(true);
         Assert.fail("Drain did not finish within 30 seconds");
       }
-  
+
       log.info("drain completed");
-  
+
       log.info("");
       log.info("Fetching metadata records:");
       for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
@@ -219,13 +219,13 @@ public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacIT {
           log.info(kv.getKey().toStringNoTruncate() + " " + kv.getValue());
         }
       }
-  
+
       log.info("");
       log.info("Fetching replication records:");
       for (Entry<Key,Value> kv : connMaster.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
         log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
       }
-  
+
       Scanner master = connMaster.createScanner(masterTable, Authorizations.EMPTY), peer = connPeer.createScanner(peerTable, Authorizations.EMPTY);
       Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
       Entry<Key,Value> masterEntry = null, peerEntry = null;
@@ -236,10 +236,10 @@ public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacIT {
             masterEntry.getKey().compareTo(peerEntry.getKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS));
         Assert.assertEquals(masterEntry.getValue(), peerEntry.getValue());
       }
-  
+
       log.info("Last master entry: " + masterEntry);
       log.info("Last peer entry: " + peerEntry);
-  
+
       Assert.assertFalse("Had more data to read from the master", masterIter.hasNext());
       Assert.assertFalse("Had more data to read from the peer", peerIter.hasNext());
     } finally {
@@ -377,7 +377,7 @@ public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacIT {
           Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
               .startsWith(masterTable1));
         }
-  
+
         log.info("Found {} records in {}", countTable, peerTable1);
 
         if (masterTable1Records != countTable) {
@@ -394,7 +394,7 @@ public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacIT {
           Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
               .startsWith(masterTable2));
         }
-  
+
         log.info("Found {} records in {}", countTable, peerTable2);
 
         if (masterTable2Records != countTable) {
@@ -605,7 +605,6 @@ public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacIT {
         Thread.sleep(500);
       }
 
-
       for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
         cluster.killProcess(ServerType.TABLET_SERVER, proc);
       }


[16/35] git commit: ACCUMULO-2766 fix wal group commit

Posted by el...@apache.org.
ACCUMULO-2766 fix wal group commit


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

Branch: refs/heads/ACCUMULO-378
Commit: 1e16159cdb25121123a3ea64df8a47c0eda54709
Parents: 05a64c1
Author: Keith Turner <kt...@apache.org>
Authored: Thu May 15 08:52:04 2014 -0400
Committer: Keith Turner <kt...@apache.org>
Committed: Tue Jun 3 12:52:19 2014 -0400

----------------------------------------------------------------------
 .../server/tabletserver/log/DfsLogger.java      | 60 +++++++++-----------
 1 file changed, 28 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/1e16159c/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java
index 01c2448..fb21ba5 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java
@@ -44,6 +44,7 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.util.Daemon;
+import org.apache.accumulo.core.util.LoggingRunnable;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.StringUtil;
 import org.apache.accumulo.server.logger.LogFileKey;
@@ -100,7 +101,8 @@ public class DfsLogger {
     @Override
     public void run() {
       ArrayList<DfsLogger.LogWork> work = new ArrayList<DfsLogger.LogWork>();
-      while (true) {
+      boolean sawClosedMarker = false;
+      while (!sawClosedMarker) {
         work.clear();
 
         try {
@@ -110,36 +112,20 @@ public class DfsLogger {
         }
         workQueue.drainTo(work);
 
-        synchronized (closeLock) {
-          if (!closed) {
-            try {
-              sync.invoke(logFile);
-            } catch (Exception ex) {
-              log.warn("Exception syncing " + ex);
-              for (DfsLogger.LogWork logWork : work) {
-                logWork.exception = ex;
-              }
-            }
-          } else {
-            for (DfsLogger.LogWork logWork : work) {
-              logWork.exception = new LogClosedException();
-            }
+        try {
+          sync.invoke(logFile);
+        } catch (Exception ex) {
+          log.warn("Exception syncing " + ex);
+          for (DfsLogger.LogWork logWork : work) {
+            logWork.exception = ex;
           }
         }
 
-        boolean sawClosedMarker = false;
         for (DfsLogger.LogWork logWork : work)
           if (logWork == CLOSED_MARKER)
             sawClosedMarker = true;
           else
             logWork.latch.countDown();
-
-        if (sawClosedMarker) {
-          synchronized (closeLock) {
-            closeLock.notifyAll();
-          }
-          break;
-        }
       }
     }
   }
@@ -200,6 +186,7 @@ public class DfsLogger {
   private Method sync;
   private Path logPath;
   private String logger;
+  private Daemon syncThread;
 
   public DfsLogger(ServerResources conf) throws IOException {
     this.conf = conf;
@@ -319,9 +306,9 @@ public class DfsLogger {
       throw ex;
     }
 
-    Thread t = new Daemon(new LogSyncingTask());
-    t.setName("Accumulo WALog thread " + toString());
-    t.start();
+    syncThread = new Daemon(new LoggingRunnable(log, new LogSyncingTask()));
+    syncThread.setName("Accumulo WALog thread " + toString());
+    syncThread.start();
   }
 
   private FSDataOutputStream create(FileSystem fs, Path logPath, boolean b, int buffersize, short replication, long blockSize) throws IOException {
@@ -386,14 +373,23 @@ public class DfsLogger {
       // thread to do work
       closed = true;
       workQueue.add(CLOSED_MARKER);
-      while (!workQueue.isEmpty())
-        try {
-          closeLock.wait();
-        } catch (InterruptedException e) {
-          log.info("Interrupted");
-        }
     }
 
+    // wait for background thread to finish before closing log file
+    if(syncThread != null){
+      try {
+        syncThread.join();
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    //expect workq should be empty at this point
+    if(workQueue.size() != 0){
+      log.error("WAL work queue not empty after sync thread exited");
+      throw new IllegalStateException("WAL work queue not empty after sync thread exited");
+    }
+    
     if (logFile != null)
       try {
         logFile.close();


[20/35] git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo

Posted by el...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo


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

Branch: refs/heads/ACCUMULO-378
Commit: 731abcebdf85fc363c48014d665ed85e01f0ebd1
Parents: a13f788 8be4a3d
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Jun 3 14:49:12 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Jun 3 14:49:12 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/tserver/log/DfsLogger.java  | 60 +++++++++-----------
 1 file changed, 28 insertions(+), 32 deletions(-)
----------------------------------------------------------------------



[08/35] git commit: ACCUMULO-2041 more review updates

Posted by el...@apache.org.
ACCUMULO-2041 more review updates


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

Branch: refs/heads/ACCUMULO-378
Commit: 459d3048eb39650ebff2c93734d2886a9d4869c7
Parents: 8049859
Author: Eric C. Newton <er...@gmail.com>
Authored: Mon Apr 21 16:28:01 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Jun 3 10:49:43 2014 -0400

----------------------------------------------------------------------
 .../accumulo/tserver/CountingIterator.java      |  78 ----------
 .../accumulo/tserver/tablet/Compactor.java      |  11 +-
 .../tserver/tablet/CountingIterator.java        |  78 ++++++++++
 .../accumulo/tserver/tablet/MinorCompactor.java |   9 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |  15 +-
 .../tserver/tablet/TabletCommitter.java         |   3 +
 .../accumulo/tserver/CountingIteratorTest.java  |   1 +
 .../apache/accumulo/tserver/RootFilesTest.java  | 150 -------------------
 .../accumulo/tserver/tablet/RootFilesTest.java  | 150 +++++++++++++++++++
 9 files changed, 247 insertions(+), 248 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/459d3048/server/tserver/src/main/java/org/apache/accumulo/tserver/CountingIterator.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/CountingIterator.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/CountingIterator.java
deleted file mode 100644
index e4ba076..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/CountingIterator.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * 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.tserver;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
-import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.WrappingIterator;
-
-public class CountingIterator extends WrappingIterator {
-
-  private long count;
-  private final ArrayList<CountingIterator> deepCopies;
-  private final AtomicLong entriesRead;
-
-  @Override
-  public CountingIterator deepCopy(IteratorEnvironment env) {
-    return new CountingIterator(this, env);
-  }
-
-  private CountingIterator(CountingIterator other, IteratorEnvironment env) {
-    setSource(other.getSource().deepCopy(env));
-    count = 0;
-    this.deepCopies = other.deepCopies;
-    this.entriesRead = other.entriesRead;
-    deepCopies.add(this);
-  }
-
-  public CountingIterator(SortedKeyValueIterator<Key,Value> source, AtomicLong entriesRead) {
-    deepCopies = new ArrayList<CountingIterator>();
-    this.setSource(source);
-    count = 0;
-    this.entriesRead = entriesRead;
-  }
-
-  @Override
-  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void next() throws IOException {
-    super.next();
-    count++;
-    if (count % 1024 == 0) {
-      entriesRead.addAndGet(1024);
-    }
-  }
-
-  public long getCount() {
-    long sum = 0;
-    for (CountingIterator dc : deepCopies) {
-      sum += dc.count;
-    }
-
-    return count + sum;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/459d3048/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
index 9a93be3..2eee5ea 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
@@ -58,7 +58,6 @@ import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.problems.ProblemType;
 import org.apache.accumulo.trace.instrument.Span;
 import org.apache.accumulo.trace.instrument.Trace;
-import org.apache.accumulo.tserver.CountingIterator;
 import org.apache.accumulo.tserver.InMemoryMap;
 import org.apache.accumulo.tserver.MinorCompactionReason;
 import org.apache.accumulo.tserver.TabletIteratorEnvironment;
@@ -134,15 +133,15 @@ public class Compactor implements Callable<CompactionStats> {
     return compactions;
   }
 
-  public Compactor(VolumeManager fs, Map<FileRef,DataFileValue> files, InMemoryMap imm, FileRef outputFile, boolean propogateDeletes,
-      AccumuloConfiguration acuTableConf, KeyExtent extent, CompactionEnv env, List<IteratorSetting> iterators, int reason) {
-    this.extent = extent;
-    this.fs = fs;
+  public Compactor(Tablet tablet, Map<FileRef,DataFileValue> files, InMemoryMap imm, FileRef outputFile, boolean propogateDeletes,
+      CompactionEnv env, List<IteratorSetting> iterators, int reason, AccumuloConfiguration tableConfiguation) {
+    this.extent = tablet.getExtent();
+    this.fs = tablet.getTabletServer().getFileSystem();
+    this.acuTableConf = tableConfiguation;
     this.filesToCompact = files;
     this.imm = imm;
     this.outputFile = outputFile;
     this.propogateDeletes = propogateDeletes;
-    this.acuTableConf = acuTableConf;
     this.env = env;
     this.iterators = iterators;
     this.reason = reason;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/459d3048/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CountingIterator.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CountingIterator.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CountingIterator.java
new file mode 100644
index 0000000..44b8460
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CountingIterator.java
@@ -0,0 +1,78 @@
+/*
+ * 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.tserver.tablet;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+
+public class CountingIterator extends WrappingIterator {
+
+  private long count;
+  private final ArrayList<CountingIterator> deepCopies;
+  private final AtomicLong entriesRead;
+
+  @Override
+  public CountingIterator deepCopy(IteratorEnvironment env) {
+    return new CountingIterator(this, env);
+  }
+
+  private CountingIterator(CountingIterator other, IteratorEnvironment env) {
+    setSource(other.getSource().deepCopy(env));
+    count = 0;
+    this.deepCopies = other.deepCopies;
+    this.entriesRead = other.entriesRead;
+    deepCopies.add(this);
+  }
+
+  public CountingIterator(SortedKeyValueIterator<Key,Value> source, AtomicLong entriesRead) {
+    deepCopies = new ArrayList<CountingIterator>();
+    this.setSource(source);
+    count = 0;
+    this.entriesRead = entriesRead;
+  }
+
+  @Override
+  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void next() throws IOException {
+    super.next();
+    count++;
+    if (count % 1024 == 0) {
+      entriesRead.addAndGet(1024);
+    }
+  }
+
+  public long getCount() {
+    long sum = 0;
+    for (CountingIterator dc : deepCopies) {
+      sum += dc.count;
+    }
+
+    return count + sum;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/459d3048/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
index 6636159..115aed7 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
@@ -23,7 +23,6 @@ import java.util.Random;
 
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
@@ -31,7 +30,6 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.FileRef;
-import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.problems.ProblemReport;
 import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.problems.ProblemType;
@@ -53,9 +51,8 @@ public class MinorCompactor extends Compactor {
     return Collections.singletonMap(mergeFile, dfv);
   }
   
-  public MinorCompactor(VolumeManager fs, InMemoryMap imm, FileRef mergeFile, DataFileValue dfv, FileRef outputFile, TableConfiguration acuTableConf,
-      KeyExtent extent, MinorCompactionReason mincReason) {
-    super(fs, toFileMap(mergeFile, dfv), imm, outputFile, true, acuTableConf, extent, new CompactionEnv() {
+  public MinorCompactor(Tablet tablet, InMemoryMap imm, FileRef mergeFile, DataFileValue dfv, FileRef outputFile, MinorCompactionReason mincReason, TableConfiguration tableConfig) {
+    super(tablet, toFileMap(mergeFile, dfv), imm, outputFile, true, new CompactionEnv() {
       
       @Override
       public boolean isCompactionEnabled() {
@@ -66,7 +63,7 @@ public class MinorCompactor extends Compactor {
       public IteratorScope getIteratorScope() {
         return IteratorScope.minc;
       }
-    }, Collections.<IteratorSetting>emptyList(), mincReason.ordinal());
+    }, Collections.<IteratorSetting>emptyList(), mincReason.ordinal(), tableConfig);
   }
   
   private boolean isTableDeleting() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/459d3048/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index bf9a905..dc2fc4d 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -856,7 +856,6 @@ public class Tablet implements TabletCommitter {
     ScanOptions opts = new ScanOptions(num, authorizations, this.defaultSecurityLabel, columns, ssiList, ssio, interruptFlag, isolated);
     return new Scanner(this, range, opts);
   }
-
   DataFileValue minorCompact(VolumeManager fs, InMemoryMap memTable, FileRef tmpDatafile, FileRef newDatafile, FileRef mergeFile,
       boolean hasQueueTime, long queued, CommitSession commitSession, long flushId, MinorCompactionReason mincReason) {
     boolean failed = false;
@@ -875,7 +874,7 @@ public class Tablet implements TabletCommitter {
         if (mergeFile != null)
           dfv = getDatafileManager().getDatafileSizes().get(mergeFile);
 
-        MinorCompactor compactor = new MinorCompactor(fs, memTable, mergeFile, dfv, tmpDatafile, tableConfiguration, extent, mincReason);
+        MinorCompactor compactor = new MinorCompactor(this, memTable, mergeFile, dfv, tmpDatafile, mincReason, tableConfiguration);
         stats = compactor.call();
       } finally {
         span.stop();
@@ -888,13 +887,13 @@ public class Tablet implements TabletCommitter {
         span.stop();
       }
       return new DataFileValue(stats.getFileSize(), stats.getEntriesWritten());
-    } catch (Exception E) {
+    } catch (Exception e) {
       failed = true;
-      throw new RuntimeException(E);
-    } catch (Error E) {
+      throw new RuntimeException(e);
+    } catch (Error e) {
       // Weird errors like "OutOfMemoryError" when trying to create the thread for the compaction
       failed = true;
-      throw new RuntimeException(E);
+      throw new RuntimeException(e);
     } finally {
       try {
         getTabletMemory().finalizeMinC();
@@ -1850,8 +1849,8 @@ public class Tablet implements TabletCommitter {
 
           // always propagate deletes, unless last batch
           boolean lastBatch = filesToCompact.isEmpty();
-          Compactor compactor = new Compactor(getTabletServer().getFileSystem(), copy, null, compactTmpName, lastBatch ? propogateDeletes : true, tableConf, extent, cenv,
-              compactionIterators, reason.ordinal());
+          Compactor compactor = new Compactor(this, copy, null, compactTmpName, lastBatch ? propogateDeletes : true, cenv,
+              compactionIterators, reason.ordinal(), tableConf);
 
           CompactionStats mcs = compactor.call();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/459d3048/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
index bd87a5b..a5d197c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
@@ -25,6 +25,9 @@ import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.tserver.InMemoryMap;
 import org.apache.accumulo.tserver.log.DfsLogger;
 
+/*
+ * A partial interface of Tablet to allow for testing of CommitSession without needing a real Tablet.
+ */
 public interface TabletCommitter {
 
   void abortCommit(CommitSession commitSession, List<Mutation> value);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/459d3048/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
index 302b025..154b121 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
@@ -26,6 +26,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
+import org.apache.accumulo.tserver.tablet.CountingIterator;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/459d3048/server/tserver/src/test/java/org/apache/accumulo/tserver/RootFilesTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/RootFilesTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/RootFilesTest.java
deleted file mode 100644
index 7cfe65c..0000000
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/RootFilesTest.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * 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.tserver;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.server.fs.FileRef;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.fs.VolumeManagerImpl;
-import org.apache.accumulo.tserver.tablet.RootFiles;
-import org.apache.hadoop.fs.Path;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * 
- */
-public class RootFilesTest {
-
-  @Rule
-  public TemporaryFolder tempFolder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
-
-  private class TestWrapper {
-    File rootTabletDir;
-    Set<FileRef> oldDatafiles;
-    String compactName;
-    FileRef tmpDatafile;
-    FileRef newDatafile;
-    VolumeManager vm;
-    AccumuloConfiguration conf;
-
-    TestWrapper(VolumeManager vm, AccumuloConfiguration conf, String compactName, String... inputFiles) throws IOException {
-      this.vm = vm;
-      this.conf = conf;
-
-      rootTabletDir = new File(tempFolder.newFolder(), "accumulo/tables/+r/root_tablet");
-      rootTabletDir.mkdirs();
-      oldDatafiles = new HashSet<FileRef>();
-      for (String filename : inputFiles) {
-        File file = new File(rootTabletDir, filename);
-        file.createNewFile();
-        oldDatafiles.add(new FileRef(file.toURI().toString()));
-      }
-
-      this.compactName = compactName;
-
-      File tmpFile = new File(rootTabletDir, compactName + "_tmp");
-      tmpFile.createNewFile();
-      tmpDatafile = new FileRef(tmpFile.toURI().toString());
-
-      newDatafile = new FileRef(new File(rootTabletDir, compactName).toURI().toString());
-    }
-
-    void prepareReplacement() throws IOException {
-      RootFiles.prepareReplacement(vm, new Path(rootTabletDir.toURI()), oldDatafiles, compactName);
-    }
-
-    void renameReplacement() throws IOException {
-      RootFiles.renameReplacement(vm, tmpDatafile, newDatafile);
-    }
-
-    public void finishReplacement() throws IOException {
-      RootFiles.finishReplacement(conf, vm, new Path(rootTabletDir.toURI()), oldDatafiles, compactName);
-    }
-
-    public Collection<String> cleanupReplacement(String... expectedFiles) throws IOException {
-      Collection<String> ret = RootFiles.cleanupReplacement(vm, vm.listStatus(new Path(rootTabletDir.toURI())), true);
-
-      HashSet<String> expected = new HashSet<String>();
-      for (String efile : expectedFiles)
-        expected.add(new File(rootTabletDir, efile).toURI().toString());
-
-      Assert.assertEquals(expected, new HashSet<String>(ret));
-
-      return ret;
-    }
-
-    public void assertFiles(String... files) {
-      HashSet<String> actual = new HashSet<String>();
-      for (File file : rootTabletDir.listFiles()) {
-        actual.add(file.getName());
-      }
-
-      HashSet<String> expected = new HashSet<String>();
-      expected.addAll(Arrays.asList(files));
-
-      Assert.assertEquals(expected, actual);
-    }
-  }
-
-  @SuppressWarnings("deprecation")
-  @Test
-  public void testFileReplacement() throws IOException {
-
-    ConfigurationCopy conf = new ConfigurationCopy();
-    conf.set(Property.INSTANCE_DFS_URI, "file:///");
-    conf.set(Property.INSTANCE_DFS_DIR, "/");
-
-    VolumeManager vm = VolumeManagerImpl.get(conf);
-
-    TestWrapper wrapper = new TestWrapper(vm, conf, "A00004.rf", "A00002.rf", "F00003.rf");
-    wrapper.prepareReplacement();
-    wrapper.renameReplacement();
-    wrapper.finishReplacement();
-    wrapper.assertFiles("A00004.rf");
-
-    wrapper = new TestWrapper(vm, conf, "A00004.rf", "A00002.rf", "F00003.rf");
-    wrapper.prepareReplacement();
-    wrapper.cleanupReplacement("A00002.rf", "F00003.rf");
-    wrapper.assertFiles("A00002.rf", "F00003.rf");
-
-    wrapper = new TestWrapper(vm, conf, "A00004.rf", "A00002.rf", "F00003.rf");
-    wrapper.prepareReplacement();
-    wrapper.renameReplacement();
-    wrapper.cleanupReplacement("A00004.rf");
-    wrapper.assertFiles("A00004.rf");
-
-    wrapper = new TestWrapper(vm, conf, "A00004.rf", "A00002.rf", "F00003.rf");
-    wrapper.prepareReplacement();
-    wrapper.renameReplacement();
-    wrapper.finishReplacement();
-    wrapper.cleanupReplacement("A00004.rf");
-    wrapper.assertFiles("A00004.rf");
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/459d3048/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/RootFilesTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/RootFilesTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/RootFilesTest.java
new file mode 100644
index 0000000..9c75a66
--- /dev/null
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/RootFilesTest.java
@@ -0,0 +1,150 @@
+/*
+ * 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.tserver.tablet;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
+import org.apache.accumulo.tserver.tablet.RootFiles;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * 
+ */
+public class RootFilesTest {
+
+  @Rule
+  public TemporaryFolder tempFolder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
+
+  private class TestWrapper {
+    File rootTabletDir;
+    Set<FileRef> oldDatafiles;
+    String compactName;
+    FileRef tmpDatafile;
+    FileRef newDatafile;
+    VolumeManager vm;
+    AccumuloConfiguration conf;
+
+    TestWrapper(VolumeManager vm, AccumuloConfiguration conf, String compactName, String... inputFiles) throws IOException {
+      this.vm = vm;
+      this.conf = conf;
+
+      rootTabletDir = new File(tempFolder.newFolder(), "accumulo/tables/+r/root_tablet");
+      rootTabletDir.mkdirs();
+      oldDatafiles = new HashSet<FileRef>();
+      for (String filename : inputFiles) {
+        File file = new File(rootTabletDir, filename);
+        file.createNewFile();
+        oldDatafiles.add(new FileRef(file.toURI().toString()));
+      }
+
+      this.compactName = compactName;
+
+      File tmpFile = new File(rootTabletDir, compactName + "_tmp");
+      tmpFile.createNewFile();
+      tmpDatafile = new FileRef(tmpFile.toURI().toString());
+
+      newDatafile = new FileRef(new File(rootTabletDir, compactName).toURI().toString());
+    }
+
+    void prepareReplacement() throws IOException {
+      RootFiles.prepareReplacement(vm, new Path(rootTabletDir.toURI()), oldDatafiles, compactName);
+    }
+
+    void renameReplacement() throws IOException {
+      RootFiles.renameReplacement(vm, tmpDatafile, newDatafile);
+    }
+
+    public void finishReplacement() throws IOException {
+      RootFiles.finishReplacement(conf, vm, new Path(rootTabletDir.toURI()), oldDatafiles, compactName);
+    }
+
+    public Collection<String> cleanupReplacement(String... expectedFiles) throws IOException {
+      Collection<String> ret = RootFiles.cleanupReplacement(vm, vm.listStatus(new Path(rootTabletDir.toURI())), true);
+
+      HashSet<String> expected = new HashSet<String>();
+      for (String efile : expectedFiles)
+        expected.add(new File(rootTabletDir, efile).toURI().toString());
+
+      Assert.assertEquals(expected, new HashSet<String>(ret));
+
+      return ret;
+    }
+
+    public void assertFiles(String... files) {
+      HashSet<String> actual = new HashSet<String>();
+      for (File file : rootTabletDir.listFiles()) {
+        actual.add(file.getName());
+      }
+
+      HashSet<String> expected = new HashSet<String>();
+      expected.addAll(Arrays.asList(files));
+
+      Assert.assertEquals(expected, actual);
+    }
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testFileReplacement() throws IOException {
+
+    ConfigurationCopy conf = new ConfigurationCopy();
+    conf.set(Property.INSTANCE_DFS_URI, "file:///");
+    conf.set(Property.INSTANCE_DFS_DIR, "/");
+
+    VolumeManager vm = VolumeManagerImpl.get(conf);
+
+    TestWrapper wrapper = new TestWrapper(vm, conf, "A00004.rf", "A00002.rf", "F00003.rf");
+    wrapper.prepareReplacement();
+    wrapper.renameReplacement();
+    wrapper.finishReplacement();
+    wrapper.assertFiles("A00004.rf");
+
+    wrapper = new TestWrapper(vm, conf, "A00004.rf", "A00002.rf", "F00003.rf");
+    wrapper.prepareReplacement();
+    wrapper.cleanupReplacement("A00002.rf", "F00003.rf");
+    wrapper.assertFiles("A00002.rf", "F00003.rf");
+
+    wrapper = new TestWrapper(vm, conf, "A00004.rf", "A00002.rf", "F00003.rf");
+    wrapper.prepareReplacement();
+    wrapper.renameReplacement();
+    wrapper.cleanupReplacement("A00004.rf");
+    wrapper.assertFiles("A00004.rf");
+
+    wrapper = new TestWrapper(vm, conf, "A00004.rf", "A00002.rf", "F00003.rf");
+    wrapper.prepareReplacement();
+    wrapper.renameReplacement();
+    wrapper.finishReplacement();
+    wrapper.cleanupReplacement("A00004.rf");
+    wrapper.assertFiles("A00004.rf");
+
+  }
+}


[12/35] ACCUMULO-2041 extract tablet classes to new files, move tablet-related code to o.a.a.tserver.tablet, make member variables private

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
new file mode 100644
index 0000000..9278cb2
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
@@ -0,0 +1,96 @@
+/*
+ * 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.tserver.tablet;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.trace.instrument.Span;
+import org.apache.accumulo.trace.instrument.Trace;
+import org.apache.accumulo.tserver.MinorCompactionReason;
+import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
+
+class MinorCompactionTask implements Runnable {
+
+  private final Tablet tablet;
+  private long queued;
+  private CommitSession commitSession;
+  private DataFileValue stats;
+  private FileRef mergeFile;
+  private long flushId;
+  private MinorCompactionReason mincReason;
+
+  MinorCompactionTask(Tablet tablet, FileRef mergeFile, CommitSession commitSession, long flushId, MinorCompactionReason mincReason) {
+    this.tablet = tablet;
+    queued = System.currentTimeMillis();
+    tablet.minorCompactionWaitingToStart();
+    this.commitSession = commitSession;
+    this.mergeFile = mergeFile;
+    this.flushId = flushId;
+    this.mincReason = mincReason;
+  }
+
+  @Override
+  public void run() {
+    tablet.isMinorCompactionRunning();
+    Span minorCompaction = Trace.on("minorCompaction");
+    try {
+      FileRef newMapfileLocation = tablet.getNextMapFilename(mergeFile == null ? "F" : "M");
+      FileRef tmpFileRef = new FileRef(newMapfileLocation.path() + "_tmp");
+      Span span = Trace.start("waitForCommits");
+      synchronized (tablet) {
+        commitSession.waitForCommitsToFinish();
+      }
+      span.stop();
+      span = Trace.start("start");
+      while (true) {
+        try {
+          // the purpose of the minor compaction start event is to keep track of the filename... in the case
+          // where the metadata table write for the minor compaction finishes and the process dies before
+          // writing the minor compaction finish event, then the start event+filename in metadata table will
+          // prevent recovery of duplicate data... the minor compaction start event could be written at any time
+          // before the metadata write for the minor compaction
+          tablet.getTabletServer().minorCompactionStarted(commitSession, commitSession.getWALogSeq() + 1, newMapfileLocation.path().toString());
+          break;
+        } catch (IOException e) {
+          Tablet.log.warn("Failed to write to write ahead log " + e.getMessage(), e);
+        }
+      }
+      span.stop();
+      span = Trace.start("compact");
+      this.stats = tablet.minorCompact(tablet.getTabletServer().getFileSystem(), tablet.getTabletMemory().getMinCMemTable(), tmpFileRef, newMapfileLocation, mergeFile, true, queued, commitSession, flushId,
+          mincReason);
+      span.stop();
+
+      if (tablet.needsSplit()) {
+        tablet.getTabletServer().executeSplit(tablet);
+      } else {
+        tablet.initiateMajorCompaction(MajorCompactionReason.NORMAL);
+      }
+    } catch (Throwable t) {
+      Tablet.log.error("Unknown error during minor compaction for extent: " + tablet.getExtent(), t);
+      throw new RuntimeException(t);
+    } finally {
+      tablet.minorCompactionComplete();
+      minorCompaction.data("extent", tablet.getExtent().toString());
+      minorCompaction.data("numEntries", Long.toString(this.stats.getNumEntries()));
+      minorCompaction.data("size", Long.toString(this.stats.getSize()));
+      minorCompaction.stop();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
new file mode 100644
index 0000000..6636159
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
@@ -0,0 +1,145 @@
+/*
+ * 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.tserver.tablet;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.master.state.tables.TableState;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.problems.ProblemReport;
+import org.apache.accumulo.server.problems.ProblemReports;
+import org.apache.accumulo.server.problems.ProblemType;
+import org.apache.accumulo.tserver.InMemoryMap;
+import org.apache.accumulo.tserver.MinorCompactionReason;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Logger;
+
+public class MinorCompactor extends Compactor {
+  
+  private static final Logger log = Logger.getLogger(MinorCompactor.class);
+  
+  private static final Map<FileRef,DataFileValue> EMPTY_MAP = Collections.emptyMap();
+  
+  private static Map<FileRef,DataFileValue> toFileMap(FileRef mergeFile, DataFileValue dfv) {
+    if (mergeFile == null)
+      return EMPTY_MAP;
+    
+    return Collections.singletonMap(mergeFile, dfv);
+  }
+  
+  public MinorCompactor(VolumeManager fs, InMemoryMap imm, FileRef mergeFile, DataFileValue dfv, FileRef outputFile, TableConfiguration acuTableConf,
+      KeyExtent extent, MinorCompactionReason mincReason) {
+    super(fs, toFileMap(mergeFile, dfv), imm, outputFile, true, acuTableConf, extent, new CompactionEnv() {
+      
+      @Override
+      public boolean isCompactionEnabled() {
+        return true;
+      }
+      
+      @Override
+      public IteratorScope getIteratorScope() {
+        return IteratorScope.minc;
+      }
+    }, Collections.<IteratorSetting>emptyList(), mincReason.ordinal());
+  }
+  
+  private boolean isTableDeleting() {
+    try {
+      return Tables.getTableState(HdfsZooInstance.getInstance(), extent.getTableId().toString()) == TableState.DELETING;
+    } catch (Exception e) {
+      log.warn("Failed to determine if table " + extent.getTableId() + " was deleting ", e);
+      return false; // can not get positive confirmation that its deleting.
+    }
+  }
+  
+  @Override
+  public CompactionStats call() {
+    log.debug("Begin minor compaction " + getOutputFile() + " " + getExtent());
+    
+    // output to new MapFile with a temporary name
+    int sleepTime = 100;
+    double growthFactor = 4;
+    int maxSleepTime = 1000 * 60 * 3; // 3 minutes
+    boolean reportedProblem = false;
+    
+    runningCompactions.add(this);
+    try {
+      do {
+        try {
+          CompactionStats ret = super.call();
+          
+          // log.debug(String.format("MinC %,d recs in | %,d recs out | %,d recs/sec | %6.3f secs | %,d bytes ",map.size(), entriesCompacted,
+          // (int)(map.size()/((t2 - t1)/1000.0)), (t2 - t1)/1000.0, estimatedSizeInBytes()));
+          
+          if (reportedProblem) {
+            ProblemReports.getInstance().deleteProblemReport(getExtent().getTableId().toString(), ProblemType.FILE_WRITE, getOutputFile());
+          }
+          
+          return ret;
+        } catch (IOException e) {
+          log.warn("MinC failed (" + e.getMessage() + ") to create " + getOutputFile() + " retrying ...");
+          ProblemReports.getInstance().report(new ProblemReport(getExtent().getTableId().toString(), ProblemType.FILE_WRITE, getOutputFile(), e));
+          reportedProblem = true;
+        } catch (RuntimeException e) {
+          // if this is coming from a user iterator, it is possible that the user could change the iterator config and that the
+          // minor compaction would succeed
+          log.warn("MinC failed (" + e.getMessage() + ") to create " + getOutputFile() + " retrying ...", e);
+          ProblemReports.getInstance().report(new ProblemReport(getExtent().getTableId().toString(), ProblemType.FILE_WRITE, getOutputFile(), e));
+          reportedProblem = true;
+        } catch (CompactionCanceledException e) {
+          throw new IllegalStateException(e);
+        }
+        
+        Random random = new Random();
+        
+        int sleep = sleepTime + random.nextInt(sleepTime);
+        log.debug("MinC failed sleeping " + sleep + " ms before retrying");
+        UtilWaitThread.sleep(sleep);
+        sleepTime = (int) Math.round(Math.min(maxSleepTime, sleepTime * growthFactor));
+        
+        // clean up
+        try {
+          if (getFileSystem().exists(new Path(getOutputFile()))) {
+            getFileSystem().deleteRecursively(new Path(getOutputFile()));
+          }
+        } catch (IOException e) {
+          log.warn("Failed to delete failed MinC file " + getOutputFile() + " " + e.getMessage());
+        }
+        
+        if (isTableDeleting())
+          return new CompactionStats(0, 0);
+        
+      } while (true);
+    } finally {
+      thread = null;
+      runningCompactions.remove(this);
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Rate.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Rate.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Rate.java
new file mode 100644
index 0000000..450fffe
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Rate.java
@@ -0,0 +1,60 @@
+/*
+ * 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.tserver.tablet;
+
+public class Rate {
+  private long lastCounter = -1;
+  private long lastTime = -1;
+  private double current = 0.0;
+  final double ratio;
+  
+  /**
+   * Turn a counter into an exponentially smoothed rate over time.
+   * 
+   * @param ratio
+   *          the rate at which each update influences the curve; must be (0., 1.0)
+   */
+  public Rate(double ratio) {
+    if (ratio <= 0. || ratio >= 1.0)
+      throw new IllegalArgumentException("ratio must be > 0. and < 1.0");
+    this.ratio = ratio;
+  }
+  
+  public double update(long counter) {
+    return update(System.currentTimeMillis(), counter);
+  }
+  
+  synchronized public double update(long when, long counter) {
+    if (lastCounter < 0) {
+      lastTime = when;
+      lastCounter = counter;
+      return current;
+    }
+    if (lastTime == when) {
+      throw new IllegalArgumentException("update time < last value");
+    }
+    double keep = 1. - ratio;
+    current = (keep * current + ratio * ((counter - lastCounter)) * 1000. / (when - lastTime));
+    lastTime = when;
+    lastCounter = counter;
+    return current;
+  }
+  
+  synchronized public double rate() {
+    return this.current;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/RootFiles.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/RootFiles.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/RootFiles.java
new file mode 100644
index 0000000..3a8bb08
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/RootFiles.java
@@ -0,0 +1,133 @@
+/*
+ * 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.tserver.tablet;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Set;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Logger;
+
+/**
+ * 
+ */
+public class RootFiles {
+
+  private static Logger log = Logger.getLogger(RootFiles.class);
+
+  public static void prepareReplacement(VolumeManager fs, Path location, Set<FileRef> oldDatafiles, String compactName) throws IOException {
+    for (FileRef ref : oldDatafiles) {
+      Path path = ref.path();
+      DatafileManager.rename(fs, path, new Path(location + "/delete+" + compactName + "+" + path.getName()));
+    }
+  }
+
+  public static void renameReplacement(VolumeManager fs, FileRef tmpDatafile, FileRef newDatafile) throws IOException {
+    if (fs.exists(newDatafile.path())) {
+      log.error("Target map file already exist " + newDatafile, new Exception());
+      throw new IllegalStateException("Target map file already exist " + newDatafile);
+    }
+
+    DatafileManager.rename(fs, tmpDatafile.path(), newDatafile.path());
+  }
+
+  public static void finishReplacement(AccumuloConfiguration acuTableConf, VolumeManager fs, Path location, Set<FileRef> oldDatafiles, String compactName)
+      throws IOException {
+    // start deleting files, if we do not finish they will be cleaned
+    // up later
+    for (FileRef ref : oldDatafiles) {
+      Path path = ref.path();
+      Path deleteFile = new Path(location + "/delete+" + compactName + "+" + path.getName());
+      if (acuTableConf.getBoolean(Property.GC_TRASH_IGNORE) || !fs.moveToTrash(deleteFile))
+        fs.deleteRecursively(deleteFile);
+    }
+  }
+
+  public static void replaceFiles(AccumuloConfiguration acuTableConf, VolumeManager fs, Path location, Set<FileRef> oldDatafiles, FileRef tmpDatafile,
+      FileRef newDatafile) throws IOException {
+    String compactName = newDatafile.path().getName();
+
+    prepareReplacement(fs, location, oldDatafiles, compactName);
+    renameReplacement(fs, tmpDatafile, newDatafile);
+    finishReplacement(acuTableConf, fs, location, oldDatafiles, compactName);
+  }
+
+  public static Collection<String> cleanupReplacement(VolumeManager fs, FileStatus[] files, boolean deleteTmp) throws IOException {
+    /*
+     * called in constructor and before major compactions
+     */
+    Collection<String> goodFiles = new ArrayList<String>(files.length);
+
+    for (FileStatus file : files) {
+
+      String path = file.getPath().toString();
+      if (file.getPath().toUri().getScheme() == null) {
+        // depending on the behavior of HDFS, if list status does not return fully qualified volumes then could switch to the default volume
+        throw new IllegalArgumentException("Require fully qualified paths " + file.getPath());
+      }
+
+      String filename = file.getPath().getName();
+
+      // check for incomplete major compaction, this should only occur
+      // for root tablet
+      if (filename.startsWith("delete+")) {
+        String expectedCompactedFile = path.substring(0, path.lastIndexOf("/delete+")) + "/" + filename.split("\\+")[1];
+        if (fs.exists(new Path(expectedCompactedFile))) {
+          // compaction finished, but did not finish deleting compacted files.. so delete it
+          if (!fs.deleteRecursively(file.getPath()))
+            log.warn("Delete of file: " + file.getPath().toString() + " return false");
+          continue;
+        }
+        // compaction did not finish, so put files back
+
+        // reset path and filename for rest of loop
+        filename = filename.split("\\+", 3)[2];
+        path = path.substring(0, path.lastIndexOf("/delete+")) + "/" + filename;
+
+        DatafileManager.rename(fs, file.getPath(), new Path(path));
+      }
+
+      if (filename.endsWith("_tmp")) {
+        if (deleteTmp) {
+          log.warn("cleaning up old tmp file: " + path);
+          if (!fs.deleteRecursively(file.getPath()))
+            log.warn("Delete of tmp file: " + file.getPath().toString() + " return false");
+
+        }
+        continue;
+      }
+
+      if (!filename.startsWith(Constants.MAPFILE_EXTENSION + "_") && !FileOperations.getValidExtensions().contains(filename.split("\\.")[1])) {
+        log.error("unknown file in tablet" + path);
+        continue;
+      }
+
+      goodFiles.add(path);
+    }
+
+    return goodFiles;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanBatch.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanBatch.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanBatch.java
new file mode 100644
index 0000000..0ea76d3
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanBatch.java
@@ -0,0 +1,29 @@
+/*
+ * 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.tserver.tablet;
+
+import java.util.List;
+
+public class ScanBatch {
+  public final boolean more;
+  public final List<KVEntry> results;
+
+  ScanBatch(List<KVEntry> results, boolean more) {
+    this.results = results;
+    this.more = more;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
new file mode 100644
index 0000000..980a082
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
@@ -0,0 +1,222 @@
+/*
+ * 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.tserver.tablet;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.data.Column;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.thrift.IterInfo;
+import org.apache.accumulo.core.iterators.IterationInterruptedException;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
+import org.apache.accumulo.core.iterators.system.ColumnQualifierFilter;
+import org.apache.accumulo.core.iterators.system.DeletingIterator;
+import org.apache.accumulo.core.iterators.system.InterruptibleIterator;
+import org.apache.accumulo.core.iterators.system.MultiIterator;
+import org.apache.accumulo.core.iterators.system.SourceSwitchingIterator.DataSource;
+import org.apache.accumulo.core.iterators.system.StatsIterator;
+import org.apache.accumulo.core.iterators.system.VisibilityFilter;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.tserver.FileManager.ScanFileManager;
+import org.apache.accumulo.tserver.InMemoryMap.MemoryIterator;
+import org.apache.accumulo.tserver.TabletIteratorEnvironment;
+import org.apache.accumulo.tserver.TabletServer;
+
+class ScanDataSource implements DataSource {
+
+  // data source state
+  private final Tablet tablet;
+  private ScanFileManager fileManager;
+  private SortedKeyValueIterator<Key,Value> iter;
+  private long expectedDeletionCount;
+  private List<MemoryIterator> memIters = null;
+  private long fileReservationId;
+  private AtomicBoolean interruptFlag;
+  private StatsIterator statsIterator;
+
+  private final ScanOptions options;
+
+  ScanDataSource(Tablet tablet, Authorizations authorizations, byte[] defaultLabels, HashSet<Column> columnSet, List<IterInfo> ssiList, Map<String,Map<String,String>> ssio,
+      AtomicBoolean interruptFlag) {
+    this.tablet = tablet;
+    expectedDeletionCount = tablet.getDataSourceDeletions();
+    this.options = new ScanOptions(-1, authorizations, defaultLabels, columnSet, ssiList, ssio, interruptFlag, false);
+    this.interruptFlag = interruptFlag;
+  }
+
+  ScanDataSource(Tablet tablet, ScanOptions options) {
+    this.tablet = tablet;
+    expectedDeletionCount = tablet.getDataSourceDeletions();
+    this.options = options;
+    this.interruptFlag = options.interruptFlag;
+  }
+
+  @Override
+  public DataSource getNewDataSource() {
+    if (!isCurrent()) {
+      // log.debug("Switching data sources during a scan");
+      if (memIters != null) {
+        tablet.getTabletMemory().returnIterators(memIters);
+        memIters = null;
+        tablet.getDatafileManager().returnFilesForScan(fileReservationId);
+        fileReservationId = -1;
+      }
+
+      if (fileManager != null)
+        fileManager.releaseOpenFiles(false);
+
+      expectedDeletionCount = tablet.getDataSourceDeletions();
+      iter = null;
+
+      return this;
+    } else
+      return this;
+  }
+
+  @Override
+  public boolean isCurrent() {
+    return expectedDeletionCount == tablet.getDataSourceDeletions();
+  }
+
+  @Override
+  public SortedKeyValueIterator<Key,Value> iterator() throws IOException {
+    if (iter == null)
+      iter = createIterator();
+    return iter;
+  }
+
+  private SortedKeyValueIterator<Key,Value> createIterator() throws IOException {
+
+    Map<FileRef,DataFileValue> files;
+
+    synchronized (tablet) {
+
+      if (memIters != null)
+        throw new IllegalStateException("Tried to create new scan iterator w/o releasing memory");
+
+      if (tablet.isClosed())
+        throw new TabletClosedException();
+
+      if (interruptFlag.get())
+        throw new IterationInterruptedException(tablet.getExtent().toString() + " " + interruptFlag.hashCode());
+
+      // only acquire the file manager when we know the tablet is open
+      if (fileManager == null) {
+        fileManager = tablet.getTabletResources().newScanFileManager();
+        tablet.addActiveScans(this);
+      }
+
+      if (fileManager.getNumOpenFiles() != 0)
+        throw new IllegalStateException("Tried to create new scan iterator w/o releasing files");
+
+      // set this before trying to get iterators in case
+      // getIterators() throws an exception
+      expectedDeletionCount = tablet.getDataSourceDeletions();
+
+      memIters = tablet.getTabletMemory().getIterators();
+      Pair<Long,Map<FileRef,DataFileValue>> reservation = tablet.getDatafileManager().reserveFilesForScan();
+      fileReservationId = reservation.getFirst();
+      files = reservation.getSecond();
+    }
+
+    Collection<InterruptibleIterator> mapfiles = fileManager.openFiles(files, options.isolated);
+
+    List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<SortedKeyValueIterator<Key,Value>>(mapfiles.size() + memIters.size());
+
+    iters.addAll(mapfiles);
+    iters.addAll(memIters);
+
+    for (SortedKeyValueIterator<Key,Value> skvi : iters)
+      ((InterruptibleIterator) skvi).setInterruptFlag(interruptFlag);
+
+    MultiIterator multiIter = new MultiIterator(iters, tablet.getExtent());
+
+    TabletIteratorEnvironment iterEnv = new TabletIteratorEnvironment(IteratorScope.scan, tablet.getTableConfiguration(), fileManager, files);
+
+    statsIterator = new StatsIterator(multiIter, TabletServer.seekCount, tablet.getScannedCounter());
+
+    DeletingIterator delIter = new DeletingIterator(statsIterator, false);
+
+    ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
+
+    ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, options.columnSet);
+
+    VisibilityFilter visFilter = new VisibilityFilter(colFilter, options.authorizations, options.defaultLabels);
+
+    return iterEnv.getTopLevelIterator(IteratorUtil
+        .loadIterators(IteratorScope.scan, visFilter, tablet.getExtent(), tablet.getTableConfiguration(), options.ssiList, options.ssio, iterEnv));
+  }
+
+  void close(boolean sawErrors) {
+
+    if (memIters != null) {
+      tablet.getTabletMemory().returnIterators(memIters);
+      memIters = null;
+      tablet.getDatafileManager().returnFilesForScan(fileReservationId);
+      fileReservationId = -1;
+    }
+
+    synchronized (tablet) {
+      if (tablet.removeScan(this) == 0)
+        tablet.notifyAll();
+    }
+
+    if (fileManager != null) {
+      fileManager.releaseOpenFiles(sawErrors);
+      fileManager = null;
+    }
+
+    if (statsIterator != null) {
+      statsIterator.report();
+    }
+
+  }
+
+  public void interrupt() {
+    interruptFlag.set(true);
+  }
+
+  @Override
+  public DataSource getDeepCopyDataSource(IteratorEnvironment env) {
+    throw new UnsupportedOperationException();
+  }
+
+  public void reattachFileManager() throws IOException {
+    if (fileManager != null)
+      fileManager.reattach();
+  }
+  
+  public void detachFileManager() {
+    if (fileManager != null)
+      fileManager.detach();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java
new file mode 100644
index 0000000..9382ea7
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tserver.tablet;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.data.Column;
+import org.apache.accumulo.core.data.thrift.IterInfo;
+import org.apache.accumulo.core.security.Authorizations;
+
+class ScanOptions {
+
+  final Authorizations authorizations;
+  final byte[] defaultLabels;
+  final Set<Column> columnSet;
+  final List<IterInfo> ssiList;
+  final Map<String,Map<String,String>> ssio;
+  final AtomicBoolean interruptFlag;
+  final int num;
+  final boolean isolated;
+
+  ScanOptions(int num, Authorizations authorizations, byte[] defaultLabels, Set<Column> columnSet, List<IterInfo> ssiList,
+      Map<String,Map<String,String>> ssio, AtomicBoolean interruptFlag, boolean isolated) {
+    this.num = num;
+    this.authorizations = authorizations;
+    this.defaultLabels = defaultLabels;
+    this.columnSet = columnSet;
+    this.ssiList = ssiList;
+    this.ssio = ssio;
+    this.interruptFlag = interruptFlag;
+    this.isolated = isolated;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
new file mode 100644
index 0000000..96379fc
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
@@ -0,0 +1,135 @@
+/*
+ * 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.tserver.tablet;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IterationInterruptedException;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.system.SourceSwitchingIterator;
+import org.apache.log4j.Logger;
+
+public class Scanner {
+  private static final Logger log = Logger.getLogger(Scanner.class);
+
+  private final Tablet tablet;
+  private final ScanOptions options;
+  private Range range;
+  private SortedKeyValueIterator<Key,Value> isolatedIter;
+  private ScanDataSource isolatedDataSource;
+  private boolean sawException = false;
+  private boolean scanClosed = false;
+
+  Scanner(Tablet tablet, Range range, ScanOptions options) {
+    this.tablet = tablet;
+    this.range = range;
+    this.options = options;
+  }
+
+  public synchronized ScanBatch read() throws IOException, TabletClosedException {
+
+    if (sawException)
+      throw new IllegalStateException("Tried to use scanner after exception occurred.");
+
+    if (scanClosed)
+      throw new IllegalStateException("Tried to use scanner after it was closed.");
+
+    Batch results = null;
+
+    ScanDataSource dataSource;
+
+    if (options.isolated) {
+      if (isolatedDataSource == null)
+        isolatedDataSource = new ScanDataSource(tablet, options);
+      dataSource = isolatedDataSource;
+    } else {
+      dataSource = new ScanDataSource(tablet, options);
+    }
+
+    try {
+
+      SortedKeyValueIterator<Key,Value> iter;
+
+      if (options.isolated) {
+        if (isolatedIter == null)
+          isolatedIter = new SourceSwitchingIterator(dataSource, true);
+        else
+          isolatedDataSource.reattachFileManager();
+        iter = isolatedIter;
+      } else {
+        iter = new SourceSwitchingIterator(dataSource, false);
+      }
+
+      results = tablet.nextBatch(iter, range, options.num, options.columnSet);
+
+      if (results.results == null) {
+        range = null;
+        return new ScanBatch(new ArrayList<KVEntry>(), false);
+      } else if (results.continueKey == null) {
+        return new ScanBatch(results.results, false);
+      } else {
+        range = new Range(results.continueKey, !results.skipContinueKey, range.getEndKey(), range.isEndKeyInclusive());
+        return new ScanBatch(results.results, true);
+      }
+
+    } catch (IterationInterruptedException iie) {
+      sawException = true;
+      if (tablet.isClosed())
+        throw new TabletClosedException(iie);
+      else
+        throw iie;
+    } catch (IOException ioe) {
+      if (tablet.shutdownInProgress()) {
+        log.debug("IOException while shutdown in progress ", ioe);
+        throw new TabletClosedException(ioe); // assume IOException was caused by execution of HDFS shutdown hook
+      }
+
+      sawException = true;
+      dataSource.close(true);
+      throw ioe;
+    } catch (RuntimeException re) {
+      sawException = true;
+      throw re;
+    } finally {
+      // code in finally block because always want
+      // to return mapfiles, even when exception is thrown
+      if (!options.isolated)
+        dataSource.close(false);
+      else 
+        dataSource.detachFileManager();
+      
+      if (results != null && results.results != null)
+        tablet.updateQueryStats(results.results.size(), results.numBytes);
+    }
+  }
+
+  // close and read are synchronized because can not call close on the data source while it is in use
+  // this could lead to the case where file iterators that are in use by a thread are returned
+  // to the pool... this would be bad
+  public void close() {
+    options.interruptFlag.set(true);
+    synchronized (this) {
+      scanClosed = true;
+      if (isolatedDataSource != null)
+        isolatedDataSource.close(false);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java
new file mode 100644
index 0000000..084503a
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java
@@ -0,0 +1,52 @@
+/*
+ * 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.tserver.tablet;
+
+import java.util.SortedMap;
+
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.server.master.state.TServerInstance;
+
+/**
+ * operations are disallowed while we split which is ok since splitting is fast
+ * 
+ * a minor compaction should have taken place before calling this so there should be relatively little left to compact
+ * 
+ * we just need to make sure major compactions aren't occurring if we have the major compactor thread decide who needs splitting we can avoid synchronization
+ * issues with major compactions
+ * 
+ */
+
+public class SplitInfo {
+  final String dir;
+  final SortedMap<FileRef,DataFileValue> datafiles;
+  final String time;
+  final long initFlushID;
+  final long initCompactID;
+  final TServerInstance lastLocation;
+
+  SplitInfo(String d, SortedMap<FileRef,DataFileValue> dfv, String time, long initFlushID, long initCompactID, TServerInstance lastLocation) {
+    this.dir = d;
+    this.datafiles = dfv;
+    this.time = time;
+    this.initFlushID = initFlushID;
+    this.initCompactID = initCompactID;
+    this.lastLocation = lastLocation;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitRowSpec.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitRowSpec.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitRowSpec.java
new file mode 100644
index 0000000..75cf91e
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitRowSpec.java
@@ -0,0 +1,29 @@
+/*
+ * 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.tserver.tablet;
+
+import org.apache.hadoop.io.Text;
+
+class SplitRowSpec {
+  final double splitRatio;
+  final Text row;
+
+  SplitRowSpec(double splitRatio, Text row) {
+    this.splitRatio = splitRatio;
+    this.row = row;
+  }
+}
\ No newline at end of file


[29/35] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'


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

Branch: refs/heads/ACCUMULO-378
Commit: a7884bbeaad3fd94a83052b2c5659a78d6ac912a
Parents: 168811d 66f3128
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Jun 4 16:24:00 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Jun 4 16:24:00 2014 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------



[06/35] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'


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

Branch: refs/heads/ACCUMULO-378
Commit: 9d8d69a251bae72359700bf6c16ed955cc78c89e
Parents: c9c6897 3bcea0d
Author: Josh Elser <el...@apache.org>
Authored: Mon Jun 2 23:42:07 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Jun 2 23:42:07 2014 -0400

----------------------------------------------------------------------
 test/system/continuous/master-agitator.pl | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[22/35] Merge remote-tracking branch 'origin/master' into ACCUMULO-378

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/e81eee7f/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 0000000,2be00fe..df3ddb5
mode 000000,100644..100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@@ -1,0 -1,2557 +1,2581 @@@
+ /*
+  * 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.tserver.tablet;
+ 
+ import java.io.ByteArrayInputStream;
+ import java.io.DataInputStream;
+ import java.io.FileNotFoundException;
+ import java.io.IOException;
+ import java.nio.charset.StandardCharsets;
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.Comparator;
+ import java.util.EnumSet;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Iterator;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Map.Entry;
+ import java.util.PriorityQueue;
+ import java.util.Set;
+ import java.util.SortedMap;
+ import java.util.TreeMap;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicLong;
+ import java.util.concurrent.atomic.AtomicReference;
+ import java.util.concurrent.locks.ReentrantLock;
+ 
+ import org.apache.accumulo.core.Constants;
+ import org.apache.accumulo.core.client.IteratorSetting;
+ import org.apache.accumulo.core.client.impl.ScannerImpl;
+ import org.apache.accumulo.core.conf.AccumuloConfiguration;
+ import org.apache.accumulo.core.conf.ConfigurationCopy;
+ import org.apache.accumulo.core.conf.ConfigurationObserver;
+ import org.apache.accumulo.core.conf.Property;
+ import org.apache.accumulo.core.constraints.Violations;
+ import org.apache.accumulo.core.data.ByteSequence;
+ import org.apache.accumulo.core.data.Column;
+ import org.apache.accumulo.core.data.ColumnUpdate;
+ import org.apache.accumulo.core.data.Key;
+ import org.apache.accumulo.core.data.KeyExtent;
+ import org.apache.accumulo.core.data.Mutation;
+ import org.apache.accumulo.core.data.Range;
+ import org.apache.accumulo.core.data.Value;
+ import org.apache.accumulo.core.data.thrift.IterInfo;
+ import org.apache.accumulo.core.data.thrift.MapFileInfo;
+ import org.apache.accumulo.core.file.FileOperations;
+ import org.apache.accumulo.core.file.FileSKVIterator;
+ import org.apache.accumulo.core.iterators.IterationInterruptedException;
+ import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+ import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+ import org.apache.accumulo.core.iterators.system.SourceSwitchingIterator;
+ import org.apache.accumulo.core.master.thrift.TabletLoadState;
+ import org.apache.accumulo.core.metadata.MetadataTable;
+ import org.apache.accumulo.core.metadata.RootTable;
+ import org.apache.accumulo.core.metadata.schema.DataFileValue;
+ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
+ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
+ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
++import org.apache.accumulo.core.protobuf.ProtobufUtil;
++import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
++import org.apache.accumulo.core.replication.StatusUtil;
++import org.apache.accumulo.core.replication.proto.Replication.Status;
+ import org.apache.accumulo.core.security.Authorizations;
+ import org.apache.accumulo.core.security.ColumnVisibility;
+ import org.apache.accumulo.core.security.Credentials;
+ import org.apache.accumulo.core.tabletserver.log.LogEntry;
+ import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
+ import org.apache.accumulo.core.util.LocalityGroupUtil;
+ import org.apache.accumulo.core.util.Pair;
+ import org.apache.accumulo.core.util.UtilWaitThread;
+ import org.apache.accumulo.server.client.HdfsZooInstance;
+ import org.apache.accumulo.server.conf.TableConfiguration;
+ import org.apache.accumulo.server.fs.FileRef;
+ import org.apache.accumulo.server.fs.VolumeManager;
+ import org.apache.accumulo.server.fs.VolumeManager.FileType;
+ import org.apache.accumulo.server.fs.VolumeUtil;
+ import org.apache.accumulo.server.fs.VolumeUtil.TabletFiles;
+ import org.apache.accumulo.server.master.state.TServerInstance;
+ import org.apache.accumulo.server.master.tableOps.CompactionIterators;
+ import org.apache.accumulo.server.problems.ProblemReport;
+ import org.apache.accumulo.server.problems.ProblemReports;
+ import org.apache.accumulo.server.problems.ProblemType;
+ import org.apache.accumulo.server.security.SystemCredentials;
+ import org.apache.accumulo.server.tablets.TabletTime;
+ import org.apache.accumulo.server.tablets.UniqueNameAllocator;
+ import org.apache.accumulo.server.util.FileUtil;
+ import org.apache.accumulo.server.util.MasterMetadataUtil;
+ import org.apache.accumulo.server.util.MetadataTableUtil;
++import org.apache.accumulo.server.util.ReplicationTableUtil;
+ import org.apache.accumulo.server.util.TabletOperations;
+ import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+ import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+ import org.apache.accumulo.trace.instrument.Span;
+ import org.apache.accumulo.trace.instrument.Trace;
+ import org.apache.accumulo.tserver.InMemoryMap;
+ import org.apache.accumulo.tserver.MinorCompactionReason;
+ import org.apache.accumulo.tserver.TConstraintViolationException;
+ import org.apache.accumulo.tserver.TLevel;
+ import org.apache.accumulo.tserver.TabletServer;
+ import org.apache.accumulo.tserver.TabletServer.TservConstraintEnv;
+ import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
+ import org.apache.accumulo.tserver.TabletStatsKeeper;
+ import org.apache.accumulo.tserver.TabletStatsKeeper.Operation;
+ import org.apache.accumulo.tserver.TooManyFilesException;
+ import org.apache.accumulo.tserver.compaction.CompactionPlan;
+ import org.apache.accumulo.tserver.compaction.CompactionStrategy;
+ import org.apache.accumulo.tserver.compaction.DefaultCompactionStrategy;
+ import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
+ import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
+ import org.apache.accumulo.tserver.compaction.WriteParameters;
+ import org.apache.accumulo.tserver.constraints.ConstraintChecker;
+ import org.apache.accumulo.tserver.log.DfsLogger;
+ import org.apache.accumulo.tserver.log.MutationReceiver;
+ import org.apache.accumulo.tserver.mastermessage.TabletStatusMessage;
+ import org.apache.accumulo.tserver.metrics.TabletServerMinCMetrics;
+ import org.apache.accumulo.tserver.tablet.Compactor.CompactionCanceledException;
+ import org.apache.accumulo.tserver.tablet.Compactor.CompactionEnv;
+ import org.apache.commons.codec.DecoderException;
+ import org.apache.commons.codec.binary.Hex;
+ import org.apache.hadoop.fs.FileStatus;
+ import org.apache.hadoop.fs.FileSystem;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.io.Text;
+ import org.apache.log4j.Logger;
+ import org.apache.zookeeper.KeeperException;
+ import org.apache.zookeeper.KeeperException.NoNodeException;
+ 
+ /**
+  * 
+  * Provide access to a single row range in a living TabletServer.
+  * 
+  */
+ public class Tablet implements TabletCommitter {
+   static private final Logger log = Logger.getLogger(Tablet.class);
+   static private final List<LogEntry> NO_LOG_ENTRIES = Collections.emptyList();
+ 
+   private final TabletServer tabletServer;
+   private final KeyExtent extent;
+   private final TabletResourceManager tabletResources;
+   private final DatafileManager datafileManager;
+   private final TableConfiguration tableConfiguration;
+   private final String tabletDirectory;
+   private final Path location; // absolute path of this tablets dir
+ 
+   private final TabletMemory tabletMemory;
+   
+   private final TabletTime tabletTime;
+   private final Object timeLock = new Object();
+   private long persistedTime;
+ 
+   private TServerInstance lastLocation;
+   private volatile boolean tableDirChecked = false;
+ 
+   private final AtomicLong dataSourceDeletions = new AtomicLong(0);
+   public long getDataSourceDeletions() { return dataSourceDeletions.get(); }
+   private final Set<ScanDataSource> activeScans = new HashSet<ScanDataSource>();
+   
+   private static enum CloseState {
+     OPEN,
+     CLOSING,
+     CLOSED,
+     COMPLETE
+   }
+ 
+   private volatile CloseState closeState = CloseState.OPEN;
+ 
+   private boolean updatingFlushID = false;
+ 
+   private long lastFlushID = -1;
+   private long lastCompactID = -1;
+   
+   static enum CompactionState { WAITING_TO_START, IN_PROGRESS };
+   private volatile CompactionState minorCompactionState = null;
+   private volatile CompactionState majorCompactionState = null;
+ 
+   private final Set<MajorCompactionReason> majorCompactionQueued = Collections.synchronizedSet(EnumSet.noneOf(MajorCompactionReason.class));
+   
+   private final AtomicReference<ConstraintChecker> constraintChecker = new AtomicReference<ConstraintChecker>();
+ 
+   private int writesInProgress = 0;
+ 
+   private final TabletStatsKeeper timer = new TabletStatsKeeper();
+ 
+   private final Rate queryRate = new Rate(0.2);
+   private long queryCount = 0;
+ 
+   private final Rate queryByteRate = new Rate(0.2);
+   private long queryBytes = 0;
+ 
+   private final Rate ingestRate = new Rate(0.2);
+   private long ingestCount = 0;
+ 
+   private final Rate ingestByteRate = new Rate(0.2);
+   private long ingestBytes = 0;
+ 
+   private byte[] defaultSecurityLabel = new byte[0];
+ 
++
+   private long lastMinorCompactionFinishTime = 0;
+   private long lastMapFileImportTime = 0;
+ 
+   private volatile long numEntries = 0;
+   private volatile long numEntriesInMemory = 0;
+ 
+   private final Rate scannedRate = new Rate(0.2);
+   private final AtomicLong scannedCount = new AtomicLong(0);
+ 
+   private final ConfigurationObserver configObserver;
+ 
+   private final int logId;
+   
+   public int getLogId() {
+     return logId;
+   }
+   
+   public static class LookupResult {
+     public List<Range> unfinishedRanges = new ArrayList<Range>();
+     public long bytesAdded = 0;
+     public long dataSize = 0;
+     public boolean closed = false;
+   }
+ 
+   FileRef getNextMapFilename(String prefix) throws IOException {
+     String extension = FileOperations.getNewFileExtension(tableConfiguration);
+     checkTabletDir();
+     return new FileRef(location.toString() + "/" + prefix + UniqueNameAllocator.getInstance().getNextName() + "." + extension);
+   }
+ 
+   private void checkTabletDir() throws IOException {
+     if (!tableDirChecked) {
+       FileStatus[] files = null;
+       try {
+         files = getTabletServer().getFileSystem().listStatus(location);
+       } catch (FileNotFoundException ex) {
+         // ignored
+       }
+       
+       if (files == null) {
+         if (location.getName().startsWith("c-"))
+           log.debug("Tablet " + extent + " had no dir, creating " + location); // its a clone dir...
+         else
+           log.warn("Tablet " + extent + " had no dir, creating " + location);
+       
+         getTabletServer().getFileSystem().mkdirs(location);
+       }
+       tableDirChecked = true;
+     }
+   }
+ 
+   public Tablet(TabletServer tabletServer, KeyExtent extent, TabletResourceManager trm, SplitInfo info) throws IOException {
+     this(tabletServer, new Text(info.getDir()), extent, trm, info.getDatafiles(), info.getTime(), info.getInitFlushID(), info.getInitCompactID(), info.getLastLocation());
+     splitCreationTime = System.currentTimeMillis();
+   }
+ 
+   private Tablet(TabletServer tabletServer, Text location, KeyExtent extent, TabletResourceManager trm,
+       SortedMap<FileRef,DataFileValue> datafiles, String time, long initFlushID, long initCompactID, TServerInstance lastLocation) throws IOException {
+     this(tabletServer, extent, location, trm, NO_LOG_ENTRIES, datafiles, time, lastLocation, new HashSet<FileRef>(), initFlushID,
+         initCompactID);
+   }
+ 
+   private static String lookupTime(AccumuloConfiguration conf, KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
+     SortedMap<Key,Value> entries;
+ 
+     if (extent.isRootTablet()) {
+       return null;
+     } else {
+       entries = new TreeMap<Key,Value>();
+       Text rowName = extent.getMetadataEntry();
+       for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+         if (entry.getKey().compareRow(rowName) == 0 && TabletsSection.ServerColumnFamily.TIME_COLUMN.hasColumns(entry.getKey())) {
+           entries.put(new Key(entry.getKey()), new Value(entry.getValue()));
+         }
+       }
+     }
+ 
+     if (entries.size() == 1)
+       return entries.values().iterator().next().toString();
+     return null;
+   }
+ 
+   private static SortedMap<FileRef,DataFileValue> lookupDatafiles(AccumuloConfiguration conf, VolumeManager fs, KeyExtent extent,
+       SortedMap<Key,Value> tabletsKeyValues) throws IOException {
+ 
+     TreeMap<FileRef,DataFileValue> datafiles = new TreeMap<FileRef,DataFileValue>();
+ 
+     if (extent.isRootTablet()) { // the meta0 tablet
+       Path location = new Path(MetadataTableUtil.getRootTabletDir());
+ 
+       // cleanUpFiles() has special handling for delete. files
+       FileStatus[] files = fs.listStatus(location);
+       Collection<String> goodPaths = RootFiles.cleanupReplacement(fs, files, true);
+       for (String good : goodPaths) {
+         Path path = new Path(good);
+         String filename = path.getName();
+         FileRef ref = new FileRef(location.toString() + "/" + filename, path);
+         DataFileValue dfv = new DataFileValue(0, 0);
+         datafiles.put(ref, dfv);
+       }
+     } else {
+ 
+       Text rowName = extent.getMetadataEntry();
+ 
+       String tableId = extent.isMeta() ? RootTable.ID : MetadataTable.ID;
+       ScannerImpl mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), SystemCredentials.get(), tableId, Authorizations.EMPTY);
+ 
+       // Commented out because when no data file is present, each tablet will scan through metadata table and return nothing
+       // reduced batch size to improve performance
+       // changed here after endKeys were implemented from 10 to 1000
+       mdScanner.setBatchSize(1000);
+ 
+       // leave these in, again, now using endKey for safety
+       mdScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
+ 
+       mdScanner.setRange(new Range(rowName));
+ 
+       for (Entry<Key,Value> entry : mdScanner) {
+ 
+         if (entry.getKey().compareRow(rowName) != 0) {
+           break;
+         }
+ 
+         FileRef ref = new FileRef(fs, entry.getKey());
+         datafiles.put(ref, new DataFileValue(entry.getValue().get()));
+       }
+     }
+     return datafiles;
+   }
+ 
+   private static List<LogEntry> lookupLogEntries(KeyExtent ke, SortedMap<Key,Value> tabletsKeyValues) {
+     List<LogEntry> logEntries = new ArrayList<LogEntry>();
+ 
+     if (ke.isMeta()) {
+       try {
+         logEntries = MetadataTableUtil.getLogEntries(SystemCredentials.get(), ke);
+       } catch (Exception ex) {
+         throw new RuntimeException("Unable to read tablet log entries", ex);
+       }
+     } else {
+       log.debug("Looking at metadata " + tabletsKeyValues);
+       Text row = ke.getMetadataEntry();
+       for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+         Key key = entry.getKey();
+         if (key.getRow().equals(row)) {
+           if (key.getColumnFamily().equals(LogColumnFamily.NAME)) {
+             logEntries.add(LogEntry.fromKeyValue(key, entry.getValue()));
+           }
+         }
+       }
+     }
+ 
+     log.debug("got " + logEntries + " for logs for " + ke);
+     return logEntries;
+   }
+ 
+   private static Set<FileRef> lookupScanFiles(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues, VolumeManager fs) throws IOException {
+     HashSet<FileRef> scanFiles = new HashSet<FileRef>();
+ 
+     Text row = extent.getMetadataEntry();
+     for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+       Key key = entry.getKey();
+       if (key.getRow().equals(row) && key.getColumnFamily().equals(ScanFileColumnFamily.NAME)) {
+         scanFiles.add(new FileRef(fs, key));
+       }
+     }
+ 
+     return scanFiles;
+   }
+ 
+   private static long lookupFlushID(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
+     Text row = extent.getMetadataEntry();
+     for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+       Key key = entry.getKey();
+       if (key.getRow().equals(row) && TabletsSection.ServerColumnFamily.FLUSH_COLUMN.equals(key.getColumnFamily(), key.getColumnQualifier()))
+         return Long.parseLong(entry.getValue().toString());
+     }
+ 
+     return -1;
+   }
+ 
+   private static long lookupCompactID(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
+     Text row = extent.getMetadataEntry();
+     for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+       Key key = entry.getKey();
+       if (key.getRow().equals(row) && TabletsSection.ServerColumnFamily.COMPACT_COLUMN.equals(key.getColumnFamily(), key.getColumnQualifier()))
+         return Long.parseLong(entry.getValue().toString());
+     }
+ 
+     return -1;
+   }
+ 
+   private static TServerInstance lookupLastServer(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
+     for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+       if (entry.getKey().getColumnFamily().compareTo(TabletsSection.LastLocationColumnFamily.NAME) == 0) {
+         return new TServerInstance(entry.getValue(), entry.getKey().getColumnQualifier());
+       }
+     }
+     return null;
+   }
+   
+   public Tablet(TabletServer tabletServer, KeyExtent extent, Text location, TabletResourceManager trm, SortedMap<Key,Value> tabletsKeyValues) throws IOException {
+     this(tabletServer, extent, location, trm, lookupLogEntries(extent, tabletsKeyValues), lookupDatafiles(tabletServer.getSystemConfiguration(), tabletServer.getFileSystem(),
+         extent, tabletsKeyValues), lookupTime(tabletServer.getSystemConfiguration(), extent, tabletsKeyValues), lookupLastServer(extent, tabletsKeyValues),
+         lookupScanFiles(extent, tabletsKeyValues, tabletServer.getFileSystem()), lookupFlushID(extent, tabletsKeyValues), lookupCompactID(extent, tabletsKeyValues));
+   }
+ 
+   /**
+    * yet another constructor - this one allows us to avoid costly lookups into the Metadata table if we already know the files we need - as at split time
+    */
+   private Tablet(final TabletServer tabletServer, final KeyExtent extent, final Text location, final TabletResourceManager trm, final List<LogEntry> rawLogEntries, final SortedMap<FileRef,DataFileValue> rawDatafiles, String time,
+       final TServerInstance lastLocation, Set<FileRef> scanFiles, long initFlushID, long initCompactID) throws IOException {
+ 
+     TabletFiles tabletPaths = VolumeUtil.updateTabletVolumes(tabletServer.getLock(), tabletServer.getFileSystem(), extent, new TabletFiles(location.toString(), rawLogEntries,
 -        rawDatafiles));
++        rawDatafiles), ReplicationConfigurationUtil.isEnabled(extent, tabletServer.getTableConfiguration(extent)));
+ 
+     Path locationPath;
+ 
+     if (tabletPaths.dir.contains(":")) {
+       locationPath = new Path(tabletPaths.dir.toString());
+     } else {
+       locationPath = tabletServer.getFileSystem().getFullPath(FileType.TABLE, extent.getTableId().toString() + tabletPaths.dir.toString());
+     }
+ 
+     final List<LogEntry> logEntries = tabletPaths.logEntries;
+     final SortedMap<FileRef,DataFileValue> datafiles = tabletPaths.datafiles;
+ 
+     this.location = locationPath;
+     this.lastLocation = lastLocation;
+     this.tabletDirectory = tabletPaths.dir;
+     this.tableConfiguration = tabletServer.getTableConfiguration(extent);
+ 
+     this.extent = extent;
+     this.tabletResources = trm;
+ 
+     this.lastFlushID = initFlushID;
+     this.lastCompactID = initCompactID;
+ 
+     if (extent.isRootTablet()) {
+       long rtime = Long.MIN_VALUE;
+       for (FileRef ref : datafiles.keySet()) {
+         Path path = ref.path();
+         FileSystem ns = tabletServer.getFileSystem().getVolumeByPath(path).getFileSystem();
+         FileSKVIterator reader = FileOperations.getInstance().openReader(path.toString(), true, ns, ns.getConf(), tabletServer.getTableConfiguration(extent));
+         long maxTime = -1;
+         try {
+ 
+           while (reader.hasTop()) {
+             maxTime = Math.max(maxTime, reader.getTopKey().getTimestamp());
+             reader.next();
+           }
+ 
+         } finally {
+           reader.close();
+         }
+ 
+         if (maxTime > rtime) {
+           time = TabletTime.LOGICAL_TIME_ID + "" + maxTime;
+           rtime = maxTime;
+         }
+       }
+     }
+     if (time == null && datafiles.isEmpty() && extent.equals(RootTable.OLD_EXTENT)) {
+       // recovery... old root tablet has no data, so time doesn't matter:
+       time = TabletTime.LOGICAL_TIME_ID + "" + Long.MIN_VALUE;
+     }
+ 
+     this.tabletServer = tabletServer;
+     this.logId = tabletServer.createLogId(extent);
+ 
+     setupDefaultSecurityLabels(extent);
+ 
+     tabletMemory = new TabletMemory(this);
+     tabletTime = TabletTime.getInstance(time);
+     persistedTime = tabletTime.getTime();
+ 
+     tableConfiguration.addObserver(configObserver = new ConfigurationObserver() {
+ 
+       private void reloadConstraints() {
+         constraintChecker.set(new ConstraintChecker(tableConfiguration));
+       }
+ 
+       @Override
+       public void propertiesChanged() {
+         reloadConstraints();
+ 
+         try {
+           setupDefaultSecurityLabels(extent);
+         } catch (Exception e) {
+           log.error("Failed to reload default security labels for extent: " + extent.toString());
+         }
+       }
+ 
+       @Override
+       public void propertyChanged(String prop) {
+         if (prop.startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey()))
+           reloadConstraints();
+         else if (prop.equals(Property.TABLE_DEFAULT_SCANTIME_VISIBILITY.getKey())) {
+           try {
+             log.info("Default security labels changed for extent: " + extent.toString());
+             setupDefaultSecurityLabels(extent);
+           } catch (Exception e) {
+             log.error("Failed to reload default security labels for extent: " + extent.toString());
+           }
+         }
+ 
+       }
+ 
+       @Override
+       public void sessionExpired() {
+         log.debug("Session expired, no longer updating per table props...");
+       }
+ 
+     });
+ 
+     tableConfiguration.getNamespaceConfiguration().addObserver(configObserver);
+ 
+     // Force a load of any per-table properties
+     configObserver.propertiesChanged();
+ 
+     if (!logEntries.isEmpty()) {
+       log.info("Starting Write-Ahead Log recovery for " + extent);
+       final long[] count = new long[2];
+       final CommitSession commitSession = getTabletMemory().getCommitSession();
+       count[1] = Long.MIN_VALUE;
+       try {
+         Set<String> absPaths = new HashSet<String>();
+         for (FileRef ref : datafiles.keySet())
+           absPaths.add(ref.path().toString());
+ 
+         tabletServer.recover(this.getTabletServer().getFileSystem(), extent, tableConfiguration, logEntries, absPaths, new MutationReceiver() {
+           @Override
+           public void receive(Mutation m) {
+             // LogReader.printMutation(m);
+             Collection<ColumnUpdate> muts = m.getUpdates();
+             for (ColumnUpdate columnUpdate : muts) {
+               if (!columnUpdate.hasTimestamp()) {
+                 // if it is not a user set timestamp, it must have been set
+                 // by the system
+                 count[1] = Math.max(count[1], columnUpdate.getTimestamp());
+               }
+             }
+             getTabletMemory().mutate(commitSession, Collections.singletonList(m));
+             count[0]++;
+           }
+         });
+ 
+         if (count[1] != Long.MIN_VALUE) {
+           tabletTime.useMaxTimeFromWALog(count[1]);
+         }
+         commitSession.updateMaxCommittedTime(tabletTime.getTime());
+ 
+         if (count[0] == 0) {
+           MetadataTableUtil.removeUnusedWALEntries(extent, logEntries, tabletServer.getLock());
++
++          // Ensure that we write a record marking each WAL as requiring replication to make sure we don't abandon the data
++          if (ReplicationConfigurationUtil.isEnabled(extent, tabletServer.getTableConfiguration(extent))) {
++            Status status = StatusUtil.fileClosed();
++            for (LogEntry logEntry : logEntries) {
++              log.debug("Writing closed status to metadata table for " + logEntry.logSet + " " + ProtobufUtil.toString(status));
++              ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logEntry.logSet, status);
++            }
++          }
++
+           logEntries.clear();
++        } else if (ReplicationConfigurationUtil.isEnabled(extent, tabletServer.getTableConfiguration(extent))) {
++          // The logs are about to be re-used, we need to record that they have data for this extent,
++          // but that they may get more data
++          Status status = StatusUtil.openWithUnknownLength();
++          for (LogEntry logEntry : logEntries) {
++            log.debug("Writing updated status to metadata table for " + logEntry.logSet + " " + ProtobufUtil.toString(status));
++            ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logEntry.logSet, status);
++          }
+         }
+ 
+       } catch (Throwable t) {
+         if (tableConfiguration.getBoolean(Property.TABLE_FAILURES_IGNORE)) {
+           log.warn("Error recovering from log files: ", t);
+         } else {
+           throw new RuntimeException(t);
+         }
+       }
+       // make some closed references that represent the recovered logs
+       currentLogs = new HashSet<DfsLogger>();
+       for (LogEntry logEntry : logEntries) {
+         for (String log : logEntry.logSet) {
+           currentLogs.add(new DfsLogger(tabletServer.getServerConfig(), log));
+         }
+       }
+ 
+       log.info("Write-Ahead Log recovery complete for " + this.extent + " (" + count[0] + " mutations applied, " + getTabletMemory().getNumEntries()
+           + " entries created)");
+     }
+ 
+     String contextName = tableConfiguration.get(Property.TABLE_CLASSPATH);
+     if (contextName != null && !contextName.equals("")) {
+       // initialize context classloader, instead of possibly waiting for it to initialize for a scan
+       // TODO this could hang, causing other tablets to fail to load - ACCUMULO-1292
+       AccumuloVFSClassLoader.getContextManager().getClassLoader(contextName);
+     }
+ 
+     // do this last after tablet is completely setup because it
+     // could cause major compaction to start
+     datafileManager = new DatafileManager(this, datafiles);
+ 
+     computeNumEntries();
+ 
+     getDatafileManager().removeFilesAfterScan(scanFiles);
+ 
+     // look for hints of a failure on the previous tablet server
+     if (!logEntries.isEmpty() || needsMajorCompaction(MajorCompactionReason.NORMAL)) {
+       // look for any temp files hanging around
+       removeOldTemporaryFiles();
+     }
+ 
+     log.log(TLevel.TABLET_HIST, extent + " opened");
+   }
+ 
+   private void removeOldTemporaryFiles() {
+     // remove any temporary files created by a previous tablet server
+     try {
+       for (FileStatus tmp : getTabletServer().getFileSystem().globStatus(new Path(location, "*_tmp"))) {
+         try {
+           log.debug("Removing old temp file " + tmp.getPath());
+           getTabletServer().getFileSystem().delete(tmp.getPath());
+         } catch (IOException ex) {
+           log.error("Unable to remove old temp file " + tmp.getPath() + ": " + ex);
+         }
+       }
+     } catch (IOException ex) {
+       log.error("Error scanning for old temp files in " + location);
+     }
+   }
+ 
+   private void setupDefaultSecurityLabels(KeyExtent extent) {
+     if (extent.isMeta()) {
+       defaultSecurityLabel = new byte[0];
+     } else {
+       try {
+         ColumnVisibility cv = new ColumnVisibility(tableConfiguration.get(Property.TABLE_DEFAULT_SCANTIME_VISIBILITY));
+         this.defaultSecurityLabel = cv.getExpression();
+       } catch (Exception e) {
+         log.error(e, e);
+         this.defaultSecurityLabel = new byte[0];
+       }
+     }
+   }
+ 
+   private LookupResult lookup(SortedKeyValueIterator<Key,Value> mmfi, List<Range> ranges, HashSet<Column> columnSet, ArrayList<KVEntry> results,
+       long maxResultsSize) throws IOException {
+ 
+     LookupResult lookupResult = new LookupResult();
+ 
+     boolean exceededMemoryUsage = false;
+     boolean tabletClosed = false;
+ 
+     Set<ByteSequence> cfset = null;
+     if (columnSet.size() > 0)
+       cfset = LocalityGroupUtil.families(columnSet);
+ 
+     for (Range range : ranges) {
+ 
+       if (exceededMemoryUsage || tabletClosed) {
+         lookupResult.unfinishedRanges.add(range);
+         continue;
+       }
+ 
+       int entriesAdded = 0;
+ 
+       try {
+         if (cfset != null)
+           mmfi.seek(range, cfset, true);
+         else
+           mmfi.seek(range, LocalityGroupUtil.EMPTY_CF_SET, false);
+ 
+         while (mmfi.hasTop()) {
+           Key key = mmfi.getTopKey();
+ 
+           KVEntry kve = new KVEntry(key, mmfi.getTopValue());
+           results.add(kve);
+           entriesAdded++;
+           lookupResult.bytesAdded += kve.estimateMemoryUsed();
+           lookupResult.dataSize += kve.numBytes();
+ 
+           exceededMemoryUsage = lookupResult.bytesAdded > maxResultsSize;
+ 
+           if (exceededMemoryUsage) {
+             addUnfinishedRange(lookupResult, range, key, false);
+             break;
+           }
+ 
+           mmfi.next();
+         }
+ 
+       } catch (TooManyFilesException tmfe) {
+         // treat this as a closed tablet, and let the client retry
+         log.warn("Tablet " + getExtent() + " has too many files, batch lookup can not run");
+         handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
+         tabletClosed = true;
+       } catch (IOException ioe) {
+         if (shutdownInProgress()) {
+           // assume HDFS shutdown hook caused this exception
+           log.debug("IOException while shutdown in progress ", ioe);
+           handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
+           tabletClosed = true;
+         } else {
+           throw ioe;
+         }
+       } catch (IterationInterruptedException iie) {
+         if (isClosed()) {
+           handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
+           tabletClosed = true;
+         } else {
+           throw iie;
+         }
+       } catch (TabletClosedException tce) {
+         handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
+         tabletClosed = true;
+       }
+ 
+     }
+ 
+     return lookupResult;
+   }
+ 
+   private void handleTabletClosedDuringScan(ArrayList<KVEntry> results, LookupResult lookupResult, boolean exceededMemoryUsage, Range range, int entriesAdded) {
+     if (exceededMemoryUsage)
+       throw new IllegalStateException("tablet should not exceed memory usage or close, not both");
+ 
+     if (entriesAdded > 0)
+       addUnfinishedRange(lookupResult, range, results.get(results.size() - 1).getKey(), false);
+     else
+       lookupResult.unfinishedRanges.add(range);
+ 
+     lookupResult.closed = true;
+   }
+ 
+   private void addUnfinishedRange(LookupResult lookupResult, Range range, Key key, boolean inclusiveStartKey) {
+     if (range.getEndKey() == null || key.compareTo(range.getEndKey()) < 0) {
+       Range nlur = new Range(new Key(key), inclusiveStartKey, range.getEndKey(), range.isEndKeyInclusive());
+       lookupResult.unfinishedRanges.add(nlur);
+     }
+   }
+ 
+   public static interface KVReceiver {
+     void receive(List<KVEntry> matches) throws IOException;
+   }
+ 
+   public LookupResult lookup(List<Range> ranges, HashSet<Column> columns, Authorizations authorizations, ArrayList<KVEntry> results, long maxResultSize,
+       List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, AtomicBoolean interruptFlag) throws IOException {
+ 
+     if (ranges.size() == 0) {
+       return new LookupResult();
+     }
+ 
+     ranges = Range.mergeOverlapping(ranges);
+     Collections.sort(ranges);
+ 
+     Range tabletRange = extent.toDataRange();
+     for (Range range : ranges) {
+       // do a test to see if this range falls within the tablet, if it does not
+       // then clip will throw an exception
+       tabletRange.clip(range);
+     }
+ 
+     ScanDataSource dataSource = new ScanDataSource(this, authorizations, this.defaultSecurityLabel, columns, ssiList, ssio, interruptFlag);
+ 
+     LookupResult result = null;
+ 
+     try {
+       SortedKeyValueIterator<Key,Value> iter = new SourceSwitchingIterator(dataSource);
+       result = lookup(iter, ranges, columns, results, maxResultSize);
+       return result;
+     } catch (IOException ioe) {
+       dataSource.close(true);
+       throw ioe;
+     } finally {
+       // code in finally block because always want
+       // to return mapfiles, even when exception is thrown
+       dataSource.close(false);
+ 
+       synchronized (this) {
+         queryCount += results.size();
+         if (result != null)
+           queryBytes += result.dataSize;
+       }
+     }
+   }
+ 
+   Batch nextBatch(SortedKeyValueIterator<Key,Value> iter, Range range, int num, Set<Column> columns) throws IOException {
+ 
+     // log.info("In nextBatch..");
+ 
+     List<KVEntry> results = new ArrayList<KVEntry>();
+     Key key = null;
+ 
+     Value value;
+     long resultSize = 0L;
+     long resultBytes = 0L;
+ 
+     long maxResultsSize = tableConfiguration.getMemoryInBytes(Property.TABLE_SCAN_MAXMEM);
+ 
+     if (columns.size() == 0) {
+       iter.seek(range, LocalityGroupUtil.EMPTY_CF_SET, false);
+     } else {
+       iter.seek(range, LocalityGroupUtil.families(columns), true);
+     }
+ 
+     Key continueKey = null;
+     boolean skipContinueKey = false;
+ 
+     boolean endOfTabletReached = false;
+     while (iter.hasTop()) {
+ 
+       value = iter.getTopValue();
+       key = iter.getTopKey();
+ 
+       KVEntry kvEntry = new KVEntry(key, value); // copies key and value
+       results.add(kvEntry);
+       resultSize += kvEntry.estimateMemoryUsed();
+       resultBytes += kvEntry.numBytes();
+ 
+       if (resultSize >= maxResultsSize || results.size() >= num) {
+         continueKey = new Key(key);
+         skipContinueKey = true;
+         break;
+       }
+ 
+       iter.next();
+     }
+ 
+     if (iter.hasTop() == false) {
+       endOfTabletReached = true;
+     }
+ 
+     if (endOfTabletReached) {
+       continueKey = null;
+     }
+ 
+     if (endOfTabletReached && results.size() == 0)
+       results = null;
+ 
+     return new Batch(skipContinueKey, results, continueKey, resultBytes);
+   }
+ 
+   /**
+    * Determine if a JVM shutdown is in progress.
+    * 
+    */
+   boolean shutdownInProgress() {
+     try {
+       Runtime.getRuntime().removeShutdownHook(new Thread(new Runnable() {
+         @Override
+         public void run() {}
+       }));
+     } catch (IllegalStateException ise) {
+       return true;
+     }
+ 
+     return false;
+   }
+ 
+   public Scanner createScanner(Range range, int num, Set<Column> columns, Authorizations authorizations, List<IterInfo> ssiList, Map<String,Map<String,String>> ssio,
+       boolean isolated, AtomicBoolean interruptFlag) {
+     // do a test to see if this range falls within the tablet, if it does not
+     // then clip will throw an exception
+     extent.toDataRange().clip(range);
+ 
+     ScanOptions opts = new ScanOptions(num, authorizations, this.defaultSecurityLabel, columns, ssiList, ssio, interruptFlag, isolated);
+     return new Scanner(this, range, opts);
+   }
+   DataFileValue minorCompact(VolumeManager fs, InMemoryMap memTable, FileRef tmpDatafile, FileRef newDatafile, FileRef mergeFile,
+       boolean hasQueueTime, long queued, CommitSession commitSession, long flushId, MinorCompactionReason mincReason) {
+     boolean failed = false;
+     long start = System.currentTimeMillis();
+     timer.incrementStatusMinor();
+ 
+     long count = 0;
+ 
+     try {
+       Span span = Trace.start("write");
+       CompactionStats stats;
+       try {
+         count = memTable.getNumEntries();
+ 
+         DataFileValue dfv = null;
+         if (mergeFile != null)
+           dfv = getDatafileManager().getDatafileSizes().get(mergeFile);
+ 
+         MinorCompactor compactor = new MinorCompactor(this, memTable, mergeFile, dfv, tmpDatafile, mincReason, tableConfiguration);
+         stats = compactor.call();
+       } finally {
+         span.stop();
+       }
+       span = Trace.start("bringOnline");
+       try {
+         getDatafileManager().bringMinorCompactionOnline(tmpDatafile, newDatafile, mergeFile, new DataFileValue(stats.getFileSize(), stats.getEntriesWritten()),
+             commitSession, flushId);
+       } finally {
+         span.stop();
+       }
+       return new DataFileValue(stats.getFileSize(), stats.getEntriesWritten());
+     } catch (Exception e) {
+       failed = true;
+       throw new RuntimeException(e);
+     } catch (Error e) {
+       // Weird errors like "OutOfMemoryError" when trying to create the thread for the compaction
+       failed = true;
+       throw new RuntimeException(e);
+     } finally {
+       try {
+         getTabletMemory().finalizeMinC();
+       } catch (Throwable t) {
+         log.error("Failed to free tablet memory", t);
+       }
+ 
+       if (!failed) {
+         lastMinorCompactionFinishTime = System.currentTimeMillis();
+       }
+       TabletServerMinCMetrics minCMetrics = getTabletServer().getMinCMetrics();
+       if (minCMetrics.isEnabled())
+         minCMetrics.add(TabletServerMinCMetrics.minc, (lastMinorCompactionFinishTime - start));
+       if (hasQueueTime) {
+         timer.updateTime(Operation.MINOR, queued, start, count, failed);
+         if (minCMetrics.isEnabled())
+           minCMetrics.add(TabletServerMinCMetrics.queue, (start - queued));
+       } else
+         timer.updateTime(Operation.MINOR, start, count, failed);
+     }
+   }
+ 
+   private synchronized MinorCompactionTask prepareForMinC(long flushId, MinorCompactionReason mincReason) {
+     CommitSession oldCommitSession = getTabletMemory().prepareForMinC();
+     otherLogs = currentLogs;
+     currentLogs = new HashSet<DfsLogger>();
+ 
+     FileRef mergeFile = getDatafileManager().reserveMergingMinorCompactionFile();
+ 
+     return new MinorCompactionTask(this, mergeFile, oldCommitSession, flushId, mincReason);
+ 
+   }
+ 
+   public void flush(long tableFlushID) {
+     boolean updateMetadata = false;
+     boolean initiateMinor = false;
+ 
+     try {
+ 
+       synchronized (this) {
+ 
+         // only want one thing at a time to update flush ID to ensure that metadata table and tablet in memory state are consistent
+         if (updatingFlushID)
+           return;
+ 
+         if (lastFlushID >= tableFlushID)
+           return;
+ 
+         if (isClosing() || isClosed() || getTabletMemory().memoryReservedForMinC())
+           return;
+ 
+         if (getTabletMemory().getMemTable().getNumEntries() == 0) {
+           lastFlushID = tableFlushID;
+           updatingFlushID = true;
+           updateMetadata = true;
+         } else
+           initiateMinor = true;
+       }
+ 
+       if (updateMetadata) {
+         Credentials creds = SystemCredentials.get();
+         // if multiple threads were allowed to update this outside of a sync block, then it would be
+         // a race condition
+         MetadataTableUtil.updateTabletFlushID(extent, tableFlushID, creds, getTabletServer().getLock());
+       } else if (initiateMinor)
+         initiateMinorCompaction(tableFlushID, MinorCompactionReason.USER);
+ 
+     } finally {
+       if (updateMetadata) {
+         synchronized (this) {
+           updatingFlushID = false;
+           this.notifyAll();
+         }
+       }
+     }
+ 
+   }
+ 
+   public boolean initiateMinorCompaction(MinorCompactionReason mincReason) {
+     if (isClosed()) {
+       // don't bother trying to get flush id if closed... could be closed after this check but that is ok... just trying to cut down on uneeded log messages....
+       return false;
+     }
+ 
+     // get the flush id before the new memmap is made available for write
+     long flushId;
+     try {
+       flushId = getFlushID();
+     } catch (NoNodeException e) {
+       log.info("Asked to initiate MinC when there was no flush id " + getExtent() + " " + e.getMessage());
+       return false;
+     }
+     return initiateMinorCompaction(flushId, mincReason);
+   }
+ 
+   public boolean minorCompactNow(MinorCompactionReason mincReason) {
+     long flushId;
+     try {
+       flushId = getFlushID();
+     } catch (NoNodeException e) {
+       log.info("Asked to initiate MinC when there was no flush id " + getExtent() + " " + e.getMessage());
+       return false;
+     }
+     MinorCompactionTask mct = createMinorCompactionTask(flushId, mincReason);
+     if (mct == null)
+       return false;
+     mct.run();
+     return true;
+   }
+ 
+   boolean initiateMinorCompaction(long flushId, MinorCompactionReason mincReason) {
+     MinorCompactionTask mct = createMinorCompactionTask(flushId, mincReason);
+     if (mct == null)
+       return false;
+     getTabletResources().executeMinorCompaction(mct);
+     return true;
+   }
+ 
+   private MinorCompactionTask createMinorCompactionTask(long flushId, MinorCompactionReason mincReason) {
+     MinorCompactionTask mct;
+     long t1, t2;
+ 
+     StringBuilder logMessage = null;
+ 
+     try {
+       synchronized (this) {
+         t1 = System.currentTimeMillis();
+ 
+         if (isClosing() || isClosed() || majorCompactionState == CompactionState.WAITING_TO_START || getTabletMemory().memoryReservedForMinC() || getTabletMemory().getMemTable().getNumEntries() == 0
+             || updatingFlushID) {
+ 
+           logMessage = new StringBuilder();
+ 
+           logMessage.append(extent.toString());
+           logMessage.append(" closeState " + closeState);
+           logMessage.append(" majorCompactionState " + majorCompactionState);
+           if (getTabletMemory() != null)
+             logMessage.append(" tabletMemory.memoryReservedForMinC() " + getTabletMemory().memoryReservedForMinC());
+           if (getTabletMemory() != null && getTabletMemory().getMemTable() != null)
+             logMessage.append(" tabletMemory.getMemTable().getNumEntries() " + getTabletMemory().getMemTable().getNumEntries());
+           logMessage.append(" updatingFlushID " + updatingFlushID);
+ 
+           return null;
+         }
+         // We're still recovering log entries
+         if (getDatafileManager() == null) {
+           logMessage = new StringBuilder();
+           logMessage.append(extent.toString());
+           logMessage.append(" datafileManager " + getDatafileManager());
+           return null;
+         }
+ 
+         mct = prepareForMinC(flushId, mincReason);
+         t2 = System.currentTimeMillis();
+       }
+     } finally {
+       // log outside of sync block
+       if (logMessage != null && log.isDebugEnabled())
+         log.debug(logMessage);
+     }
+ 
+     log.debug(String.format("MinC initiate lock %.2f secs", (t2 - t1) / 1000.0));
+     return mct;
+   }
+ 
+   public long getFlushID() throws NoNodeException {
+     try {
+       String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
+           + Constants.ZTABLE_FLUSH_ID;
+       return Long.parseLong(new String(ZooReaderWriter.getRetryingInstance().getData(zTablePath, null), StandardCharsets.UTF_8));
+     } catch (InterruptedException e) {
+       throw new RuntimeException(e);
+     } catch (NumberFormatException nfe) {
+       throw new RuntimeException(nfe);
+     } catch (KeeperException ke) {
+       if (ke instanceof NoNodeException) {
+         throw (NoNodeException) ke;
+       } else {
+         throw new RuntimeException(ke);
+       }
+     }
+   }
+ 
+   long getCompactionCancelID() {
+     String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
+         + Constants.ZTABLE_COMPACT_CANCEL_ID;
+ 
+     try {
+       return Long.parseLong(new String(ZooReaderWriter.getRetryingInstance().getData(zTablePath, null), StandardCharsets.UTF_8));
+     } catch (KeeperException e) {
+       throw new RuntimeException(e);
+     } catch (InterruptedException e) {
+       throw new RuntimeException(e);
+     }
+   }
+ 
+   public Pair<Long,List<IteratorSetting>> getCompactionID() throws NoNodeException {
+     try {
+       String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
+           + Constants.ZTABLE_COMPACT_ID;
+ 
+       String[] tokens = new String(ZooReaderWriter.getRetryingInstance().getData(zTablePath, null), StandardCharsets.UTF_8).split(",");
+       long compactID = Long.parseLong(tokens[0]);
+ 
+       CompactionIterators iters = new CompactionIterators();
+ 
+       if (tokens.length > 1) {
+         Hex hex = new Hex();
+         ByteArrayInputStream bais = new ByteArrayInputStream(hex.decode(tokens[1].split("=")[1].getBytes(StandardCharsets.UTF_8)));
+         DataInputStream dis = new DataInputStream(bais);
+ 
+         try {
+           iters.readFields(dis);
+         } catch (IOException e) {
+           throw new RuntimeException(e);
+         }
+ 
+         KeyExtent ke = new KeyExtent(extent.getTableId(), iters.getEndRow(), iters.getStartRow());
+ 
+         if (!ke.overlaps(extent)) {
+           // only use iterators if compaction range overlaps
+           iters = new CompactionIterators();
+         }
+       }
+ 
+       return new Pair<Long,List<IteratorSetting>>(compactID, iters.getIterators());
+     } catch (InterruptedException e) {
+       throw new RuntimeException(e);
+     } catch (NumberFormatException nfe) {
+       throw new RuntimeException(nfe);
+     } catch (KeeperException ke) {
+       if (ke instanceof NoNodeException) {
+         throw (NoNodeException) ke;
+       } else {
+         throw new RuntimeException(ke);
+       }
+     } catch (DecoderException e) {
+       throw new RuntimeException(e);
+     }
+   }
+ 
+   public synchronized void waitForMinC() {
+     getTabletMemory().waitForMinC();
+   }
+ 
+   private synchronized CommitSession finishPreparingMutations(long time) {
+     if (writesInProgress < 0) {
+       throw new IllegalStateException("waitingForLogs < 0 " + writesInProgress);
+     }
+ 
+     if (isClosed() || getTabletMemory() == null) {
+       return null;
+     }
+ 
+     writesInProgress++;
+     CommitSession commitSession = getTabletMemory().getCommitSession();
+     commitSession.incrementCommitsInProgress();
+     commitSession.updateMaxCommittedTime(time);
+     return commitSession;
+   }
+ 
+   public void checkConstraints() {
+     ConstraintChecker cc = constraintChecker.get();
+ 
+     if (cc.classLoaderChanged()) {
+       ConstraintChecker ncc = new ConstraintChecker(tableConfiguration);
+       constraintChecker.compareAndSet(cc, ncc);
+     }
+   }
+ 
+   public CommitSession prepareMutationsForCommit(TservConstraintEnv cenv, List<Mutation> mutations) throws TConstraintViolationException {
+ 
+     ConstraintChecker cc = constraintChecker.get();
+ 
+     List<Mutation> violators = null;
+     Violations violations = new Violations();
+     cenv.setExtent(extent);
+     for (Mutation mutation : mutations) {
+       Violations more = cc.check(cenv, mutation);
+       if (more != null) {
+         violations.add(more);
+         if (violators == null)
+           violators = new ArrayList<Mutation>();
+         violators.add(mutation);
+       }
+     }
+ 
+     long time = tabletTime.setUpdateTimes(mutations);
+ 
+     if (!violations.isEmpty()) {
+ 
+       HashSet<Mutation> violatorsSet = new HashSet<Mutation>(violators);
+       ArrayList<Mutation> nonViolators = new ArrayList<Mutation>();
+ 
+       for (Mutation mutation : mutations) {
+         if (!violatorsSet.contains(mutation)) {
+           nonViolators.add(mutation);
+         }
+       }
+ 
+       CommitSession commitSession = null;
+ 
+       if (nonViolators.size() > 0) {
+         // if everything is a violation, then it is expected that
+         // code calling this will not log or commit
+         commitSession = finishPreparingMutations(time);
+         if (commitSession == null)
+           return null;
+       }
+ 
+       throw new TConstraintViolationException(violations, violators, nonViolators, commitSession);
+     }
+ 
+     return finishPreparingMutations(time);
+   }
+ 
+   public synchronized void abortCommit(CommitSession commitSession, List<Mutation> value) {
+     if (writesInProgress <= 0) {
+       throw new IllegalStateException("waitingForLogs <= 0 " + writesInProgress);
+     }
+ 
+     if (isCloseComplete() || getTabletMemory() == null) {
+       throw new IllegalStateException("aborting commit when tablet is closed");
+     }
+ 
+     commitSession.decrementCommitsInProgress();
+     writesInProgress--;
+     if (writesInProgress == 0)
+       this.notifyAll();
+   }
+ 
+   public void commit(CommitSession commitSession, List<Mutation> mutations) {
+ 
+     int totalCount = 0;
+     long totalBytes = 0;
+ 
+     // write the mutation to the in memory table
+     for (Mutation mutation : mutations) {
+       totalCount += mutation.size();
+       totalBytes += mutation.numBytes();
+     }
+ 
+     getTabletMemory().mutate(commitSession, mutations);
+ 
+     synchronized (this) {
+       if (writesInProgress < 1) {
+         throw new IllegalStateException("commiting mutations after logging, but not waiting for any log messages");
+       }
+ 
+       if (isCloseComplete()) {
+         throw new IllegalStateException("tablet closed with outstanding messages to the logger");
+       }
+ 
+       getTabletMemory().updateMemoryUsageStats();
+ 
+       // decrement here in case an exception is thrown below
+       writesInProgress--;
+       if (writesInProgress == 0)
+         this.notifyAll();
+ 
+       commitSession.decrementCommitsInProgress();
+ 
+       numEntries += totalCount;
+       numEntriesInMemory += totalCount;
+       ingestCount += totalCount;
+       ingestBytes += totalBytes;
+     }
+   }
+ 
+   /**
+    * Closes the mapfiles associated with a Tablet. If saveState is true, a minor compaction is performed.
+    */
+   public void close(boolean saveState) throws IOException {
+     initiateClose(saveState, false, false);
+     completeClose(saveState, true);
+   }
+ 
+   void initiateClose(boolean saveState, boolean queueMinC, boolean disableWrites) {
+ 
+     if (!saveState && queueMinC) {
+       throw new IllegalArgumentException("Not saving state on close and requesting minor compactions queue does not make sense");
+     }
+ 
+     log.debug("initiateClose(saveState=" + saveState + " queueMinC=" + queueMinC + " disableWrites=" + disableWrites + ") " + getExtent());
+ 
+     MinorCompactionTask mct = null;
+ 
+     synchronized (this) {
+       if (isClosed() || isClosing() || isCloseComplete()) {
+         String msg = "Tablet " + getExtent() + " already " + closeState;
+         throw new IllegalStateException(msg);
+       }
+ 
+       // enter the closing state, no splits, minor, or major compactions can start
+       // should cause running major compactions to stop
+       closeState = CloseState.CLOSING;
+       this.notifyAll();
+ 
+       // determines if inserts and queries can still continue while minor compacting
+       if (disableWrites) {
+         closeState = CloseState.CLOSING;
+       }
+ 
+       // wait for major compactions to finish, setting closing to
+       // true should cause any running major compactions to abort
+       while (majorCompactionRunning()) {
+         try {
+           this.wait(50);
+         } catch (InterruptedException e) {
+           log.error(e.toString());
+         }
+       }
+ 
+       while (updatingFlushID) {
+         try {
+           this.wait(50);
+         } catch (InterruptedException e) {
+           log.error(e.toString());
+         }
+       }
+ 
+       if (!saveState || getTabletMemory().getMemTable().getNumEntries() == 0) {
+         return;
+       }
+ 
+       getTabletMemory().waitForMinC();
+ 
+       try {
+         mct = prepareForMinC(getFlushID(), MinorCompactionReason.CLOSE);
+       } catch (NoNodeException e) {
+         throw new RuntimeException(e);
+       }
+ 
+       if (queueMinC) {
+         getTabletResources().executeMinorCompaction(mct);
+         return;
+       }
+ 
+     }
+ 
+     // do minor compaction outside of synch block so that tablet can be read and written to while
+     // compaction runs
+     mct.run();
+   }
+ 
+   private boolean closeCompleting = false;
+ 
+   synchronized void completeClose(boolean saveState, boolean completeClose) throws IOException {
+ 
+     if (!isClosing() || isCloseComplete() || closeCompleting) {
+       throw new IllegalStateException("closeState = " + closeState);
+     }
+ 
+     log.debug("completeClose(saveState=" + saveState + " completeClose=" + completeClose + ") " + getExtent());
+ 
+     // ensure this method is only called once, also guards against multiple
+     // threads entering the method at the same time
+     closeCompleting = true;
+     closeState = CloseState.CLOSED;
+ 
+     // modify dataSourceDeletions so scans will try to switch data sources and fail because the tablet is closed
+     dataSourceDeletions.incrementAndGet();
+ 
+     for (ScanDataSource activeScan : activeScans) {
+       activeScan.interrupt();
+     }
+ 
+     // wait for reads and writes to complete
+     while (writesInProgress > 0 || activeScans.size() > 0) {
+       try {
+         this.wait(50);
+       } catch (InterruptedException e) {
+         log.error(e.toString());
+       }
+     }
+ 
+     getTabletMemory().waitForMinC();
+ 
+     if (saveState && getTabletMemory().getMemTable().getNumEntries() > 0) {
+       try {
+         prepareForMinC(getFlushID(), MinorCompactionReason.CLOSE).run();
+       } catch (NoNodeException e) {
+         throw new RuntimeException(e);
+       }
+     }
+ 
+     if (saveState) {
+       // at this point all tablet data is flushed, so do a consistency check
+       RuntimeException err = null;
+       for (int i = 0; i < 5; i++) {
+         try {
+           closeConsistencyCheck();
+           err = null;
+         } catch (RuntimeException t) {
+           err = t;
+           log.error("Consistency check fails, retrying " + t);
+           UtilWaitThread.sleep(500);
+         }
+       }
+       if (err != null) {
+         ProblemReports.getInstance().report(new ProblemReport(extent.getTableId().toString(), ProblemType.TABLET_LOAD, this.extent.toString(), err));
+         log.error("Tablet closed consistency check has failed for " + this.extent + " giving up and closing");
+       }
+     }
+ 
+     try {
+       getTabletMemory().getMemTable().delete(0);
+     } catch (Throwable t) {
+       log.error("Failed to delete mem table : " + t.getMessage(), t);
+     }
+ 
+     getTabletMemory().close();
+ 
+     // close map files
+     getTabletResources().close();
+ 
+     log.log(TLevel.TABLET_HIST, extent + " closed");
+ 
+     tableConfiguration.getNamespaceConfiguration().removeObserver(configObserver);
+     tableConfiguration.removeObserver(configObserver);
+ 
+     if (completeClose)
+       closeState = CloseState.COMPLETE;
+   }
+ 
+   private void closeConsistencyCheck() {
+ 
+     if (getTabletMemory().getMemTable().getNumEntries() != 0) {
+       String msg = "Closed tablet " + extent + " has " + getTabletMemory().getMemTable().getNumEntries() + " entries in memory";
+       log.error(msg);
+       throw new RuntimeException(msg);
+     }
+ 
+     if (getTabletMemory().memoryReservedForMinC()) {
+       String msg = "Closed tablet " + extent + " has minor compacting memory";
+       log.error(msg);
+       throw new RuntimeException(msg);
+     }
+ 
+     try {
+       Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>> fileLog = MetadataTableUtil.getFileAndLogEntries(SystemCredentials.get(), extent);
+ 
+       if (fileLog.getFirst().size() != 0) {
+         String msg = "Closed tablet " + extent + " has walog entries in " + MetadataTable.NAME + " " + fileLog.getFirst();
+         log.error(msg);
+         throw new RuntimeException(msg);
+       }
+ 
+       if (extent.isRootTablet()) {
+         if (!fileLog.getSecond().keySet().equals(getDatafileManager().getDatafileSizes().keySet())) {
+           String msg = "Data file in " + RootTable.NAME + " differ from in memory data " + extent + "  " + fileLog.getSecond().keySet() + "  "
+               + getDatafileManager().getDatafileSizes().keySet();
+           log.error(msg);
+           throw new RuntimeException(msg);
+         }
+       } else {
+         if (!fileLog.getSecond().equals(getDatafileManager().getDatafileSizes())) {
+           String msg = "Data file in " + MetadataTable.NAME + " differ from in memory data " + extent + "  " + fileLog.getSecond() + "  "
+               + getDatafileManager().getDatafileSizes();
+           log.error(msg);
+           throw new RuntimeException(msg);
+         }
+       }
+ 
+     } catch (Exception e) {
+       String msg = "Failed to do close consistency check for tablet " + extent;
+       log.error(msg, e);
+       throw new RuntimeException(msg, e);
+ 
+     }
+ 
+     if (otherLogs.size() != 0 || currentLogs.size() != 0) {
+       String msg = "Closed tablet " + extent + " has walog entries in memory currentLogs = " + currentLogs + "  otherLogs = " + otherLogs;
+       log.error(msg);
+       throw new RuntimeException(msg);
+     }
+ 
+     // TODO check lastFlushID and lostCompactID - ACCUMULO-1290
+   }
+ 
+   /**
+    * Returns a Path object representing the tablet's location on the DFS.
+    * 
+    * @return location
+    */
+   public Path getLocation() {
+     return location;
+   }
+ 
+   public synchronized boolean initiateMajorCompaction(MajorCompactionReason reason) {
+ 
+     if (isClosing() || isClosed() || !needsMajorCompaction(reason) || majorCompactionRunning() || majorCompactionQueued.contains(reason)) {
+       return false;
+     }
+ 
+     majorCompactionQueued.add(reason);
+ 
+     getTabletResources().executeMajorCompaction(getExtent(), new CompactionRunner(this, reason));
+ 
+     return false;
+   }
+ 
+   /**
+    * Returns true if a major compaction should be performed on the tablet.
+    * 
+    */
+   public boolean needsMajorCompaction(MajorCompactionReason reason) {
+     if (majorCompactionRunning())
+       return false;
+     if (reason == MajorCompactionReason.CHOP || reason == MajorCompactionReason.USER)
+       return true;
+     return getTabletResources().needsMajorCompaction(getDatafileManager().getDatafileSizes(), reason);
+   }
+ 
+   /**
+    * Returns an int representing the total block size of the f served by this tablet.
+    * 
+    * @return size
+    */
+   // this is the size of just the files
+   public long estimateTabletSize() {
+     long size = 0L;
+ 
+     for (DataFileValue sz : getDatafileManager().getDatafileSizes().values())
+       size += sz.getSize();
+ 
+     return size;
+   }
+ 
+   private boolean sawBigRow = false;
+   private long timeOfLastMinCWhenBigFreakinRowWasSeen = 0;
+   private long timeOfLastImportWhenBigFreakinRowWasSeen = 0;
+   private long splitCreationTime;
+ 
+   private SplitRowSpec findSplitRow(Collection<FileRef> files) {
+ 
+     // never split the root tablet
+     // check if we already decided that we can never split
+     // check to see if we're big enough to split
+ 
+     long splitThreshold = tableConfiguration.getMemoryInBytes(Property.TABLE_SPLIT_THRESHOLD);
+     if (extent.isRootTablet() || estimateTabletSize() <= splitThreshold) {
+       return null;
+     }
+ 
+     // have seen a big row before, do not bother checking unless a minor compaction or map file import has occurred.
+     if (sawBigRow) {
+       if (timeOfLastMinCWhenBigFreakinRowWasSeen != lastMinorCompactionFinishTime || timeOfLastImportWhenBigFreakinRowWasSeen != lastMapFileImportTime) {
+         // a minor compaction or map file import has occurred... check again
+         sawBigRow = false;
+       } else {
+         // nothing changed, do not split
+         return null;
+       }
+     }
+ 
+     SortedMap<Double,Key> keys = null;
+ 
+     try {
+       // we should make .25 below configurable
+       keys = FileUtil.findMidPoint(getTabletServer().getFileSystem(), getTabletServer().getSystemConfiguration(), extent.getPrevEndRow(), extent.getEndRow(), FileUtil.toPathStrings(files), .25);
+     } catch (IOException e) {
+       log.error("Failed to find midpoint " + e.getMessage());
+       return null;
+     }
+ 
+     // check to see if one row takes up most of the tablet, in which case we can not split
+     try {
+ 
+       Text lastRow;
+       if (extent.getEndRow() == null) {
+         Key lastKey = (Key) FileUtil.findLastKey(getTabletServer().getFileSystem(), getTabletServer().getSystemConfiguration(), files);
+         lastRow = lastKey.getRow();
+       } else {
+         lastRow = extent.getEndRow();
+       }
+ 
+       // check to see that the midPoint is not equal to the end key
+       if (keys.get(.5).compareRow(lastRow) == 0) {
+         if (keys.firstKey() < .5) {
+           Key candidate = keys.get(keys.firstKey());
+           if (candidate.compareRow(lastRow) != 0) {
+             // we should use this ratio in split size estimations
+             if (log.isTraceEnabled())
+               log.trace(String.format("Splitting at %6.2f instead of .5, row at .5 is same as end row%n", keys.firstKey()));
+             return new SplitRowSpec(keys.firstKey(), candidate.getRow());
+           }
+ 
+         }
+ 
+         log.warn("Cannot split tablet " + extent + " it contains a big row : " + lastRow);
+ 
+         sawBigRow = true;
+         timeOfLastMinCWhenBigFreakinRowWasSeen = lastMinorCompactionFinishTime;
+         timeOfLastImportWhenBigFreakinRowWasSeen = lastMapFileImportTime;
+ 
+         return null;
+       }
+       Key mid = keys.get(.5);
+       Text text = (mid == null) ? null : mid.getRow();
+       SortedMap<Double,Key> firstHalf = keys.headMap(.5);
+       if (firstHalf.size() > 0) {
+         Text beforeMid = firstHalf.get(firstHalf.lastKey()).getRow();
+         Text shorter = new Text();
+         int trunc = longestCommonLength(text, beforeMid);
+         shorter.set(text.getBytes(), 0, Math.min(text.getLength(), trunc + 1));
+         text = shorter;
+       }
+       return new SplitRowSpec(.5, text);
+     } catch (IOException e) {
+       // don't split now, but check again later
+       log.error("Failed to find lastkey " + e.getMessage());
+       return null;
+     }
+   }
+ 
+   private static int longestCommonLength(Text text, Text beforeMid) {
+     int common = 0;
+     while (common < text.getLength() && common < beforeMid.getLength() && text.getBytes()[common] == beforeMid.getBytes()[common]) {
+       common++;
+     }
+     return common;
+   }
+ 
+   private Map<FileRef,Pair<Key,Key>> getFirstAndLastKeys(SortedMap<FileRef,DataFileValue> allFiles) throws IOException {
+     Map<FileRef,Pair<Key,Key>> result = new HashMap<FileRef,Pair<Key,Key>>();
+     FileOperations fileFactory = FileOperations.getInstance();
+     VolumeManager fs = getTabletServer().getFileSystem();
+     for (Entry<FileRef,DataFileValue> entry : allFiles.entrySet()) {
+       FileRef file = entry.getKey();
+       FileSystem ns = fs.getVolumeByPath(file.path()).getFileSystem();
+       FileSKVIterator openReader = fileFactory.openReader(file.path().toString(), true, ns, ns.getConf(), this.getTableConfiguration());
+       try {
+         Key first = openReader.getFirstKey();
+         Key last = openReader.getLastKey();
+         result.put(file, new Pair<Key,Key>(first, last));
+       } finally {
+         openReader.close();
+       }
+     }
+     return result;
+   }
+ 
+   List<FileRef> findChopFiles(KeyExtent extent, Map<FileRef,Pair<Key,Key>> firstAndLastKeys, Collection<FileRef> allFiles) throws IOException {
+     List<FileRef> result = new ArrayList<FileRef>();
+     if (firstAndLastKeys == null) {
+       result.addAll(allFiles);
+       return result;
+     }
+ 
+     for (FileRef file : allFiles) {
+       Pair<Key,Key> pair = firstAndLastKeys.get(file);
+       if (pair == null) {
+         // file was created or imported after we obtained the first and last keys... there
+         // are a few options here... throw an exception which will cause the compaction to
+         // retry and also cause ugly error message that the admin has to ignore... could
+         // go get the first and last key, but this code is called while the tablet lock
+         // is held... or just compact the file....
+         result.add(file);
+       } else {
+         Key first = pair.getFirst();
+         Key last = pair.getSecond();
+         // If first and last are null, it's an empty file. Add it to the compact set so it goes away.
+         if ((first == null && last == null) || !extent.contains(first.getRow()) || !extent.contains(last.getRow())) {
+           result.add(file);
+         }
+       }
+     }
+     return result;
+   }
+ 
+   /**
+    * Returns true if this tablet needs to be split
+    * 
+    */
+   public synchronized boolean needsSplit() {
+     boolean ret;
+ 
+     if (isClosing() || isClosed())
+       ret = false;
+     else
+       ret = findSplitRow(getDatafileManager().getFiles()) != null;
+ 
+     return ret;
+   }
+ 
+   // BEGIN PRIVATE METHODS RELATED TO MAJOR COMPACTION
+ 
+   private boolean isCompactionEnabled() {
+     return !isClosing() && !getTabletServer().isMajorCompactionDisabled();
+   }
+ 
+   private CompactionStats _majorCompact(MajorCompactionReason reason) throws IOException, CompactionCanceledException {
+ 
+     long t1, t2, t3;
+ 
+     // acquire file info outside of tablet lock
+     CompactionStrategy strategy = Property.createInstanceFromPropertyName(tableConfiguration, Property.TABLE_COMPACTION_STRATEGY, CompactionStrategy.class,
+         new DefaultCompactionStrategy());
+     strategy.init(Property.getCompactionStrategyOptions(tableConfiguration));
+ 
+     Map<FileRef,Pair<Key,Key>> firstAndLastKeys = null;
+     if (reason == MajorCompactionReason.CHOP) {
+       firstAndLastKeys = getFirstAndLastKeys(getDatafileManager().getDatafileSizes());
+     } else if (reason != MajorCompactionReason.USER) {
+       MajorCompactionRequest request = new MajorCompactionRequest(extent, reason, getTabletServer().getFileSystem(), tableConfiguration);
+       request.setFiles(getDatafileManager().getDatafileSizes());
+       strategy.gatherInformation(request);
+     }
+ 
+     Map<FileRef,DataFileValue> filesToCompact;
+ 
+     int maxFilesToCompact = tableConfiguration.getCount(Property.TSERV_MAJC_THREAD_MAXOPEN);
+ 
+     CompactionStats majCStats = new CompactionStats();
+     CompactionPlan plan = null;
+ 
+     boolean propogateDeletes = false;
+ 
+     synchronized (this) {
+       // plan all that work that needs to be done in the sync block... then do the actual work
+       // outside the sync block
+ 
+       t1 = System.currentTimeMillis();
+ 
+       majorCompactionState = CompactionState.WAITING_TO_START;
+ 
+       getTabletMemory().waitForMinC();
+ 
+       t2 = System.currentTimeMillis();
+ 
+       majorCompactionState = null;
+       notifyAll();
+ 
+       VolumeManager fs = getTabletServer().getFileSystem();
+       if (extent.isRootTablet()) {
+         // very important that we call this before doing major compaction,
+         // otherwise deleted compacted files could possible be brought back
+         // at some point if the file they were compacted to was legitimately
+         // removed by a major compaction
+         RootFiles.cleanupReplacement(fs, fs.listStatus(this.location), false);
+       }
+       SortedMap<FileRef,DataFileValue> allFiles = getDatafileManager().getDatafileSizes();
+       List<FileRef> inputFiles = new ArrayList<FileRef>();
+       if (reason == MajorCompactionReason.CHOP) {
+         // enforce rules: files with keys outside our range need to be compacted
+         inputFiles.addAll(findChopFiles(extent, firstAndLastKeys, allFiles.keySet()));
+       } else if (reason == MajorCompactionReason.USER) {
+         inputFiles.addAll(allFiles.keySet());
+       } else {
+         MajorCompactionRequest request = new MajorCompactionRequest(extent, reason, fs, tableConfiguration);
+         request.setFiles(allFiles);
+         plan = strategy.getCompactionPlan(request);
+         if (plan != null)
+           inputFiles.addAll(plan.inputFiles);
+       }
+ 
+       if (inputFiles.isEmpty()) {
+         return majCStats;
+       }
+       // If no original files will exist at the end of the compaction, we do not have to propogate deletes
+       Set<FileRef> droppedFiles = new HashSet<FileRef>();
+       droppedFiles.addAll(inputFiles);
+       if (plan != null)
+         droppedFiles.addAll(plan.deleteFiles);
+       propogateDeletes = !(droppedFiles.equals(allFiles.keySet()));
+       log.debug("Major compaction plan: " + plan + " propogate deletes : " + propogateDeletes);
+       filesToCompact = new HashMap<FileRef,DataFileValue>(allFiles);
+       filesToCompact.keySet().retainAll(inputFiles);
+ 
+       t3 = System.currentTimeMillis();
+ 
+       getDatafileManager().reserveMajorCompactingFiles(filesToCompact.keySet());
+     }
+ 
+     try {
+ 
+       log.debug(String.format("MajC initiate lock %.2f secs, wait %.2f secs", (t3 - t2) / 1000.0, (t2 - t1) / 1000.0));
+ 
+       Pair<Long,List<IteratorSetting>> compactionId = null;
+       if (!propogateDeletes) {
+         // compacting everything, so update the compaction id in metadata
+         try {
+           compactionId = getCompactionID();
+         } catch (NoNodeException e) {
+           throw new RuntimeException(e);
+         }
+       }
+ 
+       List<IteratorSetting> compactionIterators = new ArrayList<IteratorSetting>();
+       if (compactionId != null) {
+         if (reason == MajorCompactionReason.USER) {
+           if (getCompactionCancelID() >= compactionId.getFirst()) {
+             // compaction was canceled
+             return majCStats;
+           }
+ 
+           synchronized (this) {
+             if (lastCompactID >= compactionId.getFirst())
+               // already compacted
+               return majCStats;
+           }
+         }
+ 
+         compactionIterators = compactionId.getSecond();
+       }
+ 
+       // need to handle case where only one file is being major compacted
+       while (filesToCompact.size() > 0) {
+ 
+         int numToCompact = maxFilesToCompact;
+ 
+         if (filesToCompact.size() > maxFilesToCompact && filesToCompact.size() < 2 * maxFilesToCompact) {
+           // on the second to last compaction pass, compact the minimum amount of files possible
+           numToCompact = filesToCompact.size() - maxFilesToCompact + 1;
+         }
+ 
+         Set<FileRef> smallestFiles = removeSmallest(filesToCompact, numToCompact);
+ 
+         FileRef fileName = getNextMapFilename((filesToCompact.size() == 0 && !propogateDeletes) ? "A" : "C");
+         FileRef compactTmpName = new FileRef(fileName.path().toString() + "_tmp");
+ 
+         AccumuloConfiguration tableConf = createTableConfiguration(tableConfiguration, plan);
+ 
+         Span span = Trace.start("compactFiles");
+         try {
+ 
+           CompactionEnv cenv = new CompactionEnv() {
+             @Override
+             public boolean isCompactionEnabled() {
+               return Tablet.this.isCompactionEnabled();
+             }
+ 
+             @Override
+             public IteratorScope getIteratorScope() {
+               return IteratorScope.majc;
+             }
+           };
+ 
+           HashMap<FileRef,DataFileValue> copy = new HashMap<FileRef,DataFileValue>(getDatafileManager().getDatafileSizes());
+           if (!copy.keySet().containsAll(smallestFiles))
+             throw new IllegalStateException("Cannot find data file values for " + smallestFiles);
+ 
+           copy.keySet().retainAll(smallestFiles);
+ 
+           log.debug("Starting MajC " + extent + " (" + reason + ") " + copy.keySet() + " --> " + compactTmpName + "  " + compactionIterators);
+ 
+           // always propagate deletes, unless last batch
+           boolean lastBatch = filesToCompact.isEmpty();
+           Compactor compactor = new Compactor(this, copy, null, compactTmpName, lastBatch ? propogateDeletes : true, cenv,
+               compactionIterators, reason.ordinal(), tableConf);
+ 
+           CompactionStats mcs = compactor.call();
+ 
+           span.data("files", "" + smallestFiles.size());
+           span.data("read", "" + mcs.getEntriesRead());
+           span.data("written", "" + mcs.getEntriesWritten());
+           majCStats.add(mcs);
+ 
+           if (lastBatch && plan != null && plan.deleteFiles != null) {
+             smallestFiles.addAll(plan.deleteFiles);
+           }
+           getDatafileManager().bringMajorCompactionOnline(smallestFiles, compactTmpName, fileName,
+               filesToCompact.size() == 0 && compactionId != null ? compactionId.getFirst() : null,
+               new DataFileValue(mcs.getFileSize(), mcs.getEntriesWritten()));
+ 
+           // when major compaction produces a file w/ zero entries, it will be deleted... do not want
+           // to add the deleted file
+           if (filesToCompact.size() > 0 && mcs.getEntriesWritten() > 0) {
+             filesToCompact.put(fileName, new DataFileValue(mcs.getFileSize(), mcs.getEntriesWritten()));
+           }
+         } finally {
+           span.stop();
+         }
+ 
+       }
+       return majCStats;
+     } finally {
+       synchronized (Tablet.this) {
+         getDatafileManager().clearMajorCompactingFile();
+       }
+     }
+   }
+ 
+   private AccumuloConfiguration createTableConfiguration(TableConfiguration base, CompactionPlan plan) {
+     if (plan == null || plan.writeParameters == null)
+       return base;
+     WriteParameters p = plan.writeParameters;
+     ConfigurationCopy result = new ConfigurationCopy(base);
+     if (p.getHdfsBlockSize() > 0)
+       result.set(Property.TABLE_FILE_BLOCK_SIZE, "" + p.getHdfsBlockSize());
+     if (p.getBlockSize() > 0)
+       result.set(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE, "" + p.getBlockSize());
+     if (p.getIndexBlockSize() > 0)
+       result.set(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX, "" + p.getBlockSize());
+     if (p.getCompressType() != null)
+       result.set(Property.TABLE_FILE_COMPRESSION_TYPE, p.getCompressType());
+     if (p.getReplication() != 0)
+       result.set(Property.TABLE_FILE_REPLICATION, "" + p.getReplication());
+     return result;
+   }
+ 
+   private Set<FileRef> removeSmallest(Map<FileRef,DataFileValue> filesToCompact, int maxFilesToCompact) {
+     // ensure this method works properly when multiple files have the same size
+ 
+     PriorityQueue<Pair<FileRef,Long>> fileHeap = new PriorityQueue<Pair<FileRef,Long>>(filesToCompact.size(), new Comparator<Pair<FileRef,Long>>() {
+       @Override
+       public int compare(Pair<FileRef,Long> o1, Pair<FileRef,Long> o2) {
+         if (o1.getSecond() == o2.getSecond())
+           return o1.getFirst().compareTo(o2.getFirst());
+         if (o1.getSecond() < o2.getSecond())
+           return -1;
+         return 1;
+       }
+     });
+ 
+     for (Iterator<Entry<FileRef,DataFileValue>> iterator = filesToCompact.entrySet().iterator(); iterator.hasNext();) {
+       Entry<FileRef,DataFileValue> entry = iterator.next();
+       fileHeap.add(new Pair<FileRef,Long>(entry.getKey(), entry.getValue().getSize()));
+     }
+ 
+     Set<FileRef> smallestFiles = new HashSet<FileRef>();
+     while (smallestFiles.size() < maxFilesToCompact && fileHeap.size() > 0) {
+       Pair<FileRef,Long> pair = fileHeap.remove();
+       filesToCompact.remove(pair.getFirst());
+       smallestFiles.add(pair.getFirst());
+     }
+ 
+     return smallestFiles;
+   }
+ 
+   // END PRIVATE METHODS RELATED TO MAJOR COMPACTION
+ 
+   /**
+    * Performs a major compaction on the tablet. If needsSplit() returns true, the tablet is split and a reference to the new tablet is returned.
+    */
+ 
+   CompactionStats majorCompact(MajorCompactionReason reason, long queued) {
+ 
+     CompactionStats majCStats = null;
+     boolean success = false;
+     long start = System.currentTimeMillis();
+ 
+     // Always trace majC
+     Span span = Trace.on("majorCompaction");
+ 
+     try {
+       timer.incrementStatusMajor();
+ 
+       synchronized (this) {
+         // check that compaction is still needed - defer to splitting
+         majorCompactionQueued.remove(reason);
+ 
+         if (isClosing() || isClosed ()|| !needsMajorCompaction(reason) || majorCompactionRunning() || needsSplit()) {
+           return null;
+         }
+ 
+         majorCompactionState = CompactionState.WAITING_TO_START;
+       }
+ 
+       try {
+         majCStats = _majorCompact(reason);
+         if (reason == MajorCompactionReason.CHOP) {
+           MetadataTableUtil.chopped(getExtent(), this.getTabletServer().getLock());
+           getTabletServer().enqueueMasterMessage(new TabletStatusMessage(TabletLoadState.CHOPPED, extent));
+         }
+         success = true;
+       } catch (CompactionCanceledException mcce) {
+         log.debug("Major compaction canceled, extent = " + getExtent());
+         throw new RuntimeException(mcce);
+       } catch (Throwable t) {
+         log.error("MajC Failed, extent = " + getExtent());
+         log.error("MajC Failed, message = " + (t.getMessage() == null ? t.getClass().getName() : t.getMessage()), t);
+         throw new RuntimeException(t);
+       } finally {
+         // ensure we always reset boolean, even
+         // when an exception is thrown
+         synchronized (this) {
+           majorCompactionState = null;
+           this.notifyAll();
+         }
+ 
+         Span curr = Trace.currentTrace();
+         curr.data("extent", "" + getExtent());
+         if (majCStats != null) {
+           curr.data("read", "" + majCStats.getEntriesRead());
+           curr.data("written", "" + majCStats.getEntriesWritten());
+         }
+       }
+     } finally {
+       span.stop();
+       long count = 0;
+       if (majCStats != null)
+         count = majCStats.getEntriesRead();
+       timer.updateTime(Operation.MAJOR, queued, start, count, !success);
+     }
+ 
+     return majCStats;
+   }
+ 
+   /**
+    * Returns a KeyExtent object representing this tablet's key range.
+    * 
+    * @return extent
+    */
+   public KeyExtent getExtent() {
+     return extent;
+   }
+ 
+   synchronized void computeNumEntries() {
+     Collection<DataFileValue> vals = getDatafileManager().getDatafileSizes().values();
+ 
+     long numEntries = 0;
+ 
+     for (DataFileValue tableValue : vals) {
+       numEntries += tableValue.getNumEntries();
+     }
+ 
+     this.numEntriesInMemory = getTabletMemory().getNumEntries();
+     numEntries += getTabletMemory().getNumEntries();
+ 
+     this.numEntries = numEntries;
+   }
+ 
+   public long getNumEntries() {
+     return numEntries;
+   }
+ 
+   public long getNumEntriesInMemory() {
+     return numEntriesInMemory;
+   }
+ 
+   public synchronized boolean isClosing() {
+     return closeState == CloseState.CLOSING;
+   }
+ 
+   public synchronized boolean isClosed() {
+     return closeState == CloseState.CLOSED;
+   }
+ 
+   public synchronized boolean isCloseComplete() {
+     return closeState == CloseState.COMPLETE;
+   }
+ 
+   public boolean majorCompactionRunning() {
+     return majorCompactionState == CompactionState.IN_PROGRESS;
+   }
+ 
+   public boolean isMinorCompactionQueued() {
+     return minorCompactionState == CompactionState.WAITING_TO_START;
+   }
+ 
+   public boolean isMinorCompactionRunning() {
+     return minorCompactionState == CompactionState.IN_PROGRESS; 
+   }
+ 
+   public boolean isMajorCompactionQueued() {
+     return majorCompactionQueued.size() > 0;
+   }
+ 
+   public TreeMap<KeyExtent,SplitInfo> split(byte[] sp) throws IOException {
+ 
+     if (sp != null && extent.getEndRow() != null && extent.getEndRow().equals(new Text(sp))) {
+       throw new IllegalArgumentException();
+     }
+ 
+     if (extent.isRootTablet()) {
+       String msg = "Cannot split root tablet";
+       log.warn(msg);
+       throw new RuntimeException(msg);
+     }
+ 
+     try {
+       initiateClose(true, false, false);
+     } catch (IllegalStateException ise) {
+       log.debug("File " + extent + " not splitting : " + ise.getMessage());
+       return null;
+     }
+ 
+     // obtain this info outside of synch block since it will involve opening
+     // the map files... it is ok if the set of map files changes, because
+     // this info is used for optimization... it is ok if map files are missing
+     // from the set... can still query and insert into the tablet while this
+     // map file operation is happening
+     Map<FileRef,FileUtil.FileInfo> firstAndLastRows = FileUtil.tryToGetFirstAndLastRows(getTabletServer().getFileSystem(), getTabletServer().getSystemConfiguration(), getDatafileManager().getFiles());
+ 
+     synchronized (this) {
+       // java needs tuples ...
+       TreeMap<KeyExtent,SplitInfo> newTablets = new TreeMap<KeyExtent,SplitInfo>();
+ 
+       long t1 = System.currentTimeMillis();
+ 
+       // choose a split point
+       SplitRowSpec splitPoint;
+       if (sp == null)
+         splitPoint = findSplitRow(getDatafileManager().getFiles());
+       else {
+         Text tsp = new Text(sp);
+         splitPoint = new SplitRowSpec(FileUtil.estimatePercentageLTE(getTabletServer().getFileSystem(), getTabletServer().getSystemConfiguration(), extent.getPrevEndRow(), extent.getEndRow(),
+             FileUtil.toPathStrings(getDatafileManager().getFiles()), tsp), tsp);
+       }
+ 
+       if (splitPoint == null || splitPoint.row == null) {
+         log.info("had to abort split because splitRow was null");
+         closeState = CloseState.OPEN;
+         return null;
+       }
+ 
+       closeState = CloseState.CLOSING;
+       completeClose(true, false);
+ 
+       Text midRow = splitPoint.row;
+       double splitRatio = splitPoint.splitRatio;
+ 
+       KeyExtent low = new KeyExtent(extent.getTableId(), midRow, extent.getPrevEndRow());
+       KeyExtent high = new KeyExtent(extent.getTableId(), extent.getEndRow(), midRow);
+ 
+       String lowDirectory = TabletOperations.createTabletDirectory(getTabletServer().getFileSystem(), extent.getTableId().toString(), midRow);
+ 
+       // write new tablet information to MetadataTable
+       SortedMap<FileRef,DataFileValue> lowDatafileSizes = new TreeMap<FileRef,DataFileValue>();
+       SortedMap<FileRef,DataFileValue> highDatafileSizes = new TreeMap<FileRef,DataFileValue>();
+       List<FileRef> highDatafilesToRemove = new ArrayList<FileRef>();
+ 
+       MetadataTableUtil.splitDatafiles(extent.getTableId(), midRow, splitRatio, firstAndLastRows, getDatafileManager().getDatafileSizes(), lowDatafileSizes,
+           highDatafileSizes, highDatafilesToRemove);
+ 
+       log.debug("Files for low split " + low + "  " + lowDatafileSizes.keySet());
+       log.debug("Files for high split " + high + "  " + highDatafileSizes.keySet());
+ 
+       String time = tabletTime.getMetadataValue();
+ 
+       // it is possible that some of the bulk loading flags will be deleted after being read below because the bulk load
+       // finishes.... therefore split could propagate load flags for a finished bulk load... there is a special iterator
+       // on the metadata table to clean up this type of garbage
+       Map<FileRef,Long> bulkLoadedFiles = MetadataTableUtil.getBulkFilesLoaded(SystemCredentials.get(), extent);
+ 
+       MetadataTableUtil.splitTablet(high, extent.getPrevEndRow(), splitRatio, SystemCredentials.get(), getTabletServer().getLock());
+       MasterMetadataUtil.addNewTablet(low, lowDirectory, getTabletServer().getTabletSession(), lowDatafileSizes, bulkLoadedFiles, SystemCredentials.get(), time,
+           lastFlushID, lastCompactID, getTabletServer().getLock());
+       MetadataTableUtil.finishSplit(high, highDatafileSizes, highDatafilesToRemove, SystemCredentials.get(), getTabletServer().getLock());
+ 
+       log.log(TLevel.TABLET_HIST, extent + " split " + low + " " + high);
+ 
+       newTablets.put(high, new SplitInfo(tabletDirectory, highDatafileSizes, time, lastFlushID, lastCompactID, lastLocation));
+       newTablets.put(low, new SplitInfo(lowDirectory, lowDatafileSizes, time, lastFlushID, lastCompactID, lastLocation));
+ 
+       long t2 = System.currentTimeMillis();
+ 
+       log.debug(String.format("offline split time : %6.2f secs", (t2 - t1) / 1000.0));
+ 
+       closeState = CloseState.COMPLETE;
+       return newTablets;
+     }
+   }
+ 
+   public SortedMap<FileRef,DataFileValue> getDatafiles() {
+     return getDatafileManager().getDatafileSizes();
+   }
+ 
+   public double queryRate() {
+     return queryRate.rate();
+   }
+ 
+   public double queryByteRate() {
+     return queryByteRate.rate();
+   }
+ 
+   public double ingestRate() {
+     return ingestRate.rate();
+   }
+ 
+   public double ingestByteRate() {
+     return ingestByteRate.rate();
+   }
+ 
+   public double scanRate() {
+     return scannedRate.rate();
+   }
+ 
+   public long totalQueries() {
+     return this.queryCount;
+   }
+ 
+   public long totalIngest() {
+     return this.ingestCount;
+   }
+ 
+   // synchronized?
+   public void updateRates(long now) {
+     queryRate.update(now, queryCount);
+     queryByteRate.update(now, queryBytes);
+     ingestRate.update(now, ingestCount);
+     ingestByteRate.update(now, ingestBytes);
+     scannedRate.update(now, scannedCount.get());
+   }
+ 
+   public long getSplitCreationTime() {
+     return splitCreationTime;
+   }
+ 
+   public void importMapFiles(long tid, Map<FileRef,MapFileInfo> fileMap, boolean setTime) throws IOException {
+     Map<FileRef,DataFileValue> entries = new HashMap<FileRef,DataFileValue>(fileMap.size());
+ 
+     for (Entry<FileRef,MapFileInfo> entry : fileMap.entrySet()) {
+       entries.put(entry.getKey(), new DataFileValue(entry.getValue().estimatedSize, 0l));
+     }
+ 
+     // Clients timeout and will think that this operation failed.
+     // Don't do it if we spent too long waiting for the lock
+     long now = System.currentTimeMillis();
+     synchronized (this) {
+       if (isClosed()) {
+         throw new IOException("tablet " + extent + " is closed");
+       }
+ 
+       // TODO check seems uneeded now - ACCUMULO-1291
+       long lockWait = System.currentTimeMillis() - now;
+       if (lockWait > getTabletServer().getSystemConfiguration().getTimeInMillis(Property.GENERAL_RPC_TIMEOUT)) {
+         throw new IOException("Timeout waiting " + (lockWait / 1000.) + " seconds to get tablet lock");
+       }
+ 
+       if (writesInProgress < 0)
+         throw new Illega

<TRUNCATED>

[35/35] git commit: ACCUMULO-378 More test stabilization

Posted by el...@apache.org.
ACCUMULO-378 More test stabilization


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

Branch: refs/heads/ACCUMULO-378
Commit: 856f2359004f3b4ff209fc6ff61db76d4ab4c192
Parents: ada6ce4
Author: Josh Elser <el...@apache.org>
Authored: Thu Jun 5 00:35:19 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu Jun 5 00:35:19 2014 -0400

----------------------------------------------------------------------
 .../test/replication/ReplicationTest.java         | 18 +++++++++++++++++-
 .../UnorderedWorkAssignerReplicationIT.java       |  2 +-
 2 files changed, 18 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/856f2359/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
index 04d1d6e..8554950 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
@@ -344,6 +344,22 @@ public class ReplicationTest extends ConfigurableMacIT {
     conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.READ);
 
     Set<String> tableIds = Sets.newHashSet(conn.tableOperations().tableIdMap().get(table1), conn.tableOperations().tableIdMap().get(table2));
+    Set<String> tableIdsForMetadata = Sets.newHashSet(tableIds);
+
+    s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    s.setRange(MetadataSchema.ReplicationSection.getRange());
+    iter = s.iterator();
+
+    Assert.assertTrue("Found no records in metadata table", iter.hasNext());
+    entry = iter.next();
+    Assert.assertTrue("Expected to find element in metadata table", tableIdsForMetadata.remove(entry.getKey().getColumnQualifier().toString()));
+    Assert.assertTrue("Expected to find two elements in metadata table, only found one ", iter.hasNext());
+    entry = iter.next();
+    Assert.assertTrue("Expected to find element in metadata table", tableIdsForMetadata.remove(entry.getKey().getColumnQualifier().toString()));
+    Assert.assertFalse("Expected to only find two elements in metadata table", iter.hasNext());
+
+    // Should be creating these records in replication table from metadata table every second
+    Thread.sleep(5000);
 
     // Verify that we found two replication records: one for table1 and one for table2
     s = ReplicationTable.getScanner(conn, new Authorizations());
@@ -352,7 +368,7 @@ public class ReplicationTest extends ConfigurableMacIT {
     Assert.assertTrue("Found no records in replication table", iter.hasNext());
     entry = iter.next();
     Assert.assertTrue("Expected to find element in replication table", tableIds.remove(entry.getKey().getColumnQualifier().toString()));
-    Assert.assertTrue("Expected to find two elements in replication table, didn't find " + tableIds, iter.hasNext());
+    Assert.assertTrue("Expected to find two elements in replication table, only found one ", iter.hasNext());
     entry = iter.next();
     Assert.assertTrue("Expected to find element in replication table", tableIds.remove(entry.getKey().getColumnQualifier().toString()));
     Assert.assertFalse("Expected to only find two elements in replication table", iter.hasNext());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/856f2359/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
index d561d2f..b90d7f7 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
@@ -648,7 +648,7 @@ public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacIT {
       }
 
       log.info("Found {} records in {}", countTable, peerTable2);
-      Assert.assertTrue(countTable > 0);
+      Assert.assertTrue("Did not find any records in peer", countTable > 0);
 
     } finally {
       peer1Cluster.stop();


[03/35] git commit: ACCUMULO-2635 Replace Constants.UTF8 in merged ZooKeeperInstanceTest

Posted by el...@apache.org.
ACCUMULO-2635 Replace Constants.UTF8 in merged ZooKeeperInstanceTest


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

Branch: refs/heads/ACCUMULO-378
Commit: c9c6897ae43d316465fa600a283a4d6c38675ed6
Parents: 8990e59
Author: Bill Havanki <bh...@cloudera.com>
Authored: Mon Jun 2 10:41:05 2014 -0400
Committer: Bill Havanki <bh...@cloudera.com>
Committed: Mon Jun 2 10:41:05 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/core/client/ZooKeeperInstanceTest.java  | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c9c6897a/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java b/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
index 8d86d5a..b215f4c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/ZooKeeperInstanceTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.core.client;
 
+import java.nio.charset.StandardCharsets;
 import java.util.List;
 import java.util.UUID;
 import org.apache.accumulo.core.Constants;
@@ -81,7 +82,7 @@ public class ZooKeeperInstanceTest {
 
   @Test
   public void testGetInstanceID_FromCache() {
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(StandardCharsets.UTF_8));
     expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn("yup".getBytes());
     replay(zc);
     assertEquals(IID_STRING, zki.getInstanceID());
@@ -107,7 +108,7 @@ public class ZooKeeperInstanceTest {
 
   @Test(expected = RuntimeException.class)
   public void testGetInstanceID_IDMissingForName() {
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/instance")).andReturn(IID_STRING.getBytes(StandardCharsets.UTF_8));
     expect(zc.get(Constants.ZROOT + "/" + IID_STRING)).andReturn(null);
     replay(zc);
     zki.getInstanceID();
@@ -135,8 +136,8 @@ public class ZooKeeperInstanceTest {
     children.add("child1");
     children.add("child2");
     expect(zc.getChildren(Constants.ZROOT + Constants.ZINSTANCES)).andReturn(children);
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child1")).andReturn(UUID.randomUUID().toString().getBytes(Constants.UTF8));
-    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child2")).andReturn(IID_STRING.getBytes(Constants.UTF8));
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child1")).andReturn(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8));
+    expect(zc.get(Constants.ZROOT + Constants.ZINSTANCES + "/child2")).andReturn(IID_STRING.getBytes(StandardCharsets.UTF_8));
     replay(zc);
     assertEquals("child2", zki.getInstanceName());
   }


[09/35] git commit: ACCUMULO-2041 updates from initial review

Posted by el...@apache.org.
ACCUMULO-2041 updates from initial review


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

Branch: refs/heads/ACCUMULO-378
Commit: 8049859154dc5cab5a5a0ce1d6babaf243c06922
Parents: 7db2abf
Author: Eric C. Newton <er...@gmail.com>
Authored: Mon Apr 21 15:27:15 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Jun 3 10:49:43 2014 -0400

----------------------------------------------------------------------
 .../accumulo/tserver/CountingIterator.java      | 78 ++++++++++++++++++++
 .../apache/accumulo/tserver/TabletServer.java   |  4 +-
 .../apache/accumulo/tserver/tablet/Batch.java   | 26 +++++--
 .../tserver/tablet/CompactionRunner.java        |  2 +-
 .../tserver/tablet/CompactionWatcher.java       |  1 +
 .../accumulo/tserver/tablet/Compactor.java      | 54 +-------------
 .../tserver/tablet/MinorCompactionTask.java     |  9 ++-
 .../accumulo/tserver/tablet/ScanBatch.java      | 14 +++-
 .../accumulo/tserver/tablet/ScanDataSource.java | 10 +--
 .../accumulo/tserver/tablet/ScanOptions.java    | 49 +++++++++---
 .../apache/accumulo/tserver/tablet/Scanner.java | 27 +++----
 .../accumulo/tserver/tablet/SplitInfo.java      | 38 ++++++++--
 .../apache/accumulo/tserver/tablet/Tablet.java  |  4 +-
 .../accumulo/tserver/CountingIteratorTest.java  |  1 -
 14 files changed, 213 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/CountingIterator.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/CountingIterator.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/CountingIterator.java
new file mode 100644
index 0000000..e4ba076
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/CountingIterator.java
@@ -0,0 +1,78 @@
+/*
+ * 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.tserver;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+
+public class CountingIterator extends WrappingIterator {
+
+  private long count;
+  private final ArrayList<CountingIterator> deepCopies;
+  private final AtomicLong entriesRead;
+
+  @Override
+  public CountingIterator deepCopy(IteratorEnvironment env) {
+    return new CountingIterator(this, env);
+  }
+
+  private CountingIterator(CountingIterator other, IteratorEnvironment env) {
+    setSource(other.getSource().deepCopy(env));
+    count = 0;
+    this.deepCopies = other.deepCopies;
+    this.entriesRead = other.entriesRead;
+    deepCopies.add(this);
+  }
+
+  public CountingIterator(SortedKeyValueIterator<Key,Value> source, AtomicLong entriesRead) {
+    deepCopies = new ArrayList<CountingIterator>();
+    this.setSource(source);
+    count = 0;
+    this.entriesRead = entriesRead;
+  }
+
+  @Override
+  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void next() throws IOException {
+    super.next();
+    count++;
+    if (count % 1024 == 0) {
+      entriesRead.addAndGet(1024);
+    }
+  }
+
+  public long getCount() {
+    long sum = 0;
+    for (CountingIterator dc : deepCopies) {
+      sum += dc.count;
+    }
+
+    return count + sum;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/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 1c07c44..2a453a8 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
@@ -1303,7 +1303,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         throw new RuntimeException(t);
       }
 
-      ScanResult scanResult = new ScanResult(Key.compress(bresult.results), bresult.more);
+      ScanResult scanResult = new ScanResult(Key.compress(bresult.getResults()), bresult.isMore());
 
       scanSession.entriesReturned += scanResult.results.size();
 
@@ -1859,7 +1859,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
           Value val = null;
 
-          for (KVEntry entry2 : batch.results) {
+          for (KVEntry entry2 : batch.getResults()) {
             val = entry2.getValue();
             break;
           }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Batch.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Batch.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Batch.java
index 73434c6..1a83ba4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Batch.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Batch.java
@@ -20,11 +20,11 @@ import java.util.List;
 
 import org.apache.accumulo.core.data.Key;
 
-class Batch {
-  final boolean skipContinueKey;
-  final List<KVEntry> results;
-  final Key continueKey;
-  final long numBytes;
+final class Batch {
+  private final boolean skipContinueKey;
+  private final List<KVEntry> results;
+  private final Key continueKey;
+  private final long numBytes;
   
   Batch(boolean skipContinueKey, List<KVEntry> results, Key continueKey, long numBytes) {
     this.skipContinueKey = skipContinueKey;
@@ -32,4 +32,20 @@ class Batch {
     this.continueKey = continueKey;
     this.numBytes = numBytes;
   }
+
+  public boolean isSkipContinueKey() {
+    return skipContinueKey;
+  }
+
+  public List<KVEntry> getResults() {
+    return results;
+  }
+
+  public Key getContinueKey() {
+    return continueKey;
+  }
+
+  public long getNumBytes() {
+    return numBytes;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionRunner.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionRunner.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionRunner.java
index de5a66d..1dee64b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionRunner.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionRunner.java
@@ -18,7 +18,7 @@ package org.apache.accumulo.tserver.tablet;
 
 import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
 
-class CompactionRunner implements Runnable, Comparable<CompactionRunner> {
+final class CompactionRunner implements Runnable, Comparable<CompactionRunner> {
 
   private final Tablet tablet;
   private final MajorCompactionReason reason;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionWatcher.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionWatcher.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionWatcher.java
index 1ca1f33..adc01b2 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionWatcher.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionWatcher.java
@@ -51,6 +51,7 @@ public class CompactionWatcher implements Runnable {
     this.config = config;
   }
 
+  @Override
   public void run() {
     List<CompactionInfo> runningCompactions = Compactor.getRunningCompactions();
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
index 2a3e2f4..9a93be3 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
@@ -40,11 +40,9 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVIterator;
 import org.apache.accumulo.core.file.FileSKVWriter;
-import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
-import org.apache.accumulo.core.iterators.WrappingIterator;
 import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
 import org.apache.accumulo.core.iterators.system.DeletingIterator;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
@@ -60,6 +58,7 @@ import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.problems.ProblemType;
 import org.apache.accumulo.trace.instrument.Span;
 import org.apache.accumulo.trace.instrument.Trace;
+import org.apache.accumulo.tserver.CountingIterator;
 import org.apache.accumulo.tserver.InMemoryMap;
 import org.apache.accumulo.tserver.MinorCompactionReason;
 import org.apache.accumulo.tserver.TabletIteratorEnvironment;
@@ -71,57 +70,6 @@ public class Compactor implements Callable<CompactionStats> {
   private static final Logger log = Logger.getLogger(Compactor.class);
   private static final AtomicLong nextCompactorID = new AtomicLong(0);
 
-  public static class CountingIterator extends WrappingIterator {
-
-    private long count;
-    private final ArrayList<CountingIterator> deepCopies;
-    private final AtomicLong entriesRead;
-
-    @Override
-    public CountingIterator deepCopy(IteratorEnvironment env) {
-      return new CountingIterator(this, env);
-    }
-
-    private CountingIterator(CountingIterator other, IteratorEnvironment env) {
-      setSource(other.getSource().deepCopy(env));
-      count = 0;
-      this.deepCopies = other.deepCopies;
-      this.entriesRead = other.entriesRead;
-      deepCopies.add(this);
-    }
-
-    public CountingIterator(SortedKeyValueIterator<Key,Value> source, AtomicLong entriesRead) {
-      deepCopies = new ArrayList<Compactor.CountingIterator>();
-      this.setSource(source);
-      count = 0;
-      this.entriesRead = entriesRead;
-    }
-
-    @Override
-    public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void next() throws IOException {
-      super.next();
-      count++;
-      if (count % 1024 == 0) {
-        entriesRead.addAndGet(1024);
-      }
-    }
-
-    public long getCount() {
-      long sum = 0;
-      for (CountingIterator dc : deepCopies) {
-        sum += dc.count;
-      }
-
-      return count + sum;
-    }
-  }
-
-
   public static class CompactionCanceledException extends Exception {
     private static final long serialVersionUID = 1L;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
index 9278cb2..6183824 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactionTask.java
@@ -24,9 +24,12 @@ import org.apache.accumulo.trace.instrument.Span;
 import org.apache.accumulo.trace.instrument.Trace;
 import org.apache.accumulo.tserver.MinorCompactionReason;
 import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
+import org.apache.log4j.Logger;
 
 class MinorCompactionTask implements Runnable {
 
+  private static Logger log = Logger.getLogger(MinorCompactionTask.class);
+
   private final Tablet tablet;
   private long queued;
   private CommitSession commitSession;
@@ -47,7 +50,7 @@ class MinorCompactionTask implements Runnable {
 
   @Override
   public void run() {
-    tablet.isMinorCompactionRunning();
+    tablet.minorCompactionStarted();
     Span minorCompaction = Trace.on("minorCompaction");
     try {
       FileRef newMapfileLocation = tablet.getNextMapFilename(mergeFile == null ? "F" : "M");
@@ -68,7 +71,7 @@ class MinorCompactionTask implements Runnable {
           tablet.getTabletServer().minorCompactionStarted(commitSession, commitSession.getWALogSeq() + 1, newMapfileLocation.path().toString());
           break;
         } catch (IOException e) {
-          Tablet.log.warn("Failed to write to write ahead log " + e.getMessage(), e);
+          log.warn("Failed to write to write ahead log " + e.getMessage(), e);
         }
       }
       span.stop();
@@ -83,7 +86,7 @@ class MinorCompactionTask implements Runnable {
         tablet.initiateMajorCompaction(MajorCompactionReason.NORMAL);
       }
     } catch (Throwable t) {
-      Tablet.log.error("Unknown error during minor compaction for extent: " + tablet.getExtent(), t);
+      log.error("Unknown error during minor compaction for extent: " + tablet.getExtent(), t);
       throw new RuntimeException(t);
     } finally {
       tablet.minorCompactionComplete();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanBatch.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanBatch.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanBatch.java
index 0ea76d3..dc932c6 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanBatch.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanBatch.java
@@ -18,12 +18,20 @@ package org.apache.accumulo.tserver.tablet;
 
 import java.util.List;
 
-public class ScanBatch {
-  public final boolean more;
-  public final List<KVEntry> results;
+final public class ScanBatch {
+  private final boolean more;
+  private final List<KVEntry> results;
 
   ScanBatch(List<KVEntry> results, boolean more) {
     this.results = results;
     this.more = more;
   }
+
+  public boolean isMore() {
+    return more;
+  }
+
+  public List<KVEntry> getResults() {
+    return results;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
index 980a082..5464731 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
@@ -76,7 +76,7 @@ class ScanDataSource implements DataSource {
     this.tablet = tablet;
     expectedDeletionCount = tablet.getDataSourceDeletions();
     this.options = options;
-    this.interruptFlag = options.interruptFlag;
+    this.interruptFlag = options.getInterruptFlag();
   }
 
   @Override
@@ -147,7 +147,7 @@ class ScanDataSource implements DataSource {
       files = reservation.getSecond();
     }
 
-    Collection<InterruptibleIterator> mapfiles = fileManager.openFiles(files, options.isolated);
+    Collection<InterruptibleIterator> mapfiles = fileManager.openFiles(files, options.isIsolated());
 
     List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<SortedKeyValueIterator<Key,Value>>(mapfiles.size() + memIters.size());
 
@@ -167,12 +167,12 @@ class ScanDataSource implements DataSource {
 
     ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
 
-    ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, options.columnSet);
+    ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, options.getColumnSet());
 
-    VisibilityFilter visFilter = new VisibilityFilter(colFilter, options.authorizations, options.defaultLabels);
+    VisibilityFilter visFilter = new VisibilityFilter(colFilter, options.getAuthorizations(), options.getDefaultLabels());
 
     return iterEnv.getTopLevelIterator(IteratorUtil
-        .loadIterators(IteratorScope.scan, visFilter, tablet.getExtent(), tablet.getTableConfiguration(), options.ssiList, options.ssio, iterEnv));
+        .loadIterators(IteratorScope.scan, visFilter, tablet.getExtent(), tablet.getTableConfiguration(), options.getSsiList(), options.getSsio(), iterEnv));
   }
 
   void close(boolean sawErrors) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java
index 9382ea7..07aa8e7 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanOptions.java
@@ -25,16 +25,16 @@ import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.thrift.IterInfo;
 import org.apache.accumulo.core.security.Authorizations;
 
-class ScanOptions {
+final class ScanOptions {
 
-  final Authorizations authorizations;
-  final byte[] defaultLabels;
-  final Set<Column> columnSet;
-  final List<IterInfo> ssiList;
-  final Map<String,Map<String,String>> ssio;
-  final AtomicBoolean interruptFlag;
-  final int num;
-  final boolean isolated;
+  private final Authorizations authorizations;
+  private final byte[] defaultLabels;
+  private final Set<Column> columnSet;
+  private final List<IterInfo> ssiList;
+  private final Map<String,Map<String,String>> ssio;
+  private final AtomicBoolean interruptFlag;
+  private final int num;
+  private final boolean isolated;
 
   ScanOptions(int num, Authorizations authorizations, byte[] defaultLabels, Set<Column> columnSet, List<IterInfo> ssiList,
       Map<String,Map<String,String>> ssio, AtomicBoolean interruptFlag, boolean isolated) {
@@ -48,4 +48,35 @@ class ScanOptions {
     this.isolated = isolated;
   }
 
+  public Authorizations getAuthorizations() {
+    return authorizations;
+  }
+
+  public byte[] getDefaultLabels() {
+    return defaultLabels;
+  }
+
+  public Set<Column> getColumnSet() {
+    return columnSet;
+  }
+
+  public List<IterInfo> getSsiList() {
+    return ssiList;
+  }
+
+  public Map<String,Map<String,String>> getSsio() {
+    return ssio;
+  }
+
+  public AtomicBoolean getInterruptFlag() {
+    return interruptFlag;
+  }
+
+  public int getNum() {
+    return num;
+  }
+
+  public boolean isIsolated() {
+    return isolated;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
index 96379fc..ad3fcb2 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java
@@ -56,7 +56,7 @@ public class Scanner {
 
     ScanDataSource dataSource;
 
-    if (options.isolated) {
+    if (options.isIsolated()) {
       if (isolatedDataSource == null)
         isolatedDataSource = new ScanDataSource(tablet, options);
       dataSource = isolatedDataSource;
@@ -68,7 +68,7 @@ public class Scanner {
 
       SortedKeyValueIterator<Key,Value> iter;
 
-      if (options.isolated) {
+      if (options.isIsolated()) {
         if (isolatedIter == null)
           isolatedIter = new SourceSwitchingIterator(dataSource, true);
         else
@@ -78,16 +78,16 @@ public class Scanner {
         iter = new SourceSwitchingIterator(dataSource, false);
       }
 
-      results = tablet.nextBatch(iter, range, options.num, options.columnSet);
+      results = tablet.nextBatch(iter, range, options.getNum(), options.getColumnSet());
 
-      if (results.results == null) {
+      if (results.getResults() == null) {
         range = null;
         return new ScanBatch(new ArrayList<KVEntry>(), false);
-      } else if (results.continueKey == null) {
-        return new ScanBatch(results.results, false);
+      } else if (results.getContinueKey() == null) {
+        return new ScanBatch(results.getResults(), false);
       } else {
-        range = new Range(results.continueKey, !results.skipContinueKey, range.getEndKey(), range.isEndKeyInclusive());
-        return new ScanBatch(results.results, true);
+        range = new Range(results.getContinueKey(), !results.isSkipContinueKey(), range.getEndKey(), range.isEndKeyInclusive());
+        return new ScanBatch(results.getResults(), true);
       }
 
     } catch (IterationInterruptedException iie) {
@@ -111,13 +111,14 @@ public class Scanner {
     } finally {
       // code in finally block because always want
       // to return mapfiles, even when exception is thrown
-      if (!options.isolated)
+      if (!options.isIsolated()) {
         dataSource.close(false);
-      else 
+      } else { 
         dataSource.detachFileManager();
+      }
       
-      if (results != null && results.results != null)
-        tablet.updateQueryStats(results.results.size(), results.numBytes);
+      if (results != null && results.getResults() != null)
+        tablet.updateQueryStats(results.getResults().size(), results.getNumBytes());
     }
   }
 
@@ -125,7 +126,7 @@ public class Scanner {
   // this could lead to the case where file iterators that are in use by a thread are returned
   // to the pool... this would be bad
   public void close() {
-    options.interruptFlag.set(true);
+    options.getInterruptFlag().set(true);
     synchronized (this) {
       scanClosed = true;
       if (isolatedDataSource != null)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java
index 084503a..ec84aa8 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java
@@ -32,13 +32,13 @@ import org.apache.accumulo.server.master.state.TServerInstance;
  * 
  */
 
-public class SplitInfo {
-  final String dir;
-  final SortedMap<FileRef,DataFileValue> datafiles;
-  final String time;
-  final long initFlushID;
-  final long initCompactID;
-  final TServerInstance lastLocation;
+final public class SplitInfo {
+  private final String dir;
+  private final SortedMap<FileRef,DataFileValue> datafiles;
+  private final String time;
+  private final long initFlushID;
+  private final long initCompactID;
+  private final TServerInstance lastLocation;
 
   SplitInfo(String d, SortedMap<FileRef,DataFileValue> dfv, String time, long initFlushID, long initCompactID, TServerInstance lastLocation) {
     this.dir = d;
@@ -49,4 +49,28 @@ public class SplitInfo {
     this.lastLocation = lastLocation;
   }
 
+  public String getDir() {
+    return dir;
+  }
+
+  public SortedMap<FileRef,DataFileValue> getDatafiles() {
+    return datafiles;
+  }
+
+  public String getTime() {
+    return time;
+  }
+
+  public long getInitFlushID() {
+    return initFlushID;
+  }
+
+  public long getInitCompactID() {
+    return initCompactID;
+  }
+
+  public TServerInstance getLastLocation() {
+    return lastLocation;
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 432e3a3..bf9a905 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -255,7 +255,7 @@ public class Tablet implements TabletCommitter {
   }
 
   public Tablet(TabletServer tabletServer, KeyExtent extent, TabletResourceManager trm, SplitInfo info) throws IOException {
-    this(tabletServer, new Text(info.dir), extent, trm, info.datafiles, info.time, info.initFlushID, info.initCompactID, info.lastLocation);
+    this(tabletServer, new Text(info.getDir()), extent, trm, info.getDatafiles(), info.getTime(), info.getInitFlushID(), info.getInitCompactID(), info.getLastLocation());
     splitCreationTime = System.currentTimeMillis();
   }
 
@@ -2539,7 +2539,7 @@ public class Tablet implements TabletCommitter {
     minorCompactionWaitingToStart = true;
   }
 
-  public void minorCompacationStarted() {
+  public void minorCompactionStarted() {
     minorCompactionWaitingToStart = false;
     minorCompactionInProgress = true;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/80498591/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
index 253c97e..302b025 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
@@ -26,7 +26,6 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
-import org.apache.accumulo.tserver.tablet.Compactor.CountingIterator;
 import org.junit.Assert;
 import org.junit.Test;
 


[32/35] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Posted by el...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Conflicts:
	server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java


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

Branch: refs/heads/ACCUMULO-378
Commit: 05cf918496adfd3ef1c7e9564dd879273019902e
Parents: 66f3128 6dfcf8f
Author: Josh Elser <el...@apache.org>
Authored: Wed Jun 4 16:40:18 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Jun 4 16:40:18 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/tserver/log/DfsLogger.java  | 20 +++++++-------------
 1 file changed, 7 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/05cf9184/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index 5dabedc,0000000..cca2953
mode 100644,000000..100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@@ -1,555 -1,0 +1,549 @@@
 +/*
 + * 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.tserver.log;
 +
 +import static org.apache.accumulo.tserver.logger.LogEvents.COMPACTION_FINISH;
 +import static org.apache.accumulo.tserver.logger.LogEvents.COMPACTION_START;
 +import static org.apache.accumulo.tserver.logger.LogEvents.DEFINE_TABLET;
 +import static org.apache.accumulo.tserver.logger.LogEvents.MANY_MUTATIONS;
 +import static org.apache.accumulo.tserver.logger.LogEvents.OPEN;
 +
 +import java.io.DataInputStream;
 +import java.io.DataOutputStream;
 +import java.io.IOException;
 +import java.io.OutputStream;
 +import java.lang.reflect.InvocationTargetException;
 +import java.lang.reflect.Method;
 +import java.nio.channels.ClosedChannelException;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Set;
 +import java.util.UUID;
 +import java.util.concurrent.CountDownLatch;
 +import java.util.concurrent.LinkedBlockingQueue;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.security.crypto.CryptoModule;
 +import org.apache.accumulo.core.security.crypto.CryptoModuleFactory;
 +import org.apache.accumulo.core.security.crypto.CryptoModuleParameters;
 +import org.apache.accumulo.core.security.crypto.DefaultCryptoModule;
 +import org.apache.accumulo.core.security.crypto.NoFlushOutputStream;
 +import org.apache.accumulo.core.util.Daemon;
 +import org.apache.accumulo.core.util.LoggingRunnable;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.StringUtil;
 +import org.apache.accumulo.server.ServerConstants;
 +import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.tserver.TabletMutations;
 +import org.apache.accumulo.tserver.logger.LogFileKey;
 +import org.apache.accumulo.tserver.logger.LogFileValue;
 +import org.apache.hadoop.fs.FSDataInputStream;
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.log4j.Logger;
 +
 +/**
 + * Wrap a connection to a logger.
 + * 
 + */
 +public class DfsLogger {
 +  // Package private so that LogSorter can find this
 +  static final String LOG_FILE_HEADER_V2 = "--- Log File Header (v2) ---";
 +  static final String LOG_FILE_HEADER_V3 = "--- Log File Header (v3) ---";
 +
 +  private static Logger log = Logger.getLogger(DfsLogger.class);
 +
 +  public static class LogClosedException extends IOException {
 +    private static final long serialVersionUID = 1L;
 +
 +    public LogClosedException() {
 +      super("LogClosed");
 +    }
 +  }
 +
 +  public static class DFSLoggerInputStreams {
 +
 +    private FSDataInputStream originalInput;
 +    private DataInputStream decryptingInputStream;
 +
 +    public DFSLoggerInputStreams(FSDataInputStream originalInput, DataInputStream decryptingInputStream) {
 +      this.originalInput = originalInput;
 +      this.decryptingInputStream = decryptingInputStream;
 +    }
 +
 +    public FSDataInputStream getOriginalInput() {
 +      return originalInput;
 +    }
 +
 +    public void setOriginalInput(FSDataInputStream originalInput) {
 +      this.originalInput = originalInput;
 +    }
 +
 +    public DataInputStream getDecryptingInputStream() {
 +      return decryptingInputStream;
 +    }
 +
 +    public void setDecryptingInputStream(DataInputStream decryptingInputStream) {
 +      this.decryptingInputStream = decryptingInputStream;
 +    }
 +  }
 +
 +  public interface ServerResources {
 +    AccumuloConfiguration getConfiguration();
 +
 +    VolumeManager getFileSystem();
 +
 +    Set<TServerInstance> getCurrentTServers();
 +  }
 +
 +  private final LinkedBlockingQueue<DfsLogger.LogWork> workQueue = new LinkedBlockingQueue<DfsLogger.LogWork>();
 +
 +  private final Object closeLock = new Object();
 +
 +  private static final DfsLogger.LogWork CLOSED_MARKER = new DfsLogger.LogWork(null);
 +
 +  private static final LogFileValue EMPTY = new LogFileValue();
 +
 +  private boolean closed = false;
 +
 +  private class LogSyncingTask implements Runnable {
 +
 +    @Override
 +    public void run() {
 +      ArrayList<DfsLogger.LogWork> work = new ArrayList<DfsLogger.LogWork>();
 +      boolean sawClosedMarker = false;
 +      while (!sawClosedMarker) {
 +        work.clear();
 +
 +        try {
 +          work.add(workQueue.take());
 +        } catch (InterruptedException ex) {
 +          continue;
 +        }
 +        workQueue.drainTo(work);
 +
 +        try {
 +          sync.invoke(logFile);
 +        } catch (Exception ex) {
 +          log.warn("Exception syncing " + ex);
 +          for (DfsLogger.LogWork logWork : work) {
 +            logWork.exception = ex;
 +          }
 +        }
 +
 +        for (DfsLogger.LogWork logWork : work)
 +          if (logWork == CLOSED_MARKER)
 +            sawClosedMarker = true;
 +          else
 +            logWork.latch.countDown();
 +      }
 +    }
 +  }
 +
 +  static class LogWork {
 +    CountDownLatch latch;
 +    volatile Exception exception;
 +
 +    public LogWork(CountDownLatch latch) {
 +      this.latch = latch;
 +    }
 +  }
 +
 +  public static class LoggerOperation {
 +    private final LogWork work;
 +
 +    public LoggerOperation(LogWork work) {
 +      this.work = work;
 +    }
 +
 +    public void await() throws IOException {
 +      try {
 +        work.latch.await();
 +      } catch (InterruptedException e) {
 +        throw new RuntimeException(e);
 +      }
 +
 +      if (work.exception != null) {
 +        if (work.exception instanceof IOException)
 +          throw (IOException) work.exception;
 +        else if (work.exception instanceof RuntimeException)
 +          throw (RuntimeException) work.exception;
 +        else
 +          throw new RuntimeException(work.exception);
 +      }
 +    }
 +  }
 +
 +  @Override
 +  public boolean equals(Object obj) {
 +    // filename is unique
 +    if (obj == null)
 +      return false;
 +    if (obj instanceof DfsLogger)
 +      return getFileName().equals(((DfsLogger) obj).getFileName());
 +    return false;
 +  }
 +
 +  @Override
 +  public int hashCode() {
 +    // filename is unique
 +    return getFileName().hashCode();
 +  }
 +
 +  private final ServerResources conf;
 +  private FSDataOutputStream logFile;
 +  private DataOutputStream encryptingLogFile = null;
 +  private Method sync;
 +  private String logPath;
 +  private Daemon syncThread;
 +
 +  public DfsLogger(ServerResources conf) throws IOException {
 +    this.conf = conf;
 +  }
 +
 +  public DfsLogger(ServerResources conf, String filename) throws IOException {
 +    this.conf = conf;
 +    this.logPath = filename;
 +  }
 +
 +  public static DFSLoggerInputStreams readHeaderAndReturnStream(VolumeManager fs, Path path, AccumuloConfiguration conf) throws IOException {
 +    FSDataInputStream input = fs.open(path);
 +    DataInputStream decryptingInput = null;
 +
 +    byte[] magic = DfsLogger.LOG_FILE_HEADER_V3.getBytes();
 +    byte[] magicBuffer = new byte[magic.length];
 +    input.readFully(magicBuffer);
 +    if (Arrays.equals(magicBuffer, magic)) {
 +      // additional parameters it needs from the underlying stream.
 +      String cryptoModuleClassname = input.readUTF();
 +      CryptoModule cryptoModule = CryptoModuleFactory.getCryptoModule(cryptoModuleClassname);
 +
 +      // Create the parameters and set the input stream into those parameters
 +      CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
 +      params.setEncryptedInputStream(input);
 +
 +      // Create the plaintext input stream from the encrypted one
 +      params = cryptoModule.getDecryptingInputStream(params);
 +
 +      if (params.getPlaintextInputStream() instanceof DataInputStream) {
 +        decryptingInput = (DataInputStream) params.getPlaintextInputStream();
 +      } else {
 +        decryptingInput = new DataInputStream(params.getPlaintextInputStream());
 +      }
 +    } else {
 +      input.seek(0);
 +      byte[] magicV2 = DfsLogger.LOG_FILE_HEADER_V2.getBytes();
 +      byte[] magicBufferV2 = new byte[magicV2.length];
 +      input.readFully(magicBufferV2);
 +
 +      if (Arrays.equals(magicBufferV2, magicV2)) {
 +        // Log files from 1.5 dump their options in raw to the logger files. Since we don't know the class
 +        // that needs to read those files, we can make a couple of basic assumptions. Either it's going to be
 +        // the NullCryptoModule (no crypto) or the DefaultCryptoModule.
 +
 +        // If it's null, we won't have any parameters whatsoever. First, let's attempt to read
 +        // parameters
 +        Map<String,String> opts = new HashMap<String,String>();
 +        int count = input.readInt();
 +        for (int i = 0; i < count; i++) {
 +          String key = input.readUTF();
 +          String value = input.readUTF();
 +          opts.put(key, value);
 +        }
 +
 +        if (opts.size() == 0) {
 +          // NullCryptoModule, we're done
 +          decryptingInput = input;
 +        } else {
 +
 +          // The DefaultCryptoModule will want to read the parameters from the underlying file, so we will put the file back to that spot.
 +          org.apache.accumulo.core.security.crypto.CryptoModule cryptoModule = org.apache.accumulo.core.security.crypto.CryptoModuleFactory
 +              .getCryptoModule(DefaultCryptoModule.class.getName());
 +
 +          CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
 +
 +          input.seek(0);
 +          input.readFully(magicBufferV2);
 +          params.setEncryptedInputStream(input);
 +
 +          params = cryptoModule.getDecryptingInputStream(params);
 +          if (params.getPlaintextInputStream() instanceof DataInputStream) {
 +            decryptingInput = (DataInputStream) params.getPlaintextInputStream();
 +          } else {
 +            decryptingInput = new DataInputStream(params.getPlaintextInputStream());
 +          }
 +        }
 +
 +      } else {
 +
 +        input.seek(0);
 +        decryptingInput = input;
 +      }
 +
 +    }
 +    return new DFSLoggerInputStreams(input, decryptingInput);
 +  }
 +
 +  public synchronized void open(String address) throws IOException {
 +    String filename = UUID.randomUUID().toString();
 +    String logger = StringUtil.join(Arrays.asList(address.split(":")), "+");
 +
 +    log.debug("DfsLogger.open() begin");
 +    VolumeManager fs = conf.getFileSystem();
 +
 +    logPath = fs.choose(ServerConstants.getWalDirs()) + "/" + logger + "/" + filename;
 +    try {
 +      short replication = (short) conf.getConfiguration().getCount(Property.TSERV_WAL_REPLICATION);
 +      if (replication == 0)
 +        replication = fs.getDefaultReplication(new Path(logPath));
 +      long blockSize = conf.getConfiguration().getMemoryInBytes(Property.TSERV_WAL_BLOCKSIZE);
 +      if (blockSize == 0)
 +        blockSize = (long) (conf.getConfiguration().getMemoryInBytes(Property.TSERV_WALOG_MAX_SIZE) * 1.1);
 +      if (conf.getConfiguration().getBoolean(Property.TSERV_WAL_SYNC))
 +        logFile = fs.createSyncable(new Path(logPath), 0, replication, blockSize);
 +      else
 +        logFile = fs.create(new Path(logPath), true, 0, replication, blockSize);
 +
 +      String syncMethod = conf.getConfiguration().get(Property.TSERV_WAL_SYNC_METHOD);
 +      try {
-         NoSuchMethodException e = null;
-         try {
-           // hsync: send data to datanodes and sync the data to disk
-           sync = logFile.getClass().getMethod(syncMethod);
-         } catch (NoSuchMethodException ex) {
-           log.warn("Could not find configured " + syncMethod + " method, trying to fall back to old Hadoop sync method", ex);
-           e = ex;
-         }
++        // hsync: send data to datanodes and sync the data to disk
++        sync = logFile.getClass().getMethod(syncMethod);
++      } catch (Exception ex) {
++        log.warn("Could not find configured " + syncMethod + " method, trying to fall back to old Hadoop sync method", ex);
++
 +        try {
 +          // sync: send data to datanodes
 +          sync = logFile.getClass().getMethod("sync");
-           e = null;
-         } catch (NoSuchMethodException ex) {}
-         if (e != null)
++        } catch (Exception e) {
 +          throw new RuntimeException(e);
-       } catch (Exception e) {
-         throw new RuntimeException(e);
++        }
 +      }
 +
 +      // Initialize the crypto operations.
 +      org.apache.accumulo.core.security.crypto.CryptoModule cryptoModule = org.apache.accumulo.core.security.crypto.CryptoModuleFactory.getCryptoModule(conf
 +          .getConfiguration().get(Property.CRYPTO_MODULE_CLASS));
 +
 +      // Initialize the log file with a header and the crypto params used to set up this log file.
 +      logFile.write(LOG_FILE_HEADER_V3.getBytes(Constants.UTF8));
 +
 +      CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf.getConfiguration());
 +
 +      NoFlushOutputStream nfos = new NoFlushOutputStream(logFile);
 +      params.setPlaintextOutputStream(nfos);
 +
 +      // In order to bootstrap the reading of this file later, we have to record the CryptoModule that was used to encipher it here,
 +      // so that that crypto module can re-read its own parameters.
 +
 +      logFile.writeUTF(conf.getConfiguration().get(Property.CRYPTO_MODULE_CLASS));
 +
 +      params = cryptoModule.getEncryptingOutputStream(params);
 +      OutputStream encipheringOutputStream = params.getEncryptedOutputStream();
 +
 +      // If the module just kicks back our original stream, then just use it, don't wrap it in
 +      // another data OutputStream.
 +      if (encipheringOutputStream == nfos) {
 +        log.debug("No enciphering, using raw output stream");
 +        encryptingLogFile = nfos;
 +      } else {
 +        log.debug("Enciphering found, wrapping in DataOutputStream");
 +        encryptingLogFile = new DataOutputStream(encipheringOutputStream);
 +      }
 +
 +      LogFileKey key = new LogFileKey();
 +      key.event = OPEN;
 +      key.tserverSession = filename;
 +      key.filename = filename;
 +      write(key, EMPTY);
 +      sync.invoke(logFile);
 +      log.debug("Got new write-ahead log: " + this);
 +    } catch (Exception ex) {
 +      if (logFile != null)
 +        logFile.close();
 +      logFile = null;
 +      encryptingLogFile = null;
 +      throw new IOException(ex);
 +    }
 +
 +    syncThread = new Daemon(new LoggingRunnable(log, new LogSyncingTask()));
 +    syncThread.setName("Accumulo WALog thread " + toString());
 +    syncThread.start();
 +  }
 +
 +  @Override
 +  public String toString() {
 +    String fileName = getFileName();
 +    if (fileName.contains(":"))
 +      return getLogger() + "/" + getFileName();
 +    return fileName;
 +  }
 +
 +  public String getFileName() {
 +    return logPath.toString();
 +  }
 +
 +  public void close() throws IOException {
 +
 +    synchronized (closeLock) {
 +      if (closed)
 +        return;
 +      // after closed is set to true, nothing else should be added to the queue
 +      // CLOSED_MARKER should be the last thing on the queue, therefore when the
 +      // background thread sees the marker and exits there should be nothing else
 +      // to process... so nothing should be left waiting for the background
 +      // thread to do work
 +      closed = true;
 +      workQueue.add(CLOSED_MARKER);
 +    }
 +
 +    // wait for background thread to finish before closing log file
 +    if(syncThread != null){
 +      try {
 +        syncThread.join();
 +      } catch (InterruptedException e) {
 +        throw new RuntimeException(e);
 +      }
 +    }
 +
 +    //expect workq should be empty at this point
 +    if(workQueue.size() != 0){
 +      log.error("WAL work queue not empty after sync thread exited");
 +      throw new IllegalStateException("WAL work queue not empty after sync thread exited");
 +    }
 +    
 +    if (encryptingLogFile != null)
 +      try {
 +        logFile.close();
 +      } catch (IOException ex) {
 +        log.error(ex);
 +        throw new LogClosedException();
 +      }
 +  }
 +
 +  public synchronized void defineTablet(int seq, int tid, KeyExtent tablet) throws IOException {
 +    // write this log to the METADATA table
 +    final LogFileKey key = new LogFileKey();
 +    key.event = DEFINE_TABLET;
 +    key.seq = seq;
 +    key.tid = tid;
 +    key.tablet = tablet;
 +    try {
 +      write(key, EMPTY);
 +      sync.invoke(logFile);
 +    } catch (IllegalArgumentException e) {
 +      log.error("Signature of sync method changed. Accumulo is likely incompatible with this version of Hadoop.");
 +      throw new RuntimeException(e);
 +    } catch (IllegalAccessException e) {
 +      log.error("Could not invoke sync method due to permission error.");
 +      throw new RuntimeException(e);
 +    } catch (InvocationTargetException e) {
 +      Throwable cause = e.getCause();
 +      if (cause instanceof IOException) {
 +        throw (IOException) cause;
 +      } else if (cause instanceof RuntimeException) {
 +        throw (RuntimeException) cause;
 +      } else if (cause instanceof Error) {
 +        throw (Error) cause;
 +      } else {
 +        // Cause is null, or some other checked exception that was added later.
 +        throw new RuntimeException(e);
 +      }
 +    }
 +  }
 +
 +  private synchronized void write(LogFileKey key, LogFileValue value) throws IOException {
 +    key.write(encryptingLogFile);
 +    value.write(encryptingLogFile);
 +    encryptingLogFile.flush();
 +  }
 +
 +  public LoggerOperation log(int seq, int tid, Mutation mutation) throws IOException {
 +    return logManyTablets(Collections.singletonList(new TabletMutations(tid, seq, Collections.singletonList(mutation))));
 +  }
 +
 +  private LoggerOperation logFileData(List<Pair<LogFileKey,LogFileValue>> keys) throws IOException {
 +    DfsLogger.LogWork work = new DfsLogger.LogWork(new CountDownLatch(1));
 +    synchronized (DfsLogger.this) {
 +      try {
 +        for (Pair<LogFileKey,LogFileValue> pair : keys) {
 +          write(pair.getFirst(), pair.getSecond());
 +        }
 +      } catch (ClosedChannelException ex) {
 +        throw new LogClosedException();
 +      } catch (Exception e) {
 +        log.error(e, e);
 +        work.exception = e;
 +      }
 +    }
 +
 +    synchronized (closeLock) {
 +      // use a different lock for close check so that adding to work queue does not need
 +      // to wait on walog I/O operations
 +
 +      if (closed)
 +        throw new LogClosedException();
 +      workQueue.add(work);
 +    }
 +
 +    return new LoggerOperation(work);
 +  }
 +
 +  public LoggerOperation logManyTablets(List<TabletMutations> mutations) throws IOException {
 +    List<Pair<LogFileKey,LogFileValue>> data = new ArrayList<Pair<LogFileKey,LogFileValue>>();
 +    for (TabletMutations tabletMutations : mutations) {
 +      LogFileKey key = new LogFileKey();
 +      key.event = MANY_MUTATIONS;
 +      key.seq = tabletMutations.getSeq();
 +      key.tid = tabletMutations.getTid();
 +      LogFileValue value = new LogFileValue();
 +      value.mutations = tabletMutations.getMutations();
 +      data.add(new Pair<LogFileKey,LogFileValue>(key, value));
 +    }
 +    return logFileData(data);
 +  }
 +
 +  public LoggerOperation minorCompactionFinished(int seq, int tid, String fqfn) throws IOException {
 +    LogFileKey key = new LogFileKey();
 +    key.event = COMPACTION_FINISH;
 +    key.seq = seq;
 +    key.tid = tid;
 +    return logFileData(Collections.singletonList(new Pair<LogFileKey,LogFileValue>(key, EMPTY)));
 +  }
 +
 +  public LoggerOperation minorCompactionStarted(int seq, int tid, String fqfn) throws IOException {
 +    LogFileKey key = new LogFileKey();
 +    key.event = COMPACTION_START;
 +    key.seq = seq;
 +    key.tid = tid;
 +    key.filename = fqfn;
 +    return logFileData(Collections.singletonList(new Pair<LogFileKey,LogFileValue>(key, EMPTY)));
 +  }
 +
 +  public String getLogger() {
 +    String parts[] = logPath.split("/");
 +    return StringUtil.join(Arrays.asList(parts[parts.length - 2].split("[+]")), ":");
 +  }
 +
 +}


[27/35] git commit: ACCUMULO-2840 turn audit logging back on for the IT

Posted by el...@apache.org.
ACCUMULO-2840 turn audit logging back on for the IT


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

Branch: refs/heads/ACCUMULO-378
Commit: 168811d68ba2a21cd37b91bfc491651d4f1de529
Parents: 2cad992
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Jun 4 16:17:58 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Jun 4 16:17:58 2014 -0400

----------------------------------------------------------------------
 test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/168811d6/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java b/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
index ef13f64..38869a3 100644
--- a/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
@@ -46,6 +46,7 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.server.security.AuditedSecurityOperation;
 import org.apache.accumulo.test.functional.ConfigurableMacIT;
 import org.apache.commons.io.FileUtils;
@@ -69,6 +70,10 @@ public class AuditMessageIT extends ConfigurableMacIT {
   private static final String NEW_TEST_TABLE_NAME = "oranges";
   private static final String THIRD_TEST_TABLE_NAME = "pears";
   private static final Authorizations auths = new Authorizations("private", "public");
+  
+  public void beforeClusterStart(MiniAccumuloConfigImpl cfg) throws Exception {
+    new File(cfg.getConfDir(), "auditLog.xml").delete();
+  }
 
   // Must be static to survive Junit re-initialising the class every time.
   private static String lastAuditTimestamp;


[10/35] ACCUMULO-2041 extract tablet classes to new files, move tablet-related code to o.a.a.tserver.tablet, make member variables private

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletClosedException.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletClosedException.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletClosedException.java
new file mode 100644
index 0000000..827b803
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletClosedException.java
@@ -0,0 +1,29 @@
+/*
+ * 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.tserver.tablet;
+
+public class TabletClosedException extends RuntimeException {
+  public TabletClosedException(Exception e) {
+    super(e);
+  }
+
+  public TabletClosedException() {
+    super();
+  }
+
+  private static final long serialVersionUID = 1L;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
new file mode 100644
index 0000000..bd87a5b
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
@@ -0,0 +1,48 @@
+/*
+ * 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.tserver.tablet;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.tserver.InMemoryMap;
+import org.apache.accumulo.tserver.log.DfsLogger;
+
+public interface TabletCommitter {
+
+  void abortCommit(CommitSession commitSession, List<Mutation> value);
+
+  void commit(CommitSession commitSession, List<Mutation> mutations);
+
+  boolean beginUpdatingLogsUsed(InMemoryMap memTable, Collection<DfsLogger> copy, boolean mincFinish);
+
+  void finishUpdatingLogsUsed();
+
+  TableConfiguration getTableConfiguration();
+
+  KeyExtent getExtent();
+
+  int getLogId();
+
+  boolean getUseWAL();
+
+  void updateMemoryUsageStats(long estimatedSizeInBytes, long estimatedSizeInBytes2);
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletMemory.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletMemory.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletMemory.java
new file mode 100644
index 0000000..155d6b5
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletMemory.java
@@ -0,0 +1,190 @@
+/*
+ * 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.tserver.tablet;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
+import org.apache.accumulo.tserver.InMemoryMap;
+import org.apache.accumulo.tserver.InMemoryMap.MemoryIterator;
+import org.apache.log4j.Logger;
+
+class TabletMemory implements Closeable {
+  static private final Logger log = Logger.getLogger(TabletMemory.class);
+  
+  private final TabletCommitter tablet;
+  private InMemoryMap memTable;
+  private InMemoryMap otherMemTable;
+  private InMemoryMap deletingMemTable;
+  private int nextSeq = 1;
+  private CommitSession commitSession;
+
+  TabletMemory(TabletCommitter tablet) {
+    this.tablet = tablet;
+    try {
+      memTable = new InMemoryMap(tablet.getTableConfiguration());
+    } catch (LocalityGroupConfigurationError e) {
+      throw new RuntimeException(e);
+    }
+    commitSession = new CommitSession(tablet, nextSeq, memTable);
+    nextSeq += 2;
+  }
+
+  public InMemoryMap getMemTable() {
+    return memTable;
+  }
+
+  public InMemoryMap getMinCMemTable() {
+    return otherMemTable;
+  }
+
+  public CommitSession prepareForMinC() {
+    if (otherMemTable != null) {
+      throw new IllegalStateException();
+    }
+
+    if (deletingMemTable != null) {
+      throw new IllegalStateException();
+    }
+    if (commitSession == null) {
+      throw new IllegalStateException();
+    }
+
+    otherMemTable = memTable;
+    try {
+      memTable = new InMemoryMap(tablet.getTableConfiguration());
+    } catch (LocalityGroupConfigurationError e) {
+      throw new RuntimeException(e);
+    }
+
+    CommitSession oldCommitSession = commitSession;
+    commitSession = new CommitSession(tablet, nextSeq, memTable);
+    nextSeq += 2;
+
+    tablet.updateMemoryUsageStats(memTable.estimatedSizeInBytes(), otherMemTable.estimatedSizeInBytes());
+
+    return oldCommitSession;
+  }
+
+  public void finishedMinC() {
+
+    if (otherMemTable == null) {
+      throw new IllegalStateException();
+    }
+
+    if (deletingMemTable != null) {
+      throw new IllegalStateException();
+    }
+    
+    if (commitSession == null) {
+      throw new IllegalStateException();
+    }
+
+    deletingMemTable = otherMemTable;
+
+    otherMemTable = null;
+    tablet.notifyAll();
+  }
+
+  public void finalizeMinC() {
+    if (commitSession == null) {
+      throw new IllegalStateException();
+    }
+    try {
+      deletingMemTable.delete(15000);
+    } finally {
+      synchronized (tablet) {
+        if (otherMemTable != null) {
+          throw new IllegalStateException();
+        }
+
+        if (deletingMemTable == null) {
+          throw new IllegalStateException();
+        }
+
+        deletingMemTable = null;
+
+        tablet.updateMemoryUsageStats(memTable.estimatedSizeInBytes(), 0);
+      }
+    }
+  }
+
+  public boolean memoryReservedForMinC() {
+    return otherMemTable != null || deletingMemTable != null;
+  }
+
+  public void waitForMinC() {
+    while (otherMemTable != null || deletingMemTable != null) {
+      try {
+        tablet.wait(50);
+      } catch (InterruptedException e) {
+        log.warn(e, e);
+      }
+    }
+  }
+
+  public void mutate(CommitSession cm, List<Mutation> mutations) {
+    cm.mutate(mutations);
+  }
+
+  public void updateMemoryUsageStats() {
+    long other = 0;
+    if (otherMemTable != null)
+      other = otherMemTable.estimatedSizeInBytes();
+    else if (deletingMemTable != null)
+      other = deletingMemTable.estimatedSizeInBytes();
+
+    tablet.updateMemoryUsageStats(memTable.estimatedSizeInBytes(), other);
+  }
+
+  public List<MemoryIterator> getIterators() {
+    List<MemoryIterator> toReturn = new ArrayList<MemoryIterator>(2);
+    toReturn.add(memTable.skvIterator());
+    if (otherMemTable != null)
+      toReturn.add(otherMemTable.skvIterator());
+    return toReturn;
+  }
+
+  public void returnIterators(List<MemoryIterator> iters) {
+    for (MemoryIterator iter : iters) {
+      iter.close();
+    }
+  }
+
+  public long getNumEntries() {
+    if (otherMemTable != null)
+      return memTable.getNumEntries() + otherMemTable.getNumEntries();
+    return memTable.getNumEntries();
+  }
+
+  public CommitSession getCommitSession() {
+    return commitSession;
+  }
+
+  @Override
+  public void close() throws IOException {
+    commitSession = null;
+  }
+
+  public boolean isClosed() {
+    return commitSession == null;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
index c5c3316..253c97e 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/CountingIteratorTest.java
@@ -26,7 +26,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
-import org.apache.accumulo.tserver.Compactor.CountingIterator;
+import org.apache.accumulo.tserver.tablet.Compactor.CountingIterator;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/test/java/org/apache/accumulo/tserver/RootFilesTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/RootFilesTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/RootFilesTest.java
index f216e93..7cfe65c 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/RootFilesTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/RootFilesTest.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
+import org.apache.accumulo.tserver.tablet.RootFiles;
 import org.apache.hadoop.fs.Path;
 import org.junit.Assert;
 import org.junit.Rule;


[31/35] git commit: ACCUMULO-2858 Ensure that we only look for 'sync' when we couldn't load configured method

Posted by el...@apache.org.
ACCUMULO-2858 Ensure that we only look for 'sync' when we couldn't load configured method


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

Branch: refs/heads/ACCUMULO-378
Commit: 6dfcf8ff730502abe74fa5d61126ccc2f4bb30f8
Parents: 1e16159
Author: Josh Elser <el...@apache.org>
Authored: Wed Jun 4 16:38:27 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Jun 4 16:38:27 2014 -0400

----------------------------------------------------------------------
 .../accumulo/server/tabletserver/log/DfsLogger.java   | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6dfcf8ff/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java
index fb21ba5..1f72c71 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/log/DfsLogger.java
@@ -253,17 +253,17 @@ public class DfsLogger {
 
       String syncMethod = conf.getConfiguration().get(Property.TSERV_WAL_SYNC_METHOD);
       try {
-        try {
-          // hsync: send data to datanodes and sync the data to disk
-          sync = logFile.getClass().getMethod(syncMethod);
-        } catch (NoSuchMethodException ex) {
-          log.warn("Could not find configured " + syncMethod + " method, trying to fall back to old Hadoop sync method", ex);
+        // hsync: send data to datanodes and sync the data to disk
+        sync = logFile.getClass().getMethod(syncMethod);
+      } catch (Exception ex) {
+        log.warn("Could not find configured " + syncMethod + " method, trying to fall back to old Hadoop sync method", ex);
 
+        try {
           // sync: send data to datanodes
           sync = logFile.getClass().getMethod("sync");
+        } catch (Exception e) {
+          throw new RuntimeException(e);
         }
-      } catch (Exception e) {
-        throw new RuntimeException(e);
       }
 
       // Initialize the crypto operations.


[34/35] git commit: Merge remote-tracking branch 'origin/master' into ACCUMULO-378

Posted by el...@apache.org.
Merge remote-tracking branch 'origin/master' into ACCUMULO-378

Conflicts:
	server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java


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

Branch: refs/heads/ACCUMULO-378
Commit: ada6ce464b1e9d818c06655369b30a45afa840c0
Parents: 98eb56f 47d5933
Author: Josh Elser <el...@apache.org>
Authored: Wed Jun 4 22:07:59 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Wed Jun 4 22:07:59 2014 -0400

----------------------------------------------------------------------
 .../server/GarbageCollectionLogger.java         | 103 ++++
 .../apache/accumulo/tserver/TabletServer.java   | 607 ++++++-------------
 .../accumulo/tserver/TservConstraintEnv.java    |  83 +++
 .../apache/accumulo/tserver/log/DfsLogger.java  |  20 +-
 .../tserver/session/ConditionalSession.java     |  41 ++
 .../tserver/session/MultiScanSession.java       |  62 ++
 .../accumulo/tserver/session/ScanSession.java   |  69 +++
 .../accumulo/tserver/session/Session.java       |  43 ++
 .../accumulo/tserver/session/UpdateSession.java |  56 ++
 .../apache/accumulo/tserver/tablet/Tablet.java  |  12 +-
 .../apache/accumulo/test/AuditMessageIT.java    |   5 +
 11 files changed, 649 insertions(+), 452 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ada6ce46/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index e6286ff,ee28c7f..dd3c16e
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@@ -78,11 -76,7 +76,8 @@@ import org.apache.accumulo.core.client.
  import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
  import org.apache.accumulo.core.conf.AccumuloConfiguration;
  import org.apache.accumulo.core.conf.Property;
- import org.apache.accumulo.core.constraints.Constraint.Environment;
- import org.apache.accumulo.core.constraints.Violations;
- import org.apache.accumulo.core.data.ByteSequence;
  import org.apache.accumulo.core.data.Column;
 +import org.apache.accumulo.core.data.ColumnUpdate;
  import org.apache.accumulo.core.data.ConstraintViolationSummary;
  import org.apache.accumulo.core.data.Key;
  import org.apache.accumulo.core.data.KeyExtent;
@@@ -116,8 -110,6 +111,7 @@@ import org.apache.accumulo.core.master.
  import org.apache.accumulo.core.metadata.MetadataTable;
  import org.apache.accumulo.core.metadata.RootTable;
  import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 +import org.apache.accumulo.core.replication.thrift.ReplicationServicer;
- import org.apache.accumulo.core.security.AuthorizationContainer;
  import org.apache.accumulo.core.security.Authorizations;
  import org.apache.accumulo.core.security.SecurityUtil;
  import org.apache.accumulo.core.security.thrift.TCredentials;
@@@ -213,8 -205,11 +207,13 @@@ import org.apache.accumulo.tserver.metr
  import org.apache.accumulo.tserver.metrics.TabletServerMinCMetrics;
  import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
  import org.apache.accumulo.tserver.metrics.TabletServerUpdateMetrics;
 +import org.apache.accumulo.tserver.replication.ReplicationServicerHandler;
 +import org.apache.accumulo.tserver.replication.ReplicationWorker;
+ import org.apache.accumulo.tserver.session.ConditionalSession;
+ import org.apache.accumulo.tserver.session.MultiScanSession;
+ import org.apache.accumulo.tserver.session.ScanSession;
+ import org.apache.accumulo.tserver.session.Session;
+ import org.apache.accumulo.tserver.session.UpdateSession;
  import org.apache.accumulo.tserver.tablet.CommitSession;
  import org.apache.accumulo.tserver.tablet.CompactionInfo;
  import org.apache.accumulo.tserver.tablet.CompactionWatcher;
@@@ -262,18 -258,56 +262,61 @@@ public class TabletServer extends Abstr
      return mincMetrics;
    }
  
-   private ServerConfiguration serverConfig;
-   private LogSorter logSorter = null;
+   private final ServerConfiguration serverConfig;
+   private final LogSorter logSorter;
 +  private ReplicationWorker replWorker = null;
+   private final TabletStatsKeeper statsKeeper;
+   private final AtomicInteger logIdGenerator = new AtomicInteger();
+   
+   private final VolumeManager fs;
+   public Instance getInstance() {
+     return serverConfig.getInstance();
+   }
+ 
+   private final SortedMap<KeyExtent,Tablet> onlineTablets = Collections.synchronizedSortedMap(new TreeMap<KeyExtent,Tablet>());
+   private final SortedSet<KeyExtent> unopenedTablets = Collections.synchronizedSortedSet(new TreeSet<KeyExtent>());
+   private final SortedSet<KeyExtent> openingTablets = Collections.synchronizedSortedSet(new TreeSet<KeyExtent>());
+   @SuppressWarnings("unchecked")
+   private final Map<KeyExtent,Long> recentlyUnloadedCache = Collections.synchronizedMap(new LRUMap(1000));
+ 
+   private final TabletServerResourceManager resourceManager;
+   private final SecurityOperation security;
+ 
+   private final BlockingDeque<MasterMessage> masterMessages = new LinkedBlockingDeque<MasterMessage>();
+ 
+   private Thread majorCompactorThread;
+ 
++  private HostAndPort replicationAddress;
+   private HostAndPort clientAddress;
+ 
+   private volatile boolean serverStopRequested = false;
+   private volatile boolean majorCompactorDisabled = false;
+   private volatile boolean shutdownComplete = false;
+ 
+   private ZooLock tabletServerLock;
+ 
+   private TServer server;
++  private TServer replServer;
+ 
+   private DistributedWorkQueue bulkFailedCopyQ;
+ 
+   private String lockID;
+ 
+   private static ObjectName OBJECT_NAME = null;
+ 
+   public static final AtomicLong seekCount = new AtomicLong(0);
+   
+   private final AtomicLong totalMinorCompactions = new AtomicLong(0);
  
    public TabletServer(ServerConfiguration conf, VolumeManager fs) {
      super();
      this.serverConfig = conf;
-     this.instance = conf.getInstance();
      this.fs = fs;
      AccumuloConfiguration aconf = getSystemConfiguration();
 -    this.logSorter = new LogSorter(getInstance(), fs, aconf);
++    Instance instance = getInstance();
 +    this.logSorter = new LogSorter(instance, fs, aconf);
 +    this.replWorker = new ReplicationWorker(instance, fs, aconf);
+     this.statsKeeper = new TabletStatsKeeper();
      SimpleTimer.getInstance(aconf).schedule(new Runnable() {
        @Override
        public void run() {
@@@ -3122,29 -2856,6 +2880,29 @@@
      return address;
    }
  
 +  private HostAndPort startReplicationService() throws UnknownHostException {
 +    ReplicationServicer.Iface repl = TraceWrap.service(new ReplicationServicerHandler(HdfsZooInstance.getInstance()));
 +    ReplicationServicer.Processor<ReplicationServicer.Iface> processor = new ReplicationServicer.Processor<ReplicationServicer.Iface>(repl);
 +    AccumuloConfiguration conf = getSystemConfiguration();
 +    Property maxMessageSizeProperty = (conf.get(Property.TSERV_MAX_MESSAGE_SIZE) != null ? Property.TSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
 +    ServerAddress sp = TServerUtils.startServer(conf, clientAddress.getHostText(), Property.REPLICATION_RECEIPT_SERVICE_PORT, processor,
 +        "ReplicationServicerHandler", "Replication Servicer", null, Property.REPLICATION_MIN_THREADS, Property.REPLICATION_THREADCHECK, maxMessageSizeProperty);
 +    this.replServer = sp.server;
 +    log.info("Started replication service on " + sp.address);
 +
 +    try {
 +      // The replication service is unique to the thrift service for a tserver, not just a host.
 +      // Advertise the host and port for replication service given the host and port for the tserver.
-       ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(instance) + Constants.ZREPLICATION_TSERVERS + "/" + clientAddress.toString(),
++      ZooReaderWriter.getInstance().putPersistentData(ZooUtil.getRoot(getInstance()) + Constants.ZREPLICATION_TSERVERS + "/" + clientAddress.toString(),
 +          sp.address.toString().getBytes(StandardCharsets.UTF_8), NodeExistsPolicy.OVERWRITE);
 +    } catch (Exception e) {
 +      log.error("Could not advertise replication service port", e);
 +      throw new RuntimeException(e);
 +    }
 +
 +    return sp.address;
 +  }
 +
    public ZooLock getLock() {
      return tabletServerLock;
    }
@@@ -3527,6 -3205,6 +3280,13 @@@
      return clientAddress.getHostText() + ":" + clientAddress.getPort();
    }
  
++  public String getReplicationAddressSTring() {
++    if (null == replicationAddress) {
++      return null;
++    }
++    return replicationAddress.getHostText() + ":" + replicationAddress.getPort();
++  }
++
    public TServerInstance getTabletSession() {
      String address = getClientAddressString();
      if (address == null)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ada6ce46/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------


[24/35] git commit: ACCUMULO-378 Some more test stabilization

Posted by el...@apache.org.
ACCUMULO-378 Some more test stabilization


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

Branch: refs/heads/ACCUMULO-378
Commit: 2f02d69d79dbfd5c5ca6aaa2c97bddc6317bd3c2
Parents: e81eee7
Author: Josh Elser <el...@apache.org>
Authored: Tue Jun 3 22:03:34 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Jun 3 22:03:34 2014 -0400

----------------------------------------------------------------------
 .../test/replication/ReplicationIT.java         | 23 ++++++++----
 .../test/replication/ReplicationTest.java       |  6 ++--
 .../UnorderedWorkAssignerReplicationIT.java     | 37 ++++++++++++++------
 3 files changed, 45 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2f02d69d/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
index db21586..d051e75 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@ -403,7 +403,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     }
   }
 
-  @Test(timeout = 60 * 5000)
+  @Test(timeout = 10 * 60 * 1000)
   public void dataWasReplicatedToThePeerWithoutDrain() throws Exception {
     MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
         ROOT_PASSWORD);
@@ -626,14 +626,23 @@ public class ReplicationIT extends ConfigurableMacIT {
       Assert.assertNotEquals(0, fullyReplicated);
 
       long countTable = 0l;
-      for (Entry<Key,Value> entry : connPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
-        countTable++;
-        Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
-            .startsWith(masterTable1));
+      for (int i = 0; i < 10; i++) {
+        for (Entry<Key,Value> entry : connPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
+          countTable++;
+          Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
+              .startsWith(masterTable1));
+        }
+  
+        log.info("Found {} records in {}", countTable, peerTable1);
+
+        if (0l == countTable) {
+          Thread.sleep(5000);
+        } else {
+          break;
+        }
       }
 
-      log.info("Found {} records in {}", countTable, peerTable1);
-      Assert.assertTrue(countTable > 0);
+      Assert.assertTrue("Found no records in " + peerTable1 + " in the peer cluster", countTable > 0);
 
       countTable = 0l;
       for (Entry<Key,Value> entry : connPeer.createScanner(peerTable2, Authorizations.EMPTY)) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2f02d69d/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
index 51e4e46..04d1d6e 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
@@ -94,8 +94,8 @@ public class ReplicationTest extends ConfigurableMacIT {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     // Run the master replication loop run frequently
-    cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "0");
-    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "0s");
+    cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
+    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "1M");
     cfg.setProperty(Property.GC_CYCLE_START, "1s");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "0");
@@ -1302,7 +1302,7 @@ public class ReplicationTest extends ConfigurableMacIT {
           }
         }
 
-        Thread.sleep(1000);
+        Thread.sleep(2000);
       }
 
       if (notFound) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2f02d69d/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
index 17c8179..6c21962 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerReplicationIT.java
@@ -370,23 +370,38 @@ public class UnorderedWorkAssignerReplicationIT extends ConfigurableMacIT {
       connMaster.replicationOperations().drain(masterTable2, filesFor2);
 
       long countTable = 0l;
-      for (Entry<Key,Value> entry : connPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
-        countTable++;
-        Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
-            .startsWith(masterTable1));
+      for (int i = 0; i < 5; i++) {
+        countTable = 0l;
+        for (Entry<Key,Value> entry : connPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
+          countTable++;
+          Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
+              .startsWith(masterTable1));
+        }
+  
+        log.info("Found {} records in {}", countTable, peerTable1);
+
+        if (masterTable1Records != countTable) {
+          log.warn("Did not find {} expected records in {}, only found {}", masterTable1Records, peerTable1, countTable);
+        }
       }
 
-      log.info("Found {} records in {}", countTable, peerTable1);
       Assert.assertEquals(masterTable1Records, countTable);
 
-      countTable = 0l;
-      for (Entry<Key,Value> entry : connPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
-        countTable++;
-        Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
-            .startsWith(masterTable2));
+      for (int i = 0; i < 5; i++) {
+        countTable = 0l;
+        for (Entry<Key,Value> entry : connPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
+          countTable++;
+          Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
+              .startsWith(masterTable2));
+        }
+  
+        log.info("Found {} records in {}", countTable, peerTable2);
+
+        if (masterTable2Records != countTable) {
+          log.warn("Did not find {} expected records in {}, only found {}", masterTable2Records, peerTable2, countTable);
+        }
       }
 
-      log.info("Found {} records in {}", countTable, peerTable2);
       Assert.assertEquals(masterTable2Records, countTable);
 
     } finally {


[21/35] git commit: ACCUMULO-2041 finished state management in Tablet

Posted by el...@apache.org.
ACCUMULO-2041 finished state management in Tablet


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

Branch: refs/heads/ACCUMULO-378
Commit: f280e9713ca3016cec3c082321774d579c86d51e
Parents: 731abce
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Jun 3 16:42:43 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Jun 3 16:42:43 2014 -0400

----------------------------------------------------------------------
 .../accumulo/server/tablets/TabletTime.java     |   1 -
 .../accumulo/tserver/TabletStatsKeeper.java     |   1 +
 .../accumulo/tserver/tablet/CompactionInfo.java |  16 +++
 .../apache/accumulo/tserver/tablet/Tablet.java  | 142 +++++++++----------
 4 files changed, 85 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f280e971/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java b/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java
index 7f6dcf7..e3fd8f3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/tablets/TabletTime.java
@@ -45,7 +45,6 @@ public abstract class TabletTime {
   
   public abstract String getMetadataValue();
   
-  // abstract long setUpdateTimes(Mutation mutation);
   public abstract long setUpdateTimes(List<Mutation> mutations);
   
   public abstract long getTime();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f280e971/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletStatsKeeper.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletStatsKeeper.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletStatsKeeper.java
index d914ac6..40906df 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletStatsKeeper.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletStatsKeeper.java
@@ -22,6 +22,7 @@ import org.apache.accumulo.server.util.ActionStatsUpdator;
 
 public class TabletStatsKeeper {
   
+  // suspect we need more synchronization in this class
   private ActionStats major = new ActionStats();
   private ActionStats minor = new ActionStats();
   private ActionStats split = new ActionStats();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f280e971/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionInfo.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionInfo.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionInfo.java
index ab57d65..8e9fb9b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionInfo.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionInfo.java
@@ -1,3 +1,19 @@
+/*
+ * 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.tserver.tablet;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f280e971/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index dc2fc4d..2be00fe 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -143,7 +143,7 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
  * 
  */
 public class Tablet implements TabletCommitter {
-  static final Logger log = Logger.getLogger(Tablet.class);
+  static private final Logger log = Logger.getLogger(Tablet.class);
   static private final List<LogEntry> NO_LOG_ENTRIES = Collections.emptyList();
 
   private final TabletServer tabletServer;
@@ -166,23 +166,27 @@ public class Tablet implements TabletCommitter {
   private final AtomicLong dataSourceDeletions = new AtomicLong(0);
   public long getDataSourceDeletions() { return dataSourceDeletions.get(); }
   private final Set<ScanDataSource> activeScans = new HashSet<ScanDataSource>();
+  
+  private static enum CloseState {
+    OPEN,
+    CLOSING,
+    CLOSED,
+    COMPLETE
+  }
 
-  private volatile boolean closing = false;
-  private boolean closed = false;
-  private boolean closeComplete = false;
+  private volatile CloseState closeState = CloseState.OPEN;
 
   private boolean updatingFlushID = false;
 
   private long lastFlushID = -1;
   private long lastCompactID = -1;
   
-  private volatile boolean majorCompactionInProgress = false;
-  private volatile boolean majorCompactionWaitingToStart = false;
-  private Set<MajorCompactionReason> majorCompactionQueued = Collections.synchronizedSet(EnumSet.noneOf(MajorCompactionReason.class));
-  
-  private volatile boolean minorCompactionInProgress = false;
-  private volatile boolean minorCompactionWaitingToStart = false;
+  static enum CompactionState { WAITING_TO_START, IN_PROGRESS };
+  private volatile CompactionState minorCompactionState = null;
+  private volatile CompactionState majorCompactionState = null;
 
+  private final Set<MajorCompactionReason> majorCompactionQueued = Collections.synchronizedSet(EnumSet.noneOf(MajorCompactionReason.class));
+  
   private final AtomicReference<ConstraintChecker> constraintChecker = new AtomicReference<ConstraintChecker>();
 
   private int writesInProgress = 0;
@@ -220,7 +224,7 @@ public class Tablet implements TabletCommitter {
     return logId;
   }
   
-  public class LookupResult {
+  public static class LookupResult {
     public List<Range> unfinishedRanges = new ArrayList<Range>();
     public long bytesAdded = 0;
     public long dataSize = 0;
@@ -228,7 +232,7 @@ public class Tablet implements TabletCommitter {
   }
 
   FileRef getNextMapFilename(String prefix) throws IOException {
-    String extension = FileOperations.getNewFileExtension(this.tableConfiguration);
+    String extension = FileOperations.getNewFileExtension(tableConfiguration);
     checkTabletDir();
     return new FileRef(location.toString() + "/" + prefix + UniqueNameAllocator.getInstance().getNextName() + "." + extension);
   }
@@ -237,18 +241,18 @@ public class Tablet implements TabletCommitter {
     if (!tableDirChecked) {
       FileStatus[] files = null;
       try {
-        files = getTabletServer().getFileSystem().listStatus(this.location);
+        files = getTabletServer().getFileSystem().listStatus(location);
       } catch (FileNotFoundException ex) {
         // ignored
       }
       
       if (files == null) {
-        if (this.location.getName().startsWith("c-"))
-          log.debug("Tablet " + extent + " had no dir, creating " + this.location); // its a clone dir...
+        if (location.getName().startsWith("c-"))
+          log.debug("Tablet " + extent + " had no dir, creating " + location); // its a clone dir...
         else
-          log.warn("Tablet " + extent + " had no dir, creating " + this.location);
+          log.warn("Tablet " + extent + " had no dir, creating " + location);
       
-        getTabletServer().getFileSystem().mkdirs(this.location);
+        getTabletServer().getFileSystem().mkdirs(location);
       }
       tableDirChecked = true;
     }
@@ -524,7 +528,7 @@ public class Tablet implements TabletCommitter {
     configObserver.propertiesChanged();
 
     if (!logEntries.isEmpty()) {
-      log.info("Starting Write-Ahead Log recovery for " + this.extent);
+      log.info("Starting Write-Ahead Log recovery for " + extent);
       final long[] count = new long[2];
       final CommitSession commitSession = getTabletMemory().getCommitSession();
       count[1] = Long.MIN_VALUE;
@@ -942,7 +946,7 @@ public class Tablet implements TabletCommitter {
         if (lastFlushID >= tableFlushID)
           return;
 
-        if (closing || closed || getTabletMemory().memoryReservedForMinC())
+        if (isClosing() || isClosed() || getTabletMemory().memoryReservedForMinC())
           return;
 
         if (getTabletMemory().getMemTable().getNumEntries() == 0) {
@@ -1022,15 +1026,14 @@ public class Tablet implements TabletCommitter {
       synchronized (this) {
         t1 = System.currentTimeMillis();
 
-        if (closing || closed || majorCompactionWaitingToStart || getTabletMemory().memoryReservedForMinC() || getTabletMemory().getMemTable().getNumEntries() == 0
+        if (isClosing() || isClosed() || majorCompactionState == CompactionState.WAITING_TO_START || getTabletMemory().memoryReservedForMinC() || getTabletMemory().getMemTable().getNumEntries() == 0
             || updatingFlushID) {
 
           logMessage = new StringBuilder();
 
           logMessage.append(extent.toString());
-          logMessage.append(" closing " + closing);
-          logMessage.append(" closed " + closed);
-          logMessage.append(" majorCompactionWaitingToStart " + majorCompactionWaitingToStart);
+          logMessage.append(" closeState " + closeState);
+          logMessage.append(" majorCompactionState " + majorCompactionState);
           if (getTabletMemory() != null)
             logMessage.append(" tabletMemory.memoryReservedForMinC() " + getTabletMemory().memoryReservedForMinC());
           if (getTabletMemory() != null && getTabletMemory().getMemTable() != null)
@@ -1145,8 +1148,7 @@ public class Tablet implements TabletCommitter {
       throw new IllegalStateException("waitingForLogs < 0 " + writesInProgress);
     }
 
-    if (closed || getTabletMemory() == null) {
-      // log.debug("tablet closed, can't commit");
+    if (isClosed() || getTabletMemory() == null) {
       return null;
     }
 
@@ -1217,7 +1219,7 @@ public class Tablet implements TabletCommitter {
       throw new IllegalStateException("waitingForLogs <= 0 " + writesInProgress);
     }
 
-    if (closeComplete || getTabletMemory() == null) {
+    if (isCloseComplete() || getTabletMemory() == null) {
       throw new IllegalStateException("aborting commit when tablet is closed");
     }
 
@@ -1245,7 +1247,7 @@ public class Tablet implements TabletCommitter {
         throw new IllegalStateException("commiting mutations after logging, but not waiting for any log messages");
       }
 
-      if (closed && closeComplete) {
+      if (isCloseComplete()) {
         throw new IllegalStateException("tablet closed with outstanding messages to the logger");
       }
 
@@ -1284,28 +1286,24 @@ public class Tablet implements TabletCommitter {
     MinorCompactionTask mct = null;
 
     synchronized (this) {
-      if (closed || closing || closeComplete) {
-        String msg = "Tablet " + getExtent() + " already";
-        if (closed)
-          msg += " closed";
-        if (closing)
-          msg += " closing";
-        if (closeComplete)
-          msg += " closeComplete";
+      if (isClosed() || isClosing() || isCloseComplete()) {
+        String msg = "Tablet " + getExtent() + " already " + closeState;
         throw new IllegalStateException(msg);
       }
 
       // enter the closing state, no splits, minor, or major compactions can start
       // should cause running major compactions to stop
-      closing = true;
+      closeState = CloseState.CLOSING;
       this.notifyAll();
 
       // determines if inserts and queries can still continue while minor compacting
-      closed = disableWrites;
+      if (disableWrites) {
+        closeState = CloseState.CLOSING;
+      }
 
       // wait for major compactions to finish, setting closing to
       // true should cause any running major compactions to abort
-      while (majorCompactionInProgress) {
+      while (majorCompactionRunning()) {
         try {
           this.wait(50);
         } catch (InterruptedException e) {
@@ -1349,9 +1347,8 @@ public class Tablet implements TabletCommitter {
 
   synchronized void completeClose(boolean saveState, boolean completeClose) throws IOException {
 
-    if (!closing || closeComplete || closeCompleting) {
-      throw new IllegalStateException("closing = " + closing + " closed = " + closed + " closeComplete = " + closeComplete + " closeCompleting = "
-          + closeCompleting);
+    if (!isClosing() || isCloseComplete() || closeCompleting) {
+      throw new IllegalStateException("closeState = " + closeState);
     }
 
     log.debug("completeClose(saveState=" + saveState + " completeClose=" + completeClose + ") " + getExtent());
@@ -1359,7 +1356,7 @@ public class Tablet implements TabletCommitter {
     // ensure this method is only called once, also guards against multiple
     // threads entering the method at the same time
     closeCompleting = true;
-    closed = true;
+    closeState = CloseState.CLOSED;
 
     // modify dataSourceDeletions so scans will try to switch data sources and fail because the tablet is closed
     dataSourceDeletions.incrementAndGet();
@@ -1422,7 +1419,8 @@ public class Tablet implements TabletCommitter {
     tableConfiguration.getNamespaceConfiguration().removeObserver(configObserver);
     tableConfiguration.removeObserver(configObserver);
 
-    closeComplete = completeClose;
+    if (completeClose)
+      closeState = CloseState.COMPLETE;
   }
 
   private void closeConsistencyCheck() {
@@ -1491,7 +1489,7 @@ public class Tablet implements TabletCommitter {
 
   public synchronized boolean initiateMajorCompaction(MajorCompactionReason reason) {
 
-    if (closing || closed || !needsMajorCompaction(reason) || majorCompactionInProgress || majorCompactionQueued.contains(reason)) {
+    if (isClosing() || isClosed() || !needsMajorCompaction(reason) || majorCompactionRunning() || majorCompactionQueued.contains(reason)) {
       return false;
     }
 
@@ -1507,7 +1505,7 @@ public class Tablet implements TabletCommitter {
    * 
    */
   public boolean needsMajorCompaction(MajorCompactionReason reason) {
-    if (majorCompactionInProgress)
+    if (majorCompactionRunning())
       return false;
     if (reason == MajorCompactionReason.CHOP || reason == MajorCompactionReason.USER)
       return true;
@@ -1678,7 +1676,7 @@ public class Tablet implements TabletCommitter {
   public synchronized boolean needsSplit() {
     boolean ret;
 
-    if (closing || closed)
+    if (isClosing() || isClosed())
       ret = false;
     else
       ret = findSplitRow(getDatafileManager().getFiles()) != null;
@@ -1689,7 +1687,7 @@ public class Tablet implements TabletCommitter {
   // BEGIN PRIVATE METHODS RELATED TO MAJOR COMPACTION
 
   private boolean isCompactionEnabled() {
-    return !closing && !getTabletServer().isMajorCompactionDisabled();
+    return !isClosing() && !getTabletServer().isMajorCompactionDisabled();
   }
 
   private CompactionStats _majorCompact(MajorCompactionReason reason) throws IOException, CompactionCanceledException {
@@ -1725,13 +1723,13 @@ public class Tablet implements TabletCommitter {
 
       t1 = System.currentTimeMillis();
 
-      majorCompactionWaitingToStart = true;
+      majorCompactionState = CompactionState.WAITING_TO_START;
 
       getTabletMemory().waitForMinC();
 
       t2 = System.currentTimeMillis();
 
-      majorCompactionWaitingToStart = false;
+      majorCompactionState = null;
       notifyAll();
 
       VolumeManager fs = getTabletServer().getFileSystem();
@@ -1953,11 +1951,11 @@ public class Tablet implements TabletCommitter {
         // check that compaction is still needed - defer to splitting
         majorCompactionQueued.remove(reason);
 
-        if (closing || closed || !needsMajorCompaction(reason) || majorCompactionInProgress || needsSplit()) {
+        if (isClosing() || isClosed ()|| !needsMajorCompaction(reason) || majorCompactionRunning() || needsSplit()) {
           return null;
         }
 
-        majorCompactionInProgress = true;
+        majorCompactionState = CompactionState.WAITING_TO_START;
       }
 
       try {
@@ -1978,7 +1976,7 @@ public class Tablet implements TabletCommitter {
         // ensure we always reset boolean, even
         // when an exception is thrown
         synchronized (this) {
-          majorCompactionInProgress = false;
+          majorCompactionState = null;
           this.notifyAll();
         }
 
@@ -2033,27 +2031,27 @@ public class Tablet implements TabletCommitter {
   }
 
   public synchronized boolean isClosing() {
-    return closing;
+    return closeState == CloseState.CLOSING;
   }
 
   public synchronized boolean isClosed() {
-    return closed;
+    return closeState == CloseState.CLOSED;
   }
 
   public synchronized boolean isCloseComplete() {
-    return closeComplete;
+    return closeState == CloseState.COMPLETE;
   }
 
   public boolean majorCompactionRunning() {
-    return this.majorCompactionInProgress;
+    return majorCompactionState == CompactionState.IN_PROGRESS;
   }
 
   public boolean isMinorCompactionQueued() {
-    return minorCompactionWaitingToStart;
+    return minorCompactionState == CompactionState.WAITING_TO_START;
   }
 
   public boolean isMinorCompactionRunning() {
-    return minorCompactionInProgress;
+    return minorCompactionState == CompactionState.IN_PROGRESS; 
   }
 
   public boolean isMajorCompactionQueued() {
@@ -2104,11 +2102,11 @@ public class Tablet implements TabletCommitter {
 
       if (splitPoint == null || splitPoint.row == null) {
         log.info("had to abort split because splitRow was null");
-        closing = false;
+        closeState = CloseState.OPEN;
         return null;
       }
 
-      closed = true;
+      closeState = CloseState.CLOSING;
       completeClose(true, false);
 
       Text midRow = splitPoint.row;
@@ -2151,8 +2149,7 @@ public class Tablet implements TabletCommitter {
 
       log.debug(String.format("offline split time : %6.2f secs", (t2 - t1) / 1000.0));
 
-      closeComplete = true;
-
+      closeState = CloseState.COMPLETE;
       return newTablets;
     }
   }
@@ -2213,7 +2210,7 @@ public class Tablet implements TabletCommitter {
     // Don't do it if we spent too long waiting for the lock
     long now = System.currentTimeMillis();
     synchronized (this) {
-      if (closed) {
+      if (isClosed()) {
         throw new IOException("tablet " + extent + " is closed");
       }
 
@@ -2328,7 +2325,7 @@ public class Tablet implements TabletCommitter {
     try {
       synchronized (this) {
 
-        if (closed && closeComplete) {
+        if (isCloseComplete()) {
           throw new IllegalStateException("Can not update logs of closed tablet " + extent);
         }
 
@@ -2409,12 +2406,12 @@ public class Tablet implements TabletCommitter {
       if (lastCompactID >= compactionId)
         return;
 
-      if (closing || closed || majorCompactionQueued.contains(MajorCompactionReason.USER) || majorCompactionInProgress)
+      if (isClosing() || isClosed() || majorCompactionQueued.contains(MajorCompactionReason.USER) || majorCompactionRunning())
         return;
 
       if (getDatafileManager().getDatafileSizes().size() == 0) {
         // no files, so jsut update the metadata table
-        majorCompactionInProgress = true;
+        majorCompactionState = CompactionState.IN_PROGRESS;
         updateMetadata = true;
         lastCompactID = compactionId;
       } else
@@ -2428,7 +2425,7 @@ public class Tablet implements TabletCommitter {
         MetadataTableUtil.updateTabletCompactID(extent, compactionId, SystemCredentials.get(), getTabletServer().getLock());
       } finally {
         synchronized (this) {
-          majorCompactionInProgress = false;
+          majorCompactionState = null;
           this.notifyAll();
         }
       }
@@ -2535,20 +2532,19 @@ public class Tablet implements TabletCommitter {
   }
 
   public void minorCompactionWaitingToStart() {
-    minorCompactionWaitingToStart = true;
+    minorCompactionState = CompactionState.WAITING_TO_START;
   }
 
   public void minorCompactionStarted() {
-    minorCompactionWaitingToStart = false;
-    minorCompactionInProgress = true;
+    minorCompactionState = CompactionState.IN_PROGRESS;
   }
 
   public void minorCompactionComplete() {
-    minorCompactionInProgress = false;
+    minorCompactionState = null;
   }
 
   public boolean isMajorCompactionRunning() {
-    return majorCompactionInProgress;
+    return majorCompactionState == CompactionState.IN_PROGRESS;
   }
 
   public TabletStats getTabletStats() {
@@ -2558,6 +2554,4 @@ public class Tablet implements TabletCommitter {
   public AtomicLong getScannedCounter() {
     return scannedCount;
   }
-
-  
 }


[02/35] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'

Conflicts:
	core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java


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

Branch: refs/heads/ACCUMULO-378
Commit: 8990e590ab0ccc91120b2c641d23d19d31ecf111
Parents: 0712ec9 a61e242
Author: Bill Havanki <bh...@cloudera.com>
Authored: Mon Jun 2 10:28:52 2014 -0400
Committer: Bill Havanki <bh...@cloudera.com>
Committed: Mon Jun 2 10:28:52 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/ZooKeeperInstance.java |   7 +-
 .../core/client/impl/ConditionalWriterImpl.java |   4 +-
 .../client/impl/InstanceOperationsImpl.java     |   3 +-
 .../accumulo/core/client/impl/Namespaces.java   |   3 +-
 .../core/client/impl/RootTabletLocator.java     |  14 +-
 .../accumulo/core/client/impl/ServerClient.java |  15 +-
 .../accumulo/core/client/impl/Tables.java       |   3 +-
 .../core/client/impl/ZookeeperLockChecker.java  |  10 +-
 .../core/client/ZooKeeperInstanceTest.java      | 143 +++++++++++++++++++
 .../core/client/impl/RootTabletLocatorTest.java |  61 ++++++++
 .../client/impl/ZookeeperLockCheckerTest.java   |  58 ++++++++
 .../accumulo/fate/zookeeper/ZooCache.java       |  15 +-
 .../fate/zookeeper/ZooCacheFactory.java         |  78 ++++++++++
 .../apache/accumulo/fate/zookeeper/ZooLock.java |   2 +-
 .../fate/zookeeper/ZooCacheFactoryTest.java     |  87 +++++++++++
 .../accumulo/server/client/HdfsZooInstance.java |   3 +-
 .../server/conf/NamespaceConfiguration.java     |   3 +-
 .../server/conf/TableConfiguration.java         |   3 +-
 .../accumulo/server/conf/ZooConfiguration.java  |   5 +-
 19 files changed, 476 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index e4bc876,6d35757..0d19e01
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@@ -134,9 -133,11 +135,11 @@@ public class ZooKeeperInstance implemen
     *          See {@link ClientConfiguration} which extends Configuration with convenience methods specific to Accumulo.
     * @since 1.6.0
     */
- 
    public ZooKeeperInstance(Configuration config) {
+     this(config, new ZooCacheFactory());
+   }
+   ZooKeeperInstance(Configuration config, ZooCacheFactory zcf) {
 -    ArgumentChecker.notNull(config);
 +    checkArgument(config != null, "config is null");
      if (config instanceof ClientConfiguration) {
        this.clientConf = (ClientConfiguration)config;
      } else {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
index 65eeefc,30f0266..6be1a12
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java
@@@ -27,9 -26,10 +27,10 @@@ import java.util.TreeMap
  import org.apache.accumulo.core.Constants;
  import org.apache.accumulo.core.client.Instance;
  import org.apache.accumulo.core.client.NamespaceNotFoundException;
 -import org.apache.accumulo.core.util.ArgumentChecker.Validator;
 +import org.apache.accumulo.core.util.Validator;
  import org.apache.accumulo.core.zookeeper.ZooUtil;
  import org.apache.accumulo.fate.zookeeper.ZooCache;
+ import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
  
  public class Namespaces {
    public static final String VALID_NAME_REGEX = "^\\w*$";

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
index 93ba4d9,8da530b..d3822f4
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
@@@ -16,12 -16,7 +16,10 @@@
   */
  package org.apache.accumulo.core.client.impl;
  
 +import static com.google.common.base.Preconditions.checkArgument;
 +
 +import java.nio.charset.StandardCharsets;
  import java.util.ArrayList;
- import java.util.HashMap;
- import java.util.Map;
  
  import org.apache.accumulo.core.Constants;
  import org.apache.accumulo.core.client.AccumuloException;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/server/base/src/main/java/org/apache/accumulo/server/conf/NamespaceConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/server/base/src/main/java/org/apache/accumulo/server/conf/TableConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8990e590/server/base/src/main/java/org/apache/accumulo/server/conf/ZooConfiguration.java
----------------------------------------------------------------------


[19/35] git commit: ACCUMULO-2041 provide alternative name for entries in a WAL

Posted by el...@apache.org.
ACCUMULO-2041 provide alternative name for entries in a WAL


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

Branch: refs/heads/ACCUMULO-378
Commit: a13f788d45db934412bbd45da9650110a5cd1dd0
Parents: 459d304
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Jun 3 14:48:56 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Jun 3 14:48:56 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/tserver/log/LocalWALRecovery.java    | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a13f788d/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java
index 868188a..191e41b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/LocalWALRecovery.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.WritableName;
 import org.apache.hadoop.io.SequenceFile.Reader;
 import org.apache.log4j.Logger;
 
@@ -47,8 +48,14 @@ import com.google.common.annotations.VisibleForTesting;
 /**
  * This class will attempt to rewrite any local WALs to HDFS.
  */
+@SuppressWarnings("deprecation")
 public class LocalWALRecovery implements Runnable {
   private static final Logger log = Logger.getLogger(LocalWALRecovery.class);
+  
+  static { 
+    WritableName.addName(LogFileKey.class,  org.apache.accumulo.server.logger.LogFileKey.class.getName());
+    WritableName.addName(LogFileValue.class,  org.apache.accumulo.server.logger.LogFileValue.class.getName());
+  }
 
   public static void main(String[] args) throws IOException {
     AccumuloConfiguration configuration = SiteConfiguration.getInstance(SiteConfiguration.getDefaultConfiguration());
@@ -145,7 +152,6 @@ public class LocalWALRecovery implements Runnable {
         Path localWal = new Path(file.toURI());
         FileSystem localFs = FileSystem.getLocal(fs.getConf());
         
-        @SuppressWarnings("deprecation")
         Reader reader = new SequenceFile.Reader(localFs, localWal, localFs.getConf());
         // Reader reader = new SequenceFile.Reader(localFs.getConf(), SequenceFile.Reader.file(localWal));
         Path tmp = new Path(options.destination + "/" + name + ".copy");


[11/35] ACCUMULO-2041 extract tablet classes to new files, move tablet-related code to o.a.a.tserver.tablet, make member variables private

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
new file mode 100644
index 0000000..432e3a3
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -0,0 +1,2564 @@
+/*
+ * 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.tserver.tablet;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.impl.ScannerImpl;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.ConfigurationObserver;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.constraints.Violations;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Column;
+import org.apache.accumulo.core.data.ColumnUpdate;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.thrift.IterInfo;
+import org.apache.accumulo.core.data.thrift.MapFileInfo;
+import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.iterators.IterationInterruptedException;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.system.SourceSwitchingIterator;
+import org.apache.accumulo.core.master.thrift.TabletLoadState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
+import org.apache.accumulo.core.util.LocalityGroupUtil;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.conf.TableConfiguration;
+import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManager.FileType;
+import org.apache.accumulo.server.fs.VolumeUtil;
+import org.apache.accumulo.server.fs.VolumeUtil.TabletFiles;
+import org.apache.accumulo.server.master.state.TServerInstance;
+import org.apache.accumulo.server.master.tableOps.CompactionIterators;
+import org.apache.accumulo.server.problems.ProblemReport;
+import org.apache.accumulo.server.problems.ProblemReports;
+import org.apache.accumulo.server.problems.ProblemType;
+import org.apache.accumulo.server.security.SystemCredentials;
+import org.apache.accumulo.server.tablets.TabletTime;
+import org.apache.accumulo.server.tablets.UniqueNameAllocator;
+import org.apache.accumulo.server.util.FileUtil;
+import org.apache.accumulo.server.util.MasterMetadataUtil;
+import org.apache.accumulo.server.util.MetadataTableUtil;
+import org.apache.accumulo.server.util.TabletOperations;
+import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
+import org.apache.accumulo.trace.instrument.Span;
+import org.apache.accumulo.trace.instrument.Trace;
+import org.apache.accumulo.tserver.InMemoryMap;
+import org.apache.accumulo.tserver.MinorCompactionReason;
+import org.apache.accumulo.tserver.TConstraintViolationException;
+import org.apache.accumulo.tserver.TLevel;
+import org.apache.accumulo.tserver.TabletServer;
+import org.apache.accumulo.tserver.TabletServer.TservConstraintEnv;
+import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
+import org.apache.accumulo.tserver.TabletStatsKeeper;
+import org.apache.accumulo.tserver.TabletStatsKeeper.Operation;
+import org.apache.accumulo.tserver.TooManyFilesException;
+import org.apache.accumulo.tserver.compaction.CompactionPlan;
+import org.apache.accumulo.tserver.compaction.CompactionStrategy;
+import org.apache.accumulo.tserver.compaction.DefaultCompactionStrategy;
+import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
+import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
+import org.apache.accumulo.tserver.compaction.WriteParameters;
+import org.apache.accumulo.tserver.constraints.ConstraintChecker;
+import org.apache.accumulo.tserver.log.DfsLogger;
+import org.apache.accumulo.tserver.log.MutationReceiver;
+import org.apache.accumulo.tserver.mastermessage.TabletStatusMessage;
+import org.apache.accumulo.tserver.metrics.TabletServerMinCMetrics;
+import org.apache.accumulo.tserver.tablet.Compactor.CompactionCanceledException;
+import org.apache.accumulo.tserver.tablet.Compactor.CompactionEnv;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+
+/**
+ * 
+ * Provide access to a single row range in a living TabletServer.
+ * 
+ */
+public class Tablet implements TabletCommitter {
+  static final Logger log = Logger.getLogger(Tablet.class);
+  static private final List<LogEntry> NO_LOG_ENTRIES = Collections.emptyList();
+
+  private final TabletServer tabletServer;
+  private final KeyExtent extent;
+  private final TabletResourceManager tabletResources;
+  private final DatafileManager datafileManager;
+  private final TableConfiguration tableConfiguration;
+  private final String tabletDirectory;
+  private final Path location; // absolute path of this tablets dir
+
+  private final TabletMemory tabletMemory;
+  
+  private final TabletTime tabletTime;
+  private final Object timeLock = new Object();
+  private long persistedTime;
+
+  private TServerInstance lastLocation;
+  private volatile boolean tableDirChecked = false;
+
+  private final AtomicLong dataSourceDeletions = new AtomicLong(0);
+  public long getDataSourceDeletions() { return dataSourceDeletions.get(); }
+  private final Set<ScanDataSource> activeScans = new HashSet<ScanDataSource>();
+
+  private volatile boolean closing = false;
+  private boolean closed = false;
+  private boolean closeComplete = false;
+
+  private boolean updatingFlushID = false;
+
+  private long lastFlushID = -1;
+  private long lastCompactID = -1;
+  
+  private volatile boolean majorCompactionInProgress = false;
+  private volatile boolean majorCompactionWaitingToStart = false;
+  private Set<MajorCompactionReason> majorCompactionQueued = Collections.synchronizedSet(EnumSet.noneOf(MajorCompactionReason.class));
+  
+  private volatile boolean minorCompactionInProgress = false;
+  private volatile boolean minorCompactionWaitingToStart = false;
+
+  private final AtomicReference<ConstraintChecker> constraintChecker = new AtomicReference<ConstraintChecker>();
+
+  private int writesInProgress = 0;
+
+  private final TabletStatsKeeper timer = new TabletStatsKeeper();
+
+  private final Rate queryRate = new Rate(0.2);
+  private long queryCount = 0;
+
+  private final Rate queryByteRate = new Rate(0.2);
+  private long queryBytes = 0;
+
+  private final Rate ingestRate = new Rate(0.2);
+  private long ingestCount = 0;
+
+  private final Rate ingestByteRate = new Rate(0.2);
+  private long ingestBytes = 0;
+
+  private byte[] defaultSecurityLabel = new byte[0];
+
+  private long lastMinorCompactionFinishTime = 0;
+  private long lastMapFileImportTime = 0;
+
+  private volatile long numEntries = 0;
+  private volatile long numEntriesInMemory = 0;
+
+  private final Rate scannedRate = new Rate(0.2);
+  private final AtomicLong scannedCount = new AtomicLong(0);
+
+  private final ConfigurationObserver configObserver;
+
+  private final int logId;
+  
+  public int getLogId() {
+    return logId;
+  }
+  
+  public class LookupResult {
+    public List<Range> unfinishedRanges = new ArrayList<Range>();
+    public long bytesAdded = 0;
+    public long dataSize = 0;
+    public boolean closed = false;
+  }
+
+  FileRef getNextMapFilename(String prefix) throws IOException {
+    String extension = FileOperations.getNewFileExtension(this.tableConfiguration);
+    checkTabletDir();
+    return new FileRef(location.toString() + "/" + prefix + UniqueNameAllocator.getInstance().getNextName() + "." + extension);
+  }
+
+  private void checkTabletDir() throws IOException {
+    if (!tableDirChecked) {
+      FileStatus[] files = null;
+      try {
+        files = getTabletServer().getFileSystem().listStatus(this.location);
+      } catch (FileNotFoundException ex) {
+        // ignored
+      }
+      
+      if (files == null) {
+        if (this.location.getName().startsWith("c-"))
+          log.debug("Tablet " + extent + " had no dir, creating " + this.location); // its a clone dir...
+        else
+          log.warn("Tablet " + extent + " had no dir, creating " + this.location);
+      
+        getTabletServer().getFileSystem().mkdirs(this.location);
+      }
+      tableDirChecked = true;
+    }
+  }
+
+  public Tablet(TabletServer tabletServer, KeyExtent extent, TabletResourceManager trm, SplitInfo info) throws IOException {
+    this(tabletServer, new Text(info.dir), extent, trm, info.datafiles, info.time, info.initFlushID, info.initCompactID, info.lastLocation);
+    splitCreationTime = System.currentTimeMillis();
+  }
+
+  private Tablet(TabletServer tabletServer, Text location, KeyExtent extent, TabletResourceManager trm,
+      SortedMap<FileRef,DataFileValue> datafiles, String time, long initFlushID, long initCompactID, TServerInstance lastLocation) throws IOException {
+    this(tabletServer, extent, location, trm, NO_LOG_ENTRIES, datafiles, time, lastLocation, new HashSet<FileRef>(), initFlushID,
+        initCompactID);
+  }
+
+  private static String lookupTime(AccumuloConfiguration conf, KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
+    SortedMap<Key,Value> entries;
+
+    if (extent.isRootTablet()) {
+      return null;
+    } else {
+      entries = new TreeMap<Key,Value>();
+      Text rowName = extent.getMetadataEntry();
+      for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+        if (entry.getKey().compareRow(rowName) == 0 && TabletsSection.ServerColumnFamily.TIME_COLUMN.hasColumns(entry.getKey())) {
+          entries.put(new Key(entry.getKey()), new Value(entry.getValue()));
+        }
+      }
+    }
+
+    if (entries.size() == 1)
+      return entries.values().iterator().next().toString();
+    return null;
+  }
+
+  private static SortedMap<FileRef,DataFileValue> lookupDatafiles(AccumuloConfiguration conf, VolumeManager fs, KeyExtent extent,
+      SortedMap<Key,Value> tabletsKeyValues) throws IOException {
+
+    TreeMap<FileRef,DataFileValue> datafiles = new TreeMap<FileRef,DataFileValue>();
+
+    if (extent.isRootTablet()) { // the meta0 tablet
+      Path location = new Path(MetadataTableUtil.getRootTabletDir());
+
+      // cleanUpFiles() has special handling for delete. files
+      FileStatus[] files = fs.listStatus(location);
+      Collection<String> goodPaths = RootFiles.cleanupReplacement(fs, files, true);
+      for (String good : goodPaths) {
+        Path path = new Path(good);
+        String filename = path.getName();
+        FileRef ref = new FileRef(location.toString() + "/" + filename, path);
+        DataFileValue dfv = new DataFileValue(0, 0);
+        datafiles.put(ref, dfv);
+      }
+    } else {
+
+      Text rowName = extent.getMetadataEntry();
+
+      String tableId = extent.isMeta() ? RootTable.ID : MetadataTable.ID;
+      ScannerImpl mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), SystemCredentials.get(), tableId, Authorizations.EMPTY);
+
+      // Commented out because when no data file is present, each tablet will scan through metadata table and return nothing
+      // reduced batch size to improve performance
+      // changed here after endKeys were implemented from 10 to 1000
+      mdScanner.setBatchSize(1000);
+
+      // leave these in, again, now using endKey for safety
+      mdScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
+
+      mdScanner.setRange(new Range(rowName));
+
+      for (Entry<Key,Value> entry : mdScanner) {
+
+        if (entry.getKey().compareRow(rowName) != 0) {
+          break;
+        }
+
+        FileRef ref = new FileRef(fs, entry.getKey());
+        datafiles.put(ref, new DataFileValue(entry.getValue().get()));
+      }
+    }
+    return datafiles;
+  }
+
+  private static List<LogEntry> lookupLogEntries(KeyExtent ke, SortedMap<Key,Value> tabletsKeyValues) {
+    List<LogEntry> logEntries = new ArrayList<LogEntry>();
+
+    if (ke.isMeta()) {
+      try {
+        logEntries = MetadataTableUtil.getLogEntries(SystemCredentials.get(), ke);
+      } catch (Exception ex) {
+        throw new RuntimeException("Unable to read tablet log entries", ex);
+      }
+    } else {
+      log.debug("Looking at metadata " + tabletsKeyValues);
+      Text row = ke.getMetadataEntry();
+      for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+        Key key = entry.getKey();
+        if (key.getRow().equals(row)) {
+          if (key.getColumnFamily().equals(LogColumnFamily.NAME)) {
+            logEntries.add(LogEntry.fromKeyValue(key, entry.getValue()));
+          }
+        }
+      }
+    }
+
+    log.debug("got " + logEntries + " for logs for " + ke);
+    return logEntries;
+  }
+
+  private static Set<FileRef> lookupScanFiles(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues, VolumeManager fs) throws IOException {
+    HashSet<FileRef> scanFiles = new HashSet<FileRef>();
+
+    Text row = extent.getMetadataEntry();
+    for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+      Key key = entry.getKey();
+      if (key.getRow().equals(row) && key.getColumnFamily().equals(ScanFileColumnFamily.NAME)) {
+        scanFiles.add(new FileRef(fs, key));
+      }
+    }
+
+    return scanFiles;
+  }
+
+  private static long lookupFlushID(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
+    Text row = extent.getMetadataEntry();
+    for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+      Key key = entry.getKey();
+      if (key.getRow().equals(row) && TabletsSection.ServerColumnFamily.FLUSH_COLUMN.equals(key.getColumnFamily(), key.getColumnQualifier()))
+        return Long.parseLong(entry.getValue().toString());
+    }
+
+    return -1;
+  }
+
+  private static long lookupCompactID(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
+    Text row = extent.getMetadataEntry();
+    for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+      Key key = entry.getKey();
+      if (key.getRow().equals(row) && TabletsSection.ServerColumnFamily.COMPACT_COLUMN.equals(key.getColumnFamily(), key.getColumnQualifier()))
+        return Long.parseLong(entry.getValue().toString());
+    }
+
+    return -1;
+  }
+
+  private static TServerInstance lookupLastServer(KeyExtent extent, SortedMap<Key,Value> tabletsKeyValues) {
+    for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
+      if (entry.getKey().getColumnFamily().compareTo(TabletsSection.LastLocationColumnFamily.NAME) == 0) {
+        return new TServerInstance(entry.getValue(), entry.getKey().getColumnQualifier());
+      }
+    }
+    return null;
+  }
+  
+  public Tablet(TabletServer tabletServer, KeyExtent extent, Text location, TabletResourceManager trm, SortedMap<Key,Value> tabletsKeyValues) throws IOException {
+    this(tabletServer, extent, location, trm, lookupLogEntries(extent, tabletsKeyValues), lookupDatafiles(tabletServer.getSystemConfiguration(), tabletServer.getFileSystem(),
+        extent, tabletsKeyValues), lookupTime(tabletServer.getSystemConfiguration(), extent, tabletsKeyValues), lookupLastServer(extent, tabletsKeyValues),
+        lookupScanFiles(extent, tabletsKeyValues, tabletServer.getFileSystem()), lookupFlushID(extent, tabletsKeyValues), lookupCompactID(extent, tabletsKeyValues));
+  }
+
+  /**
+   * yet another constructor - this one allows us to avoid costly lookups into the Metadata table if we already know the files we need - as at split time
+   */
+  private Tablet(final TabletServer tabletServer, final KeyExtent extent, final Text location, final TabletResourceManager trm, final List<LogEntry> rawLogEntries, final SortedMap<FileRef,DataFileValue> rawDatafiles, String time,
+      final TServerInstance lastLocation, Set<FileRef> scanFiles, long initFlushID, long initCompactID) throws IOException {
+
+    TabletFiles tabletPaths = VolumeUtil.updateTabletVolumes(tabletServer.getLock(), tabletServer.getFileSystem(), extent, new TabletFiles(location.toString(), rawLogEntries,
+        rawDatafiles));
+
+    Path locationPath;
+
+    if (tabletPaths.dir.contains(":")) {
+      locationPath = new Path(tabletPaths.dir.toString());
+    } else {
+      locationPath = tabletServer.getFileSystem().getFullPath(FileType.TABLE, extent.getTableId().toString() + tabletPaths.dir.toString());
+    }
+
+    final List<LogEntry> logEntries = tabletPaths.logEntries;
+    final SortedMap<FileRef,DataFileValue> datafiles = tabletPaths.datafiles;
+
+    this.location = locationPath;
+    this.lastLocation = lastLocation;
+    this.tabletDirectory = tabletPaths.dir;
+    this.tableConfiguration = tabletServer.getTableConfiguration(extent);
+
+    this.extent = extent;
+    this.tabletResources = trm;
+
+    this.lastFlushID = initFlushID;
+    this.lastCompactID = initCompactID;
+
+    if (extent.isRootTablet()) {
+      long rtime = Long.MIN_VALUE;
+      for (FileRef ref : datafiles.keySet()) {
+        Path path = ref.path();
+        FileSystem ns = tabletServer.getFileSystem().getVolumeByPath(path).getFileSystem();
+        FileSKVIterator reader = FileOperations.getInstance().openReader(path.toString(), true, ns, ns.getConf(), tabletServer.getTableConfiguration(extent));
+        long maxTime = -1;
+        try {
+
+          while (reader.hasTop()) {
+            maxTime = Math.max(maxTime, reader.getTopKey().getTimestamp());
+            reader.next();
+          }
+
+        } finally {
+          reader.close();
+        }
+
+        if (maxTime > rtime) {
+          time = TabletTime.LOGICAL_TIME_ID + "" + maxTime;
+          rtime = maxTime;
+        }
+      }
+    }
+    if (time == null && datafiles.isEmpty() && extent.equals(RootTable.OLD_EXTENT)) {
+      // recovery... old root tablet has no data, so time doesn't matter:
+      time = TabletTime.LOGICAL_TIME_ID + "" + Long.MIN_VALUE;
+    }
+
+    this.tabletServer = tabletServer;
+    this.logId = tabletServer.createLogId(extent);
+
+    setupDefaultSecurityLabels(extent);
+
+    tabletMemory = new TabletMemory(this);
+    tabletTime = TabletTime.getInstance(time);
+    persistedTime = tabletTime.getTime();
+
+    tableConfiguration.addObserver(configObserver = new ConfigurationObserver() {
+
+      private void reloadConstraints() {
+        constraintChecker.set(new ConstraintChecker(tableConfiguration));
+      }
+
+      @Override
+      public void propertiesChanged() {
+        reloadConstraints();
+
+        try {
+          setupDefaultSecurityLabels(extent);
+        } catch (Exception e) {
+          log.error("Failed to reload default security labels for extent: " + extent.toString());
+        }
+      }
+
+      @Override
+      public void propertyChanged(String prop) {
+        if (prop.startsWith(Property.TABLE_CONSTRAINT_PREFIX.getKey()))
+          reloadConstraints();
+        else if (prop.equals(Property.TABLE_DEFAULT_SCANTIME_VISIBILITY.getKey())) {
+          try {
+            log.info("Default security labels changed for extent: " + extent.toString());
+            setupDefaultSecurityLabels(extent);
+          } catch (Exception e) {
+            log.error("Failed to reload default security labels for extent: " + extent.toString());
+          }
+        }
+
+      }
+
+      @Override
+      public void sessionExpired() {
+        log.debug("Session expired, no longer updating per table props...");
+      }
+
+    });
+
+    tableConfiguration.getNamespaceConfiguration().addObserver(configObserver);
+
+    // Force a load of any per-table properties
+    configObserver.propertiesChanged();
+
+    if (!logEntries.isEmpty()) {
+      log.info("Starting Write-Ahead Log recovery for " + this.extent);
+      final long[] count = new long[2];
+      final CommitSession commitSession = getTabletMemory().getCommitSession();
+      count[1] = Long.MIN_VALUE;
+      try {
+        Set<String> absPaths = new HashSet<String>();
+        for (FileRef ref : datafiles.keySet())
+          absPaths.add(ref.path().toString());
+
+        tabletServer.recover(this.getTabletServer().getFileSystem(), extent, tableConfiguration, logEntries, absPaths, new MutationReceiver() {
+          @Override
+          public void receive(Mutation m) {
+            // LogReader.printMutation(m);
+            Collection<ColumnUpdate> muts = m.getUpdates();
+            for (ColumnUpdate columnUpdate : muts) {
+              if (!columnUpdate.hasTimestamp()) {
+                // if it is not a user set timestamp, it must have been set
+                // by the system
+                count[1] = Math.max(count[1], columnUpdate.getTimestamp());
+              }
+            }
+            getTabletMemory().mutate(commitSession, Collections.singletonList(m));
+            count[0]++;
+          }
+        });
+
+        if (count[1] != Long.MIN_VALUE) {
+          tabletTime.useMaxTimeFromWALog(count[1]);
+        }
+        commitSession.updateMaxCommittedTime(tabletTime.getTime());
+
+        if (count[0] == 0) {
+          MetadataTableUtil.removeUnusedWALEntries(extent, logEntries, tabletServer.getLock());
+          logEntries.clear();
+        }
+
+      } catch (Throwable t) {
+        if (tableConfiguration.getBoolean(Property.TABLE_FAILURES_IGNORE)) {
+          log.warn("Error recovering from log files: ", t);
+        } else {
+          throw new RuntimeException(t);
+        }
+      }
+      // make some closed references that represent the recovered logs
+      currentLogs = new HashSet<DfsLogger>();
+      for (LogEntry logEntry : logEntries) {
+        for (String log : logEntry.logSet) {
+          currentLogs.add(new DfsLogger(tabletServer.getServerConfig(), log));
+        }
+      }
+
+      log.info("Write-Ahead Log recovery complete for " + this.extent + " (" + count[0] + " mutations applied, " + getTabletMemory().getNumEntries()
+          + " entries created)");
+    }
+
+    String contextName = tableConfiguration.get(Property.TABLE_CLASSPATH);
+    if (contextName != null && !contextName.equals("")) {
+      // initialize context classloader, instead of possibly waiting for it to initialize for a scan
+      // TODO this could hang, causing other tablets to fail to load - ACCUMULO-1292
+      AccumuloVFSClassLoader.getContextManager().getClassLoader(contextName);
+    }
+
+    // do this last after tablet is completely setup because it
+    // could cause major compaction to start
+    datafileManager = new DatafileManager(this, datafiles);
+
+    computeNumEntries();
+
+    getDatafileManager().removeFilesAfterScan(scanFiles);
+
+    // look for hints of a failure on the previous tablet server
+    if (!logEntries.isEmpty() || needsMajorCompaction(MajorCompactionReason.NORMAL)) {
+      // look for any temp files hanging around
+      removeOldTemporaryFiles();
+    }
+
+    log.log(TLevel.TABLET_HIST, extent + " opened");
+  }
+
+  private void removeOldTemporaryFiles() {
+    // remove any temporary files created by a previous tablet server
+    try {
+      for (FileStatus tmp : getTabletServer().getFileSystem().globStatus(new Path(location, "*_tmp"))) {
+        try {
+          log.debug("Removing old temp file " + tmp.getPath());
+          getTabletServer().getFileSystem().delete(tmp.getPath());
+        } catch (IOException ex) {
+          log.error("Unable to remove old temp file " + tmp.getPath() + ": " + ex);
+        }
+      }
+    } catch (IOException ex) {
+      log.error("Error scanning for old temp files in " + location);
+    }
+  }
+
+  private void setupDefaultSecurityLabels(KeyExtent extent) {
+    if (extent.isMeta()) {
+      defaultSecurityLabel = new byte[0];
+    } else {
+      try {
+        ColumnVisibility cv = new ColumnVisibility(tableConfiguration.get(Property.TABLE_DEFAULT_SCANTIME_VISIBILITY));
+        this.defaultSecurityLabel = cv.getExpression();
+      } catch (Exception e) {
+        log.error(e, e);
+        this.defaultSecurityLabel = new byte[0];
+      }
+    }
+  }
+
+  private LookupResult lookup(SortedKeyValueIterator<Key,Value> mmfi, List<Range> ranges, HashSet<Column> columnSet, ArrayList<KVEntry> results,
+      long maxResultsSize) throws IOException {
+
+    LookupResult lookupResult = new LookupResult();
+
+    boolean exceededMemoryUsage = false;
+    boolean tabletClosed = false;
+
+    Set<ByteSequence> cfset = null;
+    if (columnSet.size() > 0)
+      cfset = LocalityGroupUtil.families(columnSet);
+
+    for (Range range : ranges) {
+
+      if (exceededMemoryUsage || tabletClosed) {
+        lookupResult.unfinishedRanges.add(range);
+        continue;
+      }
+
+      int entriesAdded = 0;
+
+      try {
+        if (cfset != null)
+          mmfi.seek(range, cfset, true);
+        else
+          mmfi.seek(range, LocalityGroupUtil.EMPTY_CF_SET, false);
+
+        while (mmfi.hasTop()) {
+          Key key = mmfi.getTopKey();
+
+          KVEntry kve = new KVEntry(key, mmfi.getTopValue());
+          results.add(kve);
+          entriesAdded++;
+          lookupResult.bytesAdded += kve.estimateMemoryUsed();
+          lookupResult.dataSize += kve.numBytes();
+
+          exceededMemoryUsage = lookupResult.bytesAdded > maxResultsSize;
+
+          if (exceededMemoryUsage) {
+            addUnfinishedRange(lookupResult, range, key, false);
+            break;
+          }
+
+          mmfi.next();
+        }
+
+      } catch (TooManyFilesException tmfe) {
+        // treat this as a closed tablet, and let the client retry
+        log.warn("Tablet " + getExtent() + " has too many files, batch lookup can not run");
+        handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
+        tabletClosed = true;
+      } catch (IOException ioe) {
+        if (shutdownInProgress()) {
+          // assume HDFS shutdown hook caused this exception
+          log.debug("IOException while shutdown in progress ", ioe);
+          handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
+          tabletClosed = true;
+        } else {
+          throw ioe;
+        }
+      } catch (IterationInterruptedException iie) {
+        if (isClosed()) {
+          handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
+          tabletClosed = true;
+        } else {
+          throw iie;
+        }
+      } catch (TabletClosedException tce) {
+        handleTabletClosedDuringScan(results, lookupResult, exceededMemoryUsage, range, entriesAdded);
+        tabletClosed = true;
+      }
+
+    }
+
+    return lookupResult;
+  }
+
+  private void handleTabletClosedDuringScan(ArrayList<KVEntry> results, LookupResult lookupResult, boolean exceededMemoryUsage, Range range, int entriesAdded) {
+    if (exceededMemoryUsage)
+      throw new IllegalStateException("tablet should not exceed memory usage or close, not both");
+
+    if (entriesAdded > 0)
+      addUnfinishedRange(lookupResult, range, results.get(results.size() - 1).getKey(), false);
+    else
+      lookupResult.unfinishedRanges.add(range);
+
+    lookupResult.closed = true;
+  }
+
+  private void addUnfinishedRange(LookupResult lookupResult, Range range, Key key, boolean inclusiveStartKey) {
+    if (range.getEndKey() == null || key.compareTo(range.getEndKey()) < 0) {
+      Range nlur = new Range(new Key(key), inclusiveStartKey, range.getEndKey(), range.isEndKeyInclusive());
+      lookupResult.unfinishedRanges.add(nlur);
+    }
+  }
+
+  public static interface KVReceiver {
+    void receive(List<KVEntry> matches) throws IOException;
+  }
+
+  public LookupResult lookup(List<Range> ranges, HashSet<Column> columns, Authorizations authorizations, ArrayList<KVEntry> results, long maxResultSize,
+      List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, AtomicBoolean interruptFlag) throws IOException {
+
+    if (ranges.size() == 0) {
+      return new LookupResult();
+    }
+
+    ranges = Range.mergeOverlapping(ranges);
+    Collections.sort(ranges);
+
+    Range tabletRange = extent.toDataRange();
+    for (Range range : ranges) {
+      // do a test to see if this range falls within the tablet, if it does not
+      // then clip will throw an exception
+      tabletRange.clip(range);
+    }
+
+    ScanDataSource dataSource = new ScanDataSource(this, authorizations, this.defaultSecurityLabel, columns, ssiList, ssio, interruptFlag);
+
+    LookupResult result = null;
+
+    try {
+      SortedKeyValueIterator<Key,Value> iter = new SourceSwitchingIterator(dataSource);
+      result = lookup(iter, ranges, columns, results, maxResultSize);
+      return result;
+    } catch (IOException ioe) {
+      dataSource.close(true);
+      throw ioe;
+    } finally {
+      // code in finally block because always want
+      // to return mapfiles, even when exception is thrown
+      dataSource.close(false);
+
+      synchronized (this) {
+        queryCount += results.size();
+        if (result != null)
+          queryBytes += result.dataSize;
+      }
+    }
+  }
+
+  Batch nextBatch(SortedKeyValueIterator<Key,Value> iter, Range range, int num, Set<Column> columns) throws IOException {
+
+    // log.info("In nextBatch..");
+
+    List<KVEntry> results = new ArrayList<KVEntry>();
+    Key key = null;
+
+    Value value;
+    long resultSize = 0L;
+    long resultBytes = 0L;
+
+    long maxResultsSize = tableConfiguration.getMemoryInBytes(Property.TABLE_SCAN_MAXMEM);
+
+    if (columns.size() == 0) {
+      iter.seek(range, LocalityGroupUtil.EMPTY_CF_SET, false);
+    } else {
+      iter.seek(range, LocalityGroupUtil.families(columns), true);
+    }
+
+    Key continueKey = null;
+    boolean skipContinueKey = false;
+
+    boolean endOfTabletReached = false;
+    while (iter.hasTop()) {
+
+      value = iter.getTopValue();
+      key = iter.getTopKey();
+
+      KVEntry kvEntry = new KVEntry(key, value); // copies key and value
+      results.add(kvEntry);
+      resultSize += kvEntry.estimateMemoryUsed();
+      resultBytes += kvEntry.numBytes();
+
+      if (resultSize >= maxResultsSize || results.size() >= num) {
+        continueKey = new Key(key);
+        skipContinueKey = true;
+        break;
+      }
+
+      iter.next();
+    }
+
+    if (iter.hasTop() == false) {
+      endOfTabletReached = true;
+    }
+
+    if (endOfTabletReached) {
+      continueKey = null;
+    }
+
+    if (endOfTabletReached && results.size() == 0)
+      results = null;
+
+    return new Batch(skipContinueKey, results, continueKey, resultBytes);
+  }
+
+  /**
+   * Determine if a JVM shutdown is in progress.
+   * 
+   */
+  boolean shutdownInProgress() {
+    try {
+      Runtime.getRuntime().removeShutdownHook(new Thread(new Runnable() {
+        @Override
+        public void run() {}
+      }));
+    } catch (IllegalStateException ise) {
+      return true;
+    }
+
+    return false;
+  }
+
+  public Scanner createScanner(Range range, int num, Set<Column> columns, Authorizations authorizations, List<IterInfo> ssiList, Map<String,Map<String,String>> ssio,
+      boolean isolated, AtomicBoolean interruptFlag) {
+    // do a test to see if this range falls within the tablet, if it does not
+    // then clip will throw an exception
+    extent.toDataRange().clip(range);
+
+    ScanOptions opts = new ScanOptions(num, authorizations, this.defaultSecurityLabel, columns, ssiList, ssio, interruptFlag, isolated);
+    return new Scanner(this, range, opts);
+  }
+
+  DataFileValue minorCompact(VolumeManager fs, InMemoryMap memTable, FileRef tmpDatafile, FileRef newDatafile, FileRef mergeFile,
+      boolean hasQueueTime, long queued, CommitSession commitSession, long flushId, MinorCompactionReason mincReason) {
+    boolean failed = false;
+    long start = System.currentTimeMillis();
+    timer.incrementStatusMinor();
+
+    long count = 0;
+
+    try {
+      Span span = Trace.start("write");
+      CompactionStats stats;
+      try {
+        count = memTable.getNumEntries();
+
+        DataFileValue dfv = null;
+        if (mergeFile != null)
+          dfv = getDatafileManager().getDatafileSizes().get(mergeFile);
+
+        MinorCompactor compactor = new MinorCompactor(fs, memTable, mergeFile, dfv, tmpDatafile, tableConfiguration, extent, mincReason);
+        stats = compactor.call();
+      } finally {
+        span.stop();
+      }
+      span = Trace.start("bringOnline");
+      try {
+        getDatafileManager().bringMinorCompactionOnline(tmpDatafile, newDatafile, mergeFile, new DataFileValue(stats.getFileSize(), stats.getEntriesWritten()),
+            commitSession, flushId);
+      } finally {
+        span.stop();
+      }
+      return new DataFileValue(stats.getFileSize(), stats.getEntriesWritten());
+    } catch (Exception E) {
+      failed = true;
+      throw new RuntimeException(E);
+    } catch (Error E) {
+      // Weird errors like "OutOfMemoryError" when trying to create the thread for the compaction
+      failed = true;
+      throw new RuntimeException(E);
+    } finally {
+      try {
+        getTabletMemory().finalizeMinC();
+      } catch (Throwable t) {
+        log.error("Failed to free tablet memory", t);
+      }
+
+      if (!failed) {
+        lastMinorCompactionFinishTime = System.currentTimeMillis();
+      }
+      TabletServerMinCMetrics minCMetrics = getTabletServer().getMinCMetrics();
+      if (minCMetrics.isEnabled())
+        minCMetrics.add(TabletServerMinCMetrics.minc, (lastMinorCompactionFinishTime - start));
+      if (hasQueueTime) {
+        timer.updateTime(Operation.MINOR, queued, start, count, failed);
+        if (minCMetrics.isEnabled())
+          minCMetrics.add(TabletServerMinCMetrics.queue, (start - queued));
+      } else
+        timer.updateTime(Operation.MINOR, start, count, failed);
+    }
+  }
+
+  private synchronized MinorCompactionTask prepareForMinC(long flushId, MinorCompactionReason mincReason) {
+    CommitSession oldCommitSession = getTabletMemory().prepareForMinC();
+    otherLogs = currentLogs;
+    currentLogs = new HashSet<DfsLogger>();
+
+    FileRef mergeFile = getDatafileManager().reserveMergingMinorCompactionFile();
+
+    return new MinorCompactionTask(this, mergeFile, oldCommitSession, flushId, mincReason);
+
+  }
+
+  public void flush(long tableFlushID) {
+    boolean updateMetadata = false;
+    boolean initiateMinor = false;
+
+    try {
+
+      synchronized (this) {
+
+        // only want one thing at a time to update flush ID to ensure that metadata table and tablet in memory state are consistent
+        if (updatingFlushID)
+          return;
+
+        if (lastFlushID >= tableFlushID)
+          return;
+
+        if (closing || closed || getTabletMemory().memoryReservedForMinC())
+          return;
+
+        if (getTabletMemory().getMemTable().getNumEntries() == 0) {
+          lastFlushID = tableFlushID;
+          updatingFlushID = true;
+          updateMetadata = true;
+        } else
+          initiateMinor = true;
+      }
+
+      if (updateMetadata) {
+        Credentials creds = SystemCredentials.get();
+        // if multiple threads were allowed to update this outside of a sync block, then it would be
+        // a race condition
+        MetadataTableUtil.updateTabletFlushID(extent, tableFlushID, creds, getTabletServer().getLock());
+      } else if (initiateMinor)
+        initiateMinorCompaction(tableFlushID, MinorCompactionReason.USER);
+
+    } finally {
+      if (updateMetadata) {
+        synchronized (this) {
+          updatingFlushID = false;
+          this.notifyAll();
+        }
+      }
+    }
+
+  }
+
+  public boolean initiateMinorCompaction(MinorCompactionReason mincReason) {
+    if (isClosed()) {
+      // don't bother trying to get flush id if closed... could be closed after this check but that is ok... just trying to cut down on uneeded log messages....
+      return false;
+    }
+
+    // get the flush id before the new memmap is made available for write
+    long flushId;
+    try {
+      flushId = getFlushID();
+    } catch (NoNodeException e) {
+      log.info("Asked to initiate MinC when there was no flush id " + getExtent() + " " + e.getMessage());
+      return false;
+    }
+    return initiateMinorCompaction(flushId, mincReason);
+  }
+
+  public boolean minorCompactNow(MinorCompactionReason mincReason) {
+    long flushId;
+    try {
+      flushId = getFlushID();
+    } catch (NoNodeException e) {
+      log.info("Asked to initiate MinC when there was no flush id " + getExtent() + " " + e.getMessage());
+      return false;
+    }
+    MinorCompactionTask mct = createMinorCompactionTask(flushId, mincReason);
+    if (mct == null)
+      return false;
+    mct.run();
+    return true;
+  }
+
+  boolean initiateMinorCompaction(long flushId, MinorCompactionReason mincReason) {
+    MinorCompactionTask mct = createMinorCompactionTask(flushId, mincReason);
+    if (mct == null)
+      return false;
+    getTabletResources().executeMinorCompaction(mct);
+    return true;
+  }
+
+  private MinorCompactionTask createMinorCompactionTask(long flushId, MinorCompactionReason mincReason) {
+    MinorCompactionTask mct;
+    long t1, t2;
+
+    StringBuilder logMessage = null;
+
+    try {
+      synchronized (this) {
+        t1 = System.currentTimeMillis();
+
+        if (closing || closed || majorCompactionWaitingToStart || getTabletMemory().memoryReservedForMinC() || getTabletMemory().getMemTable().getNumEntries() == 0
+            || updatingFlushID) {
+
+          logMessage = new StringBuilder();
+
+          logMessage.append(extent.toString());
+          logMessage.append(" closing " + closing);
+          logMessage.append(" closed " + closed);
+          logMessage.append(" majorCompactionWaitingToStart " + majorCompactionWaitingToStart);
+          if (getTabletMemory() != null)
+            logMessage.append(" tabletMemory.memoryReservedForMinC() " + getTabletMemory().memoryReservedForMinC());
+          if (getTabletMemory() != null && getTabletMemory().getMemTable() != null)
+            logMessage.append(" tabletMemory.getMemTable().getNumEntries() " + getTabletMemory().getMemTable().getNumEntries());
+          logMessage.append(" updatingFlushID " + updatingFlushID);
+
+          return null;
+        }
+        // We're still recovering log entries
+        if (getDatafileManager() == null) {
+          logMessage = new StringBuilder();
+          logMessage.append(extent.toString());
+          logMessage.append(" datafileManager " + getDatafileManager());
+          return null;
+        }
+
+        mct = prepareForMinC(flushId, mincReason);
+        t2 = System.currentTimeMillis();
+      }
+    } finally {
+      // log outside of sync block
+      if (logMessage != null && log.isDebugEnabled())
+        log.debug(logMessage);
+    }
+
+    log.debug(String.format("MinC initiate lock %.2f secs", (t2 - t1) / 1000.0));
+    return mct;
+  }
+
+  public long getFlushID() throws NoNodeException {
+    try {
+      String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
+          + Constants.ZTABLE_FLUSH_ID;
+      return Long.parseLong(new String(ZooReaderWriter.getRetryingInstance().getData(zTablePath, null), StandardCharsets.UTF_8));
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    } catch (NumberFormatException nfe) {
+      throw new RuntimeException(nfe);
+    } catch (KeeperException ke) {
+      if (ke instanceof NoNodeException) {
+        throw (NoNodeException) ke;
+      } else {
+        throw new RuntimeException(ke);
+      }
+    }
+  }
+
+  long getCompactionCancelID() {
+    String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
+        + Constants.ZTABLE_COMPACT_CANCEL_ID;
+
+    try {
+      return Long.parseLong(new String(ZooReaderWriter.getRetryingInstance().getData(zTablePath, null), StandardCharsets.UTF_8));
+    } catch (KeeperException e) {
+      throw new RuntimeException(e);
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public Pair<Long,List<IteratorSetting>> getCompactionID() throws NoNodeException {
+    try {
+      String zTablePath = Constants.ZROOT + "/" + HdfsZooInstance.getInstance().getInstanceID() + Constants.ZTABLES + "/" + extent.getTableId()
+          + Constants.ZTABLE_COMPACT_ID;
+
+      String[] tokens = new String(ZooReaderWriter.getRetryingInstance().getData(zTablePath, null), StandardCharsets.UTF_8).split(",");
+      long compactID = Long.parseLong(tokens[0]);
+
+      CompactionIterators iters = new CompactionIterators();
+
+      if (tokens.length > 1) {
+        Hex hex = new Hex();
+        ByteArrayInputStream bais = new ByteArrayInputStream(hex.decode(tokens[1].split("=")[1].getBytes(StandardCharsets.UTF_8)));
+        DataInputStream dis = new DataInputStream(bais);
+
+        try {
+          iters.readFields(dis);
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+
+        KeyExtent ke = new KeyExtent(extent.getTableId(), iters.getEndRow(), iters.getStartRow());
+
+        if (!ke.overlaps(extent)) {
+          // only use iterators if compaction range overlaps
+          iters = new CompactionIterators();
+        }
+      }
+
+      return new Pair<Long,List<IteratorSetting>>(compactID, iters.getIterators());
+    } catch (InterruptedException e) {
+      throw new RuntimeException(e);
+    } catch (NumberFormatException nfe) {
+      throw new RuntimeException(nfe);
+    } catch (KeeperException ke) {
+      if (ke instanceof NoNodeException) {
+        throw (NoNodeException) ke;
+      } else {
+        throw new RuntimeException(ke);
+      }
+    } catch (DecoderException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public synchronized void waitForMinC() {
+    getTabletMemory().waitForMinC();
+  }
+
+  private synchronized CommitSession finishPreparingMutations(long time) {
+    if (writesInProgress < 0) {
+      throw new IllegalStateException("waitingForLogs < 0 " + writesInProgress);
+    }
+
+    if (closed || getTabletMemory() == null) {
+      // log.debug("tablet closed, can't commit");
+      return null;
+    }
+
+    writesInProgress++;
+    CommitSession commitSession = getTabletMemory().getCommitSession();
+    commitSession.incrementCommitsInProgress();
+    commitSession.updateMaxCommittedTime(time);
+    return commitSession;
+  }
+
+  public void checkConstraints() {
+    ConstraintChecker cc = constraintChecker.get();
+
+    if (cc.classLoaderChanged()) {
+      ConstraintChecker ncc = new ConstraintChecker(tableConfiguration);
+      constraintChecker.compareAndSet(cc, ncc);
+    }
+  }
+
+  public CommitSession prepareMutationsForCommit(TservConstraintEnv cenv, List<Mutation> mutations) throws TConstraintViolationException {
+
+    ConstraintChecker cc = constraintChecker.get();
+
+    List<Mutation> violators = null;
+    Violations violations = new Violations();
+    cenv.setExtent(extent);
+    for (Mutation mutation : mutations) {
+      Violations more = cc.check(cenv, mutation);
+      if (more != null) {
+        violations.add(more);
+        if (violators == null)
+          violators = new ArrayList<Mutation>();
+        violators.add(mutation);
+      }
+    }
+
+    long time = tabletTime.setUpdateTimes(mutations);
+
+    if (!violations.isEmpty()) {
+
+      HashSet<Mutation> violatorsSet = new HashSet<Mutation>(violators);
+      ArrayList<Mutation> nonViolators = new ArrayList<Mutation>();
+
+      for (Mutation mutation : mutations) {
+        if (!violatorsSet.contains(mutation)) {
+          nonViolators.add(mutation);
+        }
+      }
+
+      CommitSession commitSession = null;
+
+      if (nonViolators.size() > 0) {
+        // if everything is a violation, then it is expected that
+        // code calling this will not log or commit
+        commitSession = finishPreparingMutations(time);
+        if (commitSession == null)
+          return null;
+      }
+
+      throw new TConstraintViolationException(violations, violators, nonViolators, commitSession);
+    }
+
+    return finishPreparingMutations(time);
+  }
+
+  public synchronized void abortCommit(CommitSession commitSession, List<Mutation> value) {
+    if (writesInProgress <= 0) {
+      throw new IllegalStateException("waitingForLogs <= 0 " + writesInProgress);
+    }
+
+    if (closeComplete || getTabletMemory() == null) {
+      throw new IllegalStateException("aborting commit when tablet is closed");
+    }
+
+    commitSession.decrementCommitsInProgress();
+    writesInProgress--;
+    if (writesInProgress == 0)
+      this.notifyAll();
+  }
+
+  public void commit(CommitSession commitSession, List<Mutation> mutations) {
+
+    int totalCount = 0;
+    long totalBytes = 0;
+
+    // write the mutation to the in memory table
+    for (Mutation mutation : mutations) {
+      totalCount += mutation.size();
+      totalBytes += mutation.numBytes();
+    }
+
+    getTabletMemory().mutate(commitSession, mutations);
+
+    synchronized (this) {
+      if (writesInProgress < 1) {
+        throw new IllegalStateException("commiting mutations after logging, but not waiting for any log messages");
+      }
+
+      if (closed && closeComplete) {
+        throw new IllegalStateException("tablet closed with outstanding messages to the logger");
+      }
+
+      getTabletMemory().updateMemoryUsageStats();
+
+      // decrement here in case an exception is thrown below
+      writesInProgress--;
+      if (writesInProgress == 0)
+        this.notifyAll();
+
+      commitSession.decrementCommitsInProgress();
+
+      numEntries += totalCount;
+      numEntriesInMemory += totalCount;
+      ingestCount += totalCount;
+      ingestBytes += totalBytes;
+    }
+  }
+
+  /**
+   * Closes the mapfiles associated with a Tablet. If saveState is true, a minor compaction is performed.
+   */
+  public void close(boolean saveState) throws IOException {
+    initiateClose(saveState, false, false);
+    completeClose(saveState, true);
+  }
+
+  void initiateClose(boolean saveState, boolean queueMinC, boolean disableWrites) {
+
+    if (!saveState && queueMinC) {
+      throw new IllegalArgumentException("Not saving state on close and requesting minor compactions queue does not make sense");
+    }
+
+    log.debug("initiateClose(saveState=" + saveState + " queueMinC=" + queueMinC + " disableWrites=" + disableWrites + ") " + getExtent());
+
+    MinorCompactionTask mct = null;
+
+    synchronized (this) {
+      if (closed || closing || closeComplete) {
+        String msg = "Tablet " + getExtent() + " already";
+        if (closed)
+          msg += " closed";
+        if (closing)
+          msg += " closing";
+        if (closeComplete)
+          msg += " closeComplete";
+        throw new IllegalStateException(msg);
+      }
+
+      // enter the closing state, no splits, minor, or major compactions can start
+      // should cause running major compactions to stop
+      closing = true;
+      this.notifyAll();
+
+      // determines if inserts and queries can still continue while minor compacting
+      closed = disableWrites;
+
+      // wait for major compactions to finish, setting closing to
+      // true should cause any running major compactions to abort
+      while (majorCompactionInProgress) {
+        try {
+          this.wait(50);
+        } catch (InterruptedException e) {
+          log.error(e.toString());
+        }
+      }
+
+      while (updatingFlushID) {
+        try {
+          this.wait(50);
+        } catch (InterruptedException e) {
+          log.error(e.toString());
+        }
+      }
+
+      if (!saveState || getTabletMemory().getMemTable().getNumEntries() == 0) {
+        return;
+      }
+
+      getTabletMemory().waitForMinC();
+
+      try {
+        mct = prepareForMinC(getFlushID(), MinorCompactionReason.CLOSE);
+      } catch (NoNodeException e) {
+        throw new RuntimeException(e);
+      }
+
+      if (queueMinC) {
+        getTabletResources().executeMinorCompaction(mct);
+        return;
+      }
+
+    }
+
+    // do minor compaction outside of synch block so that tablet can be read and written to while
+    // compaction runs
+    mct.run();
+  }
+
+  private boolean closeCompleting = false;
+
+  synchronized void completeClose(boolean saveState, boolean completeClose) throws IOException {
+
+    if (!closing || closeComplete || closeCompleting) {
+      throw new IllegalStateException("closing = " + closing + " closed = " + closed + " closeComplete = " + closeComplete + " closeCompleting = "
+          + closeCompleting);
+    }
+
+    log.debug("completeClose(saveState=" + saveState + " completeClose=" + completeClose + ") " + getExtent());
+
+    // ensure this method is only called once, also guards against multiple
+    // threads entering the method at the same time
+    closeCompleting = true;
+    closed = true;
+
+    // modify dataSourceDeletions so scans will try to switch data sources and fail because the tablet is closed
+    dataSourceDeletions.incrementAndGet();
+
+    for (ScanDataSource activeScan : activeScans) {
+      activeScan.interrupt();
+    }
+
+    // wait for reads and writes to complete
+    while (writesInProgress > 0 || activeScans.size() > 0) {
+      try {
+        this.wait(50);
+      } catch (InterruptedException e) {
+        log.error(e.toString());
+      }
+    }
+
+    getTabletMemory().waitForMinC();
+
+    if (saveState && getTabletMemory().getMemTable().getNumEntries() > 0) {
+      try {
+        prepareForMinC(getFlushID(), MinorCompactionReason.CLOSE).run();
+      } catch (NoNodeException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    if (saveState) {
+      // at this point all tablet data is flushed, so do a consistency check
+      RuntimeException err = null;
+      for (int i = 0; i < 5; i++) {
+        try {
+          closeConsistencyCheck();
+          err = null;
+        } catch (RuntimeException t) {
+          err = t;
+          log.error("Consistency check fails, retrying " + t);
+          UtilWaitThread.sleep(500);
+        }
+      }
+      if (err != null) {
+        ProblemReports.getInstance().report(new ProblemReport(extent.getTableId().toString(), ProblemType.TABLET_LOAD, this.extent.toString(), err));
+        log.error("Tablet closed consistency check has failed for " + this.extent + " giving up and closing");
+      }
+    }
+
+    try {
+      getTabletMemory().getMemTable().delete(0);
+    } catch (Throwable t) {
+      log.error("Failed to delete mem table : " + t.getMessage(), t);
+    }
+
+    getTabletMemory().close();
+
+    // close map files
+    getTabletResources().close();
+
+    log.log(TLevel.TABLET_HIST, extent + " closed");
+
+    tableConfiguration.getNamespaceConfiguration().removeObserver(configObserver);
+    tableConfiguration.removeObserver(configObserver);
+
+    closeComplete = completeClose;
+  }
+
+  private void closeConsistencyCheck() {
+
+    if (getTabletMemory().getMemTable().getNumEntries() != 0) {
+      String msg = "Closed tablet " + extent + " has " + getTabletMemory().getMemTable().getNumEntries() + " entries in memory";
+      log.error(msg);
+      throw new RuntimeException(msg);
+    }
+
+    if (getTabletMemory().memoryReservedForMinC()) {
+      String msg = "Closed tablet " + extent + " has minor compacting memory";
+      log.error(msg);
+      throw new RuntimeException(msg);
+    }
+
+    try {
+      Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>> fileLog = MetadataTableUtil.getFileAndLogEntries(SystemCredentials.get(), extent);
+
+      if (fileLog.getFirst().size() != 0) {
+        String msg = "Closed tablet " + extent + " has walog entries in " + MetadataTable.NAME + " " + fileLog.getFirst();
+        log.error(msg);
+        throw new RuntimeException(msg);
+      }
+
+      if (extent.isRootTablet()) {
+        if (!fileLog.getSecond().keySet().equals(getDatafileManager().getDatafileSizes().keySet())) {
+          String msg = "Data file in " + RootTable.NAME + " differ from in memory data " + extent + "  " + fileLog.getSecond().keySet() + "  "
+              + getDatafileManager().getDatafileSizes().keySet();
+          log.error(msg);
+          throw new RuntimeException(msg);
+        }
+      } else {
+        if (!fileLog.getSecond().equals(getDatafileManager().getDatafileSizes())) {
+          String msg = "Data file in " + MetadataTable.NAME + " differ from in memory data " + extent + "  " + fileLog.getSecond() + "  "
+              + getDatafileManager().getDatafileSizes();
+          log.error(msg);
+          throw new RuntimeException(msg);
+        }
+      }
+
+    } catch (Exception e) {
+      String msg = "Failed to do close consistency check for tablet " + extent;
+      log.error(msg, e);
+      throw new RuntimeException(msg, e);
+
+    }
+
+    if (otherLogs.size() != 0 || currentLogs.size() != 0) {
+      String msg = "Closed tablet " + extent + " has walog entries in memory currentLogs = " + currentLogs + "  otherLogs = " + otherLogs;
+      log.error(msg);
+      throw new RuntimeException(msg);
+    }
+
+    // TODO check lastFlushID and lostCompactID - ACCUMULO-1290
+  }
+
+  /**
+   * Returns a Path object representing the tablet's location on the DFS.
+   * 
+   * @return location
+   */
+  public Path getLocation() {
+    return location;
+  }
+
+  public synchronized boolean initiateMajorCompaction(MajorCompactionReason reason) {
+
+    if (closing || closed || !needsMajorCompaction(reason) || majorCompactionInProgress || majorCompactionQueued.contains(reason)) {
+      return false;
+    }
+
+    majorCompactionQueued.add(reason);
+
+    getTabletResources().executeMajorCompaction(getExtent(), new CompactionRunner(this, reason));
+
+    return false;
+  }
+
+  /**
+   * Returns true if a major compaction should be performed on the tablet.
+   * 
+   */
+  public boolean needsMajorCompaction(MajorCompactionReason reason) {
+    if (majorCompactionInProgress)
+      return false;
+    if (reason == MajorCompactionReason.CHOP || reason == MajorCompactionReason.USER)
+      return true;
+    return getTabletResources().needsMajorCompaction(getDatafileManager().getDatafileSizes(), reason);
+  }
+
+  /**
+   * Returns an int representing the total block size of the f served by this tablet.
+   * 
+   * @return size
+   */
+  // this is the size of just the files
+  public long estimateTabletSize() {
+    long size = 0L;
+
+    for (DataFileValue sz : getDatafileManager().getDatafileSizes().values())
+      size += sz.getSize();
+
+    return size;
+  }
+
+  private boolean sawBigRow = false;
+  private long timeOfLastMinCWhenBigFreakinRowWasSeen = 0;
+  private long timeOfLastImportWhenBigFreakinRowWasSeen = 0;
+  private long splitCreationTime;
+
+  private SplitRowSpec findSplitRow(Collection<FileRef> files) {
+
+    // never split the root tablet
+    // check if we already decided that we can never split
+    // check to see if we're big enough to split
+
+    long splitThreshold = tableConfiguration.getMemoryInBytes(Property.TABLE_SPLIT_THRESHOLD);
+    if (extent.isRootTablet() || estimateTabletSize() <= splitThreshold) {
+      return null;
+    }
+
+    // have seen a big row before, do not bother checking unless a minor compaction or map file import has occurred.
+    if (sawBigRow) {
+      if (timeOfLastMinCWhenBigFreakinRowWasSeen != lastMinorCompactionFinishTime || timeOfLastImportWhenBigFreakinRowWasSeen != lastMapFileImportTime) {
+        // a minor compaction or map file import has occurred... check again
+        sawBigRow = false;
+      } else {
+        // nothing changed, do not split
+        return null;
+      }
+    }
+
+    SortedMap<Double,Key> keys = null;
+
+    try {
+      // we should make .25 below configurable
+      keys = FileUtil.findMidPoint(getTabletServer().getFileSystem(), getTabletServer().getSystemConfiguration(), extent.getPrevEndRow(), extent.getEndRow(), FileUtil.toPathStrings(files), .25);
+    } catch (IOException e) {
+      log.error("Failed to find midpoint " + e.getMessage());
+      return null;
+    }
+
+    // check to see if one row takes up most of the tablet, in which case we can not split
+    try {
+
+      Text lastRow;
+      if (extent.getEndRow() == null) {
+        Key lastKey = (Key) FileUtil.findLastKey(getTabletServer().getFileSystem(), getTabletServer().getSystemConfiguration(), files);
+        lastRow = lastKey.getRow();
+      } else {
+        lastRow = extent.getEndRow();
+      }
+
+      // check to see that the midPoint is not equal to the end key
+      if (keys.get(.5).compareRow(lastRow) == 0) {
+        if (keys.firstKey() < .5) {
+          Key candidate = keys.get(keys.firstKey());
+          if (candidate.compareRow(lastRow) != 0) {
+            // we should use this ratio in split size estimations
+            if (log.isTraceEnabled())
+              log.trace(String.format("Splitting at %6.2f instead of .5, row at .5 is same as end row%n", keys.firstKey()));
+            return new SplitRowSpec(keys.firstKey(), candidate.getRow());
+          }
+
+        }
+
+        log.warn("Cannot split tablet " + extent + " it contains a big row : " + lastRow);
+
+        sawBigRow = true;
+        timeOfLastMinCWhenBigFreakinRowWasSeen = lastMinorCompactionFinishTime;
+        timeOfLastImportWhenBigFreakinRowWasSeen = lastMapFileImportTime;
+
+        return null;
+      }
+      Key mid = keys.get(.5);
+      Text text = (mid == null) ? null : mid.getRow();
+      SortedMap<Double,Key> firstHalf = keys.headMap(.5);
+      if (firstHalf.size() > 0) {
+        Text beforeMid = firstHalf.get(firstHalf.lastKey()).getRow();
+        Text shorter = new Text();
+        int trunc = longestCommonLength(text, beforeMid);
+        shorter.set(text.getBytes(), 0, Math.min(text.getLength(), trunc + 1));
+        text = shorter;
+      }
+      return new SplitRowSpec(.5, text);
+    } catch (IOException e) {
+      // don't split now, but check again later
+      log.error("Failed to find lastkey " + e.getMessage());
+      return null;
+    }
+  }
+
+  private static int longestCommonLength(Text text, Text beforeMid) {
+    int common = 0;
+    while (common < text.getLength() && common < beforeMid.getLength() && text.getBytes()[common] == beforeMid.getBytes()[common]) {
+      common++;
+    }
+    return common;
+  }
+
+  private Map<FileRef,Pair<Key,Key>> getFirstAndLastKeys(SortedMap<FileRef,DataFileValue> allFiles) throws IOException {
+    Map<FileRef,Pair<Key,Key>> result = new HashMap<FileRef,Pair<Key,Key>>();
+    FileOperations fileFactory = FileOperations.getInstance();
+    VolumeManager fs = getTabletServer().getFileSystem();
+    for (Entry<FileRef,DataFileValue> entry : allFiles.entrySet()) {
+      FileRef file = entry.getKey();
+      FileSystem ns = fs.getVolumeByPath(file.path()).getFileSystem();
+      FileSKVIterator openReader = fileFactory.openReader(file.path().toString(), true, ns, ns.getConf(), this.getTableConfiguration());
+      try {
+        Key first = openReader.getFirstKey();
+        Key last = openReader.getLastKey();
+        result.put(file, new Pair<Key,Key>(first, last));
+      } finally {
+        openReader.close();
+      }
+    }
+    return result;
+  }
+
+  List<FileRef> findChopFiles(KeyExtent extent, Map<FileRef,Pair<Key,Key>> firstAndLastKeys, Collection<FileRef> allFiles) throws IOException {
+    List<FileRef> result = new ArrayList<FileRef>();
+    if (firstAndLastKeys == null) {
+      result.addAll(allFiles);
+      return result;
+    }
+
+    for (FileRef file : allFiles) {
+      Pair<Key,Key> pair = firstAndLastKeys.get(file);
+      if (pair == null) {
+        // file was created or imported after we obtained the first and last keys... there
+        // are a few options here... throw an exception which will cause the compaction to
+        // retry and also cause ugly error message that the admin has to ignore... could
+        // go get the first and last key, but this code is called while the tablet lock
+        // is held... or just compact the file....
+        result.add(file);
+      } else {
+        Key first = pair.getFirst();
+        Key last = pair.getSecond();
+        // If first and last are null, it's an empty file. Add it to the compact set so it goes away.
+        if ((first == null && last == null) || !extent.contains(first.getRow()) || !extent.contains(last.getRow())) {
+          result.add(file);
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Returns true if this tablet needs to be split
+   * 
+   */
+  public synchronized boolean needsSplit() {
+    boolean ret;
+
+    if (closing || closed)
+      ret = false;
+    else
+      ret = findSplitRow(getDatafileManager().getFiles()) != null;
+
+    return ret;
+  }
+
+  // BEGIN PRIVATE METHODS RELATED TO MAJOR COMPACTION
+
+  private boolean isCompactionEnabled() {
+    return !closing && !getTabletServer().isMajorCompactionDisabled();
+  }
+
+  private CompactionStats _majorCompact(MajorCompactionReason reason) throws IOException, CompactionCanceledException {
+
+    long t1, t2, t3;
+
+    // acquire file info outside of tablet lock
+    CompactionStrategy strategy = Property.createInstanceFromPropertyName(tableConfiguration, Property.TABLE_COMPACTION_STRATEGY, CompactionStrategy.class,
+        new DefaultCompactionStrategy());
+    strategy.init(Property.getCompactionStrategyOptions(tableConfiguration));
+
+    Map<FileRef,Pair<Key,Key>> firstAndLastKeys = null;
+    if (reason == MajorCompactionReason.CHOP) {
+      firstAndLastKeys = getFirstAndLastKeys(getDatafileManager().getDatafileSizes());
+    } else if (reason != MajorCompactionReason.USER) {
+      MajorCompactionRequest request = new MajorCompactionRequest(extent, reason, getTabletServer().getFileSystem(), tableConfiguration);
+      request.setFiles(getDatafileManager().getDatafileSizes());
+      strategy.gatherInformation(request);
+    }
+
+    Map<FileRef,DataFileValue> filesToCompact;
+
+    int maxFilesToCompact = tableConfiguration.getCount(Property.TSERV_MAJC_THREAD_MAXOPEN);
+
+    CompactionStats majCStats = new CompactionStats();
+    CompactionPlan plan = null;
+
+    boolean propogateDeletes = false;
+
+    synchronized (this) {
+      // plan all that work that needs to be done in the sync block... then do the actual work
+      // outside the sync block
+
+      t1 = System.currentTimeMillis();
+
+      majorCompactionWaitingToStart = true;
+
+      getTabletMemory().waitForMinC();
+
+      t2 = System.currentTimeMillis();
+
+      majorCompactionWaitingToStart = false;
+      notifyAll();
+
+      VolumeManager fs = getTabletServer().getFileSystem();
+      if (extent.isRootTablet()) {
+        // very important that we call this before doing major compaction,
+        // otherwise deleted compacted files could possible be brought back
+        // at some point if the file they were compacted to was legitimately
+        // removed by a major compaction
+        RootFiles.cleanupReplacement(fs, fs.listStatus(this.location), false);
+      }
+      SortedMap<FileRef,DataFileValue> allFiles = getDatafileManager().getDatafileSizes();
+      List<FileRef> inputFiles = new ArrayList<FileRef>();
+      if (reason == MajorCompactionReason.CHOP) {
+        // enforce rules: files with keys outside our range need to be compacted
+        inputFiles.addAll(findChopFiles(extent, firstAndLastKeys, allFiles.keySet()));
+      } else if (reason == MajorCompactionReason.USER) {
+        inputFiles.addAll(allFiles.keySet());
+      } else {
+        MajorCompactionRequest request = new MajorCompactionRequest(extent, reason, fs, tableConfiguration);
+        request.setFiles(allFiles);
+        plan = strategy.getCompactionPlan(request);
+        if (plan != null)
+          inputFiles.addAll(plan.inputFiles);
+      }
+
+      if (inputFiles.isEmpty()) {
+        return majCStats;
+      }
+      // If no original files will exist at the end of the compaction, we do not have to propogate deletes
+      Set<FileRef> droppedFiles = new HashSet<FileRef>();
+      droppedFiles.addAll(inputFiles);
+      if (plan != null)
+        droppedFiles.addAll(plan.deleteFiles);
+      propogateDeletes = !(droppedFiles.equals(allFiles.keySet()));
+      log.debug("Major compaction plan: " + plan + " propogate deletes : " + propogateDeletes);
+      filesToCompact = new HashMap<FileRef,DataFileValue>(allFiles);
+      filesToCompact.keySet().retainAll(inputFiles);
+
+      t3 = System.currentTimeMillis();
+
+      getDatafileManager().reserveMajorCompactingFiles(filesToCompact.keySet());
+    }
+
+    try {
+
+      log.debug(String.format("MajC initiate lock %.2f secs, wait %.2f secs", (t3 - t2) / 1000.0, (t2 - t1) / 1000.0));
+
+      Pair<Long,List<IteratorSetting>> compactionId = null;
+      if (!propogateDeletes) {
+        // compacting everything, so update the compaction id in metadata
+        try {
+          compactionId = getCompactionID();
+        } catch (NoNodeException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      List<IteratorSetting> compactionIterators = new ArrayList<IteratorSetting>();
+      if (compactionId != null) {
+        if (reason == MajorCompactionReason.USER) {
+          if (getCompactionCancelID() >= compactionId.getFirst()) {
+            // compaction was canceled
+            return majCStats;
+          }
+
+          synchronized (this) {
+            if (lastCompactID >= compactionId.getFirst())
+              // already compacted
+              return majCStats;
+          }
+        }
+
+        compactionIterators = compactionId.getSecond();
+      }
+
+      // need to handle case where only one file is being major compacted
+      while (filesToCompact.size() > 0) {
+
+        int numToCompact = maxFilesToCompact;
+
+        if (filesToCompact.size() > maxFilesToCompact && filesToCompact.size() < 2 * maxFilesToCompact) {
+          // on the second to last compaction pass, compact the minimum amount of files possible
+          numToCompact = filesToCompact.size() - maxFilesToCompact + 1;
+        }
+
+        Set<FileRef> smallestFiles = removeSmallest(filesToCompact, numToCompact);
+
+        FileRef fileName = getNextMapFilename((filesToCompact.size() == 0 && !propogateDeletes) ? "A" : "C");
+        FileRef compactTmpName = new FileRef(fileName.path().toString() + "_tmp");
+
+        AccumuloConfiguration tableConf = createTableConfiguration(tableConfiguration, plan);
+
+        Span span = Trace.start("compactFiles");
+        try {
+
+          CompactionEnv cenv = new CompactionEnv() {
+            @Override
+            public boolean isCompactionEnabled() {
+              return Tablet.this.isCompactionEnabled();
+            }
+
+            @Override
+            public IteratorScope getIteratorScope() {
+              return IteratorScope.majc;
+            }
+          };
+
+          HashMap<FileRef,DataFileValue> copy = new HashMap<FileRef,DataFileValue>(getDatafileManager().getDatafileSizes());
+          if (!copy.keySet().containsAll(smallestFiles))
+            throw new IllegalStateException("Cannot find data file values for " + smallestFiles);
+
+          copy.keySet().retainAll(smallestFiles);
+
+          log.debug("Starting MajC " + extent + " (" + reason + ") " + copy.keySet() + " --> " + compactTmpName + "  " + compactionIterators);
+
+          // always propagate deletes, unless last batch
+          boolean lastBatch = filesToCompact.isEmpty();
+          Compactor compactor = new Compactor(getTabletServer().getFileSystem(), copy, null, compactTmpName, lastBatch ? propogateDeletes : true, tableConf, extent, cenv,
+              compactionIterators, reason.ordinal());
+
+          CompactionStats mcs = compactor.call();
+
+          span.data("files", "" + smallestFiles.size());
+          span.data("read", "" + mcs.getEntriesRead());
+          span.data("written", "" + mcs.getEntriesWritten());
+          majCStats.add(mcs);
+
+          if (lastBatch && plan != null && plan.deleteFiles != null) {
+            smallestFiles.addAll(plan.deleteFiles);
+          }
+          getDatafileManager().bringMajorCompactionOnline(smallestFiles, compactTmpName, fileName,
+              filesToCompact.size() == 0 && compactionId != null ? compactionId.getFirst() : null,
+              new DataFileValue(mcs.getFileSize(), mcs.getEntriesWritten()));
+
+          // when major compaction produces a file w/ zero entries, it will be deleted... do not want
+          // to add the deleted file
+          if (filesToCompact.size() > 0 && mcs.getEntriesWritten() > 0) {
+            filesToCompact.put(fileName, new DataFileValue(mcs.getFileSize(), mcs.getEntriesWritten()));
+          }
+        } finally {
+          span.stop();
+        }
+
+      }
+      return majCStats;
+    } finally {
+      synchronized (Tablet.this) {
+        getDatafileManager().clearMajorCompactingFile();
+      }
+    }
+  }
+
+  private AccumuloConfiguration createTableConfiguration(TableConfiguration base, CompactionPlan plan) {
+    if (plan == null || plan.writeParameters == null)
+      return base;
+    WriteParameters p = plan.writeParameters;
+    ConfigurationCopy result = new ConfigurationCopy(base);
+    if (p.getHdfsBlockSize() > 0)
+      result.set(Property.TABLE_FILE_BLOCK_SIZE, "" + p.getHdfsBlockSize());
+    if (p.getBlockSize() > 0)
+      result.set(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE, "" + p.getBlockSize());
+    if (p.getIndexBlockSize() > 0)
+      result.set(Property.TABLE_FILE_COMPRESSED_BLOCK_SIZE_INDEX, "" + p.getBlockSize());
+    if (p.getCompressType() != null)
+      result.set(Property.TABLE_FILE_COMPRESSION_TYPE, p.getCompressType());
+    if (p.getReplication() != 0)
+      result.set(Property.TABLE_FILE_REPLICATION, "" + p.getReplication());
+    return result;
+  }
+
+  private Set<FileRef> removeSmallest(Map<FileRef,DataFileValue> filesToCompact, int maxFilesToCompact) {
+    // ensure this method works properly when multiple files have the same size
+
+    PriorityQueue<Pair<FileRef,Long>> fileHeap = new PriorityQueue<Pair<FileRef,Long>>(filesToCompact.size(), new Comparator<Pair<FileRef,Long>>() {
+      @Override
+      public int compare(Pair<FileRef,Long> o1, Pair<FileRef,Long> o2) {
+        if (o1.getSecond() == o2.getSecond())
+          return o1.getFirst().compareTo(o2.getFirst());
+        if (o1.getSecond() < o2.getSecond())
+          return -1;
+        return 1;
+      }
+    });
+
+    for (Iterator<Entry<FileRef,DataFileValue>> iterator = filesToCompact.entrySet().iterator(); iterator.hasNext();) {
+      Entry<FileRef,DataFileValue> entry = iterator.next();
+      fileHeap.add(new Pair<FileRef,Long>(entry.getKey(), entry.getValue().getSize()));
+    }
+
+    Set<FileRef> smallestFiles = new HashSet<FileRef>();
+    while (smallestFiles.size() < maxFilesToCompact && fileHeap.size() > 0) {
+      Pair<FileRef,Long> pair = fileHeap.remove();
+      filesToCompact.remove(pair.getFirst());
+      smallestFiles.add(pair.getFirst());
+    }
+
+    return smallestFiles;
+  }
+
+  // END PRIVATE METHODS RELATED TO MAJOR COMPACTION
+
+  /**
+   * Performs a major compaction on the tablet. If needsSplit() returns true, the tablet is split and a reference to the new tablet is returned.
+   */
+
+  CompactionStats majorCompact(MajorCompactionReason reason, long queued) {
+
+    CompactionStats majCStats = null;
+    boolean success = false;
+    long start = System.currentTimeMillis();
+
+    // Always trace majC
+    Span span = Trace.on("majorCompaction");
+
+    try {
+      timer.incrementStatusMajor();
+
+      synchronized (this) {
+        // check that compaction is still needed - defer to splitting
+        majorCompactionQueued.remove(reason);
+
+        if (closing || closed || !needsMajorCompaction(reason) || majorCompactionInProgress || needsSplit()) {
+          return null;
+        }
+
+        majorCompactionInProgress = true;
+      }
+
+      try {
+        majCStats = _majorCompact(reason);
+        if (reason == MajorCompactionReason.CHOP) {
+          MetadataTableUtil.chopped(getExtent(), this.getTabletServer().getLock());
+          getTabletServer().enqueueMasterMessage(new TabletStatusMessage(TabletLoadState.CHOPPED, extent));
+        }
+        success = true;
+      } catch (CompactionCanceledException mcce) {
+        log.debug("Major compaction canceled, extent = " + getExtent());
+        throw new RuntimeException(mcce);
+      } catch (Throwable t) {
+        log.error("MajC Failed, extent = " + getExtent());
+        log.error("MajC Failed, message = " + (t.getMessage() == null ? t.getClass().getName() : t.getMessage()), t);
+        throw new RuntimeException(t);
+      } finally {
+        // ensure we always reset boolean, even
+        // when an exception is thrown
+        synchronized (this) {
+          majorCompactionInProgress = false;
+          this.notifyAll();
+        }
+
+        Span curr = Trace.currentTrace();
+        curr.data("extent", "" + getExtent());
+        if (majCStats != null) {
+          curr.data("read", "" + majCStats.getEntriesRead());
+          curr.data("written", "" + majCStats.getEntriesWritten());
+        }
+      }
+    } finally {
+      span.stop();
+      long count = 0;
+      if (majCStats != null)
+        count = majCStats.getEntriesRead();
+      timer.updateTime(Operation.MAJOR, queued, start, count, !success);
+    }
+
+    return majCStats;
+  }
+
+  /**
+   * Returns a KeyExtent object representing this tablet's key range.
+   * 
+   * @return extent
+   */
+  public KeyExtent getExtent() {
+    return extent;
+  }
+
+  synchronized void computeNumEntries() {
+    Collection<DataFileValue> vals = getDatafileManager().getDatafileSizes().values();
+
+    long numEntries = 0;
+
+    for (DataFileValue tableValue : vals) {
+      numEntries += tableValue.getNumEntries();
+    }
+
+    this.numEntriesInMemory = getTabletMemory().getNumEntries();
+    numEntries += getTabletMemory().getNumEntries();
+
+    this.numEntries = numEntries;
+  }
+
+  public long getNumEntries() {
+    return numEntries;
+  }
+
+  public long getNumEntriesInMemory() {
+    return numEntriesInMemory;
+  }
+
+  public synchronized boolean isClosing() {
+    return closing;
+  }
+
+  public synchronized boolean isClosed() {
+    return closed;
+  }
+
+  public synchronized boolean isCloseComplete() {
+    return closeComplete;
+  }
+
+  public boolean majorCompactionRunning() {
+    return this.majorCompactionInProgress;
+  }
+
+  public boolean isMinorCompactionQueued() {
+    return minorCompactionWaitingToStart;
+  }
+
+  public boolean isMinorCompactionRunning() {
+    return minorCompactionInProgress;
+  }
+
+  public boolean isMajorCompactionQueued() {
+    return majorCompactionQueued.size() > 0;
+  }
+
+  public TreeMap<KeyExtent,SplitInfo> split(byte[] sp) throws IOException {
+
+    if (sp != null && extent.getEndRow() != null && extent.getEndRow().equals(new Text(sp))) {
+      throw new IllegalArgumentException();
+    }
+
+    if (extent.isRootTablet()) {
+      String msg = "Cannot split root tablet";
+      log.warn(msg);
+      throw new RuntimeException(msg);
+    }
+
+    try {
+      initiateClose(true, false, false);
+    } catch (IllegalStateException ise) {
+      log.debug("File " + extent + " not splitting : " + ise.getMessage());
+      return null;
+    }
+
+    // obtain this info outside of synch block since it will involve opening
+    // the map files... it is ok if the set of map files changes, because
+    // this info is used for optimization... it is ok if map files are missing
+    // from the set... can still query and insert into the tablet while this
+    // map file operation is happening
+    Map<FileRef,FileUtil.FileInfo> firstAndLastRows = FileUtil.tryToGetFirstAndLastRows(getTabletServer().getFileSystem(), getTabletServer().getSystemConfiguration(), getDatafileManager().getFiles());
+
+    synchronized (this) {
+      // java needs tuples ...
+      TreeMap<KeyExtent,SplitInfo> newTablets = new TreeMap<KeyExtent,SplitInfo>();
+
+      long t1 = System.currentTimeMillis();
+
+      // choose a split point
+      SplitRowSpec splitPoint;
+      if (sp == null)
+        splitPoint = findSplitRow(getDatafileManager().getFiles());
+      else {
+        Text tsp = new Text(sp);
+        splitPoint = new SplitRowSpec(FileUtil.estimatePercentageLTE(getTabletServer().getFileSystem(), getTabletServer().getSystemConfiguration(), extent.getPrevEndRow(), extent.getEndRow(),
+            FileUtil.toPathStrings(getDatafileManager().getFiles()), tsp), tsp);
+      }
+
+      if (splitPoint == null || splitPoint.row == null) {
+        log.info("had to abort split because splitRow was null");
+        closing = false;
+        return null;
+      }
+
+      closed = true;
+      completeClose(true, false);
+
+      Text midRow = splitPoint.row;
+      double splitRatio = splitPoint.splitRatio;
+
+      KeyExtent low = new KeyExtent(extent.getTableId(), midRow, extent.getPrevEndRow());
+      KeyExtent high = new KeyExtent(extent.getTableId(), extent.getEndRow(), midRow);
+
+      String lowDirectory = TabletOperations.createTabletDirectory(getTabletServer().getFileSystem(), extent.getTableId().toString(), midRow);
+
+      // write new tablet information to MetadataTable
+      SortedMap<FileRef,DataFileValue> lowDatafileSizes = new TreeMap<FileRef,DataFileValue>();
+      SortedMap<FileRef,DataFileValue> highDatafileSizes = new TreeMap<FileRef,DataFileValue>();
+      List<FileRef> highDatafilesToRemove = new ArrayList<FileRef>();
+
+      MetadataTableUtil.splitDatafiles(extent.getTableId(), midRow, splitRatio, firstAndLastRows, getDatafileManager().getDatafileSizes(), lowDatafileSizes,
+          highDatafileSizes, highDatafilesToRemove);
+
+      log.debug("Files for low split " + low + "  " + lowDatafileSizes.keySet());
+      log.debug("Files for high split " + high + "  " + highDatafileSizes.keySet());
+
+      String time = tabletTime.getMetadataValue();
+
+      // it is possible that some of the bulk loading flags will be deleted after being read below because the bulk load
+      // finishes.... therefore split could propagate load flags for a finished bulk load... there is a special iterator
+      // on the metadata table to clean up this type of garbage
+      Map<FileRef,Long> bulkLoadedFiles = MetadataTableUtil.getBulkFilesLoaded(SystemCredentials.get(), extent);
+
+      MetadataTableUtil.splitTablet(high, extent.getPrevEndRow(), splitRatio, SystemCredentials.get(), getTabletServer().getLock());
+      MasterMetadataUtil.addNewTablet(low, lowDirectory, getTabletServer().getTabletSession(), lowDatafileSizes, bulkLoadedFiles, SystemCredentials.get(), time,
+          lastFlushID, lastCompactID, getTabletServer().getLock());
+      MetadataTableUtil.finishSplit(high, highDatafileSizes, highDatafilesToRemove, SystemCredentials.get(), getTabletServer().getLock());
+
+      log.log(TLevel.TABLET_HIST, extent + " split " + low + " " + high);
+
+      newTablets.put(high, new SplitInfo(tabletDirectory, highDatafileSizes, time, lastFlushID, lastCompactID, lastLocation));
+      newTablets.put(low, new SplitInfo(lowDirectory, lowDatafileSizes, time, lastFlushID, lastCompactID, lastLocation));
+
+      long t2 = System.currentTimeMillis();
+
+      log.debug(String.format("offline split time : %6.2f secs", (t2 - t1) / 1000.0));
+
+      closeComplete = true;
+
+      return newTablets;
+    }
+  }
+
+  public SortedMap<FileRef,DataFileValue> getDatafiles() {
+    return getDatafileManager().getDatafileSizes();
+  }
+
+  public double queryRate() {
+    return queryRate.rate();
+  }
+
+  public double queryByteRate() {
+    return queryByteRate.rate();
+  }
+
+  public double ingestRate() {
+    return ingestRate.rate();
+  }
+
+  public double ingestByteRate() {
+    return ingestByteRate.rate();
+  }
+
+  public double scanRate() {
+    return scannedRate.rate();
+  }
+
+  public long totalQueries() {
+    return this.queryCount;
+  }
+
+  public long totalIngest() {
+    return this.ingestCount;
+  }
+
+  // synchronized?
+  public void updateRates(long now) {
+    queryRate.update(now, queryCount);
+    queryByteRate.update(now, queryBytes);
+    ingestRate.update(now, ingestCount);
+    ingestByteRate.update(now, ingestBytes);
+    scannedRate.update(now, scannedCount.get());
+  }
+
+  public long getSplitCreationTime() {
+    return splitCreationTime;
+  }
+
+  public void importMapFiles(long tid, Map<FileRef,MapFileInfo> fileMap, boolean setTime) throws IOException {
+    Map<FileRef,DataFileValue> entries = new HashMap<FileRef,DataFileValue>(fileMap.size());
+
+    for (Entry<FileRef,MapFileInfo> entry : fileMap.entrySet()) {
+      entries.put(entry.getKey(), new DataFileValue(entry.getValue().estimatedSize, 0l));
+    }
+
+    // Clients timeout and will think that this operation failed.
+    // Don't do it if we spent too long waiting for the lock
+    long now = System.currentTimeMillis();
+    synchronized (this) {
+      if (closed) {
+        throw new IOException("tablet " + extent + " is closed");
+      }
+
+      // TODO check seems uneeded now - ACCUMULO-1291
+      long lockWait = System.currentTimeMillis() - now;
+      if (lockWait > getTabletServer().getSystemConfiguration().getTimeInMillis(Property.GENERAL_RPC_TIMEOUT)) {
+        throw new IOException("Timeout waiting " + (lockWait / 1000.) + " seconds to get tablet lock");
+      }
+
+      if (writesInProgress < 0)
+        throw new IllegalStateException("writesInProgress < 0 " + writesInProgress);
+
+      writesInProgress++;
+    }
+
+    try {
+      getDatafileManager().importMapFiles(tid, entries, setTime);
+      lastMapFileImportTime = System.currentTimeMillis();
+
+      if (needsSplit()) {
+        getTabletServer().executeSplit(this);
+      } else {
+        initiateMajorCompaction(MajorCompactionReason.NORMAL);
+      }
+    } finally {
+      synchronized (this) {
+        if (writesInProgress < 1)
+          throw new IllegalStateException("writesInProgress < 1 " + writesInProgress);
+
+        writesInProgress--;
+        if (writesInProgress == 0)
+          this.notifyAll();
+      }
+    }
+  }
+
+  private Set<DfsLogger> currentLogs = new HashSet<DfsLogger>();
+
+  public Set<String> getCurrentLogFiles() {
+    Set<String> result = new HashSet<String>();
+    synchronized (currentLogs) {
+      for (DfsLogger log : currentLogs) {
+        result.add(log.getFileName());
+      }
+    }
+    return result;
+  }
+
+  Set<String> beginClearingUnusedLogs() {
+    Set<String> doomed = new HashSet<String>();
+
+    ArrayList<String> otherLogsCopy = new ArrayList<String>();
+    ArrayList<String> currentLogsCopy = new ArrayList<String>();
+
+    // do not hold tablet lock while acquiring the log lock
+    logLock.lock();
+
+    synchronized (this) {
+      if (removingLogs)
+        throw new IllegalStateException("Attempted to clear logs when removal of logs in progress");
+
+      for (DfsLogger logger : otherLogs) {
+        otherLogsCopy.add(logger.toString());
+        doomed.add(logger.toString());
+      }
+
+      for (DfsLogger logger : currentLogs) {
+        currentLogsCopy.add(logger.toString());
+        doomed.remove(logger.toString());
+      }
+
+      otherLogs = Collections.emptySet();
+
+      if (doomed.size() > 0)
+        removingLogs = true;
+    }
+
+    // do debug logging outside tablet lock
+    for (String logger : otherLogsCopy) {
+      log.debug("Logs for memory compacted: " + getExtent() + " " + logger.toString());
+    }
+
+    for (String logger : currentLogsCopy) {
+      log.debug("Logs for current memory: " + getExtent() + " " + logger);
+    }
+
+    return doomed;
+  }
+
+  synchronized void finishClearingUnusedLogs() {
+    removingLogs = false;
+    logLock.unlock();
+  }
+
+  private Set<DfsLogger> otherLogs = Collections.emptySet();
+  private boolean removingLogs = false;
+
+  // this lock is basically used to synchronize writing of log info to metadata
+  private final ReentrantLock logLock = new ReentrantLock();
+
+  public synchronized int getLogCount() {
+    return currentLogs.size();
+  }
+
+  public boolean beginUpdatingLogsUsed(InMemoryMap memTable, Collection<DfsLogger> more, boolean mincFinish) {
+
+    boolean releaseLock = true;
+
+    // do not hold tablet lock while acquiring the log lock
+    logLock.lock();
+
+    try {
+      synchronized (this) {
+
+        if (closed && closeComplete) {
+          throw new IllegalStateException("Can not update logs of closed tablet " + extent);
+        }
+
+        boolean addToOther;
+
+        if (memTable == getTabletMemory().getMinCMemTable())
+          addToOther = true;
+        else if (memTable == getTabletMemory().getMemTable())
+          addToOther = false;
+        else
+          throw new IllegalArgumentException("passed in memtable that is not in use");
+
+        if (mincFinish) {
+          if (addToOther)
+            throw new IllegalStateException("Adding to other logs for mincFinish");
+          if (otherLogs.size() != 0)
+            throw new IllegalStateException("Expect other logs to be 0 when min finish, but its " + otherLogs);
+
+          // when writing a minc finish event, there is no need to add the log to metadata
+          // if nothing has been logged for the tablet since the minor compaction started
+          if (currentLogs.size()

<TRUNCATED>

[13/35] ACCUMULO-2041 extract tablet classes to new files, move tablet-related code to o.a.a.tserver.tablet, make member variables private

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java
index e13594d..d1fece5 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java
@@ -42,7 +42,7 @@ public class TabletIteratorEnvironment implements IteratorEnvironment {
   private final ArrayList<SortedKeyValueIterator<Key,Value>> topLevelIterators = new ArrayList<SortedKeyValueIterator<Key,Value>>();
   private Map<FileRef,DataFileValue> files;
   
-  TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration config) {
+  public TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration config) {
     if (scope == IteratorScope.majc)
       throw new IllegalArgumentException("must set if compaction is full");
     
@@ -52,7 +52,7 @@ public class TabletIteratorEnvironment implements IteratorEnvironment {
     this.fullMajorCompaction = false;
   }
   
-  TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration config, ScanFileManager trm, Map<FileRef,DataFileValue> files) {
+  public TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration config, ScanFileManager trm, Map<FileRef,DataFileValue> files) {
     if (scope == IteratorScope.majc)
       throw new IllegalArgumentException("must set if compaction is full");
     
@@ -63,7 +63,7 @@ public class TabletIteratorEnvironment implements IteratorEnvironment {
     this.files = files;
   }
   
-  TabletIteratorEnvironment(IteratorScope scope, boolean fullMajC, AccumuloConfiguration config) {
+  public TabletIteratorEnvironment(IteratorScope scope, boolean fullMajC, AccumuloConfiguration config) {
     if (scope != IteratorScope.majc)
       throw new IllegalArgumentException("Tried to set maj compaction type when scope was " + scope);
     
@@ -101,7 +101,7 @@ public class TabletIteratorEnvironment implements IteratorEnvironment {
     topLevelIterators.add(iter);
   }
   
-  SortedKeyValueIterator<Key,Value> getTopLevelIterator(SortedKeyValueIterator<Key,Value> iter) {
+  public SortedKeyValueIterator<Key,Value> getTopLevelIterator(SortedKeyValueIterator<Key,Value> iter) {
     if (topLevelIterators.isEmpty())
       return iter;
     ArrayList<SortedKeyValueIterator<Key,Value>> allIters = new ArrayList<SortedKeyValueIterator<Key,Value>>(topLevelIterators);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/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 144d59b..1c07c44 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
@@ -195,17 +195,7 @@ import org.apache.accumulo.trace.instrument.Span;
 import org.apache.accumulo.trace.instrument.Trace;
 import org.apache.accumulo.trace.instrument.thrift.TraceWrap;
 import org.apache.accumulo.trace.thrift.TInfo;
-import org.apache.accumulo.tserver.Compactor.CompactionInfo;
 import org.apache.accumulo.tserver.RowLocks.RowLock;
-import org.apache.accumulo.tserver.Tablet.CommitSession;
-import org.apache.accumulo.tserver.Tablet.KVEntry;
-import org.apache.accumulo.tserver.Tablet.LookupResult;
-import org.apache.accumulo.tserver.Tablet.MinorCompactionReason;
-import org.apache.accumulo.tserver.Tablet.ScanBatch;
-import org.apache.accumulo.tserver.Tablet.Scanner;
-import org.apache.accumulo.tserver.Tablet.SplitInfo;
-import org.apache.accumulo.tserver.Tablet.TConstraintViolationException;
-import org.apache.accumulo.tserver.Tablet.TabletClosedException;
 import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
 import org.apache.accumulo.tserver.TabletStatsKeeper.Operation;
 import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
@@ -221,6 +211,17 @@ import org.apache.accumulo.tserver.metrics.TabletServerMBean;
 import org.apache.accumulo.tserver.metrics.TabletServerMinCMetrics;
 import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
 import org.apache.accumulo.tserver.metrics.TabletServerUpdateMetrics;
+import org.apache.accumulo.tserver.tablet.CommitSession;
+import org.apache.accumulo.tserver.tablet.CompactionInfo;
+import org.apache.accumulo.tserver.tablet.CompactionWatcher;
+import org.apache.accumulo.tserver.tablet.Compactor;
+import org.apache.accumulo.tserver.tablet.KVEntry;
+import org.apache.accumulo.tserver.tablet.Tablet.LookupResult;
+import org.apache.accumulo.tserver.tablet.ScanBatch;
+import org.apache.accumulo.tserver.tablet.Scanner;
+import org.apache.accumulo.tserver.tablet.SplitInfo;
+import org.apache.accumulo.tserver.tablet.Tablet;
+import org.apache.accumulo.tserver.tablet.TabletClosedException;
 import org.apache.commons.collections.map.LRUMap;
 import org.apache.hadoop.fs.FSError;
 import org.apache.hadoop.fs.FileSystem;
@@ -252,7 +253,10 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
   private TabletServerLogger logger;
 
-  protected TabletServerMinCMetrics mincMetrics = new TabletServerMinCMetrics();
+  protected final TabletServerMinCMetrics mincMetrics = new TabletServerMinCMetrics();
+  public TabletServerMinCMetrics getMinCMetrics() {
+    return mincMetrics;
+  }
 
   private ServerConfiguration serverConfig;
   private LogSorter logSorter = null;
@@ -629,7 +633,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     }
   }
 
-  static class TservConstraintEnv implements Environment {
+  public static class TservConstraintEnv implements Environment {
 
     private TCredentials credentials;
     private SecurityOperation security;
@@ -641,7 +645,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       this.credentials = credentials;
     }
 
-    void setExtent(KeyExtent ke) {
+    public void setExtent(KeyExtent ke) {
       this.ke = ke;
     }
 
@@ -1659,16 +1663,16 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
             commitSession.commit(mutations);
 
-            Tablet tablet = commitSession.getTablet();
+            KeyExtent extent = commitSession.getExtent();
 
-            if (tablet == us.currentTablet) {
+            if (extent == us.currentTablet.getExtent()) {
               // because constraint violations may filter out some
               // mutations, for proper
               // accounting with the client code, need to increment
               // the count based
               // on the original number of mutations from the client
               // NOT the filtered number
-              us.successfulCommits.increment(tablet, us.queuedMutations.get(tablet).size());
+              us.successfulCommits.increment(us.currentTablet, us.queuedMutations.get(us.currentTablet).size());
             }
           }
           long t2 = System.currentTimeMillis();
@@ -2141,7 +2145,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         KeyExtent ke = entry.getKey();
         if (ke.getTableId().compareTo(text) == 0) {
           Tablet tablet = entry.getValue();
-          TabletStats stats = tablet.timer.getTabletStats();
+          TabletStats stats = tablet.getTabletStats();
           stats.extent = ke.toThrift();
           stats.ingestRate = tablet.ingestRate();
           stats.queryRate = tablet.queryRate();
@@ -2563,11 +2567,11 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     }
   }
 
-  boolean isMajorCompactionDisabled() {
+  public boolean isMajorCompactionDisabled() {
     return majorCompactorDisabled;
   }
 
-  void executeSplit(Tablet tablet) {
+  public void executeSplit(Tablet tablet) {
     resourceManager.executeSplit(tablet.getExtent(), new LoggingRunnable(log, new SplitRunner(tablet)));
   }
 
@@ -2617,7 +2621,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
             }
 
             synchronized (tablet) {
-              if (tablet.initiateMajorCompaction(MajorCompactionReason.NORMAL) || tablet.majorCompactionQueued() || tablet.majorCompactionRunning()) {
+              if (tablet.initiateMajorCompaction(MajorCompactionReason.NORMAL) || tablet.isMajorCompactionQueued() || tablet.isMajorCompactionRunning()) {
                 numMajorCompactionsInProgress++;
                 continue;
               }
@@ -2683,16 +2687,15 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
     Entry<KeyExtent,SplitInfo> first = tabletInfo.firstEntry();
     TabletResourceManager newTrm0 = resourceManager.createTabletResourceManager(first.getKey(), getTableConfiguration(first.getKey()));
-    newTablets[0] = new Tablet(first.getKey(), TabletServer.this, newTrm0, first.getValue());
+    newTablets[0] = new Tablet(TabletServer.this, first.getKey(), newTrm0, first.getValue());
 
     Entry<KeyExtent,SplitInfo> last = tabletInfo.lastEntry();
     TabletResourceManager newTrm1 = resourceManager.createTabletResourceManager(last.getKey(), getTableConfiguration(last.getKey()));
-    newTablets[1] = new Tablet(last.getKey(), TabletServer.this, newTrm1, last.getValue());
+    newTablets[1] = new Tablet(TabletServer.this, last.getKey(), newTrm1, last.getValue());
 
     // roll tablet stats over into tablet server's statsKeeper object as
     // historical data
-    statsKeeper.saveMinorTimes(tablet.timer);
-    statsKeeper.saveMajorTimes(tablet.timer);
+    statsKeeper.saveMajorMinorTimes(tablet.getTabletStats());
 
     // lose the reference to the old tablet and open two new ones
     synchronized (onlineTablets) {
@@ -2719,7 +2722,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
   private BlockingDeque<MasterMessage> masterMessages = new LinkedBlockingDeque<MasterMessage>();
 
   // add a message for the main thread to send back to the master
-  void enqueueMasterMessage(MasterMessage m) {
+  public void enqueueMasterMessage(MasterMessage m) {
     masterMessages.addLast(m);
   }
 
@@ -2808,9 +2811,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
       // roll tablet stats over into tablet server's statsKeeper object as
       // historical data
-      statsKeeper.saveMinorTimes(t.timer);
-      statsKeeper.saveMajorTimes(t.timer);
-
+      statsKeeper.saveMajorMinorTimes(t.getTabletStats());
       log.info("unloaded " + extent);
 
     }
@@ -2914,7 +2915,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         // this opens the tablet file and fills in the endKey in the
         // extent
         locationToOpen = VolumeUtil.switchRootTabletVolume(extent, locationToOpen);
-        tablet = new Tablet(TabletServer.this, locationToOpen, extent, trm, tabletsKeyValues);
+        tablet = new Tablet(TabletServer.this, extent, locationToOpen, trm, tabletsKeyValues);
         /*
          * If a minor compaction starts after a tablet opens, this indicates a log recovery occurred. This recovered data must be minor compacted.
          *
@@ -3018,7 +3019,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
   private static ObjectName OBJECT_NAME = null;
 
-  static AtomicLong seekCount = new AtomicLong(0);
+  public static final AtomicLong seekCount = new AtomicLong(0);
 
   public TabletStatsKeeper getStatsKeeper() {
     return statsKeeper;
@@ -3098,7 +3099,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     return address;
   }
 
-  ZooLock getLock() {
+  public ZooLock getLock() {
     return tabletServerLock;
   }
 
@@ -3452,7 +3453,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     return clientAddress.getHostText() + ":" + clientAddress.getPort();
   }
 
-  TServerInstance getTabletSession() {
+  public TServerInstance getTabletSession() {
     String address = getClientAddressString();
     if (address == null)
       return null;
@@ -3596,13 +3597,13 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       table.scanRate += tablet.scanRate();
       long recsInMemory = tablet.getNumEntriesInMemory();
       table.recsInMemory += recsInMemory;
-      if (tablet.minorCompactionRunning())
+      if (tablet.isMinorCompactionRunning())
         table.minors.running++;
-      if (tablet.minorCompactionQueued())
+      if (tablet.isMinorCompactionQueued())
         table.minors.queued++;
-      if (tablet.majorCompactionRunning())
+      if (tablet.isMajorCompactionRunning())
         table.majors.running++;
-      if (tablet.majorCompactionQueued())
+      if (tablet.isMajorCompactionQueued())
         table.majors.queued++;
     }
 
@@ -3775,7 +3776,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     if (this.isEnabled()) {
       int result = 0;
       for (Tablet tablet : Collections.unmodifiableCollection(onlineTablets.values())) {
-        if (tablet.majorCompactionQueued())
+        if (tablet.isMajorCompactionQueued())
           result++;
       }
       return result;
@@ -3788,7 +3789,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     if (this.isEnabled()) {
       int result = 0;
       for (Tablet tablet : Collections.unmodifiableCollection(onlineTablets.values())) {
-        if (tablet.minorCompactionRunning())
+        if (tablet.isMinorCompactionRunning())
           result++;
       }
       return result;
@@ -3801,7 +3802,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     if (this.isEnabled()) {
       int result = 0;
       for (Tablet tablet : Collections.unmodifiableCollection(onlineTablets.values())) {
-        if (tablet.minorCompactionQueued())
+        if (tablet.isMinorCompactionQueued())
           result++;
       }
       return result;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
index f26c74b..095f8d5 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServerResourceManager.java
@@ -54,11 +54,11 @@ import org.apache.accumulo.server.tabletserver.TabletState;
 import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.accumulo.trace.instrument.TraceExecutorService;
 import org.apache.accumulo.tserver.FileManager.ScanFileManager;
-import org.apache.accumulo.tserver.Tablet.MinorCompactionReason;
 import org.apache.accumulo.tserver.compaction.CompactionStrategy;
 import org.apache.accumulo.tserver.compaction.DefaultCompactionStrategy;
 import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
 import org.apache.accumulo.tserver.compaction.MajorCompactionRequest;
+import org.apache.accumulo.tserver.tablet.Tablet;
 import org.apache.log4j.Logger;
 
 /**
@@ -69,34 +69,34 @@ import org.apache.log4j.Logger;
  */
 public class TabletServerResourceManager {
 
-  private ExecutorService minorCompactionThreadPool;
-  private ExecutorService majorCompactionThreadPool;
-  private ExecutorService rootMajorCompactionThreadPool;
-  private ExecutorService defaultMajorCompactionThreadPool;
-  private ExecutorService splitThreadPool;
-  private ExecutorService defaultSplitThreadPool;
-  private ExecutorService defaultMigrationPool;
-  private ExecutorService migrationPool;
-  private ExecutorService assignmentPool;
-  private ExecutorService assignMetaDataPool;
-  private ExecutorService readAheadThreadPool;
-  private ExecutorService defaultReadAheadThreadPool;
-  private Map<String,ExecutorService> threadPools = new TreeMap<String,ExecutorService>();
+  private static final Logger log = Logger.getLogger(TabletServerResourceManager.class);
+
+  private final ExecutorService minorCompactionThreadPool;
+  private final ExecutorService majorCompactionThreadPool;
+  private final ExecutorService rootMajorCompactionThreadPool;
+  private final ExecutorService defaultMajorCompactionThreadPool;
+  private final ExecutorService splitThreadPool;
+  private final ExecutorService defaultSplitThreadPool;
+  private final ExecutorService defaultMigrationPool;
+  private final ExecutorService migrationPool;
+  private final ExecutorService assignmentPool;
+  private final ExecutorService assignMetaDataPool;
+  private final ExecutorService readAheadThreadPool;
+  private final ExecutorService defaultReadAheadThreadPool;
+  private final Map<String,ExecutorService> threadPools = new TreeMap<String,ExecutorService>();
 
   private final VolumeManager fs;
 
-  private FileManager fileManager;
+  private final FileManager fileManager;
 
-  private MemoryManager memoryManager;
+  private final MemoryManager memoryManager;
 
-  private MemoryManagementFramework memMgmt;
+  private final MemoryManagementFramework memMgmt;
 
   private final LruBlockCache _dCache;
   private final LruBlockCache _iCache;
   private final ServerConfiguration conf;
 
-  private static final Logger log = Logger.getLogger(TabletServerResourceManager.class);
-
   private ExecutorService addEs(String name, ExecutorService tp) {
     if (threadPools.containsKey(name)) {
       throw new IllegalArgumentException("Cannot create two executor services with same name " + name);
@@ -210,10 +210,10 @@ public class TabletServerResourceManager {
 
   private static class TabletStateImpl implements TabletState, Cloneable {
 
-    private long lct;
-    private Tablet tablet;
-    private long mts;
-    private long mcmts;
+    private final long lct;
+    private final Tablet tablet;
+    private final long mts;
+    private final long mcmts;
 
     public TabletStateImpl(Tablet t, long mts, long lct, long mcmts) {
       this.tablet = t;
@@ -249,11 +249,12 @@ public class TabletServerResourceManager {
 
   private class MemoryManagementFramework {
     private final Map<KeyExtent,TabletStateImpl> tabletReports;
-    private LinkedBlockingQueue<TabletStateImpl> memUsageReports;
+    private final LinkedBlockingQueue<TabletStateImpl> memUsageReports;
     private long lastMemCheckTime = System.currentTimeMillis();
     private long maxMem;
-    private Thread memoryGuardThread;
-    private Thread minorCompactionInitiatorThread;
+    private long lastMemTotal = 0;
+    private final Thread memoryGuardThread;
+    private final Thread minorCompactionInitiatorThread;
 
     MemoryManagementFramework() {
       tabletReports = Collections.synchronizedMap(new HashMap<KeyExtent,TabletStateImpl>());
@@ -287,8 +288,6 @@ public class TabletServerResourceManager {
       minorCompactionInitiatorThread.start();
     }
 
-    private long lastMemTotal = 0;
-
     private void processTabletMemStats() {
       while (true) {
         try {
@@ -494,7 +493,7 @@ public class TabletServerResourceManager {
       lastReportedCommitTime = System.currentTimeMillis();
     }
 
-    synchronized ScanFileManager newScanFileManager() {
+    public synchronized ScanFileManager newScanFileManager() {
       if (closed)
         throw new IllegalStateException("closed");
       return fileManager.newScanFileManager(extent);
@@ -504,8 +503,8 @@ public class TabletServerResourceManager {
 
     // BEGIN methods that Tablets call to manage memory
 
-    private AtomicLong lastReportedSize = new AtomicLong();
-    private AtomicLong lastReportedMincSize = new AtomicLong();
+    private final AtomicLong lastReportedSize = new AtomicLong();
+    private final AtomicLong lastReportedMincSize = new AtomicLong();
     private volatile long lastReportedCommitTime = 0;
 
     public void updateMemoryUsageStats(Tablet tablet, long size, long mincSize) {
@@ -544,7 +543,7 @@ public class TabletServerResourceManager {
     // BEGIN methods that Tablets call to make decisions about major compaction
     // when too many files are open, we may want tablets to compact down
     // to one map file
-    boolean needsMajorCompaction(SortedMap<FileRef,DataFileValue> tabletFiles, MajorCompactionReason reason) {
+    public boolean needsMajorCompaction(SortedMap<FileRef,DataFileValue> tabletFiles, MajorCompactionReason reason) {
       if (closed)
         return false;// throw new IOException("closed");
 
@@ -585,11 +584,11 @@ public class TabletServerResourceManager {
     // tablets call this method to run minor compactions,
     // this allows us to control how many minor compactions
     // run concurrently in a tablet server
-    void executeMinorCompaction(final Runnable r) {
+    public void executeMinorCompaction(final Runnable r) {
       minorCompactionThreadPool.execute(new LoggingRunnable(log, r));
     }
 
-    void close() throws IOException {
+    public void close() throws IOException {
       // always obtain locks in same order to avoid deadlock
       synchronized (TabletServerResourceManager.this) {
         synchronized (this) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletStatsKeeper.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletStatsKeeper.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletStatsKeeper.java
index 58e16be..d914ac6 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletStatsKeeper.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletStatsKeeper.java
@@ -81,6 +81,11 @@ public class TabletStatsKeeper {
     
   }
   
+  public void saveMajorMinorTimes(TabletStats t) {
+    ActionStatsUpdator.update(minor, t.minors);
+    ActionStatsUpdator.update(major, t.majors);
+  }
+  
   public void saveMinorTimes(TabletStatsKeeper t) {
     ActionStatsUpdator.update(minor, t.minor);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 871f4ae..9fec437 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -36,10 +36,10 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.tserver.Tablet.CommitSession;
 import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.TabletServer;
 import org.apache.accumulo.tserver.log.DfsLogger.LoggerOperation;
+import org.apache.accumulo.tserver.tablet.CommitSession;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
@@ -81,7 +81,7 @@ public class TabletServerLogger {
   }
 
   private static boolean enabled(CommitSession commitSession) {
-    return enabled(commitSession.getTablet().getTableConfiguration());
+    return commitSession.getUseWAL();
   }
 
   static private abstract class TestCallWithWriteLock {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Batch.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Batch.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Batch.java
new file mode 100644
index 0000000..73434c6
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Batch.java
@@ -0,0 +1,35 @@
+/*
+ * 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.tserver.tablet;
+
+import java.util.List;
+
+import org.apache.accumulo.core.data.Key;
+
+class Batch {
+  final boolean skipContinueKey;
+  final List<KVEntry> results;
+  final Key continueKey;
+  final long numBytes;
+  
+  Batch(boolean skipContinueKey, List<KVEntry> results, Key continueKey, long numBytes) {
+    this.skipContinueKey = skipContinueKey;
+    this.results = results;
+    this.continueKey = continueKey;
+    this.numBytes = numBytes;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
new file mode 100644
index 0000000..6402797
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
@@ -0,0 +1,121 @@
+/*
+ * 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.tserver.tablet;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.tserver.InMemoryMap;
+import org.apache.accumulo.tserver.log.DfsLogger;
+import org.apache.log4j.Logger;
+
+public class CommitSession {
+  
+  private static final Logger log = Logger.getLogger(CommitSession.class);
+
+  private final int seq;
+  private final InMemoryMap memTable;
+  private final TabletCommitter committer;
+
+  private int commitsInProgress;
+  private long maxCommittedTime = Long.MIN_VALUE;
+
+  CommitSession(TabletCommitter committer, int seq, InMemoryMap imm) {
+    this.seq = seq;
+    this.memTable = imm;
+    this.committer = committer;
+    commitsInProgress = 0;
+  }
+
+  public int getWALogSeq() {
+    return seq;
+  }
+
+  public void decrementCommitsInProgress() {
+    if (commitsInProgress < 1)
+      throw new IllegalStateException("commitsInProgress = " + commitsInProgress);
+
+    commitsInProgress--;
+    if (commitsInProgress == 0)
+      committer.notifyAll();
+  }
+
+  public void incrementCommitsInProgress() {
+    if (commitsInProgress < 0)
+      throw new IllegalStateException("commitsInProgress = " + commitsInProgress);
+
+    commitsInProgress++;
+  }
+
+  public void waitForCommitsToFinish() {
+    while (commitsInProgress > 0) {
+      try {
+        committer.wait(50);
+      } catch (InterruptedException e) {
+        log.warn(e, e);
+      }
+    }
+  }
+
+  public void abortCommit(List<Mutation> value) {
+    committer.abortCommit(this, value);
+  }
+
+  public void commit(List<Mutation> mutations) {
+    committer.commit(this, mutations);
+  }
+
+  public TabletCommitter getTablet() {
+    return committer;
+  }
+
+  public boolean beginUpdatingLogsUsed(ArrayList<DfsLogger> copy, boolean mincFinish) {
+    return committer.beginUpdatingLogsUsed(memTable, copy, mincFinish);
+  }
+
+  public void finishUpdatingLogsUsed() {
+    committer.finishUpdatingLogsUsed();
+  }
+
+  public int getLogId() {
+    return committer.getLogId();
+  }
+
+  public KeyExtent getExtent() {
+    return committer.getExtent();
+  }
+
+  public void updateMaxCommittedTime(long time) {
+    maxCommittedTime = Math.max(time, maxCommittedTime);
+  }
+
+  public long getMaxCommittedTime() {
+    if (maxCommittedTime == Long.MIN_VALUE)
+      throw new IllegalStateException("Tried to read max committed time when it was never set");
+    return maxCommittedTime;
+  }
+
+  public boolean getUseWAL() {
+    return committer.getUseWAL();
+  }
+
+  public void mutate(List<Mutation> mutations) {
+    memTable.mutate(mutations);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionInfo.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionInfo.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionInfo.java
new file mode 100644
index 0000000..ab57d65
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionInfo.java
@@ -0,0 +1,113 @@
+package org.apache.accumulo.tserver.tablet;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.thrift.IterInfo;
+import org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction;
+import org.apache.accumulo.core.tabletserver.thrift.CompactionReason;
+import org.apache.accumulo.core.tabletserver.thrift.CompactionType;
+import org.apache.accumulo.server.fs.FileRef;
+
+public class CompactionInfo {
+
+  private final Compactor compactor;
+  private final String localityGroup;
+  private final long entriesRead;
+  private final long entriesWritten;
+
+  CompactionInfo(Compactor compactor) {
+    this.localityGroup = compactor.getCurrentLocalityGroup();
+    this.entriesRead = compactor.getEntriesRead();
+    this.entriesWritten = compactor.getEntriesWritten();
+    this.compactor = compactor;
+  }
+
+  public long getID() {
+    return compactor.getCompactorID();
+  }
+
+  public KeyExtent getExtent() {
+    return compactor.getExtent();
+  }
+
+  public long getEntriesRead() {
+    return entriesRead;
+  }
+
+  public long getEntriesWritten() {
+    return entriesWritten;
+  }
+
+  public Thread getThread() {
+    return compactor.thread;
+  }
+
+  public String getOutputFile() {
+    return compactor.getOutputFile();
+  }
+
+  public ActiveCompaction toThrift() {
+
+    CompactionType type;
+
+    if (compactor.hasIMM())
+      if (compactor.getFilesToCompact().size() > 0)
+        type = CompactionType.MERGE;
+      else
+        type = CompactionType.MINOR;
+    else if (!compactor.willPropogateDeletes())
+      type = CompactionType.FULL;
+    else
+      type = CompactionType.MAJOR;
+
+    CompactionReason reason;
+
+    if (compactor.hasIMM())
+      switch (compactor.getMinCReason()) {
+        case USER:
+          reason = CompactionReason.USER;
+          break;
+        case CLOSE:
+          reason = CompactionReason.CLOSE;
+          break;
+        case SYSTEM:
+        default:
+          reason = CompactionReason.SYSTEM;
+          break;
+      }
+    else
+      switch (compactor.getMajorCompactionReason()) {
+        case USER:
+          reason = CompactionReason.USER;
+          break;
+        case CHOP:
+          reason = CompactionReason.CHOP;
+          break;
+        case IDLE:
+          reason = CompactionReason.IDLE;
+          break;
+        case NORMAL:
+        default:
+          reason = CompactionReason.SYSTEM;
+          break;
+      }
+
+    List<IterInfo> iiList = new ArrayList<IterInfo>();
+    Map<String,Map<String,String>> iterOptions = new HashMap<String,Map<String,String>>();
+
+    for (IteratorSetting iterSetting : compactor.getIterators()) {
+      iiList.add(new IterInfo(iterSetting.getPriority(), iterSetting.getIteratorClass(), iterSetting.getName()));
+      iterOptions.put(iterSetting.getName(), iterSetting.getOptions());
+    }
+    List<String> filesToCompact = new ArrayList<String>();
+    for (FileRef ref : compactor.getFilesToCompact())
+      filesToCompact.add(ref.toString());
+    return new ActiveCompaction(compactor.extent.toThrift(), System.currentTimeMillis() - compactor.getStartTime(), filesToCompact,
+        compactor.getOutputFile(), type, reason, localityGroup, entriesRead, entriesWritten, iiList, iterOptions);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionRunner.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionRunner.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionRunner.java
new file mode 100644
index 0000000..de5a66d
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionRunner.java
@@ -0,0 +1,76 @@
+/*
+ * 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.tserver.tablet;
+
+import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
+
+class CompactionRunner implements Runnable, Comparable<CompactionRunner> {
+
+  private final Tablet tablet;
+  private final MajorCompactionReason reason;
+  private final long queued;
+
+  public CompactionRunner(Tablet tablet, MajorCompactionReason reason) {
+    this.tablet = tablet;
+    queued = System.currentTimeMillis();
+    this.reason = reason;
+  }
+
+  @Override
+  public void run() {
+    if (tablet.getTabletServer().isMajorCompactionDisabled()) {
+      // this will make compaction tasks that were queued when shutdown was
+      // initiated exit
+      tablet.removeMajorCompactionQueuedReason(reason);
+      return;
+    }
+
+    tablet.majorCompact(reason, queued);
+
+    // if there is more work to be done, queue another major compaction
+    synchronized (tablet) {
+      if (reason == MajorCompactionReason.NORMAL && tablet.needsMajorCompaction(reason))
+        tablet.initiateMajorCompaction(reason);
+    }
+  }
+
+  // We used to synchronize on the Tablet before fetching this information,
+  // but this method is called by the compaction queue thread to re-order the compactions.
+  // The compaction queue holds a lock during this sort.
+  // A tablet lock can be held while putting itself on the queue, so we can't lock the tablet
+  // while pulling information used to sort the tablets in the queue, or we may get deadlocked.
+  // See ACCUMULO-1110.
+  private int getNumFiles() {
+    return tablet.getDatafileManager().getNumFiles();
+  }
+
+  @Override
+  public int compareTo(CompactionRunner o) {
+    int cmp = reason.compareTo(o.reason);
+    if (cmp != 0)
+      return cmp;
+
+    if (reason == MajorCompactionReason.USER || reason == MajorCompactionReason.CHOP) {
+      // for these types of compactions want to do the oldest first
+      cmp = (int) (queued - o.queued);
+      if (cmp != 0)
+        return cmp;
+    }
+
+    return o.getNumFiles() - this.getNumFiles();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionStats.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionStats.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionStats.java
new file mode 100644
index 0000000..69832e9
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionStats.java
@@ -0,0 +1,59 @@
+/*
+ * 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.tserver.tablet;
+
+public class CompactionStats {
+  private long entriesRead;
+  private long entriesWritten;
+  private long fileSize;
+  
+  CompactionStats(long er, long ew) {
+    this.setEntriesRead(er);
+    this.setEntriesWritten(ew);
+  }
+  
+  public CompactionStats() {}
+  
+  private void setEntriesRead(long entriesRead) {
+    this.entriesRead = entriesRead;
+  }
+  
+  public long getEntriesRead() {
+    return entriesRead;
+  }
+  
+  private void setEntriesWritten(long entriesWritten) {
+    this.entriesWritten = entriesWritten;
+  }
+  
+  public long getEntriesWritten() {
+    return entriesWritten;
+  }
+  
+  public void add(CompactionStats mcs) {
+    this.entriesRead += mcs.entriesRead;
+    this.entriesWritten += mcs.entriesWritten;
+  }
+  
+  public void setFileSize(long fileSize) {
+    this.fileSize = fileSize;
+  }
+  
+  public long getFileSize() {
+    return this.fileSize;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionWatcher.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionWatcher.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionWatcher.java
new file mode 100644
index 0000000..1ca1f33
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CompactionWatcher.java
@@ -0,0 +1,109 @@
+/*
+ * 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.tserver.tablet;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.util.time.SimpleTimer;
+import org.apache.log4j.Logger;
+
+/**
+ * 
+ */
+public class CompactionWatcher implements Runnable {
+  private final Map<List<Long>,ObservedCompactionInfo> observedCompactions = new HashMap<List<Long>,ObservedCompactionInfo>();
+  private final AccumuloConfiguration config;
+  private static boolean watching = false;
+  
+  private static class ObservedCompactionInfo {
+    CompactionInfo compactionInfo;
+    long firstSeen;
+    boolean loggedWarning;
+    
+    ObservedCompactionInfo(CompactionInfo ci, long time) {
+      this.compactionInfo = ci;
+      this.firstSeen = time;
+    }
+  }
+
+  public CompactionWatcher(AccumuloConfiguration config) {
+    this.config = config;
+  }
+
+  public void run() {
+    List<CompactionInfo> runningCompactions = Compactor.getRunningCompactions();
+    
+    Set<List<Long>> newKeys = new HashSet<List<Long>>();
+    
+    long time = System.currentTimeMillis();
+
+    for (CompactionInfo ci : runningCompactions) {
+      List<Long> compactionKey = Arrays.asList(ci.getID(), ci.getEntriesRead(), ci.getEntriesWritten());
+      newKeys.add(compactionKey);
+      
+      if (!observedCompactions.containsKey(compactionKey)) {
+        observedCompactions.put(compactionKey, new ObservedCompactionInfo(ci, time));
+      }
+    }
+    
+    // look for compactions that finished or made progress and logged a warning
+    HashMap<List<Long>,ObservedCompactionInfo> copy = new HashMap<List<Long>,ObservedCompactionInfo>(observedCompactions);
+    copy.keySet().removeAll(newKeys);
+    
+    for (ObservedCompactionInfo oci : copy.values()) {
+      if (oci.loggedWarning) {
+        Logger.getLogger(CompactionWatcher.class).info("Compaction of " + oci.compactionInfo.getExtent() + " is no longer stuck");
+      }
+    }
+
+    // remove any compaction that completed or made progress
+    observedCompactions.keySet().retainAll(newKeys);
+    
+    long warnTime = config.getTimeInMillis(Property.TSERV_COMPACTION_WARN_TIME);
+
+    // check for stuck compactions
+    for (ObservedCompactionInfo oci : observedCompactions.values()) {
+      if (time - oci.firstSeen > warnTime && !oci.loggedWarning) {
+        Thread compactionThread = oci.compactionInfo.getThread();
+        if (compactionThread != null) {
+          StackTraceElement[] trace = compactionThread.getStackTrace();
+          Exception e = new Exception("Possible stack trace of compaction stuck on " + oci.compactionInfo.getExtent());
+          e.setStackTrace(trace);
+          Logger.getLogger(CompactionWatcher.class).warn(
+              "Compaction of " + oci.compactionInfo.getExtent() + " to " + oci.compactionInfo.getOutputFile() + " has not made progress for at least "
+                  + (time - oci.firstSeen) + "ms", e);
+          oci.loggedWarning = true;
+        }
+      }
+    }
+  }
+
+  public static synchronized void startWatching(AccumuloConfiguration config) {
+    if (!watching) {
+      SimpleTimer.getInstance(config).schedule(new CompactionWatcher(config), 10000, 10000);
+      watching = true;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
new file mode 100644
index 0000000..2a3e2f4
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
@@ -0,0 +1,477 @@
+/*
+ * 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.tserver.tablet;
+
+import java.io.IOException;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.IteratorUtil;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
+import org.apache.accumulo.core.iterators.system.DeletingIterator;
+import org.apache.accumulo.core.iterators.system.MultiIterator;
+import org.apache.accumulo.core.iterators.system.TimeSettingIterator;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.util.LocalityGroupUtil;
+import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
+import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.problems.ProblemReport;
+import org.apache.accumulo.server.problems.ProblemReportingIterator;
+import org.apache.accumulo.server.problems.ProblemReports;
+import org.apache.accumulo.server.problems.ProblemType;
+import org.apache.accumulo.trace.instrument.Span;
+import org.apache.accumulo.trace.instrument.Trace;
+import org.apache.accumulo.tserver.InMemoryMap;
+import org.apache.accumulo.tserver.MinorCompactionReason;
+import org.apache.accumulo.tserver.TabletIteratorEnvironment;
+import org.apache.accumulo.tserver.compaction.MajorCompactionReason;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.Logger;
+
+public class Compactor implements Callable<CompactionStats> {
+  private static final Logger log = Logger.getLogger(Compactor.class);
+  private static final AtomicLong nextCompactorID = new AtomicLong(0);
+
+  public static class CountingIterator extends WrappingIterator {
+
+    private long count;
+    private final ArrayList<CountingIterator> deepCopies;
+    private final AtomicLong entriesRead;
+
+    @Override
+    public CountingIterator deepCopy(IteratorEnvironment env) {
+      return new CountingIterator(this, env);
+    }
+
+    private CountingIterator(CountingIterator other, IteratorEnvironment env) {
+      setSource(other.getSource().deepCopy(env));
+      count = 0;
+      this.deepCopies = other.deepCopies;
+      this.entriesRead = other.entriesRead;
+      deepCopies.add(this);
+    }
+
+    public CountingIterator(SortedKeyValueIterator<Key,Value> source, AtomicLong entriesRead) {
+      deepCopies = new ArrayList<Compactor.CountingIterator>();
+      this.setSource(source);
+      count = 0;
+      this.entriesRead = entriesRead;
+    }
+
+    @Override
+    public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void next() throws IOException {
+      super.next();
+      count++;
+      if (count % 1024 == 0) {
+        entriesRead.addAndGet(1024);
+      }
+    }
+
+    public long getCount() {
+      long sum = 0;
+      for (CountingIterator dc : deepCopies) {
+        sum += dc.count;
+      }
+
+      return count + sum;
+    }
+  }
+
+
+  public static class CompactionCanceledException extends Exception {
+    private static final long serialVersionUID = 1L;
+  }
+
+  public interface CompactionEnv {
+    
+    boolean isCompactionEnabled();
+
+    IteratorScope getIteratorScope();
+  }
+
+  private final Map<FileRef,DataFileValue> filesToCompact;
+  private final InMemoryMap imm;
+  private final FileRef outputFile;
+  private final boolean propogateDeletes;
+  private final AccumuloConfiguration acuTableConf;
+  private final CompactionEnv env;
+  private final VolumeManager fs;
+  protected final KeyExtent extent;
+  private final List<IteratorSetting> iterators;
+
+  // things to report
+  private String currentLocalityGroup = "";
+  private final long startTime;
+
+  private int reason;
+
+  private final AtomicLong entriesRead = new AtomicLong(0);
+  private final AtomicLong entriesWritten = new AtomicLong(0);
+  private final DateFormat dateFormatter = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS");
+
+  // a unique id to identify a compactor
+  private final long compactorID = nextCompactorID.getAndIncrement();
+  protected volatile Thread thread;
+
+  public long getCompactorID() { return compactorID; }
+
+  private synchronized void setLocalityGroup(String name) {
+    this.currentLocalityGroup = name;
+  }
+  
+  public synchronized String getCurrentLocalityGroup() {
+    return currentLocalityGroup;
+  }
+
+  private void clearStats() {
+    entriesRead.set(0);
+    entriesWritten.set(0);
+  }
+
+  protected static final Set<Compactor> runningCompactions = Collections.synchronizedSet(new HashSet<Compactor>());
+
+  public static List<CompactionInfo> getRunningCompactions() {
+    ArrayList<CompactionInfo> compactions = new ArrayList<CompactionInfo>();
+
+    synchronized (runningCompactions) {
+      for (Compactor compactor : runningCompactions) {
+        compactions.add(new CompactionInfo(compactor));
+      }
+    }
+
+    return compactions;
+  }
+
+  public Compactor(VolumeManager fs, Map<FileRef,DataFileValue> files, InMemoryMap imm, FileRef outputFile, boolean propogateDeletes,
+      AccumuloConfiguration acuTableConf, KeyExtent extent, CompactionEnv env, List<IteratorSetting> iterators, int reason) {
+    this.extent = extent;
+    this.fs = fs;
+    this.filesToCompact = files;
+    this.imm = imm;
+    this.outputFile = outputFile;
+    this.propogateDeletes = propogateDeletes;
+    this.acuTableConf = acuTableConf;
+    this.env = env;
+    this.iterators = iterators;
+    this.reason = reason;
+
+    startTime = System.currentTimeMillis();
+  }
+
+  public VolumeManager getFileSystem() {
+    return fs;
+  }
+
+  KeyExtent getExtent() {
+    return extent;
+  }
+
+  String getOutputFile() {
+    return outputFile.toString();
+  }
+
+  MajorCompactionReason getMajorCompactionReason() { return MajorCompactionReason.values()[reason]; }
+
+  @Override
+  public CompactionStats call() throws IOException, CompactionCanceledException {
+
+    FileSKVWriter mfw = null;
+
+    CompactionStats majCStats = new CompactionStats();
+
+    boolean remove = runningCompactions.add(this);
+
+    clearStats();
+
+    String oldThreadName = Thread.currentThread().getName();
+    String newThreadName = "MajC compacting " + extent.toString() + " started " + dateFormatter.format(new Date()) + " file: " + outputFile;
+    Thread.currentThread().setName(newThreadName);
+    thread = Thread.currentThread();
+    try {
+      FileOperations fileFactory = FileOperations.getInstance();
+      FileSystem ns = this.fs.getVolumeByPath(outputFile.path()).getFileSystem();
+      mfw = fileFactory.openWriter(outputFile.path().toString(), ns, ns.getConf(), acuTableConf);
+
+      Map<String,Set<ByteSequence>> lGroups;
+      try {
+        lGroups = LocalityGroupUtil.getLocalityGroups(acuTableConf);
+      } catch (LocalityGroupConfigurationError e) {
+        throw new IOException(e);
+      }
+
+      long t1 = System.currentTimeMillis();
+
+      HashSet<ByteSequence> allColumnFamilies = new HashSet<ByteSequence>();
+
+      if (mfw.supportsLocalityGroups()) {
+        for (Entry<String,Set<ByteSequence>> entry : lGroups.entrySet()) {
+          setLocalityGroup(entry.getKey());
+          compactLocalityGroup(entry.getKey(), entry.getValue(), true, mfw, majCStats);
+          allColumnFamilies.addAll(entry.getValue());
+        }
+      }
+
+      setLocalityGroup("");
+      compactLocalityGroup(null, allColumnFamilies, false, mfw, majCStats);
+
+      long t2 = System.currentTimeMillis();
+
+      FileSKVWriter mfwTmp = mfw;
+      mfw = null; // set this to null so we do not try to close it again in finally if the close fails
+      mfwTmp.close(); // if the close fails it will cause the compaction to fail
+
+      // Verify the file, since hadoop 0.20.2 sometimes lies about the success of close()
+      try {
+        FileSKVIterator openReader = fileFactory.openReader(outputFile.path().toString(), false, ns, ns.getConf(), acuTableConf);
+        openReader.close();
+      } catch (IOException ex) {
+        log.error("Verification of successful compaction fails!!! " + extent + " " + outputFile, ex);
+        throw ex;
+      }
+
+      log.debug(String.format("Compaction %s %,d read | %,d written | %,6d entries/sec | %6.3f secs", extent, majCStats.getEntriesRead(),
+          majCStats.getEntriesWritten(), (int) (majCStats.getEntriesRead() / ((t2 - t1) / 1000.0)), (t2 - t1) / 1000.0));
+
+      majCStats.setFileSize(fileFactory.getFileSize(outputFile.path().toString(), ns, ns.getConf(), acuTableConf));
+      return majCStats;
+    } catch (IOException e) {
+      log.error(e, e);
+      throw e;
+    } catch (RuntimeException e) {
+      log.error(e, e);
+      throw e;
+    } finally {
+      Thread.currentThread().setName(oldThreadName);
+      if (remove) {
+        thread = null;
+        runningCompactions.remove(this);
+      }
+
+      try {
+        if (mfw != null) {
+          // compaction must not have finished successfully, so close its output file
+          try {
+            mfw.close();
+          } finally {
+            if (!fs.deleteRecursively(outputFile.path()))
+              if (fs.exists(outputFile.path()))
+                log.error("Unable to delete " + outputFile);
+          }
+        }
+      } catch (IOException e) {
+        log.warn(e, e);
+      } catch (RuntimeException exception) {
+        log.warn(exception, exception);
+      }
+    }
+  }
+
+  private List<SortedKeyValueIterator<Key,Value>> openMapDataFiles(String lgName, ArrayList<FileSKVIterator> readers) throws IOException {
+
+    List<SortedKeyValueIterator<Key,Value>> iters = new ArrayList<SortedKeyValueIterator<Key,Value>>(filesToCompact.size());
+
+    for (FileRef mapFile : filesToCompact.keySet()) {
+      try {
+
+        FileOperations fileFactory = FileOperations.getInstance();
+        FileSystem fs = this.fs.getVolumeByPath(mapFile.path()).getFileSystem();
+        FileSKVIterator reader;
+
+        reader = fileFactory.openReader(mapFile.path().toString(), false, fs, fs.getConf(), acuTableConf);
+
+        readers.add(reader);
+
+        SortedKeyValueIterator<Key,Value> iter = new ProblemReportingIterator(extent.getTableId().toString(), mapFile.path().toString(), false, reader);
+
+        if (filesToCompact.get(mapFile).isTimeSet()) {
+          iter = new TimeSettingIterator(iter, filesToCompact.get(mapFile).getTime());
+        }
+
+        iters.add(iter);
+
+      } catch (Throwable e) {
+
+        ProblemReports.getInstance().report(new ProblemReport(extent.getTableId().toString(), ProblemType.FILE_READ, mapFile.path().toString(), e));
+
+        log.warn("Some problem opening map file " + mapFile + " " + e.getMessage(), e);
+        // failed to open some map file... close the ones that were opened
+        for (FileSKVIterator reader : readers) {
+          try {
+            reader.close();
+          } catch (Throwable e2) {
+            log.warn("Failed to close map file", e2);
+          }
+        }
+
+        readers.clear();
+
+        if (e instanceof IOException)
+          throw (IOException) e;
+        throw new IOException("Failed to open map data files", e);
+      }
+    }
+
+    return iters;
+  }
+
+  private void compactLocalityGroup(String lgName, Set<ByteSequence> columnFamilies, boolean inclusive, FileSKVWriter mfw, CompactionStats majCStats)
+      throws IOException, CompactionCanceledException {
+    ArrayList<FileSKVIterator> readers = new ArrayList<FileSKVIterator>(filesToCompact.size());
+    Span span = Trace.start("compact");
+    try {
+      long entriesCompacted = 0;
+      List<SortedKeyValueIterator<Key,Value>> iters = openMapDataFiles(lgName, readers);
+
+      if (imm != null) {
+        iters.add(imm.compactionIterator());
+      }
+
+      CountingIterator citr = new CountingIterator(new MultiIterator(iters, extent.toDataRange()), entriesRead);
+      DeletingIterator delIter = new DeletingIterator(citr, propogateDeletes);
+      ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
+
+      // if(env.getIteratorScope() )
+
+      TabletIteratorEnvironment iterEnv;
+      if (env.getIteratorScope() == IteratorScope.majc)
+        iterEnv = new TabletIteratorEnvironment(IteratorScope.majc, !propogateDeletes, acuTableConf);
+      else if (env.getIteratorScope() == IteratorScope.minc)
+        iterEnv = new TabletIteratorEnvironment(IteratorScope.minc, acuTableConf);
+      else
+        throw new IllegalArgumentException();
+
+      SortedKeyValueIterator<Key,Value> itr = iterEnv.getTopLevelIterator(IteratorUtil.loadIterators(env.getIteratorScope(), cfsi, extent, acuTableConf,
+          iterators, iterEnv));
+
+      itr.seek(extent.toDataRange(), columnFamilies, inclusive);
+
+      if (!inclusive) {
+        mfw.startDefaultLocalityGroup();
+      } else {
+        mfw.startNewLocalityGroup(lgName, columnFamilies);
+      }
+
+      Span write = Trace.start("write");
+      try {
+        while (itr.hasTop() && env.isCompactionEnabled()) {
+          mfw.append(itr.getTopKey(), itr.getTopValue());
+          itr.next();
+          entriesCompacted++;
+
+          if (entriesCompacted % 1024 == 0) {
+            // Periodically update stats, do not want to do this too often since its volatile
+            entriesWritten.addAndGet(1024);
+          }
+        }
+
+        if (itr.hasTop() && !env.isCompactionEnabled()) {
+          // cancel major compaction operation
+          try {
+            try {
+              mfw.close();
+            } catch (IOException e) {
+              log.error(e, e);
+            }
+            fs.deleteRecursively(outputFile.path());
+          } catch (Exception e) {
+            log.warn("Failed to delete Canceled compaction output file " + outputFile, e);
+          }
+          throw new CompactionCanceledException();
+        }
+
+      } finally {
+        CompactionStats lgMajcStats = new CompactionStats(citr.getCount(), entriesCompacted);
+        majCStats.add(lgMajcStats);
+        write.stop();
+      }
+
+    } finally {
+      // close sequence files opened
+      for (FileSKVIterator reader : readers) {
+        try {
+          reader.close();
+        } catch (Throwable e) {
+          log.warn("Failed to close map file", e);
+        }
+      }
+      span.stop();
+    }
+  }
+
+  Collection<FileRef> getFilesToCompact() {
+    return filesToCompact.keySet();
+  }
+
+  boolean hasIMM() {
+    return imm != null;
+  }
+
+  boolean willPropogateDeletes() {
+    return propogateDeletes;
+  }
+
+  long getEntriesRead() {
+    return entriesRead.get();
+  }
+  
+  long getEntriesWritten() {
+    return entriesWritten.get();
+  }
+
+  long getStartTime() {
+    return startTime;
+  }
+
+  Iterable<IteratorSetting> getIterators() {
+    return this.iterators;
+  }
+
+  MinorCompactionReason getMinCReason() {
+    return MinorCompactionReason.values()[reason];
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
new file mode 100644
index 0000000..2771db9
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
@@ -0,0 +1,581 @@
+/*
+ * 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.tserver.tablet;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.util.MapCounter;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
+import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.client.HdfsZooInstance;
+import org.apache.accumulo.server.fs.FileRef;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.master.state.TServerInstance;
+import org.apache.accumulo.server.security.SystemCredentials;
+import org.apache.accumulo.server.util.MasterMetadataUtil;
+import org.apache.accumulo.server.util.MetadataTableUtil;
+import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.trace.instrument.Span;
+import org.apache.accumulo.trace.instrument.Trace;
+import org.apache.accumulo.tserver.TLevel;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Logger;
+
+class DatafileManager {
+  private final Logger log = Logger.getLogger(DatafileManager.class);
+  // access to datafilesizes needs to be synchronized: see CompactionRunner#getNumFiles
+  private final Map<FileRef,DataFileValue> datafileSizes = Collections.synchronizedMap(new TreeMap<FileRef,DataFileValue>());
+  private final Tablet tablet;
+  
+  // ensure we only have one reader/writer of our bulk file notes at at time
+  private final Object bulkFileImportLock = new Object();
+
+  DatafileManager(Tablet tablet, SortedMap<FileRef,DataFileValue> datafileSizes) {
+    for (Entry<FileRef,DataFileValue> datafiles : datafileSizes.entrySet()) {
+      this.datafileSizes.put(datafiles.getKey(), datafiles.getValue());
+    }
+    this.tablet = tablet;
+  }
+
+  private FileRef mergingMinorCompactionFile = null;
+  private final Set<FileRef> filesToDeleteAfterScan = new HashSet<FileRef>();
+  private final Map<Long,Set<FileRef>> scanFileReservations = new HashMap<Long,Set<FileRef>>();
+  private final MapCounter<FileRef> fileScanReferenceCounts = new MapCounter<FileRef>();
+  private long nextScanReservationId = 0;
+  private boolean reservationsBlocked = false;
+
+  private final Set<FileRef> majorCompactingFiles = new HashSet<FileRef>();
+  
+  static void rename(VolumeManager fs, Path src, Path dst) throws IOException {
+    if (!fs.rename(src, dst)) {
+      throw new IOException("Rename " + src + " to " + dst + " returned false ");
+    }
+  }
+
+  Pair<Long,Map<FileRef,DataFileValue>> reserveFilesForScan() {
+    synchronized (tablet) {
+
+      while (reservationsBlocked) {
+        try {
+          tablet.wait(50);
+        } catch (InterruptedException e) {
+          log.warn(e, e);
+        }
+      }
+
+      Set<FileRef> absFilePaths = new HashSet<FileRef>(datafileSizes.keySet());
+
+      long rid = nextScanReservationId++;
+
+      scanFileReservations.put(rid, absFilePaths);
+
+      Map<FileRef,DataFileValue> ret = new HashMap<FileRef,DataFileValue>();
+
+      for (FileRef path : absFilePaths) {
+        fileScanReferenceCounts.increment(path, 1);
+        ret.put(path, datafileSizes.get(path));
+      }
+
+      return new Pair<Long,Map<FileRef,DataFileValue>>(rid, ret);
+    }
+  }
+
+  void returnFilesForScan(Long reservationId) {
+
+    final Set<FileRef> filesToDelete = new HashSet<FileRef>();
+
+    synchronized (tablet) {
+      Set<FileRef> absFilePaths = scanFileReservations.remove(reservationId);
+
+      if (absFilePaths == null)
+        throw new IllegalArgumentException("Unknown scan reservation id " + reservationId);
+
+      boolean notify = false;
+      for (FileRef path : absFilePaths) {
+        long refCount = fileScanReferenceCounts.decrement(path, 1);
+        if (refCount == 0) {
+          if (filesToDeleteAfterScan.remove(path))
+            filesToDelete.add(path);
+          notify = true;
+        } else if (refCount < 0)
+          throw new IllegalStateException("Scan ref count for " + path + " is " + refCount);
+      }
+
+      if (notify)
+        tablet.notifyAll();
+    }
+
+    if (filesToDelete.size() > 0) {
+      log.debug("Removing scan refs from metadata " + tablet.getExtent() + " " + filesToDelete);
+      MetadataTableUtil.removeScanFiles(tablet.getExtent(), filesToDelete, SystemCredentials.get(), tablet.getTabletServer().getLock());
+    }
+  }
+
+  void removeFilesAfterScan(Set<FileRef> scanFiles) {
+    if (scanFiles.size() == 0)
+      return;
+
+    Set<FileRef> filesToDelete = new HashSet<FileRef>();
+
+    synchronized (tablet) {
+      for (FileRef path : scanFiles) {
+        if (fileScanReferenceCounts.get(path) == 0)
+          filesToDelete.add(path);
+        else
+          filesToDeleteAfterScan.add(path);
+      }
+    }
+
+    if (filesToDelete.size() > 0) {
+      log.debug("Removing scan refs from metadata " + tablet.getExtent() + " " + filesToDelete);
+      MetadataTableUtil.removeScanFiles(tablet.getExtent(), filesToDelete, SystemCredentials.get(), tablet.getTabletServer().getLock());
+    }
+  }
+
+  private TreeSet<FileRef> waitForScansToFinish(Set<FileRef> pathsToWaitFor, boolean blockNewScans, long maxWaitTime) {
+    long startTime = System.currentTimeMillis();
+    TreeSet<FileRef> inUse = new TreeSet<FileRef>();
+
+    Span waitForScans = Trace.start("waitForScans");
+    try {
+      synchronized (tablet) {
+        if (blockNewScans) {
+          if (reservationsBlocked)
+            throw new IllegalStateException();
+
+          reservationsBlocked = true;
+        }
+
+        for (FileRef path : pathsToWaitFor) {
+          while (fileScanReferenceCounts.get(path) > 0 && System.currentTimeMillis() - startTime < maxWaitTime) {
+            try {
+              tablet.wait(100);
+            } catch (InterruptedException e) {
+              log.warn(e, e);
+            }
+          }
+        }
+
+        for (FileRef path : pathsToWaitFor) {
+          if (fileScanReferenceCounts.get(path) > 0)
+            inUse.add(path);
+        }
+
+        if (blockNewScans) {
+          reservationsBlocked = false;
+          tablet.notifyAll();
+        }
+
+      }
+    } finally {
+      waitForScans.stop();
+    }
+    return inUse;
+  }
+
+  public void importMapFiles(long tid, Map<FileRef,DataFileValue> pathsString, boolean setTime) throws IOException {
+
+    final KeyExtent extent = tablet.getExtent();
+    String bulkDir = null;
+
+    Map<FileRef,DataFileValue> paths = new HashMap<FileRef,DataFileValue>();
+    for (Entry<FileRef,DataFileValue> entry : pathsString.entrySet())
+      paths.put(entry.getKey(), entry.getValue());
+
+    for (FileRef tpath : paths.keySet()) {
+
+      boolean inTheRightDirectory = false;
+      Path parent = tpath.path().getParent().getParent();
+      for (String tablesDir : ServerConstants.getTablesDirs()) {
+        if (parent.equals(new Path(tablesDir, tablet.getExtent().getTableId().toString()))) {
+          inTheRightDirectory = true;
+          break;
+        }
+      }
+      if (!inTheRightDirectory) {
+        throw new IOException("Data file " + tpath + " not in table dirs");
+      }
+
+      if (bulkDir == null)
+        bulkDir = tpath.path().getParent().toString();
+      else if (!bulkDir.equals(tpath.path().getParent().toString()))
+        throw new IllegalArgumentException("bulk files in different dirs " + bulkDir + " " + tpath);
+
+    }
+
+    if (tablet.getExtent().isRootTablet()) {
+      throw new IllegalArgumentException("Can not import files to root tablet");
+    }
+
+    synchronized (bulkFileImportLock) {
+      Credentials creds = SystemCredentials.get();
+      Connector conn;
+      try {
+        conn = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), creds.getToken());
+      } catch (Exception ex) {
+        throw new IOException(ex);
+      }
+      // Remove any bulk files we've previously loaded and compacted away
+      List<FileRef> files = MetadataTableUtil.getBulkFilesLoaded(conn, extent, tid);
+
+      for (FileRef file : files)
+        if (paths.keySet().remove(file))
+          log.debug("Ignoring request to re-import a file already imported: " + extent + ": " + file);
+
+      if (paths.size() > 0) {
+        long bulkTime = Long.MIN_VALUE;
+        if (setTime) {
+          for (DataFileValue dfv : paths.values()) {
+            long nextTime = tablet.getAndUpdateTime();
+            if (nextTime < bulkTime)
+              throw new IllegalStateException("Time went backwards unexpectedly " + nextTime + " " + bulkTime);
+            bulkTime = nextTime;
+            dfv.setTime(bulkTime);
+          }
+        }
+        
+        tablet.updatePersistedTime(bulkTime, paths, tid);
+      }
+    }
+
+    synchronized (tablet) {
+      for (Entry<FileRef,DataFileValue> tpath : paths.entrySet()) {
+        if (datafileSizes.containsKey(tpath.getKey())) {
+          log.error("Adding file that is already in set " + tpath.getKey());
+        }
+        datafileSizes.put(tpath.getKey(), tpath.getValue());
+
+      }
+
+      tablet.getTabletResources().importedMapFiles();
+
+      tablet.computeNumEntries();
+    }
+
+    for (Entry<FileRef,DataFileValue> entry : paths.entrySet()) {
+      log.log(TLevel.TABLET_HIST, tablet.getExtent() + " import " + entry.getKey() + " " + entry.getValue());
+    }
+  }
+
+  FileRef reserveMergingMinorCompactionFile() {
+    if (mergingMinorCompactionFile != null)
+      throw new IllegalStateException("Tried to reserve merging minor compaction file when already reserved  : " + mergingMinorCompactionFile);
+
+    if (tablet.getExtent().isRootTablet())
+      return null;
+
+    int maxFiles = tablet.getTableConfiguration().getMaxFilesPerTablet();
+
+    // when a major compaction is running and we are at max files, write out
+    // one extra file... want to avoid the case where major compaction is
+    // compacting everything except for the largest file, and therefore the
+    // largest file is returned for merging.. the following check mostly
+    // avoids this case, except for the case where major compactions fail or
+    // are canceled
+    if (majorCompactingFiles.size() > 0 && datafileSizes.size() == maxFiles)
+      return null;
+
+    if (datafileSizes.size() >= maxFiles) {
+      // find the smallest file
+
+      long min = Long.MAX_VALUE;
+      FileRef minName = null;
+
+      for (Entry<FileRef,DataFileValue> entry : datafileSizes.entrySet()) {
+        if (entry.getValue().getSize() < min && !majorCompactingFiles.contains(entry.getKey())) {
+          min = entry.getValue().getSize();
+          minName = entry.getKey();
+        }
+      }
+
+      if (minName == null)
+        return null;
+
+      mergingMinorCompactionFile = minName;
+      return minName;
+    }
+
+    return null;
+  }
+
+  void unreserveMergingMinorCompactionFile(FileRef file) {
+    if ((file == null && mergingMinorCompactionFile != null) || (file != null && mergingMinorCompactionFile == null)
+        || (file != null && mergingMinorCompactionFile != null && !file.equals(mergingMinorCompactionFile)))
+      throw new IllegalStateException("Disagreement " + file + " " + mergingMinorCompactionFile);
+
+    mergingMinorCompactionFile = null;
+  }
+
+  void bringMinorCompactionOnline(FileRef tmpDatafile, FileRef newDatafile, FileRef absMergeFile, DataFileValue dfv, CommitSession commitSession, long flushId)
+      throws IOException {
+
+    IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
+    if (tablet.getExtent().isRootTablet()) {
+      try {
+        if (!zoo.isLockHeld(tablet.getTabletServer().getLock().getLockID())) {
+          throw new IllegalStateException();
+        }
+      } catch (Exception e) {
+        throw new IllegalStateException("Can not bring major compaction online, lock not held", e);
+      }
+    }
+
+    // rename before putting in metadata table, so files in metadata table should
+    // always exist
+    do {
+      try {
+        if (dfv.getNumEntries() == 0) {
+          tablet.getTabletServer().getFileSystem().deleteRecursively(tmpDatafile.path());
+        } else {
+          if (tablet.getTabletServer().getFileSystem().exists(newDatafile.path())) {
+            log.warn("Target map file already exist " + newDatafile);
+            tablet.getTabletServer().getFileSystem().deleteRecursively(newDatafile.path());
+          }
+
+          rename(tablet.getTabletServer().getFileSystem(), tmpDatafile.path(), newDatafile.path());
+        }
+        break;
+      } catch (IOException ioe) {
+        log.warn("Tablet " + tablet.getExtent() + " failed to rename " + newDatafile + " after MinC, will retry in 60 secs...", ioe);
+        UtilWaitThread.sleep(60 * 1000);
+      }
+    } while (true);
+
+    long t1, t2;
+
+    // the code below always assumes merged files are in use by scans... this must be done
+    // because the in memory list of files is not updated until after the metadata table
+    // therefore the file is available to scans until memory is updated, but want to ensure
+    // the file is not available for garbage collection... if memory were updated
+    // before this point (like major compactions do), then the following code could wait
+    // for scans to finish like major compactions do.... used to wait for scans to finish
+    // here, but that was incorrect because a scan could start after waiting but before
+    // memory was updated... assuming the file is always in use by scans leads to
+    // one uneeded metadata update when it was not actually in use
+    Set<FileRef> filesInUseByScans = Collections.emptySet();
+    if (absMergeFile != null)
+      filesInUseByScans = Collections.singleton(absMergeFile);
+
+    // very important to write delete entries outside of log lock, because
+    // this metadata write does not go up... it goes sideways or to itself
+    if (absMergeFile != null)
+      MetadataTableUtil.addDeleteEntries(tablet.getExtent(), Collections.singleton(absMergeFile), SystemCredentials.get());
+
+    Set<String> unusedWalLogs = tablet.beginClearingUnusedLogs();
+    try {
+      // the order of writing to metadata and walog is important in the face of machine/process failures
+      // need to write to metadata before writing to walog, when things are done in the reverse order
+      // data could be lost... the minor compaction start even should be written before the following metadata
+      // write is made
+
+      tablet.updateTabletDataFile(commitSession.getMaxCommittedTime(), newDatafile, absMergeFile, dfv, unusedWalLogs, filesInUseByScans, flushId);
+
+    } finally {
+      tablet.finishClearingUnusedLogs();
+    }
+
+    do {
+      try {
+        // the purpose of making this update use the new commit session, instead of the old one passed in,
+        // is because the new one will reference the logs used by current memory...
+        
+        tablet.getTabletServer().minorCompactionFinished(tablet.getTabletMemory().getCommitSession(), newDatafile.toString(), commitSession.getWALogSeq() + 2);
+        break;
+      } catch (IOException e) {
+        log.error("Failed to write to write-ahead log " + e.getMessage() + " will retry", e);
+        UtilWaitThread.sleep(1 * 1000);
+      }
+    } while (true);
+
+    synchronized (tablet) {
+      t1 = System.currentTimeMillis();
+
+      if (datafileSizes.containsKey(newDatafile)) {
+        log.error("Adding file that is already in set " + newDatafile);
+      }
+      
+      if (dfv.getNumEntries() > 0) {
+        datafileSizes.put(newDatafile, dfv);
+      }
+      
+      if (absMergeFile != null) {
+        datafileSizes.remove(absMergeFile);
+      }
+      
+      unreserveMergingMinorCompactionFile(absMergeFile);
+      
+      tablet.flushComplete(flushId);
+      
+      t2 = System.currentTimeMillis();
+    }
+
+    // must do this after list of files in memory is updated above
+    removeFilesAfterScan(filesInUseByScans);
+
+    if (absMergeFile != null)
+      log.log(TLevel.TABLET_HIST, tablet.getExtent() + " MinC [" + absMergeFile + ",memory] -> " + newDatafile);
+    else
+      log.log(TLevel.TABLET_HIST, tablet.getExtent() + " MinC [memory] -> " + newDatafile);
+    log.debug(String.format("MinC finish lock %.2f secs %s", (t2 - t1) / 1000.0, tablet.getExtent().toString()));
+    long splitSize = tablet.getTableConfiguration().getMemoryInBytes(Property.TABLE_SPLIT_THRESHOLD);
+    if (dfv.getSize() > splitSize) {
+      log.debug(String.format("Minor Compaction wrote out file larger than split threshold.  split threshold = %,d  file size = %,d", splitSize, dfv.getSize()));
+    }
+  }
+
+  public void reserveMajorCompactingFiles(Collection<FileRef> files) {
+    if (majorCompactingFiles.size() != 0)
+      throw new IllegalStateException("Major compacting files not empty " + majorCompactingFiles);
+
+    if (mergingMinorCompactionFile != null && files.contains(mergingMinorCompactionFile))
+      throw new IllegalStateException("Major compaction tried to resrve file in use by minor compaction " + mergingMinorCompactionFile);
+
+    majorCompactingFiles.addAll(files);
+  }
+
+  public void clearMajorCompactingFile() {
+    majorCompactingFiles.clear();
+  }
+
+  void bringMajorCompactionOnline(Set<FileRef> oldDatafiles, FileRef tmpDatafile, FileRef newDatafile, Long compactionId, DataFileValue dfv)
+      throws IOException {
+    final KeyExtent extent = tablet.getExtent();
+    long t1, t2;
+
+    if (!extent.isRootTablet()) {
+
+      if (tablet.getTabletServer().getFileSystem().exists(newDatafile.path())) {
+        log.error("Target map file already exist " + newDatafile, new Exception());
+        throw new IllegalStateException("Target map file already exist " + newDatafile);
+      }
+
+      // rename before putting in metadata table, so files in metadata table should
+      // always exist
+      rename(tablet.getTabletServer().getFileSystem(), tmpDatafile.path(), newDatafile.path());
+
+      if (dfv.getNumEntries() == 0) {
+        tablet.getTabletServer().getFileSystem().deleteRecursively(newDatafile.path());
+      }
+    }
+
+    TServerInstance lastLocation = null;
+    synchronized (tablet) {
+
+      t1 = System.currentTimeMillis();
+
+      IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
+
+      tablet.incrementDataSourceDeletions();
+
+      if (extent.isRootTablet()) {
+
+        waitForScansToFinish(oldDatafiles, true, Long.MAX_VALUE);
+
+        try {
+          if (!zoo.isLockHeld(tablet.getTabletServer().getLock().getLockID())) {
+            throw new IllegalStateException();
+          }
+        } catch (Exception e) {
+          throw new IllegalStateException("Can not bring major compaction online, lock not held", e);
+        }
+
+        // mark files as ready for deletion, but
+        // do not delete them until we successfully
+        // rename the compacted map file, in case
+        // the system goes down
+
+        RootFiles.replaceFiles(tablet.getTableConfiguration(), tablet.getTabletServer().getFileSystem(), tablet.getLocation(), oldDatafiles, tmpDatafile, newDatafile);
+      }
+
+      // atomically remove old files and add new file
+      for (FileRef oldDatafile : oldDatafiles) {
+        if (!datafileSizes.containsKey(oldDatafile)) {
+          log.error("file does not exist in set " + oldDatafile);
+        }
+        datafileSizes.remove(oldDatafile);
+        majorCompactingFiles.remove(oldDatafile);
+      }
+
+      if (datafileSizes.containsKey(newDatafile)) {
+        log.error("Adding file that is already in set " + newDatafile);
+      }
+
+      if (dfv.getNumEntries() > 0) {
+        datafileSizes.put(newDatafile, dfv);
+      }
+
+      // could be used by a follow on compaction in a multipass compaction
+      majorCompactingFiles.add(newDatafile);
+
+      tablet.computeNumEntries();
+
+      lastLocation = tablet.resetLastLocation();
+
+      tablet.setLastCompactionID(compactionId);
+      t2 = System.currentTimeMillis();
+    }
+
+    if (!extent.isRootTablet()) {
+      Set<FileRef> filesInUseByScans = waitForScansToFinish(oldDatafiles, false, 10000);
+      if (filesInUseByScans.size() > 0)
+        log.debug("Adding scan refs to metadata " + extent + " " + filesInUseByScans);
+      MasterMetadataUtil.replaceDatafiles(extent, oldDatafiles, filesInUseByScans, newDatafile, compactionId, dfv, SystemCredentials.get(),
+          tablet.getTabletServer().getClientAddressString(), lastLocation, tablet.getTabletServer().getLock());
+      removeFilesAfterScan(filesInUseByScans);
+    }
+
+    log.debug(String.format("MajC finish lock %.2f secs", (t2 - t1) / 1000.0));
+    log.log(TLevel.TABLET_HIST, extent + " MajC " + oldDatafiles + " --> " + newDatafile);
+  }
+
+  public SortedMap<FileRef,DataFileValue> getDatafileSizes() {
+    synchronized (tablet) {
+      TreeMap<FileRef,DataFileValue> copy = new TreeMap<FileRef,DataFileValue>(datafileSizes);
+      return Collections.unmodifiableSortedMap(copy);
+    }
+  }
+
+  public Set<FileRef> getFiles() {
+    synchronized (tablet) {
+      HashSet<FileRef> files = new HashSet<FileRef>(datafileSizes.keySet());
+      return Collections.unmodifiableSet(files);
+    }
+  }
+  
+  public int getNumFiles() {
+    return datafileSizes.size();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7db2abf1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/KVEntry.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/KVEntry.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/KVEntry.java
new file mode 100644
index 0000000..4919be9
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/KVEntry.java
@@ -0,0 +1,39 @@
+/*
+ * 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.tserver.tablet;
+
+import java.util.Arrays;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyValue;
+import org.apache.accumulo.core.data.Value;
+
+public class KVEntry extends KeyValue {
+  private static final long serialVersionUID = 1L;
+
+  public KVEntry(Key k, Value v) {
+    super(new Key(k), Arrays.copyOf(v.get(), v.get().length));
+  }
+
+  int numBytes() {
+    return getKey().getSize() + getValue().get().length;
+  }
+
+  int estimateMemoryUsed() {
+    return getKey().getSize() + getValue().get().length + (9 * 32); // overhead is 32 per object
+  }
+}
\ No newline at end of file


[26/35] git commit: ACCUMULO-2255 pulled 6 classes out of TabletServer, moved GC checking to a new class, add "final" to many members, grouped members together, changed many public items to private

Posted by el...@apache.org.
ACCUMULO-2255 pulled 6 classes out of TabletServer, moved GC checking to a new class, add "final" to many members, grouped members together, changed many public items to private


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

Branch: refs/heads/ACCUMULO-378
Commit: 2cad992911a077eca351747a5ca829f0fdf1c61a
Parents: f280e97
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Jun 4 15:58:24 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Jun 4 15:58:24 2014 -0400

----------------------------------------------------------------------
 .../server/GarbageCollectionLogger.java         | 103 ++++
 .../apache/accumulo/tserver/TabletServer.java   | 595 +++++--------------
 .../accumulo/tserver/TservConstraintEnv.java    |  83 +++
 .../apache/accumulo/tserver/log/DfsLogger.java  |  13 +-
 .../tserver/session/ConditionalSession.java     |  41 ++
 .../tserver/session/MultiScanSession.java       |  62 ++
 .../accumulo/tserver/session/ScanSession.java   |  69 +++
 .../accumulo/tserver/session/Session.java       |  43 ++
 .../accumulo/tserver/session/UpdateSession.java |  56 ++
 .../apache/accumulo/tserver/tablet/Tablet.java  |  12 +-
 10 files changed, 630 insertions(+), 447 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2cad9929/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java b/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
new file mode 100644
index 0000000..b66d0f7
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
@@ -0,0 +1,103 @@
+/*
+ * 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.server;
+
+import java.lang.management.GarbageCollectorMXBean;
+import java.lang.management.ManagementFactory;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.util.Halt;
+import org.apache.log4j.Logger;
+
+public class GarbageCollectionLogger {
+  private static final Logger log = Logger.getLogger(GarbageCollectionLogger.class);
+  
+  private final HashMap<String,Long> prevGcTime = new HashMap<String,Long>();
+  private long lastMemorySize = 0;
+  private long gcTimeIncreasedCount = 0;
+  
+  public GarbageCollectionLogger() {
+  }
+
+  public synchronized void logGCInfo(AccumuloConfiguration conf) {
+    List<GarbageCollectorMXBean> gcmBeans = ManagementFactory.getGarbageCollectorMXBeans();
+    Runtime rt = Runtime.getRuntime();
+
+    StringBuilder sb = new StringBuilder("gc");
+
+    boolean sawChange = false;
+
+    long maxIncreaseInCollectionTime = 0;
+
+    for (GarbageCollectorMXBean gcBean : gcmBeans) {
+      Long prevTime = prevGcTime.get(gcBean.getName());
+      long pt = 0;
+      if (prevTime != null) {
+        pt = prevTime;
+      }
+
+      long time = gcBean.getCollectionTime();
+
+      if (time - pt != 0) {
+        sawChange = true;
+      }
+
+      long increaseInCollectionTime = time - pt;
+      sb.append(String.format(" %s=%,.2f(+%,.2f) secs", gcBean.getName(), time / 1000.0, increaseInCollectionTime / 1000.0));
+      maxIncreaseInCollectionTime = Math.max(increaseInCollectionTime, maxIncreaseInCollectionTime);
+      prevGcTime.put(gcBean.getName(), time);
+    }
+
+    long mem = rt.freeMemory();
+    if (maxIncreaseInCollectionTime == 0) {
+      gcTimeIncreasedCount = 0;
+    } else {
+      gcTimeIncreasedCount++;
+      if (gcTimeIncreasedCount > 3 && mem < rt.maxMemory() * 0.05) {
+        log.warn("Running low on memory");
+        gcTimeIncreasedCount = 0;
+      }
+    }
+
+    if (mem > lastMemorySize) {
+      sawChange = true;
+    }
+
+    String sign = "+";
+    if (mem - lastMemorySize <= 0) {
+      sign = "";
+    }
+
+    sb.append(String.format(" freemem=%,d(%s%,d) totalmem=%,d", mem, sign, (mem - lastMemorySize), rt.totalMemory()));
+
+    if (sawChange) {
+      log.debug(sb.toString());
+    }
+
+    final long keepAliveTimeout = conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
+    if (maxIncreaseInCollectionTime > keepAliveTimeout) {
+      Halt.halt("Garbage collection may be interfering with lock keep-alive.  Halting.", -1);
+    }
+
+    lastMemorySize = mem;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2cad9929/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 2a453a8..ee28c7f 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
@@ -20,9 +20,7 @@ import static org.apache.accumulo.server.problems.ProblemType.TABLET_LOAD;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.lang.management.GarbageCollectorMXBean;
 import java.lang.management.ManagementFactory;
-import java.net.Socket;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
@@ -78,9 +76,6 @@ import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.constraints.Constraint.Environment;
-import org.apache.accumulo.core.constraints.Violations;
-import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.ConstraintViolationSummary;
 import org.apache.accumulo.core.data.Key;
@@ -115,7 +110,6 @@ import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.security.AuthorizationContainer;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SecurityUtil;
 import org.apache.accumulo.core.security.thrift.TCredentials;
@@ -141,7 +135,6 @@ import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.ServerServices;
 import org.apache.accumulo.core.util.ServerServices.Service;
 import org.apache.accumulo.core.util.SimpleThreadPool;
-import org.apache.accumulo.core.util.Stat;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
@@ -150,6 +143,7 @@ import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockWatcher;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.server.Accumulo;
+import org.apache.accumulo.server.GarbageCollectionLogger;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.client.ClientServiceHandler;
@@ -211,16 +205,21 @@ import org.apache.accumulo.tserver.metrics.TabletServerMBean;
 import org.apache.accumulo.tserver.metrics.TabletServerMinCMetrics;
 import org.apache.accumulo.tserver.metrics.TabletServerScanMetrics;
 import org.apache.accumulo.tserver.metrics.TabletServerUpdateMetrics;
+import org.apache.accumulo.tserver.session.ConditionalSession;
+import org.apache.accumulo.tserver.session.MultiScanSession;
+import org.apache.accumulo.tserver.session.ScanSession;
+import org.apache.accumulo.tserver.session.Session;
+import org.apache.accumulo.tserver.session.UpdateSession;
 import org.apache.accumulo.tserver.tablet.CommitSession;
 import org.apache.accumulo.tserver.tablet.CompactionInfo;
 import org.apache.accumulo.tserver.tablet.CompactionWatcher;
 import org.apache.accumulo.tserver.tablet.Compactor;
 import org.apache.accumulo.tserver.tablet.KVEntry;
-import org.apache.accumulo.tserver.tablet.Tablet.LookupResult;
 import org.apache.accumulo.tserver.tablet.ScanBatch;
 import org.apache.accumulo.tserver.tablet.Scanner;
 import org.apache.accumulo.tserver.tablet.SplitInfo;
 import org.apache.accumulo.tserver.tablet.Tablet;
+import org.apache.accumulo.tserver.tablet.Tablet.LookupResult;
 import org.apache.accumulo.tserver.tablet.TabletClosedException;
 import org.apache.commons.collections.map.LRUMap;
 import org.apache.hadoop.fs.FSError;
@@ -237,37 +236,78 @@ import org.apache.zookeeper.KeeperException.NoNodeException;
 
 import com.google.common.net.HostAndPort;
 
-enum ScanRunState {
-  QUEUED, RUNNING, FINISHED
-}
-
 public class TabletServer extends AbstractMetricsImpl implements org.apache.accumulo.tserver.metrics.TabletServerMBean {
-  private static final Logger log = Logger.getLogger(TabletServer.class);
-
-  private static HashMap<String,Long> prevGcTime = new HashMap<String,Long>();
-  private static long lastMemorySize = 0;
-  private static long gcTimeIncreasedCount;
+  static enum ScanRunState {
+    QUEUED, RUNNING, FINISHED
+  }
 
+  private static final Logger log = Logger.getLogger(TabletServer.class);
+  private static final String METRICS_PREFIX = "tserver";
   private static final long MAX_TIME_TO_WAIT_FOR_SCAN_RESULT_MILLIS = 1000;
   private static final long RECENTLY_SPLIT_MILLIES = 60 * 1000;
+  private static final Set<Column> EMPTY_COLUMNS = Collections.emptySet();
+
+  private final GarbageCollectionLogger gcLogger = new GarbageCollectionLogger();
+  private final TransactionWatcher watcher = new TransactionWatcher();
+  private final ZooCache masterLockCache = new ZooCache();
 
-  private TabletServerLogger logger;
+  private final TabletServerLogger logger;
 
-  protected final TabletServerMinCMetrics mincMetrics = new TabletServerMinCMetrics();
+  private final TabletServerMinCMetrics mincMetrics = new TabletServerMinCMetrics();
   public TabletServerMinCMetrics getMinCMetrics() {
     return mincMetrics;
   }
 
-  private ServerConfiguration serverConfig;
-  private LogSorter logSorter = null;
+  private final ServerConfiguration serverConfig;
+  private final LogSorter logSorter;
+  private final TabletStatsKeeper statsKeeper;
+  private final AtomicInteger logIdGenerator = new AtomicInteger();
+  
+  private final VolumeManager fs;
+  public Instance getInstance() {
+    return serverConfig.getInstance();
+  }
+
+  private final SortedMap<KeyExtent,Tablet> onlineTablets = Collections.synchronizedSortedMap(new TreeMap<KeyExtent,Tablet>());
+  private final SortedSet<KeyExtent> unopenedTablets = Collections.synchronizedSortedSet(new TreeSet<KeyExtent>());
+  private final SortedSet<KeyExtent> openingTablets = Collections.synchronizedSortedSet(new TreeSet<KeyExtent>());
+  @SuppressWarnings("unchecked")
+  private final Map<KeyExtent,Long> recentlyUnloadedCache = Collections.synchronizedMap(new LRUMap(1000));
+
+  private final TabletServerResourceManager resourceManager;
+  private final SecurityOperation security;
+
+  private final BlockingDeque<MasterMessage> masterMessages = new LinkedBlockingDeque<MasterMessage>();
+
+  private Thread majorCompactorThread;
+
+  private HostAndPort clientAddress;
+
+  private volatile boolean serverStopRequested = false;
+  private volatile boolean majorCompactorDisabled = false;
+  private volatile boolean shutdownComplete = false;
+
+  private ZooLock tabletServerLock;
+
+  private TServer server;
+
+  private DistributedWorkQueue bulkFailedCopyQ;
+
+  private String lockID;
+
+  private static ObjectName OBJECT_NAME = null;
+
+  public static final AtomicLong seekCount = new AtomicLong(0);
+  
+  private final AtomicLong totalMinorCompactions = new AtomicLong(0);
 
   public TabletServer(ServerConfiguration conf, VolumeManager fs) {
     super();
     this.serverConfig = conf;
-    this.instance = conf.getInstance();
     this.fs = fs;
     AccumuloConfiguration aconf = getSystemConfiguration();
-    this.logSorter = new LogSorter(instance, fs, aconf);
+    this.logSorter = new LogSorter(getInstance(), fs, aconf);
+    this.statsKeeper = new TabletStatsKeeper();
     SimpleTimer.getInstance(aconf).schedule(new Runnable() {
       @Override
       public void run() {
@@ -282,94 +322,27 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         }
       }
     }, 5000, 5000);
-  }
-
-  private synchronized static void logGCInfo(AccumuloConfiguration conf) {
-    List<GarbageCollectorMXBean> gcmBeans = ManagementFactory.getGarbageCollectorMXBeans();
-    Runtime rt = Runtime.getRuntime();
-
-    StringBuilder sb = new StringBuilder("gc");
-
-    boolean sawChange = false;
-
-    long maxIncreaseInCollectionTime = 0;
 
-    for (GarbageCollectorMXBean gcBean : gcmBeans) {
-      Long prevTime = prevGcTime.get(gcBean.getName());
-      long pt = 0;
-      if (prevTime != null) {
-        pt = prevTime;
-      }
-
-      long time = gcBean.getCollectionTime();
-
-      if (time - pt != 0) {
-        sawChange = true;
-      }
-
-      long increaseInCollectionTime = time - pt;
-      sb.append(String.format(" %s=%,.2f(+%,.2f) secs", gcBean.getName(), time / 1000.0, increaseInCollectionTime / 1000.0));
-      maxIncreaseInCollectionTime = Math.max(increaseInCollectionTime, maxIncreaseInCollectionTime);
-      prevGcTime.put(gcBean.getName(), time);
-    }
-
-    long mem = rt.freeMemory();
-    if (maxIncreaseInCollectionTime == 0) {
-      gcTimeIncreasedCount = 0;
-    } else {
-      gcTimeIncreasedCount++;
-      if (gcTimeIncreasedCount > 3 && mem < rt.maxMemory() * 0.05) {
-        log.warn("Running low on memory");
-        gcTimeIncreasedCount = 0;
-      }
-    }
-
-    if (mem > lastMemorySize) {
-      sawChange = true;
-    }
-
-    String sign = "+";
-    if (mem - lastMemorySize <= 0) {
-      sign = "";
-    }
-
-    sb.append(String.format(" freemem=%,d(%s%,d) totalmem=%,d", mem, sign, (mem - lastMemorySize), rt.totalMemory()));
-
-    if (sawChange) {
-      log.debug(sb.toString());
-    }
-
-    final long keepAliveTimeout = conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
-    if (maxIncreaseInCollectionTime > keepAliveTimeout) {
-      Halt.halt("Garbage collection may be interfering with lock keep-alive.  Halting.", -1);
-    }
-
-    lastMemorySize = mem;
-  }
-
-  private TabletStatsKeeper statsKeeper;
-
-  private static class Session {
-    long lastAccessTime;
-    long startTime;
-    String user;
-    String client = TServerUtils.clientAddress.get();
-    public boolean reserved;
+    security = AuditedSecurityOperation.getInstance();
 
-    public void cleanup() {}
+    long walogMaxSize = getSystemConfiguration().getMemoryInBytes(Property.TSERV_WALOG_MAX_SIZE);
+    long minBlockSize = CachedConfiguration.getInstance().getLong("dfs.namenode.fs-limits.min-block-size", 0);
+    if (minBlockSize != 0 && minBlockSize > walogMaxSize)
+      throw new RuntimeException("Unable to start TabletServer. Logger is set to use blocksize " + walogMaxSize + " but hdfs minimum block size is "
+          + minBlockSize + ". Either increase the " + Property.TSERV_WALOG_MAX_SIZE + " or decrease dfs.namenode.fs-limits.min-block-size in hdfs-site.xml.");
+    logger = new TabletServerLogger(this, walogMaxSize);
+    this.resourceManager = new TabletServerResourceManager(getInstance(), fs);
   }
 
-  private static class SessionManager {
+  public static class SessionManager {
 
-    SecureRandom random;
-    Map<Long,Session> sessions;
-    long maxIdle;
-    AccumuloConfiguration aconf;
+    private final SecureRandom random = new SecureRandom();
+    private final Map<Long,Session> sessions = new HashMap<Long,Session>();
+    private final long maxIdle;
+    private final AccumuloConfiguration aconf;
 
     SessionManager(AccumuloConfiguration conf) {
-      random = new SecureRandom();
-      sessions = new HashMap<Long,Session>();
-
+      aconf = conf;
       maxIdle = conf.getTimeInMillis(Property.TSERV_SESSION_MAXIDLE);
 
       Runnable r = new Runnable() {
@@ -380,7 +353,6 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       };
 
       SimpleTimer.getInstance(conf).schedule(r, 0, Math.max(maxIdle / 2, 1000));
-      aconf = conf;
     }
 
     synchronized long createSession(Session session, boolean reserve) {
@@ -482,7 +454,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     }
 
     private void sweep(long maxIdle) {
-      ArrayList<Session> sessionsToCleanup = new ArrayList<Session>();
+      List<Session> sessionsToCleanup = new ArrayList<Session>();
       synchronized (this) {
         Iterator<Session> iter = sessions.values().iterator();
         while (iter.hasNext()) {
@@ -568,7 +540,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
     public synchronized List<ActiveScan> getActiveScans() {
 
-      ArrayList<ActiveScan> activeScans = new ArrayList<ActiveScan>();
+      List<ActiveScan> activeScans = new ArrayList<ActiveScan>();
 
       long ct = System.currentTimeMillis();
 
@@ -597,7 +569,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
             }
           }
 
-          activeScans.add(new ActiveScan(ss.client, ss.user, ss.extent.getTableId().toString(), ct - ss.startTime, ct - ss.lastAccessTime, ScanType.SINGLE,
+          activeScans.add(new ActiveScan(ss.client, ss.getUser(), ss.extent.getTableId().toString(), ct - ss.startTime, ct - ss.lastAccessTime, ScanType.SINGLE,
               state, ss.extent.toThrift(), Translator.translate(ss.columnSet, Translators.CT), ss.ssiList, ss.ssio, ss.auths.getAuthorizationsBB()));
 
         } else if (session instanceof MultiScanSession) {
@@ -623,7 +595,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
             }
           }
 
-          activeScans.add(new ActiveScan(mss.client, mss.user, mss.threadPoolExtent.getTableId().toString(), ct - mss.startTime, ct - mss.lastAccessTime,
+          activeScans.add(new ActiveScan(mss.client, mss.getUser(), mss.threadPoolExtent.getTableId().toString(), ct - mss.startTime, ct - mss.lastAccessTime,
               ScanType.BATCH, state, mss.threadPoolExtent.toThrift(), Translator.translate(mss.columnSet, Translators.CT), mss.ssiList, mss.ssio, mss.auths
                   .getAuthorizationsBB()));
         }
@@ -633,61 +605,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     }
   }
 
-  public static class TservConstraintEnv implements Environment {
-
-    private TCredentials credentials;
-    private SecurityOperation security;
-    private Authorizations auths;
-    private KeyExtent ke;
-
-    TservConstraintEnv(SecurityOperation secOp, TCredentials credentials) {
-      this.security = secOp;
-      this.credentials = credentials;
-    }
-
-    public void setExtent(KeyExtent ke) {
-      this.ke = ke;
-    }
-
-    @Override
-    public KeyExtent getExtent() {
-      return ke;
-    }
-
-    @Override
-    public String getUser() {
-      return credentials.getPrincipal();
-    }
-
-    @Override
-    @Deprecated
-    public Authorizations getAuthorizations() {
-      if (auths == null)
-        try {
-          this.auths = security.getUserAuthorizations(credentials);
-        } catch (ThriftSecurityException e) {
-          throw new RuntimeException(e);
-        }
-      return auths;
-    }
-
-    @Override
-    public AuthorizationContainer getAuthorizationsContainer() {
-      return new AuthorizationContainer() {
-        @Override
-        public boolean contains(ByteSequence auth) {
-          try {
-            return security.userHasAuthorizations(credentials,
-                Collections.<ByteBuffer> singletonList(ByteBuffer.wrap(auth.getBackingArray(), auth.offset(), auth.length())));
-          } catch (ThriftSecurityException e) {
-            throw new RuntimeException(e);
-          }
-        }
-      };
-    }
-  }
-
-  private abstract class ScanTask<T> implements RunnableFuture<T> {
+  public abstract class ScanTask<T> implements RunnableFuture<T> {
 
     protected AtomicBoolean interruptFlag;
     protected ArrayBlockingQueue<Object> resultQueue;
@@ -785,94 +703,14 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
   }
 
-  private static class ConditionalSession extends Session {
-    public TCredentials credentials;
-    public Authorizations auths;
-    public String tableId;
-    public AtomicBoolean interruptFlag;
-
-    @Override
-    public void cleanup() {
-      interruptFlag.set(true);
-    }
-  }
-
-  private static class UpdateSession extends Session {
-    public Tablet currentTablet;
-    public MapCounter<Tablet> successfulCommits = new MapCounter<Tablet>();
-    Map<KeyExtent,Long> failures = new HashMap<KeyExtent,Long>();
-    HashMap<KeyExtent,SecurityErrorCode> authFailures = new HashMap<KeyExtent,SecurityErrorCode>();
-    public Violations violations;
-    public TCredentials credentials;
-    public long totalUpdates = 0;
-    public long flushTime = 0;
-    Stat prepareTimes = new Stat();
-    Stat walogTimes = new Stat();
-    Stat commitTimes = new Stat();
-    Stat authTimes = new Stat();
-    public Map<Tablet,List<Mutation>> queuedMutations = new HashMap<Tablet,List<Mutation>>();
-    public long queuedMutationSize = 0;
-    TservConstraintEnv cenv = null;
-  }
-
-  private static class ScanSession extends Session {
-    public KeyExtent extent;
-    public HashSet<Column> columnSet;
-    public List<IterInfo> ssiList;
-    public Map<String,Map<String,String>> ssio;
-    public Authorizations auths;
-    public long entriesReturned = 0;
-    public Stat nbTimes = new Stat();
-    public long batchCount = 0;
-    public volatile ScanTask<ScanBatch> nextBatchTask;
-    public AtomicBoolean interruptFlag;
-    public Scanner scanner;
-    public long readaheadThreshold = Constants.SCANNER_DEFAULT_READAHEAD_THRESHOLD;
-
-    @Override
-    public void cleanup() {
-      try {
-        if (nextBatchTask != null)
-          nextBatchTask.cancel(true);
-      } finally {
-        if (scanner != null)
-          scanner.close();
-      }
-    }
-
-  }
-
-  private static class MultiScanSession extends Session {
-    HashSet<Column> columnSet;
-    Map<KeyExtent,List<Range>> queries;
-    public List<IterInfo> ssiList;
-    public Map<String,Map<String,String>> ssio;
-    public Authorizations auths;
-
-    // stats
-    int numRanges;
-    int numTablets;
-    int numEntries;
-    long totalLookupTime;
-
-    public volatile ScanTask<MultiScanResult> lookupTask;
-    public KeyExtent threadPoolExtent;
-
-    @Override
-    public void cleanup() {
-      if (lookupTask != null)
-        lookupTask.cancel(true);
-    }
-  }
-
   /**
    * This little class keeps track of writes in progress and allows readers to wait for writes that started before the read. It assumes that the operation ids
    * are monotonically increasing.
    *
    */
   static class WriteTracker {
-    private static AtomicLong operationCounter = new AtomicLong(1);
-    private Map<TabletType,TreeSet<Long>> inProgressWrites = new EnumMap<TabletType,TreeSet<Long>>(TabletType.class);
+    private static final AtomicLong operationCounter = new AtomicLong(1);
+    private final Map<TabletType,TreeSet<Long>> inProgressWrites = new EnumMap<TabletType,TreeSet<Long>>(TabletType.class);
 
     WriteTracker() {
       for (TabletType ttype : TabletType.values()) {
@@ -920,7 +758,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       if (keySet.size() == 0)
         return -1;
 
-      ArrayList<KeyExtent> extents = new ArrayList<KeyExtent>(keySet.size());
+      List<KeyExtent> extents = new ArrayList<KeyExtent>(keySet.size());
 
       for (Tablet tablet : keySet)
         extents.add(tablet.getExtent());
@@ -933,26 +771,23 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     return serverConfig.getConfiguration();
   }
 
-  TransactionWatcher watcher = new TransactionWatcher();
-
   private class ThriftClientHandler extends ClientServiceHandler implements TabletClientService.Iface {
 
-    SessionManager sessionManager;
+    private final SessionManager sessionManager = new SessionManager(getSystemConfiguration());
 
-    AccumuloConfiguration acuConf = getSystemConfiguration();
+    private final AccumuloConfiguration acuConf = getSystemConfiguration();
 
-    TabletServerUpdateMetrics updateMetrics = new TabletServerUpdateMetrics();
+    private final TabletServerUpdateMetrics updateMetrics = new TabletServerUpdateMetrics();
 
-    TabletServerScanMetrics scanMetrics = new TabletServerScanMetrics();
+    private final TabletServerScanMetrics scanMetrics = new TabletServerScanMetrics();
 
-    WriteTracker writeTracker = new WriteTracker();
+    private final WriteTracker writeTracker = new WriteTracker();
 
-    private RowLocks rowLocks = new RowLocks();
+    private final RowLocks rowLocks = new RowLocks();
 
     ThriftClientHandler() {
-      super(instance, watcher, fs);
+      super(getInstance(), watcher, fs);
       log.debug(ThriftClientHandler.class.getName() + " created");
-      sessionManager = new SessionManager(getSystemConfiguration());
       // Register the metrics MBean
       try {
         updateMetrics.register();
@@ -1023,7 +858,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
           runState.set(ScanRunState.RUNNING);
 
           Thread.currentThread().setName(
-              "User: " + scanSession.user + " Start: " + scanSession.startTime + " Client: " + scanSession.client + " Tablet: " + scanSession.extent);
+              "User: " + scanSession.getUser() + " Start: " + scanSession.startTime + " Client: " + scanSession.client + " Tablet: " + scanSession.extent);
 
           Tablet tablet = onlineTablets.get(scanSession.extent);
 
@@ -1065,7 +900,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
     private class LookupTask extends ScanTask<MultiScanResult> {
 
-      private long scanID;
+      private final long scanID;
 
       LookupTask(long scanID) {
         this.scanID = scanID;
@@ -1080,20 +915,20 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
           if (isCancelled() || session == null)
             return;
 
-          TableConfiguration acuTableConf = ServerConfiguration.getTableConfiguration(instance, session.threadPoolExtent.getTableId().toString());
+          TableConfiguration acuTableConf = ServerConfiguration.getTableConfiguration(getInstance(), session.threadPoolExtent.getTableId().toString());
           long maxResultsSize = acuTableConf.getMemoryInBytes(Property.TABLE_SCAN_MAXMEM);
 
           runState.set(ScanRunState.RUNNING);
-          Thread.currentThread().setName("Client: " + session.client + " User: " + session.user + " Start: " + session.startTime + " Table: ");
+          Thread.currentThread().setName("Client: " + session.client + " User: " + session.getUser() + " Start: " + session.startTime + " Table: ");
 
           long bytesAdded = 0;
           long maxScanTime = 4000;
 
           long startTime = System.currentTimeMillis();
 
-          ArrayList<KVEntry> results = new ArrayList<KVEntry>();
+          List<KVEntry> results = new ArrayList<KVEntry>();
           Map<KeyExtent,List<Range>> failures = new HashMap<KeyExtent,List<Range>>();
-          ArrayList<KeyExtent> fullScans = new ArrayList<KeyExtent>();
+          List<KeyExtent> fullScans = new ArrayList<KeyExtent>();
           KeyExtent partScan = null;
           Key partNextKey = null;
           boolean partNextKeyInclusive = false;
@@ -1113,7 +948,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
               continue;
             }
             Thread.currentThread().setName(
-                "Client: " + session.client + " User: " + session.user + " Start: " + session.startTime + " Tablet: " + entry.getKey().toString());
+                "Client: " + session.client + " User: " + session.getUser() + " Start: " + session.startTime + " Tablet: " + entry.getKey().toString());
 
             LookupResult lookupResult;
             try {
@@ -1193,13 +1028,13 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         long readaheadThreshold) throws NotServingTabletException, ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException {
 
       String tableId = new String(textent.getTable(), StandardCharsets.UTF_8);
-      if (!security.canScan(credentials, tableId, Tables.getNamespaceId(instance, tableId), range, columns, ssiList, ssio, authorizations))
+      if (!security.canScan(credentials, tableId, Tables.getNamespaceId(getInstance(), tableId), range, columns, ssiList, ssio, authorizations))
         throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
       if (!security.userHasAuthorizations(credentials, authorizations))
         throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS);
 
-      KeyExtent extent = new KeyExtent(textent);
+      final KeyExtent extent = new KeyExtent(textent);
 
       // wait for any writes that are in flight.. this done to ensure
       // consistency across client restarts... assume a client writes
@@ -1218,20 +1053,11 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       if (tablet == null)
         throw new NotServingTabletException(textent);
 
-      ScanSession scanSession = new ScanSession();
-      scanSession.user = credentials.getPrincipal();
-      scanSession.extent = new KeyExtent(extent);
-      scanSession.columnSet = new HashSet<Column>();
-      scanSession.ssiList = ssiList;
-      scanSession.ssio = ssio;
-      scanSession.auths = new Authorizations(authorizations);
-      scanSession.interruptFlag = new AtomicBoolean();
-      scanSession.readaheadThreshold = readaheadThreshold;
-
+      Set<Column> columnSet = new HashSet<Column>();
       for (TColumn tcolumn : columns) {
-        scanSession.columnSet.add(new Column(tcolumn));
+        columnSet.add(new Column(tcolumn));
       }
-
+      final ScanSession scanSession = new ScanSession(credentials, extent, columnSet, ssiList, ssio, new Authorizations(authorizations), readaheadThreshold);
       scanSession.scanner = tablet.createScanner(new Range(range), batchSize, scanSession.columnSet, scanSession.auths, ssiList, ssio, isolated,
           scanSession.interruptFlag);
 
@@ -1324,7 +1150,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
     @Override
     public void closeScan(TInfo tinfo, long scanID) {
-      ScanSession ss = (ScanSession) sessionManager.removeSession(scanID);
+      final ScanSession ss = (ScanSession) sessionManager.removeSession(scanID);
       if (ss != null) {
         long t2 = System.currentTimeMillis();
 
@@ -1341,7 +1167,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     public InitialMultiScan startMultiScan(TInfo tinfo, TCredentials credentials, Map<TKeyExtent,List<TRange>> tbatch, List<TColumn> tcolumns,
         List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites) throws ThriftSecurityException {
       // find all of the tables that need to be scanned
-      HashSet<String> tables = new HashSet<String>();
+      final HashSet<String> tables = new HashSet<String>();
       for (TKeyExtent keyExtent : tbatch.keySet()) {
         tables.add(new String(keyExtent.getTable(), StandardCharsets.UTF_8));
       }
@@ -1351,7 +1177,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
       // check if user has permission to the tables
       for (String tableId : tables)
-        if (!security.canScan(credentials, tableId, Tables.getNamespaceId(instance, tableId), tbatch, tcolumns, ssiList, ssio, authorizations))
+        if (!security.canScan(credentials, tableId, Tables.getNamespaceId(getInstance(), tableId), tbatch, tcolumns, ssiList, ssio, authorizations))
           throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
       try {
@@ -1370,13 +1196,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       if (waitForWrites)
         writeTracker.waitForWrites(TabletType.type(batch.keySet()));
 
-      MultiScanSession mss = new MultiScanSession();
-      mss.user = credentials.getPrincipal();
-      mss.queries = batch;
-      mss.columnSet = new HashSet<Column>(tcolumns.size());
-      mss.ssiList = ssiList;
-      mss.ssio = ssio;
-      mss.auths = new Authorizations(authorizations);
+      final MultiScanSession mss = new MultiScanSession(credentials, threadPoolExtent, batch, ssiList, ssio, new Authorizations(authorizations));
 
       mss.numTablets = batch.size();
       for (List<Range> ranges : batch.values()) {
@@ -1386,8 +1206,6 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       for (TColumn tcolumn : tcolumns)
         mss.columnSet.add(new Column(tcolumn));
 
-      mss.threadPoolExtent = threadPoolExtent;
-
       long sid = sessionManager.createSession(mss, true);
 
       MultiScanResult result;
@@ -1464,13 +1282,8 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       if (updateMetrics.isEnabled())
         updateMetrics.add(TabletServerUpdateMetrics.permissionErrors, 0);
 
-      UpdateSession us = new UpdateSession();
-      us.violations = new Violations();
-      us.credentials = credentials;
-      us.cenv = new TservConstraintEnv(security, us.credentials);
-
+      UpdateSession us = new UpdateSession(new TservConstraintEnv(security, credentials), credentials);
       long sid = sessionManager.createSession(us, false);
-
       return sid;
     }
 
@@ -1488,7 +1301,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         // the failures list
         boolean sameTable = us.currentTablet != null && (us.currentTablet.getExtent().getTableId().equals(keyExtent.getTableId()));
         String tableId = keyExtent.getTableId().toString();
-        if (sameTable || security.canWrite(us.credentials, tableId, Tables.getNamespaceId(instance, tableId))) {
+        if (sameTable || security.canWrite(us.getCredentials(), tableId, Tables.getNamespaceId(getInstance(), tableId))) {
           long t2 = System.currentTimeMillis();
           us.authTimes.addStat(t2 - t1);
           us.currentTablet = onlineTablets.get(keyExtent);
@@ -1502,7 +1315,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
               updateMetrics.add(TabletServerUpdateMetrics.unknownTabletErrors, 0);
           }
         } else {
-          log.warn("Denying access to table " + keyExtent.getTableId() + " for user " + us.credentials.getPrincipal());
+          log.warn("Denying access to table " + keyExtent.getTableId() + " for user " + us.getUser());
           long t2 = System.currentTimeMillis();
           us.authTimes.addStat(t2 - t1);
           us.currentTablet = null;
@@ -1512,7 +1325,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
           return;
         }
       } catch (ThriftSecurityException e) {
-        log.error("Denying permission to check user " + us.credentials.getPrincipal() + " with user " + e.getUser(), e);
+        log.error("Denying permission to check user " + us.getUser() + " with user " + e.getUser(), e);
         long t2 = System.currentTimeMillis();
         us.authTimes.addStat(t2 - t1);
         us.currentTablet = null;
@@ -1667,11 +1480,9 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
             if (extent == us.currentTablet.getExtent()) {
               // because constraint violations may filter out some
-              // mutations, for proper
-              // accounting with the client code, need to increment
-              // the count based
-              // on the original number of mutations from the client
-              // NOT the filtered number
+              // mutations, for proper accounting with the client code, 
+              // need to increment the count based on the original 
+              // number of mutations from the client NOT the filtered number
               us.successfulCommits.increment(us.currentTablet, us.queuedMutations.get(us.currentTablet).size());
             }
           }
@@ -1711,7 +1522,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
     @Override
     public UpdateErrors closeUpdate(TInfo tinfo, long updateID) throws NoSuchScanIDException {
-      UpdateSession us = (UpdateSession) sessionManager.removeSession(updateID);
+      final UpdateSession us = (UpdateSession) sessionManager.removeSession(updateID);
       if (us == null) {
         throw new NoSuchScanIDException();
       }
@@ -1752,11 +1563,11 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     public void update(TInfo tinfo, TCredentials credentials, TKeyExtent tkeyExtent, TMutation tmutation) throws NotServingTabletException,
         ConstraintViolationException, ThriftSecurityException {
 
-      String tableId = new String(tkeyExtent.getTable(), StandardCharsets.UTF_8);
-      if (!security.canWrite(credentials, tableId, Tables.getNamespaceId(instance, tableId)))
+      final String tableId = new String(tkeyExtent.getTable(), StandardCharsets.UTF_8);
+      if (!security.canWrite(credentials, tableId, Tables.getNamespaceId(getInstance(), tableId)))
         throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
-      KeyExtent keyExtent = new KeyExtent(tkeyExtent);
-      Tablet tablet = onlineTablets.get(new KeyExtent(keyExtent));
+      final KeyExtent keyExtent = new KeyExtent(tkeyExtent);
+      final Tablet tablet = onlineTablets.get(new KeyExtent(keyExtent));
       if (tablet == null) {
         throw new NotServingTabletException(tkeyExtent);
       }
@@ -1764,13 +1575,13 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       if (!keyExtent.isMeta())
         TabletServer.this.resourceManager.waitUntilCommitsAreEnabled();
 
-      long opid = writeTracker.startWrite(TabletType.type(keyExtent));
+      final long opid = writeTracker.startWrite(TabletType.type(keyExtent));
 
       try {
-        Mutation mutation = new ServerMutation(tmutation);
-        List<Mutation> mutations = Collections.singletonList(mutation);
+        final Mutation mutation = new ServerMutation(tmutation);
+        final List<Mutation> mutations = Collections.singletonList(mutation);
 
-        Span prep = Trace.start("prep");
+        final Span prep = Trace.start("prep");
         CommitSession cs;
         try {
           cs = tablet.prepareMutationsForCommit(new TservConstraintEnv(security, credentials), mutations);
@@ -1783,7 +1594,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
         while (true) {
           try {
-            Span wal = Trace.start("wal");
+            final Span wal = Trace.start("wal");
             try {
               logger.log(cs, cs.getWALogSeq(), mutation);
             } finally {
@@ -1795,7 +1606,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
           }
         }
 
-        Span commit = Trace.start("commit");
+        final Span commit = Trace.start("commit");
         try {
           cs.commit(mutations);
         } finally {
@@ -1812,18 +1623,18 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         List<String> symbols) throws IOException {
       Iterator<Entry<KeyExtent,List<ServerConditionalMutation>>> iter = updates.entrySet().iterator();
 
-      CompressedIterators compressedIters = new CompressedIterators(symbols);
+      final CompressedIterators compressedIters = new CompressedIterators(symbols);
 
       while (iter.hasNext()) {
-        Entry<KeyExtent,List<ServerConditionalMutation>> entry = iter.next();
-        Tablet tablet = onlineTablets.get(entry.getKey());
+        final Entry<KeyExtent,List<ServerConditionalMutation>> entry = iter.next();
+        final Tablet tablet = onlineTablets.get(entry.getKey());
 
         if (tablet == null || tablet.isClosed()) {
           for (ServerConditionalMutation scm : entry.getValue())
             results.add(new TCMResult(scm.getID(), TCMStatus.IGNORED));
           iter.remove();
         } else {
-          List<ServerConditionalMutation> okMutations = new ArrayList<ServerConditionalMutation>(entry.getValue().size());
+          final List<ServerConditionalMutation> okMutations = new ArrayList<ServerConditionalMutation>(entry.getValue().size());
 
           for (ServerConditionalMutation scm : entry.getValue()) {
             if (checkCondition(results, cs, compressedIters, tablet, scm))
@@ -1836,12 +1647,10 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       }
     }
 
-    boolean checkCondition(ArrayList<TCMResult> results, ConditionalSession cs, CompressedIterators compressedIters, Tablet tablet,
+    private boolean checkCondition(ArrayList<TCMResult> results, ConditionalSession cs, CompressedIterators compressedIters, Tablet tablet,
         ServerConditionalMutation scm) throws IOException {
       boolean add = true;
 
-      Set<Column> emptyCols = Collections.emptySet();
-
       for (TCondition tc : scm.getConditions()) {
 
         Range range;
@@ -1852,7 +1661,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
         IterConfig ic = compressedIters.decompress(tc.iterators);
 
-        Scanner scanner = tablet.createScanner(range, 1, emptyCols, cs.auths, ic.ssiList, ic.ssio, false, cs.interruptFlag);
+        Scanner scanner = tablet.createScanner(range, 1, EMPTY_COLUMNS, cs.auths, ic.ssiList, ic.ssio, false, cs.interruptFlag);
 
         try {
           ScanBatch batch = scanner.read();
@@ -2015,7 +1824,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         throws ThriftSecurityException, TException {
 
       Authorizations userauths = null;
-      if (!security.canConditionallyUpdate(credentials, tableId, Tables.getNamespaceId(instance, tableId), authorizations))
+      if (!security.canConditionallyUpdate(credentials, tableId, Tables.getNamespaceId(getInstance(), tableId), authorizations))
         throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
       userauths = security.getUserAuthorizations(credentials);
@@ -2023,11 +1832,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         if (!userauths.contains(ByteBufferUtil.toBytes(auth)))
           throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.BAD_AUTHORIZATIONS);
 
-      ConditionalSession cs = new ConditionalSession();
-      cs.auths = new Authorizations(authorizations);
-      cs.credentials = credentials;
-      cs.tableId = tableId;
-      cs.interruptFlag = new AtomicBoolean();
+      ConditionalSession cs = new ConditionalSession(credentials, new Authorizations(authorizations), tableId);
 
       long sid = sessionManager.createSession(cs, false);
       return new TConditionalSession(sid, lockID, sessionManager.getMaxIdleTime());
@@ -2099,7 +1904,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       String tableId = new String(ByteBufferUtil.toBytes(tkeyExtent.table));
       String namespaceId;
       try {
-        namespaceId = Tables.getNamespaceId(instance, tableId);
+        namespaceId = Tables.getNamespaceId(getInstance(), tableId);
       } catch (IllegalArgumentException ex) {
         // table does not exist, try to educate the client
         throw new NotServingTabletException(tkeyExtent);
@@ -2157,8 +1962,6 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       return result;
     }
 
-    private ZooCache masterLockCache = new ZooCache();
-
     private void checkPermission(TCredentials credentials, String lock, final String request) throws ThriftSecurityException {
       boolean fatal = false;
       try {
@@ -2179,7 +1982,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
           Halt.halt(1, new Runnable() {
             @Override
             public void run() {
-              logGCInfo(getSystemConfiguration());
+              gcLogger.logGCInfo(getSystemConfiguration());
             }
           });
         }
@@ -2195,13 +1998,13 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
           @Override
           public void run() {
             log.info("Tablet server no longer holds lock during checkPermission() : " + request + ", exiting");
-            logGCInfo(getSystemConfiguration());
+            gcLogger.logGCInfo(getSystemConfiguration());
           }
         });
       }
 
       if (lock != null) {
-        ZooUtil.LockID lid = new ZooUtil.LockID(ZooUtil.getRoot(instance) + Constants.ZMASTER_LOCK, lock);
+        ZooUtil.LockID lid = new ZooUtil.LockID(ZooUtil.getRoot(getInstance()) + Constants.ZMASTER_LOCK, lock);
 
         try {
           if (!ZooLock.isLockHeld(masterLockCache, lid)) {
@@ -2380,7 +2183,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
         @Override
         public void run() {
           log.info("Master requested tablet server halt");
-          logGCInfo(getSystemConfiguration());
+          gcLogger.logGCInfo(getSystemConfiguration());
           serverStopRequested = true;
           try {
             tabletServerLock.unlock();
@@ -2715,20 +2518,14 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     return tabletInfo;
   }
 
-  public long lastPingTime = System.currentTimeMillis();
-  public Socket currentMaster;
-
-  // a queue to hold messages that are to be sent back to the master
-  private BlockingDeque<MasterMessage> masterMessages = new LinkedBlockingDeque<MasterMessage>();
-
   // add a message for the main thread to send back to the master
   public void enqueueMasterMessage(MasterMessage m) {
     masterMessages.addLast(m);
   }
 
   private class UnloadTabletHandler implements Runnable {
-    private KeyExtent extent;
-    private boolean saveState;
+    private final KeyExtent extent;
+    private final boolean saveState;
 
     public UnloadTabletHandler(KeyExtent extent, boolean saveState) {
       this.extent = extent;
@@ -2818,15 +2615,15 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
   }
 
   private class AssignmentHandler implements Runnable {
-    private KeyExtent extent;
-    private int retryAttempt = 0;
+    private final KeyExtent extent;
+    private final int retryAttempt;
 
     public AssignmentHandler(KeyExtent extent) {
-      this.extent = extent;
+      this(extent, 0);
     }
 
     public AssignmentHandler(KeyExtent extent, int retryAttempt) {
-      this(extent);
+      this.extent = extent;
       this.retryAttempt = retryAttempt;
     }
 
@@ -2985,46 +2782,6 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     }
   }
 
-  private VolumeManager fs;
-  private Instance instance;
-
-  private final SortedMap<KeyExtent,Tablet> onlineTablets = Collections.synchronizedSortedMap(new TreeMap<KeyExtent,Tablet>());
-  private final SortedSet<KeyExtent> unopenedTablets = Collections.synchronizedSortedSet(new TreeSet<KeyExtent>());
-  private final SortedSet<KeyExtent> openingTablets = Collections.synchronizedSortedSet(new TreeSet<KeyExtent>());
-  @SuppressWarnings("unchecked")
-  private final Map<KeyExtent,Long> recentlyUnloadedCache = Collections.synchronizedMap(new LRUMap(1000));
-
-  private Thread majorCompactorThread;
-
-  // used for stopping the server and MasterListener thread
-  private volatile boolean serverStopRequested = false;
-
-  private HostAndPort clientAddress;
-
-  private TabletServerResourceManager resourceManager;
-  private SecurityOperation security;
-  private volatile boolean majorCompactorDisabled = false;
-
-  private volatile boolean shutdownComplete = false;
-
-  private ZooLock tabletServerLock;
-
-  private TServer server;
-
-  private DistributedWorkQueue bulkFailedCopyQ;
-
-  private String lockID;
-
-  private static final String METRICS_PREFIX = "tserver";
-
-  private static ObjectName OBJECT_NAME = null;
-
-  public static final AtomicLong seekCount = new AtomicLong(0);
-
-  public TabletStatsKeeper getStatsKeeper() {
-    return statsKeeper;
-  }
-
   public void addLoggersToMetadata(List<DfsLogger> logs, KeyExtent extent, int id) {
     if (!this.onlineTablets.containsKey(extent)) {
       log.info("Not adding " + logs.size() + " logs for extent " + extent + " as alias " + id + " tablet is offline");
@@ -3059,7 +2816,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
   private String getMasterAddress() {
     try {
-      List<String> locations = instance.getMasterLocations();
+      List<String> locations = getInstance().getMasterLocations();
       if (locations.size() == 0)
         return null;
       return locations.get(0);
@@ -3106,7 +2863,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
   private void announceExistence() {
     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     try {
-      String zPath = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + getClientAddressString();
+      String zPath = ZooUtil.getRoot(getInstance()) + Constants.ZTSERVERS + "/" + getClientAddressString();
 
       zoo.putPersistentData(zPath, new byte[] {}, NodeExistsPolicy.SKIP);
 
@@ -3121,7 +2878,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
             public void run() {
               if (!serverStopRequested)
                 log.fatal("Lost tablet server lock (reason = " + reason + "), exiting.");
-              logGCInfo(getSystemConfiguration());
+              gcLogger.logGCInfo(getSystemConfiguration());
             }
           });
         }
@@ -3144,7 +2901,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
         if (tabletServerLock.tryLock(lw, lockContent)) {
           log.debug("Obtained tablet server lock " + tabletServerLock.getLockPath());
-          lockID = tabletServerLock.getLockID().serialize(ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/");
+          lockID = tabletServerLock.getLockID().serialize(ZooUtil.getRoot(getInstance()) + Constants.ZTSERVERS + "/");
           return;
         }
         log.info("Waiting for tablet server lock");
@@ -3172,7 +2929,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
     ThreadPoolExecutor distWorkQThreadPool = new SimpleThreadPool(getSystemConfiguration().getCount(Property.TSERV_WORKQ_THREADS), "distributed work queue");
 
-    bulkFailedCopyQ = new DistributedWorkQueue(ZooUtil.getRoot(instance) + Constants.ZBULK_FAILED_COPYQ, getSystemConfiguration());
+    bulkFailedCopyQ = new DistributedWorkQueue(ZooUtil.getRoot(getInstance()) + Constants.ZBULK_FAILED_COPYQ, getSystemConfiguration());
     try {
       bulkFailedCopyQ.startProcessing(new BulkFailedCopyProcessor(), distWorkQThreadPool);
     } catch (Exception e1) {
@@ -3217,12 +2974,11 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
           TServiceClient client = iface;
 
           // if while loop does not execute at all and mm != null,
-          // then
-          // finally block should place mm back on queue
+          // then finally block should place mm back on queue
           while (!serverStopRequested && mm != null && client != null && client.getOutputProtocol() != null
               && client.getOutputProtocol().getTransport() != null && client.getOutputProtocol().getTransport().isOpen()) {
             try {
-              mm.send(SystemCredentials.get().toThrift(instance), getClientAddressString(), iface);
+              mm.send(SystemCredentials.get().toThrift(getInstance()), getClientAddressString(), iface);
               mm = null;
             } catch (TException ex) {
               log.warn("Error sending message: queuing message again");
@@ -3259,10 +3015,8 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
     // wait for shutdown
     // if the main thread exits oldServer the master listener, the JVM will
-    // kill the
-    // other threads and finalize objects. We want the shutdown that is
-    // running
-    // in the master listener thread to complete oldServer this happens.
+    // kill the other threads and finalize objects. We want the shutdown that is
+    // running in the master listener thread to complete oldServer this happens.
     // consider making other threads daemon threads so that objects don't
     // get prematurely finalized
     synchronized (this) {
@@ -3284,7 +3038,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       log.warn("Failed to close filesystem : " + e.getMessage(), e);
     }
 
-    logGCInfo(getSystemConfiguration());
+    gcLogger.logGCInfo(getSystemConfiguration());
 
     log.info("TServerInfo: stop requested. exiting ... ");
 
@@ -3295,8 +3049,6 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     }
   }
 
-  private long totalMinorCompactions;
-
   private static Pair<Text,KeyExtent> verifyRootTablet(KeyExtent extent, TServerInstance instance) throws DistributedStoreException, AccumuloException {
     ZooTabletStateStore store = new ZooTabletStateStore();
     if (!store.iterator().hasNext()) {
@@ -3468,15 +3220,11 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
 
   public void config(String hostname) {
     log.info("Tablet server starting on " + hostname);
-    security = AuditedSecurityOperation.getInstance();
-    clientAddress = HostAndPort.fromParts(hostname, 0);
-    long walogMaxSize = getSystemConfiguration().getMemoryInBytes(Property.TSERV_WALOG_MAX_SIZE);
-    long minBlockSize = CachedConfiguration.getInstance().getLong("dfs.namenode.fs-limits.min-block-size", 0);
-    if (minBlockSize != 0 && minBlockSize > walogMaxSize)
-      throw new RuntimeException("Unable to start TabletServer. Logger is set to use blocksize " + walogMaxSize + " but hdfs minimum block size is "
-          + minBlockSize + ". Either increase the " + Property.TSERV_WALOG_MAX_SIZE + " or decrease dfs.namenode.fs-limits.min-block-size in hdfs-site.xml.");
-    logger = new TabletServerLogger(this, walogMaxSize);
+    majorCompactorThread = new Daemon(new LoggingRunnable(log, new MajorCompactor(getSystemConfiguration())));
+    majorCompactorThread.setName("Split/MajC initiator");
+    majorCompactorThread.start();
 
+    clientAddress = HostAndPort.fromParts(hostname, 0);
     try {
       AccumuloVFSClassLoader.getContextManager().setContextConfig(new ContextManager.DefaultContextsConfig(new Iterable<Entry<String,String>>() {
         @Override
@@ -3529,7 +3277,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     Runnable gcDebugTask = new Runnable() {
       @Override
       public void run() {
-        logGCInfo(getSystemConfiguration());
+        gcLogger.logGCInfo(getSystemConfiguration());
       }
     };
 
@@ -3552,19 +3300,6 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     };
 
     SimpleTimer.getInstance(aconf).schedule(constraintTask, 0, 1000);
-
-    this.resourceManager = new TabletServerResourceManager(instance, fs);
-
-    lastPingTime = System.currentTimeMillis();
-
-    currentMaster = null;
-
-    statsKeeper = new TabletStatsKeeper();
-
-    // start major compactor
-    majorCompactorThread = new Daemon(new LoggingRunnable(log, new MajorCompactor(getSystemConfiguration())));
-    majorCompactorThread.setName("Split/MajC initiator");
-    majorCompactorThread.start();
   }
 
   public TabletServerStatus getStats(Map<String,MapCounter<ScanRunState>> scanCounts) {
@@ -3674,7 +3409,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
   }
 
   public void minorCompactionFinished(CommitSession tablet, String newDatafile, int walogSeq) throws IOException {
-    totalMinorCompactions++;
+    totalMinorCompactions.incrementAndGet();
     logger.minorCompactionFinished(tablet, newDatafile, walogSeq);
   }
 
@@ -3710,8 +3445,6 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     logger.recover(fs, extent, tconf, recoveryLogs, tabletFiles, mutationReceiver);
   }
 
-  private final AtomicInteger logIdGenerator = new AtomicInteger();
-
   public int createLogId(KeyExtent tablet) {
     AccumuloConfiguration acuTableConf = getTableConfiguration(tablet);
     if (acuTableConf.getBoolean(Property.TABLE_WALOG_ENABLED)) {
@@ -3720,7 +3453,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     return -1;
   }
 
-  // / JMX methods
+  // JMX methods
 
   @Override
   public long getEntries() {
@@ -3853,7 +3586,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
   @Override
   public long getTotalMinorCompactions() {
     if (this.isEnabled())
-      return totalMinorCompactions;
+      return totalMinorCompactions.get();
     return 0;
   }
 
@@ -3891,7 +3624,7 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
   }
 
   public TableConfiguration getTableConfiguration(KeyExtent extent) {
-    return ServerConfiguration.getTableConfiguration(instance, extent.getTableId().toString());
+    return ServerConfiguration.getTableConfiguration(getInstance(), extent.getTableId().toString());
   }
 
   public DfsLogger.ServerResources getServerConfig() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2cad9929/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java
new file mode 100644
index 0000000..dbb67a9
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TservConstraintEnv.java
@@ -0,0 +1,83 @@
+/*
+ * 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.tserver;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
+import org.apache.accumulo.core.constraints.Constraint.Environment;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.security.AuthorizationContainer;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.server.security.SecurityOperation;
+
+public class TservConstraintEnv implements Environment {
+
+  private final TCredentials credentials;
+  private final SecurityOperation security;
+  private Authorizations auths;
+  private KeyExtent ke;
+
+  TservConstraintEnv(SecurityOperation secOp, TCredentials credentials) {
+    this.security = secOp;
+    this.credentials = credentials;
+  }
+
+  public void setExtent(KeyExtent ke) {
+    this.ke = ke;
+  }
+
+  @Override
+  public KeyExtent getExtent() {
+    return ke;
+  }
+
+  @Override
+  public String getUser() {
+    return credentials.getPrincipal();
+  }
+
+  @Override
+  @Deprecated
+  public Authorizations getAuthorizations() {
+    if (auths == null)
+      try {
+        this.auths = security.getUserAuthorizations(credentials);
+      } catch (ThriftSecurityException e) {
+        throw new RuntimeException(e);
+      }
+    return auths;
+  }
+
+  @Override
+  public AuthorizationContainer getAuthorizationsContainer() {
+    return new AuthorizationContainer() {
+      @Override
+      public boolean contains(ByteSequence auth) {
+        try {
+          return security.userHasAuthorizations(credentials,
+              Collections.<ByteBuffer> singletonList(ByteBuffer.wrap(auth.getBackingArray(), auth.offset(), auth.length())));
+        } catch (ThriftSecurityException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2cad9929/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index 245ba2b..a39673f 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -330,21 +330,14 @@ public class DfsLogger {
 
       String syncMethod = conf.getConfiguration().get(Property.TSERV_WAL_SYNC_METHOD);
       try {
-        NoSuchMethodException e = null;
+        // sync: send data to datanodes
+        sync = logFile.getClass().getMethod("sync");
         try {
-          // hsync: send data to datanodes and sync the data to disk
+          // hsych: send data to datanodes and sync the data to disk
           sync = logFile.getClass().getMethod(syncMethod);
         } catch (NoSuchMethodException ex) {
           log.warn("Could not find configured " + syncMethod + " method, trying to fall back to old Hadoop sync method", ex);
-          e = ex;
         }
-        try {
-          // sync: send data to datanodes
-          sync = logFile.getClass().getMethod("sync");
-          e = null;
-        } catch (NoSuchMethodException ex) {}
-        if (e != null)
-          throw new RuntimeException(e);
       } catch (Exception e) {
         throw new RuntimeException(e);
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2cad9929/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java
new file mode 100644
index 0000000..26668f6
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ConditionalSession.java
@@ -0,0 +1,41 @@
+/*
+ * 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.tserver.session;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+
+public class ConditionalSession extends Session {
+  public final TCredentials credentials;
+  public final Authorizations auths;
+  public final String tableId;
+  public final AtomicBoolean interruptFlag = new AtomicBoolean();
+  
+  public ConditionalSession(TCredentials credentials, Authorizations authorizations, String tableId) {
+    super(credentials);
+    this.credentials = credentials;
+    this.auths = authorizations;
+    this.tableId = tableId;
+  }
+
+  @Override
+  public void cleanup() {
+    interruptFlag.set(true);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2cad9929/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java
new file mode 100644
index 0000000..5d7c1fb
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/MultiScanSession.java
@@ -0,0 +1,62 @@
+/*
+ * 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.tserver.session;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.data.Column;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.thrift.IterInfo;
+import org.apache.accumulo.core.data.thrift.MultiScanResult;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.tserver.TabletServer.ScanTask;
+
+public class MultiScanSession extends Session {
+  public final KeyExtent threadPoolExtent;
+  public final HashSet<Column> columnSet = new HashSet<Column>();
+  public final Map<KeyExtent,List<Range>> queries;
+  public final List<IterInfo> ssiList;
+  public final Map<String,Map<String,String>> ssio;
+  public final Authorizations auths;
+
+  // stats
+  public int numRanges;
+  public int numTablets;
+  public int numEntries;
+  public long totalLookupTime;
+
+  public volatile ScanTask<MultiScanResult> lookupTask;
+
+  public MultiScanSession(TCredentials credentials, KeyExtent threadPoolExtent, Map<KeyExtent,List<Range>> queries, List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, Authorizations authorizations) {
+    super(credentials);
+    this.queries = queries;
+    this.ssiList = ssiList;
+    this.ssio = ssio;
+    this.auths = authorizations;
+    this.threadPoolExtent = threadPoolExtent;
+  }
+
+  @Override
+  public void cleanup() {
+    if (lookupTask != null)
+      lookupTask.cancel(true);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2cad9929/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ScanSession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ScanSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ScanSession.java
new file mode 100644
index 0000000..72d8b67
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/ScanSession.java
@@ -0,0 +1,69 @@
+/*
+ * 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.tserver.session;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.data.Column;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.thrift.IterInfo;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.util.Stat;
+import org.apache.accumulo.tserver.TabletServer.ScanTask;
+import org.apache.accumulo.tserver.tablet.ScanBatch;
+import org.apache.accumulo.tserver.tablet.Scanner;
+
+public class ScanSession extends Session {
+  public final Stat nbTimes = new Stat();
+  public final KeyExtent extent;
+  public final Set<Column> columnSet;
+  public final List<IterInfo> ssiList;
+  public final Map<String,Map<String,String>> ssio;
+  public final Authorizations auths;
+  public final AtomicBoolean interruptFlag = new AtomicBoolean();
+  public long entriesReturned = 0;
+  public long batchCount = 0;
+  public volatile ScanTask<ScanBatch> nextBatchTask;
+  public Scanner scanner;
+  public final long readaheadThreshold;
+  
+  public ScanSession(TCredentials credentials, KeyExtent extent, Set<Column> columnSet, List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, Authorizations authorizations, long readaheadThreshold) {
+    super(credentials);
+    this.extent = extent;
+    this.columnSet = columnSet;
+    this.ssiList = ssiList;
+    this.ssio = ssio;
+    this.auths = authorizations;
+    this.readaheadThreshold = readaheadThreshold;
+  }
+
+  @Override
+  public void cleanup() {
+    try {
+      if (nextBatchTask != null)
+        nextBatchTask.cancel(true);
+    } finally {
+      if (scanner != null)
+        scanner.close();
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2cad9929/server/tserver/src/main/java/org/apache/accumulo/tserver/session/Session.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/Session.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/Session.java
new file mode 100644
index 0000000..8f85fc2
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/Session.java
@@ -0,0 +1,43 @@
+/*
+ * 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.tserver.session;
+
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.server.util.TServerUtils;
+
+public class Session {
+  public final String client;
+  public long lastAccessTime;
+  public long startTime;
+  public boolean reserved;
+  private final TCredentials credentials;
+  
+  public Session(TCredentials credentials) {
+    this.credentials = credentials;
+    this.client = TServerUtils.clientAddress.get();
+  }
+  
+  public String getUser() {
+    return credentials.getPrincipal();
+  }
+  
+  public TCredentials getCredentials() {
+    return credentials;
+  }
+
+  public void cleanup() {}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2cad9929/server/tserver/src/main/java/org/apache/accumulo/tserver/session/UpdateSession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/session/UpdateSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/UpdateSession.java
new file mode 100644
index 0000000..bc04a85
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/session/UpdateSession.java
@@ -0,0 +1,56 @@
+/*
+ * 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.tserver.session;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
+import org.apache.accumulo.core.constraints.Violations;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.util.MapCounter;
+import org.apache.accumulo.core.util.Stat;
+import org.apache.accumulo.tserver.TservConstraintEnv;
+import org.apache.accumulo.tserver.tablet.Tablet;
+
+public class UpdateSession extends Session {
+  public final TservConstraintEnv cenv;
+  public final MapCounter<Tablet> successfulCommits = new MapCounter<Tablet>();
+  public final Map<KeyExtent,Long> failures = new HashMap<KeyExtent,Long>();
+  public final HashMap<KeyExtent,SecurityErrorCode> authFailures = new HashMap<KeyExtent,SecurityErrorCode>();
+  public final Stat prepareTimes = new Stat();
+  public final Stat walogTimes = new Stat();
+  public final Stat commitTimes = new Stat();
+  public final Stat authTimes = new Stat();
+  public final Map<Tablet,List<Mutation>> queuedMutations = new HashMap<Tablet,List<Mutation>>();
+  public final Violations violations;
+  
+  public Tablet currentTablet = null;
+  public long totalUpdates = 0;
+  public long flushTime = 0;
+  public long queuedMutationSize = 0;
+  
+  public UpdateSession(TservConstraintEnv env, TCredentials credentials) {
+    super(credentials);
+    this.cenv = env;
+    this.violations = new Violations();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2cad9929/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 2be00fe..c0a0ab3 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -109,11 +109,11 @@ import org.apache.accumulo.tserver.MinorCompactionReason;
 import org.apache.accumulo.tserver.TConstraintViolationException;
 import org.apache.accumulo.tserver.TLevel;
 import org.apache.accumulo.tserver.TabletServer;
-import org.apache.accumulo.tserver.TabletServer.TservConstraintEnv;
 import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
 import org.apache.accumulo.tserver.TabletStatsKeeper;
 import org.apache.accumulo.tserver.TabletStatsKeeper.Operation;
 import org.apache.accumulo.tserver.TooManyFilesException;
+import org.apache.accumulo.tserver.TservConstraintEnv;
 import org.apache.accumulo.tserver.compaction.CompactionPlan;
 import org.apache.accumulo.tserver.compaction.CompactionStrategy;
 import org.apache.accumulo.tserver.compaction.DefaultCompactionStrategy;
@@ -637,7 +637,7 @@ public class Tablet implements TabletCommitter {
     }
   }
 
-  private LookupResult lookup(SortedKeyValueIterator<Key,Value> mmfi, List<Range> ranges, HashSet<Column> columnSet, ArrayList<KVEntry> results,
+  private LookupResult lookup(SortedKeyValueIterator<Key,Value> mmfi, List<Range> ranges, HashSet<Column> columnSet, List<KVEntry> results,
       long maxResultsSize) throws IOException {
 
     LookupResult lookupResult = new LookupResult();
@@ -714,7 +714,7 @@ public class Tablet implements TabletCommitter {
     return lookupResult;
   }
 
-  private void handleTabletClosedDuringScan(ArrayList<KVEntry> results, LookupResult lookupResult, boolean exceededMemoryUsage, Range range, int entriesAdded) {
+  private void handleTabletClosedDuringScan(List<KVEntry> results, LookupResult lookupResult, boolean exceededMemoryUsage, Range range, int entriesAdded) {
     if (exceededMemoryUsage)
       throw new IllegalStateException("tablet should not exceed memory usage or close, not both");
 
@@ -737,7 +737,7 @@ public class Tablet implements TabletCommitter {
     void receive(List<KVEntry> matches) throws IOException;
   }
 
-  public LookupResult lookup(List<Range> ranges, HashSet<Column> columns, Authorizations authorizations, ArrayList<KVEntry> results, long maxResultSize,
+  public LookupResult lookup(List<Range> ranges, HashSet<Column> columns, Authorizations authorizations, List<KVEntry> results, long maxResultSize,
       List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, AtomicBoolean interruptFlag) throws IOException {
 
     if (ranges.size() == 0) {
@@ -1286,7 +1286,7 @@ public class Tablet implements TabletCommitter {
     MinorCompactionTask mct = null;
 
     synchronized (this) {
-      if (isClosed() || isClosing() || isCloseComplete()) {
+      if (isClosed() || isClosing()) {
         String msg = "Tablet " + getExtent() + " already " + closeState;
         throw new IllegalStateException(msg);
       }
@@ -2035,7 +2035,7 @@ public class Tablet implements TabletCommitter {
   }
 
   public synchronized boolean isClosed() {
-    return closeState == CloseState.CLOSED;
+    return closeState == CloseState.CLOSED || closeState == CloseState.COMPLETE;
   }
 
   public synchronized boolean isCloseComplete() {


[05/35] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Posted by el...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT


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

Branch: refs/heads/ACCUMULO-378
Commit: 3bcea0deeb50df23eadf748614b3581bbd4ba35d
Parents: a61e242 05a64c1
Author: Josh Elser <el...@apache.org>
Authored: Mon Jun 2 23:42:00 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon Jun 2 23:42:00 2014 -0400

----------------------------------------------------------------------
 test/system/continuous/master-agitator.pl | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------