You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by zy...@apache.org on 2013/02/03 11:38:37 UTC

[1/5] git commit: TS-1006: Introduce a reclaimable InkFreeList

TS-1006: Introduce a reclaimable InkFreeList

Here are some advantages of this new memory pool:
* Introduce per-thread and per-Class cache(pool), all
  operations are lockfree at runtime.
* Memory size in cache can shrink automatically.
* In the same thread, all Class with similar size will share
  the same cache.
* Use wait-free queue(InkAtomicList) for synchronization.
  In a producer-consumer alloc-free scenario, it won’t lead to
  thread cache of producer starve.

This memory pool is disable by default. We can enable it by configuring
TS as following:
$ ./configure --enable-reclaimable-freelist

NOTE: '--enable-reclaimable-freelist' option will be ignored if
      '--disable-freelist' option is given.

Signed-off-by: Yunkai Zhang <qi...@taobao.com>
Signed-off-by: Zhao Yongming <mi...@gmail.com>


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

Branch: refs/heads/master
Commit: 795e50df6021b3fea54f83ebe5b64de4a8e589fd
Parents: 4397abf
Author: Yunkai Zhang <qi...@taobao.com>
Authored: Sun Nov 18 01:44:19 2012 +0800
Committer: Zhao Yongming <mi...@gmail.com>
Committed: Sun Feb 3 10:52:58 2013 +0800

----------------------------------------------------------------------
 iocore/eventsystem/I_IOBuffer.h |    8 +
 lib/ts/Allocator.h              |   32 +-
 lib/ts/Makefile.am              |    2 +
 lib/ts/ink_queue.cc             |   40 ++-
 lib/ts/ink_queue.h              |   31 ++-
 lib/ts/ink_queue_ext.cc         |  621 ++++++++++++++++++++++++++++++++++
 lib/ts/ink_queue_ext.h          |  135 ++++++++
 lib/ts/test_freelist.cc         |    2 +-
 8 files changed, 832 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/795e50df/iocore/eventsystem/I_IOBuffer.h
----------------------------------------------------------------------
diff --git a/iocore/eventsystem/I_IOBuffer.h b/iocore/eventsystem/I_IOBuffer.h
index 4ce82cd..a6b8b01 100644
--- a/iocore/eventsystem/I_IOBuffer.h
+++ b/iocore/eventsystem/I_IOBuffer.h
@@ -61,14 +61,22 @@ enum AllocType
   DEFAULT_ALLOC, CONSTANT
 };
 #ifndef TS_MICRO
+#if TS_USE_RECLAIMABLE_FREELIST
+#define DEFAULT_BUFFER_NUMBER        64
+#else
 #define DEFAULT_BUFFER_NUMBER        128
+#endif
 #define DEFAULT_HUGE_BUFFER_NUMBER   32
 #define MAX_MIOBUFFER_READERS        5
 #define DEFAULT_BUFFER_ALIGNMENT     8192       // should be disk/page size
 #define DEFAULT_BUFFER_BASE_SIZE     128
 #else
 #define DEFAULT_BUFFER_NUMBER        4
+#if TS_USE_RECLAIMABLE_FREELIST
+#define DEFAULT_HUGE_BUFFER_NUMBER   4
+#else
 #define DEFAULT_HUGE_BUFFER_NUMBER   32
+#endif
 #define MAX_MIOBUFFER_READERS        3
 #define DEFAULT_BUFFER_BASE_SIZE     128
 #define DEFAULT_BUFFER_ALIGNMENT     8  // should be disk/page size

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/795e50df/lib/ts/Allocator.h
----------------------------------------------------------------------
diff --git a/lib/ts/Allocator.h b/lib/ts/Allocator.h
index 5b38e43..033f5cd 100644
--- a/lib/ts/Allocator.h
+++ b/lib/ts/Allocator.h
@@ -58,19 +58,19 @@ public:
   void *
   alloc_void()
   {
-    return ink_freelist_new(&this->fl);
+    return ink_freelist_new(this->fl);
   }
 
   /** Deallocate a block of memory allocated by the Allocator. */
   void
   free_void(void *ptr)
   {
-    ink_freelist_free(&this->fl, ptr);
+    ink_freelist_free(this->fl, ptr);
   }
 
   Allocator()
   {
-    memset(&fl, 0, sizeof fl);
+    fl = NULL;
   }
 
   /**
@@ -81,20 +81,22 @@ public:
     @param chunk_size number of units to be allocated if free pool is empty.
     @param alignment of objects must be a power of 2.
   */
-  Allocator(const char *name, unsigned int element_size, unsigned int chunk_size = 128, unsigned int alignment = 8)
+  Allocator(const char *name, unsigned int element_size,
+            unsigned int chunk_size = 128, unsigned int alignment = 8)
   {
-    ink_freelist_init(&fl, name, element_size, chunk_size, 0, alignment);
+    ink_freelist_init(&fl, name, element_size, chunk_size, alignment);
   }
 
   /** Re-initialize the parameters of the allocator. */
   void
-  re_init(const char *name, unsigned int element_size, unsigned int chunk_size, unsigned int alignment)
+  re_init(const char *name, unsigned int element_size,
+          unsigned int chunk_size, unsigned int alignment)
   {
-    ink_freelist_init(&this->fl, name, element_size, chunk_size, 0, alignment);
+    ink_freelist_init(&this->fl, name, element_size, chunk_size, alignment);
   }
 
 protected:
-  InkFreeList fl;
+  InkFreeList *fl;
 };
 
 /**
@@ -112,7 +114,7 @@ public:
   C*
   alloc()
   {
-    void *ptr = ink_freelist_new(&this->fl);
+    void *ptr = ink_freelist_new(this->fl);
 
     memcpy(ptr, (void *)&this->proto.typeObject, sizeof(C));
     return (C *) ptr;
@@ -126,7 +128,7 @@ public:
   void
   free(C * ptr)
   {
-    ink_freelist_free(&this->fl, ptr);
+    ink_freelist_free(this->fl, ptr);
   }
 
   /**
@@ -158,9 +160,10 @@ public:
     @param chunk_size number of units to be allocated if free pool is empty.
     @param alignment of objects must be a power of 2.
   */
-  ClassAllocator(const char *name, unsigned int chunk_size = 128, unsigned int alignment = 16)
+  ClassAllocator(const char *name, unsigned int chunk_size = 128,
+                 unsigned int alignment = 16)
   {
-    ink_freelist_init(&this->fl, name, RND16(sizeof(C)), chunk_size, 0, RND16(alignment));
+    ink_freelist_init(&this->fl, name, sizeof(C), chunk_size, RND16(alignment));
   }
 
   struct
@@ -187,7 +190,7 @@ public:
   C*
   alloc()
   {
-    void *ptr = ink_freelist_new(&this->fl);
+    void *ptr = ink_freelist_new(this->fl);
 
     if (!_instantiate) {
       memcpy(ptr, (void *)&this->proto.typeObject, sizeof(C));
@@ -206,7 +209,8 @@ public:
     @param instantiate_func
 
   */
-  SparceClassAllocator(const char *name, unsigned int chunk_size = 128, unsigned int alignment = 16,
+  SparceClassAllocator(const char *name, unsigned int chunk_size = 128,
+                       unsigned int alignment = 16,
                        void (*instantiate_func) (C * proto, C * instance) = NULL)
     : ClassAllocator<C>(name, chunk_size, alignment)
   {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/795e50df/lib/ts/Makefile.am
----------------------------------------------------------------------
diff --git a/lib/ts/Makefile.am b/lib/ts/Makefile.am
index 107fc4c..a1a5f1e 100644
--- a/lib/ts/Makefile.am
+++ b/lib/ts/Makefile.am
@@ -83,6 +83,8 @@ libtsutil_la_SOURCES = \
   ink_port.h \
   ink_queue.cc \
   ink_queue.h \
+  ink_queue_ext.cc \
+  ink_queue_ext.h \
   ink_queue_utils.cc \
   ink_rand.cc \
   ink_rand.h \

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/795e50df/lib/ts/ink_queue.cc
----------------------------------------------------------------------
diff --git a/lib/ts/ink_queue.cc b/lib/ts/ink_queue.cc
index 73435e0..c8a203a 100644
--- a/lib/ts/ink_queue.cc
+++ b/lib/ts/ink_queue.cc
@@ -40,6 +40,7 @@
 #include <assert.h>
 #include <memory.h>
 #include <stdlib.h>
+#include <unistd.h>
 #include <sys/types.h>
 #include <sys/mman.h>
 #include "ink_atomic.h"
@@ -48,15 +49,9 @@
 #include "ink_error.h"
 #include "ink_assert.h"
 #include "ink_resource.h"
+#include "ink_queue_ext.h"
 
 
-typedef struct _ink_freelist_list
-{
-  InkFreeList *fl;
-  struct _ink_freelist_list *next;
-}
-ink_freelist_list;
-
 inkcoreapi volatile int64_t fastalloc_mem_in_use = 0;
 inkcoreapi volatile int64_t fastalloc_mem_total = 0;
 
@@ -79,29 +74,32 @@ inkcoreapi volatile int64_t fastalloc_mem_total = 0;
 static const int page_size = 8192;   /* sysconf (_SC_PAGESIZE); */
 #endif
 
-static ink_freelist_list *freelists = NULL;
+ink_freelist_list *freelists = NULL;
 
 inkcoreapi volatile int64_t freelist_allocated_mem = 0;
 
 #define fl_memadd(_x_) \
    ink_atomic_increment(&freelist_allocated_mem, (int64_t) (_x_));
 
-
 void
-ink_freelist_init(InkFreeList * f,
-                  const char *name, uint32_t type_size, uint32_t chunk_size, uint32_t offset, uint32_t alignment)
+ink_freelist_init(InkFreeList **fl, const char *name, uint32_t type_size,
+                  uint32_t chunk_size, uint32_t alignment)
 {
+#if TS_USE_RECLAIMABLE_FREELIST
+  return reclaimable_freelist_init(fl, name, type_size, chunk_size, alignment);
+#else
+  InkFreeList *f;
   ink_freelist_list *fll;
 
   /* its safe to add to this global list because ink_freelist_init()
      is only called from single-threaded initialization code. */
+  f = (InkFreeList *)ats_memalign(alignment, sizeof(InkFreeList));
   fll = (ink_freelist_list *)ats_malloc(sizeof(ink_freelist_list));
   fll->fl = f;
   fll->next = freelists;
   freelists = fll;
 
   f->name = name;
-  f->offset = offset;
   /* quick test for power of 2 */
   ink_assert(!(alignment & (alignment - 1)));
   f->alignment = alignment;
@@ -113,14 +111,17 @@ ink_freelist_init(InkFreeList * f,
   f->allocated = 0;
   f->allocated_base = 0;
   f->count_base = 0;
+  *fl = f;
+#endif
 }
 
 InkFreeList *
-ink_freelist_create(const char *name, uint32_t type_size, uint32_t chunk_size, uint32_t offset, uint32_t alignment)
+ink_freelist_create(const char *name, uint32_t type_size, uint32_t chunk_size,
+                    uint32_t alignment)
 {
-  InkFreeList *f = (InkFreeList *)ats_malloc(sizeof(InkFreeList));
+  InkFreeList *f;
 
-  ink_freelist_init(f, name, type_size, chunk_size, offset, alignment);
+  ink_freelist_init(&f, name, type_size, chunk_size, alignment);
   return f;
 }
 
@@ -136,6 +137,9 @@ void *
 ink_freelist_new(InkFreeList * f)
 {
 #if TS_USE_FREELIST
+#if TS_USE_RECLAIMABLE_FREELIST
+  return reclaimable_freelist_new(f);
+#else
   head_p item;
   head_p next;
   int result = 0;
@@ -220,6 +224,7 @@ ink_freelist_new(InkFreeList * f)
   ink_atomic_increment(&fastalloc_mem_in_use, (int64_t) f->type_size);
 
   return TO_PTR(FREELIST_POINTER(item));
+#endif /* TS_USE_RECLAIMABLE_FREELIST */
 #else // ! TS_USE_FREELIST
   void *newp = NULL;
 
@@ -236,6 +241,9 @@ void
 ink_freelist_free(InkFreeList * f, void *item)
 {
 #if TS_USE_FREELIST
+#if TS_USE_RECLAIMABLE_FREELIST
+  return reclaimable_freelist_free(f, item);
+#else
   volatile_void_p *adr_of_next = (volatile_void_p *) ADDRESS_OF_NEXT(item, f->offset);
   head_p h;
   head_p item_pair;
@@ -273,6 +281,7 @@ ink_freelist_free(InkFreeList * f, void *item)
 
   ink_atomic_increment((int *) &f->count, -1);
   ink_atomic_increment(&fastalloc_mem_in_use, -(int64_t) f->type_size);
+#endif /* TS_USE_RECLAIMABLE_FREELIST */
 #else
   if (f->alignment)
     ats_memalign_free(item);
@@ -444,7 +453,6 @@ ink_atomiclist_push(InkAtomicList * l, void *item)
   head_p item_pair;
   int result = 0;
   volatile void *h = NULL;
-  ink_assert(*adr_of_next == NULL);
   do {
     INK_QUEUE_LD64(head, l->head);
     h = FREELIST_POINTER(head);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/795e50df/lib/ts/ink_queue.h
----------------------------------------------------------------------
diff --git a/lib/ts/ink_queue.h b/lib/ts/ink_queue.h
index 1176b95..7d47e1d 100644
--- a/lib/ts/ink_queue.h
+++ b/lib/ts/ink_queue.h
@@ -124,27 +124,42 @@ extern "C"
 
   typedef void *void_p;
 
-  typedef struct
+#if TS_USE_RECLAIMABLE_FREELIST
+  extern float cfg_reclaim_factor;
+  extern int64_t cfg_max_overage;
+  extern int64_t cfg_enable_reclaim;
+  extern int64_t cfg_debug_filter;
+#else
+  struct _InkFreeList
   {
     volatile head_p head;
     const char *name;
     uint32_t type_size, chunk_size, count, allocated, offset, alignment;
     uint32_t allocated_base, count_base;
-  } InkFreeList, *PInkFreeList;
+  };
 
   inkcoreapi extern volatile int64_t fastalloc_mem_in_use;
   inkcoreapi extern volatile int64_t fastalloc_mem_total;
   inkcoreapi extern volatile int64_t freelist_allocated_mem;
+#endif
 
-/*
- * alignment must be a power of 2
- */
+  typedef struct _InkFreeList InkFreeList, *PInkFreeList;
+  typedef struct _ink_freelist_list
+  {
+    InkFreeList *fl;
+    struct _ink_freelist_list *next;
+  } ink_freelist_list;
+  extern ink_freelist_list *freelists;
+
+  /*
+   * alignment must be a power of 2
+   */
   InkFreeList *ink_freelist_create(const char *name, uint32_t type_size,
-                                   uint32_t chunk_size, uint32_t offset_to_next, uint32_t alignment);
+                                   uint32_t chunk_size, uint32_t alignment);
 
-  inkcoreapi void ink_freelist_init(InkFreeList * fl, const char *name,
+  inkcoreapi void ink_freelist_init(InkFreeList **fl, const char *name,
                                     uint32_t type_size, uint32_t chunk_size,
-                                    uint32_t offset_to_next, uint32_t alignment);
+                                    uint32_t alignment);
   inkcoreapi void *ink_freelist_new(InkFreeList * f);
   inkcoreapi void ink_freelist_free(InkFreeList * f, void *item);
   void ink_freelists_dump(FILE * f);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/795e50df/lib/ts/ink_queue_ext.cc
----------------------------------------------------------------------
diff --git a/lib/ts/ink_queue_ext.cc b/lib/ts/ink_queue_ext.cc
new file mode 100644
index 0000000..7060f1a
--- /dev/null
+++ b/lib/ts/ink_queue_ext.cc
@@ -0,0 +1,621 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  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.
+ */
+
+/***********************************************************************
+
+    Reclaimable freelist Implementation
+
+***********************************************************************/
+
+#include "ink_config.h"
+#include <assert.h>
+#include <memory.h>
+#include <stdlib.h>
+#include <unistd.h>
+#include <sys/types.h>
+#include <sys/mman.h>
+#include "ink_atomic.h"
+#include "ink_queue.h"
+#include "ink_memory.h"
+#include "ink_error.h"
+#include "ink_assert.h"
+#include "ink_resource.h"
+#include "ink_queue_ext.h"
+
+#if TS_USE_RECLAIMABLE_FREELIST
+
+#define CEIL(x,y)   (((x) + (y) - 1L) / (y))
+#define ROUND(x,l)  (((x) + ((l) - 1L)) & ~((l) - 1L))
+
+#define MAX_NUM_FREELIST  1024
+
+/*
+ * Configurable Variables
+ */
+float cfg_reclaim_factor = 0.3;
+int64_t cfg_max_overage = 10;
+int64_t cfg_enable_reclaim = 0;
+/*
+ * Debug filter bit mask:
+ *  bit 0: reclaim in ink_freelist_new
+ *  bit 1: reclaim in ink_freelist_free
+ *  bit 2: fetch memory from thread cache
+ */
+int64_t cfg_debug_filter;
+
+static uint32_t page_size;
+static uint32_t nr_freelist;
+static uint64_t total_mem_in_byte;
+static __thread InkThreadCache *ThreadCaches[MAX_NUM_FREELIST];
+
+static inline pthread_t thread_id(void)
+{
+  static __thread pthread_t tid;
+
+  return tid?tid:(tid = pthread_self());
+}
+#define THREAD_ID thread_id()
+#define MAX_CHUNK_BYTE_SIZE (page_size << 8)
+
+/*
+ * For debug
+ */
+#if 1
+#define show_info(tag, f, pCache) \
+  __show_info(stdout, __FILE__, __LINE__, tag, f, pCache)
+#else
+#define show_info(tag, f, pCache) \
+  __silent(__FILE__, __LINE__, tag, f, pCache)
+#endif
+#define error_info(tag, f, pCache) \
+  __show_info(stderr, __FILE__, __LINE__, tag, f, pCache)
+
+static inline void
+__silent(const char *file, int line,
+         const char *tag, InkFreeList *f, InkThreadCache *pCache) {}
+
+static inline void
+__show_info(FILE *fp, const char *file, int line,
+            const char *tag, InkFreeList *f, InkThreadCache *pCache)
+{
+
+  fprintf(fp, "[%lx:%02u][%s:%05d][%s] %6.2fM t:%-8uf:%-4u m:%-4u avg:%-6.1f"
+          " M:%-4u csbase:%-4u csize:%-4u tsize:%-6u cbsize:%u\n",
+         (long)THREAD_ID, f->thread_cache_idx, file, line, tag,
+         ((double)total_mem_in_byte/1024/1024),
+         pCache->nr_total,
+         pCache->nr_free,
+         pCache->nr_min,
+         pCache->nr_average,
+         pCache->nr_malloc,
+         f->chunk_size_base,
+         f->chunk_size,
+         f->type_size,
+         f->chunk_byte_size);
+}
+
+static inline void
+memory_alignment_init(InkFreeList *f, uint32_t type_size, uint32_t chunk_size,
+                      uint32_t alignment)
+{
+  uint32_t chunk_byte_size, user_alignment;
+
+  f->chunk_size_base = chunk_size;
+  user_alignment = alignment;
+  chunk_size = 1;
+
+  /*
+   * limit the size of each chunk and resize alignment.
+   * 1) when size of chunk > MAX_CHUNK_BYTE_SIZE:
+   *    alignment = page_size;
+   * 2) when size of chunk <= MAX_CHUNK_BYTE_SIZE:
+   *    alignment = (2^N * page_size),
+   *    alignment should not larger than MAX_CHUNK_BYTE_SIZE
+   */
+  alignment = page_size;
+  chunk_byte_size = ROUND(type_size + sizeof(InkChunkInfo), page_size);
+  if (chunk_byte_size <= MAX_CHUNK_BYTE_SIZE) {
+
+    chunk_byte_size = ROUND(type_size * f->chunk_size_base
+                            + sizeof(InkChunkInfo), page_size);
+
+    if (chunk_byte_size > MAX_CHUNK_BYTE_SIZE) {
+      chunk_size = (MAX_CHUNK_BYTE_SIZE - sizeof(InkChunkInfo)) / type_size;
+      chunk_byte_size = ROUND(type_size * chunk_size + sizeof(InkChunkInfo),
+                              page_size);
+    } else
+      chunk_size = (chunk_byte_size - sizeof(InkChunkInfo)) / type_size;
+
+    if (chunk_size > 1) {
+      /* make alignment to be (2^N * page_size),
+       * but not larger than MAX_CHUNK_BYTE_SIZE */
+      while (alignment < chunk_byte_size)
+        alignment <<= 1;
+    }
+  }
+
+  if (user_alignment > alignment) {
+    alignment = page_size;
+    while (alignment < user_alignment)
+      alignment <<= 1;
+  }
+  ink_release_assert(alignment <= MAX_CHUNK_BYTE_SIZE);
+
+  f->alignment = alignment;
+  f->type_size = type_size;
+  f->chunk_size = chunk_size;
+  f->chunk_addr_mask = ~((uintptr_t)(alignment - 1));
+  f->chunk_byte_size = chunk_byte_size;
+
+  return;
+}
+
+/*
+ * mmap_align allocates _size_ bytes and returns a pointer to the
+ * allocated memory, which address will be a multiple of _alignment_.
+ *  1)the _size_ must be a multiple of page_size;
+ *  2)the _alignment_ must be a power of page_size;
+ */
+static void*
+mmap_align(size_t size, size_t alignment) {
+  uintptr_t ptr;
+  size_t adjust, extra = 0;
+
+  ink_assert(size % page_size == 0);
+
+  /* ask for extra memory if alignment > page_size */
+  if (alignment > page_size) {
+    extra = alignment - page_size;
+  }
+  void* result = mmap(NULL, size + extra,
+                      PROT_READ|PROT_WRITE,
+                      MAP_PRIVATE|MAP_ANONYMOUS,
+                      -1, 0);
+  if (result == MAP_FAILED) {
+      xdump();
+      ink_fatal(1, "Failed to mmap %zu bytes, %s", size, strerror(errno));
+  }
+
+  /* adjust the return memory so it is aligned */
+  adjust = 0;
+  ptr = (uintptr_t)result;
+  if ((ptr & (alignment - 1)) != 0) {
+    adjust = alignment - (ptr & (alignment - 1));
+  }
+
+  /* return the unused memory to the system */
+  if (adjust > 0) {
+    munmap((void*)ptr, adjust);
+  }
+  if (adjust < extra) {
+    munmap((void*)(ptr + adjust + size), extra - adjust);
+  }
+
+  ptr += adjust;
+  ink_assert((ptr & (alignment -1)) == 0);
+  return (void*)ptr;
+}
+
+static inline InkChunkInfo *
+get_chunk_info_addr(InkFreeList *f, void *item)
+{
+  uintptr_t chunk_addr;
+
+  if (f->chunk_size > 1)
+    chunk_addr = (uintptr_t)item & f->chunk_addr_mask;
+  else
+    chunk_addr = (uintptr_t)item;
+
+  return (InkChunkInfo *)(chunk_addr + f->type_size * f->chunk_size);
+}
+
+static inline InkChunkInfo *
+ink_chunk_create(InkFreeList *f, InkThreadCache *pCache)
+{
+  uint32_t i;
+  uint32_t type_size, chunk_size;
+  void *chunk_addr, *curr, *next;
+  InkChunkInfo *pChunk;
+
+  chunk_addr = mmap_align(f->chunk_byte_size, f->alignment);
+  pChunk = (InkChunkInfo *)((char *)chunk_addr + f->type_size * f->chunk_size);
+
+  type_size = f->type_size;
+  chunk_size = f->chunk_size;
+
+  pChunk->tid = THREAD_ID;
+  pChunk->head = chunk_addr;
+  pChunk->type_size = type_size;
+  pChunk->chunk_size = chunk_size;
+  pChunk->length = f->chunk_byte_size;
+  pChunk->allocated = 0;
+  pChunk->pThreadCache = pCache;
+  pChunk->link = Link<InkChunkInfo>();
+
+  curr = pChunk->head;
+  pChunk->inner_free_list = curr;
+  for (i = 1; i < chunk_size; i++) {
+    next = (void *)((char *)curr + type_size);
+    *(void **)curr = next;
+    curr = next;
+  }
+  *(void **)curr = NULL;
+
+  ink_atomic_increment(&f->count, chunk_size);
+  ink_atomic_increment(&total_mem_in_byte, f->chunk_byte_size);
+
+  pCache->free_chunk_list.push(pChunk);
+  pCache->nr_free_chunks++;
+  return pChunk;
+}
+
+static inline void
+ink_chunk_delete(InkFreeList *f, InkThreadCache *pCache, InkChunkInfo *pChunk)
+{
+  void *chunk_addr = pChunk->head;
+
+  ink_assert(pChunk->allocated == 0);
+
+  pCache->free_chunk_list.remove(pChunk);
+  pCache->nr_free_chunks--;
+
+  if (unlikely(munmap(chunk_addr, f->chunk_byte_size))) {
+      xdump();
+      ink_fatal(1, "Failed to munmap %u bytes, %s", f->chunk_byte_size,
+                strerror(errno));
+  }
+
+  ink_atomic_increment((int *)&f->count, -f->chunk_size);
+
+  /*
+   * TODO: I had used ink_atomic_increment() here, but it would
+   * lead to incorrect value in linux OS, I don't know why:
+   *  ink_atomic_decrement((int64_t *)&total_mem_in_byte, -f->chunk_byte_size);
+   *
+   * So I create a new wrap, ink_atomic_decrement(), in ink_atomic.h,
+   * it works well. But we should create the same wrap for other OS.
+   */
+  ink_atomic_decrement(&total_mem_in_byte, f->chunk_byte_size);
+}
+
+static inline void *
+malloc_whole_chunk(InkFreeList *f, InkThreadCache *pCache, InkChunkInfo *pChunk)
+{
+  uint32_t i;
+  uint32_t type_size, chunk_size;
+  void *next, *item;
+
+  ink_assert(pChunk->allocated == 0);
+
+  type_size = f->type_size;
+  chunk_size = f->chunk_size;
+
+  item = pChunk->head;
+  for (i = 1; i < chunk_size; i++) {
+    next = (void *)((char *)item + i * type_size);
+    ink_atomic_increment(&pCache->nr_free, 1);
+    ink_atomiclist_push(&pCache->outer_free_list, next);
+  }
+
+  pChunk->allocated += chunk_size;
+  pChunk->inner_free_list = NULL;
+  pCache->nr_total += chunk_size;
+
+  ink_atomic_increment(&f->allocated, chunk_size);
+
+  return item;
+}
+
+static inline void *
+malloc_from_chunk(InkFreeList *f, InkThreadCache *pCache, InkChunkInfo *pChunk)
+{
+  void *item;
+
+  if ((item = pChunk->inner_free_list)) {
+    pChunk->inner_free_list  = *(void **)item;
+    pChunk->allocated++;
+    pCache->nr_total++;
+    ink_atomic_increment(&f->allocated, 1);
+  }
+
+  return item;
+}
+
+static inline void
+free_to_chunk(InkFreeList *f, InkThreadCache *pCache, void *item)
+{
+  InkChunkInfo *pChunk;
+
+  pChunk = get_chunk_info_addr(f, item);
+  pChunk->allocated--;
+  ink_atomic_increment((int *)&f->allocated, -1);
+  pCache->nr_total--;
+
+  *(void **)item = pChunk->inner_free_list;
+  pChunk->inner_free_list = item;
+
+  if (pChunk->allocated == 0)
+    ink_chunk_delete(f, pCache, pChunk);
+}
+
+static inline void *
+malloc_from_cache(InkFreeList *f, InkThreadCache *pCache, uint32_t nr)
+{
+  void *item;
+  InkChunkInfo *pChunk;
+
+  pChunk = pCache->free_chunk_list.head;
+  while (pChunk) {
+    while ((item = malloc_from_chunk(f, pCache, pChunk))) {
+      if (--nr == 0)
+        return item;
+
+      ink_atomic_increment(&pCache->nr_free, 1);
+      ink_atomiclist_push(&pCache->outer_free_list, item);
+    }
+    pChunk = pChunk->link.next;
+  }
+
+  pChunk = ink_chunk_create(f, pCache);
+  if (nr == f->chunk_size)
+    return malloc_whole_chunk(f, pCache, pChunk);
+
+  while ((item = malloc_from_chunk(f, pCache, pChunk))) {
+    if (--nr == 0)
+      return item;
+
+    ink_atomic_increment(&pCache->nr_free, 1);
+    ink_atomiclist_push(&pCache->outer_free_list, item);
+  }
+
+  ink_assert(0);
+  return NULL;
+}
+
+static inline void
+free_to_cache(InkFreeList *f, InkThreadCache *pCache, void *item, uint32_t nr)
+{
+  uint32_t n = nr;
+
+  if (item)
+    free_to_chunk(f, pCache, item);
+
+  while (n && (item = ink_atomiclist_pop(&pCache->outer_free_list))) {
+    free_to_chunk(f, pCache, item);
+    n--;
+  }
+  ink_atomic_increment((int *)&pCache->nr_free, -(nr - n));
+}
+
+static inline void
+refresh_average_info(InkThreadCache *pCache)
+{
+  uint32_t nr_free;
+  float nr_average;
+
+  nr_free = pCache->nr_free;
+  nr_average = pCache->nr_average;
+
+  if (pCache->status == 1 || nr_free < pCache->nr_min)
+    pCache->nr_min = nr_free;
+
+  pCache->nr_average = (nr_average * (1 - cfg_reclaim_factor)) +
+                       (nr_free * cfg_reclaim_factor);
+}
+
+static inline bool
+need_to_reclaim(InkFreeList *f, InkThreadCache *pCache, pthread_t tid)
+{
+  if (!cfg_enable_reclaim)
+    return false;
+
+  if(pCache->nr_free >= pCache->nr_average &&
+     pCache->nr_total > f->chunk_size_base) {
+    if (pCache->nr_overage++ >= cfg_max_overage) {
+      pCache->nr_overage = 0;
+      return true;
+    }
+    return false;
+  }
+
+  pCache->nr_overage = 0;
+  return false;
+}
+
+void
+reclaimable_freelist_init(InkFreeList **fl, const char *name,
+                          uint32_t type_size, uint32_t chunk_size,
+                          uint32_t alignment)
+{
+  InkFreeList *f;
+  ink_freelist_list *fll = freelists;
+
+  /* quick test for power of 2 */
+  ink_assert(!(alignment & (alignment - 1)));
+
+  if (!page_size)
+    page_size = sysconf(_SC_PAGESIZE);
+
+  /* NOTE: it's safe to operate on this global list because
+   * ink_freelist_init() is only called from single-threaded
+   * initialization code. */
+  while (fll) {
+    /* Reuse InkFreeList if it has the same aligned type_size. */
+    if (fll->fl->type_size == type_size) {
+      fll->fl->refcnt++;
+      *fl = fll->fl;
+      return;
+    }
+    fll = fll->next;
+  }
+
+  f = (InkFreeList *)ats_memalign(alignment, sizeof(InkFreeList));
+  fll = (ink_freelist_list *)ats_memalign(alignment, sizeof(ink_freelist_list));
+  fll->fl = f;
+  fll->next = freelists;
+  freelists = fll;
+
+  f->name = name;
+  f->count = 0;
+  f->allocated = 0;
+  f->allocated_base = 0;
+  f->count_base = 0;
+
+  memory_alignment_init(f, type_size, chunk_size, alignment);
+
+  f->refcnt = 1;
+  f->pThreadCache = NULL;
+  f->nr_thread_cache = 0;
+  f->thread_cache_idx = nr_freelist++;
+  ink_assert(f->thread_cache_idx < MAX_NUM_FREELIST);
+  ink_mutex_init(&f->lock, "InkFreeList Lock");
+
+  *fl = f;
+}
+
+void *
+reclaimable_freelist_new(InkFreeList *f)
+{
+  void *ptr;
+  uint32_t i, nr;
+  uint32_t old_value;
+  uint32_t num_to_move;
+  InkChunkInfo *pChunk = NULL;
+  InkThreadCache *pCache, *pNextCache;
+
+  /* no thread cache, create it */
+  if (unlikely((pCache = ThreadCaches[f->thread_cache_idx]) == NULL)) {
+    pCache = (InkThreadCache *) ats_calloc(1, sizeof(InkThreadCache));
+
+    pCache->f = f;
+    pCache->free_chunk_list = DLL<InkChunkInfo>();
+
+    /* this lock will only be accessed when initializing
+     * thread cache, so it won't damage performance */
+    ink_mutex_acquire(&f->lock);
+    ink_atomiclist_init(&pCache->outer_free_list, f->name, 0);
+
+    nr = CEIL(f->chunk_size_base, f->chunk_size);
+    for (i = 0; i < nr; i++) {
+      pChunk = ink_chunk_create(f, pCache);
+    }
+
+    pCache->nr_malloc = 1;
+
+    ThreadCaches[f->thread_cache_idx] = pCache;
+
+    if (f->pThreadCache) {
+      /* we will loop pCache.next without lock, following
+       * statement's sequence is important for us. */
+      pCache->next = f->pThreadCache;
+      pCache->prev = f->pThreadCache->prev;
+      f->pThreadCache->prev = pCache;
+      f->pThreadCache->prev->next = pCache;
+    } else {
+      pCache->next = pCache;
+      pCache->prev = pCache;
+    }
+
+    f->pThreadCache = pCache;
+    f->nr_thread_cache++;
+
+    ink_mutex_release(&f->lock);
+
+    ptr = malloc_whole_chunk(f, pCache, pChunk);
+    return ptr;
+  }
+
+  pCache->status = 0;
+
+  /* priority to fetch memory from outer_free_list */
+  if ((ptr = ink_atomiclist_pop(&pCache->outer_free_list))) {
+    old_value = ink_atomic_increment((int *)&pCache->nr_free, -1);
+    ink_release_assert(old_value > 0);
+    ink_atomic_increment(&pCache->nr_malloc, 1);
+    return ptr;
+  }
+
+  /* try to steal memory from other thread's outer_free_list */
+  pNextCache = pCache->next;
+  while (pNextCache != pCache) {
+    if ((ptr = ink_atomiclist_pop(&pNextCache->outer_free_list))) {
+          old_value = ink_atomic_increment((int *)&pNextCache->nr_free, -1);
+          ink_release_assert(old_value > 0);
+          ink_atomic_increment(&pNextCache->nr_malloc, 1);
+          return ptr;
+    }
+    pNextCache = pNextCache->next;
+  }
+
+  /* try to reclaim memory from all caches in the same thread */
+  for (i = 0; i < nr_freelist; i++) {
+    if ((pNextCache = ThreadCaches[i]) == NULL)
+      continue;
+
+    if (need_to_reclaim(pNextCache->f, pNextCache, THREAD_ID)) {
+      if (cfg_debug_filter & 0x1)
+        show_info("F", pNextCache->f, pNextCache);
+
+      num_to_move = MIN(pNextCache->nr_average, pNextCache->nr_free);
+
+      free_to_cache(pNextCache->f, pNextCache, NULL, num_to_move);
+
+      if (cfg_debug_filter & 0x1)
+        show_info("-", pNextCache->f, pNextCache);
+
+      refresh_average_info(pNextCache);
+    }
+  }
+
+  /* finally, fetch from thread local cache */
+  if (cfg_debug_filter & 0x2)
+    show_info("M", f, pCache);
+  ptr = malloc_from_cache(f, pCache, f->chunk_size);
+  if (cfg_debug_filter & 0x2)
+    show_info("+", f, pCache);
+
+  refresh_average_info(pCache);
+  ink_atomic_increment(&pCache->nr_malloc, 1);
+  return ptr;
+}
+
+void
+reclaimable_freelist_free(InkFreeList *f, void *item)
+{
+  InkChunkInfo *pChunk;
+  InkThreadCache *pCache;
+
+  if (item == NULL)
+    return;
+
+  pChunk = get_chunk_info_addr(f, item);
+  pCache = pChunk->pThreadCache;
+
+  ink_atomic_increment((int *)&pCache->nr_malloc, -1);
+  if (ink_atomic_cas((int *)&pCache->status, 0, 1))
+    refresh_average_info(pCache);
+
+  ink_atomic_increment(&pCache->nr_free, 1);
+  ink_atomiclist_push(&pCache->outer_free_list, item);
+}
+#endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/795e50df/lib/ts/ink_queue_ext.h
----------------------------------------------------------------------
diff --git a/lib/ts/ink_queue_ext.h b/lib/ts/ink_queue_ext.h
new file mode 100644
index 0000000..fbb2704
--- /dev/null
+++ b/lib/ts/ink_queue_ext.h
@@ -0,0 +1,135 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  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.
+ */
+
+#ifndef _ink_queue_ext_h_
+#define _ink_queue_ext_h_
+
+/***********************************************************************
+
+    Head file of Reclaimable freelist
+
+***********************************************************************/
+
+#include "List.h"
+#include "ink_queue.h"
+
+#ifdef __cplusplus
+extern "C"
+{
+#endif                          /* __cplusplus */
+#if TS_USE_RECLAIMABLE_FREELIST
+  struct _InkThreadCache;
+  struct _InkFreeList;
+
+  typedef struct _InkChunkInfo
+  {
+    pthread_t tid;
+
+    uint32_t type_size;
+    uint32_t chunk_size;
+    uint32_t allocated;
+    uint32_t length;
+
+    /*
+     * inner free list will only be
+     * accessed by creator-thread
+     */
+    void *inner_free_list;
+    void *head;
+
+    struct _InkThreadCache *pThreadCache;
+
+    LINK(_InkChunkInfo, link);
+  } InkChunkInfo;
+
+  typedef struct _InkThreadCache
+  {
+    struct _InkFreeList *f;
+
+    /* outer free list will be accessed by:
+     * - creator-thread, asa producer-thread
+     * - consumer-thread
+     * - neighbor-thread
+     */
+    InkAtomicList outer_free_list;
+
+    /* using for memory reclaim algorithm */
+    float nr_average;
+    uint32_t nr_total;
+    uint32_t nr_free;
+    uint32_t nr_min;
+    uint32_t nr_overage;
+    uint32_t nr_malloc;
+
+    /* represent the status(state) of allocator: Malloc-ing(0) or Free-ing(1),
+     * I use it as an simple state machine - calculating the minimum of free
+     * memory only when the status change from Malloc-ing to Free-ing.
+     */
+    uint32_t status;
+
+    uint32_t nr_free_chunks;
+    DLL<InkChunkInfo> free_chunk_list;
+
+    _InkThreadCache *prev, *next;
+  } InkThreadCache;
+
+  typedef struct _InkFreeList
+  {
+    uint32_t thread_cache_idx;
+
+    uint32_t refcnt;
+    const char *name;
+
+    uint32_t type_size;
+    uint32_t alignment;
+
+    /* number of elements in one chunk */
+    uint32_t chunk_size;
+    /* total byte size of one chuck */
+    uint32_t chunk_byte_size;
+    /* chunk_addr = (uintptr_t)ptr & chunk_addr_mask */
+    uintptr_t chunk_addr_mask;
+
+    uint32_t count;
+    uint32_t allocated;
+    uint32_t allocated_base;
+    uint32_t count_base;
+    uint32_t chunk_size_base;
+
+    uint32_t nr_thread_cache;
+    InkThreadCache *pThreadCache;
+    InkMutex lock;
+  } InkFreeList, *PInkFreeList;
+
+  /* reclaimable freelist API */
+  void reclaimable_freelist_init(InkFreeList **fl, const char *name,
+                                 uint32_t type_size, uint32_t chunk_size,
+                                 uint32_t alignment);
+  void *reclaimable_freelist_new(InkFreeList *f);
+  void reclaimable_freelist_free(InkFreeList *f, void *item);
+#endif /* END OF TS_USE_RECLAIMABLE_FREELIST */
+#ifdef __cplusplus
+}
+#endif /* __cplusplus */
+
+#endif /* _ink_queue_ext_h_ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/795e50df/lib/ts/test_freelist.cc
----------------------------------------------------------------------
diff --git a/lib/ts/test_freelist.cc b/lib/ts/test_freelist.cc
index eb37dac..d28ecb4 100644
--- a/lib/ts/test_freelist.cc
+++ b/lib/ts/test_freelist.cc
@@ -77,7 +77,7 @@ main(int argc, char *argv[])
 {
   int i;
 
-  flist = ink_freelist_create("woof", 64, 256, 0, 8);
+  flist = ink_freelist_create("woof", 64, 256, 8);
 
   for (i = 0; i < NTHREADS; i++) {
     fprintf(stderr, "Create thread %d\n", i);