You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ja...@apache.org on 2015/12/22 16:06:24 UTC

[01/13] drill git commit: DRILL-4134: Clean up close behaviors.

Repository: drill
Updated Branches:
  refs/heads/master abdfc6a96 -> 809f4620d


DRILL-4134: Clean up close behaviors.


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

Branch: refs/heads/master
Commit: a466e4bd4d31520d69a490bbfd2eff1936e217f4
Parents: abdfc6a
Author: Chris Westin <cw...@yahoo.com>
Authored: Wed Nov 11 14:57:16 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Dec 21 23:52:22 2015 -0800

----------------------------------------------------------------------
 .../physical/impl/union/UnionAllRecordBatch.java   | 14 ++++++++++++--
 .../exec/record/RawFragmentBatchProvider.java      |  3 +++
 .../apache/drill/exec/record/WritableBatch.java    | 17 +++++++++--------
 .../exec/work/batch/AbstractDataCollector.java     | 16 +++++-----------
 4 files changed, 29 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a466e4bd/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
----------------------------------------------------------------------
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 51d74cc..0d47a57 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
@@ -84,6 +84,14 @@ public class UnionAllRecordBatch extends AbstractRecordBatch<UnionAll> {
   }
 
   @Override
+  public void kill(boolean sendUpstream) {
+    if(current != null) {
+      current.kill(sendUpstream);
+      current = null;
+    }
+  }
+
+  @Override
   protected void killIncoming(boolean sendUpstream) {
     unionAllInput.getLeftRecordBatch().kill(sendUpstream);
     unionAllInput.getRightRecordBatch().kill(sendUpstream);
@@ -270,8 +278,10 @@ public class UnionAllRecordBatch extends AbstractRecordBatch<UnionAll> {
 
   // This method is used by inner class to clear the current record batch
   private void clearCurrentRecordBatch() {
-    for(VectorWrapper<?> v: current) {
-      v.clear();
+    if (current != null) {
+      for(final VectorWrapper<?> v: current) {
+        v.clear();
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/a466e4bd/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
----------------------------------------------------------------------
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 14db502..7a0be5e 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 interface RawFragmentBatchProvider extends AutoCloseable{
 
   public RawFragmentBatch getNext() throws IOException, InterruptedException;
   public void kill(FragmentContext context);
+
+  @Override
+  public void close(); // Suppress the "throws Exception".
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/a466e4bd/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
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 324829a..488ba69 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
@@ -32,8 +32,8 @@ import com.google.common.collect.Lists;
 /**
  * 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;
@@ -170,15 +170,16 @@ public class WritableBatch {
     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);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/a466e4bd/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java
----------------------------------------------------------------------
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 3f2e3f2..1579c2b 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 abstract class AbstractDataCollector implements DataCollector{
 
     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);
     }
   }
 


[03/13] drill git commit: DRILL-4134: Add new allocator

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
new file mode 100644
index 0000000..bb8f0ee
--- /dev/null
+++ b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
@@ -0,0 +1,1654 @@
+/**
+ * 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.DrillBuf;
+import io.netty.buffer.PooledByteBufAllocatorL;
+import io.netty.buffer.UnsafeDirectLittleEndian;
+
+import java.util.Collection;
+import java.util.IdentityHashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.common.DrillAutoCloseables;
+import org.apache.drill.common.HistoricalLog;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
+import org.apache.drill.exec.util.AssertionUtil;
+import org.apache.drill.exec.util.Pointer;
+import org.slf4j.Logger;
+
+import com.google.common.base.Preconditions;
+
+// TODO(cwestin) javadoc
+// TODO(cwestin) add allocator implementation options tried
+public abstract class BaseAllocator implements BufferAllocator {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseAllocator.class);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(BaseAllocator.class);
+
+  private static final AtomicInteger idGenerator = new AtomicInteger(0);
+  private static final Object ALLOCATOR_LOCK = new Object();
+
+  public static final String CHILD_BUFFER_INJECTION_SITE = "child.buffer";
+
+  static final boolean DEBUG = AssertionUtil.isAssertionsEnabled()
+      || Boolean.getBoolean(ExecConstants.DEBUG_ALLOCATOR);
+  private static final PooledByteBufAllocatorL INNER_ALLOCATOR = PooledByteBufAllocatorL.DEFAULT;
+
+  private long allocated; // the amount of memory this allocator has given out to its clients (including children)
+  private long owned; // the amount of memory this allocator has obtained from its parent
+  private long peakAllocated; // the most memory this allocator has given out during its lifetime
+  private long bufferAllocation; // the amount of memory used just for directly allocated buffers, not children
+
+  private boolean isClosed = false; // the allocator has been closed
+
+  private final long maxAllocation; // the maximum amount of memory this allocator will give out
+  private final long chunkSize; // size of secondary chunks to allocate
+  private final AllocationPolicyAgent policyAgent;
+  private final BaseAllocator parentAllocator;
+  private final AllocatorOwner allocatorOwner;
+  protected final int id = idGenerator.incrementAndGet(); // unique ID assigned to each allocator
+  private final DrillBuf empty;
+  private final AllocationPolicy allocationPolicy;
+  private final InnerBufferLedger bufferLedger = new InnerBufferLedger();
+
+  // members used purely for debugging
+  private int getsFromParent;
+  private int putsToParent;
+  private final IdentityHashMap<UnsafeDirectLittleEndian, BufferLedger> allocatedBuffers;
+  private final IdentityHashMap<BaseAllocator, Object> childAllocators;
+  private final IdentityHashMap<Reservation, Object> reservations;
+  private long preallocSpace;
+
+  private final HistoricalLog historicalLog;
+
+  private static BaseAllocator getBaseAllocator(final BufferAllocator bufferAllocator) {
+    if (!(bufferAllocator instanceof BaseAllocator)) {
+      throw new IllegalArgumentException("expected a BaseAllocator instance, but got a "
+          + bufferAllocator.getClass().getName());
+    }
+    return (BaseAllocator) bufferAllocator;
+  }
+
+  // TODO move allocation policy and agent to outside of allocator
+  private static class PerFragmentAllocationPolicy implements AllocationPolicy {
+    static class Globals {
+      private long maxBufferAllocation = 0;
+      private final AtomicInteger limitingRoots = new AtomicInteger(0);
+    }
+
+    private final Globals globals = new Globals();
+
+    @Override
+    public AllocationPolicyAgent newAgent() {
+      return new PerFragmentAllocationPolicyAgent(globals);
+    }
+  }
+
+  /**
+   * AllocationPolicy that allows each fragment running on a drillbit to share an
+   * equal amount of direct memory, regardless of whether or not those fragments
+   * belong to the same query.
+   */
+  public static final AllocationPolicy POLICY_PER_FRAGMENT = new PerFragmentAllocationPolicy();
+
+  /**
+   * String name of {@link #POLICY_PER_FRAGMENT} policy.
+   */
+  public static final String POLICY_PER_FRAGMENT_NAME = "per-fragment";
+
+  private static class PerFragmentAllocationPolicyAgent implements AllocationPolicyAgent {
+    private final PerFragmentAllocationPolicy.Globals globals;
+    private boolean limitingRoot; // this is a limiting root; see F_LIMITING_ROOT
+
+    PerFragmentAllocationPolicyAgent(PerFragmentAllocationPolicy.Globals globals) {
+      this.globals = globals;
+    }
+
+    @Override
+    public void close() {
+      if (limitingRoot) {
+        // now there's one fewer active root
+        final int rootCount = globals.limitingRoots.decrementAndGet();
+
+        synchronized(globals) {
+          /*
+           * If the rootCount went to zero, we don't need to worry about setting the
+           * maxBufferAllocation, because there aren't any allocators to reference it;
+           * the next allocator to get created will set it appropriately.
+           */
+          if (rootCount != 0) {
+            globals.maxBufferAllocation = RootAllocator.getMaxDirect() / rootCount;
+          }
+        }
+      }
+    }
+
+    @Override
+    public void checkNewAllocator(BufferAllocator parentAllocator,
+        long initReservation, long maxAllocation, int flags) {
+/*
+      Preconditions.checkArgument(parentAllocator != null, "parent allocator can't be null");
+      Preconditions.checkArgument(parentAllocator instanceof BaseAllocator, "Parent allocator must be a BaseAllocator");
+*/
+
+//      final BaseAllocator baseAllocator = (BaseAllocator) parentAllocator;
+
+      // this is synchronized to protect maxBufferAllocation
+      synchronized(POLICY_PER_FRAGMENT) {
+        // initialize maxBufferAllocation the very first time we call this
+        if (globals.maxBufferAllocation == 0) {
+          globals.maxBufferAllocation = RootAllocator.getMaxDirect();
+        }
+
+        if (limitingRoot = ((flags & F_LIMITING_ROOT) != 0)) {
+          // figure out the new current per-allocator limit
+          globals.maxBufferAllocation = RootAllocator.getMaxDirect() / (globals.limitingRoots.get() + 1);
+        }
+
+        if (initReservation > 0) {
+          if (initReservation > globals.maxBufferAllocation) {
+            throw new OutOfMemoryRuntimeException(
+                String.format("can't fulfill initReservation request at this time "
+                    + "(initReservation = %d > maxBufferAllocation = %d)",
+                initReservation, globals.maxBufferAllocation));
+          }
+        }
+      }
+    }
+
+    @Override
+    public long getMemoryLimit(BufferAllocator bufferAllocator) {
+      synchronized(POLICY_PER_FRAGMENT) {
+        return globals.maxBufferAllocation;
+      }
+    }
+
+    @Override
+    public void initializeAllocator(final BufferAllocator bufferAllocator) {
+      final BaseAllocator baseAllocator = getBaseAllocator(bufferAllocator);
+
+      if (limitingRoot) {
+        globals.limitingRoots.incrementAndGet();
+      }
+    }
+
+    @Override
+    public boolean shouldReleaseToParent(final BufferAllocator bufferAllocator) {
+      final BaseAllocator baseAllocator = getBaseAllocator(bufferAllocator);
+      return (baseAllocator.owned + baseAllocator.chunkSize > globals.maxBufferAllocation);
+    }
+  }
+
+  private static class LocalMaxAllocationPolicy implements AllocationPolicy {
+    // this agent is stateless, so we can always use the same one
+    private static final AllocationPolicyAgent AGENT = new LocalMaxAllocationPolicyAgent();
+
+    @Override
+    public AllocationPolicyAgent newAgent() {
+      return AGENT;
+    }
+  }
+
+  /**
+   * AllocationPolicy that imposes no limits on how much direct memory fragments
+   * may allocate. LOCAL_MAX refers to the only limit that is enforced, which is
+   * the maxAllocation specified at allocators' creation.
+   *
+   * <p>This policy ignores the value of {@link BufferAllocator#F_LIMITING_ROOT}.</p>
+   */
+  public static final AllocationPolicy POLICY_LOCAL_MAX = new LocalMaxAllocationPolicy();
+
+  /**
+   * String name of {@link #POLICY_LOCAL_MAX} allocation policy.
+   */
+  public static final String POLICY_LOCAL_MAX_NAME = "local-max";
+
+  private static class LocalMaxAllocationPolicyAgent implements AllocationPolicyAgent {
+    @Override
+    public void close() throws Exception {
+    }
+
+    @Override
+    public void checkNewAllocator(BufferAllocator parentAllocator,
+        long initReservation, long maxAllocation, int flags) {
+    }
+
+    @Override
+    public long getMemoryLimit(BufferAllocator bufferAllocator) {
+      final BaseAllocator baseAllocator = (BaseAllocator) bufferAllocator;
+      return baseAllocator.maxAllocation;
+    }
+
+    @Override
+    public void initializeAllocator(BufferAllocator bufferAllocator) {
+    }
+
+    @Override
+    public boolean shouldReleaseToParent(BufferAllocator bufferAllocator) {
+      // since there are no shared limits, release space whenever we can
+      return true;
+    }
+  }
+
+  // TODO(DRILL-2698) POLICY_PER_QUERY
+
+  protected BaseAllocator(
+      final BaseAllocator parentAllocator,
+      final AllocatorOwner allocatorOwner,
+      final AllocationPolicy allocationPolicy,
+      final long initReservation,
+      final long maxAllocation,
+      final int flags) throws OutOfMemoryRuntimeException {
+    Preconditions.checkArgument(allocatorOwner != null, "allocatorOwner must be non-null");
+    Preconditions.checkArgument(initReservation >= 0,
+        "the initial reservation size must be non-negative");
+    Preconditions.checkArgument(maxAllocation >= 0,
+        "the maximum allocation limit mjst be non-negative");
+    Preconditions.checkArgument(initReservation <= maxAllocation,
+        "the initial reservation size must be <= the maximum allocation");
+
+    if (initReservation > 0) {
+      if (parentAllocator == null) {
+        throw new IllegalStateException(
+            "can't reserve memory without a parent allocator");
+      }
+    }
+
+    // check to see if we can create this new allocator (the check throws if it's not ok)
+    final AllocationPolicyAgent policyAgent = allocationPolicy.newAgent();
+    policyAgent.checkNewAllocator(parentAllocator, initReservation, maxAllocation, flags);
+
+    if ((initReservation > 0) && !parentAllocator.reserve(this, initReservation, 0)) {
+      throw new OutOfMemoryRuntimeException(
+          "can't fulfill initial reservation of size (unavailable from parent)" + initReservation);
+    }
+
+    /*
+     * Figure out how much more to ask for from our parent if we're out.
+     * Secondary allocations from the parent will be done in integral multiples of
+     * chunkSize. We also use this to determine when to hand back memory to our
+     * parent in order to create hysteresis. This reduces contention on the parent's
+     * lock
+     */
+    if (initReservation == 0) {
+      chunkSize = maxAllocation / 8;
+    } else {
+      chunkSize = initReservation;
+    }
+
+    this.parentAllocator = parentAllocator;
+    this.allocatorOwner = allocatorOwner;
+    this.allocationPolicy = allocationPolicy;
+    this.policyAgent = policyAgent;
+    this.maxAllocation = maxAllocation;
+
+    // the root allocator owns all of its memory; anything else just owns it's initial reservation
+    owned = parentAllocator == null ? maxAllocation : initReservation;
+    empty = DrillBuf.getEmpty(new EmptyLedger(), this);
+
+    if (DEBUG) {
+      allocatedBuffers = new IdentityHashMap<>();
+      childAllocators = new IdentityHashMap<>();
+      reservations = new IdentityHashMap<>();
+      historicalLog = new HistoricalLog(4, "allocator[%d]", id);
+
+      historicalLog.recordEvent("created by \"%s\", owned = %d", allocatorOwner.toString(), owned);
+    } else {
+      allocatedBuffers = null;
+      childAllocators = null;
+      reservations = null;
+      historicalLog = null;
+    }
+
+    // now that we're not in danger of throwing an exception, we can take this step
+    policyAgent.initializeAllocator(this);
+  }
+
+  /**
+   * Allocators without a parent must provide an implementation of this so
+   * that they may reserve additional space even though they don't have a
+   * parent they can fall back to.
+   *
+   * <p>Prior to calling this, BaseAllocator has verified that this won't violate
+   * the maxAllocation for this allocator.</p>
+   *
+   * @param nBytes the amount of space to reserve
+   * @param ignoreMax ignore the maximum allocation limit;
+   *   see {@link ChildLedger#reserve(long, boolean)}.
+   * @return true if the request can be met, false otherwise
+   */
+  protected boolean canIncreaseOwned(final long nBytes, final int flags) {
+    if (parentAllocator == null) {
+      return false;
+    }
+
+    return parentAllocator.reserve(this, nBytes, flags);
+  }
+
+  /**
+   * Reserve space for the child allocator from this allocator.
+   *
+   * @param childAllocator child allocator making the request, or null
+   *  if this is not for a child
+   * @param nBytes how much to reserve
+   * @param flags one or more of RESERVE_F_* flags or'ed together
+   * @return true if the reservation can be satisfied, false otherwise
+   */
+  private static final int RESERVE_F_IGNORE_MAX = 0x0001;
+  private boolean reserve(final BaseAllocator childAllocator,
+      final long nBytes, final int flags) {
+    Preconditions.checkArgument(nBytes >= 0,
+        "the number of bytes to reserve must be non-negative");
+
+    // we can always fulfill an empty request
+    if (nBytes == 0) {
+      return true;
+    }
+
+    final boolean ignoreMax = (flags & RESERVE_F_IGNORE_MAX) != 0;
+
+    synchronized(ALLOCATOR_LOCK) {
+      if (isClosed) {
+        throw new AllocatorClosedException(String.format("Attempt to use closed allocator[%d]", id));
+      }
+
+      final long ownAtLeast = allocated + nBytes;
+      // Are we allowed to hand out this much?
+      if (!ignoreMax && (ownAtLeast > maxAllocation)) {
+        return false;
+      }
+
+      // do we need more from our parent first?
+      if (ownAtLeast > owned) {
+        /*
+         * Allocate space in integral multiples of chunkSize, as long as it doesn't exceed
+         * the maxAllocation.
+         */
+        final long needAdditional = ownAtLeast - owned;
+        final long getInChunks = (1 + ((needAdditional - 1) / chunkSize)) * chunkSize;
+        final long getFromParent;
+        if (getInChunks + owned <= maxAllocation) {
+          getFromParent = getInChunks;
+        } else {
+          getFromParent = needAdditional;
+        }
+        if (!canIncreaseOwned(getFromParent, flags)) {
+          return false;
+        }
+        owned += getFromParent;
+
+        if (DEBUG) {
+          ++getsFromParent;
+          historicalLog.recordEvent("increased owned by %d, now owned == %d", needAdditional, owned);
+        }
+      }
+
+      if (DEBUG) {
+        if (owned < ownAtLeast) {
+          throw new IllegalStateException("don't own enough memory to satisfy request");
+        }
+        if (allocated > owned) {
+          throw new IllegalStateException(
+              String.format("more memory allocated (%d) than owned (%d)", allocated, owned));
+        }
+
+        historicalLog.recordEvent("allocator[%d] allocated increased by nBytes == %d to %d",
+            id, nBytes, allocated + nBytes);
+      }
+
+      allocated += nBytes;
+
+      if (allocated > peakAllocated) {
+        peakAllocated = allocated;
+      }
+
+      return true;
+    }
+  }
+
+  private void releaseBytes(final long nBytes) {
+    Preconditions.checkArgument(nBytes >= 0,
+        "the number of bytes being released must be non-negative");
+
+    synchronized(ALLOCATOR_LOCK) {
+      allocated -= nBytes;
+
+      if (DEBUG) {
+        historicalLog.recordEvent("allocator[%d] released nBytes == %d, allocated now %d",
+            id, nBytes, allocated);
+      }
+
+      /*
+       * Return space to our parent if our allocation is over the currently allowed amount.
+       */
+      final boolean releaseToParent = (parentAllocator != null)
+          && (owned > maxAllocation) && policyAgent.shouldReleaseToParent(this);
+      if (releaseToParent) {
+        final long canFree = owned - maxAllocation;
+        parentAllocator.releaseBytes(canFree);
+        owned -= canFree;
+
+        if (DEBUG) {
+          ++putsToParent;
+          historicalLog.recordEvent("returned %d to parent, now owned == %d", canFree, owned);
+        }
+      }
+    }
+  }
+
+  private void releaseBuffer(final DrillBuf drillBuf) {
+    Preconditions.checkArgument(drillBuf != null,
+        "the DrillBuf being released can't be null");
+
+    final ByteBuf byteBuf = drillBuf.unwrap();
+    final int udleMaxCapacity = byteBuf.maxCapacity();
+
+    synchronized(ALLOCATOR_LOCK) {
+      bufferAllocation -= udleMaxCapacity;
+      releaseBytes(udleMaxCapacity);
+
+      if (DEBUG) {
+        // make sure the buffer came from this allocator
+        final Object object = allocatedBuffers.remove(byteBuf);
+        if (object == null) {
+          historicalLog.logHistory(logger);
+          drillBuf.logHistory(logger);
+          throw new IllegalStateException("Released buffer did not belong to this allocator");
+        }
+      }
+    }
+  }
+
+  private void childClosed(final BaseAllocator childAllocator) {
+    Preconditions.checkArgument(childAllocator != null, "child allocator can't be null");
+
+    if (DEBUG) {
+      synchronized(ALLOCATOR_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");
+        }
+
+        try {
+          verifyAllocator();
+        } catch(Exception e) {
+          /*
+           * If there was a problem with verification, the history of the closed
+           * child may also be useful.
+           */
+          logger.debug("allocator[" + id + "]: exception while closing the following child");
+          childAllocator.historicalLog.logHistory(logger);
+
+          // Continue with the verification exception throwing.
+          throw e;
+        }
+      }
+    }
+  }
+
+  /**
+   * TODO(DRILL-2740) We use this to bypass the regular accounting for the
+   * empty DrillBuf, because it is treated specially at this time. Once that
+   * is remedied, this should be able to go away.
+   */
+  private class EmptyLedger implements BufferLedger {
+    @Override
+    public PooledByteBufAllocatorL getUnderlyingAllocator() {
+      return INNER_ALLOCATOR;
+    }
+
+    @Override
+    public void release(final DrillBuf drillBuf) {
+      if (DEBUG) {
+        if (drillBuf != empty) {
+          throw new IllegalStateException("The empty buffer's ledger is being used to release something else");
+        }
+      }
+    }
+
+    @Override
+    public BufferLedger shareWith(Pointer<DrillBuf> pDrillBuf,
+        BufferLedger otherLedger, BufferAllocator otherAllocator, DrillBuf drillBuf,
+        int index, int length, int drillBufFlags) {
+      // As a special case, we allow sharing with the same allocator so that slicing works.
+      if (otherAllocator != BaseAllocator.this) {
+        throw new UnsupportedOperationException("The empty buffer can't be shared");
+      }
+
+      pDrillBuf.value = drillBuf;
+      return otherLedger;
+    }
+
+    @Override
+    public boolean transferTo(BufferAllocator newAlloc,
+        Pointer<BufferLedger> pNewLedger, DrillBuf drillBuf) {
+      throw new UnsupportedOperationException("The empty buffer's ownership can't be changed");
+    }
+  }
+
+  private class InnerBufferLedger implements BufferLedger {
+    @Override
+    public PooledByteBufAllocatorL getUnderlyingAllocator() {
+      return INNER_ALLOCATOR;
+    }
+
+    @Override
+    public void release(final DrillBuf drillBuf) {
+      releaseBuffer(drillBuf);
+    }
+
+    @Override
+    public BufferLedger shareWith(final Pointer<DrillBuf> pDrillBuf,
+        final BufferLedger otherLedger, final BufferAllocator otherAllocator,
+        final DrillBuf drillBuf, final int index, final int length, final int drillBufFlags) {
+      final BaseAllocator baseAllocator = (BaseAllocator) otherAllocator;
+      synchronized(ALLOCATOR_LOCK) {
+        if (baseAllocator.isClosed) {
+          throw new AllocatorClosedException(
+              String.format("Attempt to use closed allocator[%d]", baseAllocator.id));
+        }
+
+        /*
+         * If this is called, then the buffer isn't yet shared, and should
+         * become so.
+         */
+        final SharedBufferLedger sharedLedger = new SharedBufferLedger(drillBuf, BaseAllocator.this);
+
+        // Create the new wrapping DrillBuf.
+        final DrillBuf newBuf =
+            new DrillBuf(sharedLedger, otherAllocator, drillBuf, index, length, drillBufFlags);
+        sharedLedger.addMapping(newBuf, baseAllocator);
+
+        if (DEBUG) {
+          final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap();
+          historicalLog.recordEvent("InnerBufferLedger(allocator[%d]).shareWith(..., "
+              + "otherAllocator[%d], drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}, ...)",
+              BaseAllocator.this.id, baseAllocator.id, drillBuf.getId(),
+              System.identityHashCode(udle));
+
+          final BaseAllocator drillBufAllocator = (BaseAllocator) drillBuf.getAllocator();
+          if (BaseAllocator.this != drillBufAllocator) {
+            historicalLog.logHistory(logger);
+            drillBuf.logHistory(logger);
+            throw new IllegalStateException(String.format(
+                "DrillBuf's allocator([%d]) doesn't match this(allocator[%d])",
+                drillBufAllocator.id, BaseAllocator.this.id));
+          }
+
+          // Replace the ledger for the existing buffer.
+          final BufferLedger thisLedger = allocatedBuffers.put(udle, sharedLedger);
+
+          // If we throw any of these exceptions, we need to clean up newBuf.
+          if (thisLedger == null) {
+            newBuf.release();
+            historicalLog.logHistory(logger);
+            drillBuf.logHistory(logger);
+            throw new IllegalStateException("Buffer to be shared is unknown to the source allocator");
+          }
+          if (thisLedger != this) {
+            newBuf.release();
+            historicalLog.logHistory(logger);
+            drillBuf.logHistory(logger);
+            throw new IllegalStateException("Buffer's ledger was not the one it should be");
+          }
+        }
+
+        pDrillBuf.value = newBuf;
+        return sharedLedger;
+      }
+    }
+
+    @Override
+    public boolean transferTo(final BufferAllocator newAlloc,
+        final Pointer<BufferLedger> pNewLedger, final DrillBuf drillBuf) {
+      Preconditions.checkArgument(newAlloc != null, "New allocator cannot be null");
+      Preconditions.checkArgument(newAlloc != BaseAllocator.this,
+          "New allocator is same as current");
+      Preconditions.checkArgument(newAlloc instanceof BaseAllocator,
+          "New allocator isn't a BaseAllocator");
+      Preconditions.checkArgument(pNewLedger.value != null, "Candidate new ledger can't be null");
+      Preconditions.checkArgument(drillBuf != null, "DrillBuf can't be null");
+
+      final BaseAllocator newAllocator = (BaseAllocator) newAlloc;
+      synchronized(ALLOCATOR_LOCK) {
+        if (newAllocator.isClosed) {
+          throw new AllocatorClosedException(
+              String.format("Attempt to use closed allocator[%d]", newAllocator.id));
+        }
+
+        return BaseAllocator.transferTo(newAllocator, pNewLedger.value, drillBuf);
+      }
+    }
+  }
+
+  /**
+   * Transfer ownership of a buffer from one allocator to another.
+   *
+   * <p>Assumes the allocatorLock is held.</p>
+   *
+   * @param newAllocator the new allocator
+   * @param newLedger the new ledger to use (which could be shared)
+   * @param drillBuf the buffer
+   * @return true if the buffer's transfer didn't exceed the new owner's maximum
+   *   allocation limit
+   */
+  private static boolean transferTo(final BaseAllocator newAllocator,
+      final BufferLedger newLedger, final DrillBuf drillBuf) {
+    final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap();
+    final int udleMaxCapacity = udle.maxCapacity();
+
+    synchronized(ALLOCATOR_LOCK) {
+      // Account for the space and track the buffer.
+      newAllocator.reserveForBuf(udleMaxCapacity);
+
+      if (DEBUG) {
+        final Object object = newAllocator.allocatedBuffers.put(udle, newLedger);
+        if (object != null) {
+          newAllocator.historicalLog.logHistory(logger);
+          drillBuf.logHistory(logger);
+          throw new IllegalStateException("Buffer unexpectedly found in new allocator");
+        }
+      }
+
+      // Remove from the old allocator.
+      final BaseAllocator oldAllocator = (BaseAllocator) drillBuf.getAllocator();
+      oldAllocator.releaseBuffer(drillBuf);
+
+      if (DEBUG) {
+        final Object object = oldAllocator.allocatedBuffers.get(udle);
+        if (object != null) {
+          oldAllocator.historicalLog.logHistory(logger);
+          drillBuf.logHistory(logger);
+          throw new IllegalStateException("Buffer was not removed from old allocator");
+        }
+
+        oldAllocator.historicalLog.recordEvent("BaseAllocator.transferTo(otherAllocator[%d], ..., "
+            + "drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}) oldAllocator[%d]",
+            newAllocator.id, drillBuf.getId(), System.identityHashCode(drillBuf.unwrap()),
+            oldAllocator.id);
+        newAllocator.historicalLog.recordEvent("BaseAllocator.transferTo(otherAllocator[%d], ..., "
+            + "drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}) oldAllocator[%d]",
+            newAllocator.id, drillBuf.getId(), System.identityHashCode(drillBuf.unwrap()),
+            oldAllocator.id);
+      }
+
+      return newAllocator.allocated < newAllocator.maxAllocation;
+    }
+  }
+
+  private static class SharedBufferLedger implements BufferLedger {
+    private volatile BaseAllocator owningAllocator;
+    private final IdentityHashMap<DrillBuf, BaseAllocator> bufferMap = new IdentityHashMap<>();
+
+    private final HistoricalLog historicalLog;
+
+    public SharedBufferLedger(final DrillBuf drillBuf, final BaseAllocator baseAllocator) {
+      if (DEBUG) {
+        historicalLog = new HistoricalLog(4,
+            "SharedBufferLedger for DrillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}",
+            drillBuf.getId(), System.identityHashCode(drillBuf.unwrap()));
+      } else {
+        historicalLog = null;
+      }
+      addMapping(drillBuf, baseAllocator);
+      owningAllocator = baseAllocator;
+
+      if (DEBUG) {
+        checkBufferMap();
+      }
+    }
+
+    private synchronized void addMapping(final DrillBuf drillBuf, final BaseAllocator baseAllocator) {
+      bufferMap.put(drillBuf, baseAllocator);
+
+      if (DEBUG) {
+        historicalLog.recordEvent("addMapping(DrillBuf[%d], allocator[%d])", drillBuf.getId(), baseAllocator.id);
+      }
+    }
+
+    private synchronized void logBufferHistories(final Logger logger) {
+      final Set<Map.Entry<DrillBuf, BaseAllocator>> bufsToCheck = bufferMap.entrySet();
+      for(final Map.Entry<DrillBuf, BaseAllocator> mapEntry : bufsToCheck) {
+        final DrillBuf drillBuf = mapEntry.getKey();
+        drillBuf.logHistory(logger);
+      }
+    }
+
+    private synchronized void checkBufferMap() {
+      boolean foundOwner = false;
+      final Set<Map.Entry<DrillBuf, BaseAllocator>> bufsToCheck = bufferMap.entrySet();
+      for(final Map.Entry<DrillBuf, BaseAllocator> mapEntry : bufsToCheck) {
+        final DrillBuf drillBuf = mapEntry.getKey();
+        final BaseAllocator bufferAllocator = mapEntry.getValue();
+
+        final Object object = bufferAllocator.allocatedBuffers.get(drillBuf.unwrap());
+        if (bufferAllocator == owningAllocator) {
+          foundOwner = true;
+          if (object == null) {
+            historicalLog.logHistory(logger);
+            logBufferHistories(logger);
+            throw new IllegalStateException(
+                String.format("Shared buffer DrillBuf[%d] not found in owning allocator[%d]",
+                    drillBuf.getId(), bufferAllocator.id));
+          }
+        } else {
+          if (object != null) {
+            historicalLog.logHistory(logger);
+            logBufferHistories(logger);
+            throw new IllegalStateException(
+                String.format("Shared buffer DrillBuf[%d] not found in non-owning allocator[%d]",
+                    drillBuf.getId(), bufferAllocator.id));
+
+          }
+        }
+      }
+
+      if (!foundOwner && !bufferMap.isEmpty()) {
+        historicalLog.logHistory(logger);
+        logBufferHistories(logger);
+        owningAllocator.historicalLog.logHistory(logger);
+        throw new IllegalStateException(
+            String.format("Did not find owning allocator[%d] in bufferMap", owningAllocator.id));
+      }
+    }
+
+    @Override
+    public PooledByteBufAllocatorL getUnderlyingAllocator() {
+      return INNER_ALLOCATOR;
+    }
+
+    @Override
+    public void release(final DrillBuf drillBuf) {
+      Preconditions.checkArgument(drillBuf != null, "drillBuf can't be null");
+
+      /*
+       * This is the only method on the shared ledger that can be entered without
+       * having first come through an outside method on BaseAllocator (such
+       * as takeOwnership() or shareOwnership()), all of which get the allocatorLock.
+       * Operations in the below require the allocatorLock. We also need to synchronize
+       * on this object to protect the bufferMap. In order to avoid a deadlock with other
+       * methods, we have to get the allocatorLock first, as will be done in all the
+       * other cases.
+       */
+      synchronized(ALLOCATOR_LOCK) {
+        synchronized(this) {
+          final Object bufferObject = bufferMap.remove(drillBuf);
+          if (DEBUG) {
+            if (bufferObject == null) {
+              historicalLog.logHistory(logger, String.format("release(DrillBuf[%d])", drillBuf.getId()));
+              drillBuf.logHistory(logger);
+              throw new IllegalStateException("Buffer not found in SharedBufferLedger's buffer map");
+            }
+          }
+
+          /*
+           * If there are other buffers in the bufferMap that share this buffer's fate,
+           * remove them, since they are also now invalid. As we examine buffers, take note
+           * of any others that don't share this one's fate, but which belong to the same
+           * allocator; if we find any such, then we can avoid transferring ownership at this
+           * time.
+           */
+          final BaseAllocator bufferAllocator = (BaseAllocator) drillBuf.getAllocator();
+          final List<DrillBuf> sameAllocatorSurvivors = new LinkedList<>();
+          if (!bufferMap.isEmpty()) {
+            /*
+             * We're going to be modifying bufferMap (if we find any other related buffers);
+             * in order to avoid getting a ConcurrentModificationException, we can't do it
+             * on the same iteration we use to examine the buffers, so we use an intermediate
+             * list to figure out which ones we have to remove.
+             */
+            final Set<Map.Entry<DrillBuf, BaseAllocator>> bufsToCheck = bufferMap.entrySet();
+            final List<DrillBuf> sharedFateBuffers = new LinkedList<>();
+            for(final Map.Entry<DrillBuf, BaseAllocator> mapEntry : bufsToCheck) {
+              final DrillBuf otherBuf = mapEntry.getKey();
+              if (otherBuf.hasSharedFate(drillBuf)) {
+                sharedFateBuffers.add(otherBuf);
+              } else {
+                final BaseAllocator otherAllocator = mapEntry.getValue();
+                if (otherAllocator == bufferAllocator) {
+                  sameAllocatorSurvivors.add(otherBuf);
+                }
+              }
+            }
+
+            final int nSharedFate = sharedFateBuffers.size();
+            if (nSharedFate > 0) {
+              final int[] sharedIds = new int[nSharedFate];
+              int index = 0;
+              for(final DrillBuf bufToRemove : sharedFateBuffers) {
+                sharedIds[index++] = bufToRemove.getId();
+                bufferMap.remove(bufToRemove);
+              }
+
+              if (DEBUG) {
+                final StringBuilder sb = new StringBuilder();
+                for(final DrillBuf bufToRemove : sharedFateBuffers) {
+                  sb.append(String.format("DrillBuf[%d], ", bufToRemove.getId()));
+                }
+                sb.setLength(sb.length() - 2); // Chop off the trailing comma and space.
+                historicalLog.recordEvent("removed shared fate buffers " + sb.toString());
+              }
+            }
+          }
+
+          if (sameAllocatorSurvivors.isEmpty()) {
+            /*
+             * If that was the owning allocator, then we need to transfer ownership to
+             * another allocator (any one) that is part of the sharing set.
+             *
+             * When we release the buffer back to the allocator, release the root buffer,
+             */
+            if (bufferAllocator == owningAllocator) {
+              if (bufferMap.isEmpty()) {
+                /*
+                 * There are no other allocators available to transfer to, so
+                 * release the space to the owner.
+                 */
+                bufferAllocator.releaseBuffer(drillBuf);
+              } else {
+                // Pick another allocator, and transfer ownership to that.
+                final Collection<BaseAllocator> allocators = bufferMap.values();
+                final Iterator<BaseAllocator> allocatorIter = allocators.iterator();
+                if (!allocatorIter.hasNext()) {
+                  historicalLog.logHistory(logger);
+                  throw new IllegalStateException("Shared ledger buffer map is non-empty, but not iterable");
+                }
+                final BaseAllocator nextAllocator = allocatorIter.next();
+                BaseAllocator.transferTo(nextAllocator, this, drillBuf);
+                owningAllocator = nextAllocator;
+
+                if (DEBUG) {
+                  if (owningAllocator == bufferAllocator) {
+                    historicalLog.logHistory(logger);
+                    owningAllocator.historicalLog.logHistory(logger);
+                    drillBuf.logHistory(logger);
+                    throw new IllegalStateException("Shared buffer release transfer to same owner");
+                  }
+
+                  final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap();
+                  final Object oldObject = bufferAllocator.allocatedBuffers.get(udle);
+                  if (oldObject != null) {
+                    historicalLog.logHistory(logger);
+                    bufferAllocator.historicalLog.logHistory(logger);
+                    owningAllocator.historicalLog.logHistory(logger);
+                    drillBuf.logHistory(logger);
+
+                    throw new IllegalStateException("Inconsistent shared buffer release state (old owner)");
+                  }
+
+                  final Object newObject = owningAllocator.allocatedBuffers.get(udle);
+                  if (newObject == null) {
+                    historicalLog.logHistory(logger);
+                    bufferAllocator.historicalLog.logHistory(logger);
+                    owningAllocator.historicalLog.logHistory(logger);
+                    drillBuf.logHistory(logger);
+
+                    throw new IllegalStateException("Inconsistent shared buffer release state (new owner)");
+                  }
+                }
+              }
+            }
+          }
+        }
+      }
+
+      if (DEBUG) {
+        checkBufferMap();
+      }
+    }
+
+    @Override
+    public BufferLedger shareWith(final Pointer<DrillBuf> pDrillBuf,
+        final BufferLedger otherLedger, final BufferAllocator otherAllocator,
+        final DrillBuf drillBuf, final int index, final int length, final int drillBufFlags) {
+      final BaseAllocator baseAllocator = (BaseAllocator) otherAllocator;
+      if (baseAllocator.isClosed) {
+        throw new AllocatorClosedException(
+            String.format("Attempt to use closed allocator[%d]", baseAllocator.id));
+      }
+
+      synchronized(ALLOCATOR_LOCK) {
+        /*
+         * This buffer is already shared, but we want to add more sharers.
+         *
+         * Create the new wrapper.
+         */
+        final DrillBuf newBuf = new DrillBuf(this, otherAllocator, drillBuf, index, length, drillBufFlags);
+        if (DEBUG) {
+          final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap();
+          baseAllocator.historicalLog.recordEvent("SharedBufferLedger.shareWith(..., otherAllocator[%d], "
+              + "drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}, ...)",
+              baseAllocator.id, drillBuf.getId(), System.identityHashCode(udle));
+
+          // Make sure the current ownership is still correct.
+          final Object object = owningAllocator.allocatedBuffers.get(udle); // This may not be protectable w/o ALLOCATOR_LOCK.
+          if (object == null) {
+            newBuf.release();
+            historicalLog.logHistory(logger);
+            owningAllocator.historicalLog.logHistory(logger);
+            drillBuf.logHistory(logger);
+            throw new IllegalStateException("Buffer not found in owning allocator");
+          }
+        }
+
+        addMapping(newBuf, baseAllocator);
+        pDrillBuf.value = newBuf;
+
+        if (DEBUG) {
+          checkBufferMap();
+        }
+
+        return this;
+      }
+    }
+
+    @Override
+    public boolean transferTo(final BufferAllocator newAlloc,
+        final Pointer<BufferLedger> pNewLedger, final DrillBuf drillBuf) {
+      Preconditions.checkArgument(newAlloc != null, "New allocator cannot be null");
+      Preconditions.checkArgument(newAlloc instanceof BaseAllocator,
+          "New allocator isn't a BaseAllocator");
+      Preconditions.checkArgument(pNewLedger.value != null, "Candidate new ledger can't be null");
+      Preconditions.checkArgument(drillBuf != null, "DrillBuf can't be null");
+
+      final BaseAllocator newAllocator = (BaseAllocator) newAlloc;
+      if (newAllocator.isClosed) {
+        throw new AllocatorClosedException(String.format(
+            "Attempt to use closed allocator[%d]", newAllocator.id));
+      }
+
+      // This doesn't need the ALLOCATOR_LOCK, because it will already be held.
+      synchronized(this) {
+        try {
+          // Modify the buffer mapping to reflect the virtual transfer.
+          final BaseAllocator oldAllocator = bufferMap.put(drillBuf, newAllocator);
+          if (oldAllocator == null) {
+            final BaseAllocator bufAllocator = (BaseAllocator) drillBuf.getAllocator();
+            historicalLog.logHistory(logger);
+            bufAllocator.historicalLog.logHistory(logger);
+            drillBuf.logHistory(logger);
+            throw new IllegalStateException("No previous entry in SharedBufferLedger for drillBuf");
+          }
+
+          // Whatever happens, this is the new ledger.
+          pNewLedger.value = this;
+
+          /*
+           * If the oldAllocator was the owner, then transfer ownership to the new allocator.
+           */
+          if (oldAllocator == owningAllocator) {
+            owningAllocator = newAllocator;
+            return BaseAllocator.transferTo(newAllocator, this, drillBuf);
+          }
+
+          // Even though we didn't do a real transfer, tell if it would have fit the limit.
+          final int udleMaxCapacity = drillBuf.unwrap().maxCapacity();
+          return newAllocator.allocated + udleMaxCapacity < newAllocator.maxAllocation;
+        } finally {
+          if (DEBUG) {
+            checkBufferMap();
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public DrillBuf buffer(int size) {
+    return buffer(size, size);
+  }
+
+  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());
+  }
+
+  @Override
+  public DrillBuf buffer(final int minSize, final int maxSize) {
+    Preconditions.checkArgument(minSize >= 0,
+        "the minimimum requested size must be non-negative");
+    Preconditions.checkArgument(maxSize >= 0,
+        "the maximum requested size must be non-negative");
+    Preconditions.checkArgument(minSize <= maxSize,
+        "the minimum requested size must be <= the maximum requested size");
+
+    if (DEBUG) {
+      injector.injectUnchecked(allocatorOwner.getExecutionControls(), CHILD_BUFFER_INJECTION_SITE);
+    }
+
+    // we can always return an empty buffer
+    if (minSize == 0) {
+      return getEmpty();
+    }
+
+    synchronized(ALLOCATOR_LOCK) {
+      // Don't allow the allocation if it will take us over the limit.
+      final long allocatedWas = allocated;
+      if (!reserve(null, maxSize, 0)) {
+        throw new OutOfMemoryRuntimeException(createErrorMsg(this, minSize));
+      }
+
+      final long reserved = allocated - allocatedWas;
+      assert reserved == maxSize;
+
+      final UnsafeDirectLittleEndian buffer = INNER_ALLOCATOR.directBuffer(minSize, maxSize);
+      final int actualSize = buffer.maxCapacity();
+      if (actualSize > maxSize) {
+        final int extraSize = actualSize - maxSize;
+        reserve(null, extraSize, RESERVE_F_IGNORE_MAX);
+      }
+
+      final DrillBuf wrapped = new DrillBuf(bufferLedger, this, buffer);
+      buffer.release(); // Should have been retained by the DrillBuf constructor.
+      assert buffer.refCnt() == 1 : "buffer was not retained by DrillBuf";
+      assert allocated <= owned : "allocated more memory than owned";
+
+      bufferAllocation += maxSize;
+      if (allocated > peakAllocated) {
+        peakAllocated = allocated;
+      }
+
+      if (allocatedBuffers != null) {
+        allocatedBuffers.put(buffer, bufferLedger);
+      }
+
+      return wrapped;
+    }
+  }
+
+  @Override
+  public ByteBufAllocator getUnderlyingAllocator() {
+    return INNER_ALLOCATOR;
+  }
+
+  @Override
+  public BufferAllocator newChildAllocator(final AllocatorOwner allocatorOwner,
+      final long initReservation, final long maxAllocation, final int flags) {
+    synchronized(ALLOCATOR_LOCK) {
+      final BaseAllocator childAllocator =
+          new ChildAllocator(this, allocatorOwner, allocationPolicy,
+              initReservation, maxAllocation, flags);
+
+      if (DEBUG) {
+        childAllocators.put(childAllocator, childAllocator);
+        historicalLog.recordEvent("allocator[%d] created new child allocator[%d]",
+            id, childAllocator.id);
+      }
+
+      return childAllocator;
+    }
+  }
+
+  @Override
+  public BufferAllocator getChildAllocator(FragmentContext fragmentContext,
+      final long initialReservation, final long maximumAllocation,
+      final boolean applyFragmentLimit) {
+    return newChildAllocator(allocatorOwner, initialReservation, maximumAllocation,
+        (applyFragmentLimit ? F_LIMITING_ROOT : 0));
+  }
+
+  /**
+   * Reserve space for a DrillBuf for an ownership transfer.
+   *
+   * @param drillBuf the buffer to reserve space for
+   */
+  private void reserveForBuf(final int maxCapacity) {
+    final boolean reserved = reserve(null, maxCapacity, RESERVE_F_IGNORE_MAX);
+    if (DEBUG) {
+      if (!reserved) {
+        throw new IllegalStateException("reserveForBuf() failed");
+      }
+    }
+  }
+
+  @Override
+  public boolean takeOwnership(final DrillBuf drillBuf) {
+    // If already owned by this, there's nothing to do.
+    if (this == drillBuf.getAllocator()) {
+      return true;
+    }
+
+    synchronized(ALLOCATOR_LOCK) {
+      return drillBuf.transferTo(this, bufferLedger);
+    }
+  }
+
+  @Override
+  public boolean shareOwnership(final DrillBuf drillBuf, final Pointer<DrillBuf> bufOut) {
+    synchronized(ALLOCATOR_LOCK) {
+      bufOut.value = drillBuf.shareWith(bufferLedger, this, 0, drillBuf.capacity());
+      return allocated < maxAllocation;
+    }
+  }
+
+  /*
+   * It's not clear why we'd allow anyone to set their own limit, need to see why this is used;
+   * this also doesn't make sense when the limits are constantly shifting, nor for other
+   * allocation policies.
+   */
+  @Deprecated
+  @Override
+  public void setFragmentLimit(long fragmentLimit) {
+    throw new UnsupportedOperationException("unimplemented:BaseAllocator.setFragmentLimit()");
+  }
+
+  /**
+   * Get the fragment limit. This was originally meant to be the maximum amount
+   * of memory the currently running fragment (which owns this allocator or
+   * its ancestor) may use. Note that the value may vary up and down over time
+   * as fragments come and go on the node.
+   *
+   * <p>This is deprecated because the concept is not entirely stable. This
+   * only makes sense for one particular memory allocation policy, which is the
+   * one that sets limits on what fragments on a node may use by dividing up all
+   * the memory evenly between all the fragments (see {@see #POLICY_PER_FRAGMENT}).
+   * Other allocation policies, such as the one that limits memory on a
+   * per-query-per-node basis, wouldn't have a value for this. But we need to have
+   * something until we figure out what to eplace this with because it is used by
+   * some operators (such as ExternalSortBatch) to determine how much memory they
+   * can use before they have to spill to disk.</p>
+   *
+   * @return the fragment limit
+   */
+  @Deprecated
+  @Override
+  public long getFragmentLimit() {
+    return policyAgent.getMemoryLimit(this);
+  }
+
+  @Override
+  public void close() {
+    /*
+     * Some owners may close more than once because of complex cleanup and shutdown
+     * procedures.
+     */
+    if (isClosed) {
+      return;
+    }
+
+    synchronized(ALLOCATOR_LOCK) {
+      if (DEBUG) {
+        verifyAllocator();
+
+        // are there outstanding child allocators?
+        if (!childAllocators.isEmpty()) {
+          for(final BaseAllocator childAllocator : childAllocators.keySet()) {
+            if (childAllocator.isClosed) {
+              logger.debug(String.format(
+                  "Closed child allocator[%d] on parent allocator[%d]'s child list",
+                  childAllocator.id, id));
+            }
+          }
+
+          historicalLog.logHistory(logger);
+          logChildren();
+
+          throw new IllegalStateException(
+              String.format("Allocator[%d] closed with outstanding child allocators", id));
+        }
+
+        // are there outstanding buffers?
+        final int allocatedCount = allocatedBuffers.size();
+        if (allocatedCount > 0) {
+          historicalLog.logHistory(logger);
+          logBuffers();
+
+          throw new IllegalStateException(
+              String.format("Allocator[%d] closed with outstanding buffers allocated (%d)",
+                  id, allocatedCount));
+        }
+
+        if (reservations.size() != 0) {
+          historicalLog.logHistory(logger);
+          logReservations(ReservationsLog.ALL);
+
+          throw new IllegalStateException(
+              String.format("Allocator closed with outstanding reservations (%d)", reservations.size()));
+        }
+
+        /* TODO(DRILL-2740)
+        // We should be the only client holding a reference to empty now.
+        final int emptyRefCnt = empty.refCnt();
+        if (emptyRefCnt != 1) {
+          final String msg = "empty buffer refCnt() == " + emptyRefCnt + " (!= 1)";
+          final StringWriter stringWriter = new StringWriter();
+          stringWriter.write(msg);
+          stringWriter.write('\n');
+          empty.writeState(stringWriter);
+          logger.debug(stringWriter.toString());
+          throw new IllegalStateException(msg);
+        }
+        */
+      }
+
+      // Is there unaccounted-for outstanding allocation?
+      if (allocated > 0) {
+        if (DEBUG) {
+          historicalLog.logHistory(logger);
+        }
+        throw new IllegalStateException(
+            String.format("Unaccounted for outstanding allocation (%d)", allocated));
+      }
+
+      // Any unclaimed reservations?
+      if (preallocSpace > 0) {
+        if (DEBUG) {
+          historicalLog.logHistory(logger);
+        }
+        throw new IllegalStateException(
+            String.format("Unclaimed preallocation space (%d)", preallocSpace));
+      }
+
+      /*
+       * Let go of the empty buffer. If the allocator has been closed more than once,
+       * this may not be necessary, so check to avoid illegal states.
+       */
+      final int emptyCount = empty.refCnt();
+      if (emptyCount > 0) {
+        empty.release(emptyCount);
+      }
+
+      DrillAutoCloseables.closeNoChecked(policyAgent);
+
+      // Inform our parent allocator that we've closed.
+      if (parentAllocator != null) {
+        parentAllocator.releaseBytes(owned);
+        owned = 0;
+        parentAllocator.childClosed(this);
+      }
+
+      if (DEBUG) {
+        historicalLog.recordEvent("closed");
+        logger.debug(String.format(
+            "closed allocator[%d]; getsFromParent == %d, putsToParent == %d",
+            id, getsFromParent, putsToParent));
+      }
+
+      isClosed = true;
+    }
+  }
+
+  /**
+   * Log information about child allocators; only works if DEBUG
+   */
+  private void logChildren() {
+    logger.debug(String.format("allocator[%d] open child allocators BEGIN", id));
+    final Set<BaseAllocator> allocators = childAllocators.keySet();
+    for(final BaseAllocator childAllocator : allocators) {
+      childAllocator.historicalLog.logHistory(logger);
+    }
+    logger.debug(String.format("allocator[%d] open child allocators END", id));
+  }
+
+  private void logBuffers() {
+    final StringBuilder sb = new StringBuilder();
+    final Set<UnsafeDirectLittleEndian> udleSet = allocatedBuffers.keySet();
+
+    sb.append("allocator[");
+    sb.append(Integer.toString(id));
+    sb.append("], ");
+    sb.append(Integer.toString(udleSet.size()));
+    sb.append(" allocated buffers\n");
+
+    for(final UnsafeDirectLittleEndian udle : udleSet) {
+      sb.append(udle.toString());
+      sb.append("[identityHashCode == ");
+      sb.append(Integer.toString(System.identityHashCode(udle)));
+      sb.append("]\n");
+
+      final Collection<DrillBuf> drillBufs = DrillBuf.unwrappedGet(udle);
+      for(DrillBuf drillBuf : drillBufs) {
+        drillBuf.logHistory(logger);
+      }
+    }
+
+    logger.debug(sb.toString());
+  }
+
+  private enum ReservationsLog {
+    ALL,
+    UNUSED,
+  }
+
+  private void logReservations(final ReservationsLog reservationsLog) {
+    final StringBuilder sb = new StringBuilder();
+    sb.append(String.format("allocator[%d] reservations BEGIN", id));
+
+    final Set<Reservation> reservations = this.reservations.keySet();
+    for(final Reservation reservation : reservations) {
+      if ((reservationsLog == ReservationsLog.ALL)
+          || ((reservationsLog == ReservationsLog.UNUSED) && (!reservation.isUsed()))) {
+        reservation.writeHistoryToBuilder(sb);
+      }
+    }
+
+    sb.append(String.format("allocator[%d] reservations END", id));
+
+    logger.debug(sb.toString());
+  }
+
+  @Override
+  public long getAllocatedMemory() {
+    return allocated;
+  }
+
+  @Override
+  public int getId() {
+    return id;
+  }
+
+  @Override
+  public long getPeakMemoryAllocation() {
+    return peakAllocated;
+  }
+
+  @Override
+  public DrillBuf getEmpty() {
+    empty.retain(1);
+    // TODO(DRILL-2740) update allocatedBuffers
+    return empty;
+  }
+
+  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(ALLOCATOR_LOCK) {
+          reservations.put(this, this);
+        }
+      } else {
+        historicalLog = null;
+      }
+    }
+
+    @Override
+    public void close() {
+      if (DEBUG) {
+        if (!isClosed()) {
+          final Object object;
+          synchronized(ALLOCATOR_LOCK) {
+            object = reservations.remove(this);
+          }
+          if (object == null) {
+            final StringBuilder sb = new StringBuilder();
+            writeHistoryToBuilder(sb);
+
+            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 boolean reserved;
+      synchronized(ALLOCATOR_LOCK) {
+        reserved = BaseAllocator.this.reserve(null, nBytes, 0);
+        if (reserved) {
+          preallocSpace += nBytes;
+        }
+      }
+
+      if (DEBUG) {
+        historicalLog.recordEvent("reserve(%d) => %s", nBytes, Boolean.toString(reserved));
+      }
+
+      return reserved;
+    }
+
+    @Override
+    protected DrillBuf allocate(int nBytes) {
+      /*
+       * 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.
+       */
+      synchronized(ALLOCATOR_LOCK) {
+        BaseAllocator.this.allocated -= nBytes;
+        final DrillBuf drillBuf = BaseAllocator.this.buffer(nBytes);
+        preallocSpace -= nBytes;
+
+        if (DEBUG) {
+          historicalLog.recordEvent("allocate() => %s",
+              drillBuf == null ? "null" : String.format("DrillBuf[%d]", drillBuf.getId()));
+        }
+
+        return drillBuf;
+      }
+    }
+
+    @Override
+    protected void releaseReservation(int nBytes) {
+      synchronized(ALLOCATOR_LOCK) {
+        releaseBytes(nBytes);
+        preallocSpace -= nBytes;
+      }
+
+      if (DEBUG) {
+        historicalLog.recordEvent("releaseReservation(%d)", nBytes);
+      }
+    }
+
+    private String getState() {
+      return String.format("size == %d, isUsed == %s", getSize(), Boolean.toString(isUsed()));
+    }
+
+    private void writeToBuilder(final StringBuilder sb) {
+      sb.append(String.format("reservation[%d]: ", System.identityHashCode(this)));
+      sb.append(getState());
+    }
+
+    /**
+     * Only works for DEBUG
+     *
+     * @param sb builder to write to
+     */
+    private void writeHistoryToBuilder(final StringBuilder sb) {
+      historicalLog.buildHistory(sb, getState());
+    }
+  }
+
+  @Override
+  public AllocationReservation newReservation() {
+    return new Reservation();
+  }
+
+  /**
+   * Verifies the accounting state of the allocator. Only works for DEBUG.
+   *
+   * @throws IllegalStateException when any problems are found
+   */
+  protected 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
+   */
+  protected void verifyAllocator(
+      final IdentityHashMap<UnsafeDirectLittleEndian, BaseAllocator> buffersSeen) {
+    synchronized(ALLOCATOR_LOCK) {
+      // verify purely local accounting
+      if (allocated > owned) {
+        historicalLog.logHistory(logger);
+        throw new IllegalStateException("Allocator (id = " + id + ") has allocated more than it owns");
+      }
+
+      // the empty buffer should still be empty
+      final long emptyCapacity = empty.maxCapacity();
+      if (emptyCapacity != 0) {
+        throw new IllegalStateException("empty buffer maxCapacity() == " + emptyCapacity + " (!= 0)");
+      }
+
+      // The remaining tests can only be performed if we're in debug mode.
+      if (!DEBUG) {
+        return;
+      }
+
+      // 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 += childAllocator.owned;
+      }
+      if (childTotal > allocated) {
+        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 (" + allocated + ')');
+      }
+
+      // Furthermore, the amount I've allocated should be that plus buffers I've allocated.
+      long bufferTotal = 0;
+      final Set<UnsafeDirectLittleEndian> udleSet = allocatedBuffers.keySet();
+      for(final UnsafeDirectLittleEndian udle : udleSet) {
+        /*
+         * 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 (reservedTotal != preallocSpace) {
+        logReservations(ReservationsLog.UNUSED);
+
+        throw new IllegalStateException(
+            String.format("This allocator's reservedTotal(%d) doesn't match preallocSpace (%d)",
+                reservedTotal, preallocSpace));
+      }
+
+      if (bufferTotal + reservedTotal + childTotal != allocated) {
+        final StringBuilder sb = new StringBuilder();
+        sb.append("allocator[");
+        sb.append(Integer.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, udleSet);
+        }
+
+        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(Integer.toString(childAllocator.id));
+            sb.append("] owned ");
+            sb.append(Long.toString(childAllocator.owned));
+            sb.append('\n');
+          }
+        }
+
+        if (reservedTotal != 0) {
+          sb.append(String.format("reserved total : ", reservedTotal));
+          for(final Reservation reservation : reservationSet) {
+            reservation.writeToBuilder(sb);
+            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));
+      }
+    }
+  }
+
+  private void dumpBuffers(final StringBuilder sb, final Set<UnsafeDirectLittleEndian> udleSet) {
+    for(final UnsafeDirectLittleEndian udle : udleSet) {
+      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 boolean isDebug() {
+    return DEBUG;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BufferLedger.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BufferLedger.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BufferLedger.java
new file mode 100644
index 0000000..d649940
--- /dev/null
+++ b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BufferLedger.java
@@ -0,0 +1,94 @@
+/**
+ * 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.exec.util.Pointer;
+
+import io.netty.buffer.DrillBuf;
+import io.netty.buffer.PooledByteBufAllocatorL;
+
+/**
+ * BufferLedger is an interface meant to facility the private
+ * exchange of information between a DrillBuf and its owning
+ * allocator. To that end, a number of DrillBuf constructors
+ * and methods take a BufferLedger as an argument, yet there
+ * are no public implementations of BufferLedger; they all
+ * come from inner classes implemented by allocators, ensuring
+ * that allocators can give DrillBufs the access they need when
+ * they are created or asked to perform complex operations such
+ * as ownership sharing or transfers.
+ */
+public interface BufferLedger {
+  /**
+   * Get the underlying pooled allocator used by this ledger's
+   * allocator.
+   *
+   * <p>This is usually used to create the shared singleton
+   * empty buffer. Don't use it to create random buffers, because
+   * they won't be tracked, and we won't be able to find leaks.</p>
+   *
+   * @return the underlying pooled allocator
+   */
+  public PooledByteBufAllocatorL getUnderlyingAllocator();
+
+  /**
+   * Return a buffer's memory to the allocator.
+   *
+   * @param drillBuf the DrillBuf that was freed
+   */
+  public void release(DrillBuf drillBuf);
+
+  /**
+   * Share ownership of a buffer with another allocator. As far as reader
+   * and writer index positions go, this acts like a new slice that is owned
+   * by the target allocator, but which has it's own lifetime (i.e., it doesn't
+   * share the fate of the original buffer, unlike real slices).
+   *
+   * @param pDrillBuf returns the new DrillBuf that is shared
+   * @param otherLedger the ledger the new DrillBuf should use
+   * @param otherAllocator the new allocator-owner
+   * @param drillBuf the original DrillBuf to be shared
+   * @param index the starting index to be shared (as for slicing)
+   * @param length the length to be shared (as for slicing)
+   * @param drillBufFlags private flags passed through from the allocator
+   *   (this call originates with a call to BufferAllocator.shareOwnership()).
+   * @return the ledger the calling DrillBuf must use from this point forward;
+   *   this may not match it's original ledger, as allocators provide multiple
+   *   implementations of ledgers to cope with sharing and slicing
+   */
+  public BufferLedger shareWith(Pointer<DrillBuf> pDrillBuf,
+      BufferLedger otherLedger, BufferAllocator otherAllocator,
+      DrillBuf drillBuf, int index, int length, int drillBufFlags);
+
+  /**
+   * Transfer the ownership of a buffer to another allocator. This doesn't change
+   * any of the buffer's reader or writer positions or size, just which allocator
+   * owns it. The reference count stays the same.
+   *
+   * @param newAlloc the new allocator (the one to transfer to)
+   * @param pNewLedger a Pointer<> initialized with a candidate ledger; this
+   *   may be used, or it may not, depending on the sharing state of the buffer.
+   *   The caller is required to use whatever ledger is in pNewLedger on return
+   * @param drillBuf the buffer to transfer
+   * @return true if the transfer kept the target allocator within its maximum
+   *   allocation limit; false if the allocator now owns more memory than its
+   *   creation-time maximum
+   */
+  public boolean transferTo(final BufferAllocator newAlloc,
+      final Pointer<BufferLedger> pNewLedger, final DrillBuf drillBuf);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java
new file mode 100644
index 0000000..37475fd
--- /dev/null
+++ b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.memory;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.testing.ExecutionControls;
+
+/**
+ * An {@link AllocatorOwner} implementation that supports linking ownership to
+ * a parent object. This can be convenient for identification purposes, when the
+ * parent of the allocator's owner is a better handle for debugging.
+ *
+ * <p>The implementation of {@link #getExecutionControls()} returns the childOwner's
+ * response to getExecutionControls().</p>
+ */
+public class ChainedAllocatorOwner implements AllocatorOwner {
+  private final AllocatorOwner childOwner;
+  private final AllocatorOwner parentOwner;
+
+  /**
+   * Constructor.
+   *
+   * @param childOwner the owner of the allocator
+   * @param parentOwner the object that owns or created the childOwner
+   */
+  public ChainedAllocatorOwner(AllocatorOwner childOwner, AllocatorOwner parentOwner) {
+    this.childOwner = childOwner;
+    this.parentOwner = parentOwner;
+  }
+
+  @Override
+  public String toString() {
+    return childOwner + "(owned by " + parentOwner + ')';
+  }
+
+  @Override
+  public ExecutionControls getExecutionControls() {
+    return childOwner.getExecutionControls();
+  }
+
+  @Override
+  public FragmentContext getFragmentContext() {
+    return childOwner.getFragmentContext();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java
new file mode 100644
index 0000000..8636d26
--- /dev/null
+++ b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java
@@ -0,0 +1,47 @@
+/**
+ * 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, AllocatorOwner allocatorOwner,
+      AllocationPolicy allocationPolicy, long initReservation, long maxAllocation, int flags) {
+    super(parentAllocator, allocatorOwner, allocationPolicy, initReservation, maxAllocation, flags);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocator.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocator.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocator.java
new file mode 100644
index 0000000..63b9987
--- /dev/null
+++ b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocator.java
@@ -0,0 +1,125 @@
+/**
+ * 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.lang.management.ManagementFactory;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.PooledByteBufAllocatorL;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.testing.ExecutionControls;
+
+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 {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootAllocator.class);
+
+  // TODO these statics, and others in BaseAllocator, may be a problem for multiple in-process Drillbits
+  private static final PooledByteBufAllocatorL innerAllocator = PooledByteBufAllocatorL.DEFAULT;
+  private static long maxDirect;
+
+  public static AllocationPolicy getAllocationPolicy() {
+    final String policyName = System.getProperty(ExecConstants.ALLOCATION_POLICY,
+        BaseAllocator.POLICY_LOCAL_MAX_NAME); // TODO try with PER_FRAGMENT_NAME
+
+    switch(policyName) {
+    case POLICY_PER_FRAGMENT_NAME:
+      return POLICY_PER_FRAGMENT;
+    case POLICY_LOCAL_MAX_NAME:
+      return POLICY_LOCAL_MAX;
+    default:
+      throw new IllegalArgumentException("Unrecognized allocation policy name \"" + policyName + "\"");
+    }
+  }
+
+  public RootAllocator(final DrillConfig drillConfig) {
+    this(getAllocationPolicy(), 0, Math.min(
+        DrillConfig.getMaxDirectMemory(), drillConfig.getLong(ExecConstants.TOP_LEVEL_MAX_ALLOC)), 0);
+  }
+
+  public static long getMaxDirect() {
+    return maxDirect;
+  }
+
+  /**
+   * Provide statistics via JMX for each RootAllocator.
+   */
+  private class AllocatorsStats implements AllocatorsStatsMXBean {
+    @Override
+    public long getMaxDirectMemory() {
+      return maxDirect;
+    }
+  }
+
+  private static class RootAllocatorOwner implements AllocatorOwner {
+    @Override
+    public ExecutionControls getExecutionControls() {
+      return null;
+    }
+
+    @Override
+    public FragmentContext getFragmentContext() {
+      return null;
+    }
+  }
+
+  @VisibleForTesting
+  public RootAllocator(final AllocationPolicy allocationPolicy,
+      final long initAllocation, final long maxReservation, final int flags) {
+    super(null, new RootAllocatorOwner(), allocationPolicy, initAllocation, maxDirect = maxReservation, flags);
+    assert (flags & F_LIMITING_ROOT) == 0 : "the RootAllocator shouldn't be a limiting root";
+
+    try {
+      final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      final ObjectName objectName = new ObjectName("org.apache.drill.exec.memory:Allocators=" + id);
+      final AllocatorsStats mbean = new AllocatorsStats();
+      mbs.registerMBean(mbean, objectName);
+    } catch(Exception e) {
+      logger.info("Exception setting up AllocatorsStatsMBean", e);
+    }
+  }
+
+  @Override
+  public ByteBufAllocator getUnderlyingAllocator() {
+    return innerAllocator;
+  }
+
+  @Override
+  protected boolean canIncreaseOwned(final long nBytes, final int flags) {
+    // the end total has already been checked against maxAllocation, so we can just return true
+    return true;
+  }
+
+  /**
+   * Verify the accounting state of the allocation system.
+   */
+  @VisibleForTesting
+  public void verify() {
+    verifyAllocator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java
----------------------------------------------------------------------
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
index a413e4a..078fc21 100644
--- 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
@@ -19,8 +19,6 @@ 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.
@@ -34,34 +32,12 @@ public class RootAllocatorFactory {
    * @return a new root allocator
    */
   public static BufferAllocator newRoot(final DrillConfig drillConfig) {
-/* TODO(DRILL-1942)
+/* TODO(cwestin)
     if (BaseAllocator.DEBUG) {
-      return new TopLevelAllocator(drillConfig);
+      return new RootAllocator(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);
+    return new RootAllocator(drillConfig);
+    // TODO(cwestin) return new TopLevelAllocator(drillConfig);
   }
-*/
 }


[13/13] drill git commit: DRILL-4134: Allocator Improvements

Posted by ja...@apache.org.
DRILL-4134: Allocator Improvements

- make Allocator mostly lockless
- change BaseAllocator maps to direct references
- add documentation around memory management model
- move transfer and ownership methods to DrillBuf
- Improve debug messaging.
- Fix/revert sort changes
- Remove unused fragment limit flag
- Add time to HistoricalLog events
- Remove reservation amount from RootAllocator constructor (since not allowed)
- Fix concurrency issue where allocator is closing at same moment as incoming batch transfer, causing leaked memory and/or query failure.
- Add new AutoCloseables.close(Iterable<AutoCloseable>)
- Remove extraneous DataResponseHandler and Impl (and update TestBitRpc to use smarter mock of FragmentManager)
- Remove the concept of poison pill record batches, using instead FragmentContext.isOverMemoryLimit()
- Update incoming data batches so that they are transferred under protection of a close lock
- Improve field names in IncomingBuffers and move synchronization to collectors as opposed to IncomingBuffers (also change decrementing to decrementToZero rather than two part check).

This closes #238.


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

Branch: refs/heads/master
Commit: 809f4620d7d82c72240212de13b993049550959d
Parents: 53dcabe
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun Nov 15 17:26:02 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Dec 21 23:52:32 2015 -0800

----------------------------------------------------------------------
 .../org/apache/drill/common/AutoCloseables.java |    2 +-
 .../org/apache/drill/common/HistoricalLog.java  |   68 +-
 .../org/apache/drill/common/StackTrace.java     |    9 +-
 .../common/concurrent/AutoCloseableLock.java    |   43 +
 .../hive/HiveDrillNativeScanBatchCreator.java   |    8 +-
 distribution/pom.xml                            |    5 -
 distribution/src/assemble/bin.xml               |    1 -
 exec/java-exec/pom.xml                          |    2 +-
 .../expr/fn/impl/StringFunctionHelpers.java     |    4 +-
 .../apache/drill/exec/expr/fn/impl/XXHash.java  |    4 +-
 .../drill/exec/memory/RootAllocatorFactory.java |   40 +
 .../drill/exec/ops/BufferManagerImpl.java       |    3 +-
 .../apache/drill/exec/ops/FragmentContext.java  |   63 +-
 .../drill/exec/ops/OperatorContextImpl.java     |   17 +-
 .../org/apache/drill/exec/ops/QueryContext.java |   14 +-
 .../drill/exec/physical/impl/BaseRootExec.java  |    2 +-
 .../drill/exec/physical/impl/ScanBatch.java     |    2 +-
 .../exec/physical/impl/SingleSenderCreator.java |    3 +-
 .../BroadcastSenderRootExec.java                |    3 +-
 .../impl/mergereceiver/MergingRecordBatch.java  |    5 +-
 .../PartitionSenderRootExec.java                |    3 +-
 .../impl/sort/SortRecordBatchBuilder.java       |   49 +-
 .../physical/impl/trace/TraceRecordBatch.java   |    7 +-
 .../impl/union/UnionAllRecordBatch.java         |   15 +-
 .../UnorderedReceiverBatch.java                 |    7 +-
 .../physical/impl/xsort/ExternalSortBatch.java  |   51 +-
 .../impl/xsort/PriorityQueueCopier.java         |    3 +-
 .../impl/xsort/PriorityQueueCopierTemplate.java |    8 +-
 .../drill/exec/record/AbstractRecordBatch.java  |    4 +-
 .../drill/exec/record/RecordBatchLoader.java    |    6 +-
 .../apache/drill/exec/record/WritableBatch.java |    5 +-
 .../drill/exec/rpc/control/ControlClient.java   |    2 +-
 .../drill/exec/rpc/control/ControlServer.java   |    2 +-
 .../apache/drill/exec/rpc/data/DataClient.java  |    2 +-
 .../exec/rpc/data/DataConnectionCreator.java    |   14 +-
 .../exec/rpc/data/DataResponseHandler.java      |   34 -
 .../exec/rpc/data/DataResponseHandlerImpl.java  |   58 -
 .../apache/drill/exec/rpc/data/DataServer.java  |  109 +-
 .../drill/exec/rpc/data/IncomingDataBatch.java  |   73 +
 .../apache/drill/exec/rpc/user/UserServer.java  |    4 +-
 .../org/apache/drill/exec/server/Drillbit.java  |    2 +-
 .../drill/exec/service/ServiceEngine.java       |    6 +-
 .../exec/store/dfs/easy/EasyFormatPlugin.java   |   10 +-
 .../exec/store/parquet/ParquetRecordWriter.java |    2 +-
 .../store/parquet/ParquetScanBatchCreator.java  |    6 +-
 .../drill/exec/store/sys/MemoryIterator.java    |    4 +-
 .../org/apache/drill/exec/work/WorkManager.java |    8 -
 .../exec/work/batch/AbstractDataCollector.java  |   15 +-
 .../exec/work/batch/BaseRawBatchBuffer.java     |   28 +-
 .../drill/exec/work/batch/IncomingBuffers.java  |   82 +-
 .../exec/work/batch/SpoolingRawBatchBuffer.java |   12 +-
 .../work/batch/UnlimitedRawBatchBuffer.java     |    2 +-
 .../apache/drill/exec/work/foreman/Foreman.java |    7 +-
 .../exec/work/fragment/FragmentManager.java     |    4 +-
 .../work/fragment/NonRootFragmentManager.java   |    4 +-
 .../exec/work/fragment/RootFragmentManager.java |    4 +-
 .../java/org/apache/drill/BaseTestQuery.java    |   13 +-
 .../apache/drill/TestAllocationException.java   |   71 -
 .../drill/exec/memory/TestAllocators.java       |  273 ++-
 .../drill/exec/memory/TestBaseAllocator.java    |  651 -------
 .../exec/physical/impl/TestOptiqPlans.java      |    3 +-
 .../exec/record/vector/TestValueVector.java     |   23 +-
 .../apache/drill/exec/server/TestBitRpc.java    |   84 +-
 .../testing/TestCountDownLatchInjection.java    |   13 +-
 .../exec/testing/TestExceptionInjection.java    |   21 +-
 .../drill/exec/testing/TestPauseInjection.java  |   17 +-
 .../drill/exec/testing/TestResourceLeak.java    |   18 +-
 .../complex/writer/TestPromotableWriter.java    |    2 -
 exec/memory/base/pom.xml                        |   10 +
 .../src/main/java/io/netty/buffer/DrillBuf.java |  714 ++------
 .../java/io/netty/buffer/ExpandableByteBuf.java |   55 +
 .../java/io/netty/buffer/FakeAllocator.java     |  167 --
 .../main/java/io/netty/buffer/LargeBuffer.java  |  297 +---
 .../io/netty/buffer/MutableWrappedByteBuf.java  |  336 ++++
 .../netty/buffer/PooledByteBufAllocatorL.java   |  312 ++--
 .../netty/buffer/UnsafeDirectLittleEndian.java  |  316 ++--
 .../apache/drill/exec/memory/Accountant.java    |  274 +++
 .../org/apache/drill/exec/memory/Accountor.java |   43 -
 .../exec/memory/AllocationReservation.java      |   86 +
 .../exec/memory/AllocatorClosedException.java   |   31 +
 .../drill/exec/memory/AllocatorManager.java     |  356 ++++
 .../apache/drill/exec/memory/BaseAllocator.java |  739 ++++++++
 .../drill/exec/memory/BoundsChecking.java       |    1 +
 .../drill/exec/memory/BufferAllocator.java      |  155 +-
 .../drill/exec/memory/ChildAllocator.java       |   53 +
 .../exec/memory/DrillByteBufAllocator.java      |  141 ++
 .../apache/drill/exec/memory/LimitConsumer.java |   28 -
 .../java/org/apache/drill/exec/memory/README.md |  121 ++
 .../apache/drill/exec/memory/RootAllocator.java |   39 +
 .../apache/drill/exec/memory/package-info.java  |   24 +
 .../base/src/main/resources/drill-module.conf   |   25 +
 .../drill/exec/memory/TestAccountant.java       |  163 ++
 .../drill/exec/memory/TestBaseAllocator.java    |  645 +++++++
 .../apache/drill/exec/memory/TestEndianess.java |   43 +
 exec/memory/impl/pom.xml                        |   68 -
 .../apache/drill/exec/memory/AccountorImpl.java |  437 -----
 .../drill/exec/memory/AllocationPolicy.java     |   38 -
 .../exec/memory/AllocationPolicyAgent.java      |   69 -
 .../exec/memory/AllocationReservation.java      |  152 --
 .../exec/memory/AllocatorClosedException.java   |   31 -
 .../drill/exec/memory/AllocatorOwner.java       |   40 -
 .../exec/memory/AllocatorsStatsMXBean.java      |   34 -
 .../drill/exec/memory/AtomicRemainder.java      |  215 ---
 .../apache/drill/exec/memory/BaseAllocator.java | 1654 ------------------
 .../apache/drill/exec/memory/BufferLedger.java  |   94 -
 .../exec/memory/ChainedAllocatorOwner.java      |   60 -
 .../drill/exec/memory/ChildAllocator.java       |   47 -
 .../apache/drill/exec/memory/RootAllocator.java |  125 --
 .../drill/exec/memory/RootAllocatorFactory.java |   43 -
 .../drill/exec/memory/TopLevelAllocator.java    |  408 -----
 .../impl/src/main/resources/drill-module.conf   |   25 -
 .../apache/drill/exec/memory/TestEndianess.java |   43 -
 exec/memory/pom.xml                             |    1 -
 .../exec/rpc/BasicClientWithConnection.java     |    2 +-
 .../exec/expr/fn/impl/ByteFunctionHelpers.java  |    8 +-
 .../apache/drill/exec/util/DecimalUtility.java  |   23 +-
 .../org/apache/drill/exec/proto/BitData.java    |   94 +-
 .../apache/drill/exec/proto/SchemaBitData.java  |    7 -
 .../exec/proto/beans/FragmentRecordBatch.java   |   23 -
 protocol/src/main/protobuf/BitData.proto        |    1 -
 120 files changed, 4394 insertions(+), 6661 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/common/src/main/java/org/apache/drill/common/AutoCloseables.java
----------------------------------------------------------------------
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 43cbbf5..fcdfe14 100644
--- a/common/src/main/java/org/apache/drill/common/AutoCloseables.java
+++ b/common/src/main/java/org/apache/drill/common/AutoCloseables.java
@@ -68,7 +68,7 @@ public class AutoCloseables {
    * Closes all autoCloseables if not null and suppresses subsequent exceptions if more than one
    * @param autoCloseables the closeables to close
    */
-  public static void close(Collection<? extends AutoCloseable> ac) throws Exception {
+  public static void close(Iterable<? extends AutoCloseable> ac) throws Exception {
     Exception topLevelException = null;
     for (AutoCloseable closeable : ac) {
       try {

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/common/src/main/java/org/apache/drill/common/HistoricalLog.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/HistoricalLog.java b/common/src/main/java/org/apache/drill/common/HistoricalLog.java
index 49b7c33..3331215 100644
--- a/common/src/main/java/org/apache/drill/common/HistoricalLog.java
+++ b/common/src/main/java/org/apache/drill/common/HistoricalLog.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.common;
 
+import java.util.Arrays;
 import java.util.LinkedList;
 
 import org.slf4j.Logger;
@@ -30,9 +31,11 @@ public class HistoricalLog {
   private static class Event {
     private final String note; // the event text
     private final StackTrace stackTrace; // where the event occurred
+    private final long time;
 
     public Event(final String note) {
       this.note = note;
+      this.time = System.nanoTime();
       stackTrace = new StackTrace();
     }
   }
@@ -106,8 +109,8 @@ public class HistoricalLog {
    *
    * @param sb {@link StringBuilder} to write to
    */
-  public void buildHistory(final StringBuilder sb) {
-    buildHistory(sb, null);
+  public void buildHistory(final StringBuilder sb, boolean includeStackTrace) {
+    buildHistory(sb, 0, includeStackTrace);
   }
 
   /**
@@ -119,34 +122,50 @@ public class HistoricalLog {
    * @param additional an extra string that will be written between the identifying
    *     information and the history; often used for a current piece of state
    */
-  public synchronized void buildHistory(final StringBuilder sb, final CharSequence additional) {
-    sb.append('\n')
-        .append(idString);
 
-    if (additional != null) {
-      sb.append('\n')
-          .append(additional)
-          .append('\n');
-    }
+  /**
+   *
+   * @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');
 
-    sb.append(" event log\n");
 
     if (firstEvent != null) {
-      sb.append("  ")
+      sb.append(innerIndentation)
+          .append(firstEvent.time)
+          .append(' ')
           .append(firstEvent.note)
           .append('\n');
-      firstEvent.stackTrace.writeToBuilder(sb, 4);
+      if (includeStackTrace) {
+        firstEvent.stackTrace.writeToBuilder(sb, indent + 2);
+      }
 
       for(final Event event : history) {
         if (event == firstEvent) {
           continue;
         }
-
-        sb.append("  ")
+        sb.append(innerIndentation)
+            .append("  ")
+            .append(event.time)
+            .append(' ')
             .append(event.note)
             .append('\n');
 
-        event.stackTrace.writeToBuilder(sb, 4);
+        if (includeStackTrace) {
+          event.stackTrace.writeToBuilder(sb, indent + 2);
+          sb.append('\n');
+        }
       }
     }
   }
@@ -157,23 +176,10 @@ public class HistoricalLog {
    * events with their stack traces.
    *
    * @param logger {@link Logger} 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
    */
-  public void logHistory(final Logger logger, final CharSequence additional) {
+  public void logHistory(final Logger logger) {
     final StringBuilder sb = new StringBuilder();
-    buildHistory(sb, additional);
+    buildHistory(sb, 0, true);
     logger.debug(sb.toString());
   }
-
-  /**
-   * 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) {
-    logHistory(logger, null);
-  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/common/src/main/java/org/apache/drill/common/StackTrace.java
----------------------------------------------------------------------
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 de64ed7..1dbffe6 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 class 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

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/common/src/main/java/org/apache/drill/common/concurrent/AutoCloseableLock.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/concurrent/AutoCloseableLock.java b/common/src/main/java/org/apache/drill/common/concurrent/AutoCloseableLock.java
new file mode 100644
index 0000000..91d50b4
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/concurrent/AutoCloseableLock.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.common.concurrent;
+
+import java.util.concurrent.locks.Lock;
+
+/**
+ * Simple wrapper class that allows Locks to be released via an try-with-resources block.
+ */
+public 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();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
index 1a76136..9f53971 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveDrillNativeScanBatchCreator.java
@@ -24,8 +24,6 @@ import java.util.Map.Entry;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
@@ -55,6 +53,9 @@ import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.BlockMetaData;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
 @SuppressWarnings("unused")
 public class HiveDrillNativeScanBatchCreator implements BatchCreator<HiveDrillNativeParquetSubScan> {
 
@@ -94,8 +95,7 @@ public class HiveDrillNativeScanBatchCreator implements BatchCreator<HiveDrillNa
       }
     }
 
-    final OperatorContext oContext = context.newOperatorContext(config,
-        false /* ScanBatch is not subject to fragment memory limit */);
+    final OperatorContext oContext = context.newOperatorContext(config);
 
     int currentPartitionIndex = 0;
     final List<RecordReader> readers = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/distribution/pom.xml
----------------------------------------------------------------------
diff --git a/distribution/pom.xml b/distribution/pom.xml
index f8aced5..5aaf09d 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -45,11 +45,6 @@
       <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>
       <version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/distribution/src/assemble/bin.xml
----------------------------------------------------------------------
diff --git a/distribution/src/assemble/bin.xml b/distribution/src/assemble/bin.xml
index f8ada9b..449ac6c 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>

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 3cd2585..efb549c 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -289,7 +289,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>

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctionHelpers.java
----------------------------------------------------------------------
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 c919f28..3d0f170 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 @@ import com.google.common.base.Charsets;
 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 class StringFunctionHelpers {
   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);

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/XXHash.java
----------------------------------------------------------------------
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 033e2cb..0124b2f 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 @@ import com.google.common.primitives.UnsignedLongs;
 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 final class XXHash {
   }
 
   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);
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java
new file mode 100644
index 0000000..4fad668
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java
@@ -0,0 +1,40 @@
+/**
+ * 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;
+
+public class RootAllocatorFactory {
+
+  public static final String TOP_LEVEL_MAX_ALLOC = "drill.memory.top.max";
+
+  /**
+   * Constructor to prevent instantiation of this static utility class.
+   */
+  private RootAllocatorFactory() {}
+
+  /**
+   * Create a new Root Allocator
+   * @param drillConfig
+   *          the DrillConfig
+   * @return a new root allocator
+   */
+  public static BufferAllocator newRoot(final DrillConfig drillConfig) {
+    return new RootAllocator(Math.min(DrillConfig.getMaxDirectMemory(), drillConfig.getLong(TOP_LEVEL_MAX_ALLOC)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/ops/BufferManagerImpl.java
----------------------------------------------------------------------
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 003fc9b..b8c6796 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 class BufferManagerImpl implements BufferManager {
   }
 
   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;
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
----------------------------------------------------------------------
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 a773c22..304ecf1 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.ClassGenerator;
 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;
@@ -159,8 +158,10 @@ public class FragmentContext implements AutoCloseable, UdfUtilities {
     // 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
     try {
-      allocator = context.getAllocator().getChildAllocator(new AsLimitConsumer(), fragment.getMemInitial(),
-          fragment.getMemMax(), true);
+      allocator = context.getAllocator().newChildAllocator(
+          "frag:" + QueryIdHelper.getFragmentId(fragment.getHandle()),
+          fragment.getMemInitial(),
+          fragment.getMemMax());
       Preconditions.checkNotNull(allocator, "Unable to acuqire allocator");
     } catch (final OutOfMemoryException e) {
       throw UserException.memoryError(e)
@@ -174,31 +175,6 @@ public class FragmentContext implements AutoCloseable, UdfUtilities {
     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 +273,6 @@ public class FragmentContext implements AutoCloseable, UdfUtilities {
     return frag;
   }
 
-  public LimitConsumer asLimitConsumer() {
-    return new AsLimitConsumer();
-  }
-
-
   /**
    * Get this fragment's allocator.
    * @return the allocator
@@ -314,11 +285,19 @@ public class FragmentContext implements AutoCloseable, UdfUtilities {
     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) throws OutOfMemoryException {
+    return allocator.newChildAllocator(
+        "op:" + QueryIdHelper.getFragmentId(fragment.getHandle()) + ":" + operatorId + ":" + operatorName,
+        initialReservation,
+        maximumReservation
+        );
+  }
+
+  public boolean isOverMemoryLimit() {
+    return allocator.isOverLimit();
   }
 
   public <T> T getImplementationClass(final ClassGenerator<T> cg)
@@ -361,16 +340,16 @@ public class FragmentContext implements AutoCloseable, UdfUtilities {
     return buffers;
   }
 
-  public OperatorContext newOperatorContext(PhysicalOperator popConfig, OperatorStats stats, boolean applyFragmentLimit)
+  public OperatorContext newOperatorContext(PhysicalOperator popConfig, OperatorStats stats)
       throws OutOfMemoryException {
-    OperatorContextImpl context = new OperatorContextImpl(popConfig, this, stats, applyFragmentLimit);
+    OperatorContextImpl context = new OperatorContextImpl(popConfig, this, stats);
     contexts.add(context);
     return context;
   }
 
-  public OperatorContext newOperatorContext(PhysicalOperator popConfig, boolean applyFragmentLimit)
+  public OperatorContext newOperatorContext(PhysicalOperator popConfig)
       throws OutOfMemoryException {
-    OperatorContextImpl context = new OperatorContextImpl(popConfig, this, applyFragmentLimit);
+    OperatorContextImpl context = new OperatorContextImpl(popConfig, this);
     contexts.add(context);
     return context;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java
----------------------------------------------------------------------
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 a3ec6bd..8217afd 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
@@ -48,7 +48,6 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
   private final PhysicalOperator popConfig;
   private final OperatorStats stats;
   private final BufferManager manager;
-  private final boolean applyFragmentLimit;
   private DrillFileSystem fs;
   private final ExecutorService executor;
 
@@ -60,21 +59,23 @@ class OperatorContextImpl extends OperatorContext implements AutoCloseable {
    */
   private ListeningExecutorService delegatePool;
 
-  public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, boolean applyFragmentLimit) throws OutOfMemoryException {
-    this.applyFragmentLimit=applyFragmentLimit;
-    this.allocator = context.getNewChildAllocator(popConfig.getInitialAllocation(), popConfig.getMaxAllocation(), applyFragmentLimit);
+  public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context) throws OutOfMemoryException {
+    this.allocator = context.getNewChildAllocator(popConfig.getClass().getSimpleName(),
+        popConfig.getOperatorId(), popConfig.getInitialAllocation(), popConfig.getMaxAllocation());
     this.popConfig = popConfig;
     this.manager = new BufferManagerImpl(allocator);
 
-    OpProfileDef def = new OpProfileDef(popConfig.getOperatorId(), popConfig.getOperatorType(), getChildCount(popConfig));
+    OpProfileDef def =
+        new OpProfileDef(popConfig.getOperatorId(), popConfig.getOperatorType(), getChildCount(popConfig));
     stats = context.getStats().newOperatorStats(def, allocator);
     executionControls = context.getExecutionControls();
     executor = context.getDrillbitContext().getExecutor();
   }
 
-  public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, OperatorStats stats, boolean applyFragmentLimit) throws OutOfMemoryException {
-    this.applyFragmentLimit=applyFragmentLimit;
-    this.allocator = context.getNewChildAllocator(popConfig.getInitialAllocation(), popConfig.getMaxAllocation(), applyFragmentLimit);
+  public OperatorContextImpl(PhysicalOperator popConfig, FragmentContext context, OperatorStats stats)
+      throws OutOfMemoryException {
+    this.allocator = context.getNewChildAllocator(popConfig.getClass().getSimpleName(),
+        popConfig.getOperatorId(), popConfig.getInitialAllocation(), popConfig.getMaxAllocation());
     this.popConfig = popConfig;
     this.manager = new BufferManagerImpl(allocator);
     this.stats     = stats;

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
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 e4f50a1..51a581a 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.DrillConfig;
 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 org.apache.drill.exec.util.Utilities;
 
 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,7 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext {
    */
   private boolean closed = false;
 
-  public QueryContext(final UserSession session, final DrillbitContext drillbitContext) {
+  public QueryContext(final UserSession session, final DrillbitContext drillbitContext, QueryId queryId) {
     this.drillbitContext = drillbitContext;
     this.session = session;
     queryOptions = new QueryOptionManager(session.getOptions());
@@ -92,9 +91,10 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext {
     queryContextInfo = Utilities.createQueryContextInfo(session.getDefaultSchemaName());
     contextInformation = new ContextInformation(session.getCredentials(), queryContextInfo);
 
-    allocator = drillbitContext.getAllocator().getChildAllocator(null, plannerSettings.getInitialPlanningMemorySize(),
-        plannerSettings.getPlanningMemoryLimit(), false);
-    // 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();

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java
index ee6475c..f720f8e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BaseRootExec.java
@@ -42,7 +42,7 @@ public abstract class BaseRootExec implements RootExec {
   private List<CloseableRecordBatch> operators;
 
   public BaseRootExec(final FragmentContext fragmentContext, final PhysicalOperator config) throws OutOfMemoryException {
-    this.oContext = fragmentContext.newOperatorContext(config, stats, true);
+    this.oContext = fragmentContext.newOperatorContext(config, stats);
     stats = new OperatorStats(new OpProfileDef(config.getOperatorId(),
         config.getOperatorType(), OperatorContext.getChildCount(config)),
         oContext.getAllocator());

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 6c763d6..73bebd1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -134,7 +134,7 @@ public class ScanBatch implements CloseableRecordBatch {
                    Iterator<RecordReader> readers)
       throws ExecutionSetupException {
     this(subScanConfig, context,
-        context.newOperatorContext(subScanConfig, false /* ScanBatch is not subject to fragment memory limit */),
+        context.newOperatorContext(subScanConfig),
         readers, Collections.<String[]> emptyList(), Collections.<Integer> emptyList());
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
index 73f3f95..23e97d0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
@@ -21,7 +21,6 @@ import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.ops.AccountingDataTunnel;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.MetricDef;
@@ -66,7 +65,7 @@ public class SingleSenderCreator implements RootCreator<SingleSender>{
     }
 
     public SingleSenderRootExec(FragmentContext context, RecordBatch batch, SingleSender config) throws OutOfMemoryException {
-      super(context, context.newOperatorContext(config, null, false), config);
+      super(context, context.newOperatorContext(config, null), config);
       this.incoming = batch;
       assert incoming != null;
       handle = context.getHandle();

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java
index c731e38..c88c72d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/broadcastsender/BroadcastSenderRootExec.java
@@ -20,7 +20,6 @@ package org.apache.drill.exec.physical.impl.broadcastsender;
 import java.util.List;
 
 import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.ops.AccountingDataTunnel;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.MetricDef;
@@ -61,7 +60,7 @@ public class BroadcastSenderRootExec extends BaseRootExec {
   public BroadcastSenderRootExec(FragmentContext context,
                                  RecordBatch incoming,
                                  BroadcastSender config) throws OutOfMemoryException {
-    super(context, context.newOperatorContext(config, null, false), config);
+    super(context, context.newOperatorContext(config, null), config);
     this.ok = true;
     this.incoming = incoming;
     this.config = config;

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
index 1cc598a..8db24af 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
@@ -26,6 +26,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.PriorityQueue;
 
+import org.apache.calcite.rel.RelFieldCollation.Direction;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
@@ -76,7 +77,6 @@ import org.apache.drill.exec.vector.AllocationHelper;
 import org.apache.drill.exec.vector.CopyUtil;
 import org.apache.drill.exec.vector.FixedWidthVector;
 import org.apache.drill.exec.vector.ValueVector;
-import org.apache.calcite.rel.RelFieldCollation.Direction;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
@@ -126,8 +126,7 @@ public class MergingRecordBatch extends AbstractRecordBatch<MergingReceiverPOP>
   public MergingRecordBatch(final FragmentContext context,
                             final MergingReceiverPOP config,
                             final RawFragmentBatchProvider[] fragProviders) throws OutOfMemoryException {
-    super(config, context, true, context.newOperatorContext(config, false));
-    //super(config, context);
+    super(config, context, true, context.newOperatorContext(config));
     this.fragProviders = fragProviders;
     this.context = context;
     this.outgoingContainer = new VectorContainer(oContext);

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index 5e9e84c..cfe1b80 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -32,7 +32,6 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.ClassGenerator;
 import org.apache.drill.exec.expr.CodeGenerator;
 import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
-import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.ops.AccountingDataTunnel;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.MetricDef;
@@ -100,7 +99,7 @@ public class PartitionSenderRootExec extends BaseRootExec {
   public PartitionSenderRootExec(FragmentContext context,
                                  RecordBatch incoming,
                                  HashPartitionSender operator) throws OutOfMemoryException {
-    super(context, context.newOperatorContext(operator, null, false), operator);
+    super(context, context.newOperatorContext(operator, null), operator);
     this.incoming = incoming;
     this.operator = operator;
     this.context = context;

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
----------------------------------------------------------------------
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 9f452cb..f2302ce 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
@@ -21,18 +21,9 @@ 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.AutoCloseablePointer;
-import org.apache.drill.common.DrillAutoCloseables;
 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.ops.FragmentContext;
@@ -48,7 +39,6 @@ import org.apache.drill.exec.vector.ValueVector;
 
 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);
@@ -56,12 +46,14 @@ public class SortRecordBatchBuilder implements AutoCloseable {
   private final ArrayListMultimap<BatchSchema, RecordBatchData> batches = ArrayListMultimap.create();
 
   private int recordCount;
-  private final AutoCloseablePointer<SelectionVector4> pSV4 = new AutoCloseablePointer<>();
-  private final AllocationReservation allocationReservation;
   private long runningBatches;
+  private SelectionVector4 sv4;
+  private BufferAllocator allocator;
+  final AllocationReservation reservation;
 
-  public SortRecordBatchBuilder(final BufferAllocator allocator) {
-    allocationReservation = allocator.newReservation();
+  public SortRecordBatchBuilder(BufferAllocator a) {
+    this.allocator = a;
+    this.reservation = a.newReservation();
   }
 
   private long getSize(VectorAccessible batch) {
@@ -94,7 +86,7 @@ public class SortRecordBatchBuilder implements AutoCloseable {
     if (runningBatches >= Character.MAX_VALUE) {
       return false; // allowed in batch.
     }
-    if (!allocationReservation.add(batch.getRecordCount() * 4)) {
+    if (!reservation.add(batch.getRecordCount() * 4)) {
       return false;  // sv allocation available.
     }
 
@@ -117,7 +109,7 @@ public class SortRecordBatchBuilder implements AutoCloseable {
       logger.error(errMsg);
       throw new DrillRuntimeException(errMsg);
     }
-    if (!allocationReservation.add(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);
@@ -160,14 +152,13 @@ public class SortRecordBatchBuilder implements AutoCloseable {
       assert false : "Invalid to have an empty set of batches with no schemas.";
     }
 
-    final DrillBuf svBuffer = allocationReservation.buffer();
+    final DrillBuf svBuffer = reservation.allocateBuffer();
     if (svBuffer == null) {
       throw new OutOfMemoryError("Failed to allocate direct memory for SV4 vector in SortRecordBatchBuilder.");
     }
-    pSV4.assignNoChecked(new SelectionVector4(svBuffer, recordCount, Character.MAX_VALUE));
+    sv4 = new SelectionVector4(svBuffer, recordCount, Character.MAX_VALUE);
     BatchSchema schema = batches.keySet().iterator().next();
     List<RecordBatchData> data = batches.get(schema);
-    final SelectionVector4 sv4 = pSV4.get();
 
     // now we're going to generate the sv4 pointers
     switch (schema.getSelectionVectorMode()) {
@@ -216,16 +207,21 @@ public class SortRecordBatchBuilder implements AutoCloseable {
   }
 
   public SelectionVector4 getSv4() {
-    return pSV4.get();
+    return sv4;
   }
 
   public void clear() {
-    DrillAutoCloseables.closeNoChecked(allocationReservation);
     for (RecordBatchData d : batches.values()) {
       d.container.clear();
     }
-    batches.clear();
-    pSV4.assignNoChecked(null);
+    if (sv4 != null) {
+      sv4.clear();
+    }
+  }
+
+  @Override
+  public void close() {
+    reservation.close();
   }
 
   public List<VectorContainer> getHeldRecordBatches() {
@@ -252,11 +248,4 @@ public class SortRecordBatchBuilder implements AutoCloseable {
     // We need 4 bytes (SV4) for each record.
     return recordCount * 4;
   }
-
-  @Override
-  public void close() {
-    clear();
-    allocationReservation.close();
-    DrillAutoCloseables.closeNoChecked(pSV4);
-  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/trace/TraceRecordBatch.java
----------------------------------------------------------------------
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 78e83d6..ff83cc9 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.common.exceptions.ExecutionSetupException;
 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 @@ public class TraceRecordBatch extends AbstractSingleRecordBatch<Trace> {
 
   private SelectionVector2 sv = null;
 
+  private final BufferAllocator localAllocator;
+
   /* Tag associated with each trace operator */
   final String traceTag;
 
@@ -70,7 +73,7 @@ public class TraceRecordBatch extends AbstractSingleRecordBatch<Trace> {
     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);
     String fileName = getFileName();
 
     /* Create the log file we will dump to and initialize the file descriptors */
@@ -116,7 +119,7 @@ public class TraceRecordBatch extends AbstractSingleRecordBatch<Trace> {
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
-    batch.reconstructContainer(container);
+    batch.reconstructContainer(localAllocator, container);
     if (incomingHasSv2) {
       sv = wrap.getSv2();
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/union/UnionAllRecordBatch.java
----------------------------------------------------------------------
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 0d47a57..55b9afe 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.CodeGenerator;
 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;
@@ -84,14 +83,6 @@ public class UnionAllRecordBatch extends AbstractRecordBatch<UnionAll> {
   }
 
   @Override
-  public void kill(boolean sendUpstream) {
-    if(current != null) {
-      current.kill(sendUpstream);
-      current = null;
-    }
-  }
-
-  @Override
   protected void killIncoming(boolean sendUpstream) {
     unionAllInput.getLeftRecordBatch().kill(sendUpstream);
     unionAllInput.getRightRecordBatch().kill(sendUpstream);
@@ -278,10 +269,8 @@ public class UnionAllRecordBatch extends AbstractRecordBatch<UnionAll> {
 
   // This method is used by inner class to clear the current record batch
   private void clearCurrentRecordBatch() {
-    if (current != null) {
-      for(final VectorWrapper<?> v: current) {
-        v.clear();
-      }
+    for(VectorWrapper<?> v: current) {
+      v.clear();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
index 00cf295..be7d4ed 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/unorderedreceiver/UnorderedReceiverBatch.java
@@ -79,7 +79,7 @@ public class UnorderedReceiverBatch implements CloseableRecordBatch {
     this.context = context;
     // In normal case, batchLoader does not require an allocator. However, in case of splitAndTransfer of a value vector,
     // we may need an allocator for the new offset vector. Therefore, here we pass the context's allocator to batchLoader.
-    oContext = context.newOperatorContext(config, false);
+    oContext = context.newOperatorContext(config);
     this.batchLoader = new RecordBatchLoader(oContext.getAllocator());
 
     this.stats = oContext.getStats();
@@ -159,7 +159,8 @@ public class UnorderedReceiverBatch implements CloseableRecordBatch {
         batch = getNextBatch();
 
         // skip over empty batches. we do this since these are basically control messages.
-        while (batch != null && !batch.getHeader().getIsOutOfMemory() && batch.getHeader().getDef().getRecordCount() == 0 && (!first || batch.getHeader().getDef().getFieldCount() == 0)) {
+        while (batch != null && batch.getHeader().getDef().getRecordCount() == 0
+            && (!first || batch.getHeader().getDef().getFieldCount() == 0)) {
           batch = getNextBatch();
         }
       } finally {
@@ -176,7 +177,7 @@ public class UnorderedReceiverBatch implements CloseableRecordBatch {
         return IterOutcome.NONE;
       }
 
-      if (batch.getHeader().getIsOutOfMemory()) {
+      if (context.isOverMemoryLimit()) {
         return IterOutcome.OUT_OF_MEMORY;
       }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
----------------------------------------------------------------------
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 1ac82bf..4dbd92d 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 class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     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 class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       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 class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
           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 class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
         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);
@@ -587,7 +588,6 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     } finally {
       hyperBatch.clear();
     }
-    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);
@@ -595,25 +595,12 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     return newGroup;
   }
 
-  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;

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopier.java
----------------------------------------------------------------------
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 e0d9c2d..ec590c2 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 interface PriorityQueueCopier extends AutoCloseable {
       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
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/PriorityQueueCopierTemplate.java
----------------------------------------------------------------------
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 891907a..5090b33 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 @@ package org.apache.drill.exec.physical.impl.xsort;
 
 import io.netty.buffer.DrillBuf;
 
+import java.io.IOException;
 import java.util.List;
 
 import javax.inject.Named;
@@ -66,7 +67,6 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
     allocateVectors(targetRecordCount);
     for (int outgoingIndex = 0; outgoingIndex < targetRecordCount; outgoingIndex++) {
       if (queueSize == 0) {
-        close();
         return 0;
       }
       int compoundIndex = vector4.get(0);
@@ -96,7 +96,7 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
   }
 
   @Override
-  public void close() {
+  public void close() throws IOException {
     vector4.clear();
     for (final VectorWrapper<?> w: outgoing) {
       w.getValueVector().clear();
@@ -104,6 +104,10 @@ public abstract class PriorityQueueCopierTemplate implements PriorityQueueCopier
     for (final VectorWrapper<?> w : hyperBatch) {
       w.clear();
     }
+
+    for (BatchGroup batchGroup : batchGroups) {
+      batchGroup.close();
+    }
   }
 
   private void siftUp() {

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
index 229fe65..998665c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java
@@ -46,11 +46,11 @@ public abstract class AbstractRecordBatch<T extends PhysicalOperator> implements
   protected BatchState state;
 
   protected AbstractRecordBatch(final T popConfig, final FragmentContext context) throws OutOfMemoryException {
-    this(popConfig, context, true, context.newOperatorContext(popConfig, true));
+    this(popConfig, context, true, context.newOperatorContext(popConfig));
   }
 
   protected AbstractRecordBatch(final T popConfig, final FragmentContext context, final boolean buildSchema) throws OutOfMemoryException {
-    this(popConfig, context, buildSchema, context.newOperatorContext(popConfig, true));
+    this(popConfig, context, buildSchema, context.newOperatorContext(popConfig));
   }
 
   protected AbstractRecordBatch(final T popConfig, final FragmentContext context, final boolean buildSchema,

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
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 ed86358..13164b9 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.RecordBatchDef;
 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 class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
         newVectors.add(vector);
       }
 
-      Preconditions.checkArgument(buf == null || bufOffset == buf.capacity());
 
       // rebuild the schema.
       final SchemaBuilder builder = BatchSchema.newBuilder();

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
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 488ba69..d39ce5e 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 io.netty.buffer.DrillBuf;
 
 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;
@@ -58,7 +59,7 @@ public class WritableBatch implements AutoCloseable {
     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 class WritableBatch implements AutoCloseable {
         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;

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlClient.java
----------------------------------------------------------------------
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 915509f..cc238f5 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 class ControlClient extends BasicClient<RpcType, ControlConnection, BitCo
   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,

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlServer.java
----------------------------------------------------------------------
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 81c886a..a786469 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 class ControlServer extends BasicServer<RpcType, ControlConnection>{
   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;

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataClient.java
----------------------------------------------------------------------
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 d7611b9..9db551b 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 class DataClient extends BasicClient<RpcType, DataClientConnection, BitCl
   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,

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataConnectionCreator.java
----------------------------------------------------------------------
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 5874b31..b2a51ca 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,9 +21,11 @@ import java.io.Closeable;
 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;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
 
 import com.google.common.collect.Maps;
 import com.google.common.io.Closeables;
@@ -37,20 +39,23 @@ public class DataConnectionCreator implements Closeable {
   private volatile DataServer server;
   private final BootStrapContext context;
   private final WorkEventBus workBus;
-  private final DataResponseHandler dataHandler;
+  private final WorkerBee bee;
   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) {
+  public DataConnectionCreator(BootStrapContext context, WorkEventBus workBus, WorkerBee bee, boolean allowPortHunting) {
     super();
     this.context = context;
     this.workBus = workBus;
-    this.dataHandler = dataHandler;
+    this.bee = bee;
     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, bee);
     int port = server.bind(partialEndpoint.getControlPort() + 1, allowPortHunting);
     DrillbitEndpoint completeEndpoint = partialEndpoint.toBuilder().setDataPort(port).build();
     return completeEndpoint;
@@ -68,6 +73,7 @@ public class DataConnectionCreator implements Closeable {
   @Override
   public void close() {
     Closeables.closeQuietly(server);
+    dataAllocator.close();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandler.java
deleted file mode 100644
index 39dcc90..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandler.java
+++ /dev/null
@@ -1,34 +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.rpc.data;
-
-import io.netty.buffer.DrillBuf;
-
-import java.io.IOException;
-
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
-import org.apache.drill.exec.work.fragment.FragmentManager;
-
-public interface DataResponseHandler {
-
-  public void handle(FragmentManager manager, FragmentRecordBatch fragmentBatch,
-      DrillBuf data, AckSender sender) throws FragmentSetupException, IOException;
-
-  public void informOutOfMemory();
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandlerImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandlerImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandlerImpl.java
deleted file mode 100644
index e18b94c..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataResponseHandlerImpl.java
+++ /dev/null
@@ -1,58 +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.rpc.data;
-
-import io.netty.buffer.DrillBuf;
-
-import java.io.IOException;
-
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
-import org.apache.drill.exec.record.RawFragmentBatch;
-import org.apache.drill.exec.work.WorkManager.WorkerBee;
-import org.apache.drill.exec.work.fragment.FragmentManager;
-
-public class DataResponseHandlerImpl implements DataResponseHandler{
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DataResponseHandlerImpl.class);
-  private final WorkerBee bee;
-
-  public DataResponseHandlerImpl(final WorkerBee bee) {
-    this.bee = bee;
-  }
-
-  @Override
-  public void informOutOfMemory() {
-    logger.error("Out of memory outside any particular fragment.");
-  }
-
-  @Override
-  public void handle(final FragmentManager manager, final FragmentRecordBatch fragmentBatch,
-      final DrillBuf data, final AckSender sender) throws FragmentSetupException, IOException {
-//      logger.debug("Fragment Batch received {}", fragmentBatch);
-
-    final boolean canRun = manager.handle(new RawFragmentBatch(fragmentBatch, data, sender));
-    if (canRun) {
-//    logger.debug("Arriving batch means local batch can run, starting local batch.");
-      /*
-       * If we've reached the canRun threshold, we'll proceed. This expects handler.handle() to
-       * only return a single true.
-       */
-      bee.startFragmentPendingRemote(manager);
-    }
-  }
-}


[04/13] drill git commit: DRILL-4134: Add new allocator

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
----------------------------------------------------------------------
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 b85502b..d244b26 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,28 +27,40 @@ import java.nio.ByteOrder;
 import java.nio.channels.GatheringByteChannel;
 import java.nio.channels.ScatteringByteChannel;
 import java.nio.charset.Charset;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.IdentityHashMap;
+import java.util.LinkedList;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.drill.common.HistoricalLog;
 import org.apache.drill.exec.memory.Accountor;
-import org.apache.drill.exec.memory.BoundsChecking;
+import org.apache.drill.exec.memory.BaseAllocator;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.BufferManager;
-
+import org.apache.drill.exec.memory.BufferLedger;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.util.AssertionUtil;
+import org.apache.drill.exec.util.Pointer;
+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 boolean BOUNDS_CHECKING_ENABLED = AssertionUtil.BOUNDS_CHECKING_ENABLED;
+  private static final boolean DEBUG = BaseAllocator.isDebug();
+  private static final AtomicInteger idGenerator = new AtomicInteger(0);
 
-  private final ByteBuf b;
+  private final ByteBuf byteBuf;
   private final long addr;
   private final int offset;
-  private final boolean rootBuffer;
-  private final AtomicLong rootRefCnt = new AtomicLong(1);
+  private final int flags;
+  private final AtomicInteger rootRefCnt;
   private volatile BufferAllocator allocator;
-  private volatile Accountor acct;
-  private volatile int length;
 
   // TODO - cleanup
   // The code is partly shared and partly copy-pasted between
@@ -56,28 +68,153 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   // 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 final boolean oldWorld; // Indicates that we're operating with TopLevelAllocator.
+  private final boolean rootBuffer;
+  private volatile Accountor acct;
   private BufferManager bufManager;
+  @Deprecated private OperatorContext operatorContext;
+  @Deprecated private FragmentContext fragmentContext;
+
+  private volatile BufferLedger bufferLedger;
+  private volatile int length; // TODO this just seems to duplicate .capacity()
+
+  // members used purely for debugging
+  // TODO once we have a reduced number of constructors, move these to DEBUG clauses in them
+  private final int id = idGenerator.incrementAndGet();
+  private final HistoricalLog historicalLog = DEBUG ? new HistoricalLog(4, "DrillBuf[%d]", id) : null;
+  private final static IdentityHashMap<UnsafeDirectLittleEndian, Collection<DrillBuf>> unwrappedMap =
+      DEBUG ? new IdentityHashMap<UnsafeDirectLittleEndian, Collection<DrillBuf>>() : null;
+
+  // TODO(cwestin) javadoc
+  private void unwrappedPut() {
+    final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) byteBuf;
+    synchronized(unwrappedMap) {
+      Collection<DrillBuf> drillBufs = unwrappedMap.get(udle);
+      if (drillBufs == null) {
+        drillBufs = new LinkedList<DrillBuf>();
+        unwrappedMap.put(udle, drillBufs);
+      }
+
+      drillBufs.add(this);
+    }
+  }
+
+  // TODO(cwestin) javadoc
+  public static Collection<DrillBuf> unwrappedGet(final UnsafeDirectLittleEndian udle) {
+    synchronized(unwrappedMap) {
+      final Collection<DrillBuf> drillBufs = unwrappedMap.get(udle);
+      if (drillBufs == null) {
+        return Collections.emptyList();
+      }
+      return new LinkedList<DrillBuf>(drillBufs);
+    }
+  }
+
+  // TODO(cwestin) javadoc
+  private static boolean unwrappedRemove(final DrillBuf drillBuf) {
+    final ByteBuf byteBuf = drillBuf.unwrap();
+    if (!(byteBuf instanceof UnsafeDirectLittleEndian)) {
+      return false;
+    }
+
+    final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) byteBuf;
+    synchronized(unwrappedMap) {
+      Collection<DrillBuf> drillBufs = unwrappedMap.get(udle);
+      if (drillBufs == null) {
+        return false;
+      }
+      final Object object = drillBufs.remove(drillBuf);
+      if (drillBufs.isEmpty()) {
+        unwrappedMap.remove(udle);
+      }
+      return object != null;
+    }
+  }
 
   public DrillBuf(BufferAllocator allocator, Accountor a, UnsafeDirectLittleEndian b) {
     super(b.maxCapacity());
-    this.b = b;
+    this.byteBuf = b;
     this.addr = b.memoryAddress();
     this.acct = a;
     this.length = b.capacity();
     this.offset = 0;
     this.rootBuffer = true;
     this.allocator = allocator;
+
+    // members from the new world order
+    flags = 0;
+    rootRefCnt = null;
+    oldWorld = true;
+  }
+
+  // TODO(cwestin) javadoc
+  public DrillBuf(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator,
+      final UnsafeDirectLittleEndian byteBuf) {
+    super(byteBuf.maxCapacity());
+    this.byteBuf = byteBuf;
+    byteBuf.retain(1);
+    this.bufferLedger = bufferLedger;
+    addr = byteBuf.memoryAddress();
+    allocator = bufferAllocator;
+    length = byteBuf.capacity();
+    offset = 0;
+    flags = 0;
+    rootRefCnt = new AtomicInteger(1);
+    oldWorld = false;
+
+    // members from the old world order
+    rootBuffer = false;
+    acct = null;
+
+    if (DEBUG) {
+      unwrappedPut();
+      historicalLog.recordEvent(
+          "DrillBuf(BufferLedger, BufferAllocator[%d], UnsafeDirectLittleEndian[identityHashCode == "
+              + "%d](%s)) => rootRefCnt identityHashCode == %d",
+              bufferAllocator.getId(), System.identityHashCode(byteBuf), byteBuf.toString(),
+              System.identityHashCode(rootRefCnt));
+    }
   }
 
   private DrillBuf(BufferAllocator allocator, Accountor a) {
     super(0);
-    this.b = new EmptyByteBuf(allocator.getUnderlyingAllocator()).order(ByteOrder.LITTLE_ENDIAN);
+    this.byteBuf = 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;
+
+    // members from the new world order
+    flags = 0;
+    rootRefCnt = null;
+    oldWorld = true;
+  }
+
+  private DrillBuf(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator) {
+    super(0);
+    this.bufferLedger = bufferLedger;
+    allocator = bufferAllocator;
+
+    byteBuf = new EmptyByteBuf(bufferLedger.getUnderlyingAllocator()).order(ByteOrder.LITTLE_ENDIAN);
+    length = 0;
+    addr = 0;
+    flags = 0;
+    rootRefCnt = new AtomicInteger(1);
+    offset = 0;
+
+    // members from the old world order
+    rootBuffer = false;
+    acct = null;
+    oldWorld = false;
+
+    if (DEBUG) {
+      // We don't put the empty buffers in the unwrappedMap.
+      historicalLog.recordEvent(
+          "DrillBuf(BufferLedger, BufferAllocator[%d])  => rootRefCnt identityHashCode == %d",
+          bufferAllocator.getId(), System.identityHashCode(rootRefCnt));
+    }
   }
 
   /**
@@ -96,7 +233,6 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     b.unwrap().unwrap().retain();
   }
 
-
   private DrillBuf(DrillBuf buffer, int index, int length) {
     this(buffer.allocator, null, buffer, buffer, index, length, false);
   }
@@ -105,6 +241,7 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     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) {
@@ -114,16 +251,138 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     this.length = length;
     writerIndex(length);
 
-    this.b = replacement;
+    this.byteBuf = replacement;
     this.addr = buffer.memoryAddress() + index;
     this.offset = index;
     this.acct = a;
     this.length = length;
     this.rootBuffer = root;
     this.allocator = allocator;
+
+    // members from the new world order
+    flags = 0;
+    rootRefCnt = null;
+    oldWorld = true;
   }
 
+  /**
+   * Indicate a shared refcount, as per http://netty.io/wiki/reference-counted-objects.html#wiki-h3-5
+   */
+  private final static int F_DERIVED = 0x0002;
+
+  // TODO(cwestin) javadoc
+  /**
+   * Used for sharing.
+   *
+   * @param bufferLedger
+   * @param bufferAllocator
+   * @param originalBuf
+   * @param index
+   * @param length
+   * @param flags
+   */
+  public DrillBuf(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator,
+      final DrillBuf originalBuf, final int index, final int length, final int flags) {
+    this(bufferAllocator, bufferLedger, getUnderlyingUdle(originalBuf),
+        originalBuf, index + originalBuf.offset, length, flags);
+  }
+
+  /**
+   * Unwraps a DrillBuf until the underlying UnsafeDirectLittleEndian buffer is
+   * found.
+   *
+   * @param originalBuf the original DrillBuf
+   * @return the underlying UnsafeDirectLittleEndian ByteBuf
+   */
+  private static ByteBuf getUnderlyingUdle(final DrillBuf originalBuf) {
+    int count = 1;
+    ByteBuf unwrapped = originalBuf.unwrap();
+    while(!(unwrapped instanceof UnsafeDirectLittleEndian)
+        && (!(unwrapped instanceof EmptyByteBuf))) {
+      unwrapped = unwrapped.unwrap();
+      ++count;
+    }
+
+    if (DEBUG) {
+      if (count > 1) {
+        throw new IllegalStateException("UnsafeDirectLittleEndian is wrapped more than one level");
+      }
+    }
+
+    return unwrapped;
+  }
+
+  // TODO(cwestin) javadoc
+  /*
+   * TODO the replacement argument becomes an UnsafeDirectLittleEndian;
+   * buffer argument may go away if it is determined to be unnecessary after all
+   * the deprecated stuff is removed (I suspect only the replacement argument is
+   * necessary then).
+   */
+  private DrillBuf(BufferAllocator allocator, BufferLedger bufferLedger,
+      ByteBuf replacement, DrillBuf buffer, int index, int length, int flags) {
+    super(replacement.maxCapacity());
+
+    // members from the old world order
+    rootBuffer = false;
+    acct = null;
+    oldWorld = false;
+
+    if (index < 0 || index > (replacement.maxCapacity() - length)) {
+      throw new IndexOutOfBoundsException(replacement.toString() + ".slice(" + index + ", " + length + ')');
+    }
+
+    this.flags = flags;
+
+    this.length = length; // capacity()
+    writerIndex(length);
+
+    byteBuf = replacement;
+    if ((flags & F_DERIVED) == 0) {
+      replacement.retain(1);
+    }
+
+    addr = replacement.memoryAddress() + index;
+    offset = index;
+    this.bufferLedger = bufferLedger;
+    if (!(buffer instanceof DrillBuf)) {
+      throw new IllegalArgumentException("DrillBuf slicing can only be performed on other DrillBufs");
+    }
+
+    if ((flags & F_DERIVED) != 0) {
+      final DrillBuf rootBuf = (DrillBuf) buffer;
+      rootRefCnt = rootBuf.rootRefCnt;
+    } else {
+      rootRefCnt = new AtomicInteger(1);
+    }
+
+    this.allocator = allocator;
+
+    if (DEBUG) {
+      unwrappedPut();
+      historicalLog.recordEvent(
+          "DrillBuf(BufferAllocator[%d], BufferLedger, ByteBuf[identityHashCode == "
+              + "%d](%s), DrillBuf[%d], index = %d, length = %d, flags = 0x%08x)"
+              + " => rootRefCnt identityHashCode == %d",
+          allocator.getId(), System.identityHashCode(replacement), replacement.toString(),
+          buffer.id, index, length, flags, System.identityHashCode(rootRefCnt));
+    }
+  }
+
+  @Deprecated
+  public void setOperatorContext(OperatorContext c) {
+    this.operatorContext = c;
+  }
+
+  @Deprecated
+  public void setFragmentContext(FragmentContext c) {
+    this.fragmentContext = c;
+  }
+
+  // TODO(DRILL-3331)
   public void setBufferManager(BufferManager bufManager) {
+    Preconditions.checkState(this.bufManager == null,
+        "the BufferManager for a buffer can only be set once");
     this.bufManager = bufManager;
   }
 
@@ -131,47 +390,50 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     return allocator;
   }
 
-  public DrillBuf reallocIfNeeded(int size) {
+  public DrillBuf reallocIfNeeded(final int size) {
+    Preconditions.checkArgument(size >= 0, "reallocation size must be non-negative");
+
     if (this.capacity() >= size) {
       return this;
     }
 
-    if (bufManager != null) {
+    if (operatorContext != null) {
+      return operatorContext.replace(this, size);
+    } else if(fragmentContext != null) {
+      return fragmentContext.replace(this, size);
+    } else if (bufManager != null) {
       return bufManager.replace(this, 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 (oldWorld) {
+      if(rootBuffer){
+        return (int) this.rootRefCnt.get();
+      }else{
+        return byteBuf.refCnt();
+      }
     }
 
+    return rootRefCnt.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 (DEBUG) {
+        historicalLog.logHistory(logger);
+      }
       throw new IndexOutOfBoundsException(String.format(
               "index: %d, length: %d (expected: range(0, %d))", index, fieldLength, capacity()));
     }
@@ -186,7 +448,7 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
    * @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){
+  public void checkBytes(int start, int end) {
     if (BOUNDS_CHECKING_ENABLED) {
       checkIndexD(start, end - start);
     }
@@ -198,18 +460,51 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     }
   }
 
-  private void chk(int index) {
-    if (BOUNDS_CHECKING_ENABLED) {
-      checkIndexD(index);
-    }
-  }
-
   private void ensure(int width) {
     if (BOUNDS_CHECKING_ENABLED) {
       ensureWritable(width);
     }
   }
 
+  /**
+   * Used by allocators to transfer ownership from one allocator to another.
+   *
+   * @param newLedger the new ledger the buffer should use going forward
+   * @param newAllocator the new allocator
+   * @return whether or not the buffer fits the receiving allocator's allocation limit
+   */
+  public boolean transferTo(final BufferAllocator newAllocator, final BufferLedger newLedger) {
+    final Pointer<BufferLedger> pNewLedger = new Pointer<>(newLedger);
+    final boolean fitsAllocation = bufferLedger.transferTo(newAllocator, pNewLedger, this);
+    allocator = newAllocator;
+    bufferLedger = pNewLedger.value;
+    return fitsAllocation;
+  }
+
+  /**
+   * DrillBuf's implementation of sharing buffer functionality, to be accessed from
+   * {@link BufferAllocator#shareOwnership(DrillBuf, Pointer)}. See that function
+   * for more information.
+   *
+   * @param otherLedger the ledger belonging to the other allocator to share with
+   * @param otherAllocator the other allocator to be shared with
+   * @param index the starting index (for slicing capability)
+   * @param length the length (for slicing capability)
+   * @return the new DrillBuf (wrapper)
+   */
+  public DrillBuf shareWith(final BufferLedger otherLedger, final BufferAllocator otherAllocator,
+      final int index, final int length) {
+    return shareWith(otherLedger, otherAllocator, index, length, 0);
+  }
+
+  // TODO(cwestin) javadoc
+  private DrillBuf shareWith(final BufferLedger otherLedger, final BufferAllocator otherAllocator,
+      final int index, final int length, final int flags) {
+    final Pointer<DrillBuf> pDrillBuf = new Pointer<>();
+    bufferLedger = bufferLedger.shareWith(pDrillBuf, otherLedger, otherAllocator, this, index, length, flags);
+    return pDrillBuf.value;
+  }
+
   public boolean transferAccounting(Accountor target) {
     if (rootBuffer) {
       boolean outcome = acct.transferTo(target, this, length);
@@ -221,7 +516,7 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   }
 
   @Override
-  public synchronized boolean release() {
+  public boolean release() {
     return release(1);
   }
 
@@ -230,20 +525,44 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
    */
   @Override
   public synchronized boolean release(int decrement) {
+    Preconditions.checkArgument(decrement > 0,
+        "release(%d) argument is not positive", decrement);
+    if (DEBUG) {
+      historicalLog.recordEvent("release(%d)", decrement);
+    }
 
-    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;
+    if (oldWorld) {
+      if(rootBuffer){
+        final long newRefCnt = this.rootRefCnt.addAndGet(-decrement);
+        Preconditions.checkArgument(newRefCnt > -1, "Buffer has negative reference count.");
+        if (newRefCnt == 0) {
+          byteBuf.release(decrement);
+          acct.release(this, length);
+          return true;
+        }else{
+          return false;
+        }
       }else{
-        return false;
+        return byteBuf.release(decrement);
       }
-    }else{
-      return b.release(decrement);
     }
+
+    final int refCnt = rootRefCnt.addAndGet(-decrement);
+    Preconditions.checkState(refCnt >= 0, "DrillBuf[%d] refCnt has gone negative", id);
+    if (refCnt == 0) {
+      bufferLedger.release(this);
+
+      if (DEBUG) {
+        unwrappedRemove(this);
+      }
+
+      // release the underlying buffer
+      byteBuf.release(1);
+
+      return true;
+    }
+
+    return false;
   }
 
   @Override
@@ -253,31 +572,45 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
 
   @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;
+    if (oldWorld) {
+      if (rootBuffer) {
+        if (newCapacity == length) {
+          return this;
+        } else if (newCapacity < length) {
+          byteBuf.capacity(newCapacity);
+          int diff = length - byteBuf.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("Accounting byte buf doesn't support increasing allocations.");
+        throw new UnsupportedOperationException("Non root bufs doen't support changing allocations.");
       }
-    } else {
-      throw new UnsupportedOperationException("Non root bufs doen't support changing allocations.");
     }
-  }
 
-  @Override
-  public int maxCapacity() {
-    return length;
+    if ((flags & F_DERIVED) != 0) {
+      throw new UnsupportedOperationException("Derived buffers don't support resizing.");
+    }
+
+    if (newCapacity == length) {
+      return this;
+    }
+
+    if (newCapacity < length) {
+      byteBuf.capacity(newCapacity);
+      final int diff = length - byteBuf.capacity();
+      length -= diff;
+      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 +620,12 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
 
   @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 +640,7 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
 
   @Override
   public ByteBuf readSlice(int length) {
-    ByteBuf slice = slice(readerIndex(), length);
+    final ByteBuf slice = slice(readerIndex(), length);
     readerIndex(readerIndex() + length);
     return slice;
   }
@@ -329,16 +660,42 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     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;
+    if (oldWorld) {
+      DrillBuf buf = new DrillBuf(this, index, length);
+      buf.writerIndex = length;
+      return buf;
+    }
+
+    /*
+     * 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 buf = shareWith(bufferLedger, allocator, index, length, F_DERIVED);
+    buf.writerIndex(length);
     return buf;
   }
 
   @Override
   public DrillBuf duplicate() {
-    return new DrillBuf(this, 0, length);
+    if (oldWorld) {
+      return new DrillBuf(this, 0, length);
+    }
+
+    return slice(0, length);
   }
 
   @Override
@@ -353,12 +710,12 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
 
   @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 +730,17 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
 
   @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
@@ -397,17 +754,26 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   }
 
   @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, rootRefCnt identityHashCode == %d}",
+            id, System.identityHashCode(byteBuf), System.identityHashCode(rootRefCnt));
+
     if (length == 0) {
-      return "";
+      return basics;
     }
 
-    ByteBuffer nioBuffer;
+    final ByteBuffer nioBuffer;
     if (nioBufferCount() == 1) {
       nioBuffer = nioBuffer(index, length);
     } else {
@@ -416,7 +782,7 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
       nioBuffer.flip();
     }
 
-    return ByteBufUtil.decodeString(nioBuffer, charset);
+    return basics + '\n' + ByteBufUtil.decodeString(nioBuffer, charset);
   }
 
   @Override
@@ -431,12 +797,22 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   }
 
   @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 (DEBUG) {
+      historicalLog.recordEvent("retain(%d)", increment);
     }
+
+    if (oldWorld) {
+      if(rootBuffer){
+        this.rootRefCnt.addAndGet(increment);
+      }else{
+        byteBuf.retain(increment);
+      }
+      return this;
+    }
+
+    rootRefCnt.addAndGet(increment);
     return this;
   }
 
@@ -448,7 +824,7 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   @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 +851,7 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   @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 +959,13 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
 
   @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 +1033,19 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
 
   @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 +1053,12 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
 
   @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 +1069,12 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
           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 +1083,24 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
 
   @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
@@ -737,8 +1113,33 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     return new DrillBuf(allocator, a);
   }
 
+  public static DrillBuf getEmpty(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator) {
+    return new DrillBuf(bufferLedger, bufferAllocator);
+  }
+
+  /**
+   * Find out if this is a "root buffer." This is obsolete terminology
+   * based on the original implementation of DrillBuf, which would layer
+   * DrillBufs on top of other DrillBufs when slicing (or duplicating).
+   * The buffer at the bottom of the layer was the "root buffer." However,
+   * the current implementation flattens such references to always make
+   * DrillBufs that are wrap a single buffer underneath, and slices and
+   * their original source have a shared fate as per
+   * http://netty.io/wiki/reference-counted-objects.html#wiki-h3-5, so
+   * this concept isn't really meaningful anymore. But there are callers
+   * that want to know a buffer's original size, and whether or not it
+   * is "primal" in some sense. Perhaps this just needs a new name that
+   * indicates that the buffer was an "original" and not a slice.
+   *
+   * @return whether or not the buffer is an original
+   */
+  @Deprecated
   public boolean isRootBuffer() {
-    return rootBuffer;
+    if (oldWorld) {
+      return rootBuffer;
+    }
+
+    return (flags & F_DERIVED) == 0;
   }
 
   @Override
@@ -746,4 +1147,72 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     release();
   }
 
+  /**
+   * Indicates whether this DrillBuf and the supplied one have a "shared fate."
+   * Having a "shared fate" indicates that the two DrillBufs share a reference
+   * count, and will both be released at the same time if either of them is
+   * released.
+   * @param otherBuf the other buffer to check against
+   * @return true if the two buffers have a shared fate, false otherwise
+   */
+  public boolean hasSharedFate(final DrillBuf otherBuf) {
+    return rootRefCnt == otherBuf.rootRefCnt;
+  }
+
+  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 int 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 void logHistoryForUdle(final Logger logger, final UnsafeDirectLittleEndian udle) {
+    final Collection<DrillBuf> drillBufs = unwrappedGet(udle);
+    for(final DrillBuf drillBuf : drillBufs) {
+      drillBuf.logHistory(logger);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/base/src/main/java/io/netty/buffer/FakeAllocator.java
----------------------------------------------------------------------
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
index 53ca91c..b8d0fb2 100644
--- a/exec/memory/base/src/main/java/io/netty/buffer/FakeAllocator.java
+++ b/exec/memory/base/src/main/java/io/netty/buffer/FakeAllocator.java
@@ -17,10 +17,11 @@
  */
 package io.netty.buffer;
 
-import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.memory.Accountor;
+import org.apache.drill.exec.memory.AllocationReservation;
+import org.apache.drill.exec.memory.AllocatorOwner;
 import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.memory.LimitConsumer;
+import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.util.Pointer;
 
 class FakeAllocator implements BufferAllocator {
@@ -46,7 +47,7 @@ class FakeAllocator implements BufferAllocator {
   }
 
   @Override
-  public BufferAllocator getChildAllocator(LimitConsumer consumer, long initialReservation, long maximumReservation,
+  public BufferAllocator getChildAllocator(FragmentContext context, long initialReservation, long maximumReservation,
                                            boolean applyFragmentLimit)
       throws OutOfMemoryException {
     throw new UnsupportedOperationException();
@@ -63,22 +64,12 @@ class FakeAllocator implements BufferAllocator {
   }
 
   @Override
-  public PreAllocator getNewPreAllocator() {
+  public void setFragmentLimit(long l) {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public void resetLimits() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void setLimit(long l) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public long getLimit(){
+  public long getFragmentLimit(){
     throw new UnsupportedOperationException();
   }
 
@@ -96,9 +87,10 @@ class FakeAllocator implements BufferAllocator {
     return 0;
   }
 
-  static class FakeAccountor implements Accountor {
+  static class FakeAccountor extends Accountor {
 
     public FakeAccountor() {
+      super(null, false, null, null, 0, 0, true);
     }
 
     @Override
@@ -138,50 +130,38 @@ class FakeAllocator implements BufferAllocator {
 
     @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;
-    }
+    public void release(DrillBuf buf, long size) {
 
-    @Override
-    public long resetFragmentLimits() {
-      return 0;
     }
 
     @Override
-    public void setFragmentLimit(long add) {
-      throw new UnsupportedOperationException();
-    }
+    public void close() {
 
-    @Override
-    public long getFragmentLimit() {
-      return 0;
     }
+  }
 
+  @Override
+  public BufferAllocator newChildAllocator(AllocatorOwner allocatorOwner,
+      long initReservation, long maxAllocation, int flags) {
+    throw new UnsupportedOperationException();
+  }
 
+  @Override
+  public boolean shareOwnership(DrillBuf buf, Pointer<DrillBuf> bufOut) {
+    throw new UnsupportedOperationException();
   }
 
   @Override
-  public boolean takeOwnership(DrillBuf buf, Pointer<DrillBuf> bufOut) {
+  public int getId() {
     throw new UnsupportedOperationException();
   }
 
+  @Override
+  public AllocationReservation newReservation() {
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java
----------------------------------------------------------------------
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 559f06d..419aef3 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
@@ -27,10 +27,68 @@ 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 IdentityHashMap<UnsafeDirectLittleEndian, StackTrace> bufferMap = new IdentityHashMap<>();
+
+  @Override
+  public boolean release() {
+    return release(1);
+  }
+
+  @Override
+  public boolean release(int decrement) {
+    boolean released = super.release(decrement);
+    if (TRACK_BUFFERS) {
+      if (released) {
+        final Object object;
+        synchronized (bufferMap) {
+          object = bufferMap.remove(this);
+        }
+        if (object == null) {
+          throw new IllegalStateException("no such buffer");
+        }
+
+        if (initCap != -1) {
+          bufferCount.decrementAndGet();
+          bufferSize.addAndGet(-initCap);
+        }
+      }
+    }
+
+    return released;
+  }
+
+
+  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) {
+    synchronized (bufferMap) {
+      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(LargeBuffer buf) {
     this(buf, true);
   }
@@ -50,7 +108,7 @@ public final class UnsafeDirectLittleEndian extends WrappedByteBuf {
       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;

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
----------------------------------------------------------------------
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 a708e92..7d14b94 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,167 @@ package org.apache.drill.exec.memory;
 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.ops.FragmentContext;
 import org.apache.drill.exec.util.Pointer;
 
 /**
- * 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
+   * @param size The size in bytes.
+   * @return a new DrillBuf, or null if the request can't be satisfied
+   * @throws OutOfMemoryRuntimeException 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.
+   * 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
+   * @return a new DrillBuf, or null if the request can't be satisfied
+   * @throws OutOfMemoryRuntimeException if buffer cannot be allocated
    */
-  public abstract DrillBuf buffer(int minSize, int maxSize);
+  public DrillBuf buffer(int minSize, int maxSize);
 
-  public abstract ByteBufAllocator getUnderlyingAllocator();
+  /**
+   * Returns the allocator this allocator falls back to when it needs more memory.
+   *
+   * @return the underlying allocator used by this allocator
+   */
+  public ByteBufAllocator getUnderlyingAllocator();
 
   /**
    * Create a child allocator nested below this one.
    *
-   * @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
+   * @param context - the owner or this 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
    */
-  public abstract BufferAllocator getChildAllocator(LimitConsumer limitListener, long initialReservation,
-      long maximumReservation, boolean applyFragmentLimit) throws OutOfMemoryException;
+  @Deprecated
+  public BufferAllocator getChildAllocator(FragmentContext context, long initialReservation,
+      long maximumReservation, boolean applyFragmentLimit);
 
   /**
-   * Take over ownership of fragment accounting.  Always takes over ownership.
-   * @param buf
-   * @return false if over allocation.
+   * Flag: this allocator is a limiting sub-tree root, meaning that the maxAllocation for
+   * it applies to all its descendant child allocators. In low memory situations, the limits
+   * for sub-tree roots may be adjusted down so that they evenly share the total amount of
+   * direct memory across all the sub-tree roots.
    */
-  public boolean takeOwnership(DrillBuf buf) ;
+  public final static int F_LIMITING_ROOT = 0x0001;
 
   /**
-   * Take over ownership of fragment accounting.  Always takes over ownership.
-   * @param buf
-   * @return false if over allocation.
+   * Create a new child allocator.
+   *
+   * @param allocatorOwner the allocator owner
+   * @param initReservation the initial space reservation (obtained from this allocator)
+   * @param maxAllocation maximum amount of space the new allocator can allocate
+   * @param flags one or more of BufferAllocator.F_* flags
+   * @return the new allocator, or null if it can't be created
    */
-  public boolean takeOwnership(DrillBuf buf, Pointer<DrillBuf> bufOut);
-
-  public PreAllocator getNewPreAllocator();
-
-  //public void addFragmentContext(FragmentContext c);
+  public BufferAllocator newChildAllocator(AllocatorOwner allocatorOwner,
+      long initReservation, long maxAllocation, int flags);
 
   /**
-   * For Top Level Allocators. Reset the fragment limits for all allocators
+   * Take over ownership of the given buffer, adjusting accounting accordingly.
+   * This allocator always takes over ownership.
+   *
+   * @param buf the buffer to take over
+   * @return false if over allocation
    */
-  public void resetLimits();
+  public boolean takeOwnership(DrillBuf buf);
 
   /**
-   * For Child allocators to set the Fragment limit for the corresponding fragment allocator.
-   * @param l the new fragment limit
+   * Share ownership of a buffer between allocators.
+   *
+   * @param buf the buffer
+   * @param bufOut a new DrillBuf owned by this allocator, but sharing the same underlying buffer
+   * @return false if over allocation.
    */
-  public void setLimit(long l);
-
-  public long getLimit();
-
+  public boolean shareOwnership(DrillBuf buf, Pointer<DrillBuf> bufOut);
 
   /**
    * Not thread safe.
    *
    * WARNING: unclaimed pre-allocations leak memory. If you call preAllocate(), you must
    * make sure to ultimately try to get the buffer and release it.
+   *
+   * For Child allocators to set their Fragment limits.
+   *
+   * @param fragmentLimit the new fragment limit
    */
-  public interface PreAllocator {
-    public boolean preAllocate(int bytes);
-
-    public DrillBuf getAllocation();
-  }
+  @Deprecated // happens automatically, and via allocation policies
+  public void setFragmentLimit(long fragmentLimit);
 
   /**
-   * @param bytes
-   * @return
+   * Returns the current fragment limit.
+   *
+   * @return the current fragment limit
+   */
+  /*
+   * TODO should be replaced with something more general because of
+   * the availability of multiple allocation policies
+   *
+   * TODO We should also have a getRemainingMemory() so operators
+   * can query how much more is left to allocate. That could be
+   * tricky.
    */
+  @Deprecated
+  public long getFragmentLimit();
 
   /**
+   * 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
    */
+  public int getId();
 
   /**
    * 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.
    */
   @Override
-  public abstract void close();
+  public void close() throws Exception;
 
-  public abstract long getAllocatedMemory();
+  /**
+   * Returns the amount of memory currently allocated from this allocator.
+   *
+   * @return the amount of memory currently allocated
+   */
+  public long getAllocatedMemory();
 
-  public abstract long getPeakMemoryAllocation();
+  /**
+   * Returns the peak amount of memory allocated from this allocator.
+   *
+   * @return the peak amount of memory allocated
+   */
+  public long getPeakMemoryAllocation();
 
+  /**
+   * Returns an empty DrillBuf.
+   *
+   * @return an empty DrillBuf
+   */
   public DrillBuf getEmpty();
+
+  /**
+   * 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();
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java
new file mode 100644
index 0000000..4f1a1bd
--- /dev/null
+++ b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java
@@ -0,0 +1,38 @@
+/**
+ * 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;
+
+/**
+ * Implicitly specifies an allocation policy by providing a factory method to
+ * create an enforcement agent.
+ *
+ * <p>Allocation policies are meant to be global, and may not work properly if
+ * different allocators are given different policies. These are designed to
+ * be supplied to the root-most allocator only, and then shared with descendant
+ * (child) allocators.</p>
+ */
+public interface AllocationPolicy {
+  /**
+   * Create an allocation policy enforcement agent. Each newly created allocator should
+   * call this in order to obtain its own agent.
+   *
+   * @return the newly instantiated agent; if an agent's implementation is stateless,
+   *   this may return a sharable singleton
+   */
+  AllocationPolicyAgent newAgent();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java
new file mode 100644
index 0000000..ad51ee6
--- /dev/null
+++ b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.memory;
+
+/**
+ * Per-allocator enforcement agent for allocation policies; created by
+ * {@link AllocationPolicy#newAgent()}.
+ */
+public interface AllocationPolicyAgent extends AutoCloseable {
+  /**
+   * Checks to see if creating a new allocator using the given specifications
+   * is allowed; should throw an exception if not.
+   *
+   * @param parentAllocator the parent allocator
+   * @param initReservation initial reservation the allocator should have
+   * @param maxAllocation the maximum allocation the allocator will allow
+   * @param flags the allocation option flags
+   * @throws OutOfMemoryException if the new allocator shouldn't be created
+   */
+  void checkNewAllocator(BufferAllocator parentAllocator,
+      long initReservation, long maxAllocation, int flags);
+
+  /**
+   * Get the currently applicable memory limit for the provided allocator.
+   * The interpretation of this value varies with the allocation policy in
+   * use, and each policy should describe what to expect.
+   *
+   * @param bufferAllocator the allocator
+   * @return the memory limit
+   */
+  long getMemoryLimit(BufferAllocator bufferAllocator);
+
+  /**
+   * Initialize the agent for a newly created allocator. Should be called from
+   * the allocator's constructor to initialize the agent for the allocator.
+   *
+   * @param bufferAllocator the newly created allocator.
+   */
+  void initializeAllocator(BufferAllocator bufferAllocator);
+
+  /**
+   * Indicate if any available memory owned by this allocator should
+   * be released to its parent. Allocators may use this to limit the
+   * amount of unused memory they retain for future requests; agents may
+   * request that memory be returned if there is currently a high demand
+   * for memory that other allocators could use if this allocator
+   * doesn't need it.
+   *
+   * @param bufferAllocator
+   * @return true if available memory owned by this allocator should be given
+   *   back to its parent
+   */
+  boolean shouldReleaseToParent(BufferAllocator bufferAllocator);
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java
new file mode 100644
index 0000000..1803572
--- /dev/null
+++ b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java
@@ -0,0 +1,152 @@
+/**
+ * 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.base.Preconditions;
+
+import io.netty.buffer.DrillBuf;
+
+/**
+ * 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.
+ */
+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.
+   */
+  protected 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");
+
+    if (!reserve(nBytes)) {
+      return false;
+    }
+
+    this.nBytes += nBytes;
+    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
+   */
+  protected 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
+   */
+  protected 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
+   */
+  protected 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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java
new file mode 100644
index 0000000..8bf2a99
--- /dev/null
+++ b/exec/memory/impl/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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java
new file mode 100644
index 0000000..f2d3df9
--- /dev/null
+++ b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java
@@ -0,0 +1,40 @@
+/**
+ * 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.exec.ops.FragmentContext;
+import org.apache.drill.exec.testing.ExecutionControls;
+
+/**
+ * This interface provides a means for allocator owners to inject services
+ * required by allocators, as well as to identify themselves for debugging purposes.
+ * Identification is done by overriding the implementation of
+ * {#link {@link Object#toString()}.
+ */
+public interface AllocatorOwner {
+  /**
+   * Get the current ExecutionControls from the allocator's owner.
+   *
+   * @return the current execution controls; may return null if this isn't
+   *   possible
+   */
+  ExecutionControls getExecutionControls();
+
+  @Deprecated // Only for TopLevelAllocator and its friends.
+  FragmentContext getFragmentContext();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java
new file mode 100644
index 0000000..00d8c4f
--- /dev/null
+++ b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java
@@ -0,0 +1,34 @@
+/**
+ * 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;
+
+/**
+ * JMX bean interface for global allocator statistics.
+ */
+// TODO use Stats infrastructure instead of JMX beans
+public interface AllocatorsStatsMXBean {
+  /**
+   * Get the maximum amount of direct memory that can be used.
+   *
+   * <p>This is determined by what is available, or by the drillbit
+   * configuration, if it specifies a value.</p>
+   *
+   * @return the amount of direct memory that can be used
+   */
+  public long getMaxDirectMemory();
+}


[06/13] drill git commit: DRILL-4134: Allocator Improvements

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java
----------------------------------------------------------------------
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 36bcacf..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java
+++ /dev/null
@@ -1,408 +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 java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.testing.ControlsInjector;
-import org.apache.drill.exec.testing.ControlsInjectorFactory;
-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);
-  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(TopLevelAllocator.class);
-  public static final String CHILD_BUFFER_INJECTION_SITE = "child.buffer";
-
-  public static long MAXIMUM_DIRECT_MEMORY;
-
-  private static final boolean ENABLE_ACCOUNTING = AssertionUtil.isAssertionsEnabled();
-  private final Map<ChildAllocator, StackTraceElement[]> childrenMap;
-  private final PooledByteBufAllocatorL innerAllocator = PooledByteBufAllocatorL.DEFAULT;
-  private final Accountor acct;
-  private final boolean errorOnLeak;
-  private final DrillBuf empty;
-
-  private final AtomicInteger idGenerator = new AtomicInteger(0);
-
-  private TopLevelAllocator(DrillConfig config, long maximumAllocation, boolean errorOnLeak){
-    MAXIMUM_DIRECT_MEMORY = maximumAllocation;
-    this.errorOnLeak = errorOnLeak;
-    this.acct = new Accountor(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(ExecConstants.TOP_LEVEL_MAX_ALLOC)),
-        config.getBoolean(ExecConstants.ERROR_ON_MEMORY_LEAK)
-        );
-  }
-
-  @Override
-  public int getId() {
-    return idGenerator.incrementAndGet();
-  }
-
-  @Override
-  public boolean takeOwnership(DrillBuf buf) {
-    return buf.transferAccounting(acct);
-  }
-
-  @Override
-  public boolean shareOwnership(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 min, int max) {
-    if (min == 0) {
-      return empty;
-    }
-    if(!acct.reserve(min)) {
-      throw new OutOfMemoryRuntimeException(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 OutOfMemoryRuntimeException(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 newChildAllocator(AllocatorOwner allocatorOwner,
-      long initialReservation, long maximumReservation, int flags) {
-    return getChildAllocator(allocatorOwner.getFragmentContext(), initialReservation,
-        maximumReservation, (flags & BufferAllocator.F_LIMITING_ROOT) != 0);
-  }
-
-    @Override
-  public BufferAllocator getChildAllocator(FragmentContext context, 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 OutOfMemoryRuntimeException(
-          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(context, acct, maximumReservation, initialReservation, childrenMap, applyFragmentLimit);
-    if(ENABLE_ACCOUNTING){
-      childrenMap.put(allocator, Thread.currentThread().getStackTrace());
-    }
-
-    return allocator;
-  }
-
-  @Override
-  public void setFragmentLimit(long limit){
-    acct.setFragmentLimit(limit);
-  }
-
-  @Override
-  public long getFragmentLimit(){
-    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 Accountor childAcct;
-    private Map<ChildAllocator, StackTraceElement[]> children = new HashMap<>();
-    private boolean closed = false;
-    private FragmentContext fragmentContext;
-    private Map<ChildAllocator, StackTraceElement[]> thisMap;
-
-    public ChildAllocator(FragmentContext context,
-                          Accountor parentAccountor,
-                          long max,
-                          long pre,
-                          Map<ChildAllocator,
-                          StackTraceElement[]> map,
-        boolean applyFragmentLimit) {
-      assert max >= pre;
-      DrillConfig drillConf = context != null ? context.getConfig() : null;
-      childAcct = new Accountor(drillConf, errorOnLeak, context, parentAccountor, max, pre, applyFragmentLimit);
-      this.fragmentContext=context;
-      thisMap = map;
-      this.empty = DrillBuf.getEmpty(this, childAcct);
-    }
-
-    @Override
-    public int getId() {
-      return idGenerator.incrementAndGet();
-    }
-
-    @Override
-    public boolean takeOwnership(DrillBuf buf) {
-      return buf.transferAccounting(childAcct);
-    }
-
-    @Override
-    public boolean shareOwnership(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 (ENABLE_ACCOUNTING) {
-        injector.injectUnchecked(fragmentContext, CHILD_BUFFER_INJECTION_SITE);
-      }
-
-      if (size == 0) {
-        return empty;
-      }
-      if(!childAcct.reserve(size)) {
-        throw new OutOfMemoryRuntimeException(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 OutOfMemoryRuntimeException(createErrorMsg(this, size), e);
-        } else {
-          throw e;
-        }
-      }
-    }
-
-    @Override
-    public DrillBuf buffer(int size) {
-      return buffer(size, size);
-    }
-
-    @Override
-    public ByteBufAllocator getUnderlyingAllocator() {
-      return innerAllocator;
-    }
-
-    @Override
-    public BufferAllocator newChildAllocator(AllocatorOwner allocatorOwner,
-        long initialReservation, long maximumReservation, int flags) {
-      return getChildAllocator(allocatorOwner.getFragmentContext(), initialReservation,
-          maximumReservation, (flags & BufferAllocator.F_LIMITING_ROOT) != 0);
-    }
-
-    @Override
-    public BufferAllocator getChildAllocator(FragmentContext context,
-        long initialReservation, long maximumReservation, boolean applyFragmentLimit) {
-      if (!childAcct.reserve(initialReservation)) {
-        throw new OutOfMemoryRuntimeException(
-            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(context, childAcct, maximumReservation, initialReservation, null, applyFragmentLimit);
-      this.children.put(newChildAllocator, Thread.currentThread().getStackTrace());
-      return newChildAllocator;
-    }
-
-    @Override
-    public AllocationReservation newReservation() {
-      return new PreAlloc(this, this.childAcct);
-    }
-
-    @Override
-    public void setFragmentLimit(long limit){
-      childAcct.setFragmentLimit(limit);
-    }
-
-    @Override
-    public long getFragmentLimit(){
-      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");
-            }
-
-
-            final FragmentHandle handle = fragmentContext.getHandle();
-            IllegalStateException e = new IllegalStateException(String.format(
-                    "Failure while trying to close child allocator: Child level allocators not closed. Fragment %d:%d. Stack trace: \n %s",
-                    handle.getMajorFragmentId(), handle.getMinorFragmentId(), 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;
-    }
-  }
-
-  @Override
-  public AllocationReservation newReservation() {
-    return new PreAlloc(this, this.acct);
-  }
-
-  public class PreAlloc extends AllocationReservation {
-    int bytes = 0;
-    final Accountor acct;
-    final BufferAllocator allocator;
-    private PreAlloc(BufferAllocator allocator, Accountor acct) {
-      this.acct = acct;
-      this.allocator = allocator;
-    }
-
-    @Override
-    protected boolean reserve(int bytes) {
-      if (!acct.reserve(bytes)) {
-        return false;
-      }
-
-      this.bytes += bytes;
-      return true;
-    }
-
-    @Override
-    protected DrillBuf allocate(int bytes) {
-      assert this.bytes == bytes : "allocation size mismatch";
-      DrillBuf b = new DrillBuf(allocator, acct, innerAllocator.directBuffer(bytes, bytes));
-      acct.reserved(bytes, b);
-      return b;
-    }
-
-    @Override
-    protected void releaseReservation(int nBytes) {
-      acct.release(nBytes);
-    }
-  }
-
-  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());
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/resources/drill-module.conf b/exec/memory/impl/src/main/resources/drill-module.conf
deleted file mode 100644
index 593ef8e..0000000
--- a/exec/memory/impl/src/main/resources/drill-module.conf
+++ /dev/null
@@ -1,25 +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.
-
-//  This file tells Drill to consider this module when class path scanning.
-//  This file can also include any supplementary configuration information.
-//  This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
-drill: {
-  memory: {
-    debug.error_on_leak: true,
-    top.max: 1000000000000
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/test/java/org/apache/drill/exec/memory/TestEndianess.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/test/java/org/apache/drill/exec/memory/TestEndianess.java b/exec/memory/impl/src/test/java/org/apache/drill/exec/memory/TestEndianess.java
deleted file mode 100644
index 2028a23..0000000
--- a/exec/memory/impl/src/test/java/org/apache/drill/exec/memory/TestEndianess.java
+++ /dev/null
@@ -1,43 +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 static org.junit.Assert.assertEquals;
-import io.netty.buffer.ByteBuf;
-
-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 {
-  @Test
-  public void testLittleEndian() {
-    final DrillConfig drillConfig = DrillConfig.create();
-    final BufferAllocator a = RootAllocatorFactory.newRoot(drillConfig);
-    final ByteBuf b = a.buffer(4);
-    b.setInt(0, 35);
-    assertEquals(b.getByte(0), 35);
-    assertEquals(b.getByte(1), 0);
-    assertEquals(b.getByte(2), 0);
-    assertEquals(b.getByte(3), 0);
-    b.release();
-    DrillAutoCloseables.closeNoChecked(a);
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/pom.xml
----------------------------------------------------------------------
diff --git a/exec/memory/pom.xml b/exec/memory/pom.xml
index 2738f34..2a98f1f 100644
--- a/exec/memory/pom.xml
+++ b/exec/memory/pom.xml
@@ -30,6 +30,5 @@
 
   <modules>
     <module>base</module>
-    <module>impl</module>
   </modules>
 </project>

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/rpc/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
----------------------------------------------------------------------
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 1cacf9c..bc79677 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 abstract class BasicClientWithConnection<T extends EnumLite, HANDSHAKE_SE
 
   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;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/vector/src/main/java/org/apache/drill/exec/expr/fn/impl/ByteFunctionHelpers.java
----------------------------------------------------------------------
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 8a330b8..bd06ba5 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 @@ import com.google.common.primitives.UnsignedLongs;
 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 @@ public class ByteFunctionHelpers {
    * @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 @@ public class ByteFunctionHelpers {
    * @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 @@ public class ByteFunctionHelpers {
    * @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);

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/vector/src/main/java/org/apache/drill/exec/util/DecimalUtility.java
----------------------------------------------------------------------
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 0922b22..e8130ec 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 class DecimalUtility extends CoreDecimalUtility{
         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 class DecimalUtility extends CoreDecimalUtility{
      * 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 class DecimalUtility extends CoreDecimalUtility{
         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++) {

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/protocol/src/main/java/org/apache/drill/exec/proto/BitData.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/BitData.java b/protocol/src/main/java/org/apache/drill/exec/proto/BitData.java
index 5de0a07..0731975 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/BitData.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/BitData.java
@@ -1099,16 +1099,6 @@ public final class BitData {
      * <code>optional bool isLastBatch = 7;</code>
      */
     boolean getIsLastBatch();
-
-    // optional bool isOutOfMemory = 8 [default = false];
-    /**
-     * <code>optional bool isOutOfMemory = 8 [default = false];</code>
-     */
-    boolean hasIsOutOfMemory();
-    /**
-     * <code>optional bool isOutOfMemory = 8 [default = false];</code>
-     */
-    boolean getIsOutOfMemory();
   }
   /**
    * Protobuf type {@code exec.bit.data.FragmentRecordBatch}
@@ -1228,11 +1218,6 @@ public final class BitData {
               isLastBatch_ = input.readBool();
               break;
             }
-            case 64: {
-              bitField0_ |= 0x00000040;
-              isOutOfMemory_ = input.readBool();
-              break;
-            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -1407,22 +1392,6 @@ public final class BitData {
       return isLastBatch_;
     }
 
-    // optional bool isOutOfMemory = 8 [default = false];
-    public static final int ISOUTOFMEMORY_FIELD_NUMBER = 8;
-    private boolean isOutOfMemory_;
-    /**
-     * <code>optional bool isOutOfMemory = 8 [default = false];</code>
-     */
-    public boolean hasIsOutOfMemory() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
-    }
-    /**
-     * <code>optional bool isOutOfMemory = 8 [default = false];</code>
-     */
-    public boolean getIsOutOfMemory() {
-      return isOutOfMemory_;
-    }
-
     private void initFields() {
       queryId_ = org.apache.drill.exec.proto.UserBitShared.QueryId.getDefaultInstance();
       receivingMajorFragmentId_ = 0;
@@ -1431,7 +1400,6 @@ public final class BitData {
       sendingMinorFragmentId_ = 0;
       def_ = org.apache.drill.exec.proto.UserBitShared.RecordBatchDef.getDefaultInstance();
       isLastBatch_ = false;
-      isOutOfMemory_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -1466,9 +1434,6 @@ public final class BitData {
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         output.writeBool(7, isLastBatch_);
       }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeBool(8, isOutOfMemory_);
-      }
       getUnknownFields().writeTo(output);
     }
 
@@ -1511,10 +1476,6 @@ public final class BitData {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(7, isLastBatch_);
       }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(8, isOutOfMemory_);
-      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -1655,8 +1616,6 @@ public final class BitData {
         bitField0_ = (bitField0_ & ~0x00000020);
         isLastBatch_ = false;
         bitField0_ = (bitField0_ & ~0x00000040);
-        isOutOfMemory_ = false;
-        bitField0_ = (bitField0_ & ~0x00000080);
         return this;
       }
 
@@ -1722,10 +1681,6 @@ public final class BitData {
           to_bitField0_ |= 0x00000020;
         }
         result.isLastBatch_ = isLastBatch_;
-        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
-          to_bitField0_ |= 0x00000040;
-        }
-        result.isOutOfMemory_ = isOutOfMemory_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1770,9 +1725,6 @@ public final class BitData {
         if (other.hasIsLastBatch()) {
           setIsLastBatch(other.getIsLastBatch());
         }
-        if (other.hasIsOutOfMemory()) {
-          setIsOutOfMemory(other.getIsOutOfMemory());
-        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -2232,39 +2184,6 @@ public final class BitData {
         return this;
       }
 
-      // optional bool isOutOfMemory = 8 [default = false];
-      private boolean isOutOfMemory_ ;
-      /**
-       * <code>optional bool isOutOfMemory = 8 [default = false];</code>
-       */
-      public boolean hasIsOutOfMemory() {
-        return ((bitField0_ & 0x00000080) == 0x00000080);
-      }
-      /**
-       * <code>optional bool isOutOfMemory = 8 [default = false];</code>
-       */
-      public boolean getIsOutOfMemory() {
-        return isOutOfMemory_;
-      }
-      /**
-       * <code>optional bool isOutOfMemory = 8 [default = false];</code>
-       */
-      public Builder setIsOutOfMemory(boolean value) {
-        bitField0_ |= 0x00000080;
-        isOutOfMemory_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bool isOutOfMemory = 8 [default = false];</code>
-       */
-      public Builder clearIsOutOfMemory() {
-        bitField0_ = (bitField0_ & ~0x00000080);
-        isOutOfMemory_ = false;
-        onChanged();
-        return this;
-      }
-
       // @@protoc_insertion_point(builder_scope:exec.bit.data.FragmentRecordBatch)
     }
 
@@ -2305,18 +2224,17 @@ public final class BitData {
       "itShared.proto\"]\n\022BitClientHandshake\022\023\n\013" +
       "rpc_version\030\001 \001(\005\0222\n\007channel\030\002 \001(\0162\027.exe" +
       "c.shared.RpcChannel:\010BIT_DATA\")\n\022BitServ" +
-      "erHandshake\022\023\n\013rpc_version\030\001 \001(\005\"\252\002\n\023Fra" +
+      "erHandshake\022\023\n\013rpc_version\030\001 \001(\005\"\214\002\n\023Fra" +
       "gmentRecordBatch\022&\n\010query_id\030\001 \001(\0132\024.exe" +
       "c.shared.QueryId\022#\n\033receiving_major_frag" +
       "ment_id\030\002 \001(\005\022#\n\033receiving_minor_fragmen" +
       "t_id\030\003 \003(\005\022!\n\031sending_major_fragment_id\030",
       "\004 \001(\005\022!\n\031sending_minor_fragment_id\030\005 \001(\005" +
       "\022(\n\003def\030\006 \001(\0132\033.exec.shared.RecordBatchD" +
-      "ef\022\023\n\013isLastBatch\030\007 \001(\010\022\034\n\risOutOfMemory" +
-      "\030\010 \001(\010:\005false*D\n\007RpcType\022\r\n\tHANDSHAKE\020\000\022" +
-      "\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\024\n\020REQ_RECORD_BATC" +
-      "H\020\003B(\n\033org.apache.drill.exec.protoB\007BitD" +
-      "ataH\001"
+      "ef\022\023\n\013isLastBatch\030\007 \001(\010*D\n\007RpcType\022\r\n\tHA" +
+      "NDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\024\n\020REQ_R" +
+      "ECORD_BATCH\020\003B(\n\033org.apache.drill.exec.p" +
+      "rotoB\007BitDataH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -2340,7 +2258,7 @@ public final class BitData {
           internal_static_exec_bit_data_FragmentRecordBatch_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_bit_data_FragmentRecordBatch_descriptor,
-              new java.lang.String[] { "QueryId", "ReceivingMajorFragmentId", "ReceivingMinorFragmentId", "SendingMajorFragmentId", "SendingMinorFragmentId", "Def", "IsLastBatch", "IsOutOfMemory", });
+              new java.lang.String[] { "QueryId", "ReceivingMajorFragmentId", "ReceivingMinorFragmentId", "SendingMajorFragmentId", "SendingMinorFragmentId", "Def", "IsLastBatch", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitData.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitData.java b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitData.java
index 9803079..5684daf 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitData.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/SchemaBitData.java
@@ -280,8 +280,6 @@ public final class SchemaBitData
 
                 if(message.hasIsLastBatch())
                     output.writeBool(7, message.getIsLastBatch(), false);
-                if(message.hasIsOutOfMemory())
-                    output.writeBool(8, message.getIsOutOfMemory(), false);
             }
             public boolean isInitialized(org.apache.drill.exec.proto.BitData.FragmentRecordBatch message)
             {
@@ -344,9 +342,6 @@ public final class SchemaBitData
                         case 7:
                             builder.setIsLastBatch(input.readBool());
                             break;
-                        case 8:
-                            builder.setIsOutOfMemory(input.readBool());
-                            break;
                         default:
                             input.handleUnknownField(number, this);
                     }
@@ -394,7 +389,6 @@ public final class SchemaBitData
                 case 5: return "sendingMinorFragmentId";
                 case 6: return "def";
                 case 7: return "isLastBatch";
-                case 8: return "isOutOfMemory";
                 default: return null;
             }
         }
@@ -413,7 +407,6 @@ public final class SchemaBitData
             fieldMap.put("sendingMinorFragmentId", 5);
             fieldMap.put("def", 6);
             fieldMap.put("isLastBatch", 7);
-            fieldMap.put("isOutOfMemory", 8);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/protocol/src/main/java/org/apache/drill/exec/proto/beans/FragmentRecordBatch.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/beans/FragmentRecordBatch.java b/protocol/src/main/java/org/apache/drill/exec/proto/beans/FragmentRecordBatch.java
index 61689f3..4b32361 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/beans/FragmentRecordBatch.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/beans/FragmentRecordBatch.java
@@ -48,7 +48,6 @@ public final class FragmentRecordBatch implements Externalizable, Message<Fragme
 
     static final FragmentRecordBatch DEFAULT_INSTANCE = new FragmentRecordBatch();
 
-    static final Boolean DEFAULT_IS_OUT_OF_MEMORY = new Boolean(false);
     
     private QueryId queryId;
     private int receivingMajorFragmentId;
@@ -57,7 +56,6 @@ public final class FragmentRecordBatch implements Externalizable, Message<Fragme
     private int sendingMinorFragmentId;
     private RecordBatchDef def;
     private Boolean isLastBatch;
-    private Boolean isOutOfMemory = DEFAULT_IS_OUT_OF_MEMORY;
 
     public FragmentRecordBatch()
     {
@@ -157,19 +155,6 @@ public final class FragmentRecordBatch implements Externalizable, Message<Fragme
         return this;
     }
 
-    // isOutOfMemory
-
-    public Boolean getIsOutOfMemory()
-    {
-        return isOutOfMemory;
-    }
-
-    public FragmentRecordBatch setIsOutOfMemory(Boolean isOutOfMemory)
-    {
-        this.isOutOfMemory = isOutOfMemory;
-        return this;
-    }
-
     // java serialization
 
     public void readExternal(ObjectInput in) throws IOException
@@ -249,9 +234,6 @@ public final class FragmentRecordBatch implements Externalizable, Message<Fragme
                 case 7:
                     message.isLastBatch = input.readBool();
                     break;
-                case 8:
-                    message.isOutOfMemory = input.readBool();
-                    break;
                 default:
                     input.handleUnknownField(number, this);
             }   
@@ -289,9 +271,6 @@ public final class FragmentRecordBatch implements Externalizable, Message<Fragme
 
         if(message.isLastBatch != null)
             output.writeBool(7, message.isLastBatch, false);
-
-        if(message.isOutOfMemory != null && message.isOutOfMemory != DEFAULT_IS_OUT_OF_MEMORY)
-            output.writeBool(8, message.isOutOfMemory, false);
     }
 
     public String getFieldName(int number)
@@ -305,7 +284,6 @@ public final class FragmentRecordBatch implements Externalizable, Message<Fragme
             case 5: return "sendingMinorFragmentId";
             case 6: return "def";
             case 7: return "isLastBatch";
-            case 8: return "isOutOfMemory";
             default: return null;
         }
     }
@@ -326,7 +304,6 @@ public final class FragmentRecordBatch implements Externalizable, Message<Fragme
         __fieldMap.put("sendingMinorFragmentId", 5);
         __fieldMap.put("def", 6);
         __fieldMap.put("isLastBatch", 7);
-        __fieldMap.put("isOutOfMemory", 8);
     }
     
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/protocol/src/main/protobuf/BitData.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/BitData.proto b/protocol/src/main/protobuf/BitData.proto
index 2b76ce0..8724c4f 100644
--- a/protocol/src/main/protobuf/BitData.proto
+++ b/protocol/src/main/protobuf/BitData.proto
@@ -32,5 +32,4 @@ message FragmentRecordBatch{
   optional int32 sending_minor_fragment_id = 5;
   optional exec.shared.RecordBatchDef def = 6;
   optional bool isLastBatch = 7;
-  optional bool isOutOfMemory = 8 [ default = false ];
 }


[07/13] drill git commit: DRILL-4134: Allocator Improvements

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
deleted file mode 100644
index bb8f0ee..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
+++ /dev/null
@@ -1,1654 +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.ByteBufAllocator;
-import io.netty.buffer.DrillBuf;
-import io.netty.buffer.PooledByteBufAllocatorL;
-import io.netty.buffer.UnsafeDirectLittleEndian;
-
-import java.util.Collection;
-import java.util.IdentityHashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.drill.common.DrillAutoCloseables;
-import org.apache.drill.common.HistoricalLog;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.testing.ControlsInjector;
-import org.apache.drill.exec.testing.ControlsInjectorFactory;
-import org.apache.drill.exec.util.AssertionUtil;
-import org.apache.drill.exec.util.Pointer;
-import org.slf4j.Logger;
-
-import com.google.common.base.Preconditions;
-
-// TODO(cwestin) javadoc
-// TODO(cwestin) add allocator implementation options tried
-public abstract class BaseAllocator implements BufferAllocator {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseAllocator.class);
-  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(BaseAllocator.class);
-
-  private static final AtomicInteger idGenerator = new AtomicInteger(0);
-  private static final Object ALLOCATOR_LOCK = new Object();
-
-  public static final String CHILD_BUFFER_INJECTION_SITE = "child.buffer";
-
-  static final boolean DEBUG = AssertionUtil.isAssertionsEnabled()
-      || Boolean.getBoolean(ExecConstants.DEBUG_ALLOCATOR);
-  private static final PooledByteBufAllocatorL INNER_ALLOCATOR = PooledByteBufAllocatorL.DEFAULT;
-
-  private long allocated; // the amount of memory this allocator has given out to its clients (including children)
-  private long owned; // the amount of memory this allocator has obtained from its parent
-  private long peakAllocated; // the most memory this allocator has given out during its lifetime
-  private long bufferAllocation; // the amount of memory used just for directly allocated buffers, not children
-
-  private boolean isClosed = false; // the allocator has been closed
-
-  private final long maxAllocation; // the maximum amount of memory this allocator will give out
-  private final long chunkSize; // size of secondary chunks to allocate
-  private final AllocationPolicyAgent policyAgent;
-  private final BaseAllocator parentAllocator;
-  private final AllocatorOwner allocatorOwner;
-  protected final int id = idGenerator.incrementAndGet(); // unique ID assigned to each allocator
-  private final DrillBuf empty;
-  private final AllocationPolicy allocationPolicy;
-  private final InnerBufferLedger bufferLedger = new InnerBufferLedger();
-
-  // members used purely for debugging
-  private int getsFromParent;
-  private int putsToParent;
-  private final IdentityHashMap<UnsafeDirectLittleEndian, BufferLedger> allocatedBuffers;
-  private final IdentityHashMap<BaseAllocator, Object> childAllocators;
-  private final IdentityHashMap<Reservation, Object> reservations;
-  private long preallocSpace;
-
-  private final HistoricalLog historicalLog;
-
-  private static BaseAllocator getBaseAllocator(final BufferAllocator bufferAllocator) {
-    if (!(bufferAllocator instanceof BaseAllocator)) {
-      throw new IllegalArgumentException("expected a BaseAllocator instance, but got a "
-          + bufferAllocator.getClass().getName());
-    }
-    return (BaseAllocator) bufferAllocator;
-  }
-
-  // TODO move allocation policy and agent to outside of allocator
-  private static class PerFragmentAllocationPolicy implements AllocationPolicy {
-    static class Globals {
-      private long maxBufferAllocation = 0;
-      private final AtomicInteger limitingRoots = new AtomicInteger(0);
-    }
-
-    private final Globals globals = new Globals();
-
-    @Override
-    public AllocationPolicyAgent newAgent() {
-      return new PerFragmentAllocationPolicyAgent(globals);
-    }
-  }
-
-  /**
-   * AllocationPolicy that allows each fragment running on a drillbit to share an
-   * equal amount of direct memory, regardless of whether or not those fragments
-   * belong to the same query.
-   */
-  public static final AllocationPolicy POLICY_PER_FRAGMENT = new PerFragmentAllocationPolicy();
-
-  /**
-   * String name of {@link #POLICY_PER_FRAGMENT} policy.
-   */
-  public static final String POLICY_PER_FRAGMENT_NAME = "per-fragment";
-
-  private static class PerFragmentAllocationPolicyAgent implements AllocationPolicyAgent {
-    private final PerFragmentAllocationPolicy.Globals globals;
-    private boolean limitingRoot; // this is a limiting root; see F_LIMITING_ROOT
-
-    PerFragmentAllocationPolicyAgent(PerFragmentAllocationPolicy.Globals globals) {
-      this.globals = globals;
-    }
-
-    @Override
-    public void close() {
-      if (limitingRoot) {
-        // now there's one fewer active root
-        final int rootCount = globals.limitingRoots.decrementAndGet();
-
-        synchronized(globals) {
-          /*
-           * If the rootCount went to zero, we don't need to worry about setting the
-           * maxBufferAllocation, because there aren't any allocators to reference it;
-           * the next allocator to get created will set it appropriately.
-           */
-          if (rootCount != 0) {
-            globals.maxBufferAllocation = RootAllocator.getMaxDirect() / rootCount;
-          }
-        }
-      }
-    }
-
-    @Override
-    public void checkNewAllocator(BufferAllocator parentAllocator,
-        long initReservation, long maxAllocation, int flags) {
-/*
-      Preconditions.checkArgument(parentAllocator != null, "parent allocator can't be null");
-      Preconditions.checkArgument(parentAllocator instanceof BaseAllocator, "Parent allocator must be a BaseAllocator");
-*/
-
-//      final BaseAllocator baseAllocator = (BaseAllocator) parentAllocator;
-
-      // this is synchronized to protect maxBufferAllocation
-      synchronized(POLICY_PER_FRAGMENT) {
-        // initialize maxBufferAllocation the very first time we call this
-        if (globals.maxBufferAllocation == 0) {
-          globals.maxBufferAllocation = RootAllocator.getMaxDirect();
-        }
-
-        if (limitingRoot = ((flags & F_LIMITING_ROOT) != 0)) {
-          // figure out the new current per-allocator limit
-          globals.maxBufferAllocation = RootAllocator.getMaxDirect() / (globals.limitingRoots.get() + 1);
-        }
-
-        if (initReservation > 0) {
-          if (initReservation > globals.maxBufferAllocation) {
-            throw new OutOfMemoryRuntimeException(
-                String.format("can't fulfill initReservation request at this time "
-                    + "(initReservation = %d > maxBufferAllocation = %d)",
-                initReservation, globals.maxBufferAllocation));
-          }
-        }
-      }
-    }
-
-    @Override
-    public long getMemoryLimit(BufferAllocator bufferAllocator) {
-      synchronized(POLICY_PER_FRAGMENT) {
-        return globals.maxBufferAllocation;
-      }
-    }
-
-    @Override
-    public void initializeAllocator(final BufferAllocator bufferAllocator) {
-      final BaseAllocator baseAllocator = getBaseAllocator(bufferAllocator);
-
-      if (limitingRoot) {
-        globals.limitingRoots.incrementAndGet();
-      }
-    }
-
-    @Override
-    public boolean shouldReleaseToParent(final BufferAllocator bufferAllocator) {
-      final BaseAllocator baseAllocator = getBaseAllocator(bufferAllocator);
-      return (baseAllocator.owned + baseAllocator.chunkSize > globals.maxBufferAllocation);
-    }
-  }
-
-  private static class LocalMaxAllocationPolicy implements AllocationPolicy {
-    // this agent is stateless, so we can always use the same one
-    private static final AllocationPolicyAgent AGENT = new LocalMaxAllocationPolicyAgent();
-
-    @Override
-    public AllocationPolicyAgent newAgent() {
-      return AGENT;
-    }
-  }
-
-  /**
-   * AllocationPolicy that imposes no limits on how much direct memory fragments
-   * may allocate. LOCAL_MAX refers to the only limit that is enforced, which is
-   * the maxAllocation specified at allocators' creation.
-   *
-   * <p>This policy ignores the value of {@link BufferAllocator#F_LIMITING_ROOT}.</p>
-   */
-  public static final AllocationPolicy POLICY_LOCAL_MAX = new LocalMaxAllocationPolicy();
-
-  /**
-   * String name of {@link #POLICY_LOCAL_MAX} allocation policy.
-   */
-  public static final String POLICY_LOCAL_MAX_NAME = "local-max";
-
-  private static class LocalMaxAllocationPolicyAgent implements AllocationPolicyAgent {
-    @Override
-    public void close() throws Exception {
-    }
-
-    @Override
-    public void checkNewAllocator(BufferAllocator parentAllocator,
-        long initReservation, long maxAllocation, int flags) {
-    }
-
-    @Override
-    public long getMemoryLimit(BufferAllocator bufferAllocator) {
-      final BaseAllocator baseAllocator = (BaseAllocator) bufferAllocator;
-      return baseAllocator.maxAllocation;
-    }
-
-    @Override
-    public void initializeAllocator(BufferAllocator bufferAllocator) {
-    }
-
-    @Override
-    public boolean shouldReleaseToParent(BufferAllocator bufferAllocator) {
-      // since there are no shared limits, release space whenever we can
-      return true;
-    }
-  }
-
-  // TODO(DRILL-2698) POLICY_PER_QUERY
-
-  protected BaseAllocator(
-      final BaseAllocator parentAllocator,
-      final AllocatorOwner allocatorOwner,
-      final AllocationPolicy allocationPolicy,
-      final long initReservation,
-      final long maxAllocation,
-      final int flags) throws OutOfMemoryRuntimeException {
-    Preconditions.checkArgument(allocatorOwner != null, "allocatorOwner must be non-null");
-    Preconditions.checkArgument(initReservation >= 0,
-        "the initial reservation size must be non-negative");
-    Preconditions.checkArgument(maxAllocation >= 0,
-        "the maximum allocation limit mjst be non-negative");
-    Preconditions.checkArgument(initReservation <= maxAllocation,
-        "the initial reservation size must be <= the maximum allocation");
-
-    if (initReservation > 0) {
-      if (parentAllocator == null) {
-        throw new IllegalStateException(
-            "can't reserve memory without a parent allocator");
-      }
-    }
-
-    // check to see if we can create this new allocator (the check throws if it's not ok)
-    final AllocationPolicyAgent policyAgent = allocationPolicy.newAgent();
-    policyAgent.checkNewAllocator(parentAllocator, initReservation, maxAllocation, flags);
-
-    if ((initReservation > 0) && !parentAllocator.reserve(this, initReservation, 0)) {
-      throw new OutOfMemoryRuntimeException(
-          "can't fulfill initial reservation of size (unavailable from parent)" + initReservation);
-    }
-
-    /*
-     * Figure out how much more to ask for from our parent if we're out.
-     * Secondary allocations from the parent will be done in integral multiples of
-     * chunkSize. We also use this to determine when to hand back memory to our
-     * parent in order to create hysteresis. This reduces contention on the parent's
-     * lock
-     */
-    if (initReservation == 0) {
-      chunkSize = maxAllocation / 8;
-    } else {
-      chunkSize = initReservation;
-    }
-
-    this.parentAllocator = parentAllocator;
-    this.allocatorOwner = allocatorOwner;
-    this.allocationPolicy = allocationPolicy;
-    this.policyAgent = policyAgent;
-    this.maxAllocation = maxAllocation;
-
-    // the root allocator owns all of its memory; anything else just owns it's initial reservation
-    owned = parentAllocator == null ? maxAllocation : initReservation;
-    empty = DrillBuf.getEmpty(new EmptyLedger(), this);
-
-    if (DEBUG) {
-      allocatedBuffers = new IdentityHashMap<>();
-      childAllocators = new IdentityHashMap<>();
-      reservations = new IdentityHashMap<>();
-      historicalLog = new HistoricalLog(4, "allocator[%d]", id);
-
-      historicalLog.recordEvent("created by \"%s\", owned = %d", allocatorOwner.toString(), owned);
-    } else {
-      allocatedBuffers = null;
-      childAllocators = null;
-      reservations = null;
-      historicalLog = null;
-    }
-
-    // now that we're not in danger of throwing an exception, we can take this step
-    policyAgent.initializeAllocator(this);
-  }
-
-  /**
-   * Allocators without a parent must provide an implementation of this so
-   * that they may reserve additional space even though they don't have a
-   * parent they can fall back to.
-   *
-   * <p>Prior to calling this, BaseAllocator has verified that this won't violate
-   * the maxAllocation for this allocator.</p>
-   *
-   * @param nBytes the amount of space to reserve
-   * @param ignoreMax ignore the maximum allocation limit;
-   *   see {@link ChildLedger#reserve(long, boolean)}.
-   * @return true if the request can be met, false otherwise
-   */
-  protected boolean canIncreaseOwned(final long nBytes, final int flags) {
-    if (parentAllocator == null) {
-      return false;
-    }
-
-    return parentAllocator.reserve(this, nBytes, flags);
-  }
-
-  /**
-   * Reserve space for the child allocator from this allocator.
-   *
-   * @param childAllocator child allocator making the request, or null
-   *  if this is not for a child
-   * @param nBytes how much to reserve
-   * @param flags one or more of RESERVE_F_* flags or'ed together
-   * @return true if the reservation can be satisfied, false otherwise
-   */
-  private static final int RESERVE_F_IGNORE_MAX = 0x0001;
-  private boolean reserve(final BaseAllocator childAllocator,
-      final long nBytes, final int flags) {
-    Preconditions.checkArgument(nBytes >= 0,
-        "the number of bytes to reserve must be non-negative");
-
-    // we can always fulfill an empty request
-    if (nBytes == 0) {
-      return true;
-    }
-
-    final boolean ignoreMax = (flags & RESERVE_F_IGNORE_MAX) != 0;
-
-    synchronized(ALLOCATOR_LOCK) {
-      if (isClosed) {
-        throw new AllocatorClosedException(String.format("Attempt to use closed allocator[%d]", id));
-      }
-
-      final long ownAtLeast = allocated + nBytes;
-      // Are we allowed to hand out this much?
-      if (!ignoreMax && (ownAtLeast > maxAllocation)) {
-        return false;
-      }
-
-      // do we need more from our parent first?
-      if (ownAtLeast > owned) {
-        /*
-         * Allocate space in integral multiples of chunkSize, as long as it doesn't exceed
-         * the maxAllocation.
-         */
-        final long needAdditional = ownAtLeast - owned;
-        final long getInChunks = (1 + ((needAdditional - 1) / chunkSize)) * chunkSize;
-        final long getFromParent;
-        if (getInChunks + owned <= maxAllocation) {
-          getFromParent = getInChunks;
-        } else {
-          getFromParent = needAdditional;
-        }
-        if (!canIncreaseOwned(getFromParent, flags)) {
-          return false;
-        }
-        owned += getFromParent;
-
-        if (DEBUG) {
-          ++getsFromParent;
-          historicalLog.recordEvent("increased owned by %d, now owned == %d", needAdditional, owned);
-        }
-      }
-
-      if (DEBUG) {
-        if (owned < ownAtLeast) {
-          throw new IllegalStateException("don't own enough memory to satisfy request");
-        }
-        if (allocated > owned) {
-          throw new IllegalStateException(
-              String.format("more memory allocated (%d) than owned (%d)", allocated, owned));
-        }
-
-        historicalLog.recordEvent("allocator[%d] allocated increased by nBytes == %d to %d",
-            id, nBytes, allocated + nBytes);
-      }
-
-      allocated += nBytes;
-
-      if (allocated > peakAllocated) {
-        peakAllocated = allocated;
-      }
-
-      return true;
-    }
-  }
-
-  private void releaseBytes(final long nBytes) {
-    Preconditions.checkArgument(nBytes >= 0,
-        "the number of bytes being released must be non-negative");
-
-    synchronized(ALLOCATOR_LOCK) {
-      allocated -= nBytes;
-
-      if (DEBUG) {
-        historicalLog.recordEvent("allocator[%d] released nBytes == %d, allocated now %d",
-            id, nBytes, allocated);
-      }
-
-      /*
-       * Return space to our parent if our allocation is over the currently allowed amount.
-       */
-      final boolean releaseToParent = (parentAllocator != null)
-          && (owned > maxAllocation) && policyAgent.shouldReleaseToParent(this);
-      if (releaseToParent) {
-        final long canFree = owned - maxAllocation;
-        parentAllocator.releaseBytes(canFree);
-        owned -= canFree;
-
-        if (DEBUG) {
-          ++putsToParent;
-          historicalLog.recordEvent("returned %d to parent, now owned == %d", canFree, owned);
-        }
-      }
-    }
-  }
-
-  private void releaseBuffer(final DrillBuf drillBuf) {
-    Preconditions.checkArgument(drillBuf != null,
-        "the DrillBuf being released can't be null");
-
-    final ByteBuf byteBuf = drillBuf.unwrap();
-    final int udleMaxCapacity = byteBuf.maxCapacity();
-
-    synchronized(ALLOCATOR_LOCK) {
-      bufferAllocation -= udleMaxCapacity;
-      releaseBytes(udleMaxCapacity);
-
-      if (DEBUG) {
-        // make sure the buffer came from this allocator
-        final Object object = allocatedBuffers.remove(byteBuf);
-        if (object == null) {
-          historicalLog.logHistory(logger);
-          drillBuf.logHistory(logger);
-          throw new IllegalStateException("Released buffer did not belong to this allocator");
-        }
-      }
-    }
-  }
-
-  private void childClosed(final BaseAllocator childAllocator) {
-    Preconditions.checkArgument(childAllocator != null, "child allocator can't be null");
-
-    if (DEBUG) {
-      synchronized(ALLOCATOR_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");
-        }
-
-        try {
-          verifyAllocator();
-        } catch(Exception e) {
-          /*
-           * If there was a problem with verification, the history of the closed
-           * child may also be useful.
-           */
-          logger.debug("allocator[" + id + "]: exception while closing the following child");
-          childAllocator.historicalLog.logHistory(logger);
-
-          // Continue with the verification exception throwing.
-          throw e;
-        }
-      }
-    }
-  }
-
-  /**
-   * TODO(DRILL-2740) We use this to bypass the regular accounting for the
-   * empty DrillBuf, because it is treated specially at this time. Once that
-   * is remedied, this should be able to go away.
-   */
-  private class EmptyLedger implements BufferLedger {
-    @Override
-    public PooledByteBufAllocatorL getUnderlyingAllocator() {
-      return INNER_ALLOCATOR;
-    }
-
-    @Override
-    public void release(final DrillBuf drillBuf) {
-      if (DEBUG) {
-        if (drillBuf != empty) {
-          throw new IllegalStateException("The empty buffer's ledger is being used to release something else");
-        }
-      }
-    }
-
-    @Override
-    public BufferLedger shareWith(Pointer<DrillBuf> pDrillBuf,
-        BufferLedger otherLedger, BufferAllocator otherAllocator, DrillBuf drillBuf,
-        int index, int length, int drillBufFlags) {
-      // As a special case, we allow sharing with the same allocator so that slicing works.
-      if (otherAllocator != BaseAllocator.this) {
-        throw new UnsupportedOperationException("The empty buffer can't be shared");
-      }
-
-      pDrillBuf.value = drillBuf;
-      return otherLedger;
-    }
-
-    @Override
-    public boolean transferTo(BufferAllocator newAlloc,
-        Pointer<BufferLedger> pNewLedger, DrillBuf drillBuf) {
-      throw new UnsupportedOperationException("The empty buffer's ownership can't be changed");
-    }
-  }
-
-  private class InnerBufferLedger implements BufferLedger {
-    @Override
-    public PooledByteBufAllocatorL getUnderlyingAllocator() {
-      return INNER_ALLOCATOR;
-    }
-
-    @Override
-    public void release(final DrillBuf drillBuf) {
-      releaseBuffer(drillBuf);
-    }
-
-    @Override
-    public BufferLedger shareWith(final Pointer<DrillBuf> pDrillBuf,
-        final BufferLedger otherLedger, final BufferAllocator otherAllocator,
-        final DrillBuf drillBuf, final int index, final int length, final int drillBufFlags) {
-      final BaseAllocator baseAllocator = (BaseAllocator) otherAllocator;
-      synchronized(ALLOCATOR_LOCK) {
-        if (baseAllocator.isClosed) {
-          throw new AllocatorClosedException(
-              String.format("Attempt to use closed allocator[%d]", baseAllocator.id));
-        }
-
-        /*
-         * If this is called, then the buffer isn't yet shared, and should
-         * become so.
-         */
-        final SharedBufferLedger sharedLedger = new SharedBufferLedger(drillBuf, BaseAllocator.this);
-
-        // Create the new wrapping DrillBuf.
-        final DrillBuf newBuf =
-            new DrillBuf(sharedLedger, otherAllocator, drillBuf, index, length, drillBufFlags);
-        sharedLedger.addMapping(newBuf, baseAllocator);
-
-        if (DEBUG) {
-          final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap();
-          historicalLog.recordEvent("InnerBufferLedger(allocator[%d]).shareWith(..., "
-              + "otherAllocator[%d], drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}, ...)",
-              BaseAllocator.this.id, baseAllocator.id, drillBuf.getId(),
-              System.identityHashCode(udle));
-
-          final BaseAllocator drillBufAllocator = (BaseAllocator) drillBuf.getAllocator();
-          if (BaseAllocator.this != drillBufAllocator) {
-            historicalLog.logHistory(logger);
-            drillBuf.logHistory(logger);
-            throw new IllegalStateException(String.format(
-                "DrillBuf's allocator([%d]) doesn't match this(allocator[%d])",
-                drillBufAllocator.id, BaseAllocator.this.id));
-          }
-
-          // Replace the ledger for the existing buffer.
-          final BufferLedger thisLedger = allocatedBuffers.put(udle, sharedLedger);
-
-          // If we throw any of these exceptions, we need to clean up newBuf.
-          if (thisLedger == null) {
-            newBuf.release();
-            historicalLog.logHistory(logger);
-            drillBuf.logHistory(logger);
-            throw new IllegalStateException("Buffer to be shared is unknown to the source allocator");
-          }
-          if (thisLedger != this) {
-            newBuf.release();
-            historicalLog.logHistory(logger);
-            drillBuf.logHistory(logger);
-            throw new IllegalStateException("Buffer's ledger was not the one it should be");
-          }
-        }
-
-        pDrillBuf.value = newBuf;
-        return sharedLedger;
-      }
-    }
-
-    @Override
-    public boolean transferTo(final BufferAllocator newAlloc,
-        final Pointer<BufferLedger> pNewLedger, final DrillBuf drillBuf) {
-      Preconditions.checkArgument(newAlloc != null, "New allocator cannot be null");
-      Preconditions.checkArgument(newAlloc != BaseAllocator.this,
-          "New allocator is same as current");
-      Preconditions.checkArgument(newAlloc instanceof BaseAllocator,
-          "New allocator isn't a BaseAllocator");
-      Preconditions.checkArgument(pNewLedger.value != null, "Candidate new ledger can't be null");
-      Preconditions.checkArgument(drillBuf != null, "DrillBuf can't be null");
-
-      final BaseAllocator newAllocator = (BaseAllocator) newAlloc;
-      synchronized(ALLOCATOR_LOCK) {
-        if (newAllocator.isClosed) {
-          throw new AllocatorClosedException(
-              String.format("Attempt to use closed allocator[%d]", newAllocator.id));
-        }
-
-        return BaseAllocator.transferTo(newAllocator, pNewLedger.value, drillBuf);
-      }
-    }
-  }
-
-  /**
-   * Transfer ownership of a buffer from one allocator to another.
-   *
-   * <p>Assumes the allocatorLock is held.</p>
-   *
-   * @param newAllocator the new allocator
-   * @param newLedger the new ledger to use (which could be shared)
-   * @param drillBuf the buffer
-   * @return true if the buffer's transfer didn't exceed the new owner's maximum
-   *   allocation limit
-   */
-  private static boolean transferTo(final BaseAllocator newAllocator,
-      final BufferLedger newLedger, final DrillBuf drillBuf) {
-    final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap();
-    final int udleMaxCapacity = udle.maxCapacity();
-
-    synchronized(ALLOCATOR_LOCK) {
-      // Account for the space and track the buffer.
-      newAllocator.reserveForBuf(udleMaxCapacity);
-
-      if (DEBUG) {
-        final Object object = newAllocator.allocatedBuffers.put(udle, newLedger);
-        if (object != null) {
-          newAllocator.historicalLog.logHistory(logger);
-          drillBuf.logHistory(logger);
-          throw new IllegalStateException("Buffer unexpectedly found in new allocator");
-        }
-      }
-
-      // Remove from the old allocator.
-      final BaseAllocator oldAllocator = (BaseAllocator) drillBuf.getAllocator();
-      oldAllocator.releaseBuffer(drillBuf);
-
-      if (DEBUG) {
-        final Object object = oldAllocator.allocatedBuffers.get(udle);
-        if (object != null) {
-          oldAllocator.historicalLog.logHistory(logger);
-          drillBuf.logHistory(logger);
-          throw new IllegalStateException("Buffer was not removed from old allocator");
-        }
-
-        oldAllocator.historicalLog.recordEvent("BaseAllocator.transferTo(otherAllocator[%d], ..., "
-            + "drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}) oldAllocator[%d]",
-            newAllocator.id, drillBuf.getId(), System.identityHashCode(drillBuf.unwrap()),
-            oldAllocator.id);
-        newAllocator.historicalLog.recordEvent("BaseAllocator.transferTo(otherAllocator[%d], ..., "
-            + "drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}) oldAllocator[%d]",
-            newAllocator.id, drillBuf.getId(), System.identityHashCode(drillBuf.unwrap()),
-            oldAllocator.id);
-      }
-
-      return newAllocator.allocated < newAllocator.maxAllocation;
-    }
-  }
-
-  private static class SharedBufferLedger implements BufferLedger {
-    private volatile BaseAllocator owningAllocator;
-    private final IdentityHashMap<DrillBuf, BaseAllocator> bufferMap = new IdentityHashMap<>();
-
-    private final HistoricalLog historicalLog;
-
-    public SharedBufferLedger(final DrillBuf drillBuf, final BaseAllocator baseAllocator) {
-      if (DEBUG) {
-        historicalLog = new HistoricalLog(4,
-            "SharedBufferLedger for DrillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}",
-            drillBuf.getId(), System.identityHashCode(drillBuf.unwrap()));
-      } else {
-        historicalLog = null;
-      }
-      addMapping(drillBuf, baseAllocator);
-      owningAllocator = baseAllocator;
-
-      if (DEBUG) {
-        checkBufferMap();
-      }
-    }
-
-    private synchronized void addMapping(final DrillBuf drillBuf, final BaseAllocator baseAllocator) {
-      bufferMap.put(drillBuf, baseAllocator);
-
-      if (DEBUG) {
-        historicalLog.recordEvent("addMapping(DrillBuf[%d], allocator[%d])", drillBuf.getId(), baseAllocator.id);
-      }
-    }
-
-    private synchronized void logBufferHistories(final Logger logger) {
-      final Set<Map.Entry<DrillBuf, BaseAllocator>> bufsToCheck = bufferMap.entrySet();
-      for(final Map.Entry<DrillBuf, BaseAllocator> mapEntry : bufsToCheck) {
-        final DrillBuf drillBuf = mapEntry.getKey();
-        drillBuf.logHistory(logger);
-      }
-    }
-
-    private synchronized void checkBufferMap() {
-      boolean foundOwner = false;
-      final Set<Map.Entry<DrillBuf, BaseAllocator>> bufsToCheck = bufferMap.entrySet();
-      for(final Map.Entry<DrillBuf, BaseAllocator> mapEntry : bufsToCheck) {
-        final DrillBuf drillBuf = mapEntry.getKey();
-        final BaseAllocator bufferAllocator = mapEntry.getValue();
-
-        final Object object = bufferAllocator.allocatedBuffers.get(drillBuf.unwrap());
-        if (bufferAllocator == owningAllocator) {
-          foundOwner = true;
-          if (object == null) {
-            historicalLog.logHistory(logger);
-            logBufferHistories(logger);
-            throw new IllegalStateException(
-                String.format("Shared buffer DrillBuf[%d] not found in owning allocator[%d]",
-                    drillBuf.getId(), bufferAllocator.id));
-          }
-        } else {
-          if (object != null) {
-            historicalLog.logHistory(logger);
-            logBufferHistories(logger);
-            throw new IllegalStateException(
-                String.format("Shared buffer DrillBuf[%d] not found in non-owning allocator[%d]",
-                    drillBuf.getId(), bufferAllocator.id));
-
-          }
-        }
-      }
-
-      if (!foundOwner && !bufferMap.isEmpty()) {
-        historicalLog.logHistory(logger);
-        logBufferHistories(logger);
-        owningAllocator.historicalLog.logHistory(logger);
-        throw new IllegalStateException(
-            String.format("Did not find owning allocator[%d] in bufferMap", owningAllocator.id));
-      }
-    }
-
-    @Override
-    public PooledByteBufAllocatorL getUnderlyingAllocator() {
-      return INNER_ALLOCATOR;
-    }
-
-    @Override
-    public void release(final DrillBuf drillBuf) {
-      Preconditions.checkArgument(drillBuf != null, "drillBuf can't be null");
-
-      /*
-       * This is the only method on the shared ledger that can be entered without
-       * having first come through an outside method on BaseAllocator (such
-       * as takeOwnership() or shareOwnership()), all of which get the allocatorLock.
-       * Operations in the below require the allocatorLock. We also need to synchronize
-       * on this object to protect the bufferMap. In order to avoid a deadlock with other
-       * methods, we have to get the allocatorLock first, as will be done in all the
-       * other cases.
-       */
-      synchronized(ALLOCATOR_LOCK) {
-        synchronized(this) {
-          final Object bufferObject = bufferMap.remove(drillBuf);
-          if (DEBUG) {
-            if (bufferObject == null) {
-              historicalLog.logHistory(logger, String.format("release(DrillBuf[%d])", drillBuf.getId()));
-              drillBuf.logHistory(logger);
-              throw new IllegalStateException("Buffer not found in SharedBufferLedger's buffer map");
-            }
-          }
-
-          /*
-           * If there are other buffers in the bufferMap that share this buffer's fate,
-           * remove them, since they are also now invalid. As we examine buffers, take note
-           * of any others that don't share this one's fate, but which belong to the same
-           * allocator; if we find any such, then we can avoid transferring ownership at this
-           * time.
-           */
-          final BaseAllocator bufferAllocator = (BaseAllocator) drillBuf.getAllocator();
-          final List<DrillBuf> sameAllocatorSurvivors = new LinkedList<>();
-          if (!bufferMap.isEmpty()) {
-            /*
-             * We're going to be modifying bufferMap (if we find any other related buffers);
-             * in order to avoid getting a ConcurrentModificationException, we can't do it
-             * on the same iteration we use to examine the buffers, so we use an intermediate
-             * list to figure out which ones we have to remove.
-             */
-            final Set<Map.Entry<DrillBuf, BaseAllocator>> bufsToCheck = bufferMap.entrySet();
-            final List<DrillBuf> sharedFateBuffers = new LinkedList<>();
-            for(final Map.Entry<DrillBuf, BaseAllocator> mapEntry : bufsToCheck) {
-              final DrillBuf otherBuf = mapEntry.getKey();
-              if (otherBuf.hasSharedFate(drillBuf)) {
-                sharedFateBuffers.add(otherBuf);
-              } else {
-                final BaseAllocator otherAllocator = mapEntry.getValue();
-                if (otherAllocator == bufferAllocator) {
-                  sameAllocatorSurvivors.add(otherBuf);
-                }
-              }
-            }
-
-            final int nSharedFate = sharedFateBuffers.size();
-            if (nSharedFate > 0) {
-              final int[] sharedIds = new int[nSharedFate];
-              int index = 0;
-              for(final DrillBuf bufToRemove : sharedFateBuffers) {
-                sharedIds[index++] = bufToRemove.getId();
-                bufferMap.remove(bufToRemove);
-              }
-
-              if (DEBUG) {
-                final StringBuilder sb = new StringBuilder();
-                for(final DrillBuf bufToRemove : sharedFateBuffers) {
-                  sb.append(String.format("DrillBuf[%d], ", bufToRemove.getId()));
-                }
-                sb.setLength(sb.length() - 2); // Chop off the trailing comma and space.
-                historicalLog.recordEvent("removed shared fate buffers " + sb.toString());
-              }
-            }
-          }
-
-          if (sameAllocatorSurvivors.isEmpty()) {
-            /*
-             * If that was the owning allocator, then we need to transfer ownership to
-             * another allocator (any one) that is part of the sharing set.
-             *
-             * When we release the buffer back to the allocator, release the root buffer,
-             */
-            if (bufferAllocator == owningAllocator) {
-              if (bufferMap.isEmpty()) {
-                /*
-                 * There are no other allocators available to transfer to, so
-                 * release the space to the owner.
-                 */
-                bufferAllocator.releaseBuffer(drillBuf);
-              } else {
-                // Pick another allocator, and transfer ownership to that.
-                final Collection<BaseAllocator> allocators = bufferMap.values();
-                final Iterator<BaseAllocator> allocatorIter = allocators.iterator();
-                if (!allocatorIter.hasNext()) {
-                  historicalLog.logHistory(logger);
-                  throw new IllegalStateException("Shared ledger buffer map is non-empty, but not iterable");
-                }
-                final BaseAllocator nextAllocator = allocatorIter.next();
-                BaseAllocator.transferTo(nextAllocator, this, drillBuf);
-                owningAllocator = nextAllocator;
-
-                if (DEBUG) {
-                  if (owningAllocator == bufferAllocator) {
-                    historicalLog.logHistory(logger);
-                    owningAllocator.historicalLog.logHistory(logger);
-                    drillBuf.logHistory(logger);
-                    throw new IllegalStateException("Shared buffer release transfer to same owner");
-                  }
-
-                  final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap();
-                  final Object oldObject = bufferAllocator.allocatedBuffers.get(udle);
-                  if (oldObject != null) {
-                    historicalLog.logHistory(logger);
-                    bufferAllocator.historicalLog.logHistory(logger);
-                    owningAllocator.historicalLog.logHistory(logger);
-                    drillBuf.logHistory(logger);
-
-                    throw new IllegalStateException("Inconsistent shared buffer release state (old owner)");
-                  }
-
-                  final Object newObject = owningAllocator.allocatedBuffers.get(udle);
-                  if (newObject == null) {
-                    historicalLog.logHistory(logger);
-                    bufferAllocator.historicalLog.logHistory(logger);
-                    owningAllocator.historicalLog.logHistory(logger);
-                    drillBuf.logHistory(logger);
-
-                    throw new IllegalStateException("Inconsistent shared buffer release state (new owner)");
-                  }
-                }
-              }
-            }
-          }
-        }
-      }
-
-      if (DEBUG) {
-        checkBufferMap();
-      }
-    }
-
-    @Override
-    public BufferLedger shareWith(final Pointer<DrillBuf> pDrillBuf,
-        final BufferLedger otherLedger, final BufferAllocator otherAllocator,
-        final DrillBuf drillBuf, final int index, final int length, final int drillBufFlags) {
-      final BaseAllocator baseAllocator = (BaseAllocator) otherAllocator;
-      if (baseAllocator.isClosed) {
-        throw new AllocatorClosedException(
-            String.format("Attempt to use closed allocator[%d]", baseAllocator.id));
-      }
-
-      synchronized(ALLOCATOR_LOCK) {
-        /*
-         * This buffer is already shared, but we want to add more sharers.
-         *
-         * Create the new wrapper.
-         */
-        final DrillBuf newBuf = new DrillBuf(this, otherAllocator, drillBuf, index, length, drillBufFlags);
-        if (DEBUG) {
-          final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) drillBuf.unwrap();
-          baseAllocator.historicalLog.recordEvent("SharedBufferLedger.shareWith(..., otherAllocator[%d], "
-              + "drillBuf[%d]{UnsafeDirectLittleEndian[identityHashCode == %d]}, ...)",
-              baseAllocator.id, drillBuf.getId(), System.identityHashCode(udle));
-
-          // Make sure the current ownership is still correct.
-          final Object object = owningAllocator.allocatedBuffers.get(udle); // This may not be protectable w/o ALLOCATOR_LOCK.
-          if (object == null) {
-            newBuf.release();
-            historicalLog.logHistory(logger);
-            owningAllocator.historicalLog.logHistory(logger);
-            drillBuf.logHistory(logger);
-            throw new IllegalStateException("Buffer not found in owning allocator");
-          }
-        }
-
-        addMapping(newBuf, baseAllocator);
-        pDrillBuf.value = newBuf;
-
-        if (DEBUG) {
-          checkBufferMap();
-        }
-
-        return this;
-      }
-    }
-
-    @Override
-    public boolean transferTo(final BufferAllocator newAlloc,
-        final Pointer<BufferLedger> pNewLedger, final DrillBuf drillBuf) {
-      Preconditions.checkArgument(newAlloc != null, "New allocator cannot be null");
-      Preconditions.checkArgument(newAlloc instanceof BaseAllocator,
-          "New allocator isn't a BaseAllocator");
-      Preconditions.checkArgument(pNewLedger.value != null, "Candidate new ledger can't be null");
-      Preconditions.checkArgument(drillBuf != null, "DrillBuf can't be null");
-
-      final BaseAllocator newAllocator = (BaseAllocator) newAlloc;
-      if (newAllocator.isClosed) {
-        throw new AllocatorClosedException(String.format(
-            "Attempt to use closed allocator[%d]", newAllocator.id));
-      }
-
-      // This doesn't need the ALLOCATOR_LOCK, because it will already be held.
-      synchronized(this) {
-        try {
-          // Modify the buffer mapping to reflect the virtual transfer.
-          final BaseAllocator oldAllocator = bufferMap.put(drillBuf, newAllocator);
-          if (oldAllocator == null) {
-            final BaseAllocator bufAllocator = (BaseAllocator) drillBuf.getAllocator();
-            historicalLog.logHistory(logger);
-            bufAllocator.historicalLog.logHistory(logger);
-            drillBuf.logHistory(logger);
-            throw new IllegalStateException("No previous entry in SharedBufferLedger for drillBuf");
-          }
-
-          // Whatever happens, this is the new ledger.
-          pNewLedger.value = this;
-
-          /*
-           * If the oldAllocator was the owner, then transfer ownership to the new allocator.
-           */
-          if (oldAllocator == owningAllocator) {
-            owningAllocator = newAllocator;
-            return BaseAllocator.transferTo(newAllocator, this, drillBuf);
-          }
-
-          // Even though we didn't do a real transfer, tell if it would have fit the limit.
-          final int udleMaxCapacity = drillBuf.unwrap().maxCapacity();
-          return newAllocator.allocated + udleMaxCapacity < newAllocator.maxAllocation;
-        } finally {
-          if (DEBUG) {
-            checkBufferMap();
-          }
-        }
-      }
-    }
-  }
-
-  @Override
-  public DrillBuf buffer(int size) {
-    return buffer(size, size);
-  }
-
-  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());
-  }
-
-  @Override
-  public DrillBuf buffer(final int minSize, final int maxSize) {
-    Preconditions.checkArgument(minSize >= 0,
-        "the minimimum requested size must be non-negative");
-    Preconditions.checkArgument(maxSize >= 0,
-        "the maximum requested size must be non-negative");
-    Preconditions.checkArgument(minSize <= maxSize,
-        "the minimum requested size must be <= the maximum requested size");
-
-    if (DEBUG) {
-      injector.injectUnchecked(allocatorOwner.getExecutionControls(), CHILD_BUFFER_INJECTION_SITE);
-    }
-
-    // we can always return an empty buffer
-    if (minSize == 0) {
-      return getEmpty();
-    }
-
-    synchronized(ALLOCATOR_LOCK) {
-      // Don't allow the allocation if it will take us over the limit.
-      final long allocatedWas = allocated;
-      if (!reserve(null, maxSize, 0)) {
-        throw new OutOfMemoryRuntimeException(createErrorMsg(this, minSize));
-      }
-
-      final long reserved = allocated - allocatedWas;
-      assert reserved == maxSize;
-
-      final UnsafeDirectLittleEndian buffer = INNER_ALLOCATOR.directBuffer(minSize, maxSize);
-      final int actualSize = buffer.maxCapacity();
-      if (actualSize > maxSize) {
-        final int extraSize = actualSize - maxSize;
-        reserve(null, extraSize, RESERVE_F_IGNORE_MAX);
-      }
-
-      final DrillBuf wrapped = new DrillBuf(bufferLedger, this, buffer);
-      buffer.release(); // Should have been retained by the DrillBuf constructor.
-      assert buffer.refCnt() == 1 : "buffer was not retained by DrillBuf";
-      assert allocated <= owned : "allocated more memory than owned";
-
-      bufferAllocation += maxSize;
-      if (allocated > peakAllocated) {
-        peakAllocated = allocated;
-      }
-
-      if (allocatedBuffers != null) {
-        allocatedBuffers.put(buffer, bufferLedger);
-      }
-
-      return wrapped;
-    }
-  }
-
-  @Override
-  public ByteBufAllocator getUnderlyingAllocator() {
-    return INNER_ALLOCATOR;
-  }
-
-  @Override
-  public BufferAllocator newChildAllocator(final AllocatorOwner allocatorOwner,
-      final long initReservation, final long maxAllocation, final int flags) {
-    synchronized(ALLOCATOR_LOCK) {
-      final BaseAllocator childAllocator =
-          new ChildAllocator(this, allocatorOwner, allocationPolicy,
-              initReservation, maxAllocation, flags);
-
-      if (DEBUG) {
-        childAllocators.put(childAllocator, childAllocator);
-        historicalLog.recordEvent("allocator[%d] created new child allocator[%d]",
-            id, childAllocator.id);
-      }
-
-      return childAllocator;
-    }
-  }
-
-  @Override
-  public BufferAllocator getChildAllocator(FragmentContext fragmentContext,
-      final long initialReservation, final long maximumAllocation,
-      final boolean applyFragmentLimit) {
-    return newChildAllocator(allocatorOwner, initialReservation, maximumAllocation,
-        (applyFragmentLimit ? F_LIMITING_ROOT : 0));
-  }
-
-  /**
-   * Reserve space for a DrillBuf for an ownership transfer.
-   *
-   * @param drillBuf the buffer to reserve space for
-   */
-  private void reserveForBuf(final int maxCapacity) {
-    final boolean reserved = reserve(null, maxCapacity, RESERVE_F_IGNORE_MAX);
-    if (DEBUG) {
-      if (!reserved) {
-        throw new IllegalStateException("reserveForBuf() failed");
-      }
-    }
-  }
-
-  @Override
-  public boolean takeOwnership(final DrillBuf drillBuf) {
-    // If already owned by this, there's nothing to do.
-    if (this == drillBuf.getAllocator()) {
-      return true;
-    }
-
-    synchronized(ALLOCATOR_LOCK) {
-      return drillBuf.transferTo(this, bufferLedger);
-    }
-  }
-
-  @Override
-  public boolean shareOwnership(final DrillBuf drillBuf, final Pointer<DrillBuf> bufOut) {
-    synchronized(ALLOCATOR_LOCK) {
-      bufOut.value = drillBuf.shareWith(bufferLedger, this, 0, drillBuf.capacity());
-      return allocated < maxAllocation;
-    }
-  }
-
-  /*
-   * It's not clear why we'd allow anyone to set their own limit, need to see why this is used;
-   * this also doesn't make sense when the limits are constantly shifting, nor for other
-   * allocation policies.
-   */
-  @Deprecated
-  @Override
-  public void setFragmentLimit(long fragmentLimit) {
-    throw new UnsupportedOperationException("unimplemented:BaseAllocator.setFragmentLimit()");
-  }
-
-  /**
-   * Get the fragment limit. This was originally meant to be the maximum amount
-   * of memory the currently running fragment (which owns this allocator or
-   * its ancestor) may use. Note that the value may vary up and down over time
-   * as fragments come and go on the node.
-   *
-   * <p>This is deprecated because the concept is not entirely stable. This
-   * only makes sense for one particular memory allocation policy, which is the
-   * one that sets limits on what fragments on a node may use by dividing up all
-   * the memory evenly between all the fragments (see {@see #POLICY_PER_FRAGMENT}).
-   * Other allocation policies, such as the one that limits memory on a
-   * per-query-per-node basis, wouldn't have a value for this. But we need to have
-   * something until we figure out what to eplace this with because it is used by
-   * some operators (such as ExternalSortBatch) to determine how much memory they
-   * can use before they have to spill to disk.</p>
-   *
-   * @return the fragment limit
-   */
-  @Deprecated
-  @Override
-  public long getFragmentLimit() {
-    return policyAgent.getMemoryLimit(this);
-  }
-
-  @Override
-  public void close() {
-    /*
-     * Some owners may close more than once because of complex cleanup and shutdown
-     * procedures.
-     */
-    if (isClosed) {
-      return;
-    }
-
-    synchronized(ALLOCATOR_LOCK) {
-      if (DEBUG) {
-        verifyAllocator();
-
-        // are there outstanding child allocators?
-        if (!childAllocators.isEmpty()) {
-          for(final BaseAllocator childAllocator : childAllocators.keySet()) {
-            if (childAllocator.isClosed) {
-              logger.debug(String.format(
-                  "Closed child allocator[%d] on parent allocator[%d]'s child list",
-                  childAllocator.id, id));
-            }
-          }
-
-          historicalLog.logHistory(logger);
-          logChildren();
-
-          throw new IllegalStateException(
-              String.format("Allocator[%d] closed with outstanding child allocators", id));
-        }
-
-        // are there outstanding buffers?
-        final int allocatedCount = allocatedBuffers.size();
-        if (allocatedCount > 0) {
-          historicalLog.logHistory(logger);
-          logBuffers();
-
-          throw new IllegalStateException(
-              String.format("Allocator[%d] closed with outstanding buffers allocated (%d)",
-                  id, allocatedCount));
-        }
-
-        if (reservations.size() != 0) {
-          historicalLog.logHistory(logger);
-          logReservations(ReservationsLog.ALL);
-
-          throw new IllegalStateException(
-              String.format("Allocator closed with outstanding reservations (%d)", reservations.size()));
-        }
-
-        /* TODO(DRILL-2740)
-        // We should be the only client holding a reference to empty now.
-        final int emptyRefCnt = empty.refCnt();
-        if (emptyRefCnt != 1) {
-          final String msg = "empty buffer refCnt() == " + emptyRefCnt + " (!= 1)";
-          final StringWriter stringWriter = new StringWriter();
-          stringWriter.write(msg);
-          stringWriter.write('\n');
-          empty.writeState(stringWriter);
-          logger.debug(stringWriter.toString());
-          throw new IllegalStateException(msg);
-        }
-        */
-      }
-
-      // Is there unaccounted-for outstanding allocation?
-      if (allocated > 0) {
-        if (DEBUG) {
-          historicalLog.logHistory(logger);
-        }
-        throw new IllegalStateException(
-            String.format("Unaccounted for outstanding allocation (%d)", allocated));
-      }
-
-      // Any unclaimed reservations?
-      if (preallocSpace > 0) {
-        if (DEBUG) {
-          historicalLog.logHistory(logger);
-        }
-        throw new IllegalStateException(
-            String.format("Unclaimed preallocation space (%d)", preallocSpace));
-      }
-
-      /*
-       * Let go of the empty buffer. If the allocator has been closed more than once,
-       * this may not be necessary, so check to avoid illegal states.
-       */
-      final int emptyCount = empty.refCnt();
-      if (emptyCount > 0) {
-        empty.release(emptyCount);
-      }
-
-      DrillAutoCloseables.closeNoChecked(policyAgent);
-
-      // Inform our parent allocator that we've closed.
-      if (parentAllocator != null) {
-        parentAllocator.releaseBytes(owned);
-        owned = 0;
-        parentAllocator.childClosed(this);
-      }
-
-      if (DEBUG) {
-        historicalLog.recordEvent("closed");
-        logger.debug(String.format(
-            "closed allocator[%d]; getsFromParent == %d, putsToParent == %d",
-            id, getsFromParent, putsToParent));
-      }
-
-      isClosed = true;
-    }
-  }
-
-  /**
-   * Log information about child allocators; only works if DEBUG
-   */
-  private void logChildren() {
-    logger.debug(String.format("allocator[%d] open child allocators BEGIN", id));
-    final Set<BaseAllocator> allocators = childAllocators.keySet();
-    for(final BaseAllocator childAllocator : allocators) {
-      childAllocator.historicalLog.logHistory(logger);
-    }
-    logger.debug(String.format("allocator[%d] open child allocators END", id));
-  }
-
-  private void logBuffers() {
-    final StringBuilder sb = new StringBuilder();
-    final Set<UnsafeDirectLittleEndian> udleSet = allocatedBuffers.keySet();
-
-    sb.append("allocator[");
-    sb.append(Integer.toString(id));
-    sb.append("], ");
-    sb.append(Integer.toString(udleSet.size()));
-    sb.append(" allocated buffers\n");
-
-    for(final UnsafeDirectLittleEndian udle : udleSet) {
-      sb.append(udle.toString());
-      sb.append("[identityHashCode == ");
-      sb.append(Integer.toString(System.identityHashCode(udle)));
-      sb.append("]\n");
-
-      final Collection<DrillBuf> drillBufs = DrillBuf.unwrappedGet(udle);
-      for(DrillBuf drillBuf : drillBufs) {
-        drillBuf.logHistory(logger);
-      }
-    }
-
-    logger.debug(sb.toString());
-  }
-
-  private enum ReservationsLog {
-    ALL,
-    UNUSED,
-  }
-
-  private void logReservations(final ReservationsLog reservationsLog) {
-    final StringBuilder sb = new StringBuilder();
-    sb.append(String.format("allocator[%d] reservations BEGIN", id));
-
-    final Set<Reservation> reservations = this.reservations.keySet();
-    for(final Reservation reservation : reservations) {
-      if ((reservationsLog == ReservationsLog.ALL)
-          || ((reservationsLog == ReservationsLog.UNUSED) && (!reservation.isUsed()))) {
-        reservation.writeHistoryToBuilder(sb);
-      }
-    }
-
-    sb.append(String.format("allocator[%d] reservations END", id));
-
-    logger.debug(sb.toString());
-  }
-
-  @Override
-  public long getAllocatedMemory() {
-    return allocated;
-  }
-
-  @Override
-  public int getId() {
-    return id;
-  }
-
-  @Override
-  public long getPeakMemoryAllocation() {
-    return peakAllocated;
-  }
-
-  @Override
-  public DrillBuf getEmpty() {
-    empty.retain(1);
-    // TODO(DRILL-2740) update allocatedBuffers
-    return empty;
-  }
-
-  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(ALLOCATOR_LOCK) {
-          reservations.put(this, this);
-        }
-      } else {
-        historicalLog = null;
-      }
-    }
-
-    @Override
-    public void close() {
-      if (DEBUG) {
-        if (!isClosed()) {
-          final Object object;
-          synchronized(ALLOCATOR_LOCK) {
-            object = reservations.remove(this);
-          }
-          if (object == null) {
-            final StringBuilder sb = new StringBuilder();
-            writeHistoryToBuilder(sb);
-
-            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 boolean reserved;
-      synchronized(ALLOCATOR_LOCK) {
-        reserved = BaseAllocator.this.reserve(null, nBytes, 0);
-        if (reserved) {
-          preallocSpace += nBytes;
-        }
-      }
-
-      if (DEBUG) {
-        historicalLog.recordEvent("reserve(%d) => %s", nBytes, Boolean.toString(reserved));
-      }
-
-      return reserved;
-    }
-
-    @Override
-    protected DrillBuf allocate(int nBytes) {
-      /*
-       * 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.
-       */
-      synchronized(ALLOCATOR_LOCK) {
-        BaseAllocator.this.allocated -= nBytes;
-        final DrillBuf drillBuf = BaseAllocator.this.buffer(nBytes);
-        preallocSpace -= nBytes;
-
-        if (DEBUG) {
-          historicalLog.recordEvent("allocate() => %s",
-              drillBuf == null ? "null" : String.format("DrillBuf[%d]", drillBuf.getId()));
-        }
-
-        return drillBuf;
-      }
-    }
-
-    @Override
-    protected void releaseReservation(int nBytes) {
-      synchronized(ALLOCATOR_LOCK) {
-        releaseBytes(nBytes);
-        preallocSpace -= nBytes;
-      }
-
-      if (DEBUG) {
-        historicalLog.recordEvent("releaseReservation(%d)", nBytes);
-      }
-    }
-
-    private String getState() {
-      return String.format("size == %d, isUsed == %s", getSize(), Boolean.toString(isUsed()));
-    }
-
-    private void writeToBuilder(final StringBuilder sb) {
-      sb.append(String.format("reservation[%d]: ", System.identityHashCode(this)));
-      sb.append(getState());
-    }
-
-    /**
-     * Only works for DEBUG
-     *
-     * @param sb builder to write to
-     */
-    private void writeHistoryToBuilder(final StringBuilder sb) {
-      historicalLog.buildHistory(sb, getState());
-    }
-  }
-
-  @Override
-  public AllocationReservation newReservation() {
-    return new Reservation();
-  }
-
-  /**
-   * Verifies the accounting state of the allocator. Only works for DEBUG.
-   *
-   * @throws IllegalStateException when any problems are found
-   */
-  protected 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
-   */
-  protected void verifyAllocator(
-      final IdentityHashMap<UnsafeDirectLittleEndian, BaseAllocator> buffersSeen) {
-    synchronized(ALLOCATOR_LOCK) {
-      // verify purely local accounting
-      if (allocated > owned) {
-        historicalLog.logHistory(logger);
-        throw new IllegalStateException("Allocator (id = " + id + ") has allocated more than it owns");
-      }
-
-      // the empty buffer should still be empty
-      final long emptyCapacity = empty.maxCapacity();
-      if (emptyCapacity != 0) {
-        throw new IllegalStateException("empty buffer maxCapacity() == " + emptyCapacity + " (!= 0)");
-      }
-
-      // The remaining tests can only be performed if we're in debug mode.
-      if (!DEBUG) {
-        return;
-      }
-
-      // 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 += childAllocator.owned;
-      }
-      if (childTotal > allocated) {
-        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 (" + allocated + ')');
-      }
-
-      // Furthermore, the amount I've allocated should be that plus buffers I've allocated.
-      long bufferTotal = 0;
-      final Set<UnsafeDirectLittleEndian> udleSet = allocatedBuffers.keySet();
-      for(final UnsafeDirectLittleEndian udle : udleSet) {
-        /*
-         * 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 (reservedTotal != preallocSpace) {
-        logReservations(ReservationsLog.UNUSED);
-
-        throw new IllegalStateException(
-            String.format("This allocator's reservedTotal(%d) doesn't match preallocSpace (%d)",
-                reservedTotal, preallocSpace));
-      }
-
-      if (bufferTotal + reservedTotal + childTotal != allocated) {
-        final StringBuilder sb = new StringBuilder();
-        sb.append("allocator[");
-        sb.append(Integer.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, udleSet);
-        }
-
-        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(Integer.toString(childAllocator.id));
-            sb.append("] owned ");
-            sb.append(Long.toString(childAllocator.owned));
-            sb.append('\n');
-          }
-        }
-
-        if (reservedTotal != 0) {
-          sb.append(String.format("reserved total : ", reservedTotal));
-          for(final Reservation reservation : reservationSet) {
-            reservation.writeToBuilder(sb);
-            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));
-      }
-    }
-  }
-
-  private void dumpBuffers(final StringBuilder sb, final Set<UnsafeDirectLittleEndian> udleSet) {
-    for(final UnsafeDirectLittleEndian udle : udleSet) {
-      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 boolean isDebug() {
-    return DEBUG;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BufferLedger.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BufferLedger.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BufferLedger.java
deleted file mode 100644
index d649940..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/BufferLedger.java
+++ /dev/null
@@ -1,94 +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.exec.util.Pointer;
-
-import io.netty.buffer.DrillBuf;
-import io.netty.buffer.PooledByteBufAllocatorL;
-
-/**
- * BufferLedger is an interface meant to facility the private
- * exchange of information between a DrillBuf and its owning
- * allocator. To that end, a number of DrillBuf constructors
- * and methods take a BufferLedger as an argument, yet there
- * are no public implementations of BufferLedger; they all
- * come from inner classes implemented by allocators, ensuring
- * that allocators can give DrillBufs the access they need when
- * they are created or asked to perform complex operations such
- * as ownership sharing or transfers.
- */
-public interface BufferLedger {
-  /**
-   * Get the underlying pooled allocator used by this ledger's
-   * allocator.
-   *
-   * <p>This is usually used to create the shared singleton
-   * empty buffer. Don't use it to create random buffers, because
-   * they won't be tracked, and we won't be able to find leaks.</p>
-   *
-   * @return the underlying pooled allocator
-   */
-  public PooledByteBufAllocatorL getUnderlyingAllocator();
-
-  /**
-   * Return a buffer's memory to the allocator.
-   *
-   * @param drillBuf the DrillBuf that was freed
-   */
-  public void release(DrillBuf drillBuf);
-
-  /**
-   * Share ownership of a buffer with another allocator. As far as reader
-   * and writer index positions go, this acts like a new slice that is owned
-   * by the target allocator, but which has it's own lifetime (i.e., it doesn't
-   * share the fate of the original buffer, unlike real slices).
-   *
-   * @param pDrillBuf returns the new DrillBuf that is shared
-   * @param otherLedger the ledger the new DrillBuf should use
-   * @param otherAllocator the new allocator-owner
-   * @param drillBuf the original DrillBuf to be shared
-   * @param index the starting index to be shared (as for slicing)
-   * @param length the length to be shared (as for slicing)
-   * @param drillBufFlags private flags passed through from the allocator
-   *   (this call originates with a call to BufferAllocator.shareOwnership()).
-   * @return the ledger the calling DrillBuf must use from this point forward;
-   *   this may not match it's original ledger, as allocators provide multiple
-   *   implementations of ledgers to cope with sharing and slicing
-   */
-  public BufferLedger shareWith(Pointer<DrillBuf> pDrillBuf,
-      BufferLedger otherLedger, BufferAllocator otherAllocator,
-      DrillBuf drillBuf, int index, int length, int drillBufFlags);
-
-  /**
-   * Transfer the ownership of a buffer to another allocator. This doesn't change
-   * any of the buffer's reader or writer positions or size, just which allocator
-   * owns it. The reference count stays the same.
-   *
-   * @param newAlloc the new allocator (the one to transfer to)
-   * @param pNewLedger a Pointer<> initialized with a candidate ledger; this
-   *   may be used, or it may not, depending on the sharing state of the buffer.
-   *   The caller is required to use whatever ledger is in pNewLedger on return
-   * @param drillBuf the buffer to transfer
-   * @return true if the transfer kept the target allocator within its maximum
-   *   allocation limit; false if the allocator now owns more memory than its
-   *   creation-time maximum
-   */
-  public boolean transferTo(final BufferAllocator newAlloc,
-      final Pointer<BufferLedger> pNewLedger, final DrillBuf drillBuf);
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java
deleted file mode 100644
index 37475fd..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChainedAllocatorOwner.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.memory;
-
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.testing.ExecutionControls;
-
-/**
- * An {@link AllocatorOwner} implementation that supports linking ownership to
- * a parent object. This can be convenient for identification purposes, when the
- * parent of the allocator's owner is a better handle for debugging.
- *
- * <p>The implementation of {@link #getExecutionControls()} returns the childOwner's
- * response to getExecutionControls().</p>
- */
-public class ChainedAllocatorOwner implements AllocatorOwner {
-  private final AllocatorOwner childOwner;
-  private final AllocatorOwner parentOwner;
-
-  /**
-   * Constructor.
-   *
-   * @param childOwner the owner of the allocator
-   * @param parentOwner the object that owns or created the childOwner
-   */
-  public ChainedAllocatorOwner(AllocatorOwner childOwner, AllocatorOwner parentOwner) {
-    this.childOwner = childOwner;
-    this.parentOwner = parentOwner;
-  }
-
-  @Override
-  public String toString() {
-    return childOwner + "(owned by " + parentOwner + ')';
-  }
-
-  @Override
-  public ExecutionControls getExecutionControls() {
-    return childOwner.getExecutionControls();
-  }
-
-  @Override
-  public FragmentContext getFragmentContext() {
-    return childOwner.getFragmentContext();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java
deleted file mode 100644
index 8636d26..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java
+++ /dev/null
@@ -1,47 +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;
-
-/**
- * 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, AllocatorOwner allocatorOwner,
-      AllocationPolicy allocationPolicy, long initReservation, long maxAllocation, int flags) {
-    super(parentAllocator, allocatorOwner, allocationPolicy, initReservation, maxAllocation, flags);
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocator.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocator.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocator.java
deleted file mode 100644
index 63b9987..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocator.java
+++ /dev/null
@@ -1,125 +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.lang.management.ManagementFactory;
-
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.PooledByteBufAllocatorL;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.testing.ExecutionControls;
-
-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 {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootAllocator.class);
-
-  // TODO these statics, and others in BaseAllocator, may be a problem for multiple in-process Drillbits
-  private static final PooledByteBufAllocatorL innerAllocator = PooledByteBufAllocatorL.DEFAULT;
-  private static long maxDirect;
-
-  public static AllocationPolicy getAllocationPolicy() {
-    final String policyName = System.getProperty(ExecConstants.ALLOCATION_POLICY,
-        BaseAllocator.POLICY_LOCAL_MAX_NAME); // TODO try with PER_FRAGMENT_NAME
-
-    switch(policyName) {
-    case POLICY_PER_FRAGMENT_NAME:
-      return POLICY_PER_FRAGMENT;
-    case POLICY_LOCAL_MAX_NAME:
-      return POLICY_LOCAL_MAX;
-    default:
-      throw new IllegalArgumentException("Unrecognized allocation policy name \"" + policyName + "\"");
-    }
-  }
-
-  public RootAllocator(final DrillConfig drillConfig) {
-    this(getAllocationPolicy(), 0, Math.min(
-        DrillConfig.getMaxDirectMemory(), drillConfig.getLong(ExecConstants.TOP_LEVEL_MAX_ALLOC)), 0);
-  }
-
-  public static long getMaxDirect() {
-    return maxDirect;
-  }
-
-  /**
-   * Provide statistics via JMX for each RootAllocator.
-   */
-  private class AllocatorsStats implements AllocatorsStatsMXBean {
-    @Override
-    public long getMaxDirectMemory() {
-      return maxDirect;
-    }
-  }
-
-  private static class RootAllocatorOwner implements AllocatorOwner {
-    @Override
-    public ExecutionControls getExecutionControls() {
-      return null;
-    }
-
-    @Override
-    public FragmentContext getFragmentContext() {
-      return null;
-    }
-  }
-
-  @VisibleForTesting
-  public RootAllocator(final AllocationPolicy allocationPolicy,
-      final long initAllocation, final long maxReservation, final int flags) {
-    super(null, new RootAllocatorOwner(), allocationPolicy, initAllocation, maxDirect = maxReservation, flags);
-    assert (flags & F_LIMITING_ROOT) == 0 : "the RootAllocator shouldn't be a limiting root";
-
-    try {
-      final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-      final ObjectName objectName = new ObjectName("org.apache.drill.exec.memory:Allocators=" + id);
-      final AllocatorsStats mbean = new AllocatorsStats();
-      mbs.registerMBean(mbean, objectName);
-    } catch(Exception e) {
-      logger.info("Exception setting up AllocatorsStatsMBean", e);
-    }
-  }
-
-  @Override
-  public ByteBufAllocator getUnderlyingAllocator() {
-    return innerAllocator;
-  }
-
-  @Override
-  protected boolean canIncreaseOwned(final long nBytes, final int flags) {
-    // the end total has already been checked against maxAllocation, so we can just return true
-    return true;
-  }
-
-  /**
-   * Verify the accounting state of the allocation system.
-   */
-  @VisibleForTesting
-  public void verify() {
-    verifyAllocator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java
----------------------------------------------------------------------
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 078fc21..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/RootAllocatorFactory.java
+++ /dev/null
@@ -1,43 +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;
-
-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(cwestin)
-    if (BaseAllocator.DEBUG) {
-      return new RootAllocator(drillConfig);
-    }
-*/
-    return new RootAllocator(drillConfig);
-    // TODO(cwestin) return new TopLevelAllocator(drillConfig);
-  }
-}


[05/13] drill git commit: DRILL-4134: Add new allocator

Posted by ja...@apache.org.
DRILL-4134: Add new allocator


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

Branch: refs/heads/master
Commit: 53dcabeb83f53c8e29aff9c9282eaaa20a8b27ee
Parents: a466e4b
Author: Chris Westin <cw...@yahoo.com>
Authored: Wed Nov 11 14:57:47 2015 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Dec 21 23:52:28 2015 -0800

----------------------------------------------------------------------
 .../drill/common/AutoCloseablePointer.java      |  112 ++
 .../apache/drill/common/DrillCloseables.java    |   55 +
 .../org/apache/drill/common/HistoricalLog.java  |  179 ++
 .../org/apache/drill/exec/ops/QueryContext.java |    8 +-
 .../exec/physical/impl/TopN/TopNBatch.java      |    5 +-
 .../impl/sort/SortRecordBatchBuilder.java       |   53 +-
 .../apache/drill/exec/rpc/data/DataServer.java  |   21 +-
 .../java/org/apache/drill/BaseTestQuery.java    |   29 +
 .../java/org/apache/drill/TestTpchLimit0.java   |    6 +
 .../drill/exec/memory/TestAllocators.java       |   26 +-
 .../drill/exec/memory/TestBaseAllocator.java    |  651 +++++++
 .../drill/exec/testing/TestResourceLeak.java    |   19 +-
 .../src/main/java/io/netty/buffer/DrillBuf.java |  671 +++++--
 .../java/io/netty/buffer/FakeAllocator.java     |   68 +-
 .../netty/buffer/UnsafeDirectLittleEndian.java  |   60 +-
 .../drill/exec/memory/BufferAllocator.java      |  166 +-
 .../drill/exec/memory/AllocationPolicy.java     |   38 +
 .../exec/memory/AllocationPolicyAgent.java      |   69 +
 .../exec/memory/AllocationReservation.java      |  152 ++
 .../exec/memory/AllocatorClosedException.java   |   31 +
 .../drill/exec/memory/AllocatorOwner.java       |   40 +
 .../exec/memory/AllocatorsStatsMXBean.java      |   34 +
 .../apache/drill/exec/memory/BaseAllocator.java | 1654 ++++++++++++++++++
 .../apache/drill/exec/memory/BufferLedger.java  |   94 +
 .../exec/memory/ChainedAllocatorOwner.java      |   60 +
 .../drill/exec/memory/ChildAllocator.java       |   47 +
 .../apache/drill/exec/memory/RootAllocator.java |  125 ++
 .../drill/exec/memory/RootAllocatorFactory.java |   32 +-
 .../drill/exec/memory/TopLevelAllocator.java    |  160 +-
 29 files changed, 4317 insertions(+), 348 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/common/src/main/java/org/apache/drill/common/AutoCloseablePointer.java
----------------------------------------------------------------------
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 0000000..3a0ac4a
--- /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);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/common/src/main/java/org/apache/drill/common/DrillCloseables.java
----------------------------------------------------------------------
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 0000000..289066b
--- /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);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/common/src/main/java/org/apache/drill/common/HistoricalLog.java
----------------------------------------------------------------------
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 0000000..49b7c33
--- /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.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) {
+    buildHistory(sb, null);
+  }
+
+  /**
+   * 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
+   */
+  public synchronized void buildHistory(final StringBuilder sb, final CharSequence additional) {
+    sb.append('\n')
+        .append(idString);
+
+    if (additional != null) {
+      sb.append('\n')
+          .append(additional)
+          .append('\n');
+    }
+
+    sb.append(" event log\n");
+
+    if (firstEvent != null) {
+      sb.append("  ")
+          .append(firstEvent.note)
+          .append('\n');
+      firstEvent.stackTrace.writeToBuilder(sb, 4);
+
+      for(final Event event : history) {
+        if (event == firstEvent) {
+          continue;
+        }
+
+        sb.append("  ")
+            .append(event.note)
+            .append('\n');
+
+        event.stackTrace.writeToBuilder(sb, 4);
+      }
+    }
+  }
+
+  /**
+   * 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
+   * @param additional an extra string that will be written between the identifying
+   *     information and the history; often used for a current piece of state
+   */
+  public void logHistory(final Logger logger, final CharSequence additional) {
+    final StringBuilder sb = new StringBuilder();
+    buildHistory(sb, additional);
+    logger.debug(sb.toString());
+  }
+
+  /**
+   * 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) {
+    logHistory(logger, null);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
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 3bcd111..e4f50a1 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
@@ -92,12 +92,8 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext {
     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);
-    }
+    allocator = drillbitContext.getAllocator().getChildAllocator(null, plannerSettings.getInitialPlanningMemorySize(),
+        plannerSettings.getPlanningMemoryLimit(), false);
     // TODO(DRILL-1942) the new allocator has this capability built-in, so this can be removed once that is available
     bufferManager = new BufferManagerImpl(this.allocator);
     viewExpansionContext = new ViewExpansionContext(this);

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
----------------------------------------------------------------------
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 8c4cf21..c287bc3 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.Iterator;
 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;
@@ -59,7 +61,6 @@ import org.apache.drill.exec.record.selection.SelectionVector2;
 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;
@@ -323,7 +324,7 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
       builder.getSv4().clear();
       selectionVector4.clear();
     } finally {
-      builder.close();
+      DrillAutoCloseables.closeNoChecked(builder);
     }
     logger.debug("Took {} us to purge", watch.elapsed(TimeUnit.MICROSECONDS));
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
----------------------------------------------------------------------
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 e5d1e4e..9f452cb 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,24 @@
  */
 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.AutoCloseablePointer;
+import org.apache.drill.common.DrillAutoCloseables;
 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;
@@ -52,15 +56,12 @@ public class SortRecordBatchBuilder implements AutoCloseable {
   private final ArrayListMultimap<BatchSchema, RecordBatchData> batches = ArrayListMultimap.create();
 
   private int recordCount;
+  private final AutoCloseablePointer<SelectionVector4> pSV4 = new AutoCloseablePointer<>();
+  private final AllocationReservation allocationReservation;
   private long runningBatches;
-  private SelectionVector4 sv4;
-  private BufferAllocator allocator;
-  final PreAllocator svAllocator;
-  private boolean svAllocatorUsed = false;
 
-  public SortRecordBatchBuilder(BufferAllocator a) {
-    this.allocator = a;
-    this.svAllocator = a.getNewPreAllocator();
+  public SortRecordBatchBuilder(final BufferAllocator allocator) {
+    allocationReservation = allocator.newReservation();
   }
 
   private long getSize(VectorAccessible batch) {
@@ -93,7 +94,7 @@ public class SortRecordBatchBuilder implements AutoCloseable {
     if (runningBatches >= Character.MAX_VALUE) {
       return false; // allowed in batch.
     }
-    if (!svAllocator.preAllocate(batch.getRecordCount()*4)) {
+    if (!allocationReservation.add(batch.getRecordCount() * 4)) {
       return false;  // sv allocation available.
     }
 
@@ -116,7 +117,7 @@ public class SortRecordBatchBuilder implements AutoCloseable {
       logger.error(errMsg);
       throw new DrillRuntimeException(errMsg);
     }
-    if(!svAllocator.preAllocate(rbd.getRecordCount()*4)) {
+    if (!allocationReservation.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,14 +160,14 @@ public class SortRecordBatchBuilder implements AutoCloseable {
       assert false : "Invalid to have an empty set of batches with no schemas.";
     }
 
-    final DrillBuf svBuffer = svAllocator.getAllocation();
+    final DrillBuf svBuffer = allocationReservation.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);
+    pSV4.assignNoChecked(new SelectionVector4(svBuffer, recordCount, Character.MAX_VALUE));
     BatchSchema schema = batches.keySet().iterator().next();
     List<RecordBatchData> data = batches.get(schema);
+    final SelectionVector4 sv4 = pSV4.get();
 
     // now we're going to generate the sv4 pointers
     switch (schema.getSelectionVectorMode()) {
@@ -215,27 +216,16 @@ public class SortRecordBatchBuilder implements AutoCloseable {
   }
 
   public SelectionVector4 getSv4() {
-    return sv4;
+    return pSV4.get();
   }
 
   public void clear() {
+    DrillAutoCloseables.closeNoChecked(allocationReservation);
     for (RecordBatchData d : batches.values()) {
       d.container.clear();
     }
-    if (sv4 != null) {
-      sv4.clear();
-    }
-  }
-
-  @Override
-  public void close() {
-    // Don't leak unused pre-allocated memory.
-    if (!svAllocatorUsed) {
-      final DrillBuf drillBuf = svAllocator.getAllocation();
-      if (drillBuf != null) {
-        drillBuf.release();
-      }
-    }
+    batches.clear();
+    pSV4.assignNoChecked(null);
   }
 
   public List<VectorContainer> getHeldRecordBatches() {
@@ -262,4 +252,11 @@ public class SortRecordBatchBuilder implements AutoCloseable {
     // We need 4 bytes (SV4) for each record.
     return recordCount * 4;
   }
+
+  @Override
+  public void close() {
+    clear();
+    allocationReservation.close();
+    DrillAutoCloseables.closeNoChecked(pSV4);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java
----------------------------------------------------------------------
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 ed69699..56e2ff2 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
@@ -26,6 +26,7 @@ import io.netty.util.concurrent.GenericFutureListener;
 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;
@@ -171,7 +172,25 @@ public class DataServer extends BasicServer<RpcType, BitServerConnection> {
 
     final boolean withinMemoryEnvelope;
 
-    withinMemoryEnvelope = allocator.takeOwnership(body, out);
+    try {
+      withinMemoryEnvelope = allocator.shareOwnership((DrillBuf) body, out);
+    } 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.

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
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 42345c0..46f0526 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
@@ -31,6 +31,9 @@ import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.io.Files;
+
+import org.apache.drill.common.DrillAutoCloseables;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.scanner.ClassPathScanner;
@@ -42,6 +45,7 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 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.RootAllocator;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
@@ -68,6 +72,11 @@ import com.google.common.base.Preconditions;
 import com.google.common.io.Files;
 import com.google.common.io.Resources;
 
+import static org.hamcrest.core.StringContains.containsString;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
 public class BaseTestQuery extends ExecTest {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseTestQuery.class);
 
@@ -249,6 +258,26 @@ public class BaseTestQuery extends ExecTest {
     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();
+          if (!(bufferAllocator instanceof RootAllocator)) {
+            throw new IllegalStateException("The DrillbitContext's allocator is not a RootAllocator");
+          }
+          final RootAllocator rootAllocator = (RootAllocator) bufferAllocator;
+          rootAllocator.verify();
+        }
+      }
+    }
+  }
+
   @AfterClass
   public static void closeClient() throws IOException {
     if (client != null) {

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/java-exec/src/test/java/org/apache/drill/TestTpchLimit0.java
----------------------------------------------------------------------
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 22471c8..6d2fbf0 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;");

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java
----------------------------------------------------------------------
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 aa0919a..2389dc9 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,21 +18,30 @@
 
 package org.apache.drill.exec.memory;
 
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import com.google.common.collect.Lists;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
 import io.netty.buffer.DrillBuf;
 
 import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 
+import io.netty.buffer.UnsafeDirectLittleEndian;
+
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OpProfileDef;
@@ -43,6 +52,7 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
 import org.apache.drill.exec.proto.BitControl;
+import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.server.Drillbit;
@@ -67,7 +77,7 @@ public class TestAllocators extends DrillTest {
     }
   };
 
-  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
@@ -254,12 +264,14 @@ public class TestAllocators extends DrillTest {
     ((AutoCloseable) oContext22).close();
 
     // Fragment 3 asks for more and fails
+    boolean outOfMem = false;
     try {
-      oContext31.getAllocator().buffer(4400000);
+      oContext31.getAllocator().buffer(44000000);
       fail("Fragment 3 should fail to allocate buffer");
-    } catch (OutOfMemoryException e) {
-      // expected
+    } catch (OutOfMemoryRuntimeException 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);
@@ -286,6 +298,12 @@ public class TestAllocators extends DrillTest {
 
     bit.close();
     serviceSet.close();
+
+/*
+    // ---------------------------------------- DEBUG ----------------------------------
+    assertEquals(0, UnsafeDirectLittleEndian.getBufferCount());
+    // ---------------------------------------- DEBUG ----------------------------------
+*/
   }
 
   private void closeOp(OperatorContext c) throws Exception {

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java
new file mode 100644
index 0000000..6ea5670
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java
@@ -0,0 +1,651 @@
+/**
+ * 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.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import io.netty.buffer.DrillBuf;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.testing.ExecutionControls;
+import org.apache.drill.exec.util.Pointer;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+
+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 ------------------------------------
+*/
+
+  // Concoct ExecutionControls that won't try to inject anything.
+  @Mock private static final OptionManager optionManager = Mockito.mock(OptionManager.class);
+  static {
+    Mockito.when(optionManager.getOption(Matchers.anyString()))
+      .thenReturn(null);
+  }
+
+  @Mock private static final ExecutionControls executionControls = new ExecutionControls(optionManager, null);
+
+  private final static class NamedOwner implements AllocatorOwner {
+    private final String name;
+
+    public NamedOwner(final String name) {
+      this.name = name;
+    }
+
+    @Override
+    public String toString() {
+      return name;
+    }
+
+    @Override
+    public ExecutionControls getExecutionControls() {
+      return executionControls;
+    }
+
+    @Override
+    public FragmentContext getFragmentContext() {
+      return null;
+    }
+  }
+
+  @Test
+  public void test_privateMax() throws Exception {
+    final AllocatorOwner allocatorOwner = new NamedOwner("noLimits");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_LOCAL_MAX, 0, MAX_ALLOCATION, 0)) {
+      final DrillBuf drillBuf1 = rootAllocator.buffer(MAX_ALLOCATION / 2);
+      assertNotNull("allocation failed", drillBuf1);
+
+      try(final BufferAllocator childAllocator =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
+        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(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+        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(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      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(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      @SuppressWarnings("unused")
+      final DrillBuf drillBuf = rootAllocator.buffer(0);
+    }
+  }
+
+  @Test
+  public void testAllocator_transferOwnership() throws Exception {
+    final AllocatorOwner allocatorOwner = new NamedOwner("changeOwnership");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      final BufferAllocator childAllocator1 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+      final BufferAllocator childAllocator2 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+
+      final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 4);
+      rootAllocator.verify();
+      final boolean allocationFit = childAllocator2.takeOwnership(drillBuf1);
+      rootAllocator.verify();
+      assertTrue(allocationFit);
+
+      childAllocator1.close();
+      rootAllocator.verify();
+
+      drillBuf1.release();
+      childAllocator2.close();
+    }
+  }
+
+  @Test
+  public void testAllocator_shareOwnership() throws Exception {
+    final AllocatorOwner allocatorOwner = new NamedOwner("shareOwnership");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      final BufferAllocator childAllocator1 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+      final BufferAllocator childAllocator2 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+      final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 4);
+      rootAllocator.verify();
+      final Pointer<DrillBuf> pDrillBuf = new Pointer<>();
+      boolean allocationFit;
+
+      allocationFit = childAllocator2.shareOwnership(drillBuf1, pDrillBuf);
+      assertTrue(allocationFit);
+      rootAllocator.verify();
+      final DrillBuf drillBuf2 = pDrillBuf.value;
+      assertNotNull(drillBuf2);
+      assertNotEquals(drillBuf2, drillBuf1);
+
+      drillBuf1.release();
+      rootAllocator.verify();
+      childAllocator1.close();
+      rootAllocator.verify();
+
+      final BufferAllocator childAllocator3 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+      allocationFit = childAllocator3.shareOwnership(drillBuf2, pDrillBuf);
+      assertTrue(allocationFit);
+      final DrillBuf drillBuf3 = pDrillBuf.value;
+      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 {
+    final AllocatorOwner allocatorOwner = new NamedOwner("createChildAndUse");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      try(final BufferAllocator childAllocator =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
+        final DrillBuf drillBuf = childAllocator.buffer(512);
+        assertNotNull("allocation failed", drillBuf);
+        drillBuf.release();
+      }
+    }
+  }
+
+  @Test(expected=IllegalStateException.class)
+  public void testRootAllocator_createChildDontClose() throws Exception {
+    try {
+      final AllocatorOwner allocatorOwner = new NamedOwner("createChildDontClose");
+      try(final RootAllocator rootAllocator =
+          new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+        final BufferAllocator childAllocator =
+            rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+        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 {
+    final AllocatorOwner allocatorOwner = new NamedOwner("manyAllocations");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      try(final BufferAllocator childAllocator =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
+        allocateAndFree(childAllocator);
+      }
+    }
+  }
+
+  @Test
+  public void testAllocator_overAllocate() throws Exception {
+    final AllocatorOwner allocatorOwner = new NamedOwner("overAllocate");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      try(final BufferAllocator childAllocator =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
+        allocateAndFree(childAllocator);
+
+        try {
+          childAllocator.buffer(MAX_ALLOCATION + 1);
+          fail("allocated memory beyond max allowed");
+        } catch(OutOfMemoryRuntimeException e) {
+          // expected
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testAllocator_overAllocateParent() throws Exception {
+    final AllocatorOwner allocatorOwner = new NamedOwner("overAllocateParent");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      try(final BufferAllocator childAllocator =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
+        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(OutOfMemoryRuntimeException 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 {
+    final AllocatorOwner allocatorOwner = new NamedOwner("createSlices");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      testAllocator_sliceUpBufferAndRelease(rootAllocator, rootAllocator);
+
+      try(final BufferAllocator childAllocator =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
+        testAllocator_sliceUpBufferAndRelease(rootAllocator, childAllocator);
+      }
+      rootAllocator.verify();
+
+      testAllocator_sliceUpBufferAndRelease(rootAllocator, rootAllocator);
+
+      try(final BufferAllocator childAllocator =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
+        try(final BufferAllocator childAllocator2 =
+            childAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
+          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(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      // Populate a buffer with byte values corresponding to their indices.
+      final DrillBuf drillBuf = rootAllocator.buffer(256, 256 + 256);
+      assertEquals(256, drillBuf.capacity());
+      assertEquals(256 + 256, drillBuf.maxCapacity());
+      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(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      // Populate a buffer with byte values corresponding to their indices.
+      final DrillBuf drillBuf = rootAllocator.buffer(256, 256 + 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 {
+    final AllocatorOwner allocatorOwner = new NamedOwner("transferSliced");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      final BufferAllocator childAllocator1 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+      final BufferAllocator childAllocator2 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+
+      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();
+
+      childAllocator1.takeOwnership(drillBuf2s);
+      rootAllocator.verify();
+      childAllocator2.takeOwnership(drillBuf1s);
+      rootAllocator.verify();
+
+      drillBuf1s.release(); // releases drillBuf1
+      drillBuf2s.release(); // releases drillBuf2
+
+      childAllocator1.close();
+      childAllocator2.close();
+    }
+  }
+
+  @Test
+  public void testAllocator_shareSliced() throws Exception {
+    final AllocatorOwner allocatorOwner = new NamedOwner("transferSliced");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      final BufferAllocator childAllocator1 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+      final BufferAllocator childAllocator2 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+
+      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 Pointer<DrillBuf> pDrillBuf = new Pointer<>();
+      childAllocator1.shareOwnership(drillBuf2s, pDrillBuf);
+      final DrillBuf drillBuf2s1 = pDrillBuf.value;
+      childAllocator2.shareOwnership(drillBuf1s, pDrillBuf);
+      final DrillBuf drillBuf1s2 = pDrillBuf.value;
+      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 {
+    final AllocatorOwner allocatorOwner = new NamedOwner("transferShared");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      final BufferAllocator childAllocator1 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+      final BufferAllocator childAllocator2 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+      final BufferAllocator childAllocator3 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+
+      final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 8);
+
+      final Pointer<DrillBuf> pDrillBuf = new Pointer<>();
+      boolean allocationFit;
+
+      allocationFit = childAllocator2.shareOwnership(drillBuf1, pDrillBuf);
+      assertTrue(allocationFit);
+      rootAllocator.verify();
+      final DrillBuf drillBuf2 = pDrillBuf.value;
+      assertNotNull(drillBuf2);
+      assertNotEquals(drillBuf2, drillBuf1);
+
+      allocationFit = childAllocator3.takeOwnership(drillBuf1);
+      assertTrue(allocationFit);
+      rootAllocator.verify();
+
+      // Since childAllocator3 now has childAllocator1's buffer, 1, can close
+      childAllocator1.close();
+      rootAllocator.verify();
+
+      drillBuf2.release();
+      childAllocator2.close();
+      rootAllocator.verify();
+
+      final BufferAllocator childAllocator4 =
+          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
+      allocationFit = childAllocator4.takeOwnership(drillBuf1);
+      assertTrue(allocationFit);
+      rootAllocator.verify();
+
+      childAllocator3.close();
+      rootAllocator.verify();
+
+      drillBuf1.release();
+      childAllocator4.close();
+      rootAllocator.verify();
+    }
+  }
+
+  @Test
+  public void testAllocator_unclaimedReservation() throws Exception {
+    final AllocatorOwner allocatorOwner = new NamedOwner("unclaimedReservation");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      try(final BufferAllocator childAllocator1 =
+            rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
+        try(final AllocationReservation reservation = childAllocator1.newReservation()) {
+          assertTrue(reservation.add(64));
+        }
+        rootAllocator.verify();
+      }
+    }
+  }
+
+  @Test
+  public void testAllocator_claimedReservation() throws Exception {
+    final AllocatorOwner allocatorOwner = new NamedOwner("claimedReservation");
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
+      try(final BufferAllocator childAllocator1 =
+            rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
+        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();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestResourceLeak.java
----------------------------------------------------------------------
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 dc539c5..7ab2da2 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,6 +17,8 @@
  */
 package org.apache.drill.exec.testing;
 
+import static org.junit.Assert.fail;
+
 import com.google.common.base.Charsets;
 import com.google.common.io.Resources;
 
@@ -42,6 +44,7 @@ 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;
@@ -50,6 +53,18 @@ import java.io.IOException;
 import java.net.URL;
 import java.util.Properties;
 
+/*
+ * 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 +98,12 @@ public class TestResourceLeak extends DrillTest {
     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 {


[12/13] drill git commit: DRILL-4134: Allocator Improvements

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/DataServer.java
----------------------------------------------------------------------
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 56e2ff2..03118d7 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
@@ -24,9 +24,9 @@ import io.netty.channel.socket.SocketChannel;
 import io.netty.util.concurrent.GenericFutureListener;
 
 import java.io.IOException;
+import java.util.concurrent.ThreadLocalRandom;
 
 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;
@@ -44,7 +44,7 @@ import org.apache.drill.exec.rpc.ResponseSender;
 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.WorkManager.WorkerBee;
 import org.apache.drill.exec.work.fragment.FragmentManager;
 
 import com.google.protobuf.MessageLite;
@@ -55,16 +55,17 @@ public class DataServer extends BasicServer<RpcType, BitServerConnection> {
   private volatile ProxyCloseHandler proxyCloseHandler;
   private final BootStrapContext context;
   private final WorkEventBus workBus;
-  private final DataResponseHandler dataHandler;
+  private final WorkerBee bee;
 
-  public DataServer(BootStrapContext context, WorkEventBus workBus, DataResponseHandler dataHandler) {
+  public DataServer(BootStrapContext context, BufferAllocator alloc, WorkEventBus workBus,
+      WorkerBee bee) {
     super(
         DataRpcConfig.getMapping(context.getConfig(), context.getExecutor()),
-        context.getAllocator().getUnderlyingAllocator(),
+        alloc.getAsByteBufAllocator(),
         context.getBitLoopGroup());
     this.context = context;
     this.workBus = workBus;
-    this.dataHandler = dataHandler;
+    this.bee = bee;
   }
 
   @Override
@@ -106,9 +107,6 @@ public class DataServer extends BasicServer<RpcType, BitServerConnection> {
     };
   }
 
-  private final static FragmentRecordBatch OOM_FRAGMENT = FragmentRecordBatch.newBuilder().setIsOutOfMemory(true).build();
-
-
   private static FragmentHandle getHandle(FragmentRecordBatch batch, int index) {
     return FragmentHandle.newBuilder()
         .setQueryId(batch.getQueryId())
@@ -117,34 +115,50 @@ public class DataServer extends BasicServer<RpcType, BitServerConnection> {
         .build();
   }
 
+  private void submit(IncomingDataBatch batch, int minorStart, int minorStopExclusive) throws FragmentSetupException,
+      IOException {
+    for (int minor = minorStart; minor < minorStopExclusive; minor++) {
+      final FragmentManager manager = workBus.getFragmentManager(getHandle(batch.getHeader(), minor));
+      if (manager == null) {
+        // A missing manager means the query already terminated. We can simply drop this data.
+        continue;
+      }
+
+      final boolean canRun = manager.handle(batch);
+      if (canRun) {
+        // logger.debug("Arriving batch means local batch can run, starting local batch.");
+        /*
+         * If we've reached the canRun threshold, we'll proceed. This expects manager.handle() to only return a single
+         * true. This is guaranteed by the interface.
+         */
+        bee.startFragmentPendingRemote(manager);
+      }
+    }
+
+  }
 
   @Override
   protected void handle(BitServerConnection connection, int rpcType, ByteBuf pBody, ByteBuf body, ResponseSender sender) throws RpcException {
     assert rpcType == RpcType.REQ_RECORD_BATCH_VALUE;
 
     final FragmentRecordBatch fragmentBatch = get(pBody, FragmentRecordBatch.PARSER);
-    final int targetCount = fragmentBatch.getReceivingMinorFragmentIdCount();
+    final AckSender ack = new AckSender(sender);
+
 
-    AckSender ack = new AckSender(sender);
     // increment so we don't get false returns.
     ack.increment();
+
     try {
 
-      if(body == null){
+      final IncomingDataBatch batch = new IncomingDataBatch(fragmentBatch, (DrillBuf) body, ack);
+      final int targetCount = fragmentBatch.getReceivingMinorFragmentIdCount();
 
-        for(int minor = 0; minor < targetCount; minor++){
-          FragmentManager manager = workBus.getFragmentManager(getHandle(fragmentBatch, minor));
-          if(manager != null){
-            ack.increment();
-            dataHandler.handle(manager, fragmentBatch, null, ack);
-          }
-        }
+      // randomize who gets first transfer (and thus ownership) so memory usage is balanced when we're sharing amongst
+      // multiple fragments.
+      final int firstOwner = ThreadLocalRandom.current().nextInt(targetCount);
+      submit(batch, firstOwner, targetCount);
+      submit(batch, 0, firstOwner);
 
-      }else{
-        for (int minor = 0; minor < targetCount; minor++) {
-          send(fragmentBatch, (DrillBuf) body, minor, ack);
-        }
-      }
     } catch (IOException | FragmentSetupException e) {
       logger.error("Failure while getting fragment manager. {}",
           QueryIdHelper.getQueryIdentifiers(fragmentBatch.getQueryId(),
@@ -159,51 +173,6 @@ public class DataServer extends BasicServer<RpcType, BitServerConnection> {
     }
   }
 
-  private void send(final FragmentRecordBatch fragmentBatch, final DrillBuf body, final int minor, final AckSender ack)
-      throws FragmentSetupException, IOException {
-
-    final FragmentManager manager = workBus.getFragmentManager(getHandle(fragmentBatch, minor));
-    if (manager == null) {
-      return;
-    }
-
-    final BufferAllocator allocator = manager.getFragmentContext().getAllocator();
-    final Pointer<DrillBuf> out = new Pointer<>();
-
-    final boolean withinMemoryEnvelope;
-
-    try {
-      withinMemoryEnvelope = allocator.shareOwnership((DrillBuf) body, out);
-    } 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.
-      dataHandler.handle(manager, OOM_FRAGMENT, null, null);
-    }
-
-    ack.increment();
-    dataHandler.handle(manager, fragmentBatch, out.value, 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();
-  }
 
   private class ProxyCloseHandler implements GenericFutureListener<ChannelFuture> {
 
@@ -226,7 +195,7 @@ public class DataServer extends BasicServer<RpcType, BitServerConnection> {
     return new OutOfMemoryHandler() {
       @Override
       public void handle() {
-        dataHandler.informOutOfMemory();
+        logger.error("Out of memory in RPC layer.");
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/IncomingDataBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/IncomingDataBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/IncomingDataBatch.java
new file mode 100644
index 0000000..a9bc305
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/data/IncomingDataBatch.java
@@ -0,0 +1,73 @@
+/**
+ * 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.rpc.data;
+
+import io.netty.buffer.DrillBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
+import org.apache.drill.exec.record.RawFragmentBatch;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * An incoming batch of data. The data is held by the original allocator. Any use of the associated data must be
+ * leveraged through the use of newRawFragmentBatch().
+ */
+public class IncomingDataBatch {
+
+  private final FragmentRecordBatch header;
+  private final DrillBuf body;
+  private final AckSender sender;
+
+  /**
+   * Create a new batch. Does not impact reference counts of body.
+   *
+   * @param header
+   *          Batch header
+   * @param body
+   *          Data body. Could be null.
+   * @param sender
+   *          AckSender to use for underlying RawFragmentBatches.
+   */
+  public IncomingDataBatch(FragmentRecordBatch header, DrillBuf body, AckSender sender) {
+    Preconditions.checkNotNull(header);
+    Preconditions.checkNotNull(sender);
+    this.header = header;
+    this.body = body;
+    this.sender = sender;
+  }
+
+  /**
+   * Create a new RawFragmentBatch based on this incoming data batch that is transferred into the provided allocator.
+   * Also increments the AckSender to expect one additional return message.
+   *
+   * @param allocator
+   *          Target allocator that should be associated with data underlying this batch.
+   * @return The newly created RawFragmentBatch
+   */
+  public RawFragmentBatch newRawFragmentBatch(final BufferAllocator allocator) {
+    final DrillBuf transferredBuffer = body == null ? null : body.transferOwnership(allocator).buffer;
+    sender.increment();
+    return new RawFragmentBatch(header, transferredBuffer, sender);
+  }
+
+  public FragmentRecordBatch getHeader() {
+    return header;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
----------------------------------------------------------------------
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 ddba213..049ae0c 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 class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
 
   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

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index feebbb2..99523d6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -188,7 +188,7 @@ public class Drillbit implements AutoCloseable {
     context = new BootStrapContext(config, classpathScan);
     manager = new WorkManager(context);
     engine = new ServiceEngine(manager.getControlMessageHandler(), manager.getUserWorker(), context,
-        manager.getWorkBus(), manager.getDataHandler(), allowPortHunting);
+        manager.getWorkBus(), manager.getBee(), allowPortHunting);
 
     webServer = new WebServer(config, context.getMetrics(), manager);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
index d3b4128..e07ca90 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
@@ -38,9 +38,9 @@ import org.apache.drill.exec.rpc.control.Controller;
 import org.apache.drill.exec.rpc.control.ControllerImpl;
 import org.apache.drill.exec.rpc.control.WorkEventBus;
 import org.apache.drill.exec.rpc.data.DataConnectionCreator;
-import org.apache.drill.exec.rpc.data.DataResponseHandler;
 import org.apache.drill.exec.rpc.user.UserServer;
 import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
 import org.apache.drill.exec.work.batch.ControlMessageHandler;
 import org.apache.drill.exec.work.user.UserWorker;
 
@@ -58,7 +58,7 @@ public class ServiceEngine implements Closeable{
   private final boolean allowPortHunting;
 
   public ServiceEngine(ControlMessageHandler controlMessageHandler, UserWorker userWorker, BootStrapContext context,
-      WorkEventBus workBus, DataResponseHandler dataHandler, boolean allowPortHunting) throws DrillbitStartupException {
+      WorkEventBus workBus, WorkerBee bee, boolean allowPortHunting) throws DrillbitStartupException {
     final EventLoopGroup eventLoopGroup = TransportCheck.createEventLoopGroup(
         context.getConfig().getInt(ExecConstants.USER_SERVER_RPC_THREADS), "UserServer-");
     this.userServer = new UserServer(
@@ -69,7 +69,7 @@ public class ServiceEngine implements Closeable{
         userWorker,
         context.getExecutor());
     this.controller = new ControllerImpl(context, controlMessageHandler, allowPortHunting);
-    this.dataPool = new DataConnectionCreator(context, workBus, dataHandler, allowPortHunting);
+    this.dataPool = new DataConnectionCreator(context, workBus, bee, allowPortHunting);
     this.config = context.getConfig();
     this.allowPortHunting = allowPortHunting;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index 12e00f2..ec3cae8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -38,8 +38,8 @@ import org.apache.drill.exec.physical.base.ScanStats;
 import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
 import org.apache.drill.exec.physical.impl.ScanBatch;
 import org.apache.drill.exec.physical.impl.WriterRecordBatch;
-import org.apache.drill.exec.record.CloseableRecordBatch;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.apache.drill.exec.record.CloseableRecordBatch;
 import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractRecordReader;
@@ -53,12 +53,11 @@ import org.apache.drill.exec.store.dfs.FormatMatcher;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
 import org.apache.drill.exec.store.schedule.CompleteFileWork;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
-import org.apache.hadoop.fs.Path;
-
 public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements FormatPlugin {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EasyFormatPlugin.class);
 
@@ -161,10 +160,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
     }
 
     int numParts = 0;
-    OperatorContext oContext = context.newOperatorContext(scan, false /*
-                                                                       * ScanBatch is not subject to fragment memory
-                                                                       * limit
-                                                                       */);
+    OperatorContext oContext = context.newOperatorContext(scan);
     final DrillFileSystem dfs;
     try {
       dfs = oContext.newFileSystem(fsConf);

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
index 710edd6..8f4f5fd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordWriter.java
@@ -109,7 +109,7 @@ public class ParquetRecordWriter extends ParquetOutputRecordWriter {
 
   public ParquetRecordWriter(FragmentContext context, ParquetWriter writer) throws OutOfMemoryException{
     super();
-    this.oContext = context.newOperatorContext(writer, true);
+    this.oContext = context.newOperatorContext(writer);
     this.codecFactory = CodecFactory.createDirectCodecFactory(writer.getFormatPlugin().getFsConf(),
         new ParquetDirectByteBufferAllocator(oContext.getAllocator()), pageSize);
     this.partitionColumns = writer.getPartitionColumns();

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
index 0cb12f8..afa4fa0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
@@ -43,7 +43,6 @@ import org.apache.drill.exec.store.parquet.columnreaders.ParquetRecordReader;
 import org.apache.drill.exec.store.parquet2.DrillParquetReader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.hadoop.CodecFactory;
 import org.apache.parquet.hadoop.ParquetFileReader;
@@ -70,10 +69,7 @@ public class ParquetScanBatchCreator implements BatchCreator<ParquetRowGroupScan
       .getOption(ExecConstants.FILESYSTEM_PARTITION_COLUMN_LABEL).string_val;
     List<SchemaPath> columns = rowGroupScan.getColumns();
     List<RecordReader> readers = Lists.newArrayList();
-    OperatorContext oContext = context.newOperatorContext(rowGroupScan, false /*
-                                                                               * ScanBatch is not subject to fragment
-                                                                               * memory limit
-                                                                               */);
+    OperatorContext oContext = context.newOperatorContext(rowGroupScan);
 
     List<String[]> partitionColumns = Lists.newArrayList();
     List<Integer> selectedPartitionColumns = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/MemoryIterator.java
----------------------------------------------------------------------
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 e02b413..e624ada 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.lang.management.MemoryUsage;
 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 class MemoryIterator implements Iterator<Object> {
 
 
     memoryInfo.direct_current = context.getDrillbitContext().getAllocator().getAllocatedMemory();
-    memoryInfo.direct_max = TopLevelAllocator.MAXIMUM_DIRECT_MEMORY;
+    memoryInfo.direct_max = DrillConfig.getMaxDirectMemory();
     return memoryInfo;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
index 93fd13e..5fd6f1c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -37,8 +37,6 @@ import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.control.Controller;
 import org.apache.drill.exec.rpc.control.WorkEventBus;
 import org.apache.drill.exec.rpc.data.DataConnectionCreator;
-import org.apache.drill.exec.rpc.data.DataResponseHandler;
-import org.apache.drill.exec.rpc.data.DataResponseHandlerImpl;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.sys.PStoreProvider;
@@ -75,7 +73,6 @@ public class WorkManager implements AutoCloseable {
   private DrillbitContext dContext;
 
   private final ControlMessageHandler controlMessageWorker;
-  private final DataResponseHandler dataHandler;
   private final UserWorker userWorker;
   private final WorkerBee bee;
   private final WorkEventBus workBus;
@@ -97,7 +94,6 @@ public class WorkManager implements AutoCloseable {
     controlMessageWorker = new ControlMessageHandler(bee); // TODO getFragmentRunner(), getForemanForQueryId()
     userWorker = new UserWorker(bee); // TODO should just be an interface? addNewForeman(), getForemanForQueryId()
     statusThread = new StatusThread();
-    dataHandler = new DataResponseHandlerImpl(bee); // TODO only uses startFragmentPendingRemote()
   }
 
   public void start(
@@ -132,10 +128,6 @@ public class WorkManager implements AutoCloseable {
     return workBus;
   }
 
-  public DataResponseHandler getDataHandler() {
-    return dataHandler;
-  }
-
   public ControlMessageHandler getControlMessageHandler() {
     return controlMessageWorker;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractDataCollector.java
----------------------------------------------------------------------
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 1579c2b..8a3e5b6 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
@@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicIntegerArray;
 
 import org.apache.drill.common.AutoCloseables;
-import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.BitControl.Collector;
 import org.apache.drill.exec.record.RawFragmentBatch;
@@ -93,26 +92,18 @@ public abstract class AbstractDataCollector implements DataCollector{
   @Override
   public boolean batchArrived(int minorFragmentId, RawFragmentBatch batch)  throws IOException {
 
-    // if we received an out of memory, add an item to all the buffer queues.
-    if (batch.getHeader().getIsOutOfMemory()) {
-      for (RawBatchBuffer buffer : buffers) {
-        buffer.enqueue(batch);
-      }
-    }
-
     // check to see if we have enough fragments reporting to proceed.
-    boolean decremented = false;
+    boolean decrementedToZero = false;
     if (remainders.compareAndSet(fragmentMap.get(minorFragmentId), 0, 1)) {
       int rem = remainingRequired.decrementAndGet();
       if (rem == 0) {
-        parentAccounter.decrementAndGet();
-        decremented = true;
+        decrementedToZero = 0 == parentAccounter.decrementAndGet();
       }
     }
 
     getBuffer(minorFragmentId).enqueue(batch);
 
-    return decremented;
+    return decrementedToZero;
   }
 
 

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java
index 31f2e4a..f15a3e6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BaseRawBatchBuffer.java
@@ -88,28 +88,10 @@ public abstract class BaseRawBatchBuffer<T> implements RawBatchBuffer {
         throw new IOException("Attempted to enqueue batch after finished");
       }
     }
-    if (batch.getHeader().getIsOutOfMemory()) {
-      handleOutOfMemory(batch);
-      return;
-    }
     enqueueInner(batch);
   }
 
   /**
-   * handle the out of memory case
-   *
-   * @param batch
-   */
-  protected void handleOutOfMemory(final RawFragmentBatch batch) {
-    if (!bufferQueue.checkForOutOfMemory()) {
-      logger.debug("Adding OOM message to front of queue. Current queue size: {}", bufferQueue.size());
-      bufferQueue.addOomBatch(batch);
-    } else {
-      logger.debug("ignoring duplicate OOM message");
-    }
-  }
-
-  /**
    * implementation specific method to enqueue batch
    *
    * @param batch
@@ -202,11 +184,11 @@ public abstract class BaseRawBatchBuffer<T> implements RawBatchBuffer {
       return null;
     }
 
+    if (context.isOverMemoryLimit()) {
+      outOfMemory.set(true);
+    }
+
     if (b != null) {
-      if (b.getHeader().getIsOutOfMemory()) {
-        outOfMemory.set(true);
-        return b;
-      }
 
       upkeep(b);
 
@@ -234,7 +216,7 @@ public abstract class BaseRawBatchBuffer<T> implements RawBatchBuffer {
   }
 
   private void assertAckSent(RawFragmentBatch batch) {
-    assert batch == null || batch.isAckSent() || batch.getHeader().getIsOutOfMemory() : "Ack not sent for batch";
+    assert batch == null || batch.isAckSent() : "Ack not sent for batch";
   }
 
   private int decrementStreamCounter() {

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
index b21c61d..a516fad 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
@@ -21,13 +21,17 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.concurrent.AutoCloseableLock;
 import org.apache.drill.exec.exception.FragmentSetupException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.BitControl.Collector;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.data.IncomingDataBatch;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
@@ -38,11 +42,22 @@ import com.google.common.collect.Maps;
 public class IncomingBuffers implements AutoCloseable {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IncomingBuffers.class);
 
+  private volatile boolean closed = false;
   private final AtomicInteger streamsRemaining = new AtomicInteger(0);
   private final AtomicInteger remainingRequired;
-  private final Map<Integer, DataCollector> fragCounts;
+  private final Map<Integer, DataCollector> collectorMap;
   private final FragmentContext context;
 
+  /**
+   * Lock used to manage close and data acceptance. We should only create a local reference to incoming data in the case
+   * that the incoming buffers are !closed. As such, we need to make sure that we aren't in the process of closing the
+   * incoming buffers when data is arriving. The read lock can be shared by many incoming batches but the write lock
+   * must be exclusive to the close method.
+   */
+  private final ReadWriteLock lock = new ReentrantReadWriteLock();
+  private final AutoCloseableLock sharedIncomingBatchLock = new AutoCloseableLock(lock.readLock());
+  private final AutoCloseableLock exclusiveCloseLock = new AutoCloseableLock(lock.writeLock());
+
   public IncomingBuffers(PlanFragment fragment, FragmentContext context) {
     this.context = context;
     Map<Integer, DataCollector> collectors = Maps.newHashMap();
@@ -56,39 +71,50 @@ public class IncomingBuffers implements AutoCloseable {
     }
 
     logger.debug("Came up with a list of {} required fragments.  Fragments {}", remainingRequired.get(), collectors);
-    fragCounts = ImmutableMap.copyOf(collectors);
+    collectorMap = ImmutableMap.copyOf(collectors);
 
     // Determine the total number of incoming streams that will need to be completed before we are finished.
     int totalStreams = 0;
-    for (DataCollector bc : fragCounts.values()) {
+    for (DataCollector bc : collectorMap.values()) {
       totalStreams += bc.getTotalIncomingFragments();
     }
     assert totalStreams >= remainingRequired.get() : String.format("Total Streams %d should be more than the minimum number of streams to commence (%d).  It isn't.", totalStreams, remainingRequired.get());
     streamsRemaining.set(totalStreams);
   }
 
-  public boolean batchArrived(RawFragmentBatch batch) throws FragmentSetupException, IOException {
-    // no need to do anything if we've already enabled running.
-    // logger.debug("New Batch Arrived {}", batch);
-    if (batch.getHeader().getIsOutOfMemory()) {
-      for (DataCollector fSet : fragCounts.values()) {
-        fSet.batchArrived(0, batch);
+  public boolean batchArrived(final IncomingDataBatch incomingBatch) throws FragmentSetupException, IOException {
+
+    // we want to make sure that we only generate local record batch reference in the case that we're not closed.
+    // Otherwise we would leak memory.
+    try (AutoCloseableLock lock = sharedIncomingBatchLock.open()) {
+      if (closed) {
+        return false;
       }
-      return false;
-    }
-    if (batch.getHeader().getIsLastBatch()) {
-      streamsRemaining.decrementAndGet();
-    }
-    int sendMajorFragmentId = batch.getHeader().getSendingMajorFragmentId();
-    DataCollector fSet = fragCounts.get(sendMajorFragmentId);
-    if (fSet == null) {
-      throw new FragmentSetupException(String.format("We received a major fragment id that we were not expecting.  The id was %d. %s", sendMajorFragmentId, Arrays.toString(fragCounts.values().toArray())));
-    }
-    synchronized (this) {
-      boolean decremented = fSet.batchArrived(batch.getHeader().getSendingMinorFragmentId(), batch);
-      // we should only return true if remaining required has been decremented and is currently equal to zero.
-      return decremented && remainingRequired.get() == 0;
+
+      if (incomingBatch.getHeader().getIsLastBatch()) {
+        streamsRemaining.decrementAndGet();
+      }
+
+      final int sendMajorFragmentId = incomingBatch.getHeader().getSendingMajorFragmentId();
+      DataCollector collector = collectorMap.get(sendMajorFragmentId);
+      if (collector == null) {
+        throw new FragmentSetupException(String.format(
+            "We received a major fragment id that we were not expecting.  The id was %d. %s", sendMajorFragmentId,
+            Arrays.toString(collectorMap.values().toArray())));
+      }
+
+      synchronized (collector) {
+        final RawFragmentBatch newRawFragmentBatch = incomingBatch.newRawFragmentBatch(context.getAllocator());
+        boolean decrementedToZero = collector
+            .batchArrived(incomingBatch.getHeader().getSendingMinorFragmentId(), newRawFragmentBatch);
+        newRawFragmentBatch.release();
+
+        // we should only return true if remaining required has been decremented and is currently equal to zero.
+        return decrementedToZero;
+      }
+
     }
+
   }
 
   public int getRemainingRequired() {
@@ -100,19 +126,19 @@ public class IncomingBuffers implements AutoCloseable {
   }
 
   public RawBatchBuffer[] getBuffers(int senderMajorFragmentId) {
-    return fragCounts.get(senderMajorFragmentId).getBuffers();
+    return collectorMap.get(senderMajorFragmentId).getBuffers();
   }
 
-
-
-
   public boolean isDone() {
     return streamsRemaining.get() < 1;
   }
 
   @Override
   public void close() throws Exception {
-    AutoCloseables.close(fragCounts.values().toArray(new AutoCloseable[0]));
+    try (AutoCloseableLock lock = exclusiveCloseLock.open()) {
+      closed = true;
+      AutoCloseables.close(collectorMap.values());
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/SpoolingRawBatchBuffer.java
----------------------------------------------------------------------
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 753c75d..9915b7e 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,11 +29,9 @@ import java.util.concurrent.ThreadLocalRandom;
 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;
-import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
 import org.apache.drill.exec.proto.ExecProtos;
 import org.apache.drill.exec.proto.helper.QueryIdHelper;
 import org.apache.drill.exec.record.RawFragmentBatch;
@@ -82,9 +80,10 @@ public class SpoolingRawBatchBuffer extends BaseRawBatchBuffer<SpoolingRawBatchB
   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);
     this.threshold = context.getConfig().getLong(ExecConstants.SPOOLING_BUFFER_MEMORY);
     this.oppositeId = oppositeId;
     this.bufferIndex = bufferIndex;
@@ -224,11 +223,10 @@ public class SpoolingRawBatchBuffer extends BaseRawBatchBuffer<SpoolingRawBatchB
 
   @Override
   protected void upkeep(RawFragmentBatch batch) {
-    FragmentRecordBatch header = batch.getHeader();
-    if (header.getIsOutOfMemory()) {
+    if (context.isOverMemoryLimit()) {
       outOfMemory.set(true);
-      return;
     }
+
     DrillBuf body = batch.getBody();
     if (body != null) {
       currentSizeInMemory -= body.capacity();

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
index ef06ea8..b177770 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlimitedRawBatchBuffer.java
@@ -65,7 +65,7 @@ public class UnlimitedRawBatchBuffer extends BaseRawBatchBuffer<RawFragmentBatch
 
     @Override
     public boolean checkForOutOfMemory() {
-      return buffer.peekFirst().getHeader().getIsOutOfMemory();
+      return context.isOverMemoryLimit();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
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 71f9307..7b015a5 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;
 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 class Foreman implements Runnable {
     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 @@ public class Foreman implements Runnable {
     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 @@ public class Foreman implements Runnable {
       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);

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java
index ff348cb..20315e1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/FragmentManager.java
@@ -22,8 +22,8 @@ import java.io.IOException;
 import org.apache.drill.exec.exception.FragmentSetupException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.record.RawFragmentBatch;
 import org.apache.drill.exec.rpc.RemoteConnection;
+import org.apache.drill.exec.rpc.data.IncomingDataBatch;
 
 /**
  * The Fragment Manager is responsible managing incoming data and executing a fragment. Once enough data and resources
@@ -37,7 +37,7 @@ public interface FragmentManager {
    * @return True if the fragment has enough incoming data to be able to be run.
    * @throws FragmentSetupException, IOException
    */
-  boolean handle(RawFragmentBatch batch) throws FragmentSetupException, IOException;
+  boolean handle(IncomingDataBatch batch) throws FragmentSetupException, IOException;
 
   /**
    * Get the fragment runner for this incoming fragment. Note, this can only be requested once.

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
index 9378e51..b9cf8e8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/NonRootFragmentManager.java
@@ -26,8 +26,8 @@ import org.apache.drill.exec.exception.FragmentSetupException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.record.RawFragmentBatch;
 import org.apache.drill.exec.rpc.RemoteConnection;
+import org.apache.drill.exec.rpc.data.IncomingDataBatch;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.work.batch.IncomingBuffers;
 import org.apache.drill.exec.work.foreman.ForemanException;
@@ -69,7 +69,7 @@ public class NonRootFragmentManager implements FragmentManager {
    * @see org.apache.drill.exec.work.fragment.FragmentHandler#handle(org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle, org.apache.drill.exec.record.RawFragmentBatch)
    */
   @Override
-  public boolean handle(final RawFragmentBatch batch) throws FragmentSetupException, IOException {
+  public boolean handle(final IncomingDataBatch batch) throws FragmentSetupException, IOException {
     return buffers.batchArrived(batch);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java
index 0713398..0f7b10e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RootFragmentManager.java
@@ -24,8 +24,8 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import org.apache.drill.exec.exception.FragmentSetupException;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.record.RawFragmentBatch;
 import org.apache.drill.exec.rpc.RemoteConnection;
+import org.apache.drill.exec.rpc.data.IncomingDataBatch;
 import org.apache.drill.exec.work.batch.IncomingBuffers;
 
 // TODO a lot of this is the same as NonRootFragmentManager
@@ -46,7 +46,7 @@ public class RootFragmentManager implements FragmentManager {
   }
 
   @Override
-  public boolean handle(final RawFragmentBatch batch) throws FragmentSetupException, IOException {
+  public boolean handle(final IncomingDataBatch batch) throws FragmentSetupException, IOException {
     return buffers.batchArrived(batch);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
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 46f0526..2077d6e 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
@@ -31,9 +31,6 @@ import java.util.List;
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import com.google.common.io.Files;
-
-import org.apache.drill.common.DrillAutoCloseables;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.scanner.ClassPathScanner;
@@ -43,9 +40,9 @@ import org.apache.drill.exec.ExecTest;
 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.memory.RootAllocator;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
@@ -72,11 +69,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.io.Files;
 import com.google.common.io.Resources;
 
-import static org.hamcrest.core.StringContains.containsString;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
-
 public class BaseTestQuery extends ExecTest {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseTestQuery.class);
 
@@ -268,9 +260,6 @@ public class BaseTestQuery extends ExecTest {
         if (bit != null) {
           final DrillbitContext drillbitContext = bit.getContext();
           final BufferAllocator bufferAllocator = drillbitContext.getAllocator();
-          if (!(bufferAllocator instanceof RootAllocator)) {
-            throw new IllegalStateException("The DrillbitContext's allocator is not a RootAllocator");
-          }
           final RootAllocator rootAllocator = (RootAllocator) bufferAllocator;
           rootAllocator.verify();
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/TestAllocationException.java
----------------------------------------------------------------------
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 19613fe..0000000
--- 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");
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestAllocators.java
----------------------------------------------------------------------
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 2389dc9..322e54a 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,30 +18,19 @@
 
 package org.apache.drill.exec.memory;
 
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import com.google.common.collect.Lists;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-
+import static org.junit.Assert.fail;
 import io.netty.buffer.DrillBuf;
 
 import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 
-import io.netty.buffer.UnsafeDirectLittleEndian;
-
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.exception.OutOfMemoryException;
-import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OpProfileDef;
@@ -52,7 +41,6 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
 import org.apache.drill.exec.proto.BitControl;
-import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.server.Drillbit;
@@ -65,15 +53,16 @@ import org.apache.drill.test.DrillTest;
 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");
     }
   };
 
@@ -95,8 +84,7 @@ public class TestAllocators extends DrillTest {
 
     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);
@@ -140,8 +128,7 @@ public class TestAllocators extends DrillTest {
   public void testClearBitVector() {
     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);
@@ -163,147 +150,145 @@ public class TestAllocators extends DrillTest {
   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
-    boolean outOfMem = false;
-    try {
-      oContext31.getAllocator().buffer(44000000);
-      fail("Fragment 3 should fail to allocate buffer");
-    } catch (OutOfMemoryRuntimeException 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);
-
-    fragmentContext1.close();
-    fragmentContext2.close();
-    fragmentContext3.close();
-
-    bit.close();
-    serviceSet.close();
-
-/*
-    // ---------------------------------------- DEBUG ----------------------------------
-    assertEquals(0, UnsafeDirectLittleEndian.getBufferCount());
-    // ---------------------------------------- DEBUG ----------------------------------
-*/
+    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);
+      DrillBuf b11 = oContext11.getAllocator().buffer(1000000);
+
+      OperatorContext oContext12 = fragmentContext1.newOperatorContext(physicalOperator2, stats);
+      DrillBuf b12 = oContext12.getAllocator().buffer(500000);
+
+      OperatorContext oContext21 = fragmentContext1.newOperatorContext(physicalOperator3);
+
+      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);
+      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);
+
+      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);
+      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);
+
+      fragmentContext1.close();
+      fragmentContext2.close();
+      fragmentContext3.close();
+
+    }
   }
 
   private void closeOp(OperatorContext c) throws Exception {


[02/13] drill git commit: DRILL-4134: Add new allocator

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/53dcabeb/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/TopLevelAllocator.java
----------------------------------------------------------------------
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
index 1e96238..36bcacf 100644
--- 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
@@ -26,66 +26,71 @@ import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.concurrent.atomic.AtomicInteger;
 
 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.ExecConstants;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ControlsInjectorFactory;
 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);
+  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(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 PooledByteBufAllocatorL innerAllocator = PooledByteBufAllocatorL.DEFAULT;
+  private final Accountor acct;
   private final boolean errorOnLeak;
   private final DrillBuf empty;
-  private final DrillConfig config;
+
+  private final AtomicInteger idGenerator = new AtomicInteger(0);
 
   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.acct = new Accountor(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)
+    this(config, Math.min(DrillConfig.getMaxDirectMemory(), config.getLong(ExecConstants.TOP_LEVEL_MAX_ALLOC)),
+        config.getBoolean(ExecConstants.ERROR_ON_MEMORY_LEAK)
         );
   }
 
   @Override
+  public int getId() {
+    return idGenerator.incrementAndGet();
+  }
+
+  @Override
   public boolean takeOwnership(DrillBuf buf) {
     return buf.transferAccounting(acct);
   }
 
   @Override
-  public boolean takeOwnership(DrillBuf buf, Pointer<DrillBuf> out) {
+  public boolean shareOwnership(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 min, int max) {
     if (min == 0) {
       return empty;
     }
     if(!acct.reserve(min)) {
-      throw new OutOfMemoryException(createErrorMsg(this, min));
+      throw new OutOfMemoryRuntimeException(createErrorMsg(this, min));
     }
 
     try {
@@ -96,7 +101,7 @@ public class TopLevelAllocator implements BufferAllocator {
     } catch (OutOfMemoryError e) {
       if ("Direct buffer memory".equals(e.getMessage())) {
         acct.release(min);
-        throw new OutOfMemoryException(createErrorMsg(this, min), e);
+        throw new OutOfMemoryRuntimeException(createErrorMsg(this, min), e);
       } else {
         throw e;
       }
@@ -124,20 +129,25 @@ public class TopLevelAllocator implements BufferAllocator {
   }
 
   @Override
-  public BufferAllocator getChildAllocator(LimitConsumer limitConsumer, long initialReservation,
-      long maximumReservation,
-      boolean applyFragmentLimit) {
+  public BufferAllocator newChildAllocator(AllocatorOwner allocatorOwner,
+      long initialReservation, long maximumReservation, int flags) {
+    return getChildAllocator(allocatorOwner.getFragmentContext(), initialReservation,
+        maximumReservation, (flags & BufferAllocator.F_LIMITING_ROOT) != 0);
+  }
+
+    @Override
+  public BufferAllocator getChildAllocator(FragmentContext context, 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(
+      throw new OutOfMemoryRuntimeException(
           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);
+    ChildAllocator allocator = new ChildAllocator(context, acct, maximumReservation, initialReservation, childrenMap, applyFragmentLimit);
     if(ENABLE_ACCOUNTING){
       childrenMap.put(allocator, Thread.currentThread().getStackTrace());
     }
@@ -146,17 +156,12 @@ public class TopLevelAllocator implements BufferAllocator {
   }
 
   @Override
-  public void resetLimits() {
-    acct.resetFragmentLimits();
-  }
-
-  @Override
-  public void setLimit(long limit){
+  public void setFragmentLimit(long limit){
     acct.setFragmentLimit(limit);
   }
 
   @Override
-  public long getLimit(){
+  public long getFragmentLimit(){
     return acct.getFragmentLimit();
   }
 
@@ -190,26 +195,30 @@ public class TopLevelAllocator implements BufferAllocator {
 
   private class ChildAllocator implements BufferAllocator {
     private final DrillBuf empty;
-    private AccountorImpl childAcct;
+    private Accountor childAcct;
     private Map<ChildAllocator, StackTraceElement[]> children = new HashMap<>();
     private boolean closed = false;
-    private LimitConsumer limitConsumer;
+    private FragmentContext fragmentContext;
     private Map<ChildAllocator, StackTraceElement[]> thisMap;
-    private boolean applyFragmentLimit;
 
-    public ChildAllocator(LimitConsumer limitConsumer,
-        AccountorImpl parentAccountor,
+    public ChildAllocator(FragmentContext context,
+                          Accountor 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);
+      DrillConfig drillConf = context != null ? context.getConfig() : null;
+      childAcct = new Accountor(drillConf, errorOnLeak, context, parentAccountor, max, pre, applyFragmentLimit);
+      this.fragmentContext=context;
       thisMap = map;
-      empty = DrillBuf.getEmpty(this, childAcct);
+      this.empty = DrillBuf.getEmpty(this, childAcct);
+    }
+
+    @Override
+    public int getId() {
+      return idGenerator.incrementAndGet();
     }
 
     @Override
@@ -218,7 +227,7 @@ public class TopLevelAllocator implements BufferAllocator {
     }
 
     @Override
-    public boolean takeOwnership(DrillBuf buf, Pointer<DrillBuf> out) {
+    public boolean shareOwnership(DrillBuf buf, Pointer<DrillBuf> out) {
       DrillBuf b = new DrillBuf(this, acct, buf);
       out.value = b;
       return acct.transferIn(b, b.capacity());
@@ -227,11 +236,15 @@ public class TopLevelAllocator implements BufferAllocator {
 
     @Override
     public DrillBuf buffer(int size, int max) {
+      if (ENABLE_ACCOUNTING) {
+        injector.injectUnchecked(fragmentContext, CHILD_BUFFER_INJECTION_SITE);
+      }
+
       if (size == 0) {
         return empty;
       }
       if(!childAcct.reserve(size)) {
-        throw new OutOfMemoryException(createErrorMsg(this, size));
+        throw new OutOfMemoryRuntimeException(createErrorMsg(this, size));
       }
 
       try {
@@ -242,13 +255,14 @@ public class TopLevelAllocator implements BufferAllocator {
       } catch (OutOfMemoryError e) {
         if ("Direct buffer memory".equals(e.getMessage())) {
           childAcct.release(size);
-          throw new OutOfMemoryException(createErrorMsg(this, size), e);
+          throw new OutOfMemoryRuntimeException(createErrorMsg(this, size), e);
         } else {
           throw e;
         }
       }
     }
 
+    @Override
     public DrillBuf buffer(int size) {
       return buffer(size, size);
     }
@@ -259,39 +273,40 @@ public class TopLevelAllocator implements BufferAllocator {
     }
 
     @Override
-    public BufferAllocator getChildAllocator(LimitConsumer limitConsumer, long initialReservation,
-        long maximumReservation,
-        boolean applyFragmentLimit) {
+    public BufferAllocator newChildAllocator(AllocatorOwner allocatorOwner,
+        long initialReservation, long maximumReservation, int flags) {
+      return getChildAllocator(allocatorOwner.getFragmentContext(), initialReservation,
+          maximumReservation, (flags & BufferAllocator.F_LIMITING_ROOT) != 0);
+    }
+
+    @Override
+    public BufferAllocator getChildAllocator(FragmentContext context,
+        long initialReservation, long maximumReservation, boolean applyFragmentLimit) {
       if (!childAcct.reserve(initialReservation)) {
-        throw new OutOfMemoryException(
+        throw new OutOfMemoryRuntimeException(
             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);
+      ChildAllocator newChildAllocator = new ChildAllocator(context, 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();
+    public AllocationReservation newReservation() {
+      return new PreAlloc(this, this.childAcct);
     }
 
     @Override
-    public void setLimit(long limit){
+    public void setFragmentLimit(long limit){
       childAcct.setFragmentLimit(limit);
     }
 
     @Override
-    public long getLimit(){
+    public long getFragmentLimit(){
       return childAcct.getFragmentLimit();
     }
 
@@ -312,9 +327,10 @@ public class TopLevelAllocator implements BufferAllocator {
             }
 
 
+            final FragmentHandle handle = fragmentContext.getHandle();
             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()));
+                    "Failure while trying to close child allocator: Child level allocators not closed. Fragment %d:%d. Stack trace: \n %s",
+                    handle.getMajorFragmentId(), handle.getMinorFragmentId(), sb.toString()));
             if (errorOnLeak) {
               throw e;
             } else {
@@ -345,15 +361,14 @@ public class TopLevelAllocator implements BufferAllocator {
     public DrillBuf getEmpty() {
       return empty;
     }
-
-
   }
 
-  public PreAllocator getNewPreAllocator() {
+  @Override
+  public AllocationReservation newReservation() {
     return new PreAlloc(this, this.acct);
   }
 
-  public class PreAlloc implements PreAllocator{
+  public class PreAlloc extends AllocationReservation {
     int bytes = 0;
     final Accountor acct;
     final BufferAllocator allocator;
@@ -362,25 +377,28 @@ public class TopLevelAllocator implements BufferAllocator {
       this.allocator = allocator;
     }
 
-    /**
-     *
-     */
-    public boolean preAllocate(int bytes) {
-
+    @Override
+    protected boolean reserve(int bytes) {
       if (!acct.reserve(bytes)) {
         return false;
       }
+
       this.bytes += bytes;
       return true;
-
     }
 
-
-    public DrillBuf getAllocation() {
+    @Override
+    protected DrillBuf allocate(int bytes) {
+      assert this.bytes == bytes : "allocation size mismatch";
       DrillBuf b = new DrillBuf(allocator, acct, innerAllocator.directBuffer(bytes, bytes));
       acct.reserved(bytes, b);
       return b;
     }
+
+    @Override
+    protected void releaseReservation(int nBytes) {
+      acct.release(nBytes);
+    }
   }
 
   private static String createErrorMsg(final BufferAllocator allocator, final int size) {


[11/13] drill git commit: DRILL-4134: Allocator Improvements

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java
deleted file mode 100644
index 6ea5670..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java
+++ /dev/null
@@ -1,651 +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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import io.netty.buffer.DrillBuf;
-
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.server.options.OptionManager;
-import org.apache.drill.exec.testing.ExecutionControls;
-import org.apache.drill.exec.util.Pointer;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-
-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 ------------------------------------
-*/
-
-  // Concoct ExecutionControls that won't try to inject anything.
-  @Mock private static final OptionManager optionManager = Mockito.mock(OptionManager.class);
-  static {
-    Mockito.when(optionManager.getOption(Matchers.anyString()))
-      .thenReturn(null);
-  }
-
-  @Mock private static final ExecutionControls executionControls = new ExecutionControls(optionManager, null);
-
-  private final static class NamedOwner implements AllocatorOwner {
-    private final String name;
-
-    public NamedOwner(final String name) {
-      this.name = name;
-    }
-
-    @Override
-    public String toString() {
-      return name;
-    }
-
-    @Override
-    public ExecutionControls getExecutionControls() {
-      return executionControls;
-    }
-
-    @Override
-    public FragmentContext getFragmentContext() {
-      return null;
-    }
-  }
-
-  @Test
-  public void test_privateMax() throws Exception {
-    final AllocatorOwner allocatorOwner = new NamedOwner("noLimits");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_LOCAL_MAX, 0, MAX_ALLOCATION, 0)) {
-      final DrillBuf drillBuf1 = rootAllocator.buffer(MAX_ALLOCATION / 2);
-      assertNotNull("allocation failed", drillBuf1);
-
-      try(final BufferAllocator childAllocator =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
-        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(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-        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(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      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(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      @SuppressWarnings("unused")
-      final DrillBuf drillBuf = rootAllocator.buffer(0);
-    }
-  }
-
-  @Test
-  public void testAllocator_transferOwnership() throws Exception {
-    final AllocatorOwner allocatorOwner = new NamedOwner("changeOwnership");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      final BufferAllocator childAllocator1 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-      final BufferAllocator childAllocator2 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-
-      final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 4);
-      rootAllocator.verify();
-      final boolean allocationFit = childAllocator2.takeOwnership(drillBuf1);
-      rootAllocator.verify();
-      assertTrue(allocationFit);
-
-      childAllocator1.close();
-      rootAllocator.verify();
-
-      drillBuf1.release();
-      childAllocator2.close();
-    }
-  }
-
-  @Test
-  public void testAllocator_shareOwnership() throws Exception {
-    final AllocatorOwner allocatorOwner = new NamedOwner("shareOwnership");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      final BufferAllocator childAllocator1 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-      final BufferAllocator childAllocator2 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-      final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 4);
-      rootAllocator.verify();
-      final Pointer<DrillBuf> pDrillBuf = new Pointer<>();
-      boolean allocationFit;
-
-      allocationFit = childAllocator2.shareOwnership(drillBuf1, pDrillBuf);
-      assertTrue(allocationFit);
-      rootAllocator.verify();
-      final DrillBuf drillBuf2 = pDrillBuf.value;
-      assertNotNull(drillBuf2);
-      assertNotEquals(drillBuf2, drillBuf1);
-
-      drillBuf1.release();
-      rootAllocator.verify();
-      childAllocator1.close();
-      rootAllocator.verify();
-
-      final BufferAllocator childAllocator3 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-      allocationFit = childAllocator3.shareOwnership(drillBuf2, pDrillBuf);
-      assertTrue(allocationFit);
-      final DrillBuf drillBuf3 = pDrillBuf.value;
-      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 {
-    final AllocatorOwner allocatorOwner = new NamedOwner("createChildAndUse");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      try(final BufferAllocator childAllocator =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
-        final DrillBuf drillBuf = childAllocator.buffer(512);
-        assertNotNull("allocation failed", drillBuf);
-        drillBuf.release();
-      }
-    }
-  }
-
-  @Test(expected=IllegalStateException.class)
-  public void testRootAllocator_createChildDontClose() throws Exception {
-    try {
-      final AllocatorOwner allocatorOwner = new NamedOwner("createChildDontClose");
-      try(final RootAllocator rootAllocator =
-          new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-        final BufferAllocator childAllocator =
-            rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-        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 {
-    final AllocatorOwner allocatorOwner = new NamedOwner("manyAllocations");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      try(final BufferAllocator childAllocator =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
-        allocateAndFree(childAllocator);
-      }
-    }
-  }
-
-  @Test
-  public void testAllocator_overAllocate() throws Exception {
-    final AllocatorOwner allocatorOwner = new NamedOwner("overAllocate");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      try(final BufferAllocator childAllocator =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
-        allocateAndFree(childAllocator);
-
-        try {
-          childAllocator.buffer(MAX_ALLOCATION + 1);
-          fail("allocated memory beyond max allowed");
-        } catch(OutOfMemoryRuntimeException e) {
-          // expected
-        }
-      }
-    }
-  }
-
-  @Test
-  public void testAllocator_overAllocateParent() throws Exception {
-    final AllocatorOwner allocatorOwner = new NamedOwner("overAllocateParent");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      try(final BufferAllocator childAllocator =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
-        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(OutOfMemoryRuntimeException 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 {
-    final AllocatorOwner allocatorOwner = new NamedOwner("createSlices");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      testAllocator_sliceUpBufferAndRelease(rootAllocator, rootAllocator);
-
-      try(final BufferAllocator childAllocator =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
-        testAllocator_sliceUpBufferAndRelease(rootAllocator, childAllocator);
-      }
-      rootAllocator.verify();
-
-      testAllocator_sliceUpBufferAndRelease(rootAllocator, rootAllocator);
-
-      try(final BufferAllocator childAllocator =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
-        try(final BufferAllocator childAllocator2 =
-            childAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
-          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(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      // Populate a buffer with byte values corresponding to their indices.
-      final DrillBuf drillBuf = rootAllocator.buffer(256, 256 + 256);
-      assertEquals(256, drillBuf.capacity());
-      assertEquals(256 + 256, drillBuf.maxCapacity());
-      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(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      // Populate a buffer with byte values corresponding to their indices.
-      final DrillBuf drillBuf = rootAllocator.buffer(256, 256 + 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 {
-    final AllocatorOwner allocatorOwner = new NamedOwner("transferSliced");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      final BufferAllocator childAllocator1 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-      final BufferAllocator childAllocator2 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-
-      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();
-
-      childAllocator1.takeOwnership(drillBuf2s);
-      rootAllocator.verify();
-      childAllocator2.takeOwnership(drillBuf1s);
-      rootAllocator.verify();
-
-      drillBuf1s.release(); // releases drillBuf1
-      drillBuf2s.release(); // releases drillBuf2
-
-      childAllocator1.close();
-      childAllocator2.close();
-    }
-  }
-
-  @Test
-  public void testAllocator_shareSliced() throws Exception {
-    final AllocatorOwner allocatorOwner = new NamedOwner("transferSliced");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      final BufferAllocator childAllocator1 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-      final BufferAllocator childAllocator2 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-
-      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 Pointer<DrillBuf> pDrillBuf = new Pointer<>();
-      childAllocator1.shareOwnership(drillBuf2s, pDrillBuf);
-      final DrillBuf drillBuf2s1 = pDrillBuf.value;
-      childAllocator2.shareOwnership(drillBuf1s, pDrillBuf);
-      final DrillBuf drillBuf1s2 = pDrillBuf.value;
-      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 {
-    final AllocatorOwner allocatorOwner = new NamedOwner("transferShared");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      final BufferAllocator childAllocator1 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-      final BufferAllocator childAllocator2 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-      final BufferAllocator childAllocator3 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-
-      final DrillBuf drillBuf1 = childAllocator1.buffer(MAX_ALLOCATION / 8);
-
-      final Pointer<DrillBuf> pDrillBuf = new Pointer<>();
-      boolean allocationFit;
-
-      allocationFit = childAllocator2.shareOwnership(drillBuf1, pDrillBuf);
-      assertTrue(allocationFit);
-      rootAllocator.verify();
-      final DrillBuf drillBuf2 = pDrillBuf.value;
-      assertNotNull(drillBuf2);
-      assertNotEquals(drillBuf2, drillBuf1);
-
-      allocationFit = childAllocator3.takeOwnership(drillBuf1);
-      assertTrue(allocationFit);
-      rootAllocator.verify();
-
-      // Since childAllocator3 now has childAllocator1's buffer, 1, can close
-      childAllocator1.close();
-      rootAllocator.verify();
-
-      drillBuf2.release();
-      childAllocator2.close();
-      rootAllocator.verify();
-
-      final BufferAllocator childAllocator4 =
-          rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0);
-      allocationFit = childAllocator4.takeOwnership(drillBuf1);
-      assertTrue(allocationFit);
-      rootAllocator.verify();
-
-      childAllocator3.close();
-      rootAllocator.verify();
-
-      drillBuf1.release();
-      childAllocator4.close();
-      rootAllocator.verify();
-    }
-  }
-
-  @Test
-  public void testAllocator_unclaimedReservation() throws Exception {
-    final AllocatorOwner allocatorOwner = new NamedOwner("unclaimedReservation");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      try(final BufferAllocator childAllocator1 =
-            rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
-        try(final AllocationReservation reservation = childAllocator1.newReservation()) {
-          assertTrue(reservation.add(64));
-        }
-        rootAllocator.verify();
-      }
-    }
-  }
-
-  @Test
-  public void testAllocator_claimedReservation() throws Exception {
-    final AllocatorOwner allocatorOwner = new NamedOwner("claimedReservation");
-    try(final RootAllocator rootAllocator =
-        new RootAllocator(RootAllocator.POLICY_PER_FRAGMENT, 0, MAX_ALLOCATION, 0)) {
-      try(final BufferAllocator childAllocator1 =
-            rootAllocator.newChildAllocator(allocatorOwner, 0, MAX_ALLOCATION, 0)) {
-        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();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
index 94aa84e..7207bf2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
@@ -40,6 +40,7 @@ import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.planner.PhysicalPlanReaderTestFactory;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.control.Controller;
@@ -112,7 +113,7 @@ public class TestOptiqPlans extends ExecTest {
         workBus,
         new LocalPStoreProvider(config));
     final QueryContext qc = new QueryContext(UserSession.Builder.newBuilder().setSupportComplexTypes(true).build(),
-        bitContext);
+        bitContext, QueryId.getDefaultInstance());
     final PhysicalPlanReader reader = bitContext.getPlanReader();
     final LogicalPlan plan = reader.readLogicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
     final PhysicalPlan pp = new BasicOptimizer(qc, connection).optimize(new BasicOptimizer.BasicOptimizationContext(qc), plan);

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestValueVector.java
----------------------------------------------------------------------
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 0a9b470..96f2b33 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 io.netty.buffer.DrillBuf;
 
 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.RepeatedVarBinaryHolder;
 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.After;
 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 class TestValueVector extends ExecTest {
     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 @@ the interface to load has changed
       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);

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
----------------------------------------------------------------------
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 107f978..73ed65e 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,18 @@
 package org.apache.drill.exec.server;
 
 import static org.junit.Assert.assertTrue;
+import io.netty.buffer.ByteBuf;
 
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
+import mockit.Injectable;
+import mockit.Mock;
+import mockit.MockUp;
+import mockit.NonStrictExpectations;
+
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.SchemaPath;
@@ -35,22 +41,21 @@ import org.apache.drill.exec.exception.FragmentSetupException;
 import org.apache.drill.exec.expr.TypeHelper;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.proto.BitData.FragmentRecordBatch;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.record.FragmentWritableBatch;
 import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RawFragmentBatch;
 import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
 import org.apache.drill.exec.rpc.control.WorkEventBus;
-import org.apache.drill.exec.rpc.data.AckSender;
 import org.apache.drill.exec.rpc.data.DataConnectionManager;
-import org.apache.drill.exec.rpc.data.DataResponseHandler;
 import org.apache.drill.exec.rpc.data.DataServer;
 import org.apache.drill.exec.rpc.data.DataTunnel;
+import org.apache.drill.exec.rpc.data.IncomingDataBatch;
 import org.apache.drill.exec.vector.Float8Vector;
 import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.work.WorkManager.WorkerBee;
@@ -60,33 +65,59 @@ import org.junit.Test;
 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);
 
   @Test
-  public void testConnectionBackpressure(@Injectable WorkerBee bee, @Injectable final WorkEventBus workBus, @Injectable final FragmentManager fman, @Injectable final FragmentContext fcon) throws Exception {
+  public void testConnectionBackpressure(@Injectable WorkerBee bee, @Injectable final WorkEventBus workBus) throws Exception {
 
     DrillConfig config1 = DrillConfig.create();
     final BootStrapContext c = new BootStrapContext(config1, ClassPathScanner.fromPrescan(config1));
     DrillConfig config2 = DrillConfig.create();
     BootStrapContext c2 = new BootStrapContext(config2, ClassPathScanner.fromPrescan(config2));
 
+    final FragmentContext fcon = new MockUp<FragmentContext>(){
+      BufferAllocator getAllocator(){
+        return c.getAllocator();
+      }
+    }.getMockInstance();
+
+    final FragmentManager fman = new MockUp<FragmentManager>(){
+      int v = 0;
+
+      @Mock
+      boolean handle(IncomingDataBatch batch) throws FragmentSetupException, IOException {
+        try {
+          v++;
+          if (v % 10 == 0) {
+            System.out.println("sleeping.");
+            Thread.sleep(3000);
+          }
+        } catch (InterruptedException e) {
+
+        }
+        RawFragmentBatch rfb = batch.newRawFragmentBatch(c.getAllocator());
+        rfb.sendOk();
+        rfb.release();
+
+        return true;
+      }
+
+      public FragmentContext getFragmentContext(){
+        return fcon;
+      }
+
+    }.getMockInstance();
+
+
     new NonStrictExpectations() {{
       workBus.getFragmentManagerIfExists((FragmentHandle) any); result = fman;
       workBus.getFragmentManager( (FragmentHandle) any); result = fman;
-      fman.getFragmentContext(); result = fcon;
-      fcon.getAllocator(); result = c.getAllocator();
     }};
 
     int port = 1234;
 
-    DataResponseHandler drp = new BitComTestHandler();
-    DataServer server = new DataServer(c, workBus, drp);
+    DataServer server = new DataServer(c, c.getAllocator(), workBus, null);
 
     port = server.bind(port, true);
     DrillbitEndpoint ep = DrillbitEndpoint.newBuilder().setAddress("localhost").setDataPort(port).build();
@@ -154,31 +185,4 @@ public class TestBitRpc extends ExecTest {
     }
   }
 
-  private class BitComTestHandler implements DataResponseHandler {
-
-    int v = 0;
-
-    @Override
-    public void informOutOfMemory() {
-    }
-
-    @Override
-    public void handle(FragmentManager manager, FragmentRecordBatch fragmentBatch, DrillBuf data, AckSender sender)
-        throws FragmentSetupException, IOException {
-      // System.out.println("Received.");
-      try {
-        v++;
-        if (v % 10 == 0) {
-          System.out.println("sleeping.");
-          Thread.sleep(3000);
-        }
-      } catch (InterruptedException e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
-      }
-      sender.sendOk();
-    }
-
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java
index c911f79..44cc3a7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestCountDownLatchInjection.java
@@ -17,20 +17,21 @@
  */
 package org.apache.drill.exec.testing;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.concurrent.CountDownLatch;
+
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.concurrent.ExtendedLatch;
 import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
 import org.apache.drill.exec.proto.UserProtos.UserProperties;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.util.Pointer;
 import org.junit.Test;
 
-import java.util.concurrent.CountDownLatch;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 public class TestCountDownLatchInjection extends BaseTestQuery {
 
   private static final UserSession session = UserSession.Builder.newBuilder()
@@ -132,7 +133,7 @@ public class TestCountDownLatchInjection extends BaseTestQuery {
 
     ControlsInjectionUtil.setControls(session, controls);
 
-    final QueryContext queryContext = new QueryContext(session, bits[0].getContext());
+    final QueryContext queryContext = new QueryContext(session, bits[0].getContext(), QueryId.getDefaultInstance());
 
     final DummyClass dummyClass = new DummyClass(queryContext, trigger, threads);
     (new ThreadCreator(dummyClass, trigger, threads, countingDownTime)).start();

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
index 40620c2..84a7320 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestExceptionInjection.java
@@ -17,12 +17,18 @@
  */
 package org.apache.drill.exec.testing;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.ZookeeperHelper;
 import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserProtos.UserProperties;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.Drillbit;
@@ -30,11 +36,6 @@ import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.junit.Test;
 
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
 public class TestExceptionInjection extends BaseTestQuery {
   private static final String NO_THROW_FAIL = "Didn't throw expected exception";
 
@@ -135,7 +136,7 @@ public class TestExceptionInjection extends BaseTestQuery {
       + "}]}";
     ControlsInjectionUtil.setControls(session, jsonString);
 
-    final QueryContext context = new QueryContext(session, bits[0].getContext());
+    final QueryContext context = new QueryContext(session, bits[0].getContext(), QueryId.getDefaultInstance());
 
     // test that the exception gets thrown
     final DummyClass dummyClass = new DummyClass(context);
@@ -156,7 +157,7 @@ public class TestExceptionInjection extends BaseTestQuery {
       .build();
     ControlsInjectionUtil.setControls(session, controls);
 
-    final QueryContext context = new QueryContext(session, bits[0].getContext());
+    final QueryContext context = new QueryContext(session, bits[0].getContext(), QueryId.getDefaultInstance());
 
     // test that the expected exception (checked) gets thrown
     final DummyClass dummyClass = new DummyClass(context);
@@ -185,7 +186,7 @@ public class TestExceptionInjection extends BaseTestQuery {
       .build();
     ControlsInjectionUtil.setControls(session, controls);
 
-    final QueryContext context = new QueryContext(session, bits[0].getContext());
+    final QueryContext context = new QueryContext(session, bits[0].getContext(), QueryId.getDefaultInstance());
 
     final DummyClass dummyClass = new DummyClass(context);
 
@@ -246,7 +247,7 @@ public class TestExceptionInjection extends BaseTestQuery {
     ControlsInjectionUtil.setControls(session, controls);
 
     {
-      final QueryContext queryContext1 = new QueryContext(session, drillbitContext1);
+      final QueryContext queryContext1 = new QueryContext(session, drillbitContext1, QueryId.getDefaultInstance());
       final DummyClass class1 = new DummyClass(queryContext1);
 
       // these shouldn't throw
@@ -268,7 +269,7 @@ public class TestExceptionInjection extends BaseTestQuery {
       }
     }
     {
-      final QueryContext queryContext2 = new QueryContext(session, drillbitContext2);
+      final QueryContext queryContext2 = new QueryContext(session, drillbitContext2, QueryId.getDefaultInstance());
       final DummyClass class2 = new DummyClass(queryContext2);
 
       // these shouldn't throw

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
index f07f676..54f851a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestPauseInjection.java
@@ -17,6 +17,11 @@
  */
 package org.apache.drill.exec.testing;
 
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.concurrent.CountDownLatch;
+
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.concurrent.ExtendedLatch;
 import org.apache.drill.common.config.DrillConfig;
@@ -24,6 +29,7 @@ import org.apache.drill.exec.ZookeeperHelper;
 import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
 import org.apache.drill.exec.proto.UserProtos.UserProperties;
 import org.apache.drill.exec.rpc.user.UserSession;
@@ -34,11 +40,6 @@ import org.apache.drill.exec.util.Pointer;
 import org.junit.Test;
 import org.slf4j.Logger;
 
-import java.util.concurrent.CountDownLatch;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 public class TestPauseInjection extends BaseTestQuery {
 
   private static final UserSession session = UserSession.Builder.newBuilder()
@@ -126,7 +127,7 @@ public class TestPauseInjection extends BaseTestQuery {
 
     ControlsInjectionUtil.setControls(session, controls);
 
-    final QueryContext queryContext = new QueryContext(session, bits[0].getContext());
+    final QueryContext queryContext = new QueryContext(session, bits[0].getContext(), QueryId.getDefaultInstance());
 
     (new ResumingThread(queryContext, trigger, ex, expectedDuration)).start();
 
@@ -181,7 +182,7 @@ public class TestPauseInjection extends BaseTestQuery {
       final long expectedDuration = 1000L;
       final ExtendedLatch trigger = new ExtendedLatch(1);
       final Pointer<Exception> ex = new Pointer<>();
-      final QueryContext queryContext = new QueryContext(session, drillbitContext1);
+      final QueryContext queryContext = new QueryContext(session, drillbitContext1, QueryId.getDefaultInstance());
       (new ResumingThread(queryContext, trigger, ex, expectedDuration)).start();
 
       // test that the pause happens
@@ -199,7 +200,7 @@ public class TestPauseInjection extends BaseTestQuery {
 
     {
       final ExtendedLatch trigger = new ExtendedLatch(1);
-      final QueryContext queryContext = new QueryContext(session, drillbitContext2);
+      final QueryContext queryContext = new QueryContext(session, drillbitContext2, QueryId.getDefaultInstance());
 
       // if the resume did not happen, the test would hang
       final DummyClass dummyClass = new DummyClass(queryContext, trigger);

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/exec/testing/TestResourceLeak.java
----------------------------------------------------------------------
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 7ab2da2..d2f2590 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
@@ -18,11 +18,13 @@
 package org.apache.drill.exec.testing;
 
 import static org.junit.Assert.fail;
+import io.netty.buffer.DrillBuf;
 
-import com.google.common.base.Charsets;
-import com.google.common.io.Resources;
+import java.io.IOException;
+import java.net.URL;
+import java.util.Properties;
 
-import io.netty.buffer.DrillBuf;
+import javax.inject.Inject;
 
 import org.apache.drill.QueryTestUtil;
 import org.apache.drill.common.config.DrillConfig;
@@ -42,16 +44,12 @@ import org.apache.drill.exec.server.Drillbit;
 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)
@@ -138,7 +136,7 @@ public class TestResourceLeak extends DrillTest {
 
     @Override
     public void eval() {
-      buf.getAllocator().buffer(1);
+      buf.retain();
       out.value = in.value;
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestPromotableWriter.java
----------------------------------------------------------------------
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 60a2268..223f4ed 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 @@ package org.apache.drill.exec.vector.complex.writer;
 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;

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/pom.xml
----------------------------------------------------------------------
diff --git a/exec/memory/base/pom.xml b/exec/memory/base/pom.xml
index adec763..686a12b 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>
 
 

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/io/netty/buffer/DrillBuf.java
----------------------------------------------------------------------
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 d244b26..138495c 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,23 +27,16 @@ import java.nio.ByteOrder;
 import java.nio.channels.GatheringByteChannel;
 import java.nio.channels.ScatteringByteChannel;
 import java.nio.charset.Charset;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.IdentityHashMap;
-import java.util.LinkedList;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.drill.common.HistoricalLog;
-import org.apache.drill.exec.memory.Accountor;
+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.apache.drill.exec.memory.BufferLedger;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.OperatorContext;
-import org.apache.drill.exec.util.AssertionUtil;
-import org.apache.drill.exec.util.Pointer;
-import org.slf4j.Logger;
 
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
@@ -51,343 +44,46 @@ import com.google.common.base.Preconditions;
 public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillBuf.class);
 
-  private static final boolean BOUNDS_CHECKING_ENABLED = AssertionUtil.BOUNDS_CHECKING_ENABLED;
-  private static final boolean DEBUG = BaseAllocator.isDebug();
-  private static final AtomicInteger idGenerator = new AtomicInteger(0);
+  private static final AtomicLong idGenerator = new AtomicLong(0);
 
-  private final ByteBuf byteBuf;
+  private final long id = idGenerator.incrementAndGet();
+  private final AtomicInteger refCnt;
+  private final UnsafeDirectLittleEndian byteBuf;
   private final long addr;
   private final int offset;
-  private final int flags;
-  private final AtomicInteger rootRefCnt;
-  private volatile BufferAllocator allocator;
-
-  // 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 final boolean oldWorld; // Indicates that we're operating with TopLevelAllocator.
-  private final boolean rootBuffer;
-  private volatile Accountor acct;
-  private BufferManager bufManager;
-  @Deprecated private OperatorContext operatorContext;
-  @Deprecated private FragmentContext fragmentContext;
-
-  private volatile BufferLedger bufferLedger;
-  private volatile int length; // TODO this just seems to duplicate .capacity()
-
-  // members used purely for debugging
-  // TODO once we have a reduced number of constructors, move these to DEBUG clauses in them
-  private final int id = idGenerator.incrementAndGet();
-  private final HistoricalLog historicalLog = DEBUG ? new HistoricalLog(4, "DrillBuf[%d]", id) : null;
-  private final static IdentityHashMap<UnsafeDirectLittleEndian, Collection<DrillBuf>> unwrappedMap =
-      DEBUG ? new IdentityHashMap<UnsafeDirectLittleEndian, Collection<DrillBuf>>() : null;
-
-  // TODO(cwestin) javadoc
-  private void unwrappedPut() {
-    final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) byteBuf;
-    synchronized(unwrappedMap) {
-      Collection<DrillBuf> drillBufs = unwrappedMap.get(udle);
-      if (drillBufs == null) {
-        drillBufs = new LinkedList<DrillBuf>();
-        unwrappedMap.put(udle, drillBufs);
-      }
-
-      drillBufs.add(this);
-    }
-  }
-
-  // TODO(cwestin) javadoc
-  public static Collection<DrillBuf> unwrappedGet(final UnsafeDirectLittleEndian udle) {
-    synchronized(unwrappedMap) {
-      final Collection<DrillBuf> drillBufs = unwrappedMap.get(udle);
-      if (drillBufs == null) {
-        return Collections.emptyList();
-      }
-      return new LinkedList<DrillBuf>(drillBufs);
-    }
-  }
-
-  // TODO(cwestin) javadoc
-  private static boolean unwrappedRemove(final DrillBuf drillBuf) {
-    final ByteBuf byteBuf = drillBuf.unwrap();
-    if (!(byteBuf instanceof UnsafeDirectLittleEndian)) {
-      return false;
-    }
-
-    final UnsafeDirectLittleEndian udle = (UnsafeDirectLittleEndian) byteBuf;
-    synchronized(unwrappedMap) {
-      Collection<DrillBuf> drillBufs = unwrappedMap.get(udle);
-      if (drillBufs == null) {
-        return false;
-      }
-      final Object object = drillBufs.remove(drillBuf);
-      if (drillBufs.isEmpty()) {
-        unwrappedMap.remove(udle);
-      }
-      return object != null;
-    }
-  }
-
-  public DrillBuf(BufferAllocator allocator, Accountor a, UnsafeDirectLittleEndian b) {
-    super(b.maxCapacity());
-    this.byteBuf = b;
-    this.addr = b.memoryAddress();
-    this.acct = a;
-    this.length = b.capacity();
-    this.offset = 0;
-    this.rootBuffer = true;
-    this.allocator = allocator;
-
-    // members from the new world order
-    flags = 0;
-    rootRefCnt = null;
-    oldWorld = true;
-  }
-
-  // TODO(cwestin) javadoc
-  public DrillBuf(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator,
-      final UnsafeDirectLittleEndian byteBuf) {
+  private final BufferLedger ledger;
+  private final BufferManager bufManager;
+  private final ByteBufAllocator alloc;
+  private final boolean isEmpty;
+  private volatile int 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;
-    byteBuf.retain(1);
-    this.bufferLedger = bufferLedger;
-    addr = byteBuf.memoryAddress();
-    allocator = bufferAllocator;
-    length = byteBuf.capacity();
-    offset = 0;
-    flags = 0;
-    rootRefCnt = new AtomicInteger(1);
-    oldWorld = false;
-
-    // members from the old world order
-    rootBuffer = false;
-    acct = null;
-
-    if (DEBUG) {
-      unwrappedPut();
-      historicalLog.recordEvent(
-          "DrillBuf(BufferLedger, BufferAllocator[%d], UnsafeDirectLittleEndian[identityHashCode == "
-              + "%d](%s)) => rootRefCnt identityHashCode == %d",
-              bufferAllocator.getId(), System.identityHashCode(byteBuf), byteBuf.toString(),
-              System.identityHashCode(rootRefCnt));
-    }
-  }
-
-  private DrillBuf(BufferAllocator allocator, Accountor a) {
-    super(0);
-    this.byteBuf = 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;
-
-    // members from the new world order
-    flags = 0;
-    rootRefCnt = null;
-    oldWorld = true;
-  }
-
-  private DrillBuf(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator) {
-    super(0);
-    this.bufferLedger = bufferLedger;
-    allocator = bufferAllocator;
-
-    byteBuf = new EmptyByteBuf(bufferLedger.getUnderlyingAllocator()).order(ByteOrder.LITTLE_ENDIAN);
-    length = 0;
-    addr = 0;
-    flags = 0;
-    rootRefCnt = new AtomicInteger(1);
-    offset = 0;
-
-    // members from the old world order
-    rootBuffer = false;
-    acct = null;
-    oldWorld = false;
-
-    if (DEBUG) {
-      // We don't put the empty buffers in the unwrappedMap.
-      historicalLog.recordEvent(
-          "DrillBuf(BufferLedger, BufferAllocator[%d])  => rootRefCnt identityHashCode == %d",
-          bufferAllocator.getId(), System.identityHashCode(rootRefCnt));
-    }
-  }
-
-  /**
-   * 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 + ')');
-    }
-
-    this.length = length;
-    writerIndex(length);
-
-    this.byteBuf = replacement;
-    this.addr = buffer.memoryAddress() + index;
-    this.offset = index;
-    this.acct = a;
+    this.isEmpty = isEmpty;
+    this.bufManager = manager;
+    this.alloc = alloc;
+    this.addr = byteBuf.memoryAddress() + offset;
+    this.ledger = ledger;
     this.length = length;
-    this.rootBuffer = root;
-    this.allocator = allocator;
+    this.offset = offset;
 
-    // members from the new world order
-    flags = 0;
-    rootRefCnt = null;
-    oldWorld = true;
-  }
-
-  /**
-   * Indicate a shared refcount, as per http://netty.io/wiki/reference-counted-objects.html#wiki-h3-5
-   */
-  private final static int F_DERIVED = 0x0002;
-
-  // TODO(cwestin) javadoc
-  /**
-   * Used for sharing.
-   *
-   * @param bufferLedger
-   * @param bufferAllocator
-   * @param originalBuf
-   * @param index
-   * @param length
-   * @param flags
-   */
-  public DrillBuf(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator,
-      final DrillBuf originalBuf, final int index, final int length, final int flags) {
-    this(bufferAllocator, bufferLedger, getUnderlyingUdle(originalBuf),
-        originalBuf, index + originalBuf.offset, length, flags);
-  }
-
-  /**
-   * Unwraps a DrillBuf until the underlying UnsafeDirectLittleEndian buffer is
-   * found.
-   *
-   * @param originalBuf the original DrillBuf
-   * @return the underlying UnsafeDirectLittleEndian ByteBuf
-   */
-  private static ByteBuf getUnderlyingUdle(final DrillBuf originalBuf) {
-    int count = 1;
-    ByteBuf unwrapped = originalBuf.unwrap();
-    while(!(unwrapped instanceof UnsafeDirectLittleEndian)
-        && (!(unwrapped instanceof EmptyByteBuf))) {
-      unwrapped = unwrapped.unwrap();
-      ++count;
-    }
-
-    if (DEBUG) {
-      if (count > 1) {
-        throw new IllegalStateException("UnsafeDirectLittleEndian is wrapped more than one level");
-      }
+    if (BaseAllocator.DEBUG) {
+      historicalLog.recordEvent("create()");
     }
 
-    return unwrapped;
-  }
-
-  // TODO(cwestin) javadoc
-  /*
-   * TODO the replacement argument becomes an UnsafeDirectLittleEndian;
-   * buffer argument may go away if it is determined to be unnecessary after all
-   * the deprecated stuff is removed (I suspect only the replacement argument is
-   * necessary then).
-   */
-  private DrillBuf(BufferAllocator allocator, BufferLedger bufferLedger,
-      ByteBuf replacement, DrillBuf buffer, int index, int length, int flags) {
-    super(replacement.maxCapacity());
-
-    // members from the old world order
-    rootBuffer = false;
-    acct = null;
-    oldWorld = false;
-
-    if (index < 0 || index > (replacement.maxCapacity() - length)) {
-      throw new IndexOutOfBoundsException(replacement.toString() + ".slice(" + index + ", " + length + ')');
-    }
-
-    this.flags = flags;
-
-    this.length = length; // capacity()
-    writerIndex(length);
-
-    byteBuf = replacement;
-    if ((flags & F_DERIVED) == 0) {
-      replacement.retain(1);
-    }
-
-    addr = replacement.memoryAddress() + index;
-    offset = index;
-    this.bufferLedger = bufferLedger;
-    if (!(buffer instanceof DrillBuf)) {
-      throw new IllegalArgumentException("DrillBuf slicing can only be performed on other DrillBufs");
-    }
-
-    if ((flags & F_DERIVED) != 0) {
-      final DrillBuf rootBuf = (DrillBuf) buffer;
-      rootRefCnt = rootBuf.rootRefCnt;
-    } else {
-      rootRefCnt = new AtomicInteger(1);
-    }
-
-    this.allocator = allocator;
-
-    if (DEBUG) {
-      unwrappedPut();
-      historicalLog.recordEvent(
-          "DrillBuf(BufferAllocator[%d], BufferLedger, ByteBuf[identityHashCode == "
-              + "%d](%s), DrillBuf[%d], index = %d, length = %d, flags = 0x%08x)"
-              + " => rootRefCnt identityHashCode == %d",
-          allocator.getId(), System.identityHashCode(replacement), replacement.toString(),
-          buffer.id, index, length, flags, System.identityHashCode(rootRefCnt));
-    }
-  }
-
-  @Deprecated
-  public void setOperatorContext(OperatorContext c) {
-    this.operatorContext = c;
-  }
-
-  @Deprecated
-  public void setFragmentContext(FragmentContext c) {
-    this.fragmentContext = c;
-  }
-
-  // TODO(DRILL-3331)
-  public void setBufferManager(BufferManager bufManager) {
-    Preconditions.checkState(this.bufManager == null,
-        "the BufferManager for a buffer can only be set once");
-    this.bufManager = bufManager;
-  }
-
-  public BufferAllocator getAllocator() {
-    return allocator;
   }
 
   public DrillBuf reallocIfNeeded(final int size) {
@@ -397,11 +93,7 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
       return this;
     }
 
-    if (operatorContext != null) {
-      return operatorContext.replace(this, size);
-    } else if(fragmentContext != null) {
-      return fragmentContext.replace(this, size);
-    } else if (bufManager != null) {
+    if (bufManager != null) {
       return bufManager.replace(this, size);
     } else {
       throw new UnsupportedOperationException("Realloc is only available in the context of an operator's UDFs");
@@ -410,15 +102,11 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
 
   @Override
   public int refCnt() {
-    if (oldWorld) {
-      if(rootBuffer){
-        return (int) this.rootRefCnt.get();
-      }else{
-        return byteBuf.refCnt();
-      }
+    if (isEmpty) {
+      return 1;
+    } else {
+      return refCnt.get();
     }
-
-    return rootRefCnt.get();
   }
 
   private long addr(int index) {
@@ -431,7 +119,7 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
       throw new IllegalArgumentException("length: " + fieldLength + " (expected: >= 0)");
     }
     if (index < 0 || index > capacity() - fieldLength) {
-      if (DEBUG) {
+      if (BaseAllocator.DEBUG) {
         historicalLog.logHistory(logger);
       }
       throw new IndexOutOfBoundsException(String.format(
@@ -449,70 +137,106 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
    * @param end The exclusive endpoint of the bytes to be read.
    */
   public void checkBytes(int start, int end) {
-    if (BOUNDS_CHECKING_ENABLED) {
+    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 ensure(int width) {
-    if (BOUNDS_CHECKING_ENABLED) {
+    if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
       ensureWritable(width);
     }
   }
 
   /**
-   * Used by allocators to transfer ownership from one allocator to another.
+   * 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 the current DrillBuf except in the situation where the
+   * passed in Allocator is the same as the current buffer.
+   *
+   * 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 newLedger the new ledger the buffer should use going forward
-   * @param newAllocator the new allocator
-   * @return whether or not the buffer fits the receiving allocator's allocation limit
+   * @param allocator
+   *          The target allocator to create an association with.
+   * @return A new DrillBuf which shares the same underlying memory as this DrillBuf.
    */
-  public boolean transferTo(final BufferAllocator newAllocator, final BufferLedger newLedger) {
-    final Pointer<BufferLedger> pNewLedger = new Pointer<>(newLedger);
-    final boolean fitsAllocation = bufferLedger.transferTo(newAllocator, pNewLedger, this);
-    allocator = newAllocator;
-    bufferLedger = pNewLedger.value;
-    return fitsAllocation;
+  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);
   }
 
   /**
-   * DrillBuf's implementation of sharing buffer functionality, to be accessed from
-   * {@link BufferAllocator#shareOwnership(DrillBuf, Pointer)}. See that function
-   * for more information.
+   * 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.
    *
-   * @param otherLedger the ledger belonging to the other allocator to share with
-   * @param otherAllocator the other allocator to be shared with
-   * @param index the starting index (for slicing capability)
-   * @param length the length (for slicing capability)
-   * @return the new DrillBuf (wrapper)
+   * 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 DrillBuf shareWith(final BufferLedger otherLedger, final BufferAllocator otherAllocator,
-      final int index, final int length) {
-    return shareWith(otherLedger, otherAllocator, index, length, 0);
-  }
+  public TransferResult transferOwnership(BufferAllocator target) {
 
-  // TODO(cwestin) javadoc
-  private DrillBuf shareWith(final BufferLedger otherLedger, final BufferAllocator otherAllocator,
-      final int index, final int length, final int flags) {
-    final Pointer<DrillBuf> pDrillBuf = new Pointer<>();
-    bufferLedger = bufferLedger.shareWith(pDrillBuf, otherLedger, otherAllocator, this, index, length, flags);
-    return pDrillBuf.value;
+    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);
   }
 
-  public boolean transferAccounting(Accountor target) {
-    if (rootBuffer) {
-      boolean outcome = acct.transferTo(target, this, length);
-      acct = target;
-      return outcome;
-    } else {
-      throw new UnsupportedOperationException();
+  /**
+   * 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
@@ -525,40 +249,28 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
    */
   @Override
   public synchronized boolean release(int decrement) {
-    Preconditions.checkArgument(decrement > 0,
-        "release(%d) argument is not positive", decrement);
-    if (DEBUG) {
-      historicalLog.recordEvent("release(%d)", decrement);
+    if (isEmpty) {
+      return false;
     }
 
-    if (oldWorld) {
-      if(rootBuffer){
-        final long newRefCnt = this.rootRefCnt.addAndGet(-decrement);
-        Preconditions.checkArgument(newRefCnt > -1, "Buffer has negative reference count.");
-        if (newRefCnt == 0) {
-          byteBuf.release(decrement);
-          acct.release(this, length);
-          return true;
-        }else{
-          return false;
-        }
-      }else{
-        return byteBuf.release(decrement);
-      }
+    if (decrement < 1) {
+      throw new IllegalStateException(String.format("release(%d) argument is not positive. Buffer Info: %s",
+          decrement, toVerboseString()));
     }
 
-    final int refCnt = rootRefCnt.addAndGet(-decrement);
-    Preconditions.checkState(refCnt >= 0, "DrillBuf[%d] refCnt has gone negative", id);
-    if (refCnt == 0) {
-      bufferLedger.release(this);
+    final int refCnt = this.refCnt.addAndGet(-decrement);
 
-      if (DEBUG) {
-        unwrappedRemove(this);
-      }
+    if (BaseAllocator.DEBUG) {
+      historicalLog.recordEvent("release(%d). original value: %d", decrement, refCnt + decrement);
+    }
 
-      // release the underlying buffer
-      byteBuf.release(1);
+    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;
     }
 
@@ -571,37 +283,16 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   }
 
   @Override
-  public synchronized ByteBuf capacity(int newCapacity) {
-    if (oldWorld) {
-      if (rootBuffer) {
-        if (newCapacity == length) {
-          return this;
-        } else if (newCapacity < length) {
-          byteBuf.capacity(newCapacity);
-          int diff = length - byteBuf.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.");
-      }
-    }
-
-    if ((flags & F_DERIVED) != 0) {
-      throw new UnsupportedOperationException("Derived buffers don't support resizing.");
-    }
+  public synchronized DrillBuf capacity(int newCapacity) {
 
     if (newCapacity == length) {
       return this;
     }
 
+    Preconditions.checkArgument(newCapacity >= 0);
+
     if (newCapacity < length) {
-      byteBuf.capacity(newCapacity);
-      final int diff = length - byteBuf.capacity();
-      length -= diff;
+      length = newCapacity;
       return this;
     }
 
@@ -673,10 +364,9 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
 
   @Override
   public DrillBuf slice(int index, int length) {
-    if (oldWorld) {
-      DrillBuf buf = new DrillBuf(this, index, length);
-      buf.writerIndex = length;
-      return buf;
+
+    if (isEmpty) {
+      return this;
     }
 
     /*
@@ -684,17 +374,13 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
      * 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 buf = shareWith(bufferLedger, allocator, index, length, F_DERIVED);
-    buf.writerIndex(length);
-    return buf;
+    final DrillBuf newBuf = ledger.newDrillBuf(offset + index, length);
+    newBuf.writerIndex(length);
+    return newBuf;
   }
 
   @Override
   public DrillBuf duplicate() {
-    if (oldWorld) {
-      return new DrillBuf(this, 0, length);
-    }
-
     return slice(0, length);
   }
 
@@ -766,8 +452,8 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   @Override
   public String toString(int index, int length, Charset charset) {
     final String basics =
-        String.format("{DrillBuf[%d], udle identityHashCode == %d, rootRefCnt identityHashCode == %d}",
-            id, System.identityHashCode(byteBuf), System.identityHashCode(rootRefCnt));
+        String.format("{DrillBuf[%d], udle identityHashCode == %d, identityHashCode == %d}",
+            id, System.identityHashCode(byteBuf), System.identityHashCode(refCnt));
 
     if (length == 0) {
       return basics;
@@ -799,20 +485,16 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
   @Override
   public ByteBuf retain(int increment) {
     Preconditions.checkArgument(increment > 0, "retain(%d) argument is not positive", increment);
-    if (DEBUG) {
-      historicalLog.recordEvent("retain(%d)", increment);
-    }
 
-    if (oldWorld) {
-      if(rootBuffer){
-        this.rootRefCnt.addAndGet(increment);
-      }else{
-        byteBuf.retain(increment);
-      }
+    if (isEmpty) {
       return this;
     }
 
-    rootRefCnt.addAndGet(increment);
+    if (BaseAllocator.DEBUG) {
+      historicalLog.recordEvent("retain(%d)", increment);
+    }
+
+    refCnt.addAndGet(increment);
     return this;
   }
 
@@ -1109,65 +791,42 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
     return PlatformDependent.getByte(addr(index));
   }
 
-  public static DrillBuf getEmpty(BufferAllocator allocator, Accountor a) {
-    return new DrillBuf(allocator, a);
-  }
-
-  public static DrillBuf getEmpty(final BufferLedger bufferLedger, final BufferAllocator bufferAllocator) {
-    return new DrillBuf(bufferLedger, bufferAllocator);
+  @Override
+  public void close() {
+    release();
   }
 
   /**
-   * Find out if this is a "root buffer." This is obsolete terminology
-   * based on the original implementation of DrillBuf, which would layer
-   * DrillBufs on top of other DrillBufs when slicing (or duplicating).
-   * The buffer at the bottom of the layer was the "root buffer." However,
-   * the current implementation flattens such references to always make
-   * DrillBufs that are wrap a single buffer underneath, and slices and
-   * their original source have a shared fate as per
-   * http://netty.io/wiki/reference-counted-objects.html#wiki-h3-5, so
-   * this concept isn't really meaningful anymore. But there are callers
-   * that want to know a buffer's original size, and whether or not it
-   * is "primal" in some sense. Perhaps this just needs a new name that
-   * indicates that the buffer was an "original" and not a slice.
+   * Returns the possible memory consumed by this DrillBuf in the worse case scenario. (not shared, connected to larger
+   * underlying buffer of allocated memory)
    *
-   * @return whether or not the buffer is an original
+   * @return Size in bytes.
    */
-  @Deprecated
-  public boolean isRootBuffer() {
-    if (oldWorld) {
-      return rootBuffer;
-    }
-
-    return (flags & F_DERIVED) == 0;
-  }
-
-  @Override
-  public void close() {
-    release();
+  public int getPossibleMemoryConsumed() {
+    return ledger.getSize();
   }
 
   /**
-   * Indicates whether this DrillBuf and the supplied one have a "shared fate."
-   * Having a "shared fate" indicates that the two DrillBufs share a reference
-   * count, and will both be released at the same time if either of them is
-   * released.
-   * @param otherBuf the other buffer to check against
-   * @return true if the two buffers have a shared fate, false otherwise
+   * 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 boolean hasSharedFate(final DrillBuf otherBuf) {
-    return rootRefCnt == otherBuf.rootRefCnt;
+  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
+   * Return the buffer's byte contents in the form of a hex dump.
+   * @param start
+   *          the starting byte index
+   * @param length
+   *          how many bytes to log
+   * @return A hex dump in a String.
    */
-  public void logBytes(final Logger logger, final int start, final int length) {
+  public String toHexString(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");
@@ -1184,7 +843,7 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
       }
       sb.append('\n');
     }
-    logger.trace(sb.toString());
+    return sb.toString();
   }
 
   /**
@@ -1192,27 +851,28 @@ public final class DrillBuf extends AbstractByteBuf implements AutoCloseable {
    *
    * @return integer id
    */
-  public int getId() {
+  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 logHistoryForUdle(final Logger logger, final UnsafeDirectLittleEndian udle) {
-    final Collection<DrillBuf> drillBufs = unwrappedGet(udle);
-    for(final DrillBuf drillBuf : drillBufs) {
-      drillBuf.logHistory(logger);
+  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);
     }
   }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/io/netty/buffer/ExpandableByteBuf.java
----------------------------------------------------------------------
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 0000000..7788552
--- /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);
+    }
+  }
+
+}


[10/13] drill git commit: DRILL-4134: Allocator Improvements

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/io/netty/buffer/FakeAllocator.java
----------------------------------------------------------------------
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 b8d0fb2..0000000
--- a/exec/memory/base/src/main/java/io/netty/buffer/FakeAllocator.java
+++ /dev/null
@@ -1,167 +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.memory.Accountor;
-import org.apache.drill.exec.memory.AllocationReservation;
-import org.apache.drill.exec.memory.AllocatorOwner;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.ops.FragmentContext;
-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(FragmentContext context, 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 void setFragmentLimit(long l) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public long getFragmentLimit(){
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void close() {
-  }
-
-  @Override
-  public long getAllocatedMemory() {
-    return 0;
-  }
-
-  @Override
-  public long getPeakMemoryAllocation() {
-    return 0;
-  }
-
-  static class FakeAccountor extends Accountor {
-
-    public FakeAccountor() {
-      super(null, false, null, null, 0, 0, true);
-    }
-
-    @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) {
-
-    }
-
-    @Override
-    public void release(DrillBuf buf, long size) {
-
-    }
-
-    @Override
-    public void close() {
-
-    }
-  }
-
-  @Override
-  public BufferAllocator newChildAllocator(AllocatorOwner allocatorOwner,
-      long initReservation, long maxAllocation, int flags) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public boolean shareOwnership(DrillBuf buf, Pointer<DrillBuf> bufOut) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int getId() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public AllocationReservation newReservation() {
-    throw new UnsupportedOperationException();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/io/netty/buffer/LargeBuffer.java
----------------------------------------------------------------------
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 f1d4842..5f5e904 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
@@ -170,169 +42,6 @@ public class LargeBuffer extends AbstractByteBuf {
   }
 
   @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);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/io/netty/buffer/MutableWrappedByteBuf.java
----------------------------------------------------------------------
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 0000000..5709473
--- /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;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/io/netty/buffer/PooledByteBufAllocatorL.java
----------------------------------------------------------------------
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 2fc1bd0..47dbf59 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,227 @@ import java.lang.reflect.Field;
 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);
-
+/**
+ * The base allocator that we use for all of Drill's memory management. Returns UnsafeDirectLittleEndian buffers.
+ */
+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();
+        }
+      });
 
-  @Override
-  protected UnsafeDirectLittleEndian newDirectBuffer(int initialCapacity, int maxCapacity) {
-    PoolThreadCache cache = threadCache.get();
-    PoolArena<ByteBuffer> directArena = cache.directArena;
+      registry.register(METRIC_PREFIX + "huge.size", new Gauge<Long>() {
+        @Override
+        public Long getValue() {
+          return hugeBufferSize.get();
+        }
+      });
 
-    if (directArena != null) {
+      registry.register(METRIC_PREFIX + "huge.count", new Gauge<Long>() {
+        @Override
+        public Long getValue() {
+          return hugeBufferCount.get();
+        }
+      });
 
-      if (initialCapacity > directArena.chunkSize) {
-        // This is beyond chunk size so we'll allocate separately.
-        ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.directBuffer(initialCapacity, maxCapacity);
+      largeBuffersHist = registry.histogram(METRIC_PREFIX + "huge.hist");
+      normalBuffersHist = registry.histogram(METRIC_PREFIX + "normal.hist");
 
-        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) {
+    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 +255,10 @@ public class PooledByteBufAllocatorL extends PooledByteBufAllocator{
       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 +268,5 @@ public class PooledByteBufAllocatorL extends PooledByteBufAllocator{
     assert isAssertEnabled = true;
     ASSERT_ENABLED = isAssertEnabled;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/io/netty/buffer/UnsafeDirectLittleEndian.java
----------------------------------------------------------------------
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 419aef3..12e9907 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
@@ -23,92 +23,46 @@ import io.netty.util.internal.PlatformDependent;
 import java.nio.ByteOrder;
 import java.util.concurrent.atomic.AtomicLong;
 
+/**
+ * The underlying class we use for little-endian access to memory. Is used underneath DrillBufs to abstract away the
+ * Netty classes and underlying Netty memory management.
+ */
 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 IdentityHashMap<UnsafeDirectLittleEndian, StackTrace> bufferMap = new IdentityHashMap<>();
+  private final AtomicLong bufferCount;
+  private final AtomicLong bufferSize;
+  private final long initCap;
 
-  @Override
-  public boolean release() {
-    return release(1);
+  UnsafeDirectLittleEndian(DuplicatedByteBuf buf) {
+    this(buf, true, null, null);
   }
 
-  @Override
-  public boolean release(int decrement) {
-    boolean released = super.release(decrement);
-    if (TRACK_BUFFERS) {
-      if (released) {
-        final Object object;
-        synchronized (bufferMap) {
-          object = bufferMap.remove(this);
-        }
-        if (object == null) {
-          throw new IllegalStateException("no such buffer");
-        }
-
-        if (initCap != -1) {
-          bufferCount.decrementAndGet();
-          bufferSize.addAndGet(-initCap);
-        }
-      }
-    }
-
-    return released;
+  UnsafeDirectLittleEndian(LargeBuffer buf) {
+    this(buf, true, null, null);
   }
 
+  UnsafeDirectLittleEndian(PooledUnsafeDirectByteBuf buf, AtomicLong bufferCount, AtomicLong bufferSize) {
+    this(buf, true, bufferCount, bufferSize);
 
-  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) {
-    synchronized (bufferMap) {
-      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);
-      }
+  private UnsafeDirectLittleEndian(AbstractByteBuf buf, boolean fake, AtomicLong bufferCount, AtomicLong bufferSize) {
+    super(buf);
+    if (!NATIVE_ORDER || buf.order() != ByteOrder.BIG_ENDIAN) {
+      throw new IllegalStateException("Drill only runs on LittleEndian systems.");
     }
-  }
-  UnsafeDirectLittleEndian(LargeBuffer buf) {
-    this(buf, true);
-  }
 
-  UnsafeDirectLittleEndian(PooledUnsafeDirectByteBuf buf, AtomicLong bufferCount, AtomicLong bufferSize) {
-    this(buf, true);
     this.bufferCount = bufferCount;
     this.bufferSize = bufferSize;
 
     // initCap is used if we're tracking memory release. If we're in non-debug mode, we'll skip this.
-    this.initCap = ASSERT_ENABLED ? capacity() : -1;
-  }
+    this.initCap = ASSERT_ENABLED ? buf.capacity() : -1;
 
-  private UnsafeDirectLittleEndian(AbstractByteBuf buf, boolean fake) {
-    super(buf);
-    if (!NATIVE_ORDER || buf.order() != ByteOrder.BIG_ENDIAN) {
-      throw new IllegalStateException("Drill only runs on LittleEndian systems.");
-    }
-    wrapped = buf;
-    memoryAddress = buf.memoryAddress();
+    this.wrapped = buf;
+    this.memoryAddress = buf.memoryAddress();
   }
     private long addr(int index) {
         return memoryAddress + index;
@@ -147,142 +101,140 @@ public final class UnsafeDirectLittleEndian extends WrappedByteBuf {
   }
 
   @Override
-    public double getDouble(int index) {
-        return Double.longBitsToDouble(getLong(index));
-    }
+  public double getDouble(int index) {
+    return Double.longBitsToDouble(getLong(index));
+  }
 
-    @Override
-    public char getChar(int index) {
-        return (char) getShort(index);
-    }
+  @Override
+  public char getChar(int index) {
+    return (char) getShort(index);
+  }
 
-    @Override
-    public long getUnsignedInt(int index) {
-        return getInt(index) & 0xFFFFFFFFL;
-    }
+  @Override
+  public long getUnsignedInt(int index) {
+    return getInt(index) & 0xFFFFFFFFL;
+  }
 
-    @Override
-    public int getInt(int index) {
-//        wrapped.checkIndex(index, 4);
-        int v = PlatformDependent.getInt(addr(index));
-        return v;
-    }
+  @Override
+  public int getInt(int index) {
+    int v = PlatformDependent.getInt(addr(index));
+    return v;
+  }
 
-    @Override
-    public int getUnsignedShort(int index) {
-        return getShort(index) & 0xFFFF;
-    }
+  @Override
+  public int getUnsignedShort(int index) {
+    return getShort(index) & 0xFFFF;
+  }
 
-    @Override
-    public short getShort(int index) {
-//        wrapped.checkIndex(index, 2);
-        short v = PlatformDependent.getShort(addr(index));
-        return v;
-    }
+  @Override
+  public short getShort(int index) {
+    short v = PlatformDependent.getShort(addr(index));
+    return v;
+  }
 
-    @Override
-    public ByteBuf setShort(int index, int value) {
-        wrapped.checkIndex(index, 2);
-        _setShort(index, value);
-        return this;
-    }
+  @Override
+  public ByteBuf setShort(int index, int value) {
+    wrapped.checkIndex(index, 2);
+    _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 setInt(int index, int value) {
+    wrapped.checkIndex(index, 4);
+    _setInt(index, value);
+    return this;
+  }
 
-    @Override
-    public ByteBuf setLong(int index, long value) {
-        wrapped.checkIndex(index, 8);
-        _setLong(index, value);
-        return this;
-    }
+  @Override
+  public ByteBuf setLong(int index, long value) {
+    wrapped.checkIndex(index, 8);
+    _setLong(index, value);
+    return this;
+  }
 
-    @Override
-    public ByteBuf setChar(int index, int value) {
-        setShort(index, value);
-        return this;
-    }
+  @Override
+  public ByteBuf setChar(int index, int value) {
+    setShort(index, value);
+    return this;
+  }
 
-    @Override
-    public ByteBuf setFloat(int index, float value) {
-        setInt(index, Float.floatToRawIntBits(value));
-        return this;
-    }
+  @Override
+  public ByteBuf setFloat(int index, float value) {
+    setInt(index, Float.floatToRawIntBits(value));
+    return this;
+  }
 
-    @Override
-    public ByteBuf setDouble(int index, double value) {
-        setLong(index, Double.doubleToRawLongBits(value));
-        return this;
-    }
+  @Override
+  public ByteBuf setDouble(int index, double value) {
+    setLong(index, Double.doubleToRawLongBits(value));
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeShort(int value) {
-        wrapped.ensureWritable(2);
-        _setShort(wrapped.writerIndex, value);
-        wrapped.writerIndex += 2;
-        return this;
-    }
+  @Override
+  public ByteBuf writeShort(int value) {
+    wrapped.ensureWritable(2);
+    _setShort(wrapped.writerIndex, value);
+    wrapped.writerIndex += 2;
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeInt(int value) {
-        wrapped.ensureWritable(4);
-        _setInt(wrapped.writerIndex, value);
-        wrapped.writerIndex += 4;
-        return this;
-    }
+  @Override
+  public ByteBuf writeInt(int value) {
+    wrapped.ensureWritable(4);
+    _setInt(wrapped.writerIndex, value);
+    wrapped.writerIndex += 4;
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeLong(long value) {
-        wrapped.ensureWritable(8);
-        _setLong(wrapped.writerIndex, value);
-        wrapped.writerIndex += 8;
-        return this;
-    }
+  @Override
+  public ByteBuf writeLong(long value) {
+    wrapped.ensureWritable(8);
+    _setLong(wrapped.writerIndex, value);
+    wrapped.writerIndex += 8;
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeChar(int value) {
-        writeShort(value);
-        return this;
-    }
+  @Override
+  public ByteBuf writeChar(int value) {
+    writeShort(value);
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeFloat(float value) {
-        writeInt(Float.floatToRawIntBits(value));
-        return this;
-    }
+  @Override
+  public ByteBuf writeFloat(float value) {
+    writeInt(Float.floatToRawIntBits(value));
+    return this;
+  }
 
-    @Override
-    public ByteBuf writeDouble(double value) {
-        writeLong(Double.doubleToRawLongBits(value));
-        return this;
-    }
+  @Override
+  public ByteBuf writeDouble(double value) {
+    writeLong(Double.doubleToRawLongBits(value));
+    return this;
+  }
 
-    private void _setShort(int index, int value) {
-        PlatformDependent.putShort(addr(index), (short) value);
-    }
+  private void _setShort(int index, int value) {
+    PlatformDependent.putShort(addr(index), (short) value);
+  }
 
-    private void _setInt(int index, int value) {
-        PlatformDependent.putInt(addr(index), value);
-    }
+  private void _setInt(int index, int value) {
+    PlatformDependent.putInt(addr(index), value);
+  }
 
-    private void _setLong(int index, long value) {
-        PlatformDependent.putLong(addr(index), value);
-    }
+  private void _setLong(int index, long value) {
+    PlatformDependent.putLong(addr(index), value);
+  }
 
-    @Override
-    public byte getByte(int index) {
-      return PlatformDependent.getByte(addr(index));
-    }
+  @Override
+  public byte getByte(int index) {
+    return PlatformDependent.getByte(addr(index));
+  }
 
-    @Override
-    public ByteBuf setByte(int index, int value) {
-      PlatformDependent.putByte(addr(index), (byte) value);
-      return this;
-    }
+  @Override
+  public ByteBuf setByte(int index, int value) {
+    PlatformDependent.putByte(addr(index), (byte) value);
+    return this;
+  }
 
   @Override
   public boolean release() {
@@ -291,8 +243,8 @@ public final class UnsafeDirectLittleEndian extends WrappedByteBuf {
 
   @Override
   public boolean release(int decrement) {
-    boolean released = super.release(decrement);
-    if (released && initCap != -1) {
+    final boolean released = super.release(decrement);
+    if (ASSERT_ENABLED && released && bufferCount != null && bufferSize != null) {
       bufferCount.decrementAndGet();
       bufferSize.addAndGet(-initCap);
     }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountant.java
----------------------------------------------------------------------
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 0000000..8bcf6a0
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountant.java
@@ -0,0 +1,274 @@
+/**
+ * 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(String.format(
+            "Failure trying to allocate initial reservation for Allocator. "
+                + "Attempted to allocate %d bytes and received an outcome of %s.", reservation, outcome.name()));
+      }
+    }
+  }
+
+  /**
+   * 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)) {
+          // peak allocation changed underneath us. try again.
+          continue;
+        }
+      }
+
+      // we either succeeded to set peak allocation or we weren't above the previous peak, exit.
+      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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountor.java
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountor.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountor.java
deleted file mode 100644
index 7014a0b..0000000
--- a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/Accountor.java
+++ /dev/null
@@ -1,43 +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.DrillBuf;
-
-public interface Accountor extends AutoCloseable{
-
-  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 boolean reserve(long size);
-  public boolean forceAdditionalReservation(long size);
-
-  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();
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java
----------------------------------------------------------------------
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 0000000..65a1386
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java
@@ -0,0 +1,86 @@
+/**
+ * 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;
+
+/**
+ * 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 interface AllocationReservation extends AutoCloseable {
+
+  /**
+   * 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
+   */
+  boolean add(final int nBytes);
+
+  /**
+   * 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
+   */
+  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
+   */
+  DrillBuf allocateBuffer();
+
+  /**
+   * Get the current size of the reservation (the sum of all the add()s).
+   *
+   * @return size of the current reservation
+   */
+  int getSize();
+
+  /**
+   * Return whether or not the reservation has been used.
+   *
+   * @return whether or not the reservation has been used
+   */
+  public boolean isUsed();
+
+  /**
+   * Return whether or not the reservation has been closed.
+   *
+   * @return whether or not the reservation has been closed
+   */
+  public boolean isClosed();
+
+  public void close();
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java
----------------------------------------------------------------------
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 0000000..8bf2a99
--- /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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorManager.java
----------------------------------------------------------------------
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 0000000..5142806
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/AllocatorManager.java
@@ -0,0 +1,356 @@
+/**
+ * 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.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.drill.common.HistoricalLog;
+import org.apache.drill.common.concurrent.AutoCloseableLock;
+import org.apache.drill.exec.memory.BaseAllocator.Verbosity;
+import org.apache.drill.exec.metrics.DrillMetrics;
+import org.apache.drill.exec.ops.BufferManager;
+
+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 IdentityHashMap<BufferAllocator, BufferLedger> map = new IdentityHashMap<>();
+  private final ReadWriteLock lock = new ReentrantReadWriteLock();
+  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(final BaseAllocator allocator) {
+    if (root != allocator.root) {
+      throw new IllegalStateException(
+          "A buffer can only be associated between two allocators that share the same root.");
+    }
+
+    try (AutoCloseableLock read = readLock.open()) {
+
+      final BufferLedger ledger = map.get(allocator);
+      if (ledger != null) {
+        return ledger;
+      }
+
+    }
+    try (AutoCloseableLock write = writeLock.open()) {
+      final BufferLedger ledger = new BufferLedger(allocator, new ReleaseListener(allocator));
+      map.put(allocator, 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 BufferAllocator allocator;
+
+    public ReleaseListener(BufferAllocator allocator) {
+      this.allocator = allocator;
+    }
+
+    public void release() {
+      try (AutoCloseableLock write = writeLock.open()) {
+        final BufferLedger oldLedger = map.remove(allocator);
+        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.values().iterator().next();
+
+            // 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);
+          }
+        }
+
+
+      }
+    }
+  }
+
+  /**
+   * 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) {
+        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);
+            sb.append('\n');
+          }
+        }
+      }
+
+    }
+
+    /**
+     * 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((BaseAllocator) allocator);
+    }
+
+    /**
+     * 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
+     */
+    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[%s], UnsafeDirectLittleEndian[identityHashCode == "
+                + "%d](%s)) => ledger hc == %d",
+            allocator.name, 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


[09/13] drill git commit: DRILL-4134: Allocator Improvements

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
----------------------------------------------------------------------
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 0000000..78c3c73
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BaseAllocator.java
@@ -0,0 +1,739 @@
+/**
+ * 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 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[%s]", name);
+      hist("created by \"%s\", owned = %d", name, 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.name
+              + "] not found in parent allocator[" + name + "]'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 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 = false;
+    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;
+  }
+
+  @Override
+  public BufferAllocator newChildAllocator(
+      final String name,
+      final long initReservation,
+      final long maxAllocation) {
+    final ChildAllocator childAllocator = new ChildAllocator(this, name, initReservation, maxAllocation);
+
+    if (DEBUG) {
+      synchronized (DEBUG_LOCK) {
+        childAllocators.put(childAllocator, childAllocator);
+        historicalLog.recordEvent("allocator[%s] created new child allocator[%s]", name, childAllocator.name);
+      }
+    }
+
+    return childAllocator;
+  }
+
+  public class Reservation implements AllocationReservation {
+    private int nBytes = 0;
+    private boolean used = false;
+    private boolean closed = false;
+    private final HistoricalLog historicalLog;
+
+    public Reservation() {
+      if (DEBUG) {
+        historicalLog = new HistoricalLog("Reservation[allocator[%s], %d]", name, System.identityHashCode(this));
+        historicalLog.recordEvent("created");
+        synchronized (DEBUG_LOCK) {
+          reservations.put(this, this);
+        }
+      } else {
+        historicalLog = null;
+      }
+    }
+
+    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;
+    }
+
+    public DrillBuf allocateBuffer() {
+      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;
+    }
+
+    public int getSize() {
+      return nBytes;
+    }
+
+    public boolean isUsed() {
+      return used;
+    }
+
+    public boolean isClosed() {
+      return closed;
+    }
+
+    @Override
+    public void close() {
+      if (closed) {
+        return;
+      }
+
+      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");
+        }
+      }
+
+      if (!used) {
+        releaseReservation(nBytes);
+      }
+
+      closed = true;
+    }
+
+    public 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();
+    }
+
+    /**
+     * 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
+     */
+    private 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", String.format("DrillBuf[%d]", drillBuf.getId()));
+        }
+        success = true;
+        return drillBuf;
+      } finally {
+        if (!success) {
+          releaseBytes(nBytes);
+        }
+      }
+    }
+
+    /**
+     * Return the reservation back to the allocator without having used it.
+     *
+     * @param nBytes
+     *          the size of the reservation
+     */
+    private 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() ? 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[" + name + "] child event logs BEGIN");
+        for (final BaseAllocator childAllocator : childSet) {
+          childAllocator.historicalLog.logHistory(logger);
+        }
+        logger.debug("allocator[" + name + "] child event logs END");
+        throw new IllegalStateException(
+            "Child allocators own more memory (" + childTotal + ") than their parent (name = "
+                + name + " ) 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(name);
+        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(childAllocator.name);
+            sb.append("] owned ");
+            sb.append(Long.toString(childAllocator.getAllocatedMemory()));
+            sb.append('\n');
+          }
+        }
+
+        if (reservedTotal != 0) {
+          sb.append(String.format("reserved total : %d bytes.", 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[%s]: buffer space (%d) + prealloc space (%d) + child space (%d) != allocated (%d)",
+            name, 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');
+
+    if (DEBUG) {
+      indent(sb, level + 1).append(String.format("child allocators: %d\n", childAllocators.size()));
+      for (BaseAllocator child : childAllocators.keySet()) {
+        child.print(sb, level + 2, verbosity);
+      }
+
+      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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BoundsChecking.java
----------------------------------------------------------------------
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 a15e348..bc61182 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 @@ public class BoundsChecking {
 
   private BoundsChecking() {
   }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
----------------------------------------------------------------------
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 7d14b94..0226254 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,8 +20,8 @@ package org.apache.drill.exec.memory;
 import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.DrillBuf;
 
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.util.Pointer;
+import org.apache.drill.exec.exception.OutOfMemoryException;
+import org.apache.drill.exec.ops.BufferManager;
 
 /**
  * Wrapper class to deal with byte buffer allocation. Ensures users only use designated methods.
@@ -31,156 +31,119 @@ 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 buffer's capacity will be set to the configured size.
    *
-   * @param size The size in bytes.
+   * @param size
+   *          The size in bytes.
    * @return a new DrillBuf, or null if the request can't be satisfied
-   * @throws OutOfMemoryRuntimeException if buffer cannot be allocated
+   * @throws OutOfMemoryException
+   *           if buffer cannot be allocated
    */
   public DrillBuf buffer(int size);
 
   /**
-   * Allocate a new or reused buffer within provided range. Note that the buffer may technically be larger than the
+   * 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.
+   * @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 OutOfMemoryRuntimeException if buffer cannot be allocated
+   * @throws OutOfMemoryException
+   *           if buffer cannot be allocated
    */
-  public DrillBuf buffer(int minSize, int maxSize);
+  public DrillBuf buffer(int size, BufferManager manager);
 
   /**
    * Returns the allocator this allocator falls back to when it needs more memory.
    *
    * @return the underlying allocator used by this allocator
    */
-  public ByteBufAllocator getUnderlyingAllocator();
-
-  /**
-   * Create a child allocator nested below this one.
-   *
-   * @param context - the owner or this 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
-   */
-  @Deprecated
-  public BufferAllocator getChildAllocator(FragmentContext context, long initialReservation,
-      long maximumReservation, boolean applyFragmentLimit);
-
-  /**
-   * Flag: this allocator is a limiting sub-tree root, meaning that the maxAllocation for
-   * it applies to all its descendant child allocators. In low memory situations, the limits
-   * for sub-tree roots may be adjusted down so that they evenly share the total amount of
-   * direct memory across all the sub-tree roots.
-   */
-  public final static int F_LIMITING_ROOT = 0x0001;
+  public ByteBufAllocator getAsByteBufAllocator();
 
   /**
    * Create a new child allocator.
    *
-   * @param allocatorOwner the allocator owner
-   * @param initReservation the initial space reservation (obtained from this allocator)
-   * @param maxAllocation maximum amount of space the new allocator can allocate
-   * @param flags one or more of BufferAllocator.F_* flags
+   * @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 BufferAllocator newChildAllocator(AllocatorOwner allocatorOwner,
-      long initReservation, long maxAllocation, int flags);
+  public BufferAllocator newChildAllocator(String name, long initReservation, long maxAllocation);
 
   /**
-   * Take over ownership of the given buffer, adjusting accounting accordingly.
-   * This allocator always takes over ownership.
+   * Close and release all buffers generated from this buffer pool.
    *
-   * @param buf the buffer to take over
-   * @return false if over allocation
+   * <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);
+  @Override
+  public void close();
 
   /**
-   * Share ownership of a buffer between allocators.
+   * Returns the amount of memory currently allocated from this allocator.
    *
-   * @param buf the buffer
-   * @param bufOut a new DrillBuf owned by this allocator, but sharing the same underlying buffer
-   * @return false if over allocation.
+   * @return the amount of memory currently allocated
    */
-  public boolean shareOwnership(DrillBuf buf, Pointer<DrillBuf> bufOut);
+  public long getAllocatedMemory();
 
   /**
-   * Not thread safe.
-   *
-   * WARNING: unclaimed pre-allocations leak memory. If you call preAllocate(), you must
-   * make sure to ultimately try to get the buffer and release it.
+   * Set the maximum amount of memory this allocator is allowed to allocate.
    *
-   * For Child allocators to set their Fragment limits.
-   *
-   * @param fragmentLimit the new fragment limit
+   * @param newLimit
+   *          The new Limit to apply to allocations
    */
-  @Deprecated // happens automatically, and via allocation policies
-  public void setFragmentLimit(long fragmentLimit);
+  public void setLimit(long newLimit);
 
   /**
-   * Returns the current fragment limit.
-   *
-   * @return the current fragment limit
-   */
-  /*
-   * TODO should be replaced with something more general because of
-   * the availability of multiple allocation policies
+   * Return the current maximum limit this allocator imposes.
    *
-   * TODO We should also have a getRemainingMemory() so operators
-   * can query how much more is left to allocate. That could be
-   * tricky.
+   * @return Limit in number of bytes.
    */
-  @Deprecated
-  public long getFragmentLimit();
+  public long getLimit();
 
   /**
-   * 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>
+   * Returns the peak amount of memory allocated from this allocator.
    *
-   * @return the allocator's id
+   * @return the peak amount of memory allocated
    */
-  public int getId();
+  public long getPeakMemoryAllocation();
 
   /**
-   * Close and release all buffers generated from this buffer pool.
+   * 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}.
    *
-   * <p>When assertions are on, complains if there are any outstanding buffers; to avoid
-   * that, release all buffers before the allocator is closed.
+   * @return the newly created reservation
    */
-  @Override
-  public void close() throws Exception;
+  public AllocationReservation newReservation();
 
   /**
-   * Returns the amount of memory currently allocated from this allocator.
-   *
-   * @return the amount of memory currently allocated
+   * 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 long getAllocatedMemory();
+  public DrillBuf getEmpty();
 
   /**
-   * Returns the peak amount of memory allocated from this allocator.
-   *
-   * @return the peak amount of memory allocated
+   * 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 long getPeakMemoryAllocation();
+  public String getName();
 
   /**
-   * Returns an empty DrillBuf.
-   *
-   * @return an empty DrillBuf
+   * 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.
    */
-  public DrillBuf getEmpty();
+  public boolean isOverLimit();
 
   /**
-   * 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 a verbose string describing this allocator. If in DEBUG mode, this will also include relevant stacktraces
+   * and historical logs for underlying objects
    *
-   * @return the newly created reservation
+   * @return A very verbose description of the allocator hierarchy.
    */
-  public AllocationReservation newReservation();
+  public String toVerboseString();
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/ChildAllocator.java
----------------------------------------------------------------------
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 0000000..8fcabb1
--- /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);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/DrillByteBufAllocator.java
----------------------------------------------------------------------
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 0000000..ec423e2
--- /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.");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/LimitConsumer.java
----------------------------------------------------------------------
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/LimitConsumer.java
deleted file mode 100644
index 777d9d2..0000000
--- a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/LimitConsumer.java
+++ /dev/null
@@ -1,28 +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;
-
-public interface LimitConsumer {
-
-  public String getIdentifier();
-  public long getAllocated();
-
-  public long getLimit();
-
-  public void setLimit(long limit);
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/README.md
----------------------------------------------------------------------
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 0000000..cbb8d96
--- /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 implementation 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 is closed.
+  - `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. 
+
+### 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

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/RootAllocator.java
----------------------------------------------------------------------
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 0000000..5ab4130
--- /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 limit) {
+    super(null, "ROOT", 0, limit);
+  }
+
+  /**
+   * Verify the accounting state of the allocation system.
+   */
+  @VisibleForTesting
+  public void verify() {
+    verifyAllocator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/java/org/apache/drill/exec/memory/package-info.java
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/main/java/org/apache/drill/exec/memory/package-info.java b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/package-info.java
new file mode 100644
index 0000000..3c1b9e5
--- /dev/null
+++ b/exec/memory/base/src/main/java/org/apache/drill/exec/memory/package-info.java
@@ -0,0 +1,24 @@
+/**
+ * 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, 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;

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/main/resources/drill-module.conf b/exec/memory/base/src/main/resources/drill-module.conf
new file mode 100644
index 0000000..593ef8e
--- /dev/null
+++ b/exec/memory/base/src/main/resources/drill-module.conf
@@ -0,0 +1,25 @@
+// 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.
+
+//  This file tells Drill to consider this module when class path scanning.
+//  This file can also include any supplementary configuration information.
+//  This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
+drill: {
+  memory: {
+    debug.error_on_leak: true,
+    top.max: 1000000000000
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestAccountant.java
----------------------------------------------------------------------
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 0000000..31c733f
--- /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();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestBaseAllocator.java
----------------------------------------------------------------------
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 0000000..780d217
--- /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(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(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(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(MAX_ALLOCATION)) {
+      @SuppressWarnings("unused")
+      final DrillBuf drillBuf = rootAllocator.buffer(0);
+    }
+  }
+
+  @Test
+  public void testAllocator_transferOwnership() throws Exception {
+    try(final RootAllocator rootAllocator =
+        new RootAllocator(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(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(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(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(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(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(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(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(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(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(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(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(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(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(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.allocateBuffer();
+          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(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();
+
+    }
+  }
+}


[08/13] drill git commit: DRILL-4134: Allocator Improvements

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestEndianess.java
----------------------------------------------------------------------
diff --git a/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestEndianess.java b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestEndianess.java
new file mode 100644
index 0000000..b312301
--- /dev/null
+++ b/exec/memory/base/src/test/java/org/apache/drill/exec/memory/TestEndianess.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.memory;
+
+import static org.junit.Assert.assertEquals;
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.common.DrillAutoCloseables;
+import org.apache.drill.common.config.DrillConfig;
+import org.junit.Test;
+
+
+public class TestEndianess {
+
+  @Test
+  public void testLittleEndian() {
+    final BufferAllocator a = new RootAllocator(DrillConfig.getMaxDirectMemory());
+    final ByteBuf b = a.buffer(4);
+    b.setInt(0, 35);
+    assertEquals(b.getByte(0), 35);
+    assertEquals(b.getByte(1), 0);
+    assertEquals(b.getByte(2), 0);
+    assertEquals(b.getByte(3), 0);
+    b.release();
+    DrillAutoCloseables.closeNoChecked(a);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/pom.xml
----------------------------------------------------------------------
diff --git a/exec/memory/impl/pom.xml b/exec/memory/impl/pom.xml
deleted file mode 100644
index 94b9052..0000000
--- 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.5.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>

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AccountorImpl.java
----------------------------------------------------------------------
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 0ac93e4..0000000
--- 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;
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java
deleted file mode 100644
index 4f1a1bd..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicy.java
+++ /dev/null
@@ -1,38 +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;
-
-/**
- * Implicitly specifies an allocation policy by providing a factory method to
- * create an enforcement agent.
- *
- * <p>Allocation policies are meant to be global, and may not work properly if
- * different allocators are given different policies. These are designed to
- * be supplied to the root-most allocator only, and then shared with descendant
- * (child) allocators.</p>
- */
-public interface AllocationPolicy {
-  /**
-   * Create an allocation policy enforcement agent. Each newly created allocator should
-   * call this in order to obtain its own agent.
-   *
-   * @return the newly instantiated agent; if an agent's implementation is stateless,
-   *   this may return a sharable singleton
-   */
-  AllocationPolicyAgent newAgent();
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java
deleted file mode 100644
index ad51ee6..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationPolicyAgent.java
+++ /dev/null
@@ -1,69 +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;
-
-/**
- * Per-allocator enforcement agent for allocation policies; created by
- * {@link AllocationPolicy#newAgent()}.
- */
-public interface AllocationPolicyAgent extends AutoCloseable {
-  /**
-   * Checks to see if creating a new allocator using the given specifications
-   * is allowed; should throw an exception if not.
-   *
-   * @param parentAllocator the parent allocator
-   * @param initReservation initial reservation the allocator should have
-   * @param maxAllocation the maximum allocation the allocator will allow
-   * @param flags the allocation option flags
-   * @throws OutOfMemoryException if the new allocator shouldn't be created
-   */
-  void checkNewAllocator(BufferAllocator parentAllocator,
-      long initReservation, long maxAllocation, int flags);
-
-  /**
-   * Get the currently applicable memory limit for the provided allocator.
-   * The interpretation of this value varies with the allocation policy in
-   * use, and each policy should describe what to expect.
-   *
-   * @param bufferAllocator the allocator
-   * @return the memory limit
-   */
-  long getMemoryLimit(BufferAllocator bufferAllocator);
-
-  /**
-   * Initialize the agent for a newly created allocator. Should be called from
-   * the allocator's constructor to initialize the agent for the allocator.
-   *
-   * @param bufferAllocator the newly created allocator.
-   */
-  void initializeAllocator(BufferAllocator bufferAllocator);
-
-  /**
-   * Indicate if any available memory owned by this allocator should
-   * be released to its parent. Allocators may use this to limit the
-   * amount of unused memory they retain for future requests; agents may
-   * request that memory be returned if there is currently a high demand
-   * for memory that other allocators could use if this allocator
-   * doesn't need it.
-   *
-   * @param bufferAllocator
-   * @return true if available memory owned by this allocator should be given
-   *   back to its parent
-   */
-  boolean shouldReleaseToParent(BufferAllocator bufferAllocator);
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java
deleted file mode 100644
index 1803572..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocationReservation.java
+++ /dev/null
@@ -1,152 +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 com.google.common.base.Preconditions;
-
-import io.netty.buffer.DrillBuf;
-
-/**
- * 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.
- */
-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.
-   */
-  protected 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");
-
-    if (!reserve(nBytes)) {
-      return false;
-    }
-
-    this.nBytes += nBytes;
-    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
-   */
-  protected 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
-   */
-  protected 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
-   */
-  protected 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;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java
deleted file mode 100644
index 8bf2a99..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorClosedException.java
+++ /dev/null
@@ -1,31 +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;
-
-/**
- * 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);
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java
deleted file mode 100644
index f2d3df9..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorOwner.java
+++ /dev/null
@@ -1,40 +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.exec.ops.FragmentContext;
-import org.apache.drill.exec.testing.ExecutionControls;
-
-/**
- * This interface provides a means for allocator owners to inject services
- * required by allocators, as well as to identify themselves for debugging purposes.
- * Identification is done by overriding the implementation of
- * {#link {@link Object#toString()}.
- */
-public interface AllocatorOwner {
-  /**
-   * Get the current ExecutionControls from the allocator's owner.
-   *
-   * @return the current execution controls; may return null if this isn't
-   *   possible
-   */
-  ExecutionControls getExecutionControls();
-
-  @Deprecated // Only for TopLevelAllocator and its friends.
-  FragmentContext getFragmentContext();
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java
----------------------------------------------------------------------
diff --git a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java b/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java
deleted file mode 100644
index 00d8c4f..0000000
--- a/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AllocatorsStatsMXBean.java
+++ /dev/null
@@ -1,34 +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;
-
-/**
- * JMX bean interface for global allocator statistics.
- */
-// TODO use Stats infrastructure instead of JMX beans
-public interface AllocatorsStatsMXBean {
-  /**
-   * Get the maximum amount of direct memory that can be used.
-   *
-   * <p>This is determined by what is available, or by the drillbit
-   * configuration, if it specifies a value.</p>
-   *
-   * @return the amount of direct memory that can be used
-   */
-  public long getMaxDirectMemory();
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/809f4620/exec/memory/impl/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java
----------------------------------------------------------------------
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 0f6b8b0..0000000
--- 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.";
-}