You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/07/02 16:42:02 UTC

[jira] [Commented] (DRILL-4142) Implement Node-level Memory maximums for a query.

    [ https://issues.apache.org/jira/browse/DRILL-4142?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16530161#comment-16530161 ] 

ASF GitHub Bot commented on DRILL-4142:
---------------------------------------

jacques-n closed pull request #284: DRILL-4142: Query/Node memory maximums
URL: https://github.com/apache/drill/pull/284
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/common/src/main/java/org/apache/drill/common/AutoCloseablePointer.java b/common/src/main/java/org/apache/drill/common/AutoCloseablePointer.java
new file mode 100644
index 0000000000..3a0ac4a2db
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/AutoCloseablePointer.java
@@ -0,0 +1,112 @@
+/**
+ * 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.drill.common;
+
+/**
+ * A class similar to Pointer<>, but with features unique to holding
+ * AutoCloseable pointers. The AutoCloseablePointer<> must be closed
+ * when it will no longer be used.
+ *
+ * <p>If you're familiar with C++/Boost's shared_ptr<>, you might recognize
+ * some of the features here.</p>
+ *
+ * @param <T> type of the pointer
+ */
+public final class AutoCloseablePointer<T extends AutoCloseable> implements AutoCloseable {
+  private T pointer;
+
+  /**
+   * Constructor for a null-valued pointer.
+   */
+  public AutoCloseablePointer() {
+    pointer = null;
+  }
+
+  /**
+   * Constructor for a pointer value.
+   *
+   * @param pointer the initial pointer value
+   */
+  public AutoCloseablePointer(final T pointer) {
+    this.pointer = pointer;
+  }
+
+  @Override
+  public void close() throws Exception {
+    assign(null);
+  }
+
+  /**
+   * Get the raw pointer out for use.
+   *
+   * @return the raw pointer
+   */
+  public T get() {
+    return pointer;
+  }
+
+  /**
+   * The caller adopts the pointer; the holder is set to
+   * null, and will no longer be responsible for close()ing this pointer.
+   *
+   * @return the pointer being adopted; may be null
+   */
+  public T adopt() {
+    final T p = pointer;
+    pointer = null;
+    return p;
+  }
+
+  /**
+   * Assign a new pointer to this holder. Any currently held pointer
+   * will first be closed. If closing the currently held pointer throws
+   * an exception, the new pointer is still assigned, and the holder must still
+   * be closed to close that.
+   *
+   * <p>This makes it convenient to assign a new pointer without having to check
+   * for a previous value and worry about cleaning it up; this does all that.</p>
+   *
+   * @param newP the new pointer to hold
+   * @throws Exception any exception thrown by closing the currently held
+   *   pointer
+   */
+  public void assign(final T newP) throws Exception {
+    try {
+      if (pointer != null) {
+        pointer.close();
+      }
+    } finally {
+      pointer = newP;
+    }
+  }
+
+  /**
+   * Like {@link #assign(AutoCloseable)}, except that any exception thrown
+   * by closing the previously held pointer is wrapped with (an unchecked)
+   * {@link RuntimeException}.
+   *
+   * @param newP the new pointer to hold
+   */
+  public void assignNoChecked(final T newP) {
+    try {
+      assign(newP);
+    } catch(final Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+}
diff --git a/common/src/main/java/org/apache/drill/common/AutoCloseables.java b/common/src/main/java/org/apache/drill/common/AutoCloseables.java
index 3c4aa23f28..dd80f9e3ff 100644
--- a/common/src/main/java/org/apache/drill/common/AutoCloseables.java
+++ b/common/src/main/java/org/apache/drill/common/AutoCloseables.java
@@ -47,7 +47,7 @@ public static void close(final AutoCloseable ac, final Logger logger) {
     }
   }
 
-  public static void close(AutoCloseable[] ac) throws Exception {
+  public static void close(AutoCloseable... ac) throws Exception {
     Exception topLevelException = null;
     for (AutoCloseable closeable : ac) {
       try {
diff --git a/common/src/main/java/org/apache/drill/common/DrillCloseables.java b/common/src/main/java/org/apache/drill/common/DrillCloseables.java
new file mode 100644
index 0000000000..289066b32d
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/DrillCloseables.java
@@ -0,0 +1,55 @@
+/**
+ * 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.drill.common;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Provides additional functionality to Guava's Closeables.
+ */
+public class DrillCloseables {
+  /**
+   * Constructor. Prevents construction for class of static utilities.
+   */
+  private DrillCloseables() {
+  }
+
+  /**
+   * Close() a {@see java.io.Closeable} without throwing a (checked)
+   * {@see java.io.IOException}. This wraps the close() call with a
+   * try-catch that will rethrow an IOException wrapped with a
+   * {@see java.lang.RuntimeException}, providing a way to call close()
+   * without having to do the try-catch everywhere or propagate the IOException.
+   *
+   * <p>Guava has deprecated {@see com.google.common.io.Closeables.closeQuietly()}
+   * as described in
+   * {@link https://code.google.com/p/guava-libraries/issues/detail?id=1118}.
+   *
+   * @param closeable the Closeable to close
+   * @throws RuntimeException if an IOException occurs; the IOException is
+   *   wrapped by the RuntimeException
+   */
+  public static void closeNoChecked(final Closeable closeable) {
+    try {
+      closeable.close();
+    } catch(final IOException e) {
+      throw new RuntimeException("IOException while closing", e);
+    }
+  }
+}
diff --git a/common/src/main/java/org/apache/drill/common/HistoricalLog.java b/common/src/main/java/org/apache/drill/common/HistoricalLog.java
new file mode 100644
index 0000000000..764f1a1fd6
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/HistoricalLog.java
@@ -0,0 +1,179 @@
+/**
+ * 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.drill.common;
+
+import java.util.Arrays;
+import java.util.LinkedList;
+
+import org.slf4j.Logger;
+
+/**
+ * Utility class that can be used to log activity within a class
+ * for later logging and debugging. Supports recording events and
+ * recording the stack at the time they occur.
+ */
+public class HistoricalLog {
+  private static class Event {
+    private final String note; // the event text
+    private final StackTrace stackTrace; // where the event occurred
+
+    public Event(final String note) {
+      this.note = note;
+      stackTrace = new StackTrace();
+    }
+  }
+
+  private final LinkedList<Event> history = new LinkedList<>();
+  private final String idString; // the formatted id string
+  private Event firstEvent; // the first stack trace recorded
+  private final int limit; // the limit on the number of events kept
+
+  /**
+   * Constructor. The format string will be formatted and have its arguments
+   * substituted at the time this is called.
+   *
+   * @param idStringFormat {@link String#format} format string that can be used
+   *     to identify this object in a log. Including some kind of unique identifier
+   *     that can be associated with the object instance is best.
+   * @param args for the format string, or nothing if none are required
+   */
+  public HistoricalLog(final String idStringFormat, Object... args) {
+    this(Integer.MAX_VALUE, idStringFormat, args);
+  }
+
+  /**
+   * Constructor. The format string will be formatted and have its arguments
+   * substituted at the time this is called.
+   *
+   * <p>This form supports the specification of a limit that will limit the
+   * number of historical entries kept (which keeps down the amount of memory
+   * used). With the limit, the first entry made is always kept (under the
+   * assumption that this is the creation site of the object, which is usually
+   * interesting), and then up to the limit number of entries are kept after that.
+   * Each time a new entry is made, the oldest that is not the first is dropped.
+   * </p>
+   *
+   * @param limit the maximum number of historical entries that will be kept,
+   *   not including the first entry made
+   * @param idStringFormat {@link String#format} format string that can be used
+   *     to identify this object in a log. Including some kind of unique identifier
+   *     that can be associated with the object instance is best.
+   * @param args for the format string, or nothing if none are required
+   */
+  public HistoricalLog(final int limit, final String idStringFormat, Object... args) {
+    this.limit = limit;
+    this.idString = String.format(idStringFormat, args);
+  }
+
+  /**
+   * Record an event. Automatically captures the stack trace at the time this is
+   * called. The format string will be formatted and have its arguments substituted
+   * at the time this is called.
+   *
+   * @param noteFormat {@link String#format} format string that describes the event
+   * @param args for the format string, or nothing if none are required
+   */
+  public synchronized void recordEvent(final String noteFormat, Object... args) {
+    final String note = String.format(noteFormat, args);
+    final Event event = new Event(note);
+    if (firstEvent == null) {
+      firstEvent = event;
+    }
+    if (history.size() == limit) {
+      history.removeFirst();
+    }
+    history.add(event);
+  }
+
+  /**
+   * Write the history of this object to the given {@link StringBuilder}. The history
+   * includes the identifying string provided at construction time, and all the recorded
+   * events with their stack traces.
+   *
+   * @param sb {@link StringBuilder} to write to
+   */
+  public void buildHistory(final StringBuilder sb, boolean includeStackTrace) {
+    buildHistory(sb, 0, includeStackTrace);
+  }
+
+  /**
+   * Write the history of this object to the given {@link StringBuilder}. The history
+   * includes the identifying string provided at construction time, and all the recorded
+   * events with their stack traces.
+   *
+   * @param sb {@link StringBuilder} to write to
+   * @param additional an extra string that will be written between the identifying
+   *     information and the history; often used for a current piece of state
+   */
+
+  /**
+   *
+   * @param sb
+   * @param indexLevel
+   * @param includeStackTrace
+   */
+  public synchronized void buildHistory(final StringBuilder sb, int indent, boolean includeStackTrace) {
+    final char[] indentation = new char[indent];
+    final char[] innerIndentation = new char[indent + 2];
+    Arrays.fill(indentation, ' ');
+    Arrays.fill(innerIndentation, ' ');
+
+    sb.append(indentation)
+        .append("event log for: ")
+        .append(idString)
+        .append('\n');
+
+
+    if (firstEvent != null) {
+      sb.append(innerIndentation)
+          .append(firstEvent.note)
+          .append('\n');
+      if (includeStackTrace) {
+        firstEvent.stackTrace.writeToBuilder(sb, indent + 2);
+      }
+
+      for(final Event event : history) {
+        if (event == firstEvent) {
+          continue;
+        }
+        sb.append(innerIndentation)
+            .append("  ")
+            .append(event.note)
+            .append('\n');
+
+        if (includeStackTrace) {
+          event.stackTrace.writeToBuilder(sb, indent + 2);
+          sb.append('\n');
+        }
+      }
+    }
+  }
+
+  /**
+   * Write the history of this object to the given {@link Logger}. The history
+   * includes the identifying string provided at construction time, and all the recorded
+   * events with their stack traces.
+   *
+   * @param logger {@link Logger} to write to
+   */
+  public void logHistory(final Logger logger) {
+    final StringBuilder sb = new StringBuilder();
+    buildHistory(sb, 0, true);
+    logger.debug(sb.toString());
+  }
+}
diff --git a/common/src/main/java/org/apache/drill/common/StackTrace.java b/common/src/main/java/org/apache/drill/common/StackTrace.java
index de64ed7f8e..1dbffe6caa 100644
--- a/common/src/main/java/org/apache/drill/common/StackTrace.java
+++ b/common/src/main/java/org/apache/drill/common/StackTrace.java
@@ -36,13 +36,14 @@ public StackTrace() {
 
   /**
    * Write the stack trace to a StringBuilder.
-   *
-   * @param sb where to write it
-   * @param indent how many spaces to indent each line
+   * @param sb
+   *          where to write it
+   * @param indent
+   *          how many double spaces to indent each line
    */
   public void writeToBuilder(final StringBuilder sb, final int indent) {
     // create the indentation string
-    final char[] indentation = new char[indent];
+    final char[] indentation = new char[indent * 2];
     Arrays.fill(indentation, ' ');
 
     // write the stack trace in standard Java format
diff --git a/distribution/pom.xml b/distribution/pom.xml
index 3a95a06719..c8fa860b8b 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -44,11 +44,6 @@
       <artifactId>drill-memory-base</artifactId>
       <version>${project.version}</version>
     </dependency>    
-    <dependency>
-      <groupId>org.apache.drill.memory</groupId>
-      <artifactId>drill-memory-impl</artifactId>
-      <version>${project.version}</version>
-    </dependency> 
     <dependency>
       <groupId>org.apache.drill.exec</groupId>
       <artifactId>drill-rpc</artifactId>
diff --git a/distribution/src/assemble/bin.xml b/distribution/src/assemble/bin.xml
index 69af35324d..f5fc2860dd 100644
--- a/distribution/src/assemble/bin.xml
+++ b/distribution/src/assemble/bin.xml
@@ -89,7 +89,6 @@
         <include>org.apache.drill:drill-logical:jar</include>
         <include>org.apache.drill.exec:vector:jar</include>
         <include>org.apache.drill.memory:drill-memory-base:jar</include>
-        <include>org.apache.drill.memory:drill-memory-impl:jar</include>
         <include>org.apache.drill.exec:drill-rpc:jar</include>
         <include>org.apache.drill.exec:drill-java-exec:jar</include>
         <include>org.apache.drill.contrib.storage-hive:drill-storage-hive-core</include>
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index a5d2e11da9..eff3fcd8d9 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -284,7 +284,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.drill.memory</groupId>
-      <artifactId>drill-memory-impl</artifactId>
+      <artifactId>drill-memory-base</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctionHelpers.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctionHelpers.java
index c919f28170..3d0f170b85 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctionHelpers.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctionHelpers.java
@@ -30,8 +30,6 @@
 public class StringFunctionHelpers {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StringFunctionHelpers.class);
 
-  private static final boolean BOUNDS_CHECKING_ENABLED = BoundsChecking.BOUNDS_CHECKING_ENABLED;
-
   static final int RADIX = 10;
   static final long MAX_LONG = -Long.MAX_VALUE / RADIX;
   static final int MAX_INT = -Integer.MAX_VALUE / RADIX;
@@ -212,7 +210,7 @@ public static String toStringFromUTF16(int start, int end, DrillBuf buffer) {
   private static final ISOChronology CHRONOLOGY = org.joda.time.chrono.ISOChronology.getInstanceUTC();
 
   public static long getDate(DrillBuf buf, int start, int end){
-    if(BOUNDS_CHECKING_ENABLED){
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
       buf.checkBytes(start, end);
     }
     return memGetDate(buf.memoryAddress(), start, end);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/XXHash.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/XXHash.java
index b18daed9d3..4dd30713be 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/XXHash.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/XXHash.java
@@ -27,8 +27,6 @@
 public final class XXHash {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(XXHash.class);
 
-  private static final boolean BOUNDS_CHECKING_ENABLED = BoundsChecking.BOUNDS_CHECKING_ENABLED;
-
   static final long PRIME64_1 = UnsignedLongs.decode("11400714785074694791");
   static final long PRIME64_2 = UnsignedLongs.decode("14029467366897019727");
   static final long PRIME64_3 = UnsignedLongs.decode("1609587929392839161");
@@ -168,7 +166,7 @@ public static long hash64(double val, long seed){
   }
 
   public static long hash64(int start, int end, DrillBuf buffer, long seed){
-    if(BOUNDS_CHECKING_ENABLED){
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
       buffer.checkBytes(start, end);
     }
 
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java
similarity index 53%
rename from exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountor.java
rename to exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java
index 7014a0bc7f..227604a0af 100644
--- a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java
@@ -17,27 +17,24 @@
  */
 package org.apache.drill.exec.memory;
 
-import io.netty.buffer.DrillBuf;
+import org.apache.drill.common.config.DrillConfig;
 
-public interface Accountor extends AutoCloseable{
+public class RootAllocatorFactory {
 
-  public boolean transferTo(Accountor target, DrillBuf buf, long size);
-  public boolean transferIn(DrillBuf buf, long size);
-  public long getAvailable();
-  public long getCapacity();
-  public long getAllocation();
-  public long getPeakMemoryAllocation();
+  public static final String TOP_LEVEL_MAX_ALLOC = "drill.memory.top.max";
 
-  public boolean reserve(long size);
-  public boolean forceAdditionalReservation(long size);
+  /**
+   * Constructor to prevent instantiation of this static utility class.
+   */
+  private RootAllocatorFactory() {}
 
-  public void reserved(long expected, DrillBuf buf);
-
-  public void release(DrillBuf buf, long size);
-  public void releasePartial(DrillBuf buf, long size);
-  public long resetFragmentLimits();
-  public void close();
-
-  public void setFragmentLimit(long add);
-  public long getFragmentLimit();
+  /**
+   * Create a new Root Allocator
+   * @param drillConfig
+   *          the DrillConfig
+   * @return a new root allocator
+   */
+  public static BufferAllocator newRoot(final DrillConfig drillConfig) {
+    return new RootAllocator(0, Math.min(DrillConfig.getMaxDirectMemory(), drillConfig.getLong(TOP_LEVEL_MAX_ALLOC)));
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BufferManagerImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BufferManagerImpl.java
index 003fc9b95d..b8c679623d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BufferManagerImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BufferManagerImpl.java
@@ -55,9 +55,8 @@ public DrillBuf getManagedBuffer() {
   }
 
   public DrillBuf getManagedBuffer(int size) {
-    DrillBuf newBuf = allocator.buffer(size);
+    DrillBuf newBuf = allocator.buffer(size, this);
     managedBuffers.put(newBuf.memoryAddress(), newBuf);
-    newBuf.setBufferManager(this);
     return newBuf;
   }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index a773c2275d..7544e5e425 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -35,7 +35,6 @@
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.memory.LimitConsumer;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
@@ -46,6 +45,7 @@
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.rpc.control.ControlTunnel;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.FragmentOptionManager;
@@ -56,6 +56,7 @@
 import org.apache.drill.exec.testing.ExecutionControls;
 import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.drill.exec.work.batch.IncomingBuffers;
+import org.apache.drill.exec.work.fragment.AllocatorTree.QueryAllocator;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -156,12 +157,13 @@ public FragmentContext(final DrillbitContext dbContext, final PlanFragment fragm
 
     executionControls = new ExecutionControls(fragmentOptions, dbContext.getEndpoint());
 
-    // Add the fragment context to the root allocator.
-    // The QueryManager will call the root allocator to recalculate all the memory limits for all the fragments
+    final long maxQueryMemory = context.getOptionManager().getOption(WorkEventBus.MEMORY_MAX_PER_QUERY) * 1024 * 1024;
+    final QueryAllocator queryAllocator = context.getWorkBus()
+        .getQueryAllocator(context.getAllocator(), fragment.getHandle().getQueryId(), 0, maxQueryMemory);
+
     try {
-      allocator = context.getAllocator().getChildAllocator(new AsLimitConsumer(), fragment.getMemInitial(),
-          fragment.getMemMax(), true);
-      Preconditions.checkNotNull(allocator, "Unable to acuqire allocator");
+      allocator = queryAllocator.newMinorFragmentAllocator(fragment.getHandle(), fragment.getMemInitial(),
+          fragment.getMemMax());
     } catch (final OutOfMemoryException e) {
       throw UserException.memoryError(e)
         .addContext("Fragment", getHandle().getMajorFragmentId() + ":" + getHandle().getMinorFragmentId())
@@ -174,31 +176,6 @@ public FragmentContext(final DrillbitContext dbContext, final PlanFragment fragm
     bufferManager = new BufferManagerImpl(this.allocator);
   }
 
-  private class AsLimitConsumer implements LimitConsumer {
-    final String identifier = QueryIdHelper.getFragmentId(fragment.getHandle());
-
-    @Override
-    public String getIdentifier() {
-      return identifier;
-    }
-
-    @Override
-    public long getAllocated() {
-      return allocator.getAllocatedMemory();
-    }
-
-    @Override
-    public long getLimit() {
-      return allocator.getLimit();
-    }
-
-    @Override
-    public void setLimit(long limit) {
-      allocator.setLimit(limit);
-    }
-
-  }
-
   /**
    * TODO: Remove this constructor when removing the SimpleRootExec (DRILL-2097). This is kept only to avoid modifying
    * the long list of test files.
@@ -297,11 +274,6 @@ private String getFragIdString() {
     return frag;
   }
 
-  public LimitConsumer asLimitConsumer() {
-    return new AsLimitConsumer();
-  }
-
-
   /**
    * Get this fragment's allocator.
    * @return the allocator
@@ -314,11 +286,16 @@ public BufferAllocator getAllocator() {
     return allocator;
   }
 
-  public BufferAllocator getNewChildAllocator(final long initialReservation,
-                                              final long maximumReservation,
-                                              final boolean applyFragmentLimit) throws OutOfMemoryException {
-    return allocator.getChildAllocator(new AsLimitConsumer(), initialReservation, maximumReservation,
-        applyFragmentLimit);
+  public BufferAllocator getNewChildAllocator(final String operatorName,
+      final int operatorId,
+      final long initialReservation,
+      final long maximumReservation,
+      final boolean applyFragmentLimit) throws OutOfMemoryException {
+    return allocator.newChildAllocator(
+        "op:" + QueryIdHelper.getFragmentId(fragment.getHandle()) + ":" + operatorId + ":" + operatorName,
+        initialReservation,
+        maximumReservation
+        );
   }
 
   public <T> T getImplementationClass(final ClassGenerator<T> cg)
@@ -465,6 +442,14 @@ public Executor getExecutor(){
     return context.getExecutor();
   }
 
+  public long getFragmentMemoryLimit() {
+    return allocator.getLimit();
+  }
+
+  public void setFragmentMemoryLimit(long value) {
+    allocator.setLimit(value);
+  }
+
   /**
    * Wait for ack that all outgoing batches have been sent
    */
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
index a3ec6bd09c..3502d52056 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
@@ -62,7 +62,8 @@
 
   public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, boolean applyFragmentLimit) throws OutOfMemoryException {
     this.applyFragmentLimit=applyFragmentLimit;
-    this.allocator = context.getNewChildAllocator(popConfig.getInitialAllocation(), popConfig.getMaxAllocation(), applyFragmentLimit);
+    this.allocator = context.getNewChildAllocator(popConfig.getClass().getSimpleName(),
+        popConfig.getOperatorId(), popConfig.getInitialAllocation(), popConfig.getMaxAllocation(), applyFragmentLimit);
     this.popConfig = popConfig;
     this.manager = new BufferManagerImpl(allocator);
 
@@ -74,7 +75,8 @@ public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context,
 
   public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, OperatorStats stats, boolean applyFragmentLimit) throws OutOfMemoryException {
     this.applyFragmentLimit=applyFragmentLimit;
-    this.allocator = context.getNewChildAllocator(popConfig.getInitialAllocation(), popConfig.getMaxAllocation(), applyFragmentLimit);
+    this.allocator = context.getNewChildAllocator(popConfig.getClass().getSimpleName(),
+        popConfig.getOperatorId(), popConfig.getInitialAllocation(), popConfig.getMaxAllocation(), applyFragmentLimit);
     this.popConfig = popConfig;
     this.manager = new BufferManagerImpl(allocator);
     this.stats     = stats;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index 3bcd111628..80ed745ab0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -30,13 +30,14 @@
 import org.apache.drill.common.config.LogicalPlanPersistence;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.planner.sql.DrillOperatorTable;
 import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.options.OptionManager;
@@ -52,8 +53,6 @@
 
 import com.google.common.collect.Lists;
 
-// TODO except for a couple of tests, this is only created by Foreman
-// TODO the many methods that just return drillbitContext.getXxx() should be replaced with getDrillbitContext()
 // TODO - consider re-name to PlanningContext, as the query execution context actually appears
 // in fragment contexts
 public class QueryContext implements AutoCloseable, OptimizerRulesContext {
@@ -80,7 +79,12 @@
    */
   private boolean closed = false;
 
+  @Deprecated
   public QueryContext(final UserSession session, final DrillbitContext drillbitContext) {
+    this(session, drillbitContext, QueryId.getDefaultInstance());
+  }
+
+  public QueryContext(final UserSession session, final DrillbitContext drillbitContext, QueryId queryId) {
     this.drillbitContext = drillbitContext;
     this.session = session;
     queryOptions = new QueryOptionManager(session.getOptions());
@@ -92,13 +96,10 @@ public QueryContext(final UserSession session, final DrillbitContext drillbitCon
     queryContextInfo = Utilities.createQueryContextInfo(session.getDefaultSchemaName());
     contextInformation = new ContextInformation(session.getCredentials(), queryContextInfo);
 
-    try {
-      allocator = drillbitContext.getAllocator().getChildAllocator(null, plannerSettings.getInitialPlanningMemorySize(),
-          plannerSettings.getPlanningMemoryLimit(), false);
-    } catch (OutOfMemoryException e) {
-      throw new DrillRuntimeException("Error creating off-heap allocator for planning context.",e);
-    }
-    // TODO(DRILL-1942) the new allocator has this capability built-in, so this can be removed once that is available
+    allocator = drillbitContext.getAllocator().newChildAllocator(
+        "query:" + QueryIdHelper.getQueryId(queryId),
+        PlannerSettings.getInitialPlanningMemorySize(),
+        plannerSettings.getPlanningMemoryLimit());
     bufferManager = new BufferManagerImpl(this.allocator);
     viewExpansionContext = new ViewExpansionContext(this);
     schemaTreesToClose = Lists.newArrayList();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
index 54d2839f29..c561add5db 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
@@ -22,6 +22,8 @@
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.calcite.rel.RelFieldCollation.Direction;
+import org.apache.drill.common.DrillAutoCloseables;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
 import org.apache.drill.common.expression.LogicalExpression;
@@ -58,7 +60,6 @@
 import org.apache.drill.exec.record.selection.SelectionVector4;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.complex.AbstractContainerVector;
-import org.apache.calcite.rel.RelFieldCollation.Direction;
 
 import com.google.common.base.Stopwatch;
 import com.sun.codemodel.JConditional;
@@ -308,7 +309,7 @@ private void purge() throws SchemaChangeException {
       builder.getSv4().clear();
       selectionVector4.clear();
     } finally {
-      builder.close();
+      DrillAutoCloseables.closeNoChecked(builder);
     }
     logger.debug("Took {} us to purge", watch.elapsed(TimeUnit.MICROSECONDS));
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
index e5d1e4e505..89823b4499 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
@@ -17,20 +17,15 @@
  */
 package org.apache.drill.exec.physical.impl.sort;
 
+import io.netty.buffer.DrillBuf;
+
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
-import com.google.common.collect.Sets;
-import io.netty.buffer.DrillBuf;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.AllocationReservation;
 import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.memory.BufferAllocator.PreAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
@@ -44,7 +39,6 @@
 
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Lists;
-import org.apache.drill.exec.vector.complex.UnionVector;
 
 public class SortRecordBatchBuilder implements AutoCloseable {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SortRecordBatchBuilder.class);
@@ -55,12 +49,11 @@
   private long runningBatches;
   private SelectionVector4 sv4;
   private BufferAllocator allocator;
-  final PreAllocator svAllocator;
-  private boolean svAllocatorUsed = false;
+  final AllocationReservation reservation;
 
   public SortRecordBatchBuilder(BufferAllocator a) {
     this.allocator = a;
-    this.svAllocator = a.getNewPreAllocator();
+    this.reservation = a.newReservation();
   }
 
   private long getSize(VectorAccessible batch) {
@@ -93,7 +86,7 @@ public boolean add(VectorAccessible batch) {
     if (runningBatches >= Character.MAX_VALUE) {
       return false; // allowed in batch.
     }
-    if (!svAllocator.preAllocate(batch.getRecordCount()*4)) {
+    if (!reservation.add(batch.getRecordCount() * 4)) {
       return false;  // sv allocation available.
     }
 
@@ -116,7 +109,7 @@ public void add(RecordBatchData rbd) {
       logger.error(errMsg);
       throw new DrillRuntimeException(errMsg);
     }
-    if(!svAllocator.preAllocate(rbd.getRecordCount()*4)) {
+    if (!reservation.add(rbd.getRecordCount() * 4)) {
       final String errMsg = String.format("Failed to pre-allocate memory for SV. " + "Existing recordCount*4 = %d, " +
           "incoming batch recordCount*4 = %d", recordCount * 4, rbd.getRecordCount() * 4);
       logger.error(errMsg);
@@ -159,11 +152,10 @@ public void build(FragmentContext context, VectorContainer outputContainer) thro
       assert false : "Invalid to have an empty set of batches with no schemas.";
     }
 
-    final DrillBuf svBuffer = svAllocator.getAllocation();
+    final DrillBuf svBuffer = reservation.buffer();
     if (svBuffer == null) {
       throw new OutOfMemoryError("Failed to allocate direct memory for SV4 vector in SortRecordBatchBuilder.");
     }
-    svAllocatorUsed = true;
     sv4 = new SelectionVector4(svBuffer, recordCount, Character.MAX_VALUE);
     BatchSchema schema = batches.keySet().iterator().next();
     List<RecordBatchData> data = batches.get(schema);
@@ -229,13 +221,7 @@ public void clear() {
 
   @Override
   public void close() {
-    // Don't leak unused pre-allocated memory.
-    if (!svAllocatorUsed) {
-      final DrillBuf drillBuf = svAllocator.getAllocation();
-      if (drillBuf != null) {
-        drillBuf.release();
-      }
-    }
+    reservation.close();
   }
 
   public List<VectorContainer> getHeldRecordBatches() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/trace/TraceRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/trace/TraceRecordBatch.java
index 78e83d63e1..a4fe3bd4d9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/trace/TraceRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/trace/TraceRecordBatch.java
@@ -25,6 +25,7 @@
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.cache.VectorAccessibleSerializable;
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.Trace;
 import org.apache.drill.exec.record.AbstractSingleRecordBatch;
@@ -57,6 +58,8 @@
 
   private SelectionVector2 sv = null;
 
+  private final BufferAllocator localAllocator;
+
   /* Tag associated with each trace operator */
   final String traceTag;
 
@@ -70,7 +73,7 @@ public TraceRecordBatch(Trace pop, RecordBatch incoming, FragmentContext context
     super(pop, context, incoming);
     this.traceTag = pop.traceTag;
     logLocation = context.getConfig().getString(ExecConstants.TRACE_DUMP_DIRECTORY);
-
+    localAllocator = context.getNewChildAllocator("trace", 200, 0, Long.MAX_VALUE, false);
     String fileName = getFileName();
 
     /* Create the log file we will dump to and initialize the file descriptors */
@@ -116,7 +119,7 @@ protected IterOutcome doWork() {
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    batch.reconstructContainer(container);
+    batch.reconstructContainer(localAllocator, container);
     if (incomingHasSv2) {
       sv = wrap.getSv2();
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
index 51d74cc32b..55b9afe3ae 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
@@ -37,7 +37,6 @@
 import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
 import org.apache.drill.exec.expr.ValueVectorReadExpression;
 import org.apache.drill.exec.expr.ValueVectorWriteExpression;
-import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.config.UnionAll;
 import org.apache.drill.exec.record.AbstractRecordBatch;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index a99a986460..a2b5947d3c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -157,8 +157,8 @@ public ExternalSortBatch(ExternalSort popConfig, FragmentContext context, Record
     SPILL_BATCH_GROUP_SIZE = config.getInt(ExecConstants.EXTERNAL_SORT_SPILL_GROUP_SIZE);
     SPILL_THRESHOLD = config.getInt(ExecConstants.EXTERNAL_SORT_SPILL_THRESHOLD);
     dirs = Iterators.cycle(config.getStringList(ExecConstants.EXTERNAL_SORT_SPILL_DIRS));
-    copierAllocator = oContext.getAllocator().getChildAllocator(
-        context.asLimitConsumer(), PriorityQueueCopier.INITIAL_ALLOCATION, PriorityQueueCopier.MAX_ALLOCATION, true);
+    copierAllocator = oContext.getAllocator().newChildAllocator(oContext.getAllocator().getName() + ":copier",
+        PriorityQueueCopier.INITIAL_ALLOCATION, PriorityQueueCopier.MAX_ALLOCATION);
     FragmentHandle handle = context.getHandle();
     fileName = String.format("%s/major_fragment_%s/minor_fragment_%s/operator_%s", QueryIdHelper.getQueryId(handle.getQueryId()),
         handle.getMajorFragmentId(), handle.getMinorFragmentId(), popConfig.getOperatorId());
@@ -211,18 +211,26 @@ public void close() {
       if (sv4 != null) {
         sv4.clear();
       }
-      if (copier != null) {
-        copier.close();
-      }
-      if (copierAllocator != null) {
+
+      try {
+        if (copier != null) {
+          copier.close();
+        }
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      } finally {
         copierAllocator.close();
-      }
-      super.close();
+        super.close();
+
+        if (mSorter != null) {
+          mSorter.clear();
+        }
 
-      if(mSorter != null) {
-        mSorter.clear();
       }
+
     }
+
+
   }
 
   @Override
@@ -339,9 +347,6 @@ public IterOutcome innerNext() {
           SelectionVector2 sv2;
           if (incoming.getSchema().getSelectionVectorMode() == BatchSchema.SelectionVectorMode.TWO_BYTE) {
             sv2 = incoming.getSelectionVector2().clone();
-            if (sv2.getBuffer(false).isRootBuffer()) {
-              oContext.getAllocator().takeOwnership(sv2.getBuffer(false));
-            }
           } else {
             try {
               sv2 = newSV2();
@@ -473,10 +478,6 @@ public IterOutcome innerNext() {
         batchGroups.addAll(spilledBatchGroups);
         spilledBatchGroups = null; // no need to cleanup spilledBatchGroups, all it's batches are in batchGroups now
 
-        // copierAllocator is no longer needed now. Closing it will free memory for this operator
-        copierAllocator.close();
-        copierAllocator = null;
-
         logger.warn("Starting to merge. {} batch groups. Current allocated memory: {}", batchGroups.size(), oContext.getAllocator().getAllocatedMemory());
         VectorContainer hyperBatch = constructHyperBatch(batchGroups);
         createCopier(hyperBatch, batchGroups, container, false);
@@ -593,7 +594,6 @@ public BatchGroup mergeAndSpill(LinkedList<BatchGroup> batchGroups) throws Schem
         AutoCloseables.close(newGroup, logger);
       }
     }
-    takeOwnership(c1); // transfer ownership from copier allocator to external sort allocator
     long bufSize = getBufferSize(c1);
     totalSizeInMemory += bufSize;
     logger.debug("mergeAndSpill: final total size in memory = {}", totalSizeInMemory);
@@ -620,25 +620,13 @@ private void cleanAfterMergeAndSpill(final List<BatchGroup> batchGroups, boolean
       }
   }
 
-  private void takeOwnership(VectorAccessible batch) {
-    for (VectorWrapper<?> w : batch) {
-      DrillBuf[] bufs = w.getValueVector().getBuffers(false);
-      for (DrillBuf buf : bufs) {
-        if (buf.isRootBuffer()) {
-          oContext.getAllocator().takeOwnership(buf);
-        }
-      }
-    }
-  }
 
   private long getBufferSize(VectorAccessible batch) {
     long size = 0;
     for (VectorWrapper<?> w : batch) {
       DrillBuf[] bufs = w.getValueVector().getBuffers(false);
       for (DrillBuf buf : bufs) {
-        if (buf.isRootBuffer()) {
-          size += buf.capacity();
-        }
+        size += buf.getPossibleMemoryConsumed();
       }
     }
     return size;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopier.java
index e0d9c2d6af..ec590c26db 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopier.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.physical.impl.xsort;
 
+import java.io.IOException;
 import java.util.List;
 
 import org.apache.drill.exec.compile.TemplateClassDefinition;
@@ -38,5 +39,5 @@ public void setup(FragmentContext context, BufferAllocator allocator, VectorAcce
       new TemplateClassDefinition<>(PriorityQueueCopier.class, PriorityQueueCopierTemplate.class);
 
   @Override
-  abstract public void close(); // specify this to leave out the Exception
+  abstract public void close() throws IOException; // specify this to leave out the Exception
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
index 891907a16d..5090b335dc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
@@ -19,6 +19,7 @@
 
 import io.netty.buffer.DrillBuf;
 
+import java.io.IOException;
 import java.util.List;
 
 import javax.inject.Named;
@@ -66,7 +67,6 @@ public int next(int targetRecordCount) {
     allocateVectors(targetRecordCount);
     for (int outgoingIndex = 0; outgoingIndex < targetRecordCount; outgoingIndex++) {
       if (queueSize == 0) {
-        close();
         return 0;
       }
       int compoundIndex = vector4.get(0);
@@ -96,7 +96,7 @@ private void setValueCount(int count) {
   }
 
   @Override
-  public void close() {
+  public void close() throws IOException {
     vector4.clear();
     for (final VectorWrapper<?> w: outgoing) {
       w.getValueVector().clear();
@@ -104,6 +104,10 @@ public void close() {
     for (final VectorWrapper<?> w : hyperBatch) {
       w.clear();
     }
+
+    for (BatchGroup batchGroup : batchGroups) {
+      batchGroup.close();
+    }
   }
 
   private void siftUp() {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
index 14db502a10..7a0be5ee74 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
@@ -25,4 +25,7 @@
 
   public RawFragmentBatch getNext() throws IOException, InterruptedException;
   public void kill(FragmentContext context);
+
+  @Override
+  public void close(); // Suppress the "throws Exception".
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index ed86358d60..13164b980b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -32,13 +32,12 @@
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.ValueVector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 
 /**
  * Holds record batch loaded from record batch message.
@@ -120,7 +119,6 @@ public boolean load(RecordBatchDef def, DrillBuf buf) throws SchemaChangeExcepti
         newVectors.add(vector);
       }
 
-      Preconditions.checkArgument(buf == null || bufOffset == buf.capacity());
 
       // rebuild the schema.
       final SchemaBuilder builder = BatchSchema.newBuilder();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
index 324829ab4b..d39ce5e89e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -21,6 +21,7 @@
 
 import java.util.List;
 
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
 import org.apache.drill.exec.proto.UserBitShared.SerializedField;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
@@ -32,8 +33,8 @@
 /**
  * A specialized version of record batch that can moves out buffers and preps them for writing.
  */
-public class WritableBatch {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WritableBatch.class);
+public class WritableBatch implements AutoCloseable {
+  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WritableBatch.class);
 
   private final RecordBatchDef def;
   private final DrillBuf[] buffers;
@@ -58,7 +59,7 @@ public RecordBatchDef getDef() {
     return buffers;
   }
 
-  public void reconstructContainer(VectorContainer container) {
+  public void reconstructContainer(BufferAllocator allocator, VectorContainer container) {
     Preconditions.checkState(!cleared,
         "Attempted to reconstruct a container from a WritableBatch after it had been cleared");
     if (buffers.length > 0) { /* If we have DrillBuf's associated with value vectors */
@@ -67,7 +68,7 @@ public void reconstructContainer(VectorContainer container) {
         len += b.capacity();
       }
 
-      DrillBuf newBuf = buffers[0].getAllocator().buffer(len);
+      DrillBuf newBuf = allocator.buffer(len);
       try {
         /* Copy data from each buffer into the compound buffer */
         int offset = 0;
@@ -170,15 +171,16 @@ public static WritableBatch get(RecordBatch batch) {
     return getBatchNoHVWrap(batch.getRecordCount(), batch, sv2);
   }
 
-  public void retainBuffers() {
-    for (DrillBuf buf : buffers) {
-      buf.retain();
+  public void retainBuffers(final int increment) {
+    for (final DrillBuf buf : buffers) {
+      buf.retain(increment);
     }
   }
 
-  public void retainBuffers(int increment) {
-    for (DrillBuf buf : buffers) {
-      buf.retain(increment);
+  @Override
+  public void close() {
+    for(final DrillBuf drillBuf : buffers) {
+      drillBuf.release(1);
     }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java
index 915509fc9b..cc238f53f4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java
@@ -52,7 +52,7 @@
   public ControlClient(DrillbitEndpoint remoteEndpoint, DrillbitEndpoint localEndpoint, ControlMessageHandler handler,
       BootStrapContext context, ControlConnectionManager.CloseHandlerCreator closeHandlerFactory) {
     super(ControlRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
-        context.getAllocator().getUnderlyingAllocator(),
+        context.getAllocator().getAsByteBufAllocator(),
         context.getBitLoopGroup(),
         RpcType.HANDSHAKE,
         BitControlHandshake.class,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlServer.java
index 81c886a49f..a786469293 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlServer.java
@@ -46,7 +46,7 @@
   public ControlServer(ControlMessageHandler handler, BootStrapContext context, ConnectionManagerRegistry connectionRegistry) {
     super(
         ControlRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
-        context.getAllocator().getUnderlyingAllocator(),
+        context.getAllocator().getAsByteBufAllocator(),
         context.getBitLoopGroup());
     this.handler = handler;
     this.connectionRegistry = connectionRegistry;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/WorkEventBus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/WorkEventBus.java
index 3e461ef885..5be4d81ebc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/WorkEventBus.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/WorkEventBus.java
@@ -17,17 +17,25 @@
  */
 package org.apache.drill.exec.rpc.control;
 
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
-import org.apache.drill.exec.work.foreman.FragmentStatusListener;
+import org.apache.drill.exec.server.options.OptionValue;
+import org.apache.drill.exec.server.options.TypeValidators.RangeLongValidator;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
+import org.apache.drill.exec.work.foreman.FragmentStatusListener;
+import org.apache.drill.exec.work.fragment.AllocatorTree;
+import org.apache.drill.exec.work.fragment.AllocatorTree.CloseListener;
+import org.apache.drill.exec.work.fragment.AllocatorTree.QueryAllocator;
 import org.apache.drill.exec.work.fragment.FragmentManager;
 
 import com.google.common.cache.Cache;
@@ -37,13 +45,29 @@
 public class WorkEventBus {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WorkEventBus.class);
   private final ConcurrentMap<FragmentHandle, FragmentManager> managers = Maps.newConcurrentMap();
-  private final ConcurrentMap<QueryId, FragmentStatusListener> listeners =
-      new ConcurrentHashMap<>(16, 0.75f, 16);
+  private final Map<QueryId, QueryAllocator> queryAllocators = Maps.newHashMap();
+
+  private final ConcurrentMap<QueryId, FragmentStatusListener> listeners = new ConcurrentHashMap<>(16, 0.75f, 16);
   private final Cache<FragmentHandle, Integer> recentlyFinishedFragments = CacheBuilder.newBuilder()
           .maximumSize(10000)
           .expireAfterWrite(10, TimeUnit.MINUTES)
           .build();
 
+  public final static RangeLongValidator MEMORY_MAX_PER_QUERY =
+      new RangeLongValidator("exec.memory.max.per_query_per_node.mb", 10, Long.MAX_VALUE, 1024 * 1024) {
+
+    @Override
+    public void validate(OptionValue v) {
+      if (v.type != OptionValue.OptionType.SYSTEM) {
+        throw UserException.unsupportedError()
+            .message("Query Memory Maximums can only be set at the SYSTEM level.")
+            .build(logger);
+      }
+      super.validate(v);
+    }
+
+  };
+
   public void removeFragmentStatusListener(final QueryId queryId) {
     if (logger.isDebugEnabled()) {
       logger.debug("Removing fragment status listener for queryId {}.", QueryIdHelper.getQueryId(queryId));
@@ -109,6 +133,34 @@ public FragmentManager getFragmentManager(final FragmentHandle handle) throws Fr
         + QueryIdHelper.getQueryIdentifier(handle));
   }
 
+  public QueryAllocator getQueryAllocator(
+      BufferAllocator parentAllocator,
+      final QueryId queryId,
+      long queryReservation,
+      long queryMax){
+    // we don't want to be optimistic here since an allocator can reserve memory.
+    synchronized(queryAllocators){
+      {
+        final QueryAllocator allocator = queryAllocators.get(queryId);
+        if (allocator != null) {
+          return allocator;
+        }
+      }
+
+      final QueryAllocator newAllocator = AllocatorTree.newAllocator(parentAllocator, new CloseListener() {
+        @Override
+        public void closeEvent() {
+          synchronized (queryAllocators) {
+            QueryAllocator oldAllocator = queryAllocators.get(queryId);
+            oldAllocator.close();
+          }
+        }
+      }, queryId, queryReservation, queryMax);
+      queryAllocators.put(queryId, newAllocator);
+      return newAllocator;
+    }
+  }
+
   /**
    * Removes fragment manager (for the corresponding the handle) from the work event bus. This method can be called
    * multiple times. The manager will be removed only once (the first call).
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java
index d7611b9ece..9db551b462 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java
@@ -48,7 +48,7 @@
   public DataClient(DrillbitEndpoint remoteEndpoint, BootStrapContext context, DataConnectionManager.CloseHandlerCreator closeHandlerFactory) {
     super(
         DataRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
-        context.getAllocator().getUnderlyingAllocator(),
+        context.getAllocator().getAsByteBufAllocator(),
         context.getBitClientLoopGroup(),
         RpcType.HANDSHAKE,
         BitServerHandshake.class,
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java
index 5874b31127..451500ca49 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java
@@ -21,6 +21,7 @@
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.rpc.control.WorkEventBus;
 import org.apache.drill.exec.server.BootStrapContext;
@@ -40,6 +41,7 @@
   private final DataResponseHandler dataHandler;
   private final boolean allowPortHunting;
   private ConcurrentMap<DrillbitEndpoint, DataConnectionManager> connectionManager = Maps.newConcurrentMap();
+  private final BufferAllocator dataAllocator;
 
   public DataConnectionCreator(BootStrapContext context, WorkEventBus workBus, DataResponseHandler dataHandler, boolean allowPortHunting) {
     super();
@@ -47,10 +49,12 @@ public DataConnectionCreator(BootStrapContext context, WorkEventBus workBus, Dat
     this.workBus = workBus;
     this.dataHandler = dataHandler;
     this.allowPortHunting = allowPortHunting;
+    this.dataAllocator = context.getAllocator()
+        .newChildAllocator("rpc-data", 0, Long.MAX_VALUE);
   }
 
   public DrillbitEndpoint start(DrillbitEndpoint partialEndpoint) throws DrillbitStartupException {
-    server = new DataServer(context, workBus, dataHandler);
+    server = new DataServer(context, dataAllocator, workBus, dataHandler);
     int port = server.bind(partialEndpoint.getControlPort() + 1, allowPortHunting);
     DrillbitEndpoint completeEndpoint = partialEndpoint.toBuilder().setDataPort(port).build();
     return completeEndpoint;
@@ -68,6 +72,7 @@ public DataTunnel getTunnel(DrillbitEndpoint endpoint) {
   @Override
   public void close() {
     Closeables.closeQuietly(server);
+    dataAllocator.close();
   }
 
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java
index ed69699236..ece2cab163 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java
@@ -19,6 +19,7 @@
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
+import io.netty.buffer.DrillBuf.TransferResult;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.socket.SocketChannel;
 import io.netty.util.concurrent.GenericFutureListener;
@@ -26,6 +27,7 @@
 import java.io.IOException;
 
 import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.memory.AllocatorClosedException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.BitData.BitClientHandshake;
 import org.apache.drill.exec.proto.BitData.BitServerHandshake;
@@ -43,7 +45,6 @@
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.control.WorkEventBus;
 import org.apache.drill.exec.server.BootStrapContext;
-import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.fragment.FragmentManager;
 
 import com.google.protobuf.MessageLite;
@@ -56,10 +57,11 @@
   private final WorkEventBus workBus;
   private final DataResponseHandler dataHandler;
 
-  public DataServer(BootStrapContext context, WorkEventBus workBus, DataResponseHandler dataHandler) {
+  public DataServer(BootStrapContext context, BufferAllocator alloc, WorkEventBus workBus,
+      DataResponseHandler dataHandler) {
     super(
         DataRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
-        context.getAllocator().getUnderlyingAllocator(),
+        alloc.getAsByteBufAllocator(),
         context.getBitLoopGroup());
     this.context = context;
     this.workBus = workBus;
@@ -167,11 +169,29 @@ private void send(final FragmentRecordBatch fragmentBatch, final DrillBuf body,
     }
 
     final BufferAllocator allocator = manager.getFragmentContext().getAllocator();
-    final Pointer<DrillBuf> out = new Pointer<>();
-
     final boolean withinMemoryEnvelope;
-
-    withinMemoryEnvelope = allocator.takeOwnership(body, out);
+    final DrillBuf transferredBuffer;
+    try {
+      TransferResult result = body.transferOwnership(allocator);
+      withinMemoryEnvelope = result.allocationFit;
+      transferredBuffer = result.buffer;
+    } catch(final AllocatorClosedException e) {
+      /*
+       * It can happen that between the time we get the fragment manager and we
+       * try to transfer this buffer to it, the fragment may have been cancelled
+       * and closed. When that happens, the allocator will be closed when we
+       * attempt this. That just means we can drop this data on the floor, since
+       * the receiver no longer exists (and no longer needs it).
+       *
+       * Note that checking manager.isCancelled() before we attempt this isn't enough,
+       * because of timing: it may still be cancelled between that check and
+       * the attempt to do the memory transfer. To double check ourselves, we
+       * do check manager.isCancelled() here, after the fact; it shouldn't
+       * change again after its allocator has been closed.
+       */
+      assert manager.isCancelled();
+      return;
+    }
 
     if (!withinMemoryEnvelope) {
       // if we over reserved, we need to add poison pill before batch.
@@ -179,11 +199,11 @@ private void send(final FragmentRecordBatch fragmentBatch, final DrillBuf body,
     }
 
     ack.increment();
-    dataHandler.handle(manager, fragmentBatch, out.value, ack);
+    dataHandler.handle(manager, fragmentBatch, transferredBuffer, ack);
 
     // make sure to release the reference count we have to the new buffer.
     // dataHandler.handle should have taken any ownership it needed.
-    out.value.release();
+    transferredBuffer.release();
   }
 
   private class ProxyCloseHandler implements GenericFutureListener<ChannelFuture> {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index ddba213819..049ae0c657 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -70,7 +70,9 @@
 
   public UserServer(DrillConfig config, ScanResult classpathScan, BufferAllocator alloc, EventLoopGroup eventLoopGroup,
       UserWorker worker, Executor executor) throws DrillbitStartupException {
-    super(UserRpcConfig.getMapping(config, executor), alloc.getUnderlyingAllocator(), eventLoopGroup);
+    super(UserRpcConfig.getMapping(config, executor),
+        alloc.getAsByteBufAllocator(),
+        eventLoopGroup);
     this.worker = worker;
     this.alloc = alloc;
     // TODO: move this up
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 6843a213c2..7ea5740e1c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -17,6 +17,8 @@
  */
 package org.apache.drill.exec.server.options;
 
+import static com.google.common.base.Preconditions.checkArgument;
+
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -24,22 +26,22 @@
 import java.util.Map.Entry;
 import java.util.Set;
 
-import com.google.common.collect.Sets;
 import org.apache.commons.collections.IteratorUtils;
 import org.apache.drill.common.config.LogicalPlanPersistence;
-import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.map.CaseInsensitiveMap;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.compile.ClassTransformer;
 import org.apache.drill.exec.compile.QueryClassLoader;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.rpc.control.WorkEventBus;
 import org.apache.drill.exec.server.options.OptionValue.OptionType;
 import org.apache.drill.exec.store.sys.PStore;
 import org.apache.drill.exec.store.sys.PStoreConfig;
 import org.apache.drill.exec.store.sys.PStoreProvider;
 import org.apache.drill.exec.util.AssertionUtil;
 
-import static com.google.common.base.Preconditions.checkArgument;
+import com.google.common.collect.Sets;
 
 /**
  * {@link OptionManager} that holds options within {@link org.apache.drill.exec.server.DrillbitContext}.
@@ -130,7 +132,8 @@
       ExecConstants.ENABLE_VERBOSE_ERRORS,
       ExecConstants.ENABLE_WINDOW_FUNCTIONS_VALIDATOR,
       ClassTransformer.SCALAR_REPLACEMENT_VALIDATOR,
-      ExecConstants.ENABLE_NEW_TEXT_READER
+        ExecConstants.ENABLE_NEW_TEXT_READER,
+        WorkEventBus.MEMORY_MAX_PER_QUERY
     };
     final Map<String, OptionValidator> tmp = new HashMap<>();
     for (final OptionValidator validator : validators) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java
index e02b41385c..e624ada241 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java
@@ -24,7 +24,7 @@
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
@@ -63,7 +63,7 @@ public Object next() {
 
 
     memoryInfo.direct_current = context.getDrillbitContext().getAllocator().getAllocatedMemory();
-    memoryInfo.direct_max = TopLevelAllocator.MAXIMUM_DIRECT_MEMORY;
+    memoryInfo.direct_max = DrillConfig.getMaxDirectMemory();
     return memoryInfo;
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java
index 3f2e3f21f6..1579c2bc7a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java
@@ -71,18 +71,12 @@ public AbstractDataCollector(AtomicInteger parentAccounter,
 
     final boolean spooling = collector.getIsSpooling();
 
-    try {
-
-      for (int i = 0; i < numBuffers; i++) {
-        if (spooling) {
-          buffers[i] = new SpoolingRawBatchBuffer(context, bufferCapacity, collector.getOppositeMajorFragmentId(), i);
-        } else {
-          buffers[i] = new UnlimitedRawBatchBuffer(context, bufferCapacity, collector.getOppositeMajorFragmentId());
-        }
+    for (int i = 0; i < numBuffers; i++) {
+      if (spooling) {
+        buffers[i] = new SpoolingRawBatchBuffer(context, bufferCapacity, collector.getOppositeMajorFragmentId(), i);
+      } else {
+        buffers[i] = new UnlimitedRawBatchBuffer(context, bufferCapacity, collector.getOppositeMajorFragmentId());
       }
-    } catch (IOException | OutOfMemoryException e) {
-      logger.error("Exception", e);
-      context.fail(e);
     }
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
index 753c75db98..d90a08e90e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
@@ -29,7 +29,6 @@
 import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.BitData;
@@ -82,9 +81,10 @@
   private Path path;
   private FSDataOutputStream outputStream;
 
-  public SpoolingRawBatchBuffer(FragmentContext context, int fragmentCount, int oppositeId, int bufferIndex) throws IOException, OutOfMemoryException {
+  public SpoolingRawBatchBuffer(FragmentContext context, int fragmentCount, int oppositeId, int bufferIndex) {
     super(context, fragmentCount);
-    this.allocator = context.getNewChildAllocator(ALLOCATOR_INITIAL_RESERVATION, ALLOCATOR_MAX_RESERVATION, true);
+    this.allocator = context.getNewChildAllocator("SpoolingRawBatchBufer", 100, ALLOCATOR_INITIAL_RESERVATION,
+        ALLOCATOR_MAX_RESERVATION, true);
     this.threshold = context.getConfig().getLong(ExecConstants.SPOOLING_BUFFER_MEMORY);
     this.oppositeId = oppositeId;
     this.bufferIndex = bufferIndex;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 71f930784e..7b015a5940 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -44,7 +44,7 @@
 import org.apache.drill.exec.coord.DistributedSemaphore.DistributedLease;
 import org.apache.drill.exec.exception.OptimizerException;
 import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.memory.RootAllocatorFactory;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.opt.BasicOptimizer;
@@ -159,7 +159,7 @@ public Foreman(final WorkerBee bee, final DrillbitContext drillbitContext,
     this.closeFuture = initiatingClient.getChannel().closeFuture();
     closeFuture.addListener(closeListener);
 
-    queryContext = new QueryContext(connection.getSession(), drillbitContext);
+    queryContext = new QueryContext(connection.getSession(), drillbitContext, queryId);
     queryManager = new QueryManager(queryId, queryRequest, drillbitContext.getPersistentStoreProvider(),
         stateListener, this); // TODO reference escapes before ctor is complete via stateListener, this
 
@@ -407,7 +407,6 @@ private void runPhysicalPlan(final PhysicalPlan plan) throws ExecutionSetupExcep
     setupRootFragment(rootPlanFragment, work.getRootOperator());
 
     setupNonRootFragments(planFragments);
-    drillbitContext.getAllocator().resetLimits(); // TODO a global effect for this query?!?
 
     moveToState(QueryState.RUNNING, null);
     logger.debug("Fragments running.");
@@ -435,7 +434,7 @@ private void setupSortMemoryAllocations(final PhysicalPlan plan) {
       final OptionManager optionManager = queryContext.getOptions();
       final long maxWidthPerNode = optionManager.getOption(ExecConstants.MAX_WIDTH_PER_NODE_KEY).num_val;
       long maxAllocPerNode = Math.min(DrillConfig.getMaxDirectMemory(),
-          queryContext.getConfig().getLong(TopLevelAllocator.TOP_LEVEL_MAX_ALLOC));
+          queryContext.getConfig().getLong(RootAllocatorFactory.TOP_LEVEL_MAX_ALLOC));
       maxAllocPerNode = Math.min(maxAllocPerNode,
           optionManager.getOption(ExecConstants.MAX_QUERY_MEMORY_PER_NODE_KEY).num_val);
       final long maxSortAlloc = maxAllocPerNode / (sortList.size() * maxWidthPerNode);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AllocatorTree.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AllocatorTree.java
new file mode 100644
index 0000000000..d7c2da8031
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/AllocatorTree.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.work.fragment;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.memory.AllocatorDecorator;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.helper.QueryIdHelper;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * AllocatorTree and its inner classes creates a tree of automatically maintained allocators. As long as there is at
+ * least one outstanding minor fragment allocator, the query query allocator will be maintained. Once all allocators are
+ * closed, the QueryAllocator is automatically closed. The tree is currently:
+ *
+ * QueryAllocator => MinorFragmentAllocator => GenericAllocator
+ */
+public class AllocatorTree {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AllocatorTree.class);
+
+
+  /**
+   * Simple AllocatorDecorator that informs a close listener when the allocator is closed.
+   */
+  private abstract static class CloseListeningAllocator extends AllocatorDecorator {
+
+    final CloseListener listener;
+
+    private CloseListeningAllocator(BufferAllocator allocator, CloseListener listener) {
+      super(allocator);
+      this.listener = listener;
+    }
+
+    @Override
+    public void close() {
+      if (!super.isClosed()) {
+        super.close();
+        listener.closeEvent();
+      }
+    }
+
+  }
+
+  public static QueryAllocator newAllocator(
+      BufferAllocator parentAllocator, CloseListener listener,
+      final QueryId queryId, long queryReservation, long queryMax
+      ) {
+
+    final BufferAllocator inner = parentAllocator
+        .newChildAllocator("query:" + QueryIdHelper.getQueryId(queryId), queryReservation, queryMax);
+    return new QueryAllocator(inner, listener);
+  }
+
+  public static class QueryAllocator extends CloseListeningAllocator {
+
+    private final AtomicInteger openAllocators = new AtomicInteger(0);
+
+    private QueryAllocator(BufferAllocator allocator, CloseListener listener) {
+      super(allocator, listener);
+    }
+
+    public MinorFragmentAllocator newMinorFragmentAllocator(FragmentHandle handle, long initReservation,
+        long maxAllocation) {
+      openAllocators.incrementAndGet();
+      return new MinorFragmentAllocator(
+          super.newChildAllocator(
+              "minor:" + QueryIdHelper.getFragmentId(handle),
+              initReservation,
+              maxAllocation),
+          new Listener());
+    }
+
+    private class Listener implements CloseListener {
+      @Override
+      public void closeEvent() {
+        int outcome = openAllocators.decrementAndGet();
+        Preconditions.checkArgument(outcome > -1);
+        if (outcome == 0) {
+          listener.closeEvent();
+        }
+      }
+    }
+
+
+    @Override
+    public BufferAllocator newChildAllocator(String name, long initReservation, long maxAllocation) {
+      throw new UnsupportedOperationException("Use newFragmentAllocator().");
+    }
+
+  }
+
+  public static class MinorFragmentAllocator extends CloseListeningAllocator {
+
+    private MinorFragmentAllocator(BufferAllocator allocator, CloseListener listener) {
+      super(allocator, listener);
+    }
+
+  }
+
+  /**
+   * Interface to listen to an allocator is closed. Called after the allocator has been closed.
+   */
+  public static interface CloseListener {
+    void closeEvent();
+  }
+
+
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
index 8737c29491..7daf1a1dc8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
@@ -39,6 +39,7 @@
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.RootAllocator;
 import org.apache.drill.exec.memory.RootAllocatorFactory;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
@@ -248,6 +249,23 @@ public static TestBuilder testBuilder() {
     return new TestBuilder(allocator);
   }
 
+  /**
+   * Utility function that can be used in tests to verify the state of drillbit
+   * allocators.
+   */
+  public static void verifyAllocators() {
+    if (bits != null) {
+      for(Drillbit bit : bits) {
+        if (bit != null) {
+          final DrillbitContext drillbitContext = bit.getContext();
+          final BufferAllocator bufferAllocator = drillbitContext.getAllocator();
+          final RootAllocator rootAllocator = (RootAllocator) bufferAllocator;
+          rootAllocator.verify();
+        }
+      }
+    }
+  }
+
   @AfterClass
   public static void closeClient() throws IOException {
     if (client != null) {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestAllocationException.java b/exec/java-exec/src/test/java/org/apache/drill/TestAllocationException.java
deleted file mode 100644
index 19613fe9cc..0000000000
--- a/exec/java-exec/src/test/java/org/apache/drill/TestAllocationException.java
+++ /dev/null
@@ -1,71 +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.drill;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.memory.TopLevelAllocator;
-import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
-import org.apache.drill.exec.testing.Controls;
-import org.apache.drill.exec.testing.ControlsInjectionUtil;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * Run several tpch queries and inject an OutOfMemoryException in ScanBatch that will cause an OUT_OF_MEMORY outcome to
- * be propagated downstream. Make sure the proper "memory error" message is sent to the client.
- */
-@Ignore("Need to add exception site in memory layer that doesn't depend on other Drill code.")
-public class TestAllocationException extends BaseTestQuery {
-
-  private static final String SINGLE_MODE = "ALTER SESSION SET `planner.disable_exchanges` = true";
-
-  private void testWithException(final String fileName) throws Exception{
-    test(SINGLE_MODE);
-
-    final String controls = Controls.newBuilder()
-      .addException(TopLevelAllocator.class,
-        TopLevelAllocator.CHILD_BUFFER_INJECTION_SITE,
-        OutOfMemoryException.class,
-        200,
-        1
-      ).build();
-    ControlsInjectionUtil.setControls(client, controls);
-
-    String query = getFile(fileName);
-
-    try {
-      test(query);
-      fail("The query should have failed!");
-    } catch(UserException uex) {
-      DrillPBError error = uex.getOrCreatePBError(false);
-      assertEquals(DrillPBError.ErrorType.RESOURCE, error.getErrorType());
-      assertTrue("Error message isn't related to memory error",
-        uex.getMessage().contains(UserException.MEMORY_ERROR_MSG));
-    }
-  }
-
-  @Test
-  public void testWithOOM() throws Exception{
-    testWithException("queries/tpch/01.sql");
-  }
-}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java
index 22471c8abb..6d2fbf02ce 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java
@@ -17,12 +17,18 @@
  */
 package org.apache.drill;
 
+import org.junit.After;
 import org.junit.Ignore;
 import org.junit.Test;
 
 public class TestTpchLimit0 extends BaseTestQuery{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTpchLimit0.class);
 
+  @After
+  public void checkForLeaks() {
+    verifyAllocators();
+  }
+
   private void testLimitZero(String fileName) throws Exception {
     String query = getFile(fileName);
     query = "ALTER SESSION SET `planner.slice_target` = 1; select * from \n(" + query.replace(";", ")xyz limit 0;");
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java
index 350e02b56d..8e69b9d684 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java
@@ -18,11 +18,9 @@
 
 package org.apache.drill.exec.memory;
 
-
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
-import com.google.common.collect.Lists;
 import io.netty.buffer.DrillBuf;
 
 import java.util.Iterator;
@@ -54,19 +52,20 @@
 import org.junit.Test;
 
 import com.google.common.base.Charsets;
+import com.google.common.collect.Lists;
 import com.google.common.io.Files;
 
 public class TestAllocators extends DrillTest {
 
   private static final Properties TEST_CONFIGURATIONS = new Properties() {
     {
-      put(TopLevelAllocator.TOP_LEVEL_MAX_ALLOC, "14000000");
-      put(AccountorImpl.ENABLE_FRAGMENT_MEMORY_LIMIT, "true");
-      put(AccountorImpl.FRAGMENT_MEM_OVERCOMMIT_FACTOR, "1.1");
+      put(RootAllocatorFactory.TOP_LEVEL_MAX_ALLOC, "14000000");
+      // put(AccountorImpl.ENABLE_FRAGMENT_MEMORY_LIMIT, "true");
+      // put(AccountorImpl.FRAGMENT_MEM_OVERCOMMIT_FACTOR, "1.1");
     }
   };
 
-  private final static String planFile="/physical_allocator_test.json";
+  private final static String planFile = "/physical_allocator_test.json";
 
   /**
    * Contract for DrillBuf[] returned from getBuffers() is that buffers are returned in a reader appropriate state
@@ -84,8 +83,7 @@ public void ensureDrillBufReadIndexIsZero() throws Exception {
 
     final Properties props = new Properties() {
       {
-        put(TopLevelAllocator.TOP_LEVEL_MAX_ALLOC, "1000000");
-        put(TopLevelAllocator.ERROR_ON_MEMORY_LEAK, "true");
+        put(RootAllocatorFactory.TOP_LEVEL_MAX_ALLOC, "1000000");
       }
     };
     final DrillConfig config = DrillConfig.create(props);
@@ -129,139 +127,145 @@ public void ensureDrillBufReadIndexIsZero() throws Exception {
   public void testTransfer() throws Exception {
     final Properties props = new Properties() {
       {
-        put(TopLevelAllocator.TOP_LEVEL_MAX_ALLOC, "1000000");
-        put(TopLevelAllocator.ERROR_ON_MEMORY_LEAK, "true");
+        put(RootAllocatorFactory.TOP_LEVEL_MAX_ALLOC, "1049600");
       }
     };
     final DrillConfig config = DrillConfig.create(props);
     BufferAllocator a = RootAllocatorFactory.newRoot(config);
-    BufferAllocator b = RootAllocatorFactory.newRoot(config);
+    BufferAllocator a1 = a.newChildAllocator("a1", 0, Integer.MAX_VALUE);
+    BufferAllocator a2 = a.newChildAllocator("a2", 0, Integer.MAX_VALUE);
 
-    DrillBuf buf1 = a.buffer(1_000_000);
-    DrillBuf buf2 = b.buffer(1_000);
-    b.takeOwnership(buf1);
+    DrillBuf buf1 = a1.buffer(1_000_000);
+    DrillBuf buf2 = a2.buffer(1_000);
+    DrillBuf buf3 = buf1.transferOwnership(a2).buffer;
 
     buf1.release();
     buf2.release();
+    buf3.release();
 
+    a1.close();
+    a2.close();
     a.close();
-    b.close();
   }
 
   @Test
   public void testAllocators() throws Exception {
     // Setup a drillbit (initializes a root allocator)
     final DrillConfig config = DrillConfig.create(TEST_CONFIGURATIONS);
-    final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-    final Drillbit bit = new Drillbit(config, serviceSet);
-    bit.run();
-    final DrillbitContext bitContext = bit.getContext();
-    FunctionImplementationRegistry functionRegistry = bitContext.getFunctionImplementationRegistry();
-    StoragePluginRegistry storageRegistry = new StoragePluginRegistry(bitContext);
-
-    // Create a few Fragment Contexts
-
-    BitControl.PlanFragment.Builder pfBuilder1=BitControl.PlanFragment.newBuilder();
-    pfBuilder1.setMemInitial(1500000);
-    BitControl.PlanFragment pf1=pfBuilder1.build();
-    BitControl.PlanFragment.Builder pfBuilder2=BitControl.PlanFragment.newBuilder();
-    pfBuilder2.setMemInitial(500000);
-    BitControl.PlanFragment pf2=pfBuilder1.build();
-
-    FragmentContext fragmentContext1 = new FragmentContext(bitContext, pf1, null, functionRegistry);
-    FragmentContext fragmentContext2 = new FragmentContext(bitContext, pf2, null, functionRegistry);
-
-    // Get a few physical operators. Easiest way is to read a physical plan.
-    PhysicalPlanReader planReader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(bitContext, storageRegistry);
-    PhysicalPlan plan = planReader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(planFile), Charsets.UTF_8));
-    List<PhysicalOperator> physicalOperators = plan.getSortedOperators();
-    Iterator<PhysicalOperator> physicalOperatorIterator = physicalOperators.iterator();
-
-    PhysicalOperator physicalOperator1 = physicalOperatorIterator.next();
-    PhysicalOperator physicalOperator2 = physicalOperatorIterator.next();
-    PhysicalOperator physicalOperator3 = physicalOperatorIterator.next();
-    PhysicalOperator physicalOperator4 = physicalOperatorIterator.next();
-    PhysicalOperator physicalOperator5 = physicalOperatorIterator.next();
-    PhysicalOperator physicalOperator6 = physicalOperatorIterator.next();
-
-    // Create some bogus Operator profile defs and stats to create operator contexts
-    OpProfileDef def;
-    OperatorStats stats;
-
-    //Use some bogus operator type to create a new operator context.
-    def = new OpProfileDef(physicalOperator1.getOperatorId(), UserBitShared.CoreOperatorType.MOCK_SUB_SCAN_VALUE,
-        OperatorContext.getChildCount(physicalOperator1));
-    stats = fragmentContext1.getStats().newOperatorStats(def, fragmentContext1.getAllocator());
-
-
-    // Add a couple of Operator Contexts
-    // Initial allocation = 1000000 bytes for all operators
-    OperatorContext oContext11 = fragmentContext1.newOperatorContext(physicalOperator1, true);
-    DrillBuf b11=oContext11.getAllocator().buffer(1000000);
-
-    OperatorContext oContext12 = fragmentContext1.newOperatorContext(physicalOperator2, stats, true);
-    DrillBuf b12=oContext12.getAllocator().buffer(500000);
-
-    OperatorContext oContext21 = fragmentContext1.newOperatorContext(physicalOperator3, true);
-
-    def = new OpProfileDef(physicalOperator4.getOperatorId(), UserBitShared.CoreOperatorType.TEXT_WRITER_VALUE,
-        OperatorContext.getChildCount(physicalOperator4));
-    stats = fragmentContext2.getStats().newOperatorStats(def, fragmentContext2.getAllocator());
-    OperatorContext oContext22 = fragmentContext2.newOperatorContext(physicalOperator4, stats, true);
-    DrillBuf b22=oContext22.getAllocator().buffer(2000000);
-
-    // New Fragment begins
-    BitControl.PlanFragment.Builder pfBuilder3=BitControl.PlanFragment.newBuilder();
-    pfBuilder3.setMemInitial(1000000);
-    BitControl.PlanFragment pf3=pfBuilder3.build();
-
-    FragmentContext fragmentContext3 = new FragmentContext(bitContext, pf3, null, functionRegistry);
-
-    // New fragment starts an operator that allocates an amount within the limit
-    def = new OpProfileDef(physicalOperator5.getOperatorId(), UserBitShared.CoreOperatorType.UNION_VALUE,
-        OperatorContext.getChildCount(physicalOperator5));
-    stats = fragmentContext3.getStats().newOperatorStats(def, fragmentContext3.getAllocator());
-    OperatorContext oContext31 = fragmentContext3.newOperatorContext(physicalOperator5, stats, true);
-
-    DrillBuf b31a = oContext31.getAllocator().buffer(200000);
-
-    //Previously running operator completes
-    b22.release();
-    ((AutoCloseable) oContext22).close();
-
-    // Fragment 3 asks for more and fails
-    try {
-      oContext31.getAllocator().buffer(4400000);
-      fail("Fragment 3 should fail to allocate buffer");
-    } catch (OutOfMemoryException e) {
-      // expected
-    }
 
-    // Operator is Exempt from Fragment limits. Fragment 3 asks for more and succeeds
-    OperatorContext oContext32 = fragmentContext3.newOperatorContext(physicalOperator6, false);
-    try {
-      DrillBuf b32 = oContext32.getAllocator().buffer(4400000);
-      b32.release();
-    } catch (OutOfMemoryException e) {
-      fail("Fragment 3 failed to allocate buffer");
-    } finally {
-      closeOp(oContext32);
-    }
+    try (final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+        final Drillbit bit = new Drillbit(config, serviceSet)) {
+      ;
+      bit.run();
+      final DrillbitContext bitContext = bit.getContext();
+      FunctionImplementationRegistry functionRegistry = bitContext.getFunctionImplementationRegistry();
+      StoragePluginRegistry storageRegistry = new StoragePluginRegistry(bitContext);
+
+      // Create a few Fragment Contexts
+
+      BitControl.PlanFragment.Builder pfBuilder1 = BitControl.PlanFragment.newBuilder();
+      pfBuilder1.setMemInitial(1500000);
+      BitControl.PlanFragment pf1 = pfBuilder1.build();
+      BitControl.PlanFragment.Builder pfBuilder2 = BitControl.PlanFragment.newBuilder();
+      pfBuilder2.setMemInitial(500000);
+      BitControl.PlanFragment pf2 = pfBuilder1.build();
+
+      FragmentContext fragmentContext1 = new FragmentContext(bitContext, pf1, null, functionRegistry);
+      FragmentContext fragmentContext2 = new FragmentContext(bitContext, pf2, null, functionRegistry);
+
+      // Get a few physical operators. Easiest way is to read a physical plan.
+      PhysicalPlanReader planReader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(bitContext,
+          storageRegistry);
+      PhysicalPlan plan = planReader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(planFile),
+          Charsets.UTF_8));
+      List<PhysicalOperator> physicalOperators = plan.getSortedOperators();
+      Iterator<PhysicalOperator> physicalOperatorIterator = physicalOperators.iterator();
+
+      PhysicalOperator physicalOperator1 = physicalOperatorIterator.next();
+      PhysicalOperator physicalOperator2 = physicalOperatorIterator.next();
+      PhysicalOperator physicalOperator3 = physicalOperatorIterator.next();
+      PhysicalOperator physicalOperator4 = physicalOperatorIterator.next();
+      PhysicalOperator physicalOperator5 = physicalOperatorIterator.next();
+      PhysicalOperator physicalOperator6 = physicalOperatorIterator.next();
+
+      // Create some bogus Operator profile defs and stats to create operator contexts
+      OpProfileDef def;
+      OperatorStats stats;
+
+      // Use some bogus operator type to create a new operator context.
+      def = new OpProfileDef(physicalOperator1.getOperatorId(), UserBitShared.CoreOperatorType.MOCK_SUB_SCAN_VALUE,
+          OperatorContext.getChildCount(physicalOperator1));
+      stats = fragmentContext1.getStats().newOperatorStats(def, fragmentContext1.getAllocator());
+
+      // Add a couple of Operator Contexts
+      // Initial allocation = 1000000 bytes for all operators
+      OperatorContext oContext11 = fragmentContext1.newOperatorContext(physicalOperator1, true);
+      DrillBuf b11 = oContext11.getAllocator().buffer(1000000);
+
+      OperatorContext oContext12 = fragmentContext1.newOperatorContext(physicalOperator2, stats, true);
+      DrillBuf b12 = oContext12.getAllocator().buffer(500000);
+
+      OperatorContext oContext21 = fragmentContext1.newOperatorContext(physicalOperator3, true);
+
+      def = new OpProfileDef(physicalOperator4.getOperatorId(), UserBitShared.CoreOperatorType.TEXT_WRITER_VALUE,
+          OperatorContext.getChildCount(physicalOperator4));
+      stats = fragmentContext2.getStats().newOperatorStats(def, fragmentContext2.getAllocator());
+      OperatorContext oContext22 = fragmentContext2.newOperatorContext(physicalOperator4, stats, true);
+      DrillBuf b22 = oContext22.getAllocator().buffer(2000000);
+
+      // New Fragment begins
+      BitControl.PlanFragment.Builder pfBuilder3 = BitControl.PlanFragment.newBuilder();
+      pfBuilder3.setMemInitial(1000000);
+      BitControl.PlanFragment pf3 = pfBuilder3.build();
+
+      FragmentContext fragmentContext3 = new FragmentContext(bitContext, pf3, null, functionRegistry);
+
+      // New fragment starts an operator that allocates an amount within the limit
+      def = new OpProfileDef(physicalOperator5.getOperatorId(), UserBitShared.CoreOperatorType.UNION_VALUE,
+          OperatorContext.getChildCount(physicalOperator5));
+      stats = fragmentContext3.getStats().newOperatorStats(def, fragmentContext3.getAllocator());
+      OperatorContext oContext31 = fragmentContext3.newOperatorContext(physicalOperator5, stats, true);
+
+      DrillBuf b31a = oContext31.getAllocator().buffer(200000);
+
+      // Previously running operator completes
+      b22.release();
+      ((AutoCloseable) oContext22).close();
+
+      // Fragment 3 asks for more and fails
+      boolean outOfMem = false;
+      try {
+        oContext31.getAllocator().buffer(44000000);
+        fail("Fragment 3 should fail to allocate buffer");
+      } catch (OutOfMemoryException e) {
+        outOfMem = true; // Expected.
+      }
+      assertTrue(outOfMem);
+
+      // Operator is Exempt from Fragment limits. Fragment 3 asks for more and succeeds
+      OperatorContext oContext32 = fragmentContext3.newOperatorContext(physicalOperator6, false);
+      try {
+        DrillBuf b32 = oContext32.getAllocator().buffer(4400000);
+        b32.release();
+      } catch (OutOfMemoryException e) {
+        fail("Fragment 3 failed to allocate buffer");
+      } finally {
+        closeOp(oContext32);
+      }
 
-    b11.release();
-    closeOp(oContext11);
-    b12.release();
-    closeOp(oContext12);
-    closeOp(oContext21);
-    b31a.release();
-    closeOp(oContext31);
+      b11.release();
+      closeOp(oContext11);
+      b12.release();
+      closeOp(oContext12);
+      closeOp(oContext21);
+      b31a.release();
+      closeOp(oContext31);
 
-    fragmentContext1.close();
-    fragmentContext2.close();
-    fragmentContext3.close();
+      fragmentContext1.close();
+      fragmentContext2.close();
+      fragmentContext3.close();
 
-    bit.close();
-    serviceSet.close();
+    }
   }
 
   private void closeOp(OperatorContext c) throws Exception {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
index 0a9b470404..96f2b33984 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
@@ -24,9 +24,6 @@
 
 import java.nio.charset.Charset;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableMap;
-
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.SchemaPath;
@@ -47,9 +44,9 @@
 import org.apache.drill.exec.expr.holders.UInt1Holder;
 import org.apache.drill.exec.expr.holders.UInt4Holder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.RootAllocatorFactory;
 import org.apache.drill.exec.proto.UserBitShared;
-import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.vector.BaseValueVector;
 import org.apache.drill.exec.vector.BitVector;
@@ -67,6 +64,9 @@
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+
 public class TestValueVector extends ExecTest {
   //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestValueVector.class);
 
@@ -156,11 +156,11 @@ public void testVariableVectorReallocation() {
     final int expectedOffsetSize = 10;
     try {
       vector.allocateNew(expectedAllocationInBytes, 10);
-      assertEquals(expectedOffsetSize, vector.getValueCapacity());
-      assertEquals(expectedAllocationInBytes, vector.getBuffer().capacity());
+      assertTrue(expectedOffsetSize <= vector.getValueCapacity());
+      assertTrue(expectedAllocationInBytes <= vector.getBuffer().capacity());
       vector.reAlloc();
-      assertEquals(expectedOffsetSize * 2, vector.getValueCapacity());
-      assertEquals(expectedAllocationInBytes * 2, vector.getBuffer().capacity());
+      assertTrue(expectedOffsetSize * 2 <= vector.getValueCapacity());
+      assertTrue(expectedAllocationInBytes * 2 <= vector.getBuffer().capacity());
     } finally {
       vector.close();
     }
@@ -666,8 +666,11 @@ public void testVVInitialCapacity() throws Exception {
       for (int i = 0; i < valueVectors.length; i++) {
         final ValueVector vv = valueVectors[i];
         final int vvCapacity = vv.getValueCapacity();
-        assertEquals(String.format("Incorrect value capacity for %s [%d]", vv.getField(), vvCapacity),
-            initialCapacity, vvCapacity);
+
+        // this can't be equality because Nullables will be allocated using power of two sized buffers (thus need 1025
+        // spots in one vector > power of two is 2048, available capacity will be 2048 => 2047)
+        assertTrue(String.format("Incorrect value capacity for %s [%d]", vv.getField(), vvCapacity),
+            initialCapacity <= vvCapacity);
       }
     } finally {
       AutoCloseables.close(valueVectors);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
index 107f9784ce..5c5c6302b3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
@@ -18,12 +18,17 @@
 package org.apache.drill.exec.server;
 
 import static org.junit.Assert.assertTrue;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.DrillBuf;
 
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
+import mockit.Injectable;
+import mockit.NonStrictExpectations;
+
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;
@@ -60,11 +65,6 @@
 import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
-import mockit.Injectable;
-import mockit.NonStrictExpectations;
-
 public class TestBitRpc extends ExecTest {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestBitRpc.class);
 
@@ -86,7 +86,7 @@ public void testConnectionBackpressure(@Injectable WorkerBee bee, @Injectable fi
     int port = 1234;
 
     DataResponseHandler drp = new BitComTestHandler();
-    DataServer server = new DataServer(c, workBus, drp);
+    DataServer server = new DataServer(c, c.getAllocator(), workBus, drp);
 
     port = server.bind(port, true);
     DrillbitEndpoint ep = DrillbitEndpoint.newBuilder().setAddress("localhost").setDataPort(port).build();
@@ -173,8 +173,7 @@ public void handle(FragmentManager manager, FragmentRecordBatch fragmentBatch, D
           Thread.sleep(3000);
         }
       } catch (InterruptedException e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
+
       }
       sender.sendOk();
     }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestResourceLeak.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestResourceLeak.java
index dc539c58ea..d2f2590df4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestResourceLeak.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestResourceLeak.java
@@ -17,11 +17,15 @@
  */
 package org.apache.drill.exec.testing;
 
-import com.google.common.base.Charsets;
-import com.google.common.io.Resources;
-
+import static org.junit.Assert.fail;
 import io.netty.buffer.DrillBuf;
 
+import java.io.IOException;
+import java.net.URL;
+import java.util.Properties;
+
+import javax.inject.Inject;
+
 import org.apache.drill.QueryTestUtil;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.UserRemoteException;
@@ -40,16 +44,25 @@
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.test.DrillTest;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
-import javax.inject.Inject;
-
-import java.io.IOException;
-import java.net.URL;
-import java.util.Properties;
+import com.google.common.base.Charsets;
+import com.google.common.io.Resources;
 
+/*
+ * TODO(DRILL-3170)
+ * This test had to be ignored because while the test case tpch01 works, the test
+ * fails overall because the final allocator closure again complains about
+ * outstanding resources. This could be fixed if we introduced a means to force
+ * cleanup of an allocator and all of its descendant resources. But that's a
+ * non-trivial exercise in the face of the ability to transfer ownership of
+ * slices of a buffer; we can't be sure it is safe to release an
+ * UnsafeDirectLittleEndian that an allocator believes it owns if slices of that
+ * have been transferred to another allocator.
+ */
+@Ignore
 public class TestResourceLeak extends DrillTest {
 
   private static DrillClient client;
@@ -83,12 +96,12 @@ public void tpch01() throws Exception {
     try {
       QueryTestUtil.test(client, "alter session set `planner.slice_target` = 10; " + query);
     } catch (UserRemoteException e) {
-      if (e.getMessage().contains("Attempted to close accountor")) {
+      if (e.getMessage().contains("Allocator closed with outstanding buffers allocated")) {
         return;
       }
       throw e;
     }
-    Assert.fail("Expected UserRemoteException indicating memory leak");
+    fail("Expected UserRemoteException indicating memory leak");
   }
 
   private static String getFile(String resource) throws IOException {
@@ -123,7 +136,7 @@ public void setup() {}
 
     @Override
     public void eval() {
-      buf.getAllocator().buffer(1);
+      buf.retain();
       out.value = in.value;
     }
   }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestPromotableWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestPromotableWriter.java
index 60a2268b4c..223f4edb08 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestPromotableWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestPromotableWriter.java
@@ -20,8 +20,6 @@
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.RootAllocatorFactory;
-import org.apache.drill.exec.memory.TopLevelAllocator;
-import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.store.TestOutputMutator;
 import org.apache.drill.exec.util.BatchPrinter;
 import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
diff --git a/exec/memory/base/pom.xml b/exec/memory/base/pom.xml
index c841e724ac..6ae2c50695 100644
--- a/exec/memory/base/pom.xml
+++ b/exec/memory/base/pom.xml
@@ -28,7 +28,17 @@
       <version>3.0.1</version>
     </dependency>
 
+    <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
 
+    <dependency>
+      <groupId>com.carrotsearch</groupId>
+      <artifactId>hppc</artifactId>
+      <version>0.5.2</version>
+    </dependency>
   </dependencies>
 
 
diff --git a/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java b/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
index b85502ba10..e9aa0c0da8 100644
--- a/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
+++ b/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
@@ -27,111 +27,69 @@
 import java.nio.channels.GatheringByteChannel;
 import java.nio.channels.ScatteringByteChannel;
 import java.nio.charset.Charset;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.drill.exec.memory.Accountor;
+import org.apache.drill.common.HistoricalLog;
+import org.apache.drill.exec.memory.AllocatorManager.BufferLedger;
+import org.apache.drill.exec.memory.BaseAllocator;
+import org.apache.drill.exec.memory.BaseAllocator.Verbosity;
 import org.apache.drill.exec.memory.BoundsChecking;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.BufferManager;
+import org.slf4j.Logger;
 
+import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 
 public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillBuf.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillBuf.class);
 
-  private static final boolean BOUNDS_CHECKING_ENABLED = BoundsChecking.BOUNDS_CHECKING_ENABLED;
+  private static final AtomicLong idGenerator = new AtomicLong(0);
 
-  private final ByteBuf b;
+  private final long id = idGenerator.incrementAndGet();
+  private final AtomicInteger refCnt;
+  private final UnsafeDirectLittleEndian byteBuf;
   private final long addr;
   private final int offset;
-  private final boolean rootBuffer;
-  private final AtomicLong rootRefCnt = new AtomicLong(1);
-  private volatile BufferAllocator allocator;
-  private volatile Accountor acct;
+  private final BufferLedger ledger;
+  private final BufferManager bufManager;
+  private final ByteBufAllocator alloc;
+  private final boolean isEmpty;
   private volatile int length;
 
-  // TODO - cleanup
-  // The code is partly shared and partly copy-pasted between
-  // these three types. They should be unified under one interface
-  // to share code and to remove the hacky code here to use only
-  // one of these types at a time and use null checks to find out
-  // which.
-  private BufferManager bufManager;
-
-  public DrillBuf(BufferAllocator allocator, Accountor a, UnsafeDirectLittleEndian b) {
-    super(b.maxCapacity());
-    this.b = b;
-    this.addr = b.memoryAddress();
-    this.acct = a;
-    this.length = b.capacity();
-    this.offset = 0;
-    this.rootBuffer = true;
-    this.allocator = allocator;
-  }
-
-  private DrillBuf(BufferAllocator allocator, Accountor a) {
-    super(0);
-    this.b = new EmptyByteBuf(allocator.getUnderlyingAllocator()).order(ByteOrder.LITTLE_ENDIAN);
-    this.allocator = allocator;
-    this.acct = a;
-    this.length = 0;
-    this.addr = 0;
-    this.rootBuffer = false;
-    this.offset = 0;
-  }
-
-  /**
-   * Special constructor used for RPC ownership transfer.  Takes a snapshot slice of the current buf
-   *  but points directly to the underlying UnsafeLittleEndian buffer.  Does this by calling unwrap()
-   *  twice on the provided DrillBuf and expecting an UnsafeDirectLittleEndian buffer. This operation
-   *  includes taking a new reference count on the underlying buffer and maintaining returning with a
-   *  current reference count for itself (masking the underlying reference count).
-   * @param allocator
-   * @param a Allocator used when users try to receive allocator from buffer.
-   * @param b Accountor used for accounting purposes.
-   */
-  public DrillBuf(BufferAllocator allocator, Accountor a, DrillBuf b) {
-    this(allocator, a, getUnderlying(b), b, 0, b.length, true);
-    assert b.unwrap().unwrap() instanceof UnsafeDirectLittleEndian;
-    b.unwrap().unwrap().retain();
-  }
-
-
-  private DrillBuf(DrillBuf buffer, int index, int length) {
-    this(buffer.allocator, null, buffer, buffer, index, length, false);
-  }
-
-  private static ByteBuf getUnderlying(DrillBuf b){
-    ByteBuf underlying = b.unwrap().unwrap();
-    return underlying.slice((int) (b.memoryAddress() - underlying.memoryAddress()), b.length);
-  }
-  private DrillBuf(BufferAllocator allocator, Accountor a, ByteBuf replacement, DrillBuf buffer, int index, int length, boolean root) {
-    super(length);
-    if (index < 0 || index > buffer.capacity() - length) {
-      throw new IndexOutOfBoundsException(buffer.toString() + ".slice(" + index + ", " + length + ')');
-    }
-
+  private final HistoricalLog historicalLog = BaseAllocator.DEBUG ?
+      new HistoricalLog(BaseAllocator.DEBUG_LOG_LENGTH, "DrillBuf[%d]", id) : null;
+
+  public DrillBuf(
+      final AtomicInteger refCnt,
+      final BufferLedger ledger,
+      final UnsafeDirectLittleEndian byteBuf,
+      final BufferManager manager,
+      final ByteBufAllocator alloc,
+      final int offset,
+      final int length,
+      boolean isEmpty) {
+    super(byteBuf.maxCapacity());
+    this.refCnt = refCnt;
+    this.byteBuf = byteBuf;
+    this.isEmpty = isEmpty;
+    this.bufManager = manager;
+    this.alloc = alloc;
+    this.addr = byteBuf.memoryAddress() + offset;
+    this.ledger = ledger;
     this.length = length;
-    writerIndex(length);
+    this.offset = offset;
 
-    this.b = replacement;
-    this.addr = buffer.memoryAddress() + index;
-    this.offset = index;
-    this.acct = a;
-    this.length = length;
-    this.rootBuffer = root;
-    this.allocator = allocator;
-  }
+    if (BaseAllocator.DEBUG) {
+      historicalLog.recordEvent("create()");
+    }
 
-  public void setBufferManager(BufferManager bufManager) {
-    this.bufManager = bufManager;
   }
 
-  public BufferAllocator getAllocator() {
-    return allocator;
-  }
+  public DrillBuf reallocIfNeeded(final int size) {
+    Preconditions.checkArgument(size >= 0, "reallocation size must be non-negative");
 
-  public DrillBuf reallocIfNeeded(int size) {
     if (this.capacity() >= size) {
       return this;
     }
@@ -141,37 +99,30 @@ public DrillBuf reallocIfNeeded(int size) {
     } else {
       throw new UnsupportedOperationException("Realloc is only available in the context of an operator's UDFs");
     }
-
   }
 
   @Override
   public int refCnt() {
-    if(rootBuffer){
-      return (int) this.rootRefCnt.get();
-    }else{
-      return b.refCnt();
+    if (isEmpty) {
+      return 1;
+    } else {
+      return refCnt.get();
     }
-
   }
 
   private long addr(int index) {
     return addr + index;
   }
 
-  private final void checkIndexD(int index) {
-    ensureAccessible();
-    if (index < 0 || index >= capacity()) {
-      throw new IndexOutOfBoundsException(String.format(
-              "index: %d (expected: range(0, %d))", index, capacity()));
-    }
-  }
-
   private final void checkIndexD(int index, int fieldLength) {
     ensureAccessible();
     if (fieldLength < 0) {
       throw new IllegalArgumentException("length: " + fieldLength + " (expected: >= 0)");
     }
     if (index < 0 || index > capacity() - fieldLength) {
+      if (BaseAllocator.DEBUG) {
+        historicalLog.logHistory(logger);
+      }
       throw new IndexOutOfBoundsException(String.format(
               "index: %d, length: %d (expected: range(0, %d))", index, fieldLength, capacity()));
     }
@@ -186,42 +137,110 @@ private final void checkIndexD(int index, int fieldLength) {
    * @param start The starting position of the bytes to be read.
    * @param end The exclusive endpoint of the bytes to be read.
    */
-  public void checkBytes(int start, int end){
-    if (BOUNDS_CHECKING_ENABLED) {
+  public void checkBytes(int start, int end) {
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
       checkIndexD(start, end - start);
     }
   }
 
   private void chk(int index, int width) {
-    if (BOUNDS_CHECKING_ENABLED) {
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
       checkIndexD(index, width);
     }
   }
 
-  private void chk(int index) {
-    if (BOUNDS_CHECKING_ENABLED) {
-      checkIndexD(index);
+  private void ensure(int width) {
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
+      ensureWritable(width);
     }
   }
 
-  private void ensure(int width) {
-    if (BOUNDS_CHECKING_ENABLED) {
-      ensureWritable(width);
+  /**
+   * Create a new DrillBuf that is associated with an alternative allocator for the purposes of memory ownership and
+   * accounting. This has no impact on the reference counting for this allocator.
+   *
+   * This operation has no impact on the reference count of this DrillBuf. The newly created DrillBuf with either have a
+   * reference count of 1 (in the case that this is the first time this memory is being associated with the new
+   * allocator) or the current value of the reference count + 1 for the other AllocatorManager/BufferLedger combination
+   * in the case that the provided allocator already had an association to this underlying memory.
+   *
+   * @param allocator
+   *          The target allocator to create an association with.
+   * @return A new DrillBuf which shares the same underlying memory as this DrillBuf.
+   */
+  public DrillBuf retain(BufferAllocator allocator) {
+
+    if (isEmpty) {
+      return this;
     }
+
+    if (BaseAllocator.DEBUG) {
+      historicalLog.recordEvent("retain(%s)", allocator.getName());
+    }
+    BufferLedger otherLedger = this.ledger.getLedgerForAllocator(allocator);
+    return otherLedger.newDrillBuf(offset, length, null, true);
   }
 
-  public boolean transferAccounting(Accountor target) {
-    if (rootBuffer) {
-      boolean outcome = acct.transferTo(target, this, length);
-      acct = target;
-      return outcome;
-    } else {
-      throw new UnsupportedOperationException();
+  /**
+   * Transfer the memory accounting ownership of this DrillBuf to another allocator. This will generate a new DrillBuf
+   * that carries an association with the underlying memory of this DrillBuf. If this DrillBuf is connected to the
+   * owning BufferLedger of this memory, that memory ownership/accounting will be transferred to the taret allocator. If
+   * this DrillBuf does not currently own the memory underlying it (and is only associated with it), this does not
+   * transfer any ownership to the newly created DrillBuf.
+   *
+   * This operation has no impact on the reference count of this DrillBuf. The newly created DrillBuf with either have a
+   * reference count of 1 (in the case that this is the first time this memory is being associated with the new
+   * allocator) or the current value of the reference count for the other AllocatorManager/BufferLedger combination in
+   * the case that the provided allocator already had an association to this underlying memory.
+   *
+   * Transfers will always succeed, even if that puts the other allocator into an overlimit situation. This is possible
+   * due to the fact that the original owning allocator may have allocated this memory out of a local reservation
+   * whereas the target allocator may need to allocate new memory from a parent or RootAllocator. This operation is done
+   * in a mostly-lockless but consistent manner. As such, the overlimit==true situation could occur slightly prematurely
+   * to an actual overlimit==true condition. This is simply conservative behavior which means we may return overlimit
+   * slightly sooner than is necessary.
+   *
+   * @param target
+   *          The allocator to transfer ownership to.
+   * @return A new transfer result with the impact of the transfer (whether it was overlimit) as well as the newly
+   *         created DrillBuf.
+   */
+  public TransferResult transferOwnership(BufferAllocator target) {
+
+    if (isEmpty) {
+      return new TransferResult(true, this);
     }
+
+    final BufferLedger otherLedger = this.ledger.getLedgerForAllocator(target);
+    final DrillBuf newBuf = otherLedger.newDrillBuf(offset, length, null, true);
+    final boolean allocationFit = this.ledger.transferBalance(otherLedger);
+    return new TransferResult(allocationFit, newBuf);
+  }
+
+  /**
+   * The outcome of a Transfer.
+   */
+  public class TransferResult {
+
+    /**
+     * Whether this transfer fit within the target allocator's capacity.
+     */
+    public final boolean allocationFit;
+
+    /**
+     * The newly created buffer associated with the target allocator.
+     */
+    public final DrillBuf buffer;
+
+    private TransferResult(boolean allocationFit, DrillBuf buffer) {
+      this.allocationFit = allocationFit;
+      this.buffer = buffer;
+    }
+
   }
 
   @Override
-  public synchronized boolean release() {
+  public boolean release() {
     return release(1);
   }
 
@@ -230,20 +249,31 @@ public synchronized boolean release() {
    */
   @Override
   public synchronized boolean release(int decrement) {
+    if (isEmpty) {
+      return false;
+    }
 
-    if(rootBuffer){
-      final long newRefCnt = this.rootRefCnt.addAndGet(-decrement);
-      Preconditions.checkArgument(newRefCnt > -1, "Buffer has negative reference count.");
-      if (newRefCnt == 0) {
-        b.release(decrement);
-        acct.release(this, length);
-        return true;
-      }else{
-        return false;
-      }
-    }else{
-      return b.release(decrement);
+    if (decrement < 1) {
+      throw new IllegalStateException(String.format("release(%d) argument is not positive. Buffer Info: %s",
+          decrement, toVerboseString()));
     }
+
+    if (BaseAllocator.DEBUG) {
+      historicalLog.recordEvent("release(%d)", decrement);
+    }
+
+    final int refCnt = this.refCnt.addAndGet(-decrement);
+    if (refCnt < 0) {
+      throw new IllegalStateException(
+          String.format("DrillBuf[%d] refCnt has gone negative. Buffer Info: %s", id, toVerboseString()));
+
+    }
+    if (refCnt == 0) {
+      ledger.release();
+      return true;
+    }
+
+    return false;
   }
 
   @Override
@@ -252,32 +282,25 @@ public int capacity() {
   }
 
   @Override
-  public synchronized ByteBuf capacity(int newCapacity) {
-    if (rootBuffer) {
-      if (newCapacity == length) {
-        return this;
-      } else if (newCapacity < length) {
-        b.capacity(newCapacity);
-        int diff = length - b.capacity();
-        acct.releasePartial(this, diff);
-        this.length = length - diff;
-        return this;
-      } else {
-        throw new UnsupportedOperationException("Accounting byte buf doesn't support increasing allocations.");
-      }
-    } else {
-      throw new UnsupportedOperationException("Non root bufs doen't support changing allocations.");
+  public synchronized DrillBuf capacity(int newCapacity) {
+
+    if (newCapacity == length) {
+      return this;
     }
-  }
 
-  @Override
-  public int maxCapacity() {
-    return length;
+    Preconditions.checkArgument(newCapacity >= 0);
+
+    if (newCapacity < length) {
+      length = newCapacity;
+      return this;
+    }
+
+    throw new UnsupportedOperationException("Buffers don't support resizing that increases the size.");
   }
 
   @Override
   public ByteBufAllocator alloc() {
-    return b.alloc();
+    return byteBuf.alloc();
   }
 
   @Override
@@ -287,14 +310,12 @@ public ByteOrder order() {
 
   @Override
   public ByteBuf order(ByteOrder endianness) {
-    // if(endianness != ByteOrder.LITTLE_ENDIAN) throw new
-    // UnsupportedOperationException("Drill buffers only support little endian.");
     return this;
   }
 
   @Override
   public ByteBuf unwrap() {
-    return b;
+    return byteBuf;
   }
 
   @Override
@@ -309,7 +330,7 @@ public ByteBuf readBytes(int length) {
 
   @Override
   public ByteBuf readSlice(int length) {
-    ByteBuf slice = slice(readerIndex(), length);
+    final ByteBuf slice = slice(readerIndex(), length);
     readerIndex(readerIndex() + length);
     return slice;
   }
@@ -329,16 +350,37 @@ public ByteBuf slice() {
     return slice(readerIndex(), readableBytes());
   }
 
+  public static String bufferState(final ByteBuf buf) {
+    final int cap = buf.capacity();
+    final int mcap = buf.maxCapacity();
+    final int ri = buf.readerIndex();
+    final int rb = buf.readableBytes();
+    final int wi = buf.writerIndex();
+    final int wb = buf.writableBytes();
+    return String.format("cap/max: %d/%d, ri: %d, rb: %d, wi: %d, wb: %d",
+        cap, mcap, ri, rb, wi, wb);
+  }
+
   @Override
   public DrillBuf slice(int index, int length) {
-    DrillBuf buf = new DrillBuf(this, index, length);
-    buf.writerIndex = length;
-    return buf;
+
+    if (isEmpty) {
+      return this;
+    }
+
+    /*
+     * Re the behavior of reference counting,
+     * see http://netty.io/wiki/reference-counted-objects.html#wiki-h3-5, which explains
+     * that derived buffers share their reference count with their parent
+     */
+    final DrillBuf newBuf = ledger.newDrillBuf(offset + index, length);
+    newBuf.writerIndex(length);
+    return newBuf;
   }
 
   @Override
   public DrillBuf duplicate() {
-    return new DrillBuf(this, 0, length);
+    return slice(0, length);
   }
 
   @Override
@@ -353,12 +395,12 @@ public ByteBuffer nioBuffer() {
 
   @Override
   public ByteBuffer nioBuffer(int index, int length) {
-    return b.nioBuffer(offset + index, length);
+    return byteBuf.nioBuffer(offset + index, length);
   }
 
   @Override
   public ByteBuffer internalNioBuffer(int index, int length) {
-    return b.internalNioBuffer(offset + index, length);
+    return byteBuf.internalNioBuffer(offset + index, length);
   }
 
   @Override
@@ -373,17 +415,17 @@ public ByteBuffer internalNioBuffer(int index, int length) {
 
   @Override
   public boolean hasArray() {
-    return b.hasArray();
+    return byteBuf.hasArray();
   }
 
   @Override
   public byte[] array() {
-    return b.array();
+    return byteBuf.array();
   }
 
   @Override
   public int arrayOffset() {
-    return b.arrayOffset();
+    return byteBuf.arrayOffset();
   }
 
   @Override
@@ -396,18 +438,27 @@ public long memoryAddress() {
     return this.addr;
   }
 
+  @Override
+  public String toString() {
+    return toString(0, 0, Charsets.UTF_8);
+  }
+
   @Override
   public String toString(Charset charset) {
-      return toString(readerIndex, readableBytes(), charset);
+    return toString(readerIndex, readableBytes(), charset);
   }
 
   @Override
   public String toString(int index, int length, Charset charset) {
+    final String basics =
+        String.format("{DrillBuf[%d], udle identityHashCode == %d, identityHashCode == %d}",
+            id, System.identityHashCode(byteBuf), System.identityHashCode(refCnt));
+
     if (length == 0) {
-      return "";
+      return basics;
     }
 
-    ByteBuffer nioBuffer;
+    final ByteBuffer nioBuffer;
     if (nioBufferCount() == 1) {
       nioBuffer = nioBuffer(index, length);
     } else {
@@ -416,7 +467,7 @@ public String toString(int index, int length, Charset charset) {
       nioBuffer.flip();
     }
 
-    return ByteBufUtil.decodeString(nioBuffer, charset);
+    return basics + '\n' + ByteBufUtil.decodeString(nioBuffer, charset);
   }
 
   @Override
@@ -431,12 +482,18 @@ public boolean equals(Object obj) {
   }
 
   @Override
-  public synchronized ByteBuf retain(int increment) {
-    if(rootBuffer){
-      this.rootRefCnt.addAndGet(increment);
-    }else{
-      b.retain(increment);
+  public ByteBuf retain(int increment) {
+    Preconditions.checkArgument(increment > 0, "retain(%d) argument is not positive", increment);
+
+    if (isEmpty) {
+      return this;
+    }
+
+    if (BaseAllocator.DEBUG) {
+      historicalLog.recordEvent("retain(%d)", increment);
     }
+
+    refCnt.addAndGet(increment);
     return this;
   }
 
@@ -448,7 +505,7 @@ public ByteBuf retain() {
   @Override
   public long getLong(int index) {
     chk(index, 8);
-    long v = PlatformDependent.getLong(addr(index));
+    final long v = PlatformDependent.getLong(addr(index));
     return v;
   }
 
@@ -475,7 +532,7 @@ public long getUnsignedInt(int index) {
   @Override
   public int getInt(int index) {
     chk(index, 4);
-    int v = PlatformDependent.getInt(addr(index));
+    final int v = PlatformDependent.getInt(addr(index));
     return v;
   }
 
@@ -583,13 +640,13 @@ public ByteBuf writeDouble(double value) {
 
   @Override
   public ByteBuf getBytes(int index, byte[] dst, int dstIndex, int length) {
-    b.getBytes(index + offset,  dst, dstIndex, length);
+    byteBuf.getBytes(index + offset, dst, dstIndex, length);
     return this;
   }
 
   @Override
   public ByteBuf getBytes(int index, ByteBuffer dst) {
-    b.getBytes(index + offset, dst);
+    byteBuf.getBytes(index + offset, dst);
     return this;
   }
 
@@ -657,19 +714,19 @@ protected void _setLong(int index, long value) {
 
   @Override
   public ByteBuf getBytes(int index, ByteBuf dst, int dstIndex, int length) {
-    b.getBytes(index + offset, dst, dstIndex, length);
+    byteBuf.getBytes(index + offset, dst, dstIndex, length);
     return this;
   }
 
   @Override
   public ByteBuf getBytes(int index, OutputStream out, int length) throws IOException {
-    b.getBytes(index + offset, out, length);
+    byteBuf.getBytes(index + offset, out, length);
     return this;
   }
 
   @Override
   protected int _getUnsignedMedium(int index) {
-    long addr = addr(index);
+    final long addr = addr(index);
     return (PlatformDependent.getByte(addr) & 0xff) << 16 |
             (PlatformDependent.getByte(addr + 1) & 0xff) << 8 |
             PlatformDependent.getByte(addr + 2) & 0xff;
@@ -677,12 +734,12 @@ protected int _getUnsignedMedium(int index) {
 
   @Override
   public int getBytes(int index, GatheringByteChannel out, int length) throws IOException {
-    return b.getBytes(index + offset, out, length);
+    return byteBuf.getBytes(index + offset, out, length);
   }
 
   @Override
   public ByteBuf setBytes(int index, ByteBuf src, int srcIndex, int length) {
-    b.setBytes(index + offset, src, srcIndex, length);
+    byteBuf.setBytes(index + offset, src, srcIndex, length);
     return this;
   }
 
@@ -693,12 +750,12 @@ public ByteBuf setBytes(int index, ByteBuffer src, int srcIndex, int length) {
           length);
     } else {
       if (srcIndex == 0 && src.capacity() == length) {
-        b.setBytes(index + offset, src);
+        byteBuf.setBytes(index + offset, src);
       } else {
         ByteBuffer newBuf = src.duplicate();
         newBuf.position(srcIndex);
         newBuf.limit(srcIndex + length);
-        b.setBytes(index + offset, src);
+        byteBuf.setBytes(index + offset, src);
       }
     }
 
@@ -707,24 +764,24 @@ public ByteBuf setBytes(int index, ByteBuffer src, int srcIndex, int length) {
 
   @Override
   public ByteBuf setBytes(int index, byte[] src, int srcIndex, int length) {
-    b.setBytes(index + offset, src, srcIndex, length);
+    byteBuf.setBytes(index + offset, src, srcIndex, length);
     return this;
   }
 
   @Override
   public ByteBuf setBytes(int index, ByteBuffer src) {
-    b.setBytes(index + offset, src);
+    byteBuf.setBytes(index + offset, src);
     return this;
   }
 
   @Override
   public int setBytes(int index, InputStream in, int length) throws IOException {
-    return b.setBytes(index + offset, in, length);
+    return byteBuf.setBytes(index + offset, in, length);
   }
 
   @Override
   public int setBytes(int index, ScatteringByteChannel in, int length) throws IOException {
-    return b.setBytes(index + offset, in, length);
+    return byteBuf.setBytes(index + offset, in, length);
   }
 
   @Override
@@ -733,17 +790,98 @@ public byte getByte(int index) {
     return PlatformDependent.getByte(addr(index));
   }
 
-  public static DrillBuf getEmpty(BufferAllocator allocator, Accountor a) {
-    return new DrillBuf(allocator, a);
+  @Override
+  public void close() {
+    release();
   }
 
-  public boolean isRootBuffer() {
-    return rootBuffer;
+  /**
+   * Returns the possible memory consumed by this DrillBuf in the worse case scenario. (not shared, connected to larger
+   * underlying buffer of allocated memory)
+   *
+   * @return Size in bytes.
+   */
+  public int getPossibleMemoryConsumed() {
+    return ledger.getSize();
   }
 
-  @Override
-  public void close() {
-    release();
+  /**
+   * Return that is Accounted for by this buffer (and its potentially shared siblings within the context of the
+   * associated allocator).
+   *
+   * @return Size in bytes.
+   */
+  public int getActualMemoryConsumed() {
+    return ledger.getAccountedSize();
+  }
+
+  private final static int LOG_BYTES_PER_ROW = 10;
+  /**
+   * Log this buffer's byte contents in the form of a hex dump.
+   *
+   * @param logger where to log to
+   * @param start the starting byte index
+   * @param length how many bytes to log
+   */
+  public void logBytes(final Logger logger, final int start, final int length) {
+    final int roundedStart = (start / LOG_BYTES_PER_ROW) * LOG_BYTES_PER_ROW;
+
+    final StringBuilder sb = new StringBuilder("buffer byte dump\n");
+    int index = roundedStart;
+    for(int nLogged = 0; nLogged < length; nLogged += LOG_BYTES_PER_ROW) {
+      sb.append(String.format(" [%05d-%05d]", index, index + LOG_BYTES_PER_ROW - 1));
+      for(int i = 0; i < LOG_BYTES_PER_ROW; ++i) {
+        try {
+          final byte b = getByte(index++);
+          sb.append(String.format(" 0x%02x", b));
+        } catch(IndexOutOfBoundsException ioob) {
+          sb.append(" <ioob>");
+        }
+      }
+      sb.append('\n');
+    }
+    logger.trace(sb.toString());
+  }
+
+  /**
+   * Get the integer id assigned to this DrillBuf for debugging purposes.
+   *
+   * @return integer id
+   */
+  public long getId() {
+    return id;
+  }
+
+  /**
+   * Log this buffer's history.
+   *
+   * @param logger the logger to use
+   */
+  public void logHistory(final Logger logger) {
+    if (historicalLog == null) {
+      logger.warn("DrillBuf[{}] historicalLog not available", id);
+    } else {
+      historicalLog.logHistory(logger);
+    }
+  }
+
+  public String toVerboseString() {
+    if (isEmpty) {
+      return toString();
+    }
+
+    StringBuilder sb = new StringBuilder();
+    ledger.print(sb, 0, Verbosity.LOG_WITH_STACKTRACE);
+    return sb.toString();
+  }
+
+  public void print(StringBuilder sb, int indent, Verbosity verbosity) {
+    BaseAllocator.indent(sb, indent).append(toString());
+
+    if (BaseAllocator.DEBUG && !isEmpty && verbosity.includeHistoricalLog) {
+      sb.append("\n");
+      historicalLog.buildHistory(sb, indent + 1, verbosity.includeStackTraces);
+    }
   }
 
 }
diff --git a/exec/memory/base/src/main/java/io/netty/buffer/ExpandableByteBuf.java b/exec/memory/base/src/main/java/io/netty/buffer/ExpandableByteBuf.java
new file mode 100644
index 0000000000..7788552872
--- /dev/null
+++ b/exec/memory/base/src/main/java/io/netty/buffer/ExpandableByteBuf.java
@@ -0,0 +1,55 @@
+/**
+ * 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 io.netty.buffer;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+
+/**
+ * Allows us to decorate DrillBuf to make it expandable so that we can use them in the context of the Netty framework
+ * (thus supporting RPC level memory accounting).
+ */
+public class ExpandableByteBuf extends MutableWrappedByteBuf {
+
+  private final BufferAllocator allocator;
+
+  public ExpandableByteBuf(ByteBuf buffer, BufferAllocator allocator) {
+    super(buffer);
+    this.allocator = allocator;
+  }
+
+  @Override
+  public ByteBuf copy(int index, int length) {
+    return new ExpandableByteBuf(buffer.copy(index, length), allocator);
+  }
+
+  @Override
+  public ByteBuf capacity(int newCapacity) {
+    if (newCapacity > capacity()) {
+      ByteBuf newBuf = allocator.buffer(newCapacity);
+      newBuf.writeBytes(buffer, 0, buffer.capacity());
+      newBuf.readerIndex(buffer.readerIndex());
+      newBuf.writerIndex(buffer.writerIndex());
+      buffer.release();
+      buffer = newBuf;
+      return newBuf;
+    } else {
+      return super.capacity(newCapacity);
+    }
+  }
+
+}
diff --git a/exec/memory/base/src/main/java/io/netty/buffer/FakeAllocator.java b/exec/memory/base/src/main/java/io/netty/buffer/FakeAllocator.java
deleted file mode 100644
index 53ca91c0e3..0000000000
--- a/exec/memory/base/src/main/java/io/netty/buffer/FakeAllocator.java
+++ /dev/null
@@ -1,187 +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 io.netty.buffer;
-
-import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.memory.Accountor;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.memory.LimitConsumer;
-import org.apache.drill.exec.util.Pointer;
-
-class FakeAllocator implements BufferAllocator {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FakeAllocator.class);
-
-
-  public static final Accountor FAKE_ACCOUNTOR = new FakeAccountor();
-  public static final BufferAllocator FAKE_ALLOCATOR = new FakeAllocator();
-
-  @Override
-  public DrillBuf buffer(int size) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public DrillBuf buffer(int minSize, int maxSize) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public ByteBufAllocator getUnderlyingAllocator() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public BufferAllocator getChildAllocator(LimitConsumer consumer, long initialReservation, long maximumReservation,
-                                           boolean applyFragmentLimit)
-      throws OutOfMemoryException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public DrillBuf getEmpty() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean takeOwnership(DrillBuf buf) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public PreAllocator getNewPreAllocator() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void resetLimits() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setLimit(long l) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public long getLimit(){
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void close() {
-  }
-
-  @Override
-  public long getAllocatedMemory() {
-    return 0;
-  }
-
-  @Override
-  public long getPeakMemoryAllocation() {
-    return 0;
-  }
-
-  static class FakeAccountor implements Accountor {
-
-    public FakeAccountor() {
-    }
-
-    @Override
-    public boolean transferTo(Accountor target, DrillBuf buf, long size) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public long getAvailable() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public long getCapacity() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public long getAllocation() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean reserve(long size) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean forceAdditionalReservation(long size) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void reserved(long expected, DrillBuf buf) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void releasePartial(DrillBuf buf, long size) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void release(DrillBuf buf, long size) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void close() {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean transferIn(DrillBuf buf, long size) {
-      return false;
-    }
-
-    @Override
-    public long getPeakMemoryAllocation() {
-      return 0;
-    }
-
-    @Override
-    public long resetFragmentLimits() {
-      return 0;
-    }
-
-    @Override
-    public void setFragmentLimit(long add) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public long getFragmentLimit() {
-      return 0;
-    }
-
-
-  }
-
-  @Override
-  public boolean takeOwnership(DrillBuf buf, Pointer<DrillBuf> bufOut) {
-    throw new UnsupportedOperationException();
-  }
-
-}
diff --git a/exec/memory/base/src/main/java/io/netty/buffer/LargeBuffer.java b/exec/memory/base/src/main/java/io/netty/buffer/LargeBuffer.java
index f1d48420a0..5f5e904fb0 100644
--- a/exec/memory/base/src/main/java/io/netty/buffer/LargeBuffer.java
+++ b/exec/memory/base/src/main/java/io/netty/buffer/LargeBuffer.java
@@ -17,151 +17,23 @@
  */
 package io.netty.buffer;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.channels.GatheringByteChannel;
-import java.nio.channels.ScatteringByteChannel;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
- * This is basically a complete copy of DuplicatedByteBuf. We copy because we can't override the release methods to keep
- * global track of created Large Buffers.
+ * A MutableWrappedByteBuf that also maintains a metric of the number of huge buffer bytes and counts.
  */
-public class LargeBuffer extends AbstractByteBuf {
+public class LargeBuffer extends MutableWrappedByteBuf {
 
   private final AtomicLong hugeBufferSize;
   private final AtomicLong hugeBufferCount;
 
-  @Override
-  public ByteBuffer nioBuffer(int index, int length) {
-    return unwrap().nioBuffer(index, length);
-  }
-
-  private final ByteBuf buffer;
   private final int initCap;
 
   public LargeBuffer(ByteBuf buffer, AtomicLong hugeBufferSize, AtomicLong hugeBufferCount) {
-    super(buffer.maxCapacity());
+    super(buffer);
     initCap = buffer.capacity();
     this.hugeBufferCount = hugeBufferCount;
     this.hugeBufferSize = hugeBufferSize;
-
-    if (buffer instanceof LargeBuffer) {
-      this.buffer = ((LargeBuffer) buffer).buffer;
-    } else {
-      this.buffer = buffer;
-    }
-
-    setIndex(buffer.readerIndex(), buffer.writerIndex());
-  }
-
-  @Override
-  public ByteBuf unwrap() {
-    return buffer;
-  }
-
-  @Override
-  public ByteBufAllocator alloc() {
-    return buffer.alloc();
-  }
-
-  @Override
-  public ByteOrder order() {
-    return buffer.order();
-  }
-
-  @Override
-  public boolean isDirect() {
-    return buffer.isDirect();
-  }
-
-  @Override
-  public int capacity() {
-    return buffer.capacity();
-  }
-
-  @Override
-  public ByteBuf capacity(int newCapacity) {
-    buffer.capacity(newCapacity);
-    return this;
-  }
-
-  @Override
-  public boolean hasArray() {
-    return buffer.hasArray();
-  }
-
-  @Override
-  public byte[] array() {
-    return buffer.array();
-  }
-
-  @Override
-  public int arrayOffset() {
-    return buffer.arrayOffset();
-  }
-
-  @Override
-  public boolean hasMemoryAddress() {
-    return buffer.hasMemoryAddress();
-  }
-
-  @Override
-  public long memoryAddress() {
-    return buffer.memoryAddress();
-  }
-
-  @Override
-  public byte getByte(int index) {
-    return _getByte(index);
-  }
-
-  @Override
-  protected byte _getByte(int index) {
-    return buffer.getByte(index);
-  }
-
-  @Override
-  public short getShort(int index) {
-    return _getShort(index);
-  }
-
-  @Override
-  protected short _getShort(int index) {
-    return buffer.getShort(index);
-  }
-
-  @Override
-  public int getUnsignedMedium(int index) {
-    return _getUnsignedMedium(index);
-  }
-
-  @Override
-  protected int _getUnsignedMedium(int index) {
-    return buffer.getUnsignedMedium(index);
-  }
-
-  @Override
-  public int getInt(int index) {
-    return _getInt(index);
-  }
-
-  @Override
-  protected int _getInt(int index) {
-    return buffer.getInt(index);
-  }
-
-  @Override
-  public long getLong(int index) {
-    return _getLong(index);
-  }
-
-  @Override
-  protected long _getLong(int index) {
-    return buffer.getLong(index);
   }
 
   @Override
@@ -169,169 +41,6 @@ public ByteBuf copy(int index, int length) {
     return new LargeBuffer(buffer.copy(index, length), hugeBufferSize, hugeBufferCount);
   }
 
-  @Override
-  public ByteBuf slice(int index, int length) {
-    return new SlicedByteBuf(this, index, length);
-  }
-
-  @Override
-  public ByteBuf getBytes(int index, ByteBuf dst, int dstIndex, int length) {
-    buffer.getBytes(index, dst, dstIndex, length);
-    return this;
-  }
-
-  @Override
-  public ByteBuf getBytes(int index, byte[] dst, int dstIndex, int length) {
-    buffer.getBytes(index, dst, dstIndex, length);
-    return this;
-  }
-
-  @Override
-  public ByteBuf getBytes(int index, ByteBuffer dst) {
-    buffer.getBytes(index, dst);
-    return this;
-  }
-
-  @Override
-  public ByteBuf setByte(int index, int value) {
-    _setByte(index, value);
-    return this;
-  }
-
-  @Override
-  protected void _setByte(int index, int value) {
-    buffer.setByte(index, value);
-  }
-
-  @Override
-  public ByteBuf setShort(int index, int value) {
-    _setShort(index, value);
-    return this;
-  }
-
-  @Override
-  protected void _setShort(int index, int value) {
-    buffer.setShort(index, value);
-  }
-
-  @Override
-  public ByteBuf setMedium(int index, int value) {
-    _setMedium(index, value);
-    return this;
-  }
-
-  @Override
-  protected void _setMedium(int index, int value) {
-    buffer.setMedium(index, value);
-  }
-
-  @Override
-  public ByteBuf setInt(int index, int value) {
-    _setInt(index, value);
-    return this;
-  }
-
-  @Override
-  protected void _setInt(int index, int value) {
-    buffer.setInt(index, value);
-  }
-
-  @Override
-  public ByteBuf setLong(int index, long value) {
-    _setLong(index, value);
-    return this;
-  }
-
-  @Override
-  protected void _setLong(int index, long value) {
-    buffer.setLong(index, value);
-  }
-
-  @Override
-  public ByteBuf setBytes(int index, byte[] src, int srcIndex, int length) {
-    buffer.setBytes(index, src, srcIndex, length);
-    return this;
-  }
-
-  @Override
-  public ByteBuf setBytes(int index, ByteBuf src, int srcIndex, int length) {
-    buffer.setBytes(index, src, srcIndex, length);
-    return this;
-  }
-
-  @Override
-  public ByteBuf setBytes(int index, ByteBuffer src) {
-    buffer.setBytes(index, src);
-    return this;
-  }
-
-  @Override
-  public ByteBuf getBytes(int index, OutputStream out, int length)
-      throws IOException {
-    buffer.getBytes(index, out, length);
-    return this;
-  }
-
-  @Override
-  public int getBytes(int index, GatheringByteChannel out, int length)
-      throws IOException {
-    return buffer.getBytes(index, out, length);
-  }
-
-  @Override
-  public int setBytes(int index, InputStream in, int length)
-      throws IOException {
-    return buffer.setBytes(index, in, length);
-  }
-
-  @Override
-  public int setBytes(int index, ScatteringByteChannel in, int length)
-      throws IOException {
-    return buffer.setBytes(index, in, length);
-  }
-
-  @Override
-  public int nioBufferCount() {
-    return buffer.nioBufferCount();
-  }
-
-  @Override
-  public ByteBuffer[] nioBuffers(int index, int length) {
-    return buffer.nioBuffers(index, length);
-  }
-
-  @Override
-  public ByteBuffer internalNioBuffer(int index, int length) {
-    return nioBuffer(index, length);
-  }
-
-  @Override
-  public int forEachByte(int index, int length, ByteBufProcessor processor) {
-    return buffer.forEachByte(index, length, processor);
-  }
-
-  @Override
-  public int forEachByteDesc(int index, int length, ByteBufProcessor processor) {
-    return buffer.forEachByteDesc(index, length, processor);
-  }
-
-  @Override
-  public final int refCnt() {
-    return unwrap().refCnt();
-  }
-
-  @Override
-  public final ByteBuf retain() {
-    unwrap().retain();
-    return this;
-  }
-
-  @Override
-  public final ByteBuf retain(int increment) {
-    unwrap().retain(increment);
-    return this;
-  }
-
   @Override
   public boolean release() {
     return release(1);
diff --git a/exec/memory/base/src/main/java/io/netty/buffer/MutableWrappedByteBuf.java b/exec/memory/base/src/main/java/io/netty/buffer/MutableWrappedByteBuf.java
new file mode 100644
index 0000000000..5709473135
--- /dev/null
+++ b/exec/memory/base/src/main/java/io/netty/buffer/MutableWrappedByteBuf.java
@@ -0,0 +1,336 @@
+/**
+ * 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 io.netty.buffer;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.GatheringByteChannel;
+import java.nio.channels.ScatteringByteChannel;
+
+/**
+ * This is basically a complete copy of DuplicatedByteBuf. We copy because we want to override some behaviors and make
+ * buffer mutable.
+ */
+abstract class MutableWrappedByteBuf extends AbstractByteBuf {
+
+  @Override
+  public ByteBuffer nioBuffer(int index, int length) {
+    return unwrap().nioBuffer(index, length);
+  }
+
+  ByteBuf buffer;
+
+  public MutableWrappedByteBuf(ByteBuf buffer) {
+    super(buffer.maxCapacity());
+
+    if (buffer instanceof MutableWrappedByteBuf) {
+      this.buffer = ((MutableWrappedByteBuf) buffer).buffer;
+    } else {
+      this.buffer = buffer;
+    }
+
+    setIndex(buffer.readerIndex(), buffer.writerIndex());
+  }
+
+  @Override
+  public ByteBuf unwrap() {
+    return buffer;
+  }
+
+  @Override
+  public ByteBufAllocator alloc() {
+    return buffer.alloc();
+  }
+
+  @Override
+  public ByteOrder order() {
+    return buffer.order();
+  }
+
+  @Override
+  public boolean isDirect() {
+    return buffer.isDirect();
+  }
+
+  @Override
+  public int capacity() {
+    return buffer.capacity();
+  }
+
+  @Override
+  public ByteBuf capacity(int newCapacity) {
+    buffer.capacity(newCapacity);
+    return this;
+  }
+
+  @Override
+  public boolean hasArray() {
+    return buffer.hasArray();
+  }
+
+  @Override
+  public byte[] array() {
+    return buffer.array();
+  }
+
+  @Override
+  public int arrayOffset() {
+    return buffer.arrayOffset();
+  }
+
+  @Override
+  public boolean hasMemoryAddress() {
+    return buffer.hasMemoryAddress();
+  }
+
+  @Override
+  public long memoryAddress() {
+    return buffer.memoryAddress();
+  }
+
+  @Override
+  public byte getByte(int index) {
+    return _getByte(index);
+  }
+
+  @Override
+  protected byte _getByte(int index) {
+    return buffer.getByte(index);
+  }
+
+  @Override
+  public short getShort(int index) {
+    return _getShort(index);
+  }
+
+  @Override
+  protected short _getShort(int index) {
+    return buffer.getShort(index);
+  }
+
+  @Override
+  public int getUnsignedMedium(int index) {
+    return _getUnsignedMedium(index);
+  }
+
+  @Override
+  protected int _getUnsignedMedium(int index) {
+    return buffer.getUnsignedMedium(index);
+  }
+
+  @Override
+  public int getInt(int index) {
+    return _getInt(index);
+  }
+
+  @Override
+  protected int _getInt(int index) {
+    return buffer.getInt(index);
+  }
+
+  @Override
+  public long getLong(int index) {
+    return _getLong(index);
+  }
+
+  @Override
+  protected long _getLong(int index) {
+    return buffer.getLong(index);
+  }
+
+  @Override
+  public abstract ByteBuf copy(int index, int length);
+
+  @Override
+  public ByteBuf slice(int index, int length) {
+    return new SlicedByteBuf(this, index, length);
+  }
+
+  @Override
+  public ByteBuf getBytes(int index, ByteBuf dst, int dstIndex, int length) {
+    buffer.getBytes(index, dst, dstIndex, length);
+    return this;
+  }
+
+  @Override
+  public ByteBuf getBytes(int index, byte[] dst, int dstIndex, int length) {
+    buffer.getBytes(index, dst, dstIndex, length);
+    return this;
+  }
+
+  @Override
+  public ByteBuf getBytes(int index, ByteBuffer dst) {
+    buffer.getBytes(index, dst);
+    return this;
+  }
+
+  @Override
+  public ByteBuf setByte(int index, int value) {
+    _setByte(index, value);
+    return this;
+  }
+
+  @Override
+  protected void _setByte(int index, int value) {
+    buffer.setByte(index, value);
+  }
+
+  @Override
+  public ByteBuf setShort(int index, int value) {
+    _setShort(index, value);
+    return this;
+  }
+
+  @Override
+  protected void _setShort(int index, int value) {
+    buffer.setShort(index, value);
+  }
+
+  @Override
+  public ByteBuf setMedium(int index, int value) {
+    _setMedium(index, value);
+    return this;
+  }
+
+  @Override
+  protected void _setMedium(int index, int value) {
+    buffer.setMedium(index, value);
+  }
+
+  @Override
+  public ByteBuf setInt(int index, int value) {
+    _setInt(index, value);
+    return this;
+  }
+
+  @Override
+  protected void _setInt(int index, int value) {
+    buffer.setInt(index, value);
+  }
+
+  @Override
+  public ByteBuf setLong(int index, long value) {
+    _setLong(index, value);
+    return this;
+  }
+
+  @Override
+  protected void _setLong(int index, long value) {
+    buffer.setLong(index, value);
+  }
+
+  @Override
+  public ByteBuf setBytes(int index, byte[] src, int srcIndex, int length) {
+    buffer.setBytes(index, src, srcIndex, length);
+    return this;
+  }
+
+  @Override
+  public ByteBuf setBytes(int index, ByteBuf src, int srcIndex, int length) {
+    buffer.setBytes(index, src, srcIndex, length);
+    return this;
+  }
+
+  @Override
+  public ByteBuf setBytes(int index, ByteBuffer src) {
+    buffer.setBytes(index, src);
+    return this;
+  }
+
+  @Override
+  public ByteBuf getBytes(int index, OutputStream out, int length)
+      throws IOException {
+    buffer.getBytes(index, out, length);
+    return this;
+  }
+
+  @Override
+  public int getBytes(int index, GatheringByteChannel out, int length)
+      throws IOException {
+    return buffer.getBytes(index, out, length);
+  }
+
+  @Override
+  public int setBytes(int index, InputStream in, int length)
+      throws IOException {
+    return buffer.setBytes(index, in, length);
+  }
+
+  @Override
+  public int setBytes(int index, ScatteringByteChannel in, int length)
+      throws IOException {
+    return buffer.setBytes(index, in, length);
+  }
+
+  @Override
+  public int nioBufferCount() {
+    return buffer.nioBufferCount();
+  }
+
+  @Override
+  public ByteBuffer[] nioBuffers(int index, int length) {
+    return buffer.nioBuffers(index, length);
+  }
+
+  @Override
+  public ByteBuffer internalNioBuffer(int index, int length) {
+    return nioBuffer(index, length);
+  }
+
+  @Override
+  public int forEachByte(int index, int length, ByteBufProcessor processor) {
+    return buffer.forEachByte(index, length, processor);
+  }
+
+  @Override
+  public int forEachByteDesc(int index, int length, ByteBufProcessor processor) {
+    return buffer.forEachByteDesc(index, length, processor);
+  }
+
+  @Override
+  public final int refCnt() {
+    return unwrap().refCnt();
+  }
+
+  @Override
+  public final ByteBuf retain() {
+    unwrap().retain();
+    return this;
+  }
+
+  @Override
+  public final ByteBuf retain(int increment) {
+    unwrap().retain(increment);
+    return this;
+  }
+
+  @Override
+  public boolean release() {
+    return release(1);
+  }
+
+  @Override
+  public boolean release(int decrement) {
+    boolean released = unwrap().release(decrement);
+    return released;
+  }
+
+}
diff --git a/exec/memory/base/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java b/exec/memory/base/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java
index 2fc1bd0057..48f8cdf0bc 100644
--- a/exec/memory/base/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java
+++ b/exec/memory/base/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java
@@ -23,193 +23,246 @@
 import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.drill.exec.exception.OutOfMemoryException;
+
 import com.codahale.metrics.Gauge;
 import com.codahale.metrics.Histogram;
 import com.codahale.metrics.Metric;
 import com.codahale.metrics.MetricFilter;
 import com.codahale.metrics.MetricRegistry;
 
-public class PooledByteBufAllocatorL extends PooledByteBufAllocator{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PooledByteBufAllocatorL.class);
-
+public class PooledByteBufAllocatorL {
   private static final org.slf4j.Logger memoryLogger = org.slf4j.LoggerFactory.getLogger("drill.allocator");
+
   private static final int MEMORY_LOGGER_FREQUENCY_SECONDS = 60;
 
+
   private static final String METRIC_PREFIX = "drill.allocator.";
+
   private final MetricRegistry registry;
   private final AtomicLong hugeBufferSize = new AtomicLong(0);
   private final AtomicLong hugeBufferCount = new AtomicLong(0);
   private final AtomicLong normalBufferSize = new AtomicLong(0);
   private final AtomicLong normalBufferCount = new AtomicLong(0);
 
-  private final PoolArena<ByteBuffer>[] directArenas;
-  private final MemoryStatusThread statusThread;
-  private final Histogram largeBuffersHist;
-  private final Histogram normalBuffersHist;
+  public final InnerAllocator allocator;
+  public final UnsafeDirectLittleEndian empty;
 
   public PooledByteBufAllocatorL(MetricRegistry registry) {
-    super(true);
     this.registry = registry;
+    allocator = new InnerAllocator();
+    empty = new UnsafeDirectLittleEndian(new DuplicatedByteBuf(Unpooled.EMPTY_BUFFER));
+  }
+
+  public UnsafeDirectLittleEndian allocate(int size) {
     try {
-      Field f = PooledByteBufAllocator.class.getDeclaredField("directArenas");
-      f.setAccessible(true);
-      this.directArenas = (PoolArena<ByteBuffer>[]) f.get(this);
-    } catch (Exception e) {
-      throw new RuntimeException("Failure while initializing allocator.  Unable to retrieve direct arenas field.", e);
+      return allocator.directBuffer(size, size);
+    } catch (OutOfMemoryError e) {
+      throw new OutOfMemoryException("Failure allocating buffer.", e);
     }
 
-    if (memoryLogger.isTraceEnabled()) {
-      statusThread = new MemoryStatusThread();
-      statusThread.start();
-    } else {
-      statusThread = null;
-    }
-    removeOldMetrics();
+  }
 
-    registry.register(METRIC_PREFIX + "normal.size", new Gauge<Long>() {
-      @Override
-      public Long getValue() {
-        return normalBufferSize.get();
-      }
-    });
+  public int getChunkSize() {
+    return allocator.chunkSize;
+  }
 
-    registry.register(METRIC_PREFIX + "normal.count", new Gauge<Long>() {
-      @Override
-      public Long getValue() {
-        return normalBufferCount.get();
-      }
-    });
+  private class InnerAllocator extends PooledByteBufAllocator {
 
-    registry.register(METRIC_PREFIX + "huge.size", new Gauge<Long>() {
-      @Override
-      public Long getValue() {
-        return hugeBufferSize.get();
-      }
-    });
 
-    registry.register(METRIC_PREFIX + "huge.count", new Gauge<Long>() {
-      @Override
-      public Long getValue() {
-        return hugeBufferCount.get();
-      }
-    });
+    private final PoolArena<ByteBuffer>[] directArenas;
+    private final MemoryStatusThread statusThread;
+    private final Histogram largeBuffersHist;
+    private final Histogram normalBuffersHist;
+    private final int chunkSize;
 
-    largeBuffersHist = registry.histogram(METRIC_PREFIX + "huge.hist");
-    normalBuffersHist = registry.histogram(METRIC_PREFIX + "normal.hist");
+    public InnerAllocator() {
+      super(true);
 
-  }
+      try {
+        Field f = PooledByteBufAllocator.class.getDeclaredField("directArenas");
+        f.setAccessible(true);
+        this.directArenas = (PoolArena<ByteBuffer>[]) f.get(this);
+      } catch (Exception e) {
+        throw new RuntimeException("Failure while initializing allocator.  Unable to retrieve direct arenas field.", e);
+      }
 
-  private synchronized void removeOldMetrics() {
-    registry.removeMatching(new MetricFilter() {
-      @Override
-      public boolean matches(String name, Metric metric) {
-        return name.startsWith("drill.allocator.");
+      this.chunkSize = directArenas[0].chunkSize;
+
+      if (memoryLogger.isTraceEnabled()) {
+        statusThread = new MemoryStatusThread();
+        statusThread.start();
+      } else {
+        statusThread = null;
       }
+      removeOldMetrics();
 
-    });
-  }
+      registry.register(METRIC_PREFIX + "normal.size", new Gauge<Long>() {
+        @Override
+        public Long getValue() {
+          return normalBufferSize.get();
+        }
+      });
 
-  @Override
-  protected ByteBuf newHeapBuffer(int initialCapacity, int maxCapacity) {
-    throw new UnsupportedOperationException("Drill doesn't support using heap buffers.");
-  }
+      registry.register(METRIC_PREFIX + "normal.count", new Gauge<Long>() {
+        @Override
+        public Long getValue() {
+          return normalBufferCount.get();
+        }
+      });
+
+      registry.register(METRIC_PREFIX + "huge.size", new Gauge<Long>() {
+        @Override
+        public Long getValue() {
+          return hugeBufferSize.get();
+        }
+      });
 
-  @Override
-  protected UnsafeDirectLittleEndian newDirectBuffer(int initialCapacity, int maxCapacity) {
-    PoolThreadCache cache = threadCache.get();
-    PoolArena<ByteBuffer> directArena = cache.directArena;
+      registry.register(METRIC_PREFIX + "huge.count", new Gauge<Long>() {
+        @Override
+        public Long getValue() {
+          return hugeBufferCount.get();
+        }
+      });
 
-    if (directArena != null) {
+      largeBuffersHist = registry.histogram(METRIC_PREFIX + "huge.hist");
+      normalBuffersHist = registry.histogram(METRIC_PREFIX + "normal.hist");
 
-      if (initialCapacity > directArena.chunkSize) {
-        // This is beyond chunk size so we'll allocate separately.
-        ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.directBuffer(initialCapacity, maxCapacity);
+    }
 
-        hugeBufferCount.incrementAndGet();
-        hugeBufferSize.addAndGet(buf.capacity());
-        largeBuffersHist.update(buf.capacity());
-        // logger.debug("Allocating huge buffer of size {}", initialCapacity, new Exception());
-        return new UnsafeDirectLittleEndian(new LargeBuffer(buf, hugeBufferSize, hugeBufferCount));
 
-      } else {
-        // within chunk, use arena.
-        ByteBuf buf = directArena.allocate(cache, initialCapacity, maxCapacity);
-        if (!(buf instanceof PooledUnsafeDirectByteBuf)) {
-          fail();
+    private synchronized void removeOldMetrics() {
+      registry.removeMatching(new MetricFilter() {
+        @Override
+        public boolean matches(String name, Metric metric) {
+          return name.startsWith("drill.allocator.");
         }
 
-        normalBuffersHist.update(buf.capacity());
-        if (ASSERT_ENABLED) {
-          normalBufferSize.addAndGet(buf.capacity());
-          normalBufferCount.incrementAndGet();
+      });
+    }
+
+    private UnsafeDirectLittleEndian newDirectBufferL(int initialCapacity, int maxCapacity) {
+      PoolThreadCache cache = threadCache.get();
+      PoolArena<ByteBuffer> directArena = cache.directArena;
+
+      if (directArena != null) {
+
+        if (initialCapacity > directArena.chunkSize) {
+          // This is beyond chunk size so we'll allocate separately.
+          ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.directBuffer(initialCapacity, maxCapacity);
+
+          hugeBufferCount.incrementAndGet();
+          hugeBufferSize.addAndGet(buf.capacity());
+          largeBuffersHist.update(buf.capacity());
+          // logger.debug("Allocating huge buffer of size {}", initialCapacity, new Exception());
+          return new UnsafeDirectLittleEndian(new LargeBuffer(buf, hugeBufferSize, hugeBufferCount));
+
+        } else {
+          // within chunk, use arena.
+          ByteBuf buf = directArena.allocate(cache, initialCapacity, maxCapacity);
+          if (!(buf instanceof PooledUnsafeDirectByteBuf)) {
+            fail();
+          }
+
+          normalBuffersHist.update(buf.capacity());
+          if (ASSERT_ENABLED) {
+            normalBufferSize.addAndGet(buf.capacity());
+            normalBufferCount.incrementAndGet();
+          }
+
+          return new UnsafeDirectLittleEndian((PooledUnsafeDirectByteBuf) buf, normalBufferCount,
+              normalBufferSize);
         }
 
-        return new UnsafeDirectLittleEndian((PooledUnsafeDirectByteBuf) buf, normalBufferCount, normalBufferSize);
+      } else {
+        throw fail();
       }
-
-    } else {
-      throw fail();
     }
-  }
-
-  private UnsupportedOperationException fail() {
-    return new UnsupportedOperationException(
-        "Drill requries that the JVM used supports access sun.misc.Unsafe.  This platform didn't provide that functionality.");
-  }
 
+    private UnsupportedOperationException fail() {
+      return new UnsupportedOperationException(
+          "Drill requries that the JVM used supports access sun.misc.Unsafe.  This platform didn't provide that functionality.");
+    }
 
-  @Override
-  public UnsafeDirectLittleEndian directBuffer(int initialCapacity, int maxCapacity) {
+    public UnsafeDirectLittleEndian directBuffer(int initialCapacity, int maxCapacity) {
       if (initialCapacity == 0 && maxCapacity == 0) {
-          newDirectBuffer(initialCapacity, maxCapacity);
+        newDirectBuffer(initialCapacity, maxCapacity);
       }
       validate(initialCapacity, maxCapacity);
-      return newDirectBuffer(initialCapacity, maxCapacity);
-  }
+      return newDirectBufferL(initialCapacity, maxCapacity);
+    }
 
-  @Override
-  public ByteBuf heapBuffer(int initialCapacity, int maxCapacity) {
-    throw new UnsupportedOperationException("Drill doesn't support using heap buffers.");
-  }
+    @Override
+    public ByteBuf heapBuffer(int initialCapacity, int maxCapacity) {
+      throw new UnsupportedOperationException("Drill doesn't support using heap buffers.");
+    }
 
 
-  private static void validate(int initialCapacity, int maxCapacity) {
-    if (initialCapacity < 0) {
+    private void validate(int initialCapacity, int maxCapacity) {
+      if (initialCapacity < 0) {
         throw new IllegalArgumentException("initialCapacity: " + initialCapacity + " (expectd: 0+)");
-    }
-    if (initialCapacity > maxCapacity) {
+      }
+      if (initialCapacity > maxCapacity) {
         throw new IllegalArgumentException(String.format(
-                "initialCapacity: %d (expected: not greater than maxCapacity(%d)",
-                initialCapacity, maxCapacity));
+            "initialCapacity: %d (expected: not greater than maxCapacity(%d)",
+            initialCapacity, maxCapacity));
+      }
     }
-  }
 
-  private class MemoryStatusThread extends Thread {
+    private class MemoryStatusThread extends Thread {
 
-    public MemoryStatusThread() {
-      super("memory-status-logger");
-      this.setDaemon(true);
-      this.setName("allocation.logger");
-    }
+      public MemoryStatusThread() {
+        super("memory-status-logger");
+        this.setDaemon(true);
+        this.setName("allocation.logger");
+      }
 
-    @Override
-    public void run() {
-      while (true) {
-        memoryLogger.trace("Memory Usage: \n{}", PooledByteBufAllocatorL.this.toString());
-        try {
-          Thread.sleep(MEMORY_LOGGER_FREQUENCY_SECONDS * 1000);
-        } catch (InterruptedException e) {
-          return;
-        }
+      @Override
+      public void run() {
+        while (true) {
+          memoryLogger.trace("Memory Usage: \n{}", PooledByteBufAllocatorL.this.toString());
+          try {
+            Thread.sleep(MEMORY_LOGGER_FREQUENCY_SECONDS * 1000);
+          } catch (InterruptedException e) {
+            return;
+          }
 
+        }
       }
+
     }
 
-  }
+    public void checkAndReset() {
+      if (hugeBufferCount.get() != 0 || normalBufferCount.get() != 0) {
+        StringBuilder buf = new StringBuilder();
+        buf.append("Large buffers outstanding: ");
+        buf.append(hugeBufferCount.get());
+        buf.append(" totaling ");
+        buf.append(hugeBufferSize.get());
+        buf.append(" bytes.");
+        buf.append('\n');
+        buf.append("Normal buffers outstanding: ");
+        buf.append(normalBufferCount.get());
+        buf.append(" totaling ");
+        buf.append(normalBufferSize.get());
+        buf.append(" bytes.");
+        hugeBufferCount.set(0);
+        normalBufferCount.set(0);
+        hugeBufferSize.set(0);
+        normalBufferSize.set(0);
+        throw new IllegalStateException(buf.toString());
+      }
+    }
 
-  public void checkAndReset() {
-    if (hugeBufferCount.get() != 0 || normalBufferCount.get() != 0) {
+    public String toString() {
       StringBuilder buf = new StringBuilder();
+      buf.append(directArenas.length);
+      buf.append(" direct arena(s):");
+      buf.append(StringUtil.NEWLINE);
+      for (PoolArena<ByteBuffer> a : directArenas) {
+        buf.append(a);
+      }
+
       buf.append("Large buffers outstanding: ");
       buf.append(hugeBufferCount.get());
       buf.append(" totaling ");
@@ -221,35 +274,10 @@ public void checkAndReset() {
       buf.append(" totaling ");
       buf.append(normalBufferSize.get());
       buf.append(" bytes.");
-      hugeBufferCount.set(0);
-      normalBufferCount.set(0);
-      hugeBufferSize.set(0);
-      normalBufferSize.set(0);
-      throw new IllegalStateException(buf.toString());
+      return buf.toString();
     }
-  }
 
-  public String toString() {
-    StringBuilder buf = new StringBuilder();
-    buf.append(directArenas.length);
-    buf.append(" direct arena(s):");
-    buf.append(StringUtil.NEWLINE);
-    for (PoolArena<ByteBuffer> a : directArenas) {
-      buf.append(a);
-    }
 
-    buf.append("Large buffers outstanding: ");
-    buf.append(this.hugeBufferCount.get());
-    buf.append(" totaling ");
-    buf.append(this.hugeBufferSize.get());
-    buf.append(" bytes.");
-    buf.append('\n');
-    buf.append("Normal buffers outstanding: ");
-    buf.append(this.normalBufferCount.get());
-    buf.append(" totaling ");
-    buf.append(this.normalBufferSize.get());
-    buf.append(" bytes.");
-    return buf.toString();
   }
 
   public static final boolean ASSERT_ENABLED;
@@ -259,4 +287,5 @@ public String toString() {
     assert isAssertEnabled = true;
     ASSERT_ENABLED = isAssertEnabled;
   }
+
 }
diff --git a/exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java b/exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java
index 559f06ddd0..e2b2bd33a4 100644
--- a/exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java
+++ b/exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java
@@ -21,21 +21,63 @@
 import io.netty.util.internal.PlatformDependent;
 
 import java.nio.ByteOrder;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.drill.common.StackTrace;
+import org.slf4j.Logger;
+
 public final class UnsafeDirectLittleEndian extends WrappedByteBuf {
   private static final boolean NATIVE_ORDER = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN;
   private final AbstractByteBuf wrapped;
   private final long memoryAddress;
+  private static final boolean TRACK_BUFFERS = false;
   private AtomicLong bufferCount;
   private AtomicLong bufferSize;
   private long initCap = -1;
 
+  private final static Map<UnsafeDirectLittleEndian, StackTrace> bufferMap = Collections
+      .synchronizedMap(new IdentityHashMap<UnsafeDirectLittleEndian, StackTrace>());
+
+  public static int getBufferCount() {
+    return bufferMap.size();
+  }
+
+  public static void releaseBuffers() {
+    synchronized(bufferMap) {
+      final Set<UnsafeDirectLittleEndian> bufferSet = bufferMap.keySet();
+      final LinkedList<UnsafeDirectLittleEndian> bufferList = new LinkedList<>(bufferSet);
+      while(!bufferList.isEmpty()) {
+        final UnsafeDirectLittleEndian udle = bufferList.removeFirst();
+        udle.release(udle.refCnt());
+      }
+    }
+  }
+
+  public static void logBuffers(final Logger logger) {
+    int count = 0;
+    final Set<UnsafeDirectLittleEndian> bufferSet = bufferMap.keySet();
+    for (final UnsafeDirectLittleEndian udle : bufferSet) {
+      final StackTrace stackTrace = bufferMap.get(udle);
+      ++count;
+      logger.debug("#" + count + " active buffer allocated at\n" + stackTrace);
+    }
+  }
+
+  UnsafeDirectLittleEndian(DuplicatedByteBuf buf) {
+    this(buf, true);
+  }
+
   UnsafeDirectLittleEndian(LargeBuffer buf) {
     this(buf, true);
   }
 
-  UnsafeDirectLittleEndian(PooledUnsafeDirectByteBuf buf, AtomicLong bufferCount, AtomicLong bufferSize) {
+  UnsafeDirectLittleEndian(PooledUnsafeDirectByteBuf buf, AtomicLong bufferCount,
+      AtomicLong bufferSize) {
     this(buf, true);
     this.bufferCount = bufferCount;
     this.bufferSize = bufferSize;
@@ -50,7 +92,7 @@ private UnsafeDirectLittleEndian(AbstractByteBuf buf, boolean fake) {
       throw new IllegalStateException("Drill only runs on LittleEndian systems.");
     }
     wrapped = buf;
-    this.memoryAddress = buf.memoryAddress();
+    memoryAddress = buf.memoryAddress();
   }
     private long addr(int index) {
         return memoryAddress + index;
@@ -89,156 +131,139 @@ public ByteBuf order(ByteOrder endianness) {
   }
 
   @Override
-    public double getDouble(int index) {
-        return Double.longBitsToDouble(getLong(index));
-    }
-
-    @Override
-    public char getChar(int index) {
-        return (char) getShort(index);
-    }
-
-    @Override
-    public long getUnsignedInt(int index) {
-        return getInt(index) & 0xFFFFFFFFL;
-    }
+  public double getDouble(int index) {
+    return Double.longBitsToDouble(getLong(index));
+  }
 
-    @Override
-    public int getInt(int index) {
-//        wrapped.checkIndex(index, 4);
-        int v = PlatformDependent.getInt(addr(index));
-        return v;
-    }
+  @Override
+  public char getChar(int index) {
+    return (char) getShort(index);
+  }
 
-    @Override
-    public int getUnsignedShort(int index) {
-        return getShort(index) & 0xFFFF;
-    }
+  @Override
+  public long getUnsignedInt(int index) {
+    return getInt(index) & 0xFFFFFFFFL;
+  }
 
-    @Override
-    public short getShort(int index) {
-//        wrapped.checkIndex(index, 2);
-        short v = PlatformDependent.getShort(addr(index));
-        return v;
-    }
+  @Override
+  public int getInt(int index) {
+    int v = PlatformDependent.getInt(addr(index));
+    return v;
+  }
 
-    @Override
-    public ByteBuf setShort(int index, int value) {
-        wrapped.checkIndex(index, 2);
-        _setShort(index, value);
-        return this;
-    }
+  @Override
+  public int getUnsignedShort(int index) {
+    return getShort(index) & 0xFFFF;
+  }
 
-    @Override
-    public ByteBuf setInt(int index, int value) {
-        wrapped.checkIndex(index, 4);
-        _setInt(index, value);
-        return this;
-    }
+  @Override
+  public short getShort(int index) {
+    short v = PlatformDependent.getShort(addr(index));
+    return v;
+  }
 
-    @Override
-    public ByteBuf setLong(int index, long value) {
-        wrapped.checkIndex(index, 8);
-        _setLong(index, value);
-        return this;
-    }
+  @Override
+  public ByteBuf setShort(int index, int value) {
+    wrapped.checkIndex(index, 2);
+    _setShort(index, value);
+    return this;
+  }
 
-    @Override
-    public ByteBuf setChar(int index, int value) {
-        setShort(index, value);
-        return this;
-    }
+  @Override
+  public ByteBuf setInt(int index, int value) {
+    wrapped.checkIndex(index, 4);
+    _setInt(index, value);
+    return this;
+  }
 
-    @Override
-    public ByteBuf setFloat(int index, float value) {
-        setInt(index, Float.floatToRawIntBits(value));
-        return this;
-    }
+  @Override
+  public ByteBuf setLong(int index, long value) {
+    wrapped.checkIndex(index, 8);
+    _setLong(index, value);
+    return this;
+  }
 
-    @Override
-    public ByteBuf setDouble(int index, double value) {
-        setLong(index, Double.doubleToRawLongBits(value));
-        return this;
-    }
+  @Override
+  public ByteBuf setChar(int index, int value) {
+    setShort(index, value);
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeShort(int value) {
-        wrapped.ensureWritable(2);
-        _setShort(wrapped.writerIndex, value);
-        wrapped.writerIndex += 2;
-        return this;
-    }
+  @Override
+  public ByteBuf setFloat(int index, float value) {
+    setInt(index, Float.floatToRawIntBits(value));
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeInt(int value) {
-        wrapped.ensureWritable(4);
-        _setInt(wrapped.writerIndex, value);
-        wrapped.writerIndex += 4;
-        return this;
-    }
+  @Override
+  public ByteBuf setDouble(int index, double value) {
+    setLong(index, Double.doubleToRawLongBits(value));
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeLong(long value) {
-        wrapped.ensureWritable(8);
-        _setLong(wrapped.writerIndex, value);
-        wrapped.writerIndex += 8;
-        return this;
-    }
+  @Override
+  public ByteBuf writeShort(int value) {
+    wrapped.ensureWritable(2);
+    _setShort(wrapped.writerIndex, value);
+    wrapped.writerIndex += 2;
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeChar(int value) {
-        writeShort(value);
-        return this;
-    }
+  @Override
+  public ByteBuf writeInt(int value) {
+    wrapped.ensureWritable(4);
+    _setInt(wrapped.writerIndex, value);
+    wrapped.writerIndex += 4;
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeFloat(float value) {
-        writeInt(Float.floatToRawIntBits(value));
-        return this;
-    }
+  @Override
+  public ByteBuf writeLong(long value) {
+    wrapped.ensureWritable(8);
+    _setLong(wrapped.writerIndex, value);
+    wrapped.writerIndex += 8;
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeDouble(double value) {
-        writeLong(Double.doubleToRawLongBits(value));
-        return this;
-    }
+  @Override
+  public ByteBuf writeChar(int value) {
+    writeShort(value);
+    return this;
+  }
 
-    private void _setShort(int index, int value) {
-        PlatformDependent.putShort(addr(index), (short) value);
-    }
+  @Override
+  public ByteBuf writeFloat(float value) {
+    writeInt(Float.floatToRawIntBits(value));
+    return this;
+  }
 
-    private void _setInt(int index, int value) {
-        PlatformDependent.putInt(addr(index), value);
-    }
+  @Override
+  public ByteBuf writeDouble(double value) {
+    writeLong(Double.doubleToRawLongBits(value));
+    return this;
+  }
 
-    private void _setLong(int index, long value) {
-        PlatformDependent.putLong(addr(index), value);
-    }
+  private void _setShort(int index, int value) {
+    PlatformDependent.putShort(addr(index), (short) value);
+  }
 
-    @Override
-    public byte getByte(int index) {
-      return PlatformDependent.getByte(addr(index));
-    }
+  private void _setInt(int index, int value) {
+    PlatformDependent.putInt(addr(index), value);
+  }
 
-    @Override
-    public ByteBuf setByte(int index, int value) {
-      PlatformDependent.putByte(addr(index), (byte) value);
-      return this;
-    }
+  private void _setLong(int index, long value) {
+    PlatformDependent.putLong(addr(index), value);
+  }
 
   @Override
-  public boolean release() {
-    return release(1);
+  public byte getByte(int index) {
+    return PlatformDependent.getByte(addr(index));
   }
 
   @Override
-  public boolean release(int decrement) {
-    boolean released = super.release(decrement);
-    if (released && initCap != -1) {
-      bufferCount.decrementAndGet();
-      bufferSize.addAndGet(-initCap);
-    }
-    return released;
+  public ByteBuf setByte(int index, int value) {
+    PlatformDependent.putByte(addr(index), (byte) value);
+    return this;
   }
 
   public static final boolean ASSERT_ENABLED;
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountant.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountant.java
new file mode 100644
index 0000000000..dd87e8cf1c
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountant.java
@@ -0,0 +1,272 @@
+/**
+ * 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.drill.exec.memory;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import org.apache.drill.exec.exception.OutOfMemoryException;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Provides a concurrent way to manage account for memory usage without locking. Used as basis for Allocators. All
+ * operations are threadsafe (except for close).
+ */
+@ThreadSafe
+class Accountant implements AutoCloseable {
+  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Accountant.class);
+
+  /**
+   * The parent allocator
+   */
+  protected final Accountant parent;
+
+  /**
+   * The amount of memory reserved for this allocator. Releases below this amount of memory will not be returned to the
+   * parent Accountant until this Accountant is closed.
+   */
+  protected final long reservation;
+
+  private final AtomicLong peakAllocation = new AtomicLong();
+
+  /**
+   * Maximum local memory that can be held. This can be externally updated. Changing it won't cause past memory to
+   * change but will change responses to future allocation efforts
+   */
+  private final AtomicLong allocationLimit = new AtomicLong();
+
+  /**
+   * Currently allocated amount of memory;
+   */
+  private final AtomicLong locallyHeldMemory = new AtomicLong();
+
+  public Accountant(Accountant parent, long reservation, long maxAllocation) {
+    Preconditions.checkArgument(reservation >= 0, "The initial reservation size must be non-negative.");
+    Preconditions.checkArgument(maxAllocation >= 0, "The maximum allocation limit must be non-negative.");
+    Preconditions.checkArgument(reservation <= maxAllocation,
+        "The initial reservation size must be <= the maximum allocation.");
+    Preconditions.checkArgument(reservation == 0 || parent != null, "The root accountant can't reserve memory.");
+
+    this.parent = parent;
+    this.reservation = reservation;
+    this.allocationLimit.set(maxAllocation);
+
+    if (reservation != 0) {
+      // we will allocate a reservation from our parent.
+      final AllocationOutcome outcome = parent.allocateBytes(reservation);
+      if (!outcome.isOk()) {
+        throw new OutOfMemoryException("Failure trying to allocate initial reservation for Allocator.");
+      }
+    }
+  }
+
+  /**
+   * Attempt to allocate the requested amount of memory. Either completely succeeds or completely fails. Constructs a a
+   * log of delta
+   *
+   * If it fails, no changes are made to accounting.
+   *
+   * @param size
+   *          The amount of memory to reserve in bytes.
+   * @return True if the allocation was successful, false if the allocation failed.
+   */
+  AllocationOutcome allocateBytes(long size) {
+    final AllocationOutcome outcome = allocate(size, true, false);
+    if (!outcome.isOk()) {
+      releaseBytes(size);
+    }
+    return outcome;
+  }
+
+  private void updatePeak() {
+    final long currentMemory = locallyHeldMemory.get();
+    while (true) {
+      final long previousPeak = peakAllocation.get();
+      if (currentMemory > previousPeak) {
+        if (peakAllocation.compareAndSet(previousPeak, currentMemory)) {
+          // if we're able to update peak, finish.
+          return;
+        } else {
+          // peak allocation changed underneath us. try again.
+          continue;
+        }
+      } else {
+        return;
+      }
+    }
+  }
+
+  /**
+   * Increase the accounting. Returns whether the allocation fit within limits.
+   *
+   * @param size
+   *          to increase
+   * @return Whether the allocation fit within limits.
+   */
+  boolean forceAllocate(long size) {
+    final AllocationOutcome outcome = allocate(size, true, true);
+    return outcome.isOk();
+  }
+
+  /**
+   * Internal method for allocation. This takes a forced approach to allocation to ensure that we manage reservation
+   * boundary issues consistently. Allocation is always done through the entire tree. The two options that we influence
+   * are whether the allocation should be forced and whether or not the peak memory allocation should be updated. If at
+   * some point during allocation escalation we determine that the allocation is no longer possible, we will continue to
+   * do a complete and consistent allocation but we will stop updating the peak allocation. We do this because we know
+   * that we will be directly unwinding this allocation (and thus never actually making the allocation). If force
+   * allocation is passed, then we continue to update the peak limits since we now know that this allocation will occur
+   * despite our moving past one or more limits.
+   *
+   * @param size
+   *          The size of the allocation.
+   * @param incomingUpdatePeak
+   *          Whether we should update the local peak for this allocation.
+   * @param forceAllocation
+   *          Whether we should force the allocation.
+   * @return The outcome of the allocation.
+   */
+  private AllocationOutcome allocate(final long size, final boolean incomingUpdatePeak, final boolean forceAllocation) {
+    final long newLocal = locallyHeldMemory.addAndGet(size);
+    final long beyondReservation = newLocal - reservation;
+    final boolean beyondLimit = newLocal > allocationLimit.get();
+    final boolean updatePeak = forceAllocation || (incomingUpdatePeak && !beyondLimit);
+
+    AllocationOutcome parentOutcome = AllocationOutcome.SUCCESS;
+    if (beyondReservation > 0 && parent != null) {
+      // we need to get memory from our parent.
+      final long parentRequest = Math.min(beyondReservation, size);
+      parentOutcome = parent.allocate(parentRequest, updatePeak, forceAllocation);
+    }
+
+    final AllocationOutcome finalOutcome = beyondLimit ? AllocationOutcome.FAILED_LOCAL :
+        parentOutcome.ok ? AllocationOutcome.SUCCESS : AllocationOutcome.FAILED_PARENT;
+
+    if (updatePeak) {
+      updatePeak();
+    }
+
+    return finalOutcome;
+  }
+
+  public void releaseBytes(long size) {
+    // reduce local memory. all memory released above reservation should be released up the tree.
+    final long newSize = locallyHeldMemory.addAndGet(-size);
+
+    Preconditions.checkArgument(newSize >= 0, "Accounted size went negative.");
+
+    final long originalSize = newSize + size;
+    if(originalSize > reservation && parent != null){
+      // we deallocated memory that we should release to our parent.
+      final long possibleAmountToReleaseToParent = originalSize - reservation;
+      final long actualToReleaseToParent = Math.min(size, possibleAmountToReleaseToParent);
+      parent.releaseBytes(actualToReleaseToParent);
+    }
+
+  }
+
+  /**
+   * Set the maximum amount of memory that can be allocated in the this Accountant before failing an allocation.
+   *
+   * @param newLimit
+   *          The limit in bytes.
+   */
+  public void setLimit(long newLimit) {
+    allocationLimit.set(newLimit);
+  }
+
+  public boolean isOverLimit() {
+    return getAllocatedMemory() > getLimit() || (parent != null && parent.isOverLimit());
+  }
+
+  /**
+   * Close this Accountant. This will release any reservation bytes back to a parent Accountant.
+   */
+  public void close() {
+    // return memory reservation to parent allocator.
+    if (parent != null) {
+      parent.releaseBytes(reservation);
+    }
+  }
+
+  /**
+   * Return the current limit of this Accountant.
+   *
+   * @return Limit in bytes.
+   */
+  public long getLimit() {
+    return allocationLimit.get();
+  }
+
+  /**
+   * Return the current amount of allocated memory that this Accountant is managing accounting for. Note this does not
+   * include reservation memory that hasn't been allocated.
+   *
+   * @return Currently allocate memory in bytes.
+   */
+  public long getAllocatedMemory() {
+    return locallyHeldMemory.get();
+  }
+
+  /**
+   * The peak memory allocated by this Accountant.
+   *
+   * @return The peak allocated memory in bytes.
+   */
+  public long getPeakMemoryAllocation() {
+    return peakAllocation.get();
+  }
+
+  /**
+   * Describes the type of outcome that occurred when trying to account for allocation of memory.
+   */
+  public static enum AllocationOutcome {
+
+    /**
+     * Allocation succeeded.
+     */
+    SUCCESS(true),
+
+    /**
+     * Allocation succeeded but only because the allocator was forced to move beyond a limit.
+     */
+    FORCED_SUCESS(true),
+
+    /**
+     * Allocation failed because the local allocator's limits were exceeded.
+     */
+    FAILED_LOCAL(false),
+
+    /**
+     * Allocation failed because a parent allocator's limits were exceeded.
+     */
+    FAILED_PARENT(false);
+
+    private final boolean ok;
+
+    AllocationOutcome(boolean ok) {
+      this.ok = ok;
+    }
+
+    public boolean isOk() {
+      return ok;
+    }
+  }
+}
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java
new file mode 100644
index 0000000000..58664382a1
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java
@@ -0,0 +1,155 @@
+/**
+ * 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.drill.exec.memory;
+
+import io.netty.buffer.DrillBuf;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Supports cumulative allocation reservation. Clients may increase the size of the reservation repeatedly until they
+ * call for an allocation of the current total size. The reservation can only be used once, and will throw an exception
+ * if it is used more than once.
+ * <p>
+ * For the purposes of airtight memory accounting, the reservation must be close()d whether it is used or not.
+ * This is not threadsafe.
+ */
+public abstract class AllocationReservation implements AutoCloseable {
+  private int nBytes = 0;
+  private boolean used = false;
+  private boolean closed = false;
+
+  /**
+   * Constructor. Prevent construction except by derived classes.
+   * <p>The expectation is that the derived class will be a non-static inner
+   * class in an allocator.
+   */
+  AllocationReservation() {
+  }
+
+  /**
+   * Add to the current reservation.
+   *
+   * <p>Adding may fail if the allocator is not allowed to consume any more space.
+   *
+   * @param nBytes the number of bytes to add
+   * @return true if the addition is possible, false otherwise
+   * @throws IllegalStateException if called after buffer() is used to allocate the reservation
+   */
+  public boolean add(final int nBytes) {
+    Preconditions.checkArgument(nBytes >= 0, "nBytes(%d) < 0", nBytes);
+    Preconditions.checkState(!closed, "Attempt to increase reservation after reservation has been closed");
+    Preconditions.checkState(!used, "Attempt to increase reservation after reservation has been used");
+
+    // we round up to next power of two since all reservations are done in powers of two. This may overestimate the
+    // preallocation since someone may perceive additions to be power of two. If this becomes a problem, we can look at
+    // modifying this behavior so that we maintain what we reserve and what the user asked for and make sure to only
+    // round to power of two as necessary.
+    final int nBytesTwo = BaseAllocator.nextPowerOfTwo(nBytes);
+    if (!reserve(nBytesTwo)) {
+      return false;
+    }
+
+    this.nBytes += nBytesTwo;
+    return true;
+  }
+
+  /**
+   * Requests a reservation of additional space.
+   *
+   * <p>The implementation of the allocator's inner class provides this.
+   *
+   * @param nBytes the amount to reserve
+   * @return true if the reservation can be satisfied, false otherwise
+   */
+  abstract boolean reserve(int nBytes);
+
+  /**
+   * Allocate a buffer whose size is the total of all the add()s made.
+   *
+   * <p>The allocation request can still fail, even if the amount of space
+   * requested is available, if the allocation cannot be made contiguously.
+   *
+   * @return the buffer, or null, if the request cannot be satisfied
+   * @throws IllegalStateException if called called more than once
+   */
+  public DrillBuf buffer() {
+    Preconditions.checkState(!closed, "Attempt to allocate after closed");
+    Preconditions.checkState(!used, "Attempt to allocate more than once");
+
+    final DrillBuf drillBuf = allocate(nBytes);
+    used = true;
+    return drillBuf;
+  }
+
+  /**
+   * Allocate the a buffer of the requested size.
+   *
+   * <p>The implementation of the allocator's inner class provides this.
+   *
+   * @param nBytes the size of the buffer requested
+   * @return the buffer, or null, if the request cannot be satisfied
+   */
+  abstract DrillBuf allocate(int nBytes);
+
+  @Override
+  public void close() {
+    if (closed) {
+      return;
+    }
+    if (!used) {
+      releaseReservation(nBytes);
+    }
+
+    closed = true;
+  }
+
+  /**
+   * Return the reservation back to the allocator without having used it.
+   *
+   * @param nBytes the size of the reservation
+   */
+  abstract void releaseReservation(int nBytes);
+
+  /**
+   * Get the current size of the reservation (the sum of all the add()s).
+   *
+   * @return size of the current reservation
+   */
+  public int getSize() {
+    return nBytes;
+  }
+
+  /**
+   * Return whether or not the reservation has been used.
+   *
+   * @return whether or not the reservation has been used
+   */
+  public boolean isUsed() {
+    return used;
+  }
+
+  /**
+   * Return whether or not the reservation has been closed.
+   *
+   * @return whether or not the reservation has been closed
+   */
+  public boolean isClosed() {
+    return closed;
+  }
+}
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java
new file mode 100644
index 0000000000..8bf2a995f5
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java
@@ -0,0 +1,31 @@
+/**
+ * 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.drill.exec.memory;
+
+/**
+ * Exception thrown when a closed BufferAllocator is used. Note
+ * this is an unchecked exception.
+ *
+ * @param message string associated with the cause
+ */
+@SuppressWarnings("serial")
+public class AllocatorClosedException extends RuntimeException {
+  public AllocatorClosedException(String message) {
+    super(message);
+  }
+}
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorDecorator.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorDecorator.java
new file mode 100644
index 0000000000..c93149099e
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorDecorator.java
@@ -0,0 +1,128 @@
+/**
+ * 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.drill.exec.memory;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.DrillBuf;
+
+import org.apache.drill.exec.ops.BufferManager;
+
+/**
+ * Decorator class that allows someone to wrap an Allocator implementation to enhance capabilities.
+ */
+public abstract class AllocatorDecorator implements BufferAllocator {
+
+  private final BufferAllocator allocator;
+
+  /**
+   * Constructor to create a new Decorator.
+   *
+   * @param allocator
+   *          The allocator that all calls will be passed to when using the default implementation.
+   */
+  public AllocatorDecorator(final BufferAllocator allocator) {
+    super();
+    this.allocator = allocator;
+  }
+
+  @Override
+  public DrillBuf buffer(int size) {
+    return allocator.buffer(size);
+  }
+
+  @Override
+  public DrillBuf buffer(int size, BufferManager manager) {
+    return allocator.buffer(size, manager);
+  }
+
+  @Override
+  public ByteBufAllocator getAsByteBufAllocator() {
+    return allocator.getAsByteBufAllocator();
+  }
+
+  @Override
+  public BufferAllocator newChildAllocator(String name, long initReservation, long maxAllocation) {
+    return allocator.newChildAllocator(name, initReservation, maxAllocation);
+  }
+
+  @Override
+  public void close() {
+    allocator.close();
+  }
+
+  @Override
+  public long getAllocatedMemory() {
+    return allocator.getAllocatedMemory();
+  }
+
+  @Override
+  public void setLimit(long newLimit) {
+    allocator.setLimit(newLimit);
+  }
+
+  @Override
+  public long getLimit() {
+    return allocator.getLimit();
+  }
+
+  @Override
+  public long getPeakMemoryAllocation() {
+    return allocator.getPeakMemoryAllocation();
+  }
+
+  @Override
+  public AllocationReservation newReservation() {
+    return allocator.newReservation();
+  }
+
+  @Override
+  public DrillBuf getEmpty() {
+    return allocator.getEmpty();
+  }
+
+  @Override
+  public String getName() {
+    return allocator.getName();
+  }
+
+  @Override
+  public boolean isOverLimit() {
+    return allocator.isOverLimit();
+  }
+
+  @Override
+  public String toVerboseString() {
+    return allocator.toVerboseString();
+  }
+
+  @Override
+  public boolean isClosed() {
+    return allocator.isClosed();
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <T> T unwrap(Class<T> c) {
+    if (c == AllocatorDecorator.class) {
+      return (T) this;
+    } else {
+      return allocator.unwrap(c);
+    }
+  }
+
+}
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorManager.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorManager.java
new file mode 100644
index 0000000000..72dbc1fde2
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorManager.java
@@ -0,0 +1,386 @@
+/**
+ * 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.drill.exec.memory;
+
+import static org.apache.drill.exec.memory.BaseAllocator.indent;
+import io.netty.buffer.DrillBuf;
+import io.netty.buffer.PooledByteBufAllocatorL;
+import io.netty.buffer.UnsafeDirectLittleEndian;
+
+import java.util.IdentityHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.drill.common.HistoricalLog;
+import org.apache.drill.exec.memory.BaseAllocator.Verbosity;
+import org.apache.drill.exec.metrics.DrillMetrics;
+import org.apache.drill.exec.ops.BufferManager;
+
+import com.carrotsearch.hppc.LongObjectOpenHashMap;
+import com.google.common.base.Preconditions;
+
+/**
+ * Manages the relationship between one or more allocators and a particular UDLE. Ensures that one allocator owns the
+ * memory that multiple allocators may be referencing. Manages a BufferLedger between each of its associated allocators.
+ * This class is also responsible for managing when memory is allocated and returned to the Netty-based
+ * PooledByteBufAllocatorL.
+ *
+ * The only reason that this isn't package private is we're forced to put DrillBuf in Netty's package which need access
+ * to these objects or methods.
+ *
+ * Threading: AllocatorManager manages thread-safety internally. Operations within the context of a single BufferLedger
+ * are lockless in nature and can be leveraged by multiple threads. Operations that cross the context of two ledgers
+ * will acquire a lock on the AllocatorManager instance. Important note, there is one AllocatorManager per
+ * UnsafeDirectLittleEndian buffer allocation. As such, there will be thousands of these in a typical query. The
+ * contention of acquiring a lock on AllocatorManager should be very low.
+ *
+ */
+public class AllocatorManager {
+  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AllocatorManager.class);
+
+  private static final AtomicLong LEDGER_ID_GENERATOR = new AtomicLong(0);
+  static final PooledByteBufAllocatorL INNER_ALLOCATOR = new PooledByteBufAllocatorL(DrillMetrics.getInstance());
+
+  private final RootAllocator root;
+  private volatile BufferLedger owningLedger;
+  private final int size;
+  private final UnsafeDirectLittleEndian underlying;
+  private final ReadWriteLock lock = new ReentrantReadWriteLock();
+  private final LongObjectOpenHashMap<BufferLedger> map = new LongObjectOpenHashMap<>();
+  private final AutoCloseableLock readLock = new AutoCloseableLock(lock.readLock());
+  private final AutoCloseableLock writeLock = new AutoCloseableLock(lock.writeLock());
+  private final IdentityHashMap<DrillBuf, Object> buffers =
+      BaseAllocator.DEBUG ? new IdentityHashMap<DrillBuf, Object>() : null;
+
+  AllocatorManager(BaseAllocator accountingAllocator, int size) {
+    Preconditions.checkNotNull(accountingAllocator);
+    this.root = accountingAllocator.root;
+    this.underlying = INNER_ALLOCATOR.allocate(size);
+    this.owningLedger = associate(accountingAllocator);
+    this.size = underlying.capacity();
+  }
+
+  /**
+   * Associate the existing underlying buffer with a new allocator.
+   *
+   * @param allocator
+   *          The target allocator to associate this buffer with.
+   * @return The Ledger (new or existing) that associates the underlying buffer to this new ledger.
+   */
+  public BufferLedger associate(BaseAllocator allocator) {
+    if (root != allocator.root) {
+      throw new IllegalStateException(
+          "A buffer can only be associated between two allocators that share the same root.");
+    }
+
+    final long allocatorId = allocator.getId();
+    try (AutoCloseableLock read = readLock.open()) {
+
+      final BufferLedger ledger = map.get(allocatorId);
+      if (ledger != null) {
+        return ledger;
+      }
+
+    }
+    try (AutoCloseableLock write = writeLock.open()) {
+      final BufferLedger ledger = new BufferLedger(allocator, new ReleaseListener(allocatorId));
+      map.put(allocatorId, ledger);
+      allocator.associateLedger(ledger);
+      return ledger;
+    }
+  }
+
+
+  /**
+   * The way that a particular BufferLedger communicates back to the AllocatorManager that it now longer needs to hold a
+   * reference to particular piece of memory.
+   */
+  private class ReleaseListener {
+
+    private final long allocatorId;
+
+    public ReleaseListener(long allocatorId) {
+      this.allocatorId = allocatorId;
+    }
+
+    public void release() {
+      try (AutoCloseableLock write = writeLock.open()) {
+        final BufferLedger oldLedger = map.remove(allocatorId);
+        oldLedger.allocator.dissociateLedger(oldLedger);
+
+        if (oldLedger == owningLedger) {
+          if (map.isEmpty()) {
+            // no one else owns, lets release.
+            oldLedger.allocator.releaseBytes(size);
+            underlying.release();
+          } else {
+            // we need to change the owning allocator. we've been removed so we'll get whatever is top of list
+            BufferLedger newLedger = map.iterator().next().value;
+
+            // we'll forcefully transfer the ownership and not worry about whether we exceeded the limit
+            // since this consumer can do anything with this.
+            oldLedger.transferBalance(newLedger);
+            owningLedger = newLedger;
+          }
+        }
+
+
+      }
+    }
+  }
+
+  /**
+   * Simple wrapper class that allows Locks to be released via an try-with-resources block.
+   */
+  private class AutoCloseableLock implements AutoCloseable {
+
+    private final Lock lock;
+
+    public AutoCloseableLock(Lock lock) {
+      this.lock = lock;
+    }
+
+    public AutoCloseableLock open() {
+      lock.lock();
+      return this;
+    }
+
+    @Override
+    public void close() {
+      lock.unlock();
+    }
+
+  }
+
+  /**
+   * The reference manager that binds an allocator manager to a particular BaseAllocator. Also responsible for creating
+   * a set of DrillBufs that share a common fate and set of reference counts.
+   *
+   * As with AllocatorManager, the only reason this is public is due to DrillBuf being in io.netty.buffer package.
+   */
+  public class BufferLedger {
+    private final long id = LEDGER_ID_GENERATOR.incrementAndGet(); // unique ID assigned to each ledger
+    private final AtomicInteger bufRefCnt = new AtomicInteger(0); // start at zero so we can manage request for retain
+                                                                  // correctly
+    private final BaseAllocator allocator;
+    private final ReleaseListener listener;
+    private final HistoricalLog historicalLog = BaseAllocator.DEBUG ? new HistoricalLog(BaseAllocator.DEBUG_LOG_LENGTH,
+        "BufferLedger[%d]", 1)
+        : null;
+
+    private BufferLedger(BaseAllocator allocator, ReleaseListener listener) {
+      this.allocator = allocator;
+      this.listener = listener;
+    }
+
+    /**
+     * Transfer any balance the current ledger has to the target ledger. In the case that the current ledger holds no
+     * memory, no transfer is made to the new ledger.
+     *
+     * @param target
+     *          The ledger to transfer ownership account to.
+     * @return Whether transfer fit within target ledgers limits.
+     */
+    public boolean transferBalance(BufferLedger target) {
+      Preconditions.checkNotNull(target);
+      Preconditions.checkArgument(allocator.root == target.allocator.root,
+          "You can only transfer between two allocators that share the same root.");
+
+      // since two balance transfers out from the allocator manager could cause incorrect accounting, we need to ensure
+      // that this won't happen by synchronizing on the allocator manager instance.
+      synchronized (AllocatorManager.this) {
+        if (this != owningLedger || target == this) {
+          return true;
+        }
+
+        if (BaseAllocator.DEBUG) {
+          this.historicalLog.recordEvent("transferBalance(%s)", target.allocator.name);
+          target.historicalLog.recordEvent("incoming(from %s)", owningLedger.allocator.name);
+        }
+
+        boolean overlimit = target.allocator.forceAllocate(size);
+        allocator.releaseBytes(size);
+        owningLedger = target;
+        return overlimit;
+      }
+
+    }
+
+    /**
+     * Print the current ledger state to a the provided StringBuilder.
+     *
+     * @param sb
+     *          The StringBuilder to populate.
+     * @param indent
+     *          The level of indentation to position the data.
+     * @param verbosity
+     *          The level of verbosity to print.
+     */
+    public void print(StringBuilder sb, int indent, Verbosity verbosity) {
+      indent(sb, indent)
+          .append("ledger (allocator: ")
+          .append(allocator.name)
+          .append("), isOwning: ")
+          .append(owningLedger == this)
+          .append(", size: ")
+          .append(size)
+          .append(", references: ")
+          .append(bufRefCnt.get())
+          .append('\n');
+
+      if (BaseAllocator.DEBUG) {
+        // This doesn't seem as useful as the individual buffer logs below. Removing from default presentation.
+        // if (verbosity.includeHistoricalLog) {
+        // historicalLog.buildHistory(sb, indent + 2, verbosity.includeStackTraces);
+        // }
+        synchronized (buffers) {
+          indent(sb, indent + 1).append("BufferLedger[" + id + "] holds ").append(buffers.size())
+              .append(" buffers. \n");
+          for (DrillBuf buf : buffers.keySet()) {
+            buf.print(sb, indent + 2, verbosity);
+          }
+        }
+      }
+
+    }
+
+    /**
+     * Release this ledger. This means that all reference counts associated with this ledger are no longer used. This
+     * will inform the AllocatorManager to make a decision about how to manage any memory owned by this particular
+     * BufferLedger
+     */
+    public void release() {
+      listener.release();
+    }
+
+    /**
+     * Returns the ledger associated with a particular BufferAllocator. If the BufferAllocator doesn't currently have a
+     * ledger associated with this AllocatorManager, a new one is created. This is placed on BufferLedger rather than
+     * AllocatorManager direclty because DrillBufs don't have access to AllocatorManager and they are the ones
+     * responsible for exposing the ability to associate mutliple allocators with a particular piece of underlying
+     * memory.
+     *
+     * @param allocator
+     * @return
+     */
+    public BufferLedger getLedgerForAllocator(BufferAllocator allocator) {
+      return associate(allocator.unwrap(BaseAllocator.class));
+    }
+
+    /**
+     * Create a new DrillBuf associated with this AllocatorManager and memory. Does not impact reference count.
+     * Typically used for slicing.
+     * @param offset
+     *          The offset in bytes to start this new DrillBuf.
+     * @param length
+     *          The length in bytes that this DrillBuf will provide access to.
+     * @return A new DrillBuf that shares references with all DrillBufs associated with this BufferLedger
+     */
+    public DrillBuf newDrillBuf(int offset, int length) {
+      return newDrillBuf(offset, length, null, false);
+    }
+
+    /**
+     * Create a new DrillBuf associated with this AllocatorManager and memory.
+     * @param offset
+     *          The offset in bytes to start this new DrillBuf.
+     * @param length
+     *          The length in bytes that this DrillBuf will provide access to.
+     * @param manager
+     *          An optional BufferManager argument that can be used to manage expansion of this DrillBuf
+     * @param retain
+     *          Whether or not the newly created buffer should get an additional reference count added to it.
+     * @return A new DrillBuf that shares references with all DrillBufs associated with this BufferLedger
+     * @return
+     */
+    public DrillBuf newDrillBuf(int offset, int length, BufferManager manager, boolean retain) {
+      final DrillBuf buf = new DrillBuf(
+          bufRefCnt,
+          this,
+          underlying,
+          manager,
+          allocator.getAsByteBufAllocator(),
+          offset,
+          length,
+          false);
+
+      if (retain) {
+        buf.retain();
+      }
+
+      if (BaseAllocator.DEBUG) {
+        historicalLog.recordEvent(
+            "DrillBuf(BufferLedger, BufferAllocator[%d], UnsafeDirectLittleEndian[identityHashCode == "
+                + "%d](%s)) => ledger hc == %d",
+            allocator.getId(), System.identityHashCode(buf), buf.toString(),
+            System.identityHashCode(this));
+
+        synchronized (buffers) {
+          buffers.put(buf, null);
+        }
+      }
+
+      return buf;
+
+    }
+
+    /**
+     * What is the total size (in bytes) of memory underlying this ledger.
+     *
+     * @return Size in bytes
+     */
+    public int getSize() {
+      return size;
+    }
+
+    /**
+     * How much memory is accounted for by this ledger. This is either getSize() if this is the owning ledger for the
+     * memory or zero in the case that this is not the owning ledger associated with this memory.
+     *
+     * @return Amount of accounted(owned) memory associated with this ledger.
+     */
+    public int getAccountedSize() {
+      try (AutoCloseableLock read = readLock.open()) {
+        if (owningLedger == this) {
+          return size;
+        } else {
+          return 0;
+        }
+      }
+    }
+
+    /**
+     * Package visible for debugging/verification only.
+     */
+    UnsafeDirectLittleEndian getUnderlying() {
+      return underlying;
+    }
+
+    /**
+     * Package visible for debugging/verification only.
+     */
+    boolean isOwningLedger() {
+      return this == owningLedger;
+    }
+
+  }
+
+}
\ No newline at end of file
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
new file mode 100644
index 0000000000..3fbd0a3ea9
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
@@ -0,0 +1,704 @@
+/**
+ * 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.drill.exec.memory;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.DrillBuf;
+import io.netty.buffer.UnsafeDirectLittleEndian;
+
+import java.util.Arrays;
+import java.util.IdentityHashMap;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.drill.common.HistoricalLog;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.memory.AllocatorManager.BufferLedger;
+import org.apache.drill.exec.ops.BufferManager;
+import org.apache.drill.exec.util.AssertionUtil;
+
+import com.google.common.base.Preconditions;
+
+public abstract class BaseAllocator extends Accountant implements BufferAllocator {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseAllocator.class);
+
+  public static final String DEBUG_ALLOCATOR = "drill.memory.debug.allocator";
+
+  private static final AtomicLong ID_GENERATOR = new AtomicLong(0);
+  private static final int CHUNK_SIZE = AllocatorManager.INNER_ALLOCATOR.getChunkSize();
+
+  public static final int DEBUG_LOG_LENGTH = 6;
+  public static final boolean DEBUG = AssertionUtil.isAssertionsEnabled()
+      || Boolean.parseBoolean(System.getProperty(DEBUG_ALLOCATOR, "false"));
+  private final Object DEBUG_LOCK = DEBUG ? new Object() : null;
+
+  private final BaseAllocator parentAllocator;
+  private final ByteBufAllocator thisAsByteBufAllocator;
+  private final IdentityHashMap<BaseAllocator, Object> childAllocators;
+  private final DrillBuf empty;
+
+  private volatile boolean isClosed = false; // the allocator has been closed
+
+  // Package exposed for sharing between AllocatorManger and BaseAllocator objects
+  final long id = ID_GENERATOR.incrementAndGet(); // unique ID assigned to each allocator
+  final String name;
+  final RootAllocator root;
+
+  // members used purely for debugging
+  private final IdentityHashMap<BufferLedger, Object> childLedgers;
+  private final IdentityHashMap<Reservation, Object> reservations;
+  private final HistoricalLog historicalLog;
+
+  protected BaseAllocator(
+      final BaseAllocator parentAllocator,
+      final String name,
+      final long initReservation,
+      final long maxAllocation) throws OutOfMemoryException {
+    super(parentAllocator, initReservation, maxAllocation);
+
+    if (parentAllocator != null) {
+      this.root = parentAllocator.root;
+      empty = parentAllocator.empty;
+    } else if (this instanceof RootAllocator) {
+      this.root = (RootAllocator) this;
+      empty = createEmpty();
+    } else {
+      throw new IllegalStateException("An parent allocator must either carry a root or be the root.");
+    }
+
+    this.parentAllocator = parentAllocator;
+    this.name = name;
+
+    // TODO: DRILL-4131
+    // this.thisAsByteBufAllocator = new DrillByteBufAllocator(this);
+    this.thisAsByteBufAllocator = AllocatorManager.INNER_ALLOCATOR.allocator;
+
+    if (DEBUG) {
+      childAllocators = new IdentityHashMap<>();
+      reservations = new IdentityHashMap<>();
+      childLedgers = new IdentityHashMap<>();
+      historicalLog = new HistoricalLog(DEBUG_LOG_LENGTH, "allocator[%d]", id);
+      hist("created by \"%s\", owned = %d", name.toString(), this.getAllocatedMemory());
+    } else {
+      childAllocators = null;
+      reservations = null;
+      historicalLog = null;
+      childLedgers = null;
+    }
+
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public DrillBuf getEmpty() {
+    return empty;
+  }
+
+  /**
+   * For debug/verification purposes only. Allows an AllocatorManager to tell the allocator that we have a new ledger
+   * associated with this allocator.
+   */
+  void associateLedger(BufferLedger ledger) {
+    if (DEBUG) {
+      synchronized (DEBUG_LOCK) {
+        childLedgers.put(ledger, null);
+      }
+    }
+  }
+
+  /**
+   * For debug/verification purposes only. Allows an AllocatorManager to tell the allocator that we are removing a
+   * ledger associated with this allocator
+   */
+  void dissociateLedger(BufferLedger ledger) {
+    if (DEBUG) {
+      synchronized (DEBUG_LOCK) {
+        if (!childLedgers.containsKey(ledger)) {
+          throw new IllegalStateException("Trying to remove a child ledger that doesn't exist.");
+        }
+        childLedgers.remove(ledger);
+      }
+    }
+  }
+
+  /**
+   * Track when a ChildAllocator of this BaseAllocator is closed. Used for debugging purposes.
+   *
+   * @param childAllocator
+   *          The child allocator that has been closed.
+   */
+  private void childClosed(final BaseAllocator childAllocator) {
+    if (DEBUG) {
+      Preconditions.checkArgument(childAllocator != null, "child allocator can't be null");
+
+      synchronized (DEBUG_LOCK) {
+        final Object object = childAllocators.remove(childAllocator);
+        if (object == null) {
+          childAllocator.historicalLog.logHistory(logger);
+          throw new IllegalStateException("Child allocator[" + childAllocator.id
+              + "] not found in parent allocator[" + id + "]'s childAllocators");
+        }
+      }
+    }
+  }
+
+  private static String createErrorMsg(final BufferAllocator allocator, final int rounded, final int requested) {
+    if (rounded != requested) {
+      return String.format(
+          "Unable to allocate buffer of size %d (rounded from %d) due to memory limit. Current allocation: %d",
+          rounded, requested, allocator.getAllocatedMemory());
+    } else {
+      return String.format("Unable to allocate buffer of size %d due to memory limit. Current allocation: %d",
+          rounded, allocator.getAllocatedMemory());
+    }
+  }
+
+  @Override
+  public DrillBuf buffer(final int initialRequestSize) {
+    return buffer(initialRequestSize, null);
+  }
+
+  private DrillBuf createEmpty(){
+    return new DrillBuf(new AtomicInteger(), null, AllocatorManager.INNER_ALLOCATOR.empty, null, null, 0, 0, true);
+  }
+
+  @Override
+  public DrillBuf buffer(final int initialRequestSize, BufferManager manager) {
+
+    Preconditions.checkArgument(initialRequestSize >= 0, "the minimimum requested size must be non-negative");
+    Preconditions.checkArgument(initialRequestSize >= 0, "the maximum requested size must be non-negative");
+
+    if (initialRequestSize == 0) {
+      return empty;
+    }
+
+    // round to next largest power of two if we're within a chunk since that is how our allocator operates
+    final int actualRequestSize = initialRequestSize < CHUNK_SIZE ?
+        nextPowerOfTwo(initialRequestSize)
+        : initialRequestSize;
+    AllocationOutcome outcome = this.allocateBytes(actualRequestSize);
+    if (!outcome.isOk()) {
+      throw new OutOfMemoryException(createErrorMsg(this, actualRequestSize, initialRequestSize));
+    }
+
+    boolean success = true;
+    try {
+      DrillBuf buffer = bufferWithoutReservation(actualRequestSize, manager);
+      success = true;
+      return buffer;
+    } finally {
+      if (!success) {
+        releaseBytes(actualRequestSize);
+      }
+    }
+
+  }
+
+  /**
+   * Used by usual allocation as well as for allocating a pre-reserved buffer. Skips the typical accounting associated
+   * with creating a new buffer.
+   */
+  private DrillBuf bufferWithoutReservation(final int size, BufferManager bufferManager) throws OutOfMemoryException {
+    AllocatorManager manager = new AllocatorManager(this, size);
+    BufferLedger ledger = manager.associate(this);
+    DrillBuf buffer = ledger.newDrillBuf(0, size, bufferManager, true);
+
+    // make sure that our allocation is equal to what we expected.
+    Preconditions.checkArgument(buffer.capacity() == size,
+        "Allocated capacity %d was not equal to requested capacity %d.", buffer.capacity(), size);
+
+    return buffer;
+  }
+
+  @Override
+  public ByteBufAllocator getAsByteBufAllocator() {
+    return thisAsByteBufAllocator;
+  }
+
+  /**
+   * Return a unique Id for an allocator. Id's may be recycled after a long period of time.
+   *
+   * <p>
+   * Primary use for this is for debugging output.
+   * </p>
+   *
+   * @return the allocator's id
+   */
+  long getId() {
+    return id;
+  }
+
+  @Override
+  public BufferAllocator newChildAllocator(
+      final String name,
+      final long initReservation,
+      final long maxAllocation) {
+    final ChildAllocator childAllocator = new ChildAllocator(this, name, initReservation, maxAllocation);
+
+    if (DEBUG) {
+      childAllocators.put(childAllocator, childAllocator);
+      historicalLog.recordEvent("allocator[%d] created new child allocator[%d]",
+          id, childAllocator.id);
+    }
+
+    return childAllocator;
+  }
+
+  private class Reservation extends AllocationReservation {
+    private final HistoricalLog historicalLog;
+
+    public Reservation() {
+      if (DEBUG) {
+        historicalLog = new HistoricalLog("Reservation[allocator[%d], %d]", id, System.identityHashCode(this));
+        historicalLog.recordEvent("created");
+        synchronized (DEBUG_LOCK) {
+          reservations.put(this, this);
+        }
+      } else {
+        historicalLog = null;
+      }
+    }
+
+    @Override
+    public void close() {
+      if (DEBUG) {
+        if (!isClosed()) {
+          final Object object;
+          synchronized (DEBUG_LOCK) {
+            object = reservations.remove(this);
+          }
+          if (object == null) {
+            final StringBuilder sb = new StringBuilder();
+            print(sb, 0, Verbosity.LOG_WITH_STACKTRACE);
+            logger.debug(sb.toString());
+            throw new IllegalStateException(
+                String.format("Didn't find closing reservation[%d]", System.identityHashCode(this)));
+          }
+
+          historicalLog.recordEvent("closed");
+        }
+      }
+
+      super.close();
+    }
+
+    @Override
+    protected boolean reserve(int nBytes) {
+      final AllocationOutcome outcome = BaseAllocator.this.allocateBytes(nBytes);
+
+      if (DEBUG) {
+        historicalLog.recordEvent("reserve(%d) => %s", nBytes, Boolean.toString(outcome.isOk()));
+      }
+
+      return outcome.isOk();
+    }
+
+    @Override
+    protected DrillBuf allocate(int nBytes) {
+      boolean success = false;
+
+      /*
+       * The reservation already added the requested bytes to the allocators owned and allocated bytes via reserve().
+       * This ensures that they can't go away. But when we ask for the buffer here, that will add to the allocated bytes
+       * as well, so we need to return the same number back to avoid double-counting them.
+       */
+      try {
+        final DrillBuf drillBuf = BaseAllocator.this.bufferWithoutReservation(nBytes, null);
+
+        if (DEBUG) {
+          historicalLog.recordEvent("allocate() => %s",
+              drillBuf == null ? "null" : String.format("DrillBuf[%d]", drillBuf.getId()));
+        }
+        success = true;
+        return drillBuf;
+      } finally {
+        if (!success) {
+          releaseBytes(nBytes);
+        }
+      }
+    }
+
+    @Override
+    protected void releaseReservation(int nBytes) {
+      releaseBytes(nBytes);
+
+      if (DEBUG) {
+        historicalLog.recordEvent("releaseReservation(%d)", nBytes);
+      }
+    }
+
+  }
+
+  @Override
+  public AllocationReservation newReservation() {
+    return new Reservation();
+  }
+
+
+  @Override
+  public synchronized void close() {
+    /*
+     * Some owners may close more than once because of complex cleanup and shutdown
+     * procedures.
+     */
+    if (isClosed) {
+      return;
+    }
+
+    if (DEBUG) {
+      synchronized(DEBUG_LOCK) {
+        verifyAllocator();
+
+        // are there outstanding child allocators?
+        if (!childAllocators.isEmpty()) {
+          for (final BaseAllocator childAllocator : childAllocators.keySet()) {
+            if (childAllocator.isClosed) {
+              logger.warn(String.format(
+                  "Closed child allocator[%s] on parent allocator[%s]'s child list.\n%s",
+                  childAllocator.name, name, toString()));
+            }
+          }
+
+          throw new IllegalStateException(
+              String.format("Allocator[%s] closed with outstanding child allocators.\n%s", name, toString()));
+        }
+
+        // are there outstanding buffers?
+        final int allocatedCount = childLedgers.size();
+        if (allocatedCount > 0) {
+          throw new IllegalStateException(
+              String.format("Allocator[%s] closed with outstanding buffers allocated (%d).\n%s",
+                  name, allocatedCount, toString()));
+        }
+
+        if (reservations.size() != 0) {
+          throw new IllegalStateException(
+              String.format("Allocator[%s] closed with outstanding reservations (%d).\n%s", name, reservations.size(),
+                  toString()));
+        }
+
+      }
+    }
+
+      // Is there unaccounted-for outstanding allocation?
+      final long allocated = getAllocatedMemory();
+      if (allocated > 0) {
+        throw new IllegalStateException(
+          String.format("Unaccounted for outstanding allocation (%d)\n%s", allocated, toString()));
+      }
+
+    // we need to release our memory to our parent before we tell it we've closed.
+    super.close();
+
+    // Inform our parent allocator that we've closed
+    if (parentAllocator != null) {
+      parentAllocator.childClosed(this);
+    }
+
+    if (DEBUG) {
+      historicalLog.recordEvent("closed");
+      logger.debug(String.format(
+          "closed allocator[%s].",
+          name));
+    }
+
+    isClosed = true;
+
+
+  }
+
+  public String toString() {
+    final Verbosity verbosity = logger.isTraceEnabled() || true ? Verbosity.LOG_WITH_STACKTRACE : Verbosity.BASIC;
+    final StringBuilder sb = new StringBuilder();
+    print(sb, 0, verbosity);
+    return sb.toString();
+  }
+
+  /**
+   * Provide a verbose string of the current allocator state. Includes the state of all child allocators, along with
+   * historical logs of each object and including stacktraces.
+   *
+   * @return A Verbose string of current allocator state.
+   */
+  public String toVerboseString() {
+    final StringBuilder sb = new StringBuilder();
+    print(sb, 0, Verbosity.LOG_WITH_STACKTRACE);
+    return sb.toString();
+  }
+
+  private void hist(String noteFormat, Object... args) {
+    historicalLog.recordEvent(noteFormat, args);
+  }
+
+  /**
+   * Rounds up the provided value to the nearest power of two.
+   *
+   * @param val
+   *          An integer value.
+   * @return The closest power of two of that value.
+   */
+  static int nextPowerOfTwo(int val) {
+    int highestBit = Integer.highestOneBit(val);
+    if (highestBit == val) {
+      return val;
+    } else {
+      return highestBit << 1;
+    }
+  }
+
+
+  /**
+   * Verifies the accounting state of the allocator. Only works for DEBUG.
+   *
+   * @throws IllegalStateException
+   *           when any problems are found
+   */
+  void verifyAllocator() {
+    final IdentityHashMap<UnsafeDirectLittleEndian, BaseAllocator> buffersSeen = new IdentityHashMap<>();
+    verifyAllocator(buffersSeen);
+  }
+
+  /**
+   * Verifies the accounting state of the allocator. Only works for DEBUG.
+   *
+   * <p>
+   * This overload is used for recursive calls, allowing for checking that DrillBufs are unique across all allocators
+   * that are checked.
+   * </p>
+   *
+   * @param buffersSeen
+   *          a map of buffers that have already been seen when walking a tree of allocators
+   * @throws IllegalStateException
+   *           when any problems are found
+   */
+  private void verifyAllocator(final IdentityHashMap<UnsafeDirectLittleEndian, BaseAllocator> buffersSeen) {
+    synchronized (DEBUG_LOCK) {
+
+      // The remaining tests can only be performed if we're in debug mode.
+      if (!DEBUG) {
+        return;
+      }
+
+      final long allocated = getAllocatedMemory();
+
+      // verify my direct descendants
+      final Set<BaseAllocator> childSet = childAllocators.keySet();
+      for (final BaseAllocator childAllocator : childSet) {
+        childAllocator.verifyAllocator(buffersSeen);
+      }
+
+      /*
+       * Verify my relationships with my descendants.
+       *
+       * The sum of direct child allocators' owned memory must be <= my allocated memory; my allocated memory also
+       * includes DrillBuf's directly allocated by me.
+       */
+      long childTotal = 0;
+      for (final BaseAllocator childAllocator : childSet) {
+        childTotal += Math.max(childAllocator.getAllocatedMemory(), childAllocator.reservation);
+      }
+      if (childTotal > getAllocatedMemory()) {
+        historicalLog.logHistory(logger);
+        logger.debug("allocator[" + id + "] child event logs BEGIN");
+        for (final BaseAllocator childAllocator : childSet) {
+          childAllocator.historicalLog.logHistory(logger);
+        }
+        logger.debug("allocator[" + id + "] child event logs END");
+        throw new IllegalStateException(
+            "Child allocators own more memory (" + childTotal + ") than their parent (id = "
+                + id + " ) has allocated (" + getAllocatedMemory() + ')');
+      }
+
+      // Furthermore, the amount I've allocated should be that plus buffers I've allocated.
+      long bufferTotal = 0;
+
+      final Set<BufferLedger> ledgerSet = childLedgers.keySet();
+      for (final BufferLedger ledger : ledgerSet) {
+        if (!ledger.isOwningLedger()) {
+          continue;
+        }
+
+        final UnsafeDirectLittleEndian udle = ledger.getUnderlying();
+        /*
+         * Even when shared, DrillBufs are rewrapped, so we should never see the same instance twice.
+         */
+        final BaseAllocator otherOwner = buffersSeen.get(udle);
+        if (otherOwner != null) {
+          throw new IllegalStateException("This allocator's drillBuf already owned by another allocator");
+        }
+        buffersSeen.put(udle, this);
+
+        bufferTotal += udle.maxCapacity();
+      }
+
+      // Preallocated space has to be accounted for
+      final Set<Reservation> reservationSet = reservations.keySet();
+      long reservedTotal = 0;
+      for (final Reservation reservation : reservationSet) {
+        if (!reservation.isUsed()) {
+          reservedTotal += reservation.getSize();
+        }
+      }
+
+      if (bufferTotal + reservedTotal + childTotal != getAllocatedMemory()) {
+        final StringBuilder sb = new StringBuilder();
+        sb.append("allocator[");
+        sb.append(Long.toString(id));
+        sb.append("]\nallocated: ");
+        sb.append(Long.toString(allocated));
+        sb.append(" allocated - (bufferTotal + reservedTotal + childTotal): ");
+        sb.append(Long.toString(allocated - (bufferTotal + reservedTotal + childTotal)));
+        sb.append('\n');
+
+        if (bufferTotal != 0) {
+          sb.append("buffer total: ");
+          sb.append(Long.toString(bufferTotal));
+          sb.append('\n');
+          dumpBuffers(sb, ledgerSet);
+        }
+
+        if (childTotal != 0) {
+          sb.append("child total: ");
+          sb.append(Long.toString(childTotal));
+          sb.append('\n');
+
+          for (final BaseAllocator childAllocator : childSet) {
+            sb.append("child allocator[");
+            sb.append(Long.toString(childAllocator.id));
+            sb.append("] owned ");
+            sb.append(Long.toString(childAllocator.getAllocatedMemory()));
+            sb.append('\n');
+          }
+        }
+
+        if (reservedTotal != 0) {
+          sb.append(String.format("reserved total : ", reservedTotal));
+          for (final Reservation reservation : reservationSet) {
+            reservation.historicalLog.buildHistory(sb, 0, true);
+            sb.append('\n');
+          }
+        }
+
+        logger.debug(sb.toString());
+        throw new IllegalStateException(String.format(
+            "allocator[%d]: buffer space (%d) + prealloc space (%d) + child space (%d) != allocated (%d)",
+            id, bufferTotal, reservedTotal, childTotal, allocated));
+      }
+    }
+  }
+
+  void print(StringBuilder sb, int level, Verbosity verbosity) {
+
+    indent(sb, level)
+        .append("Allocator(")
+        .append(name)
+        .append(") ")
+        .append(reservation)
+        .append('/')
+        .append(getAllocatedMemory())
+        .append('/')
+        .append(getPeakMemoryAllocation())
+        .append('/')
+        .append(getLimit())
+        .append(" (res/actual/peak/limit)")
+        .append('\n');
+
+    indent(sb, level + 1).append(String.format("child allocators: %d\n", childAllocators.size()));
+    for (BaseAllocator child : childAllocators.keySet()) {
+      child.print(sb, level + 2, verbosity);
+    }
+
+    if (DEBUG) {
+      indent(sb, level + 1).append(String.format("ledgers: %d\n", childLedgers.size()));
+      for (BufferLedger ledger : childLedgers.keySet()) {
+        ledger.print(sb, level + 2, verbosity);
+      }
+
+      final Set<Reservation> reservations = this.reservations.keySet();
+      indent(sb, level + 1).append(String.format("reservations: %d\n", reservations.size()));
+      for (final Reservation reservation : reservations) {
+        if (verbosity.includeHistoricalLog) {
+          reservation.historicalLog.buildHistory(sb, level + 3, true);
+        }
+      }
+
+    }
+
+  }
+
+  private void dumpBuffers(final StringBuilder sb, final Set<BufferLedger> ledgerSet) {
+    for (final BufferLedger ledger : ledgerSet) {
+      if (!ledger.isOwningLedger()) {
+        continue;
+      }
+      final UnsafeDirectLittleEndian udle = ledger.getUnderlying();
+      sb.append("UnsafeDirectLittleEndian[dentityHashCode == ");
+      sb.append(Integer.toString(System.identityHashCode(udle)));
+      sb.append("] size ");
+      sb.append(Integer.toString(udle.maxCapacity()));
+      sb.append('\n');
+    }
+  }
+
+
+  public static StringBuilder indent(StringBuilder sb, int indent) {
+    final char[] indentation = new char[indent * 2];
+    Arrays.fill(indentation, ' ');
+    sb.append(indentation);
+    return sb;
+  }
+
+  public static enum Verbosity {
+    BASIC(false, false), // only include basic information
+    LOG(true, false), // include basic
+    LOG_WITH_STACKTRACE(true, true) //
+    ;
+
+    public final boolean includeHistoricalLog;
+    public final boolean includeStackTraces;
+
+    Verbosity(boolean includeHistoricalLog, boolean includeStackTraces) {
+      this.includeHistoricalLog = includeHistoricalLog;
+      this.includeStackTraces = includeStackTraces;
+    }
+  }
+
+  public static boolean isDebug() {
+    return DEBUG;
+  }
+
+  public boolean isClosed() {
+    return isClosed;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <T> T unwrap(Class<T> c) {
+    if (BaseAllocator.class.isAssignableFrom(c)) {
+      return (T) this;
+    }
+
+    throw new UnsupportedOperationException("Unable to unwrap type to class: " + c.getName());
+  }
+
+}
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BoundsChecking.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BoundsChecking.java
index a15e348ce7..bc611826aa 100644
--- a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BoundsChecking.java
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BoundsChecking.java
@@ -31,4 +31,5 @@
 
   private BoundsChecking() {
   }
+
 }
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
index a708e92da3..f3a8738657 100644
--- a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
@@ -20,121 +20,144 @@
 import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.DrillBuf;
 
-import java.io.Closeable;
-
 import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.util.Pointer;
+import org.apache.drill.exec.ops.BufferManager;
 
 /**
- * Wrapper class to deal with byte buffer allocation. Ensures users only use designated methods. Also allows inser
+ * Wrapper class to deal with byte buffer allocation. Ensures users only use designated methods.
  */
-public interface BufferAllocator extends Closeable {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BufferAllocator.class);
-
+public interface BufferAllocator extends AutoCloseable {
   /**
    * Allocate a new or reused buffer of the provided size. Note that the buffer may technically be larger than the
-   * requested size for rounding purposes. However, the buffers capacity will be set to the configured size.
+   * requested size for rounding purposes. However, the buffer's capacity will be set to the configured size.
    *
    * @param size
    *          The size in bytes.
-   * @return A new ByteBuf.
-   * @throws OutOfMemoryException if buffer cannot be allocated
+   * @return a new DrillBuf, or null if the request can't be satisfied
+   * @throws OutOfMemoryException
+   *           if buffer cannot be allocated
    */
-  public abstract DrillBuf buffer(int size);
+  public DrillBuf buffer(int size);
 
   /**
-   * Allocate a new or reused buffer within provided range. Note that the buffer may technically be larger than the
-   * requested size for rounding purposes. However, the buffers capacity will be set to the configured size.
+   * Allocate a new or reused buffer of the provided size. Note that the buffer may technically be larger than the
+   * requested size for rounding purposes. However, the buffer's capacity will be set to the configured size.
    *
-   * @param minSize The minimum size in bytes.
-   * @param maxSize The maximum size in bytes.
-   * @return A new ByteBuf.
-   * @throws OutOfMemoryException if buffer cannot be allocated
+   * @param size
+   *          The size in bytes.
+   * @param manager
+   *          A buffer manager to manage reallocation.
+   * @return a new DrillBuf, or null if the request can't be satisfied
+   * @throws OutOfMemoryException
+   *           if buffer cannot be allocated
    */
-  public abstract DrillBuf buffer(int minSize, int maxSize);
-
-  public abstract ByteBufAllocator getUnderlyingAllocator();
+  public DrillBuf buffer(int size, BufferManager manager);
 
   /**
-   * Create a child allocator nested below this one.
+   * Returns the allocator this allocator falls back to when it needs more memory.
    *
-   * @param context
-   *          - BufferManager associated with the new child allocator
-   * @param initialReservation
-   *          - specified in bytes
-   * @param maximumReservation
-   *          - specified in bytes
-   * @param applyFragmentLimit
-   *          - flag to conditionally enable fragment memory limits
-   * @return - a new buffer allocator owned by the parent it was spawned from
-   * @throws OutOfMemoryException
-   *           - when off-heap memory has been exhausted
+   * @return the underlying allocator used by this allocator
    */
-  public abstract BufferAllocator getChildAllocator(LimitConsumer limitListener, long initialReservation,
-      long maximumReservation, boolean applyFragmentLimit) throws OutOfMemoryException;
+  public ByteBufAllocator getAsByteBufAllocator();
 
   /**
-   * Take over ownership of fragment accounting.  Always takes over ownership.
-   * @param buf
-   * @return false if over allocation.
+   * Create a new child allocator.
+   *
+   * @param name
+   *          the name of the allocator.
+   * @param initReservation
+   *          the initial space reservation (obtained from this allocator)
+   * @param maxAllocation
+   *          maximum amount of space the new allocator can allocate
+   * @return the new allocator, or null if it can't be created
    */
-  public boolean takeOwnership(DrillBuf buf) ;
+  public BufferAllocator newChildAllocator(String name, long initReservation, long maxAllocation);
 
   /**
-   * Take over ownership of fragment accounting.  Always takes over ownership.
-   * @param buf
-   * @return false if over allocation.
+   * Close and release all buffers generated from this buffer pool.
+   *
+   * <p>When assertions are on, complains if there are any outstanding buffers; to avoid
+   * that, release all buffers before the allocator is closed.
    */
-  public boolean takeOwnership(DrillBuf buf, Pointer<DrillBuf> bufOut);
-
-  public PreAllocator getNewPreAllocator();
-
-  //public void addFragmentContext(FragmentContext c);
+  @Override
+  public void close();
 
   /**
-   * For Top Level Allocators. Reset the fragment limits for all allocators
+   * Returns the amount of memory currently allocated from this allocator.
+   *
+   * @return the amount of memory currently allocated
    */
-  public void resetLimits();
+  public long getAllocatedMemory();
 
   /**
-   * For Child allocators to set the Fragment limit for the corresponding fragment allocator.
-   * @param l the new fragment limit
+   * Set the maximum amount of memory this allocator is allowed to allocate.
+   *
+   * @param newLimit
+   *          The new Limit to apply to allocations
    */
-  public void setLimit(long l);
+  public void setLimit(long newLimit);
 
+  /**
+   * Return the current maximum limit this allocator imposes.
+   *
+   * @return Limit in number of bytes.
+   */
   public long getLimit();
 
-
   /**
-   * Not thread safe.
+   * Returns the peak amount of memory allocated from this allocator.
    *
-   * WARNING: unclaimed pre-allocations leak memory. If you call preAllocate(), you must
-   * make sure to ultimately try to get the buffer and release it.
+   * @return the peak amount of memory allocated
    */
-  public interface PreAllocator {
-    public boolean preAllocate(int bytes);
+  public long getPeakMemoryAllocation();
 
-    public DrillBuf getAllocation();
-  }
+  /**
+   * Create an allocation reservation. A reservation is a way of building up
+   * a request for a buffer whose size is not known in advance. See
+   * {@see AllocationReservation}.
+   *
+   * @return the newly created reservation
+   */
+  public AllocationReservation newReservation();
 
   /**
-   * @param bytes
-   * @return
+   * Get a reference to the empty buffer associated with this allocator. Empty buffers are special because we don't
+   * worry about them leaking or managing reference counts on them since they don't actually point to any memory.
    */
+  public DrillBuf getEmpty();
 
   /**
-   *
+   * Return the name of this allocator. This is a human readable name that can help debugging. Typically provides
+   * coordinates about where this allocator was created
    */
+  public String getName();
 
   /**
-   * Close and release all buffers generated from this buffer pool.
+   * Return whether or not this allocator (or one if its parents) is over its limits. In the case that an allocator is
+   * over its limit, all consumers of that allocator should aggressively try to addrss the overlimit situation.
    */
-  @Override
-  public abstract void close();
+  public boolean isOverLimit();
 
-  public abstract long getAllocatedMemory();
+  /**
+   * Return a verbose string describing this allocator. If in DEBUG mode, this will also include relevant stacktraces
+   * and historical logs for underlying objects
+   *
+   * @return A very verbose description of the allocator hierarchy.
+   */
+  public String toVerboseString();
 
-  public abstract long getPeakMemoryAllocation();
+  /**
+   * Whether this allocator has been closed.
+   *
+   * @return True if allocator is closed.
+   */
+  public boolean isClosed();
 
-  public DrillBuf getEmpty();
+  /**
+   * Unwrap the class so that exposes the provided interface, if possible. Otherwise, throw Exception.
+   * @param c
+   *          The class or interface that you want this class to implement/extend.
+   * @return The instance of that class related to 'this'
+   */
+  public <T> T unwrap(Class<T> c);
 }
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java
new file mode 100644
index 0000000000..8fcabb1207
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java
@@ -0,0 +1,53 @@
+/**
+ * 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.drill.exec.memory;
+
+
+/**
+ * Child allocator class. Only slightly different from the {@see RootAllocator},
+ * in that these can't be created directly, but must be obtained from
+ * {@see BufferAllocator#newChildAllocator(AllocatorOwner, long, long, int)}.
+
+ * <p>Child allocators can only be created by the root, or other children, so
+ * this class is package private.</p>
+ */
+class ChildAllocator extends BaseAllocator {
+  /**
+   * Constructor.
+   *
+   * @param parentAllocator parent allocator -- the one creating this child
+   * @param allocatorOwner a handle to the object making the request
+   * @param allocationPolicy the allocation policy to use; the policy for all
+   *   allocators must match for each invocation of a drillbit
+   * @param initReservation initial amount of space to reserve (obtained from the parent)
+   * @param maxAllocation maximum amount of space that can be obtained from this allocator;
+   *   note this includes direct allocations (via {@see BufferAllocator#buffer(int, int)}
+   *   et al) and requests from descendant allocators. Depending on the allocation policy in
+   *   force, even less memory may be available
+   * @param flags one or more of BaseAllocator.F_* flags
+   */
+  ChildAllocator(
+      BaseAllocator parentAllocator,
+      String name,
+      long initReservation,
+      long maxAllocation) {
+    super(parentAllocator, name, initReservation, maxAllocation);
+  }
+
+
+}
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/DrillByteBufAllocator.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/DrillByteBufAllocator.java
new file mode 100644
index 0000000000..ec423e2299
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/DrillByteBufAllocator.java
@@ -0,0 +1,141 @@
+/**
+ * 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.drill.exec.memory;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.CompositeByteBuf;
+import io.netty.buffer.ExpandableByteBuf;
+
+/**
+ * An implementation of ByteBufAllocator that wraps a Drill BufferAllocator. This allows the RPC layer to be accounted
+ * and managed using Drill's BufferAllocator infrastructure. The only thin different from a typical BufferAllocator is
+ * the signature and the fact that this Allocator returns ExpandableByteBufs which enable otherwise non-expandable
+ * DrillBufs to be expandable.
+ */
+public class DrillByteBufAllocator implements ByteBufAllocator {
+
+  private static final int DEFAULT_BUFFER_SIZE = 4096;
+  private static final int DEFAULT_MAX_COMPOSITE_COMPONENTS = 16;
+
+  private final BufferAllocator allocator;
+
+  public DrillByteBufAllocator(BufferAllocator allocator) {
+    this.allocator = allocator;
+  }
+
+  @Override
+  public ByteBuf buffer() {
+    return buffer(DEFAULT_BUFFER_SIZE);
+  }
+
+  @Override
+  public ByteBuf buffer(int initialCapacity) {
+    return new ExpandableByteBuf(allocator.buffer(initialCapacity), allocator);
+  }
+
+  @Override
+  public ByteBuf buffer(int initialCapacity, int maxCapacity) {
+    return buffer(initialCapacity);
+  }
+
+  @Override
+  public ByteBuf ioBuffer() {
+    return buffer();
+  }
+
+  @Override
+  public ByteBuf ioBuffer(int initialCapacity) {
+    return buffer(initialCapacity);
+  }
+
+  @Override
+  public ByteBuf ioBuffer(int initialCapacity, int maxCapacity) {
+    return buffer(initialCapacity);
+  }
+
+  @Override
+  public ByteBuf directBuffer() {
+    return buffer();
+  }
+
+  @Override
+  public ByteBuf directBuffer(int initialCapacity) {
+    return allocator.buffer(initialCapacity);
+  }
+
+  @Override
+  public ByteBuf directBuffer(int initialCapacity, int maxCapacity) {
+    return buffer(initialCapacity, maxCapacity);
+  }
+
+  @Override
+  public CompositeByteBuf compositeBuffer() {
+    return compositeBuffer(DEFAULT_MAX_COMPOSITE_COMPONENTS);
+  }
+
+  @Override
+  public CompositeByteBuf compositeBuffer(int maxNumComponents) {
+    return new CompositeByteBuf(this, true, maxNumComponents);
+  }
+
+  @Override
+  public CompositeByteBuf compositeDirectBuffer() {
+    return compositeBuffer();
+  }
+
+  @Override
+  public CompositeByteBuf compositeDirectBuffer(int maxNumComponents) {
+    return compositeBuffer(maxNumComponents);
+  }
+
+  @Override
+  public boolean isDirectBufferPooled() {
+    return false;
+  }
+
+  @Override
+  public ByteBuf heapBuffer() {
+    throw fail();
+  }
+
+  @Override
+  public ByteBuf heapBuffer(int initialCapacity) {
+    throw fail();
+  }
+
+  @Override
+  public ByteBuf heapBuffer(int initialCapacity, int maxCapacity) {
+    throw fail();
+  }
+
+  @Override
+  public CompositeByteBuf compositeHeapBuffer() {
+    throw fail();
+  }
+
+  @Override
+  public CompositeByteBuf compositeHeapBuffer(int maxNumComponents) {
+    throw fail();
+  }
+
+  private RuntimeException fail() {
+    throw new UnsupportedOperationException("Allocator doesn't support heap-based memory.");
+  }
+
+}
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/README.md b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/README.md
new file mode 100644
index 0000000000..b52c02078e
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/README.md
@@ -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.
+-->
+# Memory: Allocation, Accounting and Management
+ 
+The memory management package contains all the memory allocation related items that Drill uses to manage memory.
+
+
+## Key Components
+Memory management can be broken into the following main components:
+
+- Memory chunk allocation and fragmentation management
+  - `PooledByteBufAllocatorL` - A LittleEndian clone of Netty's jemalloc implementation
+  - `UnsafeDirectLittleEndian` - A base level memory access interface
+  - `LargeBuffer` - A buffer backing implmentation used when working with data larger than one Netty chunk (default to 16mb)
+- Memory limits & Accounting
+  - `Accountant` - A nestable class of lockfree memory accountors.
+- Application-level memory allocation
+  - `BufferAllocator` - The public interface application users should be leveraging
+  - `BaseAllocator` - The base implementation of memory allocation, contains the meat of our the Drill allocator implementation
+  - `RootAllocator` - The root allocator. Typically only one created for a JVM
+  - `ChildAllocator` - A child allocator that derives from the root allocator
+- Buffer ownership and transfer capabilities
+  - `AllocatorManager` - Responsible for managing the relationship between multiple allocators and a single chunk of memory
+  - `BufferLedger` - Responsible for allowing maintaining the relationship between an `AllocatorManager`, a `BufferAllocator` and one or more individual `DrillBuf`s 
+- Memory access
+  - `DrillBuf` - The facade for interacting directly with a chunk of memory.
+ 
+
+## Memory Management Overview
+Drill's memory model is based on the following basic concepts:
+
+ - Memory can be allocated up to some limit. That limit could be a real limit (OS/JVM) or a locally imposed limit.
+ - Allocation operates in two phases: accounting then actual allocation. Allocation could fail at either point.
+ - Allocation failure should be recoverable. In all cases, the Allocator infrastructure should expose memory allocation failures (OS or internal limit-based) as `OutOfMemoryException`s.
+ - Any allocator can reserve memory when created. This memory shall be held such that this allocator will always be able to allocate that amount of memory.
+ - A particular application component should work to use a local allocator to understand local memory usage and better debug memory leaks.
+ - The same physical memory can be shared by multiple allocators and the allocator must provide an accounting paradigm for this purpose.
+
+## Allocator Trees
+
+Drill provides a tree-based model for memory allocation. The RootAllocator is created first, then all allocators are created as children of that allocator. The RootAllocator is responsible for being the master bookeeper for memory allocations. All other allocators are created as children of this tree. Each allocator can first determine whether it has enough local memory to satisfy a particular request. If not, the allocator can ask its parent for an additional memory allocation.
+
+## Reserving Memory
+
+Drill provides two different ways to reserve memory:
+
+  - BufferAllocator accounting reservations: 
+      When a new allocator (other than the `RootAllocator`) is initialized, it can set aside memory that it will keep locally for its lifetime. This is memory that will never be released back to its parent allocator until the allocator
+  - `AllocationReservation` via BufferAllocator.newReservation(): Allows a short-term preallocation strategy so that a particular subsystem can ensure future memory is available to support a particular request.
+  
+## Memory Ownership, Reference Counts and Sharing
+Many BufferAllocators can reference the same piece of memory at the same time. The most common situation for this is in the case of a Broadcast Join: in this situation many downstream operators in the same Drillbit will receive the same physical memory. Each of these operators will be operating within its own Allocator context. We therefore have multiple allocators all pointing at the same physical memory. It is the AllocatorManager's responsibility to ensure that in this situation, that all memory is accurately accounted for from the Root's perspective and also to ensure that the memory is correctly released once all BufferAllocators have stopped using that memory.
+
+For simplicity of accounting, we treat that memory as being used by one of the BufferAllocators associated with the memory. When that allocator releases its claim on that memory, the memory ownership is then moved to another BufferLedger belonging to the same AllocatorManager. Note that because a DrillBuf.release() is what actually causes memory ownership transfer to occur, we always precede with ownership transfer (even if that violates an allocator limit). It is the responsibility of the application owning a particular allocator to frequently confirm whether the allocator is over its memory limit (BufferAllocator.isOverLimit()) and if so, attempt to aggresively release memory to ameliorate the situation.
+
+All DrillBufs (direct or sliced) related to a single BufferLedger/BufferAllocator combination share the same reference count and either all will be valid or all will be invalid.
+
+## Object Hierarchy
+
+There are two main ways that someone can look at the object hierarchy for Drill's memory management scheme. The first is a memory based perspective as below:
+
+### Memory Perspective
+<pre>
++ AllocatorManager
+|
+|-- UnsignedDirectLittleEndian (One per AllocatorManager)
+|
+|-+ BufferLedger 1 ==> Allocator A (owning)
+| ` - DrillBuf 1
+|-+ BufferLedger 2 ==> Allocator B (non-owning)
+| ` - DrillBuf 2
+|-+ BufferLedger 3 ==> Allocator C (non-owning)
+  | - DrillBuf 3
+  | - DrillBuf 4
+  ` - DrillBuf 5
+</pre>
+
+In this picture, a piece of memory is owned by an allocator manager. An allocator manager is responsible for that piece of memory no matter which allocator(s) it is working with. An allocator manager will have relationships with a piece of raw memory (via its reference to UnsignedDirectLittleEndian) as well as references to each BufferAllocator it has a relationship to. When understanding the
+
+### Allocator Perspective
+<pre>
++ RootAllocator
+|-+ ChildAllocator 1
+| | - ChildAllocator 1.1
+| ` ...
+|
+|-+ ChildAllocator 2
+|-+ ChildAllocator 3
+| |
+| |-+ BufferLedger 1 ==> AllocatorManager 1 (owning) ==> UDLE
+| | `- DrillBuf 1
+| `-+ BufferLedger 2 ==> AllocatorManager 2 (non-owning)==> UDLE
+| 	`- DrillBuf 2
+|
+|-+ BufferLedger 3 ==> AllocatorManager 1 (non-owning)==> UDLE
+| ` - DrillBuf 3
+|-+ BufferLedger 4 ==> AllocatorManager 2 (owning) ==> UDLE
+  | - DrillBuf 4
+  | - DrillBuf 5
+  ` - DrillBuf 6
+</pre>
+
+In this picture, a RootAllocator owns three ChildAllocators. The first ChildAllocator (ChildAllocator 1) owns a subsequent ChildAllocator. ChildAllocator has two BufferLedgers/AllocatorManager references. Coincidentally, each of these AllocatorManager's is also associated with the RootAllocator. In this case, one of the these AllocatorManagers is owned by ChildAllocator 3 (AllocatorManager 1) while the other AllocatorManager (AllocatorManager 2) is owned/accounted for by the RootAllocator. Note that in this scenario, DrillBuf 1 is sharing the underlying memory as DrillBuf 3. However the subset of that memory (e.g. through slicing) might be different. Also note that DrillBuf 2 and DrillBuf 4, 5 and 6 are also sharing the same underlying memory. Also note that DrillBuf 4, 5 and 6 all share the same reference count and fate.
+
+## Debugging Issues
+The Allocator object provides a useful set of tools to better understand the status of the allocator. If in `DEBUG` mode, the allocator and supporting classes will record additional debug tracking information to better track down memory leaks and issues. To enable DEBUG mode, either enable Java assertions with `-ea` or pass the following system property to the VM when starting `-Ddrill.memory.debug.allocator=true`. The BufferAllocator also provides a `BufferAllocator.toVerboseString()` which can be used in DEBUG mode to get extensive stacktrace information and events associated with various Allocator behaviors.
\ No newline at end of file
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/RootAllocator.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/RootAllocator.java
new file mode 100644
index 0000000000..08ac5863d7
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/RootAllocator.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.drill.exec.memory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * The root allocator for using direct memory inside a Drillbit. Supports creating a
+ * tree of descendant child allocators.
+ */
+public class RootAllocator extends BaseAllocator {
+
+  public RootAllocator(final long reservation, final long limit) {
+    super(null, "ROOT", reservation, limit);
+  }
+
+  /**
+   * Verify the accounting state of the allocation system.
+   */
+  @VisibleForTesting
+  public void verify() {
+    verifyAllocator();
+  }
+}
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/LimitConsumer.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/package-info.java
similarity index 82%
rename from exec/memory/base/src/main/java/org/apache/drill/exec/memory/LimitConsumer.java
rename to exec/memory/base/src/main/java/org/apache/drill/exec/memory/package-info.java
index 777d9d2595..3c1b9e5b9d 100644
--- a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/LimitConsumer.java
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/package-info.java
@@ -15,14 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+/**
+ *  Memory Allocation, Account and Management
+ *
+ *  See the README.md file in this directory for detailed information about Drill's memory allocation subsystem.
+ *
+ */
 package org.apache.drill.exec.memory;
-
-public interface LimitConsumer {
-
-  public String getIdentifier();
-  public long getAllocated();
-
-  public long getLimit();
-
-  public void setLimit(long limit);
-}
diff --git a/exec/memory/impl/src/main/resources/drill-module.conf b/exec/memory/base/src/main/resources/drill-module.conf
similarity index 100%
rename from exec/memory/impl/src/main/resources/drill-module.conf
rename to exec/memory/base/src/main/resources/drill-module.conf
diff --git a/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestAccountant.java b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestAccountant.java
new file mode 100644
index 0000000000..31c733ffeb
--- /dev/null
+++ b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestAccountant.java
@@ -0,0 +1,163 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.memory;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.drill.exec.memory.Accountant.AllocationOutcome;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestAccountant {
+
+  @Test
+  public void basic() {
+    ensureAccurateReservations(null);
+  }
+
+  @Test
+  public void nested() {
+    final Accountant parent = new Accountant(null, 0, Long.MAX_VALUE);
+    ensureAccurateReservations(parent);
+    assertEquals(0, parent.getAllocatedMemory());
+  }
+
+  @Test
+  public void multiThread() throws InterruptedException {
+    final Accountant parent = new Accountant(null, 0, Long.MAX_VALUE);
+
+    final int numberOfThreads = 32;
+    final int loops = 100;
+    Thread[] threads = new Thread[numberOfThreads];
+
+    for (int i = 0; i < numberOfThreads; i++) {
+      Thread t = new Thread() {
+
+        @Override
+        public void run() {
+          try {
+            for (int i = 0; i < loops; i++) {
+              ensureAccurateReservations(parent);
+            }
+          } catch (Exception ex) {
+            ex.printStackTrace();
+            Assert.fail(ex.getMessage());
+          }
+        }
+
+      };
+      threads[i] = t;
+      t.start();
+    }
+
+    for (Thread thread : threads) {
+      thread.join();
+    }
+
+    assertEquals(0, parent.getAllocatedMemory());
+  }
+
+  private void ensureAccurateReservations(Accountant outsideParent) {
+    final Accountant parent = new Accountant(outsideParent, 0, 10);
+    assertEquals(0, parent.getAllocatedMemory());
+
+    final Accountant child = new Accountant(parent, 2, Long.MAX_VALUE);
+    assertEquals(2, parent.getAllocatedMemory());
+
+    {
+      AllocationOutcome first = child.allocateBytes(1);
+      assertEquals(AllocationOutcome.SUCCESS, first);
+    }
+
+    // child will have new allocation
+    assertEquals(1, child.getAllocatedMemory());
+
+    // root has no change since within reservation
+    assertEquals(2, parent.getAllocatedMemory());
+
+    {
+      AllocationOutcome first = child.allocateBytes(1);
+      assertEquals(AllocationOutcome.SUCCESS, first);
+    }
+
+    // child will have new allocation
+    assertEquals(2, child.getAllocatedMemory());
+
+    // root has no change since within reservation
+    assertEquals(2, parent.getAllocatedMemory());
+
+    child.releaseBytes(1);
+
+    // child will have new allocation
+    assertEquals(1, child.getAllocatedMemory());
+
+    // root has no change since within reservation
+    assertEquals(2, parent.getAllocatedMemory());
+
+    {
+      AllocationOutcome first = child.allocateBytes(2);
+      assertEquals(AllocationOutcome.SUCCESS, first);
+    }
+
+    // child will have new allocation
+    assertEquals(3, child.getAllocatedMemory());
+
+    // went beyond reservation, now in parent accountant
+    assertEquals(3, parent.getAllocatedMemory());
+
+    {
+      AllocationOutcome first = child.allocateBytes(7);
+      assertEquals(AllocationOutcome.SUCCESS, first);
+    }
+
+    // child will have new allocation
+    assertEquals(10, child.getAllocatedMemory());
+
+    // went beyond reservation, now in parent accountant
+    assertEquals(10, parent.getAllocatedMemory());
+
+    child.releaseBytes(9);
+
+    assertEquals(1, child.getAllocatedMemory());
+
+    // back to reservation size
+    assertEquals(2, parent.getAllocatedMemory());
+
+    AllocationOutcome first = child.allocateBytes(10);
+    assertEquals(AllocationOutcome.FAILED_PARENT, first);
+
+    // unchanged
+    assertEquals(1, child.getAllocatedMemory());
+    assertEquals(2, parent.getAllocatedMemory());
+
+    boolean withinLimit = child.forceAllocate(10);
+    assertEquals(false, withinLimit);
+
+    // at new limit
+    assertEquals(child.getAllocatedMemory(), 11);
+    assertEquals(parent.getAllocatedMemory(), 11);
+
+
+    child.releaseBytes(11);
+    assertEquals(child.getAllocatedMemory(), 0);
+    assertEquals(parent.getAllocatedMemory(), 2);
+
+    child.close();
+    parent.close();
+  }
+}
diff --git a/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java
new file mode 100644
index 0000000000..83e0d103d8
--- /dev/null
+++ b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java
@@ -0,0 +1,645 @@
+/**
+ * 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.drill.exec.memory;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import io.netty.buffer.DrillBuf;
+import io.netty.buffer.DrillBuf.TransferResult;
+
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestBaseAllocator {
+  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestBaseAllocator.class);
+
+  private final static int MAX_ALLOCATION = 8 * 1024;
+
+/*
+  // ---------------------------------------- DEBUG -----------------------------------
+
+  @After
+  public void checkBuffers() {
+    final int bufferCount = UnsafeDirectLittleEndian.getBufferCount();
+    if (bufferCount != 0) {
+      UnsafeDirectLittleEndian.logBuffers(logger);
+      UnsafeDirectLittleEndian.releaseBuffers();
+    }
+
+    assertEquals(0, bufferCount);
+  }
+
+//  @AfterClass
+//  public static void dumpBuffers() {
+//    UnsafeDirectLittleEndian.logBuffers(logger);
+//  }
+
+  // ---------------------------------------- DEBUG ------------------------------------
+*/
+
+
+  @Test
+  public void test_privateMax() throws Exception {
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(0, MAX_ALLOCATION)) {
+      final DrillBuf drillBuf1 = rootAllocator.buffer(MAX_ALLOCATION / 2);
+      assertNotNull("allocation failed", drillBuf1);
+
+      try(final BufferAllocator childAllocator =
+          rootAllocator.newChildAllocator("noLimits", 0, MAX_ALLOCATION)) {
+        final DrillBuf drillBuf2 = childAllocator.buffer(MAX_ALLOCATION / 2);
+        assertNotNull("allocation failed", drillBuf2);
+        drillBuf2.release();
+      }
+
+      drillBuf1.release();
+    }
+  }
+
+  @Test(expected=IllegalStateException.class)
+  public void testRootAllocator_closeWithOutstanding() throws Exception {
+    try {
+      try(final RootAllocator rootAllocator =
+          new RootAllocator(0, MAX_ALLOCATION)) {
+        final DrillBuf drillBuf = rootAllocator.buffer(512);
+        assertNotNull("allocation failed", drillBuf);
+      }
+    } finally {
+      /*
+       * We expect there to be one unreleased underlying buffer because we're closing
+       * without releasing it.
+       */
+/*
+      // ------------------------------- DEBUG ---------------------------------
+      final int bufferCount = UnsafeDirectLittleEndian.getBufferCount();
+      UnsafeDirectLittleEndian.releaseBuffers();
+      assertEquals(1, bufferCount);
+      // ------------------------------- DEBUG ---------------------------------
+*/
+    }
+  }
+
+  @Test
+  public void testRootAllocator_getEmpty() throws Exception {
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(0, MAX_ALLOCATION)) {
+      final DrillBuf drillBuf = rootAllocator.buffer(0);
+      assertNotNull("allocation failed", drillBuf);
+      assertEquals("capacity was non-zero", 0, drillBuf.capacity());
+      drillBuf.release();
+    }
+  }
+
+  @Ignore // TODO(DRILL-2740)
+  @Test(expected = IllegalStateException.class)
+  public void testAllocator_unreleasedEmpty() throws Exception {
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(0, MAX_ALLOCATION)) {
+      @SuppressWarnings("unused")
+      final DrillBuf drillBuf = rootAllocator.buffer(0);
+    }
+  }
+
+  @Test
+  public void testAllocator_transferOwnership() throws Exception {
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(0, MAX_ALLOCATION)) {
+      final BufferAllocator childAllocator1 =
+          rootAllocator.newChildAllocator("changeOwnership1", 0, MAX_ALLOCATION);
+      final BufferAllocator childAllocator2 =
+          rootAllocator.newChildAllocator("changeOwnership2", 0, MAX_ALLOCATION);
+
+      final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 4);
+      rootAllocator.verify();
+      TransferResult transferOwnership = drillBuf1.transferOwnership(childAllocator2);
+      final boolean allocationFit = transferOwnership.allocationFit;
+      rootAllocator.verify();
+      assertTrue(allocationFit);
+
+      drillBuf1.release();
+      childAllocator1.close();
+      rootAllocator.verify();
+
+      transferOwnership.buffer.release();
+      childAllocator2.close();
+    }
+  }
+
+  @Test
+  public void testAllocator_shareOwnership() throws Exception {
+    try (final RootAllocator rootAllocator = new RootAllocator(0, MAX_ALLOCATION)) {
+      final BufferAllocator childAllocator1 = rootAllocator.newChildAllocator("shareOwnership1", 0, MAX_ALLOCATION);
+      final BufferAllocator childAllocator2 = rootAllocator.newChildAllocator("shareOwnership2", 0, MAX_ALLOCATION);
+      final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 4);
+      rootAllocator.verify();
+
+      // share ownership of buffer.
+      final DrillBuf drillBuf2 = drillBuf1.retain(childAllocator2);
+      rootAllocator.verify();
+      assertNotNull(drillBuf2);
+      assertNotEquals(drillBuf2, drillBuf1);
+
+      // release original buffer (thus transferring ownership to allocator 2. (should leave allocator 1 in empty state)
+      drillBuf1.release();
+      rootAllocator.verify();
+      childAllocator1.close();
+      rootAllocator.verify();
+
+      final BufferAllocator childAllocator3 = rootAllocator.newChildAllocator("shareOwnership3", 0, MAX_ALLOCATION);
+      final DrillBuf drillBuf3 = drillBuf1.retain(childAllocator3);
+      assertNotNull(drillBuf3);
+      assertNotEquals(drillBuf3, drillBuf1);
+      assertNotEquals(drillBuf3, drillBuf2);
+      rootAllocator.verify();
+
+      drillBuf2.release();
+      rootAllocator.verify();
+      childAllocator2.close();
+      rootAllocator.verify();
+
+      drillBuf3.release();
+      rootAllocator.verify();
+      childAllocator3.close();
+    }
+  }
+
+  @Test
+  public void testRootAllocator_createChildAndUse() throws Exception {
+    try (final RootAllocator rootAllocator = new RootAllocator(0, MAX_ALLOCATION)) {
+      try (final BufferAllocator childAllocator = rootAllocator.newChildAllocator("createChildAndUse", 0,
+          MAX_ALLOCATION)) {
+        final DrillBuf drillBuf = childAllocator.buffer(512);
+        assertNotNull("allocation failed", drillBuf);
+        drillBuf.release();
+      }
+    }
+  }
+
+  @Test(expected=IllegalStateException.class)
+  public void testRootAllocator_createChildDontClose() throws Exception {
+    try {
+      try (final RootAllocator rootAllocator = new RootAllocator(0, MAX_ALLOCATION)) {
+        final BufferAllocator childAllocator = rootAllocator.newChildAllocator("createChildDontClose", 0,
+            MAX_ALLOCATION);
+        final DrillBuf drillBuf = childAllocator.buffer(512);
+        assertNotNull("allocation failed", drillBuf);
+      }
+    } finally {
+      /*
+       * We expect one underlying buffer because we closed a child allocator without
+       * releasing the buffer allocated from it.
+       */
+/*
+      // ------------------------------- DEBUG ---------------------------------
+      final int bufferCount = UnsafeDirectLittleEndian.getBufferCount();
+      UnsafeDirectLittleEndian.releaseBuffers();
+      assertEquals(1, bufferCount);
+      // ------------------------------- DEBUG ---------------------------------
+*/
+    }
+  }
+
+  private static void allocateAndFree(final BufferAllocator allocator) {
+    final DrillBuf drillBuf = allocator.buffer(512);
+    assertNotNull("allocation failed", drillBuf);
+    drillBuf.release();
+
+    final DrillBuf drillBuf2 = allocator.buffer(MAX_ALLOCATION);
+    assertNotNull("allocation failed", drillBuf2);
+    drillBuf2.release();
+
+    final int nBufs = 8;
+    final DrillBuf[] drillBufs = new DrillBuf[nBufs];
+    for(int i = 0; i < drillBufs.length; ++i) {
+      DrillBuf drillBufi = allocator.buffer(MAX_ALLOCATION / nBufs);
+      assertNotNull("allocation failed", drillBufi);
+      drillBufs[i] = drillBufi;
+    }
+    for(DrillBuf drillBufi : drillBufs) {
+      drillBufi.release();
+    }
+  }
+
+  @Test
+  public void testAllocator_manyAllocations() throws Exception {
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(0, MAX_ALLOCATION)) {
+      try(final BufferAllocator childAllocator =
+          rootAllocator.newChildAllocator("manyAllocations", 0, MAX_ALLOCATION)) {
+        allocateAndFree(childAllocator);
+      }
+    }
+  }
+
+  @Test
+  public void testAllocator_overAllocate() throws Exception {
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(0, MAX_ALLOCATION)) {
+      try(final BufferAllocator childAllocator =
+          rootAllocator.newChildAllocator("overAllocate", 0, MAX_ALLOCATION)) {
+        allocateAndFree(childAllocator);
+
+        try {
+          childAllocator.buffer(MAX_ALLOCATION + 1);
+          fail("allocated memory beyond max allowed");
+        } catch (OutOfMemoryException e) {
+          // expected
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testAllocator_overAllocateParent() throws Exception {
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(0, MAX_ALLOCATION)) {
+      try(final BufferAllocator childAllocator =
+          rootAllocator.newChildAllocator("overAllocateParent", 0, MAX_ALLOCATION)) {
+        final DrillBuf drillBuf1 = rootAllocator.buffer(MAX_ALLOCATION / 2);
+        assertNotNull("allocation failed", drillBuf1);
+        final DrillBuf drillBuf2 = childAllocator.buffer(MAX_ALLOCATION / 2);
+        assertNotNull("allocation failed", drillBuf2);
+
+        try {
+          childAllocator.buffer(MAX_ALLOCATION / 4);
+          fail("allocated memory beyond max allowed");
+        } catch (OutOfMemoryException e) {
+          // expected
+        }
+
+        drillBuf1.release();
+        drillBuf2.release();
+      }
+    }
+  }
+
+  private static void testAllocator_sliceUpBufferAndRelease(
+      final RootAllocator rootAllocator, final BufferAllocator bufferAllocator) {
+    final DrillBuf drillBuf1 = bufferAllocator.buffer(MAX_ALLOCATION / 2);
+    rootAllocator.verify();
+
+    final DrillBuf drillBuf2 = drillBuf1.slice(16, drillBuf1.capacity() - 32);
+    rootAllocator.verify();
+    final DrillBuf drillBuf3 = drillBuf2.slice(16, drillBuf2.capacity() - 32);
+    rootAllocator.verify();
+    @SuppressWarnings("unused")
+    final DrillBuf drillBuf4 = drillBuf3.slice(16, drillBuf3.capacity() - 32);
+    rootAllocator.verify();
+
+    drillBuf3.release(); // since they share refcounts, one is enough to release them all
+    rootAllocator.verify();
+  }
+
+  @Test
+  public void testAllocator_createSlices() throws Exception {
+    try (final RootAllocator rootAllocator = new RootAllocator(0, MAX_ALLOCATION)) {
+      testAllocator_sliceUpBufferAndRelease(rootAllocator, rootAllocator);
+
+      try (final BufferAllocator childAllocator = rootAllocator.newChildAllocator("createSlices", 0, MAX_ALLOCATION)) {
+        testAllocator_sliceUpBufferAndRelease(rootAllocator, childAllocator);
+      }
+      rootAllocator.verify();
+
+      testAllocator_sliceUpBufferAndRelease(rootAllocator, rootAllocator);
+
+      try (final BufferAllocator childAllocator = rootAllocator.newChildAllocator("createSlices", 0, MAX_ALLOCATION)) {
+        try (final BufferAllocator childAllocator2 =
+            childAllocator.newChildAllocator("createSlices", 0, MAX_ALLOCATION)) {
+          final DrillBuf drillBuf1 = childAllocator2.buffer(MAX_ALLOCATION / 8);
+          @SuppressWarnings("unused")
+          final DrillBuf drillBuf2 = drillBuf1.slice(MAX_ALLOCATION / 16, MAX_ALLOCATION / 16);
+          testAllocator_sliceUpBufferAndRelease(rootAllocator, childAllocator);
+          drillBuf1.release();
+          rootAllocator.verify();
+        }
+        rootAllocator.verify();
+
+        testAllocator_sliceUpBufferAndRelease(rootAllocator, childAllocator);
+      }
+      rootAllocator.verify();
+    }
+  }
+
+  @Test
+  public void testAllocator_sliceRanges() throws Exception {
+//    final AllocatorOwner allocatorOwner = new NamedOwner("sliceRanges");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(0, MAX_ALLOCATION)) {
+      // Populate a buffer with byte values corresponding to their indices.
+      final DrillBuf drillBuf = rootAllocator.buffer(256);
+      assertEquals(256, drillBuf.capacity());
+      assertEquals(0, drillBuf.readerIndex());
+      assertEquals(0, drillBuf.readableBytes());
+      assertEquals(0, drillBuf.writerIndex());
+      assertEquals(256, drillBuf.writableBytes());
+
+      final DrillBuf slice3 = (DrillBuf) drillBuf.slice();
+      assertEquals(0, slice3.readerIndex());
+      assertEquals(0, slice3.readableBytes());
+      assertEquals(0, slice3.writerIndex());
+//      assertEquals(256, slice3.capacity());
+//      assertEquals(256, slice3.writableBytes());
+
+      for(int i = 0; i < 256; ++i) {
+        drillBuf.writeByte(i);
+      }
+      assertEquals(0, drillBuf.readerIndex());
+      assertEquals(256, drillBuf.readableBytes());
+      assertEquals(256, drillBuf.writerIndex());
+      assertEquals(0, drillBuf.writableBytes());
+
+      final DrillBuf slice1 = (DrillBuf) drillBuf.slice();
+      assertEquals(0, slice1.readerIndex());
+      assertEquals(256, slice1.readableBytes());
+      for(int i = 0; i < 10; ++i) {
+        assertEquals(i, slice1.readByte());
+      }
+      assertEquals(256 - 10, slice1.readableBytes());
+      for(int i = 0; i < 256; ++i) {
+        assertEquals((byte) i, slice1.getByte(i));
+      }
+
+      final DrillBuf slice2 = (DrillBuf) drillBuf.slice(25, 25);
+      assertEquals(0, slice2.readerIndex());
+      assertEquals(25, slice2.readableBytes());
+      for(int i = 25; i < 50; ++i) {
+        assertEquals(i, slice2.readByte());
+      }
+
+/*
+      for(int i = 256; i > 0; --i) {
+        slice3.writeByte(i - 1);
+      }
+      for(int i = 0; i < 256; ++i) {
+        assertEquals(255 - i, slice1.getByte(i));
+      }
+*/
+
+      drillBuf.release(); // all the derived buffers share this fate
+    }
+  }
+
+  @Test
+  public void testAllocator_slicesOfSlices() throws Exception {
+//    final AllocatorOwner allocatorOwner = new NamedOwner("slicesOfSlices");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(0, MAX_ALLOCATION)) {
+      // Populate a buffer with byte values corresponding to their indices.
+      final DrillBuf drillBuf = rootAllocator.buffer(256);
+      for(int i = 0; i < 256; ++i) {
+        drillBuf.writeByte(i);
+      }
+
+      // Slice it up.
+      final DrillBuf slice0 = drillBuf.slice(0, drillBuf.capacity());
+      for(int i = 0; i < 256; ++i) {
+        assertEquals((byte) i, drillBuf.getByte(i));
+      }
+
+      final DrillBuf slice10 = slice0.slice(10, drillBuf.capacity() - 10);
+      for(int i = 10; i < 256; ++i) {
+        assertEquals((byte) i, slice10.getByte(i - 10));
+      }
+
+      final DrillBuf slice20 = slice10.slice(10, drillBuf.capacity() - 20);
+      for(int i = 20; i < 256; ++i) {
+        assertEquals((byte) i, slice20.getByte(i - 20));
+      }
+
+      final DrillBuf slice30 = slice20.slice(10,  drillBuf.capacity() - 30);
+      for(int i = 30; i < 256; ++i) {
+        assertEquals((byte) i, slice30.getByte(i - 30));
+      }
+
+      drillBuf.release();
+    }
+  }
+
+  @Test
+  public void testAllocator_transferSliced() throws Exception {
+    try (final RootAllocator rootAllocator = new RootAllocator(0, MAX_ALLOCATION)) {
+      final BufferAllocator childAllocator1 = rootAllocator.newChildAllocator("transferSliced1", 0, MAX_ALLOCATION);
+      final BufferAllocator childAllocator2 = rootAllocator.newChildAllocator("transferSliced2", 0, MAX_ALLOCATION);
+
+      final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 8);
+      final DrillBuf drillBuf2 = childAllocator2.buffer(MAX_ALLOCATION / 8);
+
+      final DrillBuf drillBuf1s = drillBuf1.slice(0, drillBuf1.capacity() / 2);
+      final DrillBuf drillBuf2s = drillBuf2.slice(0, drillBuf2.capacity() / 2);
+
+      rootAllocator.verify();
+
+      TransferResult result1 = drillBuf2s.transferOwnership(childAllocator1);
+      rootAllocator.verify();
+      TransferResult result2 = drillBuf1s.transferOwnership(childAllocator2);
+      rootAllocator.verify();
+
+      result1.buffer.release();
+      result2.buffer.release();
+
+      drillBuf1s.release(); // releases drillBuf1
+      drillBuf2s.release(); // releases drillBuf2
+
+      childAllocator1.close();
+      childAllocator2.close();
+    }
+  }
+
+  @Test
+  public void testAllocator_shareSliced() throws Exception {
+    try (final RootAllocator rootAllocator = new RootAllocator(0, MAX_ALLOCATION)) {
+      final BufferAllocator childAllocator1 = rootAllocator.newChildAllocator("transferSliced", 0, MAX_ALLOCATION);
+      final BufferAllocator childAllocator2 = rootAllocator.newChildAllocator("transferSliced", 0, MAX_ALLOCATION);
+
+      final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 8);
+      final DrillBuf drillBuf2 = childAllocator2.buffer(MAX_ALLOCATION / 8);
+
+      final DrillBuf drillBuf1s = drillBuf1.slice(0, drillBuf1.capacity() / 2);
+      final DrillBuf drillBuf2s = drillBuf2.slice(0, drillBuf2.capacity() / 2);
+
+      rootAllocator.verify();
+
+      final DrillBuf drillBuf2s1 = drillBuf2s.retain(childAllocator1);
+      final DrillBuf drillBuf1s2 = drillBuf1s.retain(childAllocator2);
+      rootAllocator.verify();
+
+      drillBuf1s.release(); // releases drillBuf1
+      drillBuf2s.release(); // releases drillBuf2
+      rootAllocator.verify();
+
+      drillBuf2s1.release(); // releases the shared drillBuf2 slice
+      drillBuf1s2.release(); // releases the shared drillBuf1 slice
+
+      childAllocator1.close();
+      childAllocator2.close();
+    }
+  }
+
+  @Test
+  public void testAllocator_transferShared() throws Exception {
+    try (final RootAllocator rootAllocator = new RootAllocator(0, MAX_ALLOCATION)) {
+      final BufferAllocator childAllocator1 = rootAllocator.newChildAllocator("transferShared1", 0, MAX_ALLOCATION);
+      final BufferAllocator childAllocator2 = rootAllocator.newChildAllocator("transferShared2", 0, MAX_ALLOCATION);
+      final BufferAllocator childAllocator3 = rootAllocator.newChildAllocator("transferShared3", 0, MAX_ALLOCATION);
+
+      final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 8);
+
+      boolean allocationFit;
+
+      DrillBuf drillBuf2 = drillBuf1.retain(childAllocator2);
+      rootAllocator.verify();
+      assertNotNull(drillBuf2);
+      assertNotEquals(drillBuf2, drillBuf1);
+
+      TransferResult result = drillBuf1.transferOwnership(childAllocator3);
+      allocationFit = result.allocationFit;
+      final DrillBuf drillBuf3 = result.buffer;
+      assertTrue(allocationFit);
+      rootAllocator.verify();
+
+      // Since childAllocator3 now has childAllocator1's buffer, 1, can close
+      drillBuf1.release();
+      childAllocator1.close();
+      rootAllocator.verify();
+
+      drillBuf2.release();
+      childAllocator2.close();
+      rootAllocator.verify();
+
+      final BufferAllocator childAllocator4 = rootAllocator.newChildAllocator("transferShared4", 0, MAX_ALLOCATION);
+      TransferResult result2 = drillBuf3.transferOwnership(childAllocator4);
+      allocationFit = result.allocationFit;
+      final DrillBuf drillBuf4 = result2.buffer;
+      assertTrue(allocationFit);
+      rootAllocator.verify();
+
+      drillBuf3.release();
+      childAllocator3.close();
+      rootAllocator.verify();
+
+      drillBuf4.release();
+      childAllocator4.close();
+      rootAllocator.verify();
+    }
+  }
+
+  @Test
+  public void testAllocator_unclaimedReservation() throws Exception {
+    try (final RootAllocator rootAllocator = new RootAllocator(0, MAX_ALLOCATION)) {
+      try (final BufferAllocator childAllocator1 =
+          rootAllocator.newChildAllocator("unclaimedReservation", 0, MAX_ALLOCATION)) {
+        try(final AllocationReservation reservation = childAllocator1.newReservation()) {
+          assertTrue(reservation.add(64));
+        }
+        rootAllocator.verify();
+      }
+    }
+  }
+
+  @Test
+  public void testAllocator_claimedReservation() throws Exception {
+    try (final RootAllocator rootAllocator = new RootAllocator(0, MAX_ALLOCATION)) {
+
+      try (final BufferAllocator childAllocator1 = rootAllocator.newChildAllocator("claimedReservation", 0,
+          MAX_ALLOCATION)) {
+
+        try (final AllocationReservation reservation = childAllocator1.newReservation()) {
+          assertTrue(reservation.add(32));
+          assertTrue(reservation.add(32));
+
+          final DrillBuf drillBuf = reservation.buffer();
+          assertEquals(64, drillBuf.capacity());
+          rootAllocator.verify();
+
+          drillBuf.release();
+          rootAllocator.verify();
+        }
+        rootAllocator.verify();
+      }
+    }
+  }
+
+  @Test
+  public void multiple() throws Exception {
+    final String owner = "test";
+    try (RootAllocator allocator = new RootAllocator(0, Long.MAX_VALUE)) {
+
+      final int op = 100000;
+
+      BufferAllocator frag1 = allocator.newChildAllocator(owner, 1500000, Long.MAX_VALUE);
+      BufferAllocator frag2 = allocator.newChildAllocator(owner, 500000, Long.MAX_VALUE);
+
+      allocator.verify();
+
+      BufferAllocator allocator11 = frag1.newChildAllocator(owner, op, Long.MAX_VALUE);
+      DrillBuf b11 = allocator11.buffer(1000000);
+
+      allocator.verify();
+
+      BufferAllocator allocator12 = frag1.newChildAllocator(owner, op, Long.MAX_VALUE);
+      DrillBuf b12 = allocator12.buffer(500000);
+
+      allocator.verify();
+
+      BufferAllocator allocator21 = frag1.newChildAllocator(owner, op, Long.MAX_VALUE);
+
+      allocator.verify();
+
+      BufferAllocator allocator22 = frag2.newChildAllocator(owner, op, Long.MAX_VALUE);
+      DrillBuf b22 = allocator22.buffer(2000000);
+
+      allocator.verify();
+
+      BufferAllocator frag3 = allocator.newChildAllocator(owner, 1000000, Long.MAX_VALUE);
+
+      allocator.verify();
+
+      BufferAllocator allocator31 = frag3.newChildAllocator(owner, op, Long.MAX_VALUE);
+      DrillBuf b31a = allocator31.buffer(200000);
+
+      allocator.verify();
+
+      // Previously running operator completes
+      b22.release();
+
+      allocator.verify();
+
+      allocator22.close();
+
+      b31a.release();
+      allocator31.close();
+
+      b12.release();
+      allocator12.close();
+
+      allocator21.close();
+
+      b11.release();
+      allocator11.close();
+
+      frag1.close();
+      frag2.close();
+      frag3.close();
+
+    }
+  }
+}
diff --git a/exec/memory/impl/src/test/java/org/apache/drill/exec/memory/TestEndianess.java b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestEndianess.java
similarity index 86%
rename from exec/memory/impl/src/test/java/org/apache/drill/exec/memory/TestEndianess.java
rename to exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestEndianess.java
index 2028a23aaf..eaefbe509a 100644
--- a/exec/memory/impl/src/test/java/org/apache/drill/exec/memory/TestEndianess.java
+++ b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestEndianess.java
@@ -22,15 +22,14 @@
 
 import org.apache.drill.common.DrillAutoCloseables;
 import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.test.DrillTest;
 import org.junit.Test;
 
 
-public class TestEndianess extends DrillTest {
+public class TestEndianess {
+
   @Test
   public void testLittleEndian() {
-    final DrillConfig drillConfig = DrillConfig.create();
-    final BufferAllocator a = RootAllocatorFactory.newRoot(drillConfig);
+    final BufferAllocator a = new RootAllocator(0, DrillConfig.getMaxDirectMemory());
     final ByteBuf b = a.buffer(4);
     b.setInt(0, 35);
     assertEquals(b.getByte(0), 35);
@@ -40,4 +39,5 @@ public void testLittleEndian() {
     b.release();
     DrillAutoCloseables.closeNoChecked(a);
   }
+
 }
diff --git a/exec/memory/impl/pom.xml b/exec/memory/impl/pom.xml
deleted file mode 100644
index b7c00acbce..0000000000
--- a/exec/memory/impl/pom.xml
+++ /dev/null
@@ -1,68 +0,0 @@
-<?xml version="1.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. -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>memory-parent</artifactId>
-    <groupId>org.apache.drill.memory</groupId>
-    <version>1.4.0-SNAPSHOT</version>
-  </parent>
-  <artifactId>drill-memory-impl</artifactId>
-  <name>exec/memory/impl</name>
-
-  <dependencies>
-
-    <dependency>
-      <groupId>org.apache.drill</groupId>
-      <artifactId>drill-protocol</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.drill</groupId>
-      <artifactId>drill-common</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.drill.memory</groupId>
-      <artifactId>drill-memory-base</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.drill</groupId>
-      <artifactId>drill-common</artifactId>
-      <version>${project.version}</version>
-      <classifier>tests</classifier>
-      <scope>test</scope>
-    </dependency>
-    
-    <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-      <version>2.5.0</version>
-    </dependency>
-    <dependency>
-      <groupId>com.codahale.metrics</groupId>
-      <artifactId>metrics-core</artifactId>
-      <version>3.0.1</version>
-    </dependency>
-
-
-  </dependencies>
-
-
-  <build>
-  </build>
-
-
-
-</project>
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AccountorImpl.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AccountorImpl.java
deleted file mode 100644
index 0ac93e4b46..0000000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AccountorImpl.java
+++ /dev/null
@@ -1,437 +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.drill.exec.memory;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.exec.util.AssertionUtil;
-
-import com.google.common.collect.LinkedListMultimap;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multimap;
-
-public class AccountorImpl implements Accountor {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AccountorImpl.class);
-
-  private static final boolean ENABLE_ACCOUNTING = AssertionUtil.isAssertionsEnabled();
-
-  public static final String ENABLE_FRAGMENT_MEMORY_LIMIT = "drill.exec.memory.enable_frag_limit";
-  public static final String FRAGMENT_MEM_OVERCOMMIT_FACTOR = "drill.exec.memory.frag_mem_overcommit_factor";
-
-  private final AtomicRemainder remainder;
-  private final long total;
-  private ConcurrentMap<ByteBuf, DebugStackTrace> buffers = Maps.newConcurrentMap();
-  private AccountorImpl parent;
-
-  private final boolean errorOnLeak;
-  // some operators are no subject to the fragment limit. They set the applyFragmentLimit to false
-
-  private final boolean enableFragmentLimit;
-  private final double  fragmentMemOvercommitFactor;
-
-  private final boolean  DEFAULT_ENABLE_FRAGMENT_LIMIT=false;
-  private final double   DEFAULT_FRAGMENT_MEM_OVERCOMMIT_FACTOR=1.5;
-
-  private final boolean applyFragmentLimit;
-
-  private final LimitConsumer limitConsumer;
-  long fragmentLimit;
-
-  private long peakMemoryAllocation = 0;
-
-  // The top level Allocator has an accountor that keeps track of all the LimitConsumers currently executing.
-  // This enables the top level accountor to calculate a new fragment limit whenever necessary.
-  private final List<LimitConsumer> limitConsumers;
-
-  public AccountorImpl(DrillConfig config, boolean errorOnLeak, LimitConsumer context, AccountorImpl parent, long max,
-      long preAllocated, boolean applyFragLimit) {
-    // TODO: fix preallocation stuff
-    this.errorOnLeak = errorOnLeak;
-    AtomicRemainder parentRemainder = parent != null ? parent.remainder : null;
-    this.parent = parent;
-
-    boolean enableFragmentLimit;
-    double  fragmentMemOvercommitFactor;
-
-    try {
-      enableFragmentLimit = config.getBoolean(ENABLE_FRAGMENT_MEMORY_LIMIT);
-      fragmentMemOvercommitFactor = config.getDouble(FRAGMENT_MEM_OVERCOMMIT_FACTOR);
-    }catch(Exception e){
-      enableFragmentLimit = DEFAULT_ENABLE_FRAGMENT_LIMIT;
-      fragmentMemOvercommitFactor = DEFAULT_FRAGMENT_MEM_OVERCOMMIT_FACTOR;
-    }
-    this.enableFragmentLimit = enableFragmentLimit;
-    this.fragmentMemOvercommitFactor = fragmentMemOvercommitFactor;
-
-
-    this.applyFragmentLimit=applyFragLimit;
-
-    this.remainder = new AtomicRemainder(errorOnLeak, parentRemainder, max, preAllocated, applyFragmentLimit);
-    this.total = max;
-    this.limitConsumer = context;
-    this.fragmentLimit=this.total; // Allow as much as possible to start with;
-    if (ENABLE_ACCOUNTING) {
-      buffers = Maps.newConcurrentMap();
-    } else {
-      buffers = null;
-    }
-    this.limitConsumers = new ArrayList<LimitConsumer>();
-    if(parent!=null && parent.parent==null){ // Only add the fragment context to the fragment level accountor
-      synchronized(this) {
-        addLimitConsumer(this.limitConsumer);
-      }
-    }
-  }
-
-  public boolean transferTo(Accountor target, DrillBuf buf, long size) {
-    return transfer(target, buf, size, true);
-  }
-
-  public boolean transferIn(DrillBuf buf, long size) {
-    return transfer(this, buf, size, false);
-  }
-
-  private boolean transfer(Accountor target, DrillBuf buf, long size, boolean release) {
-    boolean withinLimit = target.forceAdditionalReservation(size);
-    if(release){
-      release(buf, size);
-    }
-
-    if (ENABLE_ACCOUNTING) {
-      if (target instanceof AccountorImpl) {
-        ((AccountorImpl) target).buffers.put(buf, new DebugStackTrace(buf.capacity(), Thread.currentThread()
-            .getStackTrace()));
-      }
-    }
-    return withinLimit;
-  }
-
-  public long getAvailable() {
-    if (parent != null) {
-      return Math.min(parent.getAvailable(), getCapacity() - getAllocation());
-    }
-    return getCapacity() - getAllocation();
-  }
-
-  public long getCapacity() {
-    return fragmentLimit;
-  }
-
-  public long getAllocation() {
-    return remainder.getUsed();
-  }
-
-  public long getPeakMemoryAllocation() {
-    return peakMemoryAllocation;
-  }
-
-  public boolean reserve(long size) {
-    boolean status = remainder.get(size, this.applyFragmentLimit);
-    peakMemoryAllocation = Math.max(peakMemoryAllocation, getAllocation());
-    return status;
-  }
-
-  public boolean forceAdditionalReservation(long size) {
-    if (size > 0) {
-      boolean status = remainder.forceGet(size);
-      peakMemoryAllocation = Math.max(peakMemoryAllocation, getAllocation());
-      return status;
-    } else {
-      return true;
-    }
-  }
-
-  public void reserved(long expected, DrillBuf buf) {
-    // make sure to take away the additional memory that happened due to rounding.
-
-    long additional = buf.capacity() - expected;
-    if (additional > 0) {
-      remainder.forceGet(additional);
-    }
-
-    if (ENABLE_ACCOUNTING) {
-      buffers.put(buf, new DebugStackTrace(buf.capacity(), Thread.currentThread().getStackTrace()));
-    }
-
-    peakMemoryAllocation = Math.max(peakMemoryAllocation, getAllocation());
-  }
-
-
-  public void releasePartial(DrillBuf buf, long size) {
-    remainder.returnAllocation(size);
-    if (ENABLE_ACCOUNTING) {
-      if (buf != null) {
-        DebugStackTrace dst = buffers.get(buf);
-        if (dst == null) {
-          throw new IllegalStateException("Partially releasing a buffer that has already been released. Buffer: " + buf);
-        }
-        dst.size -= size;
-        if (dst.size < 0) {
-          throw new IllegalStateException("Partially releasing a buffer that has already been released. Buffer: " + buf);
-        }
-      }
-    }
-  }
-
-  void release(long size) {
-    remainder.returnAllocation(size);
-  }
-
-  public void release(DrillBuf buf, long size) {
-    remainder.returnAllocation(size);
-    if (ENABLE_ACCOUNTING) {
-      if (buf != null && buffers.remove(buf) == null) {
-        throw new IllegalStateException("Releasing a buffer that has already been released. Buffer: " + buf);
-      }
-    }
-  }
-
-  private void addLimitConsumer(LimitConsumer c) {
-    if (c == null) {
-      return;
-    }
-
-    if (parent != null){
-      parent.addLimitConsumer(c);
-    }else {
-      if(logger.isTraceEnabled()) {
-        String fragStr = c == null ? "[Null Context]" : c.getIdentifier();
-        fragStr+=" (Object Id: "+System.identityHashCode(c)+")";
-        StackTraceElement[] ste = (new Throwable()).getStackTrace();
-        StringBuffer sb = new StringBuffer();
-        for (StackTraceElement s : ste) {
-          sb.append(s.toString());
-          sb.append("\n");
-        }
-
-        logger.trace("Fragment " + fragStr + " added to root accountor.\n"+sb.toString());
-      }
-      synchronized(this) {
-        limitConsumers.add(c);
-      }
-    }
-  }
-
-  private void removeLimitConsumer(LimitConsumer c) {
-    if (c == null) {
-      return;
-    }
-
-    if (parent != null){
-      if (parent.parent==null){
-        // only fragment level allocators will have the fragment context saved
-        parent.removeLimitConsumer(c);
-      }
-    }else{
-      if(logger.isDebugEnabled()) {
-        String fragStr = c == null ? "[Null Context]" : c.getIdentifier();
-        fragStr += " (Object Id: " + System.identityHashCode(c) + ")";
-        logger.trace("Fragment " + fragStr + " removed from root accountor");
-      }
-      synchronized(this) {
-        limitConsumers.remove(c);
-      }
-    }
-  }
-
-  public long resetFragmentLimits(){
-    // returns the new capacity
-    if(!this.enableFragmentLimit){
-      return getCapacity();
-    }
-
-    if(parent!=null){
-      parent.resetFragmentLimits();
-    }else {
-      //Get remaining memory available per fragment and distribute it EQUALLY among all the fragments.
-      //Fragments get the memory limit added to the amount already allocated.
-      //This favours fragments that are already running which will get a limit greater than newly started fragments.
-      //If the already running fragments end quickly, their limits will be assigned back to the remaining fragments
-      //quickly. If they are long running, then we want to favour them with larger limits anyway.
-      synchronized (this) {
-        int nFragments = limitConsumers.size();
-        long allocatedMemory=0;
-        for (LimitConsumer fragment : limitConsumers) {
-          allocatedMemory += fragment.getAllocated();
-        }
-        if(logger.isTraceEnabled()) {
-          logger.trace("Resetting Fragment Memory Limit: total Available memory== "+total
-            +" Total Allocated Memory :"+allocatedMemory
-            +" Number of fragments: "+nFragments
-            + " fragmentMemOvercommitFactor: "+fragmentMemOvercommitFactor
-            + " Root fragment limit: "+this.fragmentLimit + "(Root obj: "+System.identityHashCode(this)+")"
-          );
-        }
-        if(nFragments>0) {
-          long rem = (total - allocatedMemory) / nFragments;
-          for (LimitConsumer fragment : limitConsumers) {
-            fragment.setLimit((long) (rem * fragmentMemOvercommitFactor));
-          }
-        }
-        if(logger.isTraceEnabled() && false){
-          StringBuffer sb= new StringBuffer();
-          sb.append("[root](0:0)");
-          sb.append("Allocated memory: ");
-          sb.append(this.getAllocation());
-          sb.append(" Fragment Limit: ");
-          sb.append(this.getFragmentLimit());
-          logger.trace(sb.toString());
-          for (LimitConsumer fragment : limitConsumers) {
-            sb= new StringBuffer();
-            sb.append('[');
-            sb.append(fragment.getIdentifier());
-            sb.append(']');
-            sb.append("Allocated memory: ");
-            sb.append(fragment.getAllocated());
-            sb.append(" Fragment Limit: ");
-            sb.append(fragment.getLimit());
-            logger.trace(sb.toString());
-          }
-          logger.trace("Resetting Complete");
-        }
-      }
-    }
-    return getCapacity();
-  }
-
-  public void close() {
-    // remove the fragment context and reset fragment limits whenever an allocator closes
-    if (parent != null && parent.parent == null && limitConsumer != null) {
-      logger.debug("Fragment " + limitConsumer.getIdentifier() + "  accountor being closed");
-      removeLimitConsumer(limitConsumer);
-    }
-    resetFragmentLimits();
-
-    if (ENABLE_ACCOUNTING && !buffers.isEmpty()) {
-      StringBuffer sb = new StringBuffer();
-      sb.append("Attempted to close accountor with ");
-      sb.append(buffers.size());
-      sb.append(" buffer(s) still allocated for ");
-      sb.append(limitConsumer.getIdentifier());
-      sb.append(".\n");
-
-      Multimap<DebugStackTrace, DebugStackTrace> multi = LinkedListMultimap.create();
-      for (DebugStackTrace t : buffers.values()) {
-        multi.put(t, t);
-      }
-
-      for (DebugStackTrace entry : multi.keySet()) {
-        Collection<DebugStackTrace> allocs = multi.get(entry);
-
-        sb.append("\n\n\tTotal ");
-        sb.append(allocs.size());
-        sb.append(" allocation(s) of byte size(s): ");
-        for (DebugStackTrace alloc : allocs) {
-          sb.append(alloc.size);
-          sb.append(", ");
-        }
-
-        sb.append("at stack location:\n");
-        entry.addToString(sb);
-      }
-      if (!buffers.isEmpty()) {
-        IllegalStateException e = new IllegalStateException(sb.toString());
-        if (errorOnLeak) {
-          throw e;
-        } else {
-          logger.warn("Memory leaked.", e);
-        }
-      }
-    }
-
-    remainder.close();
-
-  }
-
-  public void setFragmentLimit(long add) {
-    // We ADD the limit to the current allocation. If none has been allocated, this
-    // sets a new limit. If memory has already been allocated, the fragment gets its
-    // limit based on the allocation, though this might still result in reducing the
-    // limit.
-
-    if (parent != null && parent.parent==null) { // This is a fragment level accountor
-      this.fragmentLimit=getAllocation()+add;
-      this.remainder.setLimit(this.fragmentLimit);
-      logger.trace("Fragment " + limitConsumer.getIdentifier() + " memory limit set to " + this.fragmentLimit);
-    }
-  }
-
-  public long getFragmentLimit(){
-    return this.fragmentLimit;
-  }
-
-  public class DebugStackTrace {
-
-    private StackTraceElement[] elements;
-    private long size;
-
-    public DebugStackTrace(long size, StackTraceElement[] elements) {
-      super();
-      this.elements = elements;
-      this.size = size;
-    }
-
-    public void addToString(StringBuffer sb) {
-      for (int i = 3; i < elements.length; i++) {
-        sb.append("\t\t");
-        sb.append(elements[i]);
-        sb.append("\n");
-      }
-    }
-
-    @Override
-    public int hashCode() {
-      final int prime = 31;
-      int result = 1;
-      result = prime * result + Arrays.hashCode(elements);
-//      result = prime * result + (int) (size ^ (size >>> 32));
-      return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj) {
-        return true;
-      }
-      if (obj == null) {
-        return false;
-      }
-      if (getClass() != obj.getClass()) {
-        return false;
-      }
-      DebugStackTrace other = (DebugStackTrace) obj;
-      if (!Arrays.equals(elements, other.elements)) {
-        return false;
-      }
-      // weird equal where size doesn't matter for multimap purposes.
-//      if (size != other.size)
-//        return false;
-      return true;
-    }
-
-  }
-
-}
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java
deleted file mode 100644
index 0f6b8b0daa..0000000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java
+++ /dev/null
@@ -1,215 +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.drill.exec.memory;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- *
- *
- * TODO: Fix this so that preallocation can never be released back to general pool until allocator is closed.
- */
-public class AtomicRemainder {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AtomicRemainder.class);
-
-  private final AtomicRemainder parent;
-  private final AtomicLong availableShared;
-  private final AtomicLong availablePrivate;
-  private final long initTotal;
-  private final long initShared;
-  private final long initPrivate;
-  private long limit;       // An Allocator can set a variable limit less than or equal to the initTotal
-  private boolean hasLimit; // True for Atomic Remainders associated with a Fragment. May be true for Operator Level allocators some day.
-  private boolean closed = false;
-  private final boolean errorOnLeak;
-  private final boolean applyFragmentLimit;
-
-  public AtomicRemainder(boolean errorOnLeak, AtomicRemainder parent, long max, long pre, boolean applyFragLimit) {
-    this.errorOnLeak = errorOnLeak;
-    this.parent = parent;
-    this.availableShared = new AtomicLong(max - pre);
-    this.availablePrivate = new AtomicLong(pre);
-    this.initTotal = max;
-    this.initShared = max - pre;
-    this.initPrivate = pre;
-    this.limit = max;
-    this.hasLimit=false;
-    this.applyFragmentLimit=applyFragLimit; // If this is an operator that is exempt from the fragment limit, set this to false.
-//    logger.info("new AtomicRemainder. a.s. {} a.p. {} hashcode {}", availableShared, availablePrivate, hashCode(), new Exception());
-  }
-
-  public long getRemainder() {
-    return availableShared.get() + availablePrivate.get();
-  }
-
-  public long getUsed() {
-    return initTotal - getRemainder();
-  }
-
-  /**
-   * Allow an allocator to constrain the remainder to a particular limit that is lower than the initTotal.
-   * If limit is larger than initTotal, then the function will do nothing and the hasLimit flag will not be set.
-   * @param limit new remainder limit
-   */
-  public void setLimit(long limit) {
-    if(limit<initTotal){
-      this.hasLimit=true;
-      this.limit=limit;
-    }
-
-  }
-  /**
-   * Automatically allocate memory. This is used when an actual allocation happened to be larger than requested, or when
-   * a buffer has it's ownership passed to another allocator.<br>
-   * This memory has already been used up so it must be accurately accounted for in future allocations.
-   *
-   * @param size extra allocated memory that needs to be accounted for
-   */
-  public boolean forceGet(long size) {
-    if (get(size, this.applyFragmentLimit)) {
-      return true;
-    } else {
-      availableShared.addAndGet(-size);
-      if (parent != null) {
-        parent.forceGet(size);
-      }
-      return false;
-    }
-  }
-
-  public boolean get(long size, boolean applyFragmentLimitForChild) {
-    if (availablePrivate.get() < 1) {
-      // if there is no preallocated memory, we can operate normally.
-
-      // if there is a parent allocator, check it before allocating.
-      if (parent != null && !parent.get(size, this.applyFragmentLimit)) {
-        return false;
-      }
-
-      // If we need to allocate memory beyond the allowed Fragment Limit
-      if(applyFragmentLimitForChild && this.applyFragmentLimit && this.hasLimit && (getUsed()+size > this.limit)){
-        if (parent != null) {
-          parent.returnAllocation(size);
-        }
-        StackTraceElement[] ste = (new Throwable()).getStackTrace();
-        StringBuilder sb = new StringBuilder();
-        for (StackTraceElement s : ste) {
-          sb.append(s.toString());
-          sb.append("\n");
-        }
-        logger.warn("No more memory. Fragment limit ({} bytes) reached. Trying to allocate {} bytes. {} bytes " +
-          "already allocated.\n{}", limit, size, getUsed(), sb.toString());
-        return false;
-      }
-
-      // attempt to get shared memory, if fails, return false.
-      long outcome = availableShared.addAndGet(-size);
-//      assert outcome <= initShared;
-      if (outcome < 0) {
-        availableShared.addAndGet(size);
-        if (parent != null) {
-          parent.returnAllocation(size);
-        }
-        return false;
-      } else {
-//        if (DEBUG)
-//          logger.info("Get {}. a.s. {} a.p. {} hashcode {}", size, availableShared, availablePrivate, hashCode(), new Exception());
-        return true;
-      }
-
-    } else {
-      // if there is preallocated memory, use that first.
-      long unaccount = availablePrivate.addAndGet(-size);
-      if (unaccount >= 0) {
-//        if (DEBUG)
-//          logger.info("Get {}. a.s. {} a.p. {} hashcode {}", size, availableShared, availablePrivate, hashCode(), new Exception());
-        return true;
-      } else {
-
-        long additionalSpaceNeeded = -unaccount;
-        // if there is a parent allocator, check it before allocating.
-        if (parent != null && !parent.get(additionalSpaceNeeded, this.applyFragmentLimit)) {
-          // parent allocation failed, return space to private pool.
-          availablePrivate.getAndAdd(size);
-          return false;
-        }
-
-        // we got space from parent pool. lets make sure we have space locally available.
-        long account = availableShared.addAndGet(-additionalSpaceNeeded);
-        if (account >= 0) {
-          // we were succesful, move private back to zero (since we allocated using shared).
-          availablePrivate.addAndGet(additionalSpaceNeeded);
-//          if (DEBUG)
-//            logger.info("Get {}. a.s. {} a.p. {} hashcode {}", size, availableShared, availablePrivate, hashCode(), new Exception());
-          return true;
-        } else {
-          // we failed to get space from available shared. Return allocations to initial state.
-          availablePrivate.addAndGet(size);
-          availableShared.addAndGet(additionalSpaceNeeded);
-          if (parent != null) {
-            parent.returnAllocation(additionalSpaceNeeded);
-          }
-          return false;
-        }
-      }
-
-    }
-
-  }
-
-  /**
-   * Return the memory accounting to the allocation pool. Make sure to first maintain hold of the preallocated memory.
-   *
-   * @param size amount of memory returned
-   */
-  public void returnAllocation(long size) {
-    long privateSize = availablePrivate.get();
-    long privateChange = Math.min(size, initPrivate - privateSize);
-    long sharedChange = size - privateChange;
-    availablePrivate.addAndGet(privateChange);
-    availableShared.addAndGet(sharedChange);
-//    if (DEBUG)
-//      logger.info("Return allocation {}, a.s. {} a.p. {} hashcode {}", size, availableShared, availablePrivate, hashCode(), new Exception());
-    if (parent != null) {
-      parent.returnAllocation(sharedChange);
-    }
-  }
-
-  public void close() {
-    if (closed) {
-      logger.warn("Tried to close remainder, but it has already been closed", new Exception());
-      return;
-    }
-    if (availablePrivate.get() != initPrivate || availableShared.get() != initShared) {
-      IllegalStateException e = new IllegalStateException(
-          String
-              .format(ERROR, initPrivate, availablePrivate.get(), initPrivate - availablePrivate.get(), initShared, availableShared.get(), initShared - availableShared.get()));
-      if (errorOnLeak) {
-        throw e;
-      } else {
-        logger.warn("Memory leaked during query.", e);
-      }
-    }
-    if (parent != null) {
-      parent.returnAllocation(initPrivate);
-    }
-    closed = true;
-  }
-
-  static final String ERROR = "Failure while closing accountor.  Expected private and shared pools to be set to initial values.  However, one or more were not.  Stats are\n\tzone\tinit\tallocated\tdelta \n\tprivate\t%d\t%d\t%d \n\tshared\t%d\t%d\t%d.";
-}
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java
deleted file mode 100644
index a413e4a07a..0000000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java
+++ /dev/null
@@ -1,67 +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.drill.exec.memory;
-
-import org.apache.drill.common.config.DrillConfig;
-
-import com.google.common.annotations.VisibleForTesting;
-
-public class RootAllocatorFactory {
-  /**
-   * Constructor to prevent instantiation of this static utility class.
-   */
-  private RootAllocatorFactory() {}
-
-  /**
-   * Factory method.
-   *
-   * @param drillConfig the DrillConfig
-   * @return a new root allocator
-   */
-  public static BufferAllocator newRoot(final DrillConfig drillConfig) {
-/* TODO(DRILL-1942)
-    if (BaseAllocator.DEBUG) {
-      return new TopLevelAllocator(drillConfig);
-    }
-*/
-
-    return new TopLevelAllocator(drillConfig);
-  }
-
-  /**
-   * Factory method for testing, where a DrillConfig may not be available.
-   *
-   * @param allocationPolicy the allocation policy
-   * @param initAllocation initial allocation from parent
-   * @param maxReservation maximum amount of memory this can hand out
-   * @param flags one of BufferAllocator's F_* flags
-   * @return a new root allocator
-   */
-/* TODO(DRILL-1942)
-  @VisibleForTesting
-  public static BufferAllocator newRoot(
-      final AllocationPolicy allocationPolicy,
-      final long initAllocation, final long maxReservation, final int flags) {
-    if (BaseAllocator.DEBUG) {
-      return new RootAllocatorDebug(allocationPolicy, initAllocation, maxReservation, flags);
-    }
-
-    return new RootAllocator(allocationPolicy, initAllocation, maxReservation, flags);
-  }
-*/
-}
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java
deleted file mode 100644
index 1e96238fb6..0000000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java
+++ /dev/null
@@ -1,390 +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.drill.exec.memory;
-
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.DrillBuf;
-import io.netty.buffer.PooledByteBufAllocatorL;
-import io.netty.buffer.UnsafeDirectLittleEndian;
-
-import java.util.HashMap;
-import java.util.IdentityHashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.metrics.DrillMetrics;
-import org.apache.drill.exec.util.AssertionUtil;
-import org.apache.drill.exec.util.Pointer;
-
-public class TopLevelAllocator implements BufferAllocator {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TopLevelAllocator.class);
-  public static final String CHILD_BUFFER_INJECTION_SITE = "child.buffer";
-
-  private static final PooledByteBufAllocatorL ALLOCATOR = new PooledByteBufAllocatorL(DrillMetrics.getInstance());
-
-  public static final String TOP_LEVEL_MAX_ALLOC = "drill.memory.top.max";
-  public static final String ERROR_ON_MEMORY_LEAK = "drill.memory.debug.error_on_leak";
-
-  public static long MAXIMUM_DIRECT_MEMORY;
-
-  private static final boolean ENABLE_ACCOUNTING = AssertionUtil.isAssertionsEnabled();
-  private final Map<ChildAllocator, StackTraceElement[]> childrenMap;
-  private final PooledByteBufAllocatorL innerAllocator;
-  private final AccountorImpl acct;
-  private final boolean errorOnLeak;
-  private final DrillBuf empty;
-  private final DrillConfig config;
-
-  private TopLevelAllocator(DrillConfig config, long maximumAllocation, boolean errorOnLeak){
-    MAXIMUM_DIRECT_MEMORY = maximumAllocation;
-    innerAllocator = ALLOCATOR;
-    this.config=(config!=null) ? config : DrillConfig.create();
-    this.errorOnLeak = errorOnLeak;
-    this.acct = new AccountorImpl(config, errorOnLeak, null, null, maximumAllocation, 0, true);
-    this.empty = DrillBuf.getEmpty(this, acct);
-    this.childrenMap = ENABLE_ACCOUNTING ? new IdentityHashMap<ChildAllocator, StackTraceElement[]>() : null;
-  }
-
-  TopLevelAllocator(DrillConfig config) {
-    this(config, Math.min(DrillConfig.getMaxDirectMemory(), config.getLong(TOP_LEVEL_MAX_ALLOC)),
-        config.getBoolean(ERROR_ON_MEMORY_LEAK)
-        );
-  }
-
-  @Override
-  public boolean takeOwnership(DrillBuf buf) {
-    return buf.transferAccounting(acct);
-  }
-
-  @Override
-  public boolean takeOwnership(DrillBuf buf, Pointer<DrillBuf> out) {
-    DrillBuf b = new DrillBuf(this, acct, buf);
-    out.value = b;
-    return acct.transferIn(b, b.capacity());
-  }
-
-  public DrillBuf buffer(int min, int max) {
-    if (min == 0) {
-      return empty;
-    }
-    if(!acct.reserve(min)) {
-      throw new OutOfMemoryException(createErrorMsg(this, min));
-    }
-
-    try {
-      UnsafeDirectLittleEndian buffer = innerAllocator.directBuffer(min, max);
-      DrillBuf wrapped = new DrillBuf(this, acct, buffer);
-      acct.reserved(min, wrapped);
-      return wrapped;
-    } catch (OutOfMemoryError e) {
-      if ("Direct buffer memory".equals(e.getMessage())) {
-        acct.release(min);
-        throw new OutOfMemoryException(createErrorMsg(this, min), e);
-      } else {
-        throw e;
-      }
-    }
-  }
-
-  @Override
-  public DrillBuf buffer(int size) {
-    return buffer(size, size);
-  }
-
-  @Override
-  public long getAllocatedMemory() {
-    return acct.getAllocation();
-  }
-
-  @Override
-  public long getPeakMemoryAllocation() {
-    return acct.getPeakMemoryAllocation();
-  }
-
-  @Override
-  public ByteBufAllocator getUnderlyingAllocator() {
-    return innerAllocator;
-  }
-
-  @Override
-  public BufferAllocator getChildAllocator(LimitConsumer limitConsumer, long initialReservation,
-      long maximumReservation,
-      boolean applyFragmentLimit) {
-    if(!acct.reserve(initialReservation)){
-      logger.debug(String.format("You attempted to create a new child allocator with initial reservation %d but only %d bytes of memory were available.", initialReservation, acct.getCapacity() - acct.getAllocation()));
-      throw new OutOfMemoryException(
-          String
-              .format(
-                  "You attempted to create a new child allocator with initial reservation %d but only %d bytes of memory were available.",
-                  initialReservation, acct.getCapacity() - acct.getAllocation()));
-    }
-    logger.debug("New child allocator with initial reservation {}", initialReservation);
-    ChildAllocator allocator = new ChildAllocator(limitConsumer, acct, maximumReservation, initialReservation,
-        childrenMap, applyFragmentLimit);
-    if(ENABLE_ACCOUNTING){
-      childrenMap.put(allocator, Thread.currentThread().getStackTrace());
-    }
-
-    return allocator;
-  }
-
-  @Override
-  public void resetLimits() {
-    acct.resetFragmentLimits();
-  }
-
-  @Override
-  public void setLimit(long limit){
-    acct.setFragmentLimit(limit);
-  }
-
-  @Override
-  public long getLimit(){
-    return acct.getFragmentLimit();
-  }
-
-  @Override
-  public void close() {
-    if (ENABLE_ACCOUNTING) {
-      for (Entry<ChildAllocator, StackTraceElement[]> child : childrenMap.entrySet()) {
-        if (!child.getKey().isClosed()) {
-          StringBuilder sb = new StringBuilder();
-          StackTraceElement[] elements = child.getValue();
-          for (int i = 0; i < elements.length; i++) {
-            sb.append("\t\t");
-            sb.append(elements[i]);
-            sb.append("\n");
-          }
-          throw new IllegalStateException("Failure while trying to close allocator: Child level allocators not closed. Stack trace: \n" + sb);
-        }
-      }
-    }
-    acct.close();
-  }
-
-
-
-  @Override
-  public DrillBuf getEmpty() {
-    return empty;
-  }
-
-
-
-  private class ChildAllocator implements BufferAllocator {
-    private final DrillBuf empty;
-    private AccountorImpl childAcct;
-    private Map<ChildAllocator, StackTraceElement[]> children = new HashMap<>();
-    private boolean closed = false;
-    private LimitConsumer limitConsumer;
-    private Map<ChildAllocator, StackTraceElement[]> thisMap;
-    private boolean applyFragmentLimit;
-
-    public ChildAllocator(LimitConsumer limitConsumer,
-        AccountorImpl parentAccountor,
-                          long max,
-                          long pre,
-                          Map<ChildAllocator,
-                          StackTraceElement[]> map,
-        boolean applyFragmentLimit) {
-      assert max >= pre;
-      this.applyFragmentLimit = applyFragmentLimit;
-      this.limitConsumer = limitConsumer;
-      childAcct = new AccountorImpl(config, errorOnLeak, limitConsumer, parentAccountor, max, pre, applyFragmentLimit);
-      thisMap = map;
-      empty = DrillBuf.getEmpty(this, childAcct);
-    }
-
-    @Override
-    public boolean takeOwnership(DrillBuf buf) {
-      return buf.transferAccounting(childAcct);
-    }
-
-    @Override
-    public boolean takeOwnership(DrillBuf buf, Pointer<DrillBuf> out) {
-      DrillBuf b = new DrillBuf(this, acct, buf);
-      out.value = b;
-      return acct.transferIn(b, b.capacity());
-    }
-
-
-    @Override
-    public DrillBuf buffer(int size, int max) {
-      if (size == 0) {
-        return empty;
-      }
-      if(!childAcct.reserve(size)) {
-        throw new OutOfMemoryException(createErrorMsg(this, size));
-      }
-
-      try {
-        UnsafeDirectLittleEndian buffer = innerAllocator.directBuffer(size, max);
-        DrillBuf wrapped = new DrillBuf(this, childAcct, buffer);
-        childAcct.reserved(buffer.capacity(), wrapped);
-        return wrapped;
-      } catch (OutOfMemoryError e) {
-        if ("Direct buffer memory".equals(e.getMessage())) {
-          childAcct.release(size);
-          throw new OutOfMemoryException(createErrorMsg(this, size), e);
-        } else {
-          throw e;
-        }
-      }
-    }
-
-    public DrillBuf buffer(int size) {
-      return buffer(size, size);
-    }
-
-    @Override
-    public ByteBufAllocator getUnderlyingAllocator() {
-      return innerAllocator;
-    }
-
-    @Override
-    public BufferAllocator getChildAllocator(LimitConsumer limitConsumer, long initialReservation,
-        long maximumReservation,
-        boolean applyFragmentLimit) {
-      if (!childAcct.reserve(initialReservation)) {
-        throw new OutOfMemoryException(
-            String
-                .format(
-                    "You attempted to create a new child allocator with initial reservation %d but only %d bytes of memory were available.",
-                    initialReservation, childAcct.getAvailable()));
-      }
-      logger.debug("New child allocator with initial reservation {}", initialReservation);
-      ChildAllocator newChildAllocator = new ChildAllocator(limitConsumer, childAcct, maximumReservation,
-          initialReservation, null, applyFragmentLimit);
-      this.children.put(newChildAllocator, Thread.currentThread().getStackTrace());
-      return newChildAllocator;
-    }
-
-    public PreAllocator getNewPreAllocator() {
-      return new PreAlloc(this, this.childAcct);
-    }
-
-    @Override
-    public void resetLimits(){
-      childAcct.resetFragmentLimits();
-    }
-
-    @Override
-    public void setLimit(long limit){
-      childAcct.setFragmentLimit(limit);
-    }
-
-    @Override
-    public long getLimit(){
-      return childAcct.getFragmentLimit();
-    }
-
-    @Override
-    public void close() {
-      if (ENABLE_ACCOUNTING) {
-        if (thisMap != null) {
-          thisMap.remove(this);
-        }
-        for (ChildAllocator child : children.keySet()) {
-          if (!child.isClosed()) {
-            StringBuilder sb = new StringBuilder();
-            StackTraceElement[] elements = children.get(child);
-            for (int i = 1; i < elements.length; i++) {
-              sb.append("\t\t");
-              sb.append(elements[i]);
-              sb.append("\n");
-            }
-
-
-            IllegalStateException e = new IllegalStateException(String.format(
-                        "Failure while trying to close child allocator: Child level allocators not closed. Identifier: %s. Stack trace: \n %s",
-                        limitConsumer.getIdentifier(), sb.toString()));
-            if (errorOnLeak) {
-              throw e;
-            } else {
-              logger.warn("Memory leak.", e);
-            }
-          }
-        }
-      }
-      childAcct.close();
-      closed = true;
-    }
-
-    public boolean isClosed() {
-      return closed;
-    }
-
-    @Override
-    public long getAllocatedMemory() {
-      return childAcct.getAllocation();
-    }
-
-    @Override
-    public long getPeakMemoryAllocation() {
-      return childAcct.getPeakMemoryAllocation();
-    }
-
-    @Override
-    public DrillBuf getEmpty() {
-      return empty;
-    }
-
-
-  }
-
-  public PreAllocator getNewPreAllocator() {
-    return new PreAlloc(this, this.acct);
-  }
-
-  public class PreAlloc implements PreAllocator{
-    int bytes = 0;
-    final Accountor acct;
-    final BufferAllocator allocator;
-    private PreAlloc(BufferAllocator allocator, Accountor acct) {
-      this.acct = acct;
-      this.allocator = allocator;
-    }
-
-    /**
-     *
-     */
-    public boolean preAllocate(int bytes) {
-
-      if (!acct.reserve(bytes)) {
-        return false;
-      }
-      this.bytes += bytes;
-      return true;
-
-    }
-
-
-    public DrillBuf getAllocation() {
-      DrillBuf b = new DrillBuf(allocator, acct, innerAllocator.directBuffer(bytes, bytes));
-      acct.reserved(bytes, b);
-      return b;
-    }
-  }
-
-  private static String createErrorMsg(final BufferAllocator allocator, final int size) {
-    return String.format("Unable to allocate buffer of size %d due to memory limit. Current allocation: %d",
-      size, allocator.getAllocatedMemory());
-  }
-}
diff --git a/exec/memory/pom.xml b/exec/memory/pom.xml
index dac840c0b7..be603dbc30 100644
--- a/exec/memory/pom.xml
+++ b/exec/memory/pom.xml
@@ -30,6 +30,5 @@
 
   <modules>
     <module>base</module>
-    <module>impl</module>
   </modules>
 </project>
diff --git a/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java b/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
index 1cacf9caf5..bc7967729a 100644
--- a/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
+++ b/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
@@ -36,7 +36,7 @@
 
   public BasicClientWithConnection(RpcConfig rpcMapping, BufferAllocator alloc, EventLoopGroup eventLoopGroup, T handshakeType,
       Class<HANDSHAKE_RESPONSE> responseClass, Parser<HANDSHAKE_RESPONSE> handshakeParser, String connectionName) {
-    super(rpcMapping, alloc.getUnderlyingAllocator(), eventLoopGroup, handshakeType, responseClass, handshakeParser);
+    super(rpcMapping, alloc.getAsByteBufAllocator(), eventLoopGroup, handshakeType, responseClass, handshakeParser);
     this.alloc = alloc;
     this.connectionName = connectionName;
   }
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/ByteFunctionHelpers.java b/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/ByteFunctionHelpers.java
index 8a330b8ad9..bd06ba5c7c 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/ByteFunctionHelpers.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/ByteFunctionHelpers.java
@@ -29,8 +29,6 @@
 public class ByteFunctionHelpers {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ByteFunctionHelpers.class);
 
-  private static final boolean BOUNDS_CHECKING_ENABLED = BoundsChecking.BOUNDS_CHECKING_ENABLED;
-
   /**
    * Helper function to check for equality of bytes in two DrillBuffers
    *
@@ -43,7 +41,7 @@
    * @return 1 if left input is greater, -1 if left input is smaller, 0 otherwise
    */
   public static final int equal(final DrillBuf left, int lStart, int lEnd, final DrillBuf right, int rStart, int rEnd){
-    if(BOUNDS_CHECKING_ENABLED){
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
       left.checkBytes(lStart, lEnd);
       right.checkBytes(rStart, rEnd);
     }
@@ -97,7 +95,7 @@ private static final int memEqual(final long laddr, int lStart, int lEnd, final
    * @return 1 if left input is greater, -1 if left input is smaller, 0 otherwise
    */
   public static final int compare(final DrillBuf left, int lStart, int lEnd, final DrillBuf right, int rStart, int rEnd){
-    if(BOUNDS_CHECKING_ENABLED){
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
       left.checkBytes(lStart, lEnd);
       right.checkBytes(rStart, rEnd);
     }
@@ -152,7 +150,7 @@ private static final int memcmp(final long laddr, int lStart, int lEnd, final lo
    * @return 1 if left input is greater, -1 if left input is smaller, 0 otherwise
    */
   public static final int compare(final DrillBuf left, int lStart, int lEnd, final byte[] right, int rStart, final int rEnd) {
-    if(BOUNDS_CHECKING_ENABLED){
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
       left.checkBytes(lStart, lEnd);
     }
     return memcmp(left.memoryAddress(), lStart, lEnd, right, rStart, rEnd);
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java b/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
index 0922b22ab2..e8130ec0b8 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
@@ -17,7 +17,9 @@
  */
 package org.apache.drill.exec.util;
 
+import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
+import io.netty.buffer.UnpooledByteBufAllocator;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -142,7 +144,7 @@ public static StringBuilder toStringWithZeroes(long number, int desiredLength) {
         return str;
     }
 
-    public static BigDecimal getBigDecimalFromIntermediate(DrillBuf data, int startIndex, int nDecimalDigits, int scale) {
+  public static BigDecimal getBigDecimalFromIntermediate(ByteBuf data, int startIndex, int nDecimalDigits, int scale) {
 
         // In the intermediate representation we don't pad the scale with zeroes, so set truncate = false
         return getBigDecimalFromDrillBuf(data, startIndex, nDecimalDigits, scale, false);
@@ -172,7 +174,8 @@ public static BigDecimal getBigDecimalFromByteBuffer(ByteBuffer bytebuf, int sta
      * This function assumes that data is provided in a non-dense format
      * It works on both sparse and intermediate representations.
      */
-    public static BigDecimal getBigDecimalFromDrillBuf(DrillBuf data, int startIndex, int nDecimalDigits, int scale, boolean truncateScale) {
+  public static BigDecimal getBigDecimalFromDrillBuf(ByteBuf data, int startIndex, int nDecimalDigits, int scale,
+      boolean truncateScale) {
 
         // For sparse decimal type we have padded zeroes at the end, strip them while converting to BigDecimal.
         int actualDigits;
@@ -272,18 +275,24 @@ public static BigDecimal getBigDecimalFromDense(DrillBuf data, int startIndex, i
         if (sign == true) {
             intermediateBytes[0] = (byte) (intermediateBytes[0] | 0x80);
         }
-        DrillBuf intermediate = data.getAllocator().buffer(intermediateBytes.length);
+
+    final ByteBuf intermediate = UnpooledByteBufAllocator.DEFAULT.buffer(intermediateBytes.length);
+    try {
         intermediate.setBytes(0, intermediateBytes);
 
-        BigDecimal ret = getBigDecimalFromIntermediate(intermediate, 0, nDecimalDigits + 1, scale);
-        intermediate.release();
-        return ret;
+      BigDecimal ret = getBigDecimalFromIntermediate(intermediate, 0, nDecimalDigits + 1, scale);
+      return ret;
+    } finally {
+      intermediate.release();
+    }
+
     }
 
     /*
      * Function converts the BigDecimal and stores it in out internal sparse representation
      */
-    public static void getSparseFromBigDecimal(BigDecimal input, DrillBuf data, int startIndex, int scale, int precision, int nDecimalDigits) {
+  public static void getSparseFromBigDecimal(BigDecimal input, ByteBuf data, int startIndex, int scale, int precision,
+      int nDecimalDigits) {
 
         // Initialize the buffer
         for (int i = 0; i < nDecimalDigits; i++) {


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Implement Node-level Memory maximums for a query.
> -------------------------------------------------
>
>                 Key: DRILL-4142
>                 URL: https://issues.apache.org/jira/browse/DRILL-4142
>             Project: Apache Drill
>          Issue Type: New Feature
>          Components: Execution - Flow
>            Reporter: Jacques Nadeau
>            Priority: Major
>             Fix For: Future
>
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)