You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by he...@apache.org on 2017/08/13 21:35:28 UTC

incubator-impala git commit: IMPALA-5666: ASAN poisoning for MemPool and BufferPool

Repository: incubator-impala
Updated Branches:
  refs/heads/master 6c1254656 -> a99114283


IMPALA-5666: ASAN poisoning for MemPool and BufferPool

"If you poison us, do we not die?"

* Use ASAN_[UN]POISON_MEMORY_REGION to indicate to ASAN runtime that
  memory is not 'valid' from Impala's perspective (but still valid from
  kernel's perspective).

* Add this to MemPool and BufferPoolAllocator. For both, memory goes
  through the following lifecycle: when it is allocated from the OS and
  returned to the user, it must be unpoisoned. When the user returns it
  to a cache, it becomes poisoned. When the cache is freed to the OS, it
  must be unpoisoned again (otherwise future allocations elsewhere in
  the system might return poisoned memory).

* Also add checks to FreePool which uses a MemPool underneath, but has
  its own freelist cache. Fix a couple of bugs in free-pool-test that
  these checks uncovered.

Testing:

* Tests that only run if ASAN is enabled to confirm that poisoning
  catches simple use-after-return errors. These are 'death' tests which
  catch expected process exits.

Change-Id: I8d5a28dfee2b7c631981aac75524bde9acc0b36a
Reviewed-on: http://gerrit.cloudera.org:8080/7591
Tested-by: Impala Public Jenkins
Reviewed-by: Henry Robinson <he...@cloudera.com>


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

Branch: refs/heads/master
Commit: a99114283b371852254fe05eb24ac0e339cf777b
Parents: 6c12546
Author: Henry Robinson <he...@cloudera.com>
Authored: Tue Aug 1 19:55:41 2017 -0700
Committer: Henry Robinson <he...@cloudera.com>
Committed: Sun Aug 13 17:01:02 2017 +0000

----------------------------------------------------------------------
 be/CMakeLists.txt                               |  4 +
 .../runtime/bufferpool/buffer-allocator-test.cc | 17 ++++
 be/src/runtime/bufferpool/buffer-allocator.cc   |  4 +
 be/src/runtime/bufferpool/buffer-pool.h         |  8 ++
 be/src/runtime/bufferpool/free-list.h           |  1 +
 be/src/runtime/free-pool-test.cc                | 46 +++++++++-
 be/src/runtime/free-pool.h                      | 96 +++++++++++++++-----
 be/src/runtime/mem-pool-test.cc                 | 27 ++++++
 be/src/runtime/mem-pool.cc                      | 14 +--
 be/src/runtime/mem-pool.h                       |  3 +
 10 files changed, 189 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a9911428/be/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index ae84528..b0922b0 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -194,6 +194,10 @@ set(CLANG_IR_CXX_FLAGS "-emit-llvm" "-c" "-std=c++14" "-DIR_COMPILE" "-DNDEBUG"
 # -Werror: compile warnings should be errors when using the toolchain compiler.
 set(CLANG_IR_CXX_FLAGS "${CLANG_IR_CXX_FLAGS}" "-Werror")
 
+if ("${CMAKE_BUILD_TYPE}" STREQUAL "ADDRESS_SANITIZER")
+  SET(CLANG_IR_CXX_FLAGS "${CLANG_IR_CXX_FLAGS}" "-DADDRESS_SANITIZER")
+endif()
+
 IF($ENV{ENABLE_IMPALA_IR_DEBUG_INFO} STREQUAL "true")
   # -g: emit debug symbols in IR. These increase IR size and memory overhead of LLVM, but
   #     are useful for debugging codegened code and interpreting codegen disassembly

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a9911428/be/src/runtime/bufferpool/buffer-allocator-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/buffer-allocator-test.cc b/be/src/runtime/bufferpool/buffer-allocator-test.cc
index fa7ac10..7ec64ad 100644
--- a/be/src/runtime/bufferpool/buffer-allocator-test.cc
+++ b/be/src/runtime/bufferpool/buffer-allocator-test.cc
@@ -68,6 +68,23 @@ class BufferAllocatorTest : public ::testing::Test {
   ReservationTracker dummy_reservation_;
 };
 
+#ifdef ADDRESS_SANITIZER
+
+// Confirm that ASAN will catch use-after-free errors, even if the BufferAllocator caches
+// returned memory.
+TEST_F(BufferAllocatorTest, Poisoning) {
+  BufferAllocator allocator(dummy_pool_, TEST_BUFFER_LEN, 2 * TEST_BUFFER_LEN);
+  BufferHandle buffer;
+  ASSERT_OK(allocator.Allocate(&dummy_client_, TEST_BUFFER_LEN, &buffer));
+  uint8_t* data = buffer.data();
+  allocator.Free(move(buffer));
+
+  // Should trigger a ASAN failure.
+  ASSERT_DEATH({data[10] = 0;}, "use-after-poison");
+}
+
+#endif
+
 // Functional test that makes sure the free lists cache as many buffers as expected.
 TEST_F(BufferAllocatorTest, FreeListSizes) {
   // Run on core 0 to ensure that we always go to the same free list.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a9911428/be/src/runtime/bufferpool/buffer-allocator.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/buffer-allocator.cc b/be/src/runtime/bufferpool/buffer-allocator.cc
index a2e108a..0f05cf3 100644
--- a/be/src/runtime/bufferpool/buffer-allocator.cc
+++ b/be/src/runtime/bufferpool/buffer-allocator.cc
@@ -351,6 +351,7 @@ void BufferPool::BufferAllocator::Free(BufferHandle&& handle) {
   DCHECK(handle.is_open());
   handle.client_ = nullptr; // Buffer is no longer associated with a client.
   FreeBufferArena* arena = per_core_arenas_[handle.home_core_].get();
+  handle.Poison();
   arena->AddFreeBuffer(move(handle));
 }
 
@@ -398,6 +399,8 @@ int64_t BufferPool::BufferAllocator::FreeToSystem(vector<BufferHandle>&& buffers
   int64_t bytes_freed = 0;
   for (BufferHandle& buffer : buffers) {
     bytes_freed += buffer.len();
+    // Ensure that the memory is unpoisoned when it's next allocated by the system.
+    buffer.Unpoison();
     system_allocator_->Free(move(buffer));
   }
   return bytes_freed;
@@ -497,6 +500,7 @@ bool BufferPool::FreeBufferArena::PopFreeBuffer(
   FreeList* list = &lists->free_buffers;
   DCHECK_EQ(lists->num_free_buffers.Load(), list->Size());
   if (!list->PopFreeBuffer(buffer)) return false;
+  buffer->Unpoison();
   lists->num_free_buffers.Add(-1);
   lists->low_water_mark = min<int>(lists->low_water_mark, list->Size());
   return true;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a9911428/be/src/runtime/bufferpool/buffer-pool.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/buffer-pool.h b/be/src/runtime/bufferpool/buffer-pool.h
index dab875c..4798d6c 100644
--- a/be/src/runtime/bufferpool/buffer-pool.h
+++ b/be/src/runtime/bufferpool/buffer-pool.h
@@ -420,6 +420,14 @@ class BufferPool::BufferHandle {
 
   std::string DebugString() const;
 
+  /// Poison the memory associated with this handle. If ASAN is not enabled, this is a
+  /// no-op.
+  void Poison() { ASAN_POISON_MEMORY_REGION(data(), len()); }
+
+  /// Unpoison the memory associated with this handle. If ASAN is not enabled, this is a
+  /// no-op.
+  void Unpoison() { ASAN_UNPOISON_MEMORY_REGION(data(), len()); }
+
  private:
   DISALLOW_COPY_AND_ASSIGN(BufferHandle);
   friend class BufferPool;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a9911428/be/src/runtime/bufferpool/free-list.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/free-list.h b/be/src/runtime/bufferpool/free-list.h
index ec1057d..7d59649 100644
--- a/be/src/runtime/bufferpool/free-list.h
+++ b/be/src/runtime/bufferpool/free-list.h
@@ -70,6 +70,7 @@ class FreeList {
 
   /// Adds a free buffer to the list.
   void AddFreeBuffer(BufferHandle&& buffer) {
+    buffer.Poison();
     free_list_.emplace_back(std::move(buffer));
     std::push_heap(free_list_.begin(), free_list_.end(), HeapCompare);
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a9911428/be/src/runtime/free-pool-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/free-pool-test.cc b/be/src/runtime/free-pool-test.cc
index 07b676f..bfbce20 100644
--- a/be/src/runtime/free-pool-test.cc
+++ b/be/src/runtime/free-pool-test.cc
@@ -67,14 +67,14 @@ TEST(FreePoolTest, Basic) {
   // We know have 2 1 byte allocations, which were rounded up to 8 bytes. Make an 8
   // byte allocation, which can reuse one of the existing ones.
   uint8_t* p4 = pool.Allocate(2);
-  memset(p4, 2, 123);
+  memset(p4, 123, 2);
   EXPECT_EQ(mem_pool.total_allocated_bytes(), 32);
   EXPECT_TRUE(p4 == p1 || p4 == p2 || p4 == p3);
   pool.Free(p4);
 
   // Make a 9 byte allocation, which requires a new allocation.
   uint8_t* p5 = pool.Allocate(9);
-  memset(p5, 9, 123);
+  memset(p5, 123, 9);
   EXPECT_EQ(mem_pool.total_allocated_bytes(), 56);
   pool.Free(p5);
   EXPECT_TRUE(p5 != p1);
@@ -114,6 +114,48 @@ TEST(FreePoolTest, Basic) {
   mem_pool.FreeAll();
 }
 
+#ifdef ADDRESS_SANITIZER
+
+// The following tests confirm that ASAN catches invalid memory accesses thanks to the
+// FreePool manually (un)poisoning memory.
+TEST(FreePoolTest, OutOfBoundsAccess) {
+  MemTracker tracker;
+  MemPool mem_pool(&tracker);
+  FreePool pool(&mem_pool);
+
+  uint8_t* ptr = pool.Allocate(5);
+  ptr[4] = 'O';
+  ASSERT_DEATH({ptr[5] = 'X';}, "use-after-poison");
+  mem_pool.FreeAll();
+}
+
+TEST(FreePoolTest, UseAfterFree) {
+  MemTracker tracker;
+  MemPool mem_pool(&tracker);
+  FreePool pool(&mem_pool);
+
+  uint8_t* ptr = pool.Allocate(5);
+  ptr[4] = 'O';
+  pool.Free(ptr);
+  ASSERT_DEATH({ptr[0] = 'X';}, "use-after-poison");
+  mem_pool.FreeAll();
+}
+
+TEST(FreePoolTest, ReallocPoison) {
+  MemTracker tracker;
+  MemPool mem_pool(&tracker);
+  FreePool pool(&mem_pool);
+
+  uint8_t* ptr = pool.Allocate(32);
+  ptr[30] = 'O';
+  ptr = pool.Reallocate(ptr, 16);
+  ASSERT_DEATH({ptr[30] = 'X';}, "use-after-poison");
+  mem_pool.FreeAll();
+}
+
+#endif
+
+
 // In this test we make two allocations at increasing sizes and then we
 // free both to prime the pool. Then, for a few iterations, we make the same allocations
 // as we did to prime the pool in a random order. We validate that the returned

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a9911428/be/src/runtime/free-pool.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/free-pool.h b/be/src/runtime/free-pool.h
index 55774e8..f4a3e4b 100644
--- a/be/src/runtime/free-pool.h
+++ b/be/src/runtime/free-pool.h
@@ -23,9 +23,11 @@
 #include <string.h>
 #include <string>
 #include <sstream>
+#include <unordered_map>
 
 #include "common/atomic.h"
 #include "common/logging.h"
+#include "gutil/dynamic_annotations.h"
 #include "runtime/mem-pool.h"
 #include "util/bit-util.h"
 
@@ -58,38 +60,48 @@ class FreePool {
   }
 
   /// Allocates a buffer of size between [0, 2^62 - 1 - sizeof(FreeListNode)] bytes.
-  uint8_t* Allocate(int64_t size) {
-    DCHECK_GE(size, 0);
+  uint8_t* Allocate(const int64_t requested_size) {
+    DCHECK_GE(requested_size, 0);
 #ifndef NDEBUG
     if (FLAGS_stress_free_pool_alloc > 0 &&
         (alloc_counts_.Add(1) % FLAGS_stress_free_pool_alloc) == 0) {
-      return NULL;
+      return nullptr;
     }
 #endif
-    /// Return a non-NULL dummy pointer. NULL is reserved for failures.
-    if (UNLIKELY(size == 0)) return mem_pool_->EmptyAllocPtr();
+    /// Return a non-nullptr dummy pointer. nullptr is reserved for failures.
+    if (UNLIKELY(requested_size == 0)) return mem_pool_->EmptyAllocPtr();
     ++net_allocations_;
-    if (FLAGS_disable_mem_pools) return reinterpret_cast<uint8_t*>(malloc(size));
+    if (FLAGS_disable_mem_pools) {
+      return reinterpret_cast<uint8_t*>(malloc(requested_size));
+    }
     /// MemPool allocations are 8-byte aligned, so making allocations < 8 bytes
     /// doesn't save memory and eliminates opportunities to recycle allocations.
-    size = std::max<int64_t>(8, size);
-    int free_list_idx = BitUtil::Log2Ceiling64(size);
+    int64_t actual_size = std::max<int64_t>(8, requested_size);
+    int free_list_idx = BitUtil::Log2Ceiling64(actual_size);
     DCHECK_LT(free_list_idx, NUM_LISTS);
     FreeListNode* allocation = lists_[free_list_idx].next;
-    if (allocation == NULL) {
+    if (allocation == nullptr) {
       // There wasn't an existing allocation of the right size, allocate a new one.
-      size = 1LL << free_list_idx;
+      actual_size = 1LL << free_list_idx;
       allocation = reinterpret_cast<FreeListNode*>(
-          mem_pool_->Allocate(size + sizeof(FreeListNode)));
-      if (UNLIKELY(allocation == NULL)) {
+          mem_pool_->Allocate(actual_size + sizeof(FreeListNode)));
+      if (UNLIKELY(allocation == nullptr)) {
         --net_allocations_;
-        return NULL;
+        return nullptr;
       }
+      // Memory will be returned unpoisoned from MemPool. Poison the whole range and then
+      // deal with unpoisoning both allocation paths in one place below.
+      ASAN_POISON_MEMORY_REGION(allocation, sizeof(FreeListNode) + actual_size);
     } else {
       // Remove this allocation from the list.
       lists_[free_list_idx].next = allocation->next;
     }
-    DCHECK(allocation != NULL);
+    DCHECK(allocation != nullptr);
+#ifdef ADDRESS_SANITIZER
+    uint8_t* ptr = reinterpret_cast<uint8_t*>(allocation);
+    ASAN_UNPOISON_MEMORY_REGION(ptr, sizeof(FreeListNode) + requested_size);
+    alloc_to_size_[ptr + sizeof(FreeListNode)] = requested_size;
+#endif
     // Set the back node to point back to the list it came from so know where
     // to add it on Free().
     allocation->list = &lists_[free_list_idx];
@@ -97,7 +109,7 @@ class FreePool {
   }
 
   void Free(uint8_t* ptr) {
-    if (UNLIKELY(ptr == NULL || ptr == mem_pool_->EmptyAllocPtr())) return;
+    if (UNLIKELY(ptr == nullptr || ptr == mem_pool_->EmptyAllocPtr())) return;
     --net_allocations_;
     if (FLAGS_disable_mem_pools) {
       free(ptr);
@@ -111,22 +123,30 @@ class FreePool {
     // Add node to front of list.
     node->next = list->next;
     list->next = node;
+
+#ifdef ADDRESS_SANITIZER
+    int bucket_idx = (list - &lists_[0]);
+    DCHECK_LT(bucket_idx, NUM_LISTS);
+    int64_t allocation_size = 1LL << bucket_idx;
+    ASAN_POISON_MEMORY_REGION(ptr, allocation_size);
+    alloc_to_size_.erase(ptr);
+#endif
   }
 
   /// Returns an allocation that is at least 'size'. If the current allocation backing
   /// 'ptr' is big enough, 'ptr' is returned. Otherwise a new one is made and the contents
   /// of ptr are copied into it.
   ///
-  /// NULL will be returned on allocation failure. It's the caller's responsibility to
+  /// nullptr will be returned on allocation failure. It's the caller's responsibility to
   /// free the memory buffer pointed to by "ptr" in this case.
   uint8_t* Reallocate(uint8_t* ptr, int64_t size) {
 #ifndef NDEBUG
     if (FLAGS_stress_free_pool_alloc > 0 &&
         (alloc_counts_.Add(1) % FLAGS_stress_free_pool_alloc) == 0) {
-      return NULL;
+      return nullptr;
     }
 #endif
-    if (UNLIKELY(ptr == NULL || ptr == mem_pool_->EmptyAllocPtr())) return Allocate(size);
+    if (UNLIKELY(ptr == nullptr || ptr == mem_pool_->EmptyAllocPtr())) return Allocate(size);
     if (FLAGS_disable_mem_pools) {
       return reinterpret_cast<uint8_t*>(realloc(reinterpret_cast<void*>(ptr), size));
     }
@@ -141,12 +161,34 @@ class FreePool {
     int64_t allocation_size = 1LL << bucket_idx;
 
     // If it's already big enough, just return the ptr.
-    if (allocation_size >= size) return ptr;
+    if (allocation_size >= size) {
+      // Ensure that only first size bytes are unpoisoned. Need to poison whole region
+      // first in case size is smaller than original allocation's size.
+#ifdef ADDRESS_SANITIZER
+      DCHECK(alloc_to_size_.find(ptr) != alloc_to_size_.end());
+      int64_t prev_allocation_size = alloc_to_size_[ptr];
+      if (prev_allocation_size > size) {
+        // Allocation is shrinking: poison the 'freed' bytes.
+        ASAN_POISON_MEMORY_REGION(ptr + size, prev_allocation_size - size);
+      } else {
+        // Allocation is growing: unpoison the newly allocated bytes.
+        ASAN_UNPOISON_MEMORY_REGION(
+            ptr + prev_allocation_size, size - prev_allocation_size);
+      }
+      alloc_to_size_[ptr] = size;
+#endif
+      return ptr;
+    }
 
     // Make a new one. Since Allocate() already rounds up to powers of 2, this effectively
     // doubles for the caller.
     uint8_t* new_ptr = Allocate(size);
-    if (LIKELY(new_ptr != NULL)) {
+    if (LIKELY(new_ptr != nullptr)) {
+#ifdef ADDRESS_SANITIZER
+      DCHECK(alloc_to_size_.find(ptr) != alloc_to_size_.end());
+      // Unpoison the region so that we can copy the old allocation to the new one.
+      ASAN_UNPOISON_MEMORY_REGION(ptr, allocation_size);
+#endif
       memcpy(new_ptr, ptr, allocation_size);
       Free(ptr);
     }
@@ -184,14 +226,14 @@ class FreePool {
     ss << "FreePool: " << this << std::endl;
     for (int i = 0; i < NUM_LISTS; ++i) {
       FreeListNode* n = lists_[i].next;
-      if (n == NULL) continue;
+      if (n == nullptr) continue;
       ss << i << ": ";
-      while (n != NULL) {
+      while (n != nullptr) {
         uint8_t* ptr = reinterpret_cast<uint8_t*>(n);
         ptr += sizeof(FreeListNode);
         ss << reinterpret_cast<void*>(ptr);
         n = n->next;
-        if (n != NULL) ss << "->";
+        if (n != nullptr) ss << "->";
       }
       ss << std::endl;
     }
@@ -206,6 +248,14 @@ class FreePool {
   /// allocations, it makes the indexing easy.
   FreeListNode lists_[NUM_LISTS];
 
+#ifdef ADDRESS_SANITIZER
+  // For ASAN only: keep track of used bytes for each allocation (not including
+  // FreeListNode header for each chunk). This allows us to poison each byte in a chunk
+  // exactly once when reallocating from an existing chunk, rather than having to poison
+  // the entire chunk each time.
+  std::unordered_map<uint8_t*, int64_t> alloc_to_size_;
+#endif
+
   /// Diagnostic counter that tracks (# Allocates - # Frees)
   int64_t net_allocations_;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a9911428/be/src/runtime/mem-pool-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-pool-test.cc b/be/src/runtime/mem-pool-test.cc
index a39a120..4f537ab 100644
--- a/be/src/runtime/mem-pool-test.cc
+++ b/be/src/runtime/mem-pool-test.cc
@@ -175,6 +175,33 @@ TEST(MemPoolTest, Keep) {
   p2.FreeAll();
 }
 
+#ifdef ADDRESS_SANITIZER
+
+/// These tests confirm that ASAN will catch use-after-return errors (even though the
+/// memory is still valid, just cached by the MemPool).
+TEST(MemPoolTest, UseAfterClear) {
+  MemTracker tracker;
+  MemPool p(&tracker);
+  uint8_t* ptr = p.Allocate(1024);
+  ptr[10] = 'A';
+  p.Clear();
+  ASSERT_DEATH({ptr[10] = 'B';}, "use-after-poison");
+  p.FreeAll();
+}
+
+TEST(MemPoolTest, UseAfterPartialReturn) {
+  MemTracker tracker;
+  MemPool p(&tracker);
+  uint8_t* ptr = p.Allocate(1024);
+  ptr[10] = 'A';
+  p.ReturnPartialAllocation(512);
+  ptr[511] = 'B';
+  ASSERT_DEATH({ptr[512] = 'X';}, "use-after-poison");
+  p.FreeAll();
+}
+
+#endif
+
 // Tests that we can return partial allocations.
 TEST(MemPoolTest, ReturnPartial) {
   MemTracker tracker;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a9911428/be/src/runtime/mem-pool.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-pool.cc b/be/src/runtime/mem-pool.cc
index 0ca3822..1b38d37 100644
--- a/be/src/runtime/mem-pool.cc
+++ b/be/src/runtime/mem-pool.cc
@@ -75,9 +75,9 @@ MemPool::~MemPool() {
 
 void MemPool::Clear() {
   current_chunk_idx_ = -1;
-  for (std::vector<ChunkInfo>::iterator chunk = chunks_.begin();
-       chunk != chunks_.end(); ++chunk) {
-    chunk->allocated_bytes = 0;
+  for (auto& chunk: chunks_) {
+    chunk.allocated_bytes = 0;
+    ASAN_POISON_MEMORY_REGION(chunk.data, chunk.size);
   }
   total_allocated_bytes_ = 0;
   DCHECK(CheckIntegrity(false));
@@ -85,9 +85,9 @@ void MemPool::Clear() {
 
 void MemPool::FreeAll() {
   int64_t total_bytes_released = 0;
-  for (size_t i = 0; i < chunks_.size(); ++i) {
-    total_bytes_released += chunks_[i].size;
-    free(chunks_[i].data);
+  for (auto& chunk: chunks_) {
+    total_bytes_released += chunk.size;
+    free(chunk.data);
   }
   chunks_.clear();
   next_chunk_size_ = INITIAL_CHUNK_SIZE;
@@ -152,6 +152,8 @@ bool MemPool::FindChunk(int64_t min_size, bool check_limits) noexcept {
     return false;
   }
 
+  ASAN_POISON_MEMORY_REGION(buf, chunk_size);
+
   // Put it before the first free chunk. If no free chunks, it goes at the end.
   if (first_free_idx == static_cast<int>(chunks_.size())) {
     chunks_.push_back(ChunkInfo(chunk_size, buf));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/a9911428/be/src/runtime/mem-pool.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-pool.h b/be/src/runtime/mem-pool.h
index 234fc41..f0393b5 100644
--- a/be/src/runtime/mem-pool.h
+++ b/be/src/runtime/mem-pool.h
@@ -27,6 +27,7 @@
 #include <vector>
 
 #include "common/logging.h"
+#include "gutil/dynamic_annotations.h"
 #include "util/bit-util.h"
 
 namespace impala {
@@ -128,6 +129,7 @@ class MemPool {
     ChunkInfo& info = chunks_[current_chunk_idx_];
     DCHECK_GE(info.allocated_bytes, byte_size);
     info.allocated_bytes -= byte_size;
+    ASAN_POISON_MEMORY_REGION(info.data + info.allocated_bytes, byte_size);
     total_allocated_bytes_ -= byte_size;
   }
 
@@ -266,6 +268,7 @@ class MemPool {
 
     ChunkInfo& info = chunks_[current_chunk_idx_];
     uint8_t* result = info.data + info.allocated_bytes;
+    ASAN_UNPOISON_MEMORY_REGION(result, size);
     DCHECK_LE(info.allocated_bytes + size, info.size);
     info.allocated_bytes += size;
     total_allocated_bytes_ += size;