You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucy.apache.org by ma...@apache.org on 2013/07/17 16:12:29 UTC

[lucy-commits] [09/34] git commit: refs/heads/master - Migrate Lucy's util classes to IVARS.

Migrate Lucy's util classes to IVARS.

Change all Lucy's util classes to access instance vars via an IVARS
struct rather than via `self`.


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

Branch: refs/heads/master
Commit: 9bff835a8bd62e7202870c05ac510905ec488139
Parents: 458824c
Author: Marvin Humphrey <ma...@rectangular.com>
Authored: Fri Jun 28 11:30:50 2013 -0700
Committer: Marvin Humphrey <ma...@rectangular.com>
Committed: Tue Jul 16 15:50:07 2013 -0700

----------------------------------------------------------------------
 core/Lucy/Util/MemoryPool.c    | 100 ++++++++-------
 core/Lucy/Util/PriorityQueue.c | 118 ++++++++++--------
 core/Lucy/Util/SortExternal.c  | 242 +++++++++++++++++++-----------------
 3 files changed, 246 insertions(+), 214 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucy/blob/9bff835a/core/Lucy/Util/MemoryPool.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Util/MemoryPool.c b/core/Lucy/Util/MemoryPool.c
index d2bfebd..fff1dce 100644
--- a/core/Lucy/Util/MemoryPool.c
+++ b/core/Lucy/Util/MemoryPool.c
@@ -20,7 +20,7 @@
 #include "Lucy/Util/MemoryPool.h"
 
 static void
-S_init_arena(MemoryPool *self, size_t amount);
+S_init_arena(MemoryPool *self, MemoryPoolIVARS *ivars, size_t amount);
 
 #define DEFAULT_BUF_SIZE 0x100000 // 1 MiB
 
@@ -42,32 +42,35 @@ MemPool_new(uint32_t arena_size) {
 
 MemoryPool*
 MemPool_init(MemoryPool *self, uint32_t arena_size) {
-    self->arena_size = arena_size == 0 ? DEFAULT_BUF_SIZE : arena_size;
-    self->arenas     = VA_new(16);
-    self->tick       = -1;
-    self->buf        = NULL;
-    self->limit      = NULL;
-    self->consumed   = 0;
+    MemoryPoolIVARS *const ivars = MemPool_IVARS(self);
+    ivars->arena_size = arena_size == 0 ? DEFAULT_BUF_SIZE : arena_size;
+    ivars->arenas     = VA_new(16);
+    ivars->tick       = -1;
+    ivars->buf        = NULL;
+    ivars->limit      = NULL;
+    ivars->consumed   = 0;
 
     return self;
 }
 
 void
 MemPool_destroy(MemoryPool *self) {
-    DECREF(self->arenas);
+    MemoryPoolIVARS *const ivars = MemPool_IVARS(self);
+    DECREF(ivars->arenas);
     SUPER_DESTROY(self, MEMORYPOOL);
 }
 
 static void
-S_init_arena(MemoryPool *self, size_t amount) {
+S_init_arena(MemoryPool *self, MemoryPoolIVARS *ivars, size_t amount) {
+    UNUSED_VAR(self);
     ByteBuf *bb;
 
     // Indicate which arena we're using at present.
-    self->tick++;
+    ivars->tick++;
 
-    if (self->tick < (int32_t)VA_Get_Size(self->arenas)) {
+    if (ivars->tick < (int32_t)VA_Get_Size(ivars->arenas)) {
         // In recycle mode, use previously acquired memory.
-        bb = (ByteBuf*)VA_Fetch(self->arenas, self->tick);
+        bb = (ByteBuf*)VA_Fetch(ivars->arenas, ivars->tick);
         if (amount >= BB_Get_Size(bb)) {
             BB_Grow(bb, amount);
             BB_Set_Size(bb, amount);
@@ -75,63 +78,65 @@ S_init_arena(MemoryPool *self, size_t amount) {
     }
     else {
         // In add mode, get more mem from system.
-        size_t buf_size = (amount + 1) > self->arena_size
+        size_t buf_size = (amount + 1) > ivars->arena_size
                           ? (amount + 1)
-                          : self->arena_size;
+                          : ivars->arena_size;
         char *ptr = (char*)MALLOCATE(buf_size);
         bb = BB_new_steal_bytes(ptr, buf_size - 1, buf_size);
-        VA_Push(self->arenas, (Obj*)bb);
+        VA_Push(ivars->arenas, (Obj*)bb);
     }
 
     // Recalculate consumption to take into account blocked off space.
-    self->consumed = 0;
-    for (int32_t i = 0; i < self->tick; i++) {
-        ByteBuf *bb = (ByteBuf*)VA_Fetch(self->arenas, i);
-        self->consumed += BB_Get_Size(bb);
+    ivars->consumed = 0;
+    for (int32_t i = 0; i < ivars->tick; i++) {
+        ByteBuf *bb = (ByteBuf*)VA_Fetch(ivars->arenas, i);
+        ivars->consumed += BB_Get_Size(bb);
     }
 
-    self->buf   = BB_Get_Buf(bb);
-    self->limit = self->buf + BB_Get_Size(bb);
+    ivars->buf   = BB_Get_Buf(bb);
+    ivars->limit = ivars->buf + BB_Get_Size(bb);
 }
 
 size_t
 MemPool_get_consumed(MemoryPool *self) {
-    return self->consumed;
+    return MemPool_IVARS(self)->consumed;
 }
 
 void*
 MemPool_grab(MemoryPool *self, size_t amount) {
+    MemoryPoolIVARS *const ivars = MemPool_IVARS(self);
     INCREASE_TO_WORD_MULTIPLE(amount);
-    self->last_buf = self->buf;
+    ivars->last_buf = ivars->buf;
 
     // Verify that we have enough stocked up, otherwise get more.
-    self->buf += amount;
-    if (self->buf >= self->limit) {
+    ivars->buf += amount;
+    if (ivars->buf >= ivars->limit) {
         // Get enough mem from system or die trying.
-        S_init_arena(self, amount);
-        self->last_buf = self->buf;
-        self->buf += amount;
+        S_init_arena(self, ivars, amount);
+        ivars->last_buf = ivars->buf;
+        ivars->buf += amount;
     }
 
     // Track bytes we've allocated from this pool.
-    self->consumed += amount;
+    ivars->consumed += amount;
 
-    return self->last_buf;
+    return ivars->last_buf;
 }
 
 void
 MemPool_resize(MemoryPool *self, void *ptr, size_t new_amount) {
-    const size_t last_amount = self->buf - self->last_buf;
+    MemoryPoolIVARS *const ivars = MemPool_IVARS(self);
+    const size_t last_amount = ivars->buf - ivars->last_buf;
     INCREASE_TO_WORD_MULTIPLE(new_amount);
 
-    if (ptr != self->last_buf) {
+    if (ptr != ivars->last_buf) {
         THROW(ERR, "Not the last pointer allocated.");
     }
     else {
         if (new_amount <= last_amount) {
             const size_t difference = last_amount - new_amount;
-            self->buf      -= difference;
-            self->consumed -= difference;
+            ivars->buf      -= difference;
+            ivars->consumed -= difference;
         }
         else {
             THROW(ERR, "Can't resize to greater amount: %u64 > %u64",
@@ -142,28 +147,31 @@ MemPool_resize(MemoryPool *self, void *ptr, size_t new_amount) {
 
 void
 MemPool_release_all(MemoryPool *self) {
-    self->tick     = -1;
-    self->buf      = NULL;
-    self->last_buf = NULL;
-    self->limit    = NULL;
+    MemoryPoolIVARS *const ivars = MemPool_IVARS(self);
+    ivars->tick     = -1;
+    ivars->buf      = NULL;
+    ivars->last_buf = NULL;
+    ivars->limit    = NULL;
 }
 
 void
 MemPool_eat(MemoryPool *self, MemoryPool *other) {
-    if (self->buf != NULL) {
+    MemoryPoolIVARS *const ivars = MemPool_IVARS(self);
+    MemoryPoolIVARS *const ovars = MemPool_IVARS(other);
+    if (ivars->buf != NULL) {
         THROW(ERR, "Memory pool is not empty");
     }
 
     // Move active arenas from other to self.
-    for (int32_t i = 0; i <= other->tick; i++) {
-        ByteBuf *arena = (ByteBuf*)VA_Shift(other->arenas);
+    for (int32_t i = 0; i <= ovars->tick; i++) {
+        ByteBuf *arena = (ByteBuf*)VA_Shift(ovars->arenas);
         // Maybe displace existing arena.
-        VA_Store(self->arenas, i, (Obj*)arena);
+        VA_Store(ivars->arenas, i, (Obj*)arena);
     }
-    self->tick     = other->tick;
-    self->last_buf = other->last_buf;
-    self->buf      = other->buf;
-    self->limit    = other->limit;
+    ivars->tick     = ovars->tick;
+    ivars->last_buf = ovars->last_buf;
+    ivars->buf      = ovars->buf;
+    ivars->limit    = ovars->limit;
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucy/blob/9bff835a/core/Lucy/Util/PriorityQueue.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Util/PriorityQueue.c b/core/Lucy/Util/PriorityQueue.c
index 048d410..b13bce1 100644
--- a/core/Lucy/Util/PriorityQueue.c
+++ b/core/Lucy/Util/PriorityQueue.c
@@ -24,35 +24,36 @@
 // Add an element to the heap.  Throw an error if too many elements
 // are added.
 static void
-S_put(PriorityQueue *self, Obj *element);
+S_put(PriorityQueue *self, PriorityQueueIVARS *ivars, Obj *element);
 
 // Free all the elements in the heap and set size to 0.
 static void
-S_clear(PriorityQueue *self);
+S_clear(PriorityQueue *self, PriorityQueueIVARS *ivars);
 
 // Heap adjuster.
 static void
-S_up_heap(PriorityQueue *self);
+S_up_heap(PriorityQueue *self, PriorityQueueIVARS *ivars);
 
 // Heap adjuster.  Should be called when the item at the top changes.
 static void
-S_down_heap(PriorityQueue *self);
+S_down_heap(PriorityQueue *self, PriorityQueueIVARS *ivars);
 
 PriorityQueue*
 PriQ_init(PriorityQueue *self, uint32_t max_size) {
+    PriorityQueueIVARS *const ivars = PriQ_IVARS(self);
     if (max_size == UINT32_MAX) {
         THROW(ERR, "max_size too large: %u32", max_size);
     }
     uint32_t heap_size = max_size + 1;
 
     // Init.
-    self->size = 0;
+    ivars->size = 0;
 
     // Assign.
-    self->max_size = max_size;
+    ivars->max_size = max_size;
 
     // Allocate space for the heap, assign all slots to NULL.
-    self->heap = (Obj**)CALLOCATE(heap_size, sizeof(Obj*));
+    ivars->heap = (Obj**)CALLOCATE(heap_size, sizeof(Obj*));
 
     ABSTRACT_CLASS_CHECK(self, PRIORITYQUEUE);
     return self;
@@ -60,32 +61,33 @@ PriQ_init(PriorityQueue *self, uint32_t max_size) {
 
 void
 PriQ_destroy(PriorityQueue *self) {
-    if (self->heap) {
-        S_clear(self);
-        FREEMEM(self->heap);
+    PriorityQueueIVARS *const ivars = PriQ_IVARS(self);
+    if (ivars->heap) {
+        S_clear(self, ivars);
+        FREEMEM(ivars->heap);
     }
     SUPER_DESTROY(self, PRIORITYQUEUE);
 }
 
 uint32_t
 PriQ_get_size(PriorityQueue *self) {
-    return self->size;
+    return PriQ_IVARS(self)->size;
 }
 
 static void
-S_put(PriorityQueue *self, Obj *element) {
+S_put(PriorityQueue *self, PriorityQueueIVARS *ivars, Obj *element) {
     // Increment size.
-    if (self->size >= self->max_size) {
-        THROW(ERR, "PriorityQueue exceeded max_size: %u32 %u32", self->size,
-              self->max_size);
+    if (ivars->size >= ivars->max_size) {
+        THROW(ERR, "PriorityQueue exceeded max_size: %u32 %u32", ivars->size,
+              ivars->max_size);
     }
-    self->size++;
+    ivars->size++;
 
     // Put element into heap.
-    self->heap[self->size] = element;
+    ivars->heap[ivars->size] = element;
 
     // Adjust heap.
-    S_up_heap(self);
+    S_up_heap(self, ivars);
 }
 
 bool
@@ -98,22 +100,24 @@ PriQ_insert(PriorityQueue *self, Obj *element) {
 
 Obj*
 PriQ_jostle(PriorityQueue *self, Obj *element) {
+    PriorityQueueIVARS *const ivars = PriQ_IVARS(self);
+
     // Absorb element if there's a vacancy.
-    if (self->size < self->max_size) {
-        S_put(self, element);
+    if (ivars->size < ivars->max_size) {
+        S_put(self, ivars, element);
         return NULL;
     }
     // Otherwise, compete for the slot.
-    else if (self->size == 0) {
+    else if (ivars->size == 0) {
         return element;
     }
     else {
         Obj *scratch = PriQ_Peek(self);
         if (!PriQ_Less_Than(self, element, scratch)) {
             // If the new element belongs in the queue, replace something.
-            Obj *retval = self->heap[1];
-            self->heap[1] = element;
-            S_down_heap(self);
+            Obj *retval = ivars->heap[1];
+            ivars->heap[1] = element;
+            S_down_heap(self, ivars);
             return retval;
         }
         else {
@@ -124,15 +128,16 @@ PriQ_jostle(PriorityQueue *self, Obj *element) {
 
 Obj*
 PriQ_pop(PriorityQueue *self) {
-    if (self->size > 0) {
+    PriorityQueueIVARS *const ivars = PriQ_IVARS(self);
+    if (ivars->size > 0) {
         // Save the first value.
-        Obj *result = self->heap[1];
+        Obj *result = ivars->heap[1];
 
         // Move last to first and adjust heap.
-        self->heap[1] = self->heap[self->size];
-        self->heap[self->size] = NULL;
-        self->size--;
-        S_down_heap(self);
+        ivars->heap[1] = ivars->heap[ivars->size];
+        ivars->heap[ivars->size] = NULL;
+        ivars->size--;
+        S_down_heap(self, ivars);
 
         // Return the value, leaving a refcount for the caller.
         return result;
@@ -144,11 +149,12 @@ PriQ_pop(PriorityQueue *self) {
 
 VArray*
 PriQ_pop_all(PriorityQueue *self) {
-    VArray *retval = VA_new(self->size);
+    PriorityQueueIVARS *const ivars = PriQ_IVARS(self);
+    VArray *retval = VA_new(ivars->size);
 
     // Map the queue nodes onto the array in reverse order.
-    if (self->size) {
-        for (uint32_t i = self->size; i--;) {
+    if (ivars->size) {
+        for (uint32_t i = ivars->size; i--;) {
             Obj *const elem = PriQ_Pop(self);
             VA_Store(retval, i, elem);
         }
@@ -159,8 +165,9 @@ PriQ_pop_all(PriorityQueue *self) {
 
 Obj*
 PriQ_peek(PriorityQueue *self) {
-    if (self->size > 0) {
-        return self->heap[1];
+    PriorityQueueIVARS *const ivars = PriQ_IVARS(self);
+    if (ivars->size > 0) {
+        return ivars->heap[1];
     }
     else {
         return NULL;
@@ -168,62 +175,63 @@ PriQ_peek(PriorityQueue *self) {
 }
 
 static void
-S_clear(PriorityQueue *self) {
-    Obj **elem_ptr = (self->heap + 1);
+S_clear(PriorityQueue *self, PriorityQueueIVARS *ivars) {
+    UNUSED_VAR(self);
+    Obj **elem_ptr = (ivars->heap + 1);
 
     // Node 0 is held empty, to make the algo clearer.
-    for (uint32_t i = 1; i <= self->size; i++) {
+    for (uint32_t i = 1; i <= ivars->size; i++) {
         DECREF(*elem_ptr);
         *elem_ptr = NULL;
         elem_ptr++;
     }
-    self->size = 0;
+    ivars->size = 0;
 }
 
 static void
-S_up_heap(PriorityQueue *self) {
-    uint32_t i = self->size;
+S_up_heap(PriorityQueue *self, PriorityQueueIVARS *ivars) {
+    uint32_t i = ivars->size;
     uint32_t j = i >> 1;
-    Obj *const node = self->heap[i]; // save bottom node
+    Obj *const node = ivars->heap[i]; // save bottom node
 
     while (j > 0
-           && PriQ_Less_Than(self, node, self->heap[j])
+           && PriQ_Less_Than(self, node, ivars->heap[j])
           ) {
-        self->heap[i] = self->heap[j];
+        ivars->heap[i] = ivars->heap[j];
         i = j;
         j = j >> 1;
     }
-    self->heap[i] = node;
+    ivars->heap[i] = node;
 }
 
 static void
-S_down_heap(PriorityQueue *self) {
+S_down_heap(PriorityQueue *self, PriorityQueueIVARS *ivars) {
     uint32_t i = 1;
     uint32_t j = i << 1;
     uint32_t k = j + 1;
-    Obj *node = self->heap[i]; // save top node
+    Obj *node = ivars->heap[i]; // save top node
 
     // Find smaller child.
-    if (k <= self->size
-        && PriQ_Less_Than(self, self->heap[k], self->heap[j])
+    if (k <= ivars->size
+        && PriQ_Less_Than(self, ivars->heap[k], ivars->heap[j])
        ) {
         j = k;
     }
 
-    while (j <= self->size
-           && PriQ_Less_Than(self, self->heap[j], node)
+    while (j <= ivars->size
+           && PriQ_Less_Than(self, ivars->heap[j], node)
           ) {
-        self->heap[i] = self->heap[j];
+        ivars->heap[i] = ivars->heap[j];
         i = j;
         j = i << 1;
         k = j + 1;
-        if (k <= self->size
-            && PriQ_Less_Than(self, self->heap[k], self->heap[j])
+        if (k <= ivars->size
+            && PriQ_Less_Than(self, ivars->heap[k], ivars->heap[j])
            ) {
             j = k;
         }
     }
-    self->heap[i] = node;
+    ivars->heap[i] = node;
 }
 
 

http://git-wip-us.apache.org/repos/asf/lucy/blob/9bff835a/core/Lucy/Util/SortExternal.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Util/SortExternal.c b/core/Lucy/Util/SortExternal.c
index 1a220ed..6337ee1 100644
--- a/core/Lucy/Util/SortExternal.c
+++ b/core/Lucy/Util/SortExternal.c
@@ -21,41 +21,44 @@
 
 // Refill the main cache, drawing from the caches of all runs.
 static void
-S_refill_cache(SortExternal *self);
+S_refill_cache(SortExternal *self, SortExternalIVARS *ivars);
 
 // Absorb all the items which are "in-range" from all the Runs into the main
 // cache.
 static void
-S_absorb_slices(SortExternal *self, uint8_t *endpost);
+S_absorb_slices(SortExternal *self, SortExternalIVARS *ivars,
+                uint8_t *endpost);
 
 // Return the address for the item in one of the runs' caches which is the
 // highest in sort order, but which we can guarantee is lower in sort order
 // than any item which has yet to enter a run cache.
 static uint8_t*
-S_find_endpost(SortExternal *self);
+S_find_endpost(SortExternal *self, SortExternalIVARS *ivars);
 
 // Determine how many cache items are less than or equal to [endpost].
 static uint32_t
-S_find_slice_size(SortExternal *self, uint8_t *endpost);
+S_find_slice_size(SortExternal *self, SortExternalIVARS *ivars,
+                  uint8_t *endpost);
 
 SortExternal*
 SortEx_init(SortExternal *self, size_t width) {
+    SortExternalIVARS *const ivars = SortEx_IVARS(self);
     // Assign.
-    self->width        = width;
+    ivars->width        = width;
 
     // Init.
-    self->mem_thresh   = UINT32_MAX;
-    self->cache        = NULL;
-    self->cache_cap    = 0;
-    self->cache_max    = 0;
-    self->cache_tick   = 0;
-    self->scratch      = NULL;
-    self->scratch_cap  = 0;
-    self->runs         = VA_new(0);
-    self->slice_sizes  = NULL;
-    self->slice_starts = NULL;
-    self->num_slices   = 0;
-    self->flipped      = false;
+    ivars->mem_thresh   = UINT32_MAX;
+    ivars->cache        = NULL;
+    ivars->cache_cap    = 0;
+    ivars->cache_max    = 0;
+    ivars->cache_tick   = 0;
+    ivars->scratch      = NULL;
+    ivars->scratch_cap  = 0;
+    ivars->runs         = VA_new(0);
+    ivars->slice_sizes  = NULL;
+    ivars->slice_starts = NULL;
+    ivars->num_slices   = 0;
+    ivars->flipped      = false;
 
     ABSTRACT_CLASS_CHECK(self, SORTEXTERNAL);
     return self;
@@ -63,43 +66,46 @@ SortEx_init(SortExternal *self, size_t width) {
 
 void
 SortEx_destroy(SortExternal *self) {
-    FREEMEM(self->scratch);
-    FREEMEM(self->slice_sizes);
-    FREEMEM(self->slice_starts);
-    if (self->cache) {
+    SortExternalIVARS *const ivars = SortEx_IVARS(self);
+    FREEMEM(ivars->scratch);
+    FREEMEM(ivars->slice_sizes);
+    FREEMEM(ivars->slice_starts);
+    if (ivars->cache) {
         SortEx_Clear_Cache(self);
-        FREEMEM(self->cache);
+        FREEMEM(ivars->cache);
     }
-    DECREF(self->runs);
+    DECREF(ivars->runs);
     SUPER_DESTROY(self, SORTEXTERNAL);
 }
 
 void
 SortEx_clear_cache(SortExternal *self) {
-    self->cache_max    = 0;
-    self->cache_tick   = 0;
+    SortExternalIVARS *const ivars = SortEx_IVARS(self);
+    ivars->cache_max    = 0;
+    ivars->cache_tick   = 0;
 }
 
 void
 SortEx_feed(SortExternal *self, void *data) {
-    const size_t width = self->width;
-    if (self->cache_max == self->cache_cap) {
-        size_t amount = Memory_oversize(self->cache_max + 1, width);
+    SortExternalIVARS *const ivars = SortEx_IVARS(self);
+    const size_t width = ivars->width;
+    if (ivars->cache_max == ivars->cache_cap) {
+        size_t amount = Memory_oversize(ivars->cache_max + 1, width);
         SortEx_Grow_Cache(self, amount);
     }
-    uint8_t *target = self->cache + self->cache_max * width;
+    uint8_t *target = ivars->cache + ivars->cache_max * width;
     memcpy(target, data, width);
-    self->cache_max++;
+    ivars->cache_max++;
 }
 
 static INLINE void*
-SI_peek(SortExternal *self) {
-    if (self->cache_tick >= self->cache_max) {
-        S_refill_cache(self);
+SI_peek(SortExternal *self, SortExternalIVARS *ivars) {
+    if (ivars->cache_tick >= ivars->cache_max) {
+        S_refill_cache(self, ivars);
     }
 
-    if (self->cache_max > 0) {
-        return self->cache + self->cache_tick * self->width;
+    if (ivars->cache_max > 0) {
+        return ivars->cache + ivars->cache_tick * ivars->width;
     }
     else {
         return NULL;
@@ -108,95 +114,100 @@ SI_peek(SortExternal *self) {
 
 void*
 SortEx_fetch(SortExternal *self) {
-    void *address = SI_peek(self);
-    self->cache_tick++;
+    SortExternalIVARS *const ivars = SortEx_IVARS(self);
+    void *address = SI_peek(self, ivars);
+    ivars->cache_tick++;
     return address;
 }
 
 void*
 SortEx_peek(SortExternal *self) {
-    return SI_peek(self);
+    SortExternalIVARS *const ivars = SortEx_IVARS(self);
+    return SI_peek(self, ivars);
 }
 
 void
 SortEx_sort_cache(SortExternal *self) {
-    if (self->cache_tick != 0) {
-        THROW(ERR, "Cant Sort_Cache() after fetching %u32 items", self->cache_tick);
+    SortExternalIVARS *const ivars = SortEx_IVARS(self);
+    if (ivars->cache_tick != 0) {
+        THROW(ERR, "Cant Sort_Cache() after fetching %u32 items", ivars->cache_tick);
     }
-    if (self->cache_max != 0) {
+    if (ivars->cache_max != 0) {
         VTable *vtable = SortEx_Get_VTable(self);
         Cfish_Sort_Compare_t compare
             = (Cfish_Sort_Compare_t)METHOD_PTR(vtable, Lucy_SortEx_Compare);
-        if (self->scratch_cap < self->cache_cap) {
-            self->scratch_cap = self->cache_cap;
-            self->scratch = (uint8_t*)REALLOCATE(
-                                self->scratch,
-                                self->scratch_cap * self->width);
+        if (ivars->scratch_cap < ivars->cache_cap) {
+            ivars->scratch_cap = ivars->cache_cap;
+            ivars->scratch
+                = (uint8_t*)REALLOCATE(ivars->scratch,
+                                       ivars->scratch_cap * ivars->width);
         }
-        Sort_mergesort(self->cache, self->scratch, self->cache_max,
-                       self->width, compare, self);
+        Sort_mergesort(ivars->cache, ivars->scratch, ivars->cache_max,
+                       ivars->width, compare, self);
     }
 }
 
 void
 SortEx_flip(SortExternal *self) {
     SortEx_Flush(self);
-    self->flipped = true;
+    SortEx_IVARS(self)->flipped = true;
 }
 
 void
 SortEx_add_run(SortExternal *self, SortExternal *run) {
-    VA_Push(self->runs, (Obj*)run);
-    uint32_t num_runs = VA_Get_Size(self->runs);
-    self->slice_sizes = (uint32_t*)REALLOCATE(
-                            self->slice_sizes,
-                            num_runs * sizeof(uint32_t));
-    self->slice_starts = (uint8_t**)REALLOCATE(
-                             self->slice_starts,
-                             num_runs * sizeof(uint8_t*));
+    SortExternalIVARS *const ivars = SortEx_IVARS(self);
+    VA_Push(ivars->runs, (Obj*)run);
+    uint32_t num_runs = VA_Get_Size(ivars->runs);
+    ivars->slice_sizes
+        = (uint32_t*)REALLOCATE(ivars->slice_sizes,
+                                num_runs * sizeof(uint32_t));
+    ivars->slice_starts
+        = (uint8_t**)REALLOCATE(ivars->slice_starts,
+                                num_runs * sizeof(uint8_t*));
 }
 
 static void
-S_refill_cache(SortExternal *self) {
+S_refill_cache(SortExternal *self, SortExternalIVARS *ivars) {
     // Reset cache vars.
     SortEx_Clear_Cache(self);
 
     // Make sure all runs have at least one item in the cache.
     uint32_t i = 0;
-    while (i < VA_Get_Size(self->runs)) {
-        SortExternal *const run = (SortExternal*)VA_Fetch(self->runs, i);
+    while (i < VA_Get_Size(ivars->runs)) {
+        SortExternal *const run = (SortExternal*)VA_Fetch(ivars->runs, i);
         if (SortEx_Cache_Count(run) > 0 || SortEx_Refill(run) > 0) {
             i++; // Run has some elements, so keep.
         }
         else {
-            VA_Excise(self->runs, i, 1);
+            VA_Excise(ivars->runs, i, 1);
         }
     }
 
     // Absorb as many elems as possible from all runs into main cache.
-    if (VA_Get_Size(self->runs)) {
-        uint8_t *endpost = S_find_endpost(self);
-        S_absorb_slices(self, endpost);
+    if (VA_Get_Size(ivars->runs)) {
+        uint8_t *endpost = S_find_endpost(self, ivars);
+        S_absorb_slices(self, ivars, endpost);
     }
 }
 
 static uint8_t*
-S_find_endpost(SortExternal *self) {
+S_find_endpost(SortExternal *self, SortExternalIVARS *ivars) {
     uint8_t *endpost = NULL;
-    const size_t width = self->width;
+    const size_t width = ivars->width;
 
-    for (uint32_t i = 0, max = VA_Get_Size(self->runs); i < max; i++) {
+    for (uint32_t i = 0, max = VA_Get_Size(ivars->runs); i < max; i++) {
         // Get a run and retrieve the last item in its cache.
-        SortExternal *const run = (SortExternal*)VA_Fetch(self->runs, i);
-        const uint32_t tick = run->cache_max - 1;
-        if (tick >= run->cache_cap || run->cache_max < 1) {
+        SortExternal *const run = (SortExternal*)VA_Fetch(ivars->runs, i);
+        SortExternalIVARS *const run_ivars = SortEx_IVARS(run);
+        const uint32_t tick = run_ivars->cache_max - 1;
+        if (tick >= run_ivars->cache_cap || run_ivars->cache_max < 1) {
             THROW(ERR, "Invalid SortExternal cache access: %u32 %u32 %u32", tick,
-                  run->cache_max, run->cache_cap);
+                  run_ivars->cache_max, run_ivars->cache_cap);
         }
         else {
             // Cache item with the highest sort value currently held in memory
             // by the run.
-            uint8_t *candidate = run->cache + tick * width;
+            uint8_t *candidate = run_ivars->cache + tick * width;
 
             // If it's the first run, item is automatically the new endpost.
             if (i == 0) {
@@ -213,75 +224,77 @@ S_find_endpost(SortExternal *self) {
 }
 
 static void
-S_absorb_slices(SortExternal *self, uint8_t *endpost) {
-    size_t      width        = self->width;
-    uint32_t    num_runs     = VA_Get_Size(self->runs);
-    uint8_t   **slice_starts = self->slice_starts;
-    uint32_t   *slice_sizes  = self->slice_sizes;
+S_absorb_slices(SortExternal *self, SortExternalIVARS *ivars,
+                uint8_t *endpost) {
+    size_t      width        = ivars->width;
+    uint32_t    num_runs     = VA_Get_Size(ivars->runs);
+    uint8_t   **slice_starts = ivars->slice_starts;
+    uint32_t   *slice_sizes  = ivars->slice_sizes;
     VTable     *vtable       = SortEx_Get_VTable(self);
     Cfish_Sort_Compare_t compare
         = (Cfish_Sort_Compare_t)METHOD_PTR(vtable, Lucy_SortEx_Compare);
 
-    if (self->cache_max != 0) { THROW(ERR, "Can't refill unless empty"); }
+    if (ivars->cache_max != 0) { THROW(ERR, "Can't refill unless empty"); }
 
     // Move all the elements in range into the main cache as slices.
     for (uint32_t i = 0; i < num_runs; i++) {
-        SortExternal *const run = (SortExternal*)VA_Fetch(self->runs, i);
-        uint32_t slice_size = S_find_slice_size(run, endpost);
+        SortExternal *const run = (SortExternal*)VA_Fetch(ivars->runs, i);
+        SortExternalIVARS *const run_ivars = SortEx_IVARS(run);
+        uint32_t slice_size = S_find_slice_size(run, run_ivars, endpost);
 
         if (slice_size) {
             // Move slice content from run cache to main cache.
-            if (self->cache_max + slice_size > self->cache_cap) {
-                size_t cap = Memory_oversize(self->cache_max + slice_size,
+            if (ivars->cache_max + slice_size > ivars->cache_cap) {
+                size_t cap = Memory_oversize(ivars->cache_max + slice_size,
                                              width);
                 SortEx_Grow_Cache(self, cap);
             }
-            memcpy(self->cache + self->cache_max * width,
-                   run->cache + run->cache_tick * width,
+            memcpy(ivars->cache + ivars->cache_max * width,
+                   run_ivars->cache + run_ivars->cache_tick * width,
                    slice_size * width);
-            run->cache_tick += slice_size;
-            self->cache_max += slice_size;
+            run_ivars->cache_tick += slice_size;
+            ivars->cache_max += slice_size;
 
             // Track number of slices and slice sizes.
-            slice_sizes[self->num_slices++] = slice_size;
+            slice_sizes[ivars->num_slices++] = slice_size;
         }
     }
 
     // Transform slice starts from ticks to pointers.
     uint32_t total = 0;
-    for (uint32_t i = 0; i < self->num_slices; i++) {
-        slice_starts[i] = self->cache + total * width;
+    for (uint32_t i = 0; i < ivars->num_slices; i++) {
+        slice_starts[i] = ivars->cache + total * width;
         total += slice_sizes[i];
     }
 
     // The main cache now consists of several slices.  Sort the main cache,
     // but exploit the fact that each slice is already sorted.
-    if (self->scratch_cap < self->cache_cap) {
-        self->scratch_cap = self->cache_cap;
-        self->scratch = (uint8_t*)REALLOCATE(
-                            self->scratch, self->scratch_cap * width);
+    if (ivars->scratch_cap < ivars->cache_cap) {
+        ivars->scratch_cap = ivars->cache_cap;
+        ivars->scratch = (uint8_t*)REALLOCATE(
+                            ivars->scratch, ivars->scratch_cap * width);
     }
 
     // Exploit previous sorting, rather than sort cache naively.
     // Leave the first slice intact if the number of slices is odd. */
-    while (self->num_slices > 1) {
+    while (ivars->num_slices > 1) {
         uint32_t i = 0;
         uint32_t j = 0;
 
-        while (i < self->num_slices) {
-            if (self->num_slices - i >= 2) {
+        while (i < ivars->num_slices) {
+            if (ivars->num_slices - i >= 2) {
                 // Merge two consecutive slices.
                 const uint32_t merged_size = slice_sizes[i] + slice_sizes[i + 1];
                 Sort_merge(slice_starts[i], slice_sizes[i],
-                           slice_starts[i + 1], slice_sizes[i + 1], self->scratch,
-                           self->width, compare, self);
+                           slice_starts[i + 1], slice_sizes[i + 1], ivars->scratch,
+                           ivars->width, compare, self);
                 slice_sizes[j]  = merged_size;
                 slice_starts[j] = slice_starts[i];
-                memcpy(slice_starts[j], self->scratch, merged_size * width);
+                memcpy(slice_starts[j], ivars->scratch, merged_size * width);
                 i += 2;
                 j += 1;
             }
-            else if (self->num_slices - i >= 1) {
+            else if (ivars->num_slices - i >= 1) {
                 // Move single slice pointer.
                 slice_sizes[j]  = slice_sizes[i];
                 slice_starts[j] = slice_starts[i];
@@ -289,26 +302,28 @@ S_absorb_slices(SortExternal *self, uint8_t *endpost) {
                 j += 1;
             }
         }
-        self->num_slices = j;
+        ivars->num_slices = j;
     }
 
-    self->num_slices = 0;
+    ivars->num_slices = 0;
 }
 
 void
 SortEx_grow_cache(SortExternal *self, uint32_t size) {
-    if (size > self->cache_cap) {
-        self->cache = (uint8_t*)REALLOCATE(self->cache, size * self->width);
-        self->cache_cap = size;
+    SortExternalIVARS *const ivars = SortEx_IVARS(self);
+    if (size > ivars->cache_cap) {
+        ivars->cache = (uint8_t*)REALLOCATE(ivars->cache, size * ivars->width);
+        ivars->cache_cap = size;
     }
 }
 
 static uint32_t
-S_find_slice_size(SortExternal *self, uint8_t *endpost) {
-    int32_t          lo      = self->cache_tick - 1;
-    int32_t          hi      = self->cache_max;
-    uint8_t *const   cache   = self->cache;
-    const size_t     width   = self->width;
+S_find_slice_size(SortExternal *self, SortExternalIVARS *ivars,
+                  uint8_t *endpost) {
+    int32_t          lo      = ivars->cache_tick - 1;
+    int32_t          hi      = ivars->cache_max;
+    uint8_t *const   cache   = ivars->cache;
+    const size_t     width   = ivars->width;
     SortEx_Compare_t compare
         = METHOD_PTR(SortEx_Get_VTable(self), Lucy_SortEx_Compare);
 
@@ -323,17 +338,18 @@ S_find_slice_size(SortExternal *self, uint8_t *endpost) {
     // If lo is still -1, we didn't find anything.
     return lo == -1
            ? 0
-           : (lo - self->cache_tick) + 1;
+           : (lo - ivars->cache_tick) + 1;
 }
 
 void
 SortEx_set_mem_thresh(SortExternal *self, uint32_t mem_thresh) {
-    self->mem_thresh = mem_thresh;
+    SortEx_IVARS(self)->mem_thresh = mem_thresh;
 }
 
 uint32_t
 SortEx_cache_count(SortExternal *self) {
-    return self->cache_max - self->cache_tick;
+    SortExternalIVARS *const ivars = SortEx_IVARS(self);
+    return ivars->cache_max - ivars->cache_tick;
 }