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 2014/07/03 03:09:09 UTC

[1/5] git commit: refs/heads/264_dont_subclass_hash - Track memory consumption in a Counter.

Repository: lucy
Updated Branches:
  refs/heads/264_dont_subclass_hash 1622b4e4f -> d136b8675


Track memory consumption in a Counter.

Anticipating the elimination of MemoryPool, duplicate tracking of memory
consumption in a simple "Counter" object.


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

Branch: refs/heads/264_dont_subclass_hash
Commit: 1f159329056d0bc549f799f2dd5ed0461f65679e
Parents: 1622b4e
Author: Marvin Humphrey <ma...@rectangular.com>
Authored: Wed Jul 2 13:22:31 2014 -0700
Committer: Marvin Humphrey <ma...@rectangular.com>
Committed: Wed Jul 2 18:07:40 2014 -0700

----------------------------------------------------------------------
 core/Lucy/Index/SortFieldWriter.c   | 47 ++++++++++++++++++++++++--------
 core/Lucy/Index/SortFieldWriter.cfh |  5 ++--
 core/Lucy/Index/SortWriter.c        | 39 ++++++++++++++++++++++++--
 core/Lucy/Index/SortWriter.cfh      | 19 +++++++++++++
 4 files changed, 94 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucy/blob/1f159329/core/Lucy/Index/SortFieldWriter.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/SortFieldWriter.c b/core/Lucy/Index/SortFieldWriter.c
index b6faec5..d0c2291 100644
--- a/core/Lucy/Index/SortFieldWriter.c
+++ b/core/Lucy/Index/SortFieldWriter.c
@@ -29,6 +29,7 @@
 #include "Lucy/Index/SortCache/NumericSortCache.h"
 #include "Lucy/Index/SortCache/TextSortCache.h"
 #include "Lucy/Index/SortReader.h"
+#include "Lucy/Index/SortWriter.h"
 #include "Lucy/Index/ZombieKeyedHash.h"
 #include "Lucy/Plan/FieldType.h"
 #include "Lucy/Plan/Schema.h"
@@ -54,18 +55,28 @@ S_write_files(SortFieldWriter *self, OutStream *ord_out, OutStream *ix_out,
 // allocation itself will come from the MemoryPool, so the the element will be
 // deallocated via MemPool_Release_All().
 static SFWriterElem*
-S_SFWriterElem_create(MemoryPool *mem_pool, Obj *value, int32_t doc_id);
+S_SFWriterElem_create(MemoryPool *mem_pool, Counter *counter, Obj *value, int32_t doc_id);
+
+static int64_t
+SI_increase_to_word_multiple(int64_t amount) {
+    const int64_t remainder = amount % sizeof(void*);
+    if (remainder) {
+        amount += sizeof(void*);
+        amount -= remainder;
+    }
+    return amount;
+}
 
 SortFieldWriter*
 SortFieldWriter_new(Schema *schema, Snapshot *snapshot, Segment *segment,
                     PolyReader *polyreader, String *field,
-                    MemoryPool *memory_pool, size_t mem_thresh,
+                    MemoryPool *memory_pool, Counter *counter, size_t mem_thresh,
                     OutStream *temp_ord_out, OutStream *temp_ix_out,
                     OutStream *temp_dat_out) {
     SortFieldWriter *self
         = (SortFieldWriter*)VTable_Make_Obj(SORTFIELDWRITER);
     return SortFieldWriter_init(self, schema, snapshot, segment, polyreader,
-                                field, memory_pool, mem_thresh, temp_ord_out,
+                                field, memory_pool, counter, mem_thresh, temp_ord_out,
                                 temp_ix_out, temp_dat_out);
 }
 
@@ -73,7 +84,7 @@ SortFieldWriter*
 SortFieldWriter_init(SortFieldWriter *self, Schema *schema,
                      Snapshot *snapshot, Segment *segment,
                      PolyReader *polyreader, String *field,
-                     MemoryPool *memory_pool, size_t mem_thresh,
+                     MemoryPool *memory_pool, Counter *counter, size_t mem_thresh,
                      OutStream *temp_ord_out, OutStream *temp_ix_out,
                      OutStream *temp_dat_out) {
     // Init.
@@ -103,6 +114,7 @@ SortFieldWriter_init(SortFieldWriter *self, Schema *schema,
     ivars->segment      = (Segment*)INCREF(segment);
     ivars->polyreader   = (PolyReader*)INCREF(polyreader);
     ivars->mem_pool     = (MemoryPool*)INCREF(memory_pool);
+    ivars->counter      = (Counter*)INCREF(counter);
     ivars->temp_ord_out = (OutStream*)INCREF(temp_ord_out);
     ivars->temp_ix_out  = (OutStream*)INCREF(temp_ix_out);
     ivars->temp_dat_out = (OutStream*)INCREF(temp_dat_out);
@@ -158,6 +170,7 @@ SortFieldWriter_Destroy_IMP(SortFieldWriter *self) {
     DECREF(ivars->polyreader);
     DECREF(ivars->type);
     DECREF(ivars->mem_pool);
+    DECREF(ivars->counter);
     DECREF(ivars->temp_ord_out);
     DECREF(ivars->temp_ix_out);
     DECREF(ivars->temp_dat_out);
@@ -181,11 +194,18 @@ SortFieldWriter_Get_Ord_Width_IMP(SortFieldWriter *self) {
 }
 
 static Obj*
-S_find_unique_value(Hash *uniq_vals, Obj *val) {
+S_find_unique_value(Hash *uniq_vals, Counter *counter, Obj *val) {
     int32_t  hash_sum  = Obj_Hash_Sum(val);
     Obj     *uniq_val  = Hash_Find_Key(uniq_vals, val, hash_sum);
     if (!uniq_val) {
         Hash_Store(uniq_vals, val, (Obj*)CFISH_TRUE);
+        VTable *vtable = Obj_Get_VTable(val);
+        Counter_Add(counter, VTable_Get_Obj_Alloc_Size(vtable));
+        if (vtable == STRING) {
+            int64_t size = Str_Get_Size((String*)val) + 1;
+            size = SI_increase_to_word_multiple(size);
+            Counter_Add(counter, size);
+        }
         uniq_val = Hash_Find_Key(uniq_vals, val, hash_sum);
     }
     return uniq_val;
@@ -196,8 +216,8 @@ SortFieldWriter_Add_IMP(SortFieldWriter *self, int32_t doc_id, Obj *value) {
     SortFieldWriterIVARS *const ivars = SortFieldWriter_IVARS(self);
 
     // Uniq-ify the value, and record it for this document.
-    Obj *copy = S_find_unique_value(ivars->uniq_vals, value);
-    SFWriterElem *elem = S_SFWriterElem_create(ivars->mem_pool, copy, doc_id);
+    Obj *copy = S_find_unique_value(ivars->uniq_vals, ivars->counter, value);
+    SFWriterElem *elem = S_SFWriterElem_create(ivars->mem_pool, ivars->counter, copy, doc_id);
     SortFieldWriter_Feed(self, (Obj*)elem);
     ivars->count++;
 }
@@ -209,7 +229,7 @@ SortFieldWriter_Add_Segment_IMP(SortFieldWriter *self, SegReader *reader,
     SortFieldWriterIVARS *const ivars = SortFieldWriter_IVARS(self);
     SortFieldWriter *run
         = SortFieldWriter_new(ivars->schema, ivars->snapshot, ivars->segment,
-                              ivars->polyreader, ivars->field, ivars->mem_pool,
+                              ivars->polyreader, ivars->field, ivars->mem_pool, ivars->counter,
                               ivars->mem_thresh, NULL, NULL, NULL);
     SortFieldWriterIVARS *const run_ivars = SortFieldWriter_IVARS(run);
     run_ivars->sort_cache = (SortCache*)INCREF(sort_cache);
@@ -400,7 +420,7 @@ SortFieldWriter_Flush_IMP(SortFieldWriter *self) {
     SortFieldWriter_Sort_Buffer(self);
     SortFieldWriter *run
         = SortFieldWriter_new(ivars->schema, ivars->snapshot, ivars->segment,
-                              ivars->polyreader, ivars->field, ivars->mem_pool,
+                              ivars->polyreader, ivars->field, ivars->mem_pool, ivars->counter,
                               ivars->mem_thresh, NULL, NULL, NULL);
     SortFieldWriterIVARS *const run_ivars = SortFieldWriter_IVARS(run);
 
@@ -454,16 +474,16 @@ SortFieldWriter_Refill_IMP(SortFieldWriter *self) {
     }
     SortFieldWriter_Clear_Buffer(self);
     MemPool_Release_All(ivars->mem_pool);
+    Counter_Reset(ivars->counter);
     S_lazy_init_sorted_ids(self);
 
     const int32_t    null_ord   = ivars->null_ord;
-    Hash *const      uniq_vals  = ivars->uniq_vals;
     I32Array *const  doc_map    = ivars->doc_map;
     SortCache *const sort_cache = ivars->sort_cache;
 
     uint32_t count = 0;
     while (ivars->run_tick <= ivars->run_max
-           && MemPool_Get_Consumed(ivars->mem_pool) < ivars->mem_thresh
+           && Counter_Get_Value(ivars->counter) < ivars->mem_thresh
           ) {
         int32_t raw_doc_id = ivars->sorted_ids[ivars->run_tick];
         int32_t ord = SortCache_Ordinal(sort_cache, raw_doc_id);
@@ -675,7 +695,9 @@ S_flip_run(SortFieldWriter *run, size_t sub_thresh, InStream *ord_in,
     // Get our own MemoryPool, ZombieKeyedHash, and slice of mem_thresh.
     DECREF(run_ivars->uniq_vals);
     DECREF(run_ivars->mem_pool);
+    DECREF(run_ivars->counter);
     run_ivars->mem_pool   = MemPool_new(0);
+    run_ivars->counter    = Counter_new();
     run_ivars->uniq_vals  = (Hash*)ZKHash_new(run_ivars->mem_pool, run_ivars->prim_id);
     run_ivars->mem_thresh = sub_thresh;
 
@@ -755,8 +777,9 @@ S_flip_run(SortFieldWriter *run, size_t sub_thresh, InStream *ord_in,
 /***************************************************************************/
 
 static SFWriterElem*
-S_SFWriterElem_create(MemoryPool *mem_pool, Obj *value, int32_t doc_id) {
+S_SFWriterElem_create(MemoryPool *mem_pool, Counter *counter, Obj *value, int32_t doc_id) {
     size_t size = VTable_Get_Obj_Alloc_Size(SFWRITERELEM);
+    Counter_Add(counter, size);
     SFWriterElem *self = (SFWriterElem*)MemPool_Grab(mem_pool, size);
     VTable_Init_Obj(SFWRITERELEM, (Obj*)self);
     SFWriterElemIVARS *ivars = SFWriterElem_IVARS(self);

http://git-wip-us.apache.org/repos/asf/lucy/blob/1f159329/core/Lucy/Index/SortFieldWriter.cfh
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/SortFieldWriter.cfh b/core/Lucy/Index/SortFieldWriter.cfh
index 0128c61..7ca6b5a 100644
--- a/core/Lucy/Index/SortFieldWriter.cfh
+++ b/core/Lucy/Index/SortFieldWriter.cfh
@@ -27,6 +27,7 @@ class Lucy::Index::SortFieldWriter
     FieldType  *type;
     I32Array   *doc_map;
     MemoryPool *mem_pool;
+    Counter    *counter;
     int32_t     field_num;
     int32_t     null_ord;
     int8_t      prim_id;
@@ -54,14 +55,14 @@ class Lucy::Index::SortFieldWriter
 
     inert incremented SortFieldWriter*
     new(Schema *schema, Snapshot *snapshot, Segment *segment,
-        PolyReader *polyreader, String *field, MemoryPool *memory_pool,
+        PolyReader *polyreader, String *field, MemoryPool *memory_pool, Counter *counter,
         size_t mem_thresh, OutStream *temp_ord_out, OutStream *temp_ix_out,
         OutStream *temp_dat_out);
 
     inert SortFieldWriter*
     init(SortFieldWriter *self, Schema *schema, Snapshot *snapshot,
          Segment *segment, PolyReader *polyreader, String *field,
-         MemoryPool *memory_pool, size_t mem_thresh, OutStream *temp_ord_out,
+         MemoryPool *memory_pool, Counter *counter, size_t mem_thresh, OutStream *temp_ord_out,
          OutStream *temp_ix_out, OutStream *temp_dat_out);
 
     void

http://git-wip-us.apache.org/repos/asf/lucy/blob/1f159329/core/Lucy/Index/SortWriter.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/SortWriter.c b/core/Lucy/Index/SortWriter.c
index fe1641c..fc323d0 100644
--- a/core/Lucy/Index/SortWriter.c
+++ b/core/Lucy/Index/SortWriter.c
@@ -15,6 +15,7 @@
  */
 
 #define C_LUCY_SORTWRITER
+#define C_LUCY_COUNTER
 #include "Lucy/Util/ToolSet.h"
 #include <math.h>
 
@@ -63,6 +64,7 @@ SortWriter_init(SortWriter *self, Schema *schema, Snapshot *snapshot,
     ivars->temp_ix_out     = NULL;
     ivars->temp_dat_out    = NULL;
     ivars->mem_pool        = MemPool_new(0);
+    ivars->counter         = Counter_new();
     ivars->mem_thresh      = default_mem_thresh;
     ivars->flush_at_finish = false;
 
@@ -80,6 +82,7 @@ SortWriter_Destroy_IMP(SortWriter *self) {
     DECREF(ivars->temp_ix_out);
     DECREF(ivars->temp_dat_out);
     DECREF(ivars->mem_pool);
+    DECREF(ivars->counter);
     SUPER_DESTROY(self, SORTWRITER);
 }
 
@@ -123,7 +126,7 @@ S_lazy_init_field_writer(SortWriter *self, int32_t field_num) {
         String *field = Seg_Field_Name(ivars->segment, field_num);
         field_writer
             = SortFieldWriter_new(ivars->schema, ivars->snapshot, ivars->segment,
-                                  ivars->polyreader, field, ivars->mem_pool,
+                                  ivars->polyreader, field, ivars->mem_pool, ivars->counter,
                                   ivars->mem_thresh, ivars->temp_ord_out,
                                   ivars->temp_ix_out, ivars->temp_dat_out);
         VA_Store(ivars->field_writers, field_num, (Obj*)field_writer);
@@ -150,13 +153,14 @@ SortWriter_Add_Inverted_Doc_IMP(SortWriter *self, Inverter *inverter,
 
     // If our SortFieldWriters have collectively passed the memory threshold,
     // flush all of them, then release all unique values with a single action.
-    if (MemPool_Get_Consumed(ivars->mem_pool) > ivars->mem_thresh) {
+    if (Counter_Get_Value(ivars->counter) > ivars->mem_thresh) {
         for (uint32_t i = 0; i < VA_Get_Size(ivars->field_writers); i++) {
             SortFieldWriter *const field_writer
                 = (SortFieldWriter*)VA_Fetch(ivars->field_writers, i);
             if (field_writer) { SortFieldWriter_Flush(field_writer); }
         }
         MemPool_Release_All(ivars->mem_pool);
+        Counter_Reset(ivars->counter);
         ivars->flush_at_finish = true;
     }
 }
@@ -272,4 +276,35 @@ SortWriter_Format_IMP(SortWriter *self) {
     return SortWriter_current_file_format;
 }
 
+/*************************************************************************/
+
+Counter*
+Counter_new() {
+    Counter *self = (Counter*)VTable_Make_Obj(COUNTER);
+    return Counter_init(self);
+}
+
+Counter*
+Counter_init(Counter *self) {
+    CounterIVARS *ivars = Counter_IVARS(self);
+    ivars->value = 0;
+    return self;
+}
+
+int64_t
+Counter_Add_IMP(Counter *self, int64_t amount) {
+    CounterIVARS *ivars = Counter_IVARS(self);
+    ivars->value += amount;
+    return ivars->value;
+}
+
+int64_t
+Counter_Get_Value_IMP(Counter *self) {
+    return Counter_IVARS(self)->value;
+}
+
+void
+Counter_Reset_IMP(Counter *self) {
+    Counter_IVARS(self)->value = 0;
+}
 

http://git-wip-us.apache.org/repos/asf/lucy/blob/1f159329/core/Lucy/Index/SortWriter.cfh
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/SortWriter.cfh b/core/Lucy/Index/SortWriter.cfh
index ab3ec91..ba9bd35 100644
--- a/core/Lucy/Index/SortWriter.cfh
+++ b/core/Lucy/Index/SortWriter.cfh
@@ -36,6 +36,7 @@ class Lucy::Index::SortWriter inherits Lucy::Index::DataWriter {
     OutStream  *temp_ix_out;
     OutStream  *temp_dat_out;
     MemoryPool *mem_pool;
+    Counter    *counter;
     size_t      mem_thresh;
     bool        flush_at_finish;
 
@@ -73,4 +74,22 @@ class Lucy::Index::SortWriter inherits Lucy::Index::DataWriter {
     Destroy(SortWriter *self);
 }
 
+final class Lucy::Index::Counter inherits Clownfish::Obj {
+    int64_t value;
+
+    inert incremented Counter*
+    new();
+
+    inert Counter*
+    init(Counter *self);
+
+    int64_t
+    Add(Counter *self, int64_t amount);
+
+    int64_t
+    Get_Value(Counter *self);
+
+    void
+    Reset(Counter *self);
+}
 


[5/5] git commit: refs/heads/264_dont_subclass_hash - Remove now-unused ZombieKeyedHash.

Posted by ma...@apache.org.
Remove now-unused ZombieKeyedHash.


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

Branch: refs/heads/264_dont_subclass_hash
Commit: d136b8675b7ef0437262bac33fa492eb4ab1954b
Parents: d0910e4
Author: Marvin Humphrey <ma...@rectangular.com>
Authored: Wed Jul 2 18:02:53 2014 -0700
Committer: Marvin Humphrey <ma...@rectangular.com>
Committed: Wed Jul 2 18:08:45 2014 -0700

----------------------------------------------------------------------
 core/Lucy/Index/ZombieKeyedHash.c   | 107 -------------------------------
 core/Lucy/Index/ZombieKeyedHash.cfh |  37 -----------
 2 files changed, 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucy/blob/d136b867/core/Lucy/Index/ZombieKeyedHash.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/ZombieKeyedHash.c b/core/Lucy/Index/ZombieKeyedHash.c
deleted file mode 100644
index c991e66..0000000
--- a/core/Lucy/Index/ZombieKeyedHash.c
+++ /dev/null
@@ -1,107 +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.
- */
-
-#define C_LUCY_ZOMBIEKEYEDHASH
-#include "Lucy/Util/ToolSet.h"
-
-#include "Lucy/Index/ZombieKeyedHash.h"
-#include "Lucy/Plan/FieldType.h"
-#include "Lucy/Util/MemoryPool.h"
-
-ZombieKeyedHash*
-ZKHash_new(MemoryPool *memory_pool, uint8_t primitive_id) {
-    ZombieKeyedHash *self
-        = (ZombieKeyedHash*)VTable_Make_Obj(ZOMBIEKEYEDHASH);
-    Hash_init((Hash*)self, 0);
-    ZombieKeyedHashIVARS *const ivars = ZKHash_IVARS(self);
-    ivars->mem_pool = (MemoryPool*)INCREF(memory_pool);
-    ivars->prim_id  = primitive_id;
-    return self;
-}
-
-void
-ZKHash_Destroy_IMP(ZombieKeyedHash *self) {
-    ZombieKeyedHashIVARS *const ivars = ZKHash_IVARS(self);
-    DECREF(ivars->mem_pool);
-    SUPER_DESTROY(self, ZOMBIEKEYEDHASH);
-}
-
-Obj*
-ZKHash_Make_Key_IMP(ZombieKeyedHash *self, Obj *key, int32_t hash_sum) {
-    ZombieKeyedHashIVARS *const ivars = ZKHash_IVARS(self);
-    UNUSED_VAR(hash_sum);
-    Obj *retval = NULL;
-    switch (ivars->prim_id & FType_PRIMITIVE_ID_MASK) {
-        case FType_TEXT: {
-                String *source = (String*)key;
-                size_t size = SStr_size() + Str_Get_Size(source) + 1;
-                void *allocation = MemPool_Grab(ivars->mem_pool, size);
-                retval = (Obj*)SStr_new_from_str(allocation, size, source);
-            }
-            break;
-        case FType_INT32: {
-                size_t size = VTable_Get_Obj_Alloc_Size(INTEGER32);
-                Integer32 *copy
-                    = (Integer32*)MemPool_Grab(ivars->mem_pool, size);
-                VTable_Init_Obj(INTEGER32, copy);
-                Int32_init(copy, 0);
-                Int32_Mimic(copy, key);
-                retval = (Obj*)copy;
-            }
-            break;
-        case FType_INT64: {
-                size_t size = VTable_Get_Obj_Alloc_Size(INTEGER64);
-                Integer64 *copy
-                    = (Integer64*)MemPool_Grab(ivars->mem_pool, size);
-                VTable_Init_Obj(INTEGER64, copy);
-                Int64_init(copy, 0);
-                Int64_Mimic(copy, key);
-                retval = (Obj*)copy;
-            }
-            break;
-        case FType_FLOAT32: {
-                size_t size = VTable_Get_Obj_Alloc_Size(FLOAT32);
-                Float32 *copy = (Float32*)MemPool_Grab(ivars->mem_pool, size);
-                VTable_Init_Obj(FLOAT32, copy);
-                Float32_init(copy, 0);
-                Float32_Mimic(copy, key);
-                retval = (Obj*)copy;
-            }
-            break;
-        case FType_FLOAT64: {
-                size_t size = VTable_Get_Obj_Alloc_Size(FLOAT64);
-                Float64 *copy = (Float64*)MemPool_Grab(ivars->mem_pool, size);
-                VTable_Init_Obj(FLOAT64, copy);
-                Float64_init(copy, 0);
-                Float64_Mimic(copy, key);
-                retval = (Obj*)copy;
-            }
-            break;
-        default:
-            THROW(ERR, "Unrecognized primitive id: %i8", ivars->prim_id);
-    }
-
-    /* FIXME This is a hack.  It will leak memory if host objects get cached,
-     * which in the present implementation will happen as soon as the refcount
-     * reaches 4.  However, we must never call Destroy() for these objects,
-     * because they will try to free() their initial allocation, which is
-     * invalid because it's part of a MemoryPool arena. */
-    retval = INCREF(retval);
-
-    return retval;
-}
-
-

http://git-wip-us.apache.org/repos/asf/lucy/blob/d136b867/core/Lucy/Index/ZombieKeyedHash.cfh
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/ZombieKeyedHash.cfh b/core/Lucy/Index/ZombieKeyedHash.cfh
deleted file mode 100644
index a82dcbc..0000000
--- a/core/Lucy/Index/ZombieKeyedHash.cfh
+++ /dev/null
@@ -1,37 +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.
- */
-
-parcel Lucy;
-
-/** Hash which creates keys allocated from a MemoryPool rather than malloc.
- */
-class Lucy::Index::SortFieldWriter::ZombieKeyedHash nickname ZKHash
-    inherits Clownfish::Hash {
-
-    MemoryPool *mem_pool;
-    uint8_t     prim_id;
-
-    inert incremented ZombieKeyedHash*
-    new(MemoryPool *memory_pool, uint8_t primitive_id);
-
-    public void
-    Destroy(ZombieKeyedHash *self);
-
-    public incremented Obj*
-    Make_Key(ZombieKeyedHash *self, Obj *key, int32_t hash_sum);
-}
-
-


[4/5] git commit: refs/heads/264_dont_subclass_hash - Eliminate MemoryPool from SortFieldWriter.

Posted by ma...@apache.org.
Eliminate MemoryPool from SortFieldWriter.

SortFieldWriter used to allocate values from the MemoryPool, which was
also used to track memory usage.  Now it uses ordinary objects and
memory consumption is tracked in a simple Counter, so the MemoryPool is
no longer needed.


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

Branch: refs/heads/264_dont_subclass_hash
Commit: d0910e438b24b5cc9c8e04781cea96ed29f94d3d
Parents: 2f32088
Author: Marvin Humphrey <ma...@rectangular.com>
Authored: Wed Jul 2 17:50:46 2014 -0700
Committer: Marvin Humphrey <ma...@rectangular.com>
Committed: Wed Jul 2 18:08:44 2014 -0700

----------------------------------------------------------------------
 core/Lucy/Index/SortFieldWriter.c   | 22 ++++++++--------------
 core/Lucy/Index/SortFieldWriter.cfh |  5 ++---
 core/Lucy/Index/SortWriter.c        |  9 +++------
 core/Lucy/Index/SortWriter.cfh      |  1 -
 4 files changed, 13 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucy/blob/d0910e43/core/Lucy/Index/SortFieldWriter.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/SortFieldWriter.c b/core/Lucy/Index/SortFieldWriter.c
index 40c896a..fc350ba 100644
--- a/core/Lucy/Index/SortFieldWriter.c
+++ b/core/Lucy/Index/SortFieldWriter.c
@@ -36,7 +36,6 @@
 #include "Lucy/Store/InStream.h"
 #include "Lucy/Store/OutStream.h"
 #include "Clownfish/Util/Memory.h"
-#include "Lucy/Util/MemoryPool.h"
 #include "Clownfish/Util/SortUtils.h"
 
 // Prepare to read back a run.
@@ -67,13 +66,13 @@ SI_increase_to_word_multiple(int64_t amount) {
 SortFieldWriter*
 SortFieldWriter_new(Schema *schema, Snapshot *snapshot, Segment *segment,
                     PolyReader *polyreader, String *field,
-                    MemoryPool *memory_pool, Counter *counter, size_t mem_thresh,
+                    Counter *counter, size_t mem_thresh,
                     OutStream *temp_ord_out, OutStream *temp_ix_out,
                     OutStream *temp_dat_out) {
     SortFieldWriter *self
         = (SortFieldWriter*)VTable_Make_Obj(SORTFIELDWRITER);
     return SortFieldWriter_init(self, schema, snapshot, segment, polyreader,
-                                field, memory_pool, counter, mem_thresh, temp_ord_out,
+                                field, counter, mem_thresh, temp_ord_out,
                                 temp_ix_out, temp_dat_out);
 }
 
@@ -81,7 +80,7 @@ SortFieldWriter*
 SortFieldWriter_init(SortFieldWriter *self, Schema *schema,
                      Snapshot *snapshot, Segment *segment,
                      PolyReader *polyreader, String *field,
-                     MemoryPool *memory_pool, Counter *counter, size_t mem_thresh,
+                     Counter *counter, size_t mem_thresh,
                      OutStream *temp_ord_out, OutStream *temp_ix_out,
                      OutStream *temp_dat_out) {
     // Init.
@@ -109,7 +108,6 @@ SortFieldWriter_init(SortFieldWriter *self, Schema *schema,
     ivars->snapshot     = (Snapshot*)INCREF(snapshot);
     ivars->segment      = (Segment*)INCREF(segment);
     ivars->polyreader   = (PolyReader*)INCREF(polyreader);
-    ivars->mem_pool     = (MemoryPool*)INCREF(memory_pool);
     ivars->counter      = (Counter*)INCREF(counter);
     ivars->temp_ord_out = (OutStream*)INCREF(temp_ord_out);
     ivars->temp_ix_out  = (OutStream*)INCREF(temp_ix_out);
@@ -142,10 +140,10 @@ SortFieldWriter_Clear_Buffer_IMP(SortFieldWriter *self) {
     SortFieldWriter_Clear_Buffer_t super_clear_buffer
         = SUPER_METHOD_PTR(SORTFIELDWRITER, LUCY_SortFieldWriter_Clear_Buffer);
     super_clear_buffer(self);
-    // Note that we have not called MemPool_Release_All() on our memory pool.
-    // This is because the pool is shared amongst multiple SortFieldWriters
+    // Note that we have not Reset() the Counter which tracks memory usage.
+    // This is because the counter is shared amongst multiple SortFieldWriters
     // which belong to a parent SortWriter; it is the responsibility of the
-    // parent SortWriter to release the memory pool once **all** of its child
+    // parent SortWriter to reset it once **all** of its child
     // SortFieldWriters have cleared their buffers.
 }
 
@@ -160,7 +158,6 @@ SortFieldWriter_Destroy_IMP(SortFieldWriter *self) {
     DECREF(ivars->segment);
     DECREF(ivars->polyreader);
     DECREF(ivars->type);
-    DECREF(ivars->mem_pool);
     DECREF(ivars->counter);
     DECREF(ivars->temp_ord_out);
     DECREF(ivars->temp_ix_out);
@@ -220,7 +217,7 @@ SortFieldWriter_Add_Segment_IMP(SortFieldWriter *self, SegReader *reader,
     SortFieldWriterIVARS *const ivars = SortFieldWriter_IVARS(self);
     SortFieldWriter *run
         = SortFieldWriter_new(ivars->schema, ivars->snapshot, ivars->segment,
-                              ivars->polyreader, ivars->field, ivars->mem_pool, ivars->counter,
+                              ivars->polyreader, ivars->field, ivars->counter,
                               ivars->mem_thresh, NULL, NULL, NULL);
     SortFieldWriterIVARS *const run_ivars = SortFieldWriter_IVARS(run);
     run_ivars->sort_cache = (SortCache*)INCREF(sort_cache);
@@ -411,7 +408,7 @@ SortFieldWriter_Flush_IMP(SortFieldWriter *self) {
     SortFieldWriter_Sort_Buffer(self);
     SortFieldWriter *run
         = SortFieldWriter_new(ivars->schema, ivars->snapshot, ivars->segment,
-                              ivars->polyreader, ivars->field, ivars->mem_pool, ivars->counter,
+                              ivars->polyreader, ivars->field, ivars->counter,
                               ivars->mem_thresh, NULL, NULL, NULL);
     SortFieldWriterIVARS *const run_ivars = SortFieldWriter_IVARS(run);
 
@@ -464,7 +461,6 @@ SortFieldWriter_Refill_IMP(SortFieldWriter *self) {
               buf_count);
     }
     SortFieldWriter_Clear_Buffer(self);
-    MemPool_Release_All(ivars->mem_pool);
     Counter_Reset(ivars->counter);
     S_lazy_init_sorted_ids(self);
 
@@ -683,9 +679,7 @@ S_flip_run(SortFieldWriter *run, size_t sub_thresh, InStream *ord_in,
     run_ivars->flipped = true;
 
     // Get our own slice of mem_thresh.
-    DECREF(run_ivars->mem_pool);
     DECREF(run_ivars->counter);
-    run_ivars->mem_pool   = MemPool_new(0);
     run_ivars->counter    = Counter_new();
     run_ivars->mem_thresh = sub_thresh;
 

http://git-wip-us.apache.org/repos/asf/lucy/blob/d0910e43/core/Lucy/Index/SortFieldWriter.cfh
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/SortFieldWriter.cfh b/core/Lucy/Index/SortFieldWriter.cfh
index df82067..e2b996f 100644
--- a/core/Lucy/Index/SortFieldWriter.cfh
+++ b/core/Lucy/Index/SortFieldWriter.cfh
@@ -26,7 +26,6 @@ class Lucy::Index::SortFieldWriter
     PolyReader *polyreader;
     FieldType  *type;
     I32Array   *doc_map;
-    MemoryPool *mem_pool;
     Counter    *counter;
     int32_t     field_num;
     int32_t     null_ord;
@@ -55,14 +54,14 @@ class Lucy::Index::SortFieldWriter
 
     inert incremented SortFieldWriter*
     new(Schema *schema, Snapshot *snapshot, Segment *segment,
-        PolyReader *polyreader, String *field, MemoryPool *memory_pool, Counter *counter,
+        PolyReader *polyreader, String *field, Counter *counter,
         size_t mem_thresh, OutStream *temp_ord_out, OutStream *temp_ix_out,
         OutStream *temp_dat_out);
 
     inert SortFieldWriter*
     init(SortFieldWriter *self, Schema *schema, Snapshot *snapshot,
          Segment *segment, PolyReader *polyreader, String *field,
-         MemoryPool *memory_pool, Counter *counter, size_t mem_thresh, OutStream *temp_ord_out,
+         Counter *counter, size_t mem_thresh, OutStream *temp_ord_out,
          OutStream *temp_ix_out, OutStream *temp_dat_out);
 
     void

http://git-wip-us.apache.org/repos/asf/lucy/blob/d0910e43/core/Lucy/Index/SortWriter.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/SortWriter.c b/core/Lucy/Index/SortWriter.c
index fc323d0..bd3b690 100644
--- a/core/Lucy/Index/SortWriter.c
+++ b/core/Lucy/Index/SortWriter.c
@@ -34,7 +34,6 @@
 #include "Lucy/Store/InStream.h"
 #include "Lucy/Store/OutStream.h"
 #include "Clownfish/Util/Memory.h"
-#include "Lucy/Util/MemoryPool.h"
 #include "Clownfish/Util/SortUtils.h"
 
 int32_t SortWriter_current_file_format = 3;
@@ -63,7 +62,6 @@ SortWriter_init(SortWriter *self, Schema *schema, Snapshot *snapshot,
     ivars->temp_ord_out    = NULL;
     ivars->temp_ix_out     = NULL;
     ivars->temp_dat_out    = NULL;
-    ivars->mem_pool        = MemPool_new(0);
     ivars->counter         = Counter_new();
     ivars->mem_thresh      = default_mem_thresh;
     ivars->flush_at_finish = false;
@@ -81,7 +79,6 @@ SortWriter_Destroy_IMP(SortWriter *self) {
     DECREF(ivars->temp_ord_out);
     DECREF(ivars->temp_ix_out);
     DECREF(ivars->temp_dat_out);
-    DECREF(ivars->mem_pool);
     DECREF(ivars->counter);
     SUPER_DESTROY(self, SORTWRITER);
 }
@@ -126,7 +123,7 @@ S_lazy_init_field_writer(SortWriter *self, int32_t field_num) {
         String *field = Seg_Field_Name(ivars->segment, field_num);
         field_writer
             = SortFieldWriter_new(ivars->schema, ivars->snapshot, ivars->segment,
-                                  ivars->polyreader, field, ivars->mem_pool, ivars->counter,
+                                  ivars->polyreader, field, ivars->counter,
                                   ivars->mem_thresh, ivars->temp_ord_out,
                                   ivars->temp_ix_out, ivars->temp_dat_out);
         VA_Store(ivars->field_writers, field_num, (Obj*)field_writer);
@@ -152,14 +149,14 @@ SortWriter_Add_Inverted_Doc_IMP(SortWriter *self, Inverter *inverter,
     }
 
     // If our SortFieldWriters have collectively passed the memory threshold,
-    // flush all of them, then release all unique values with a single action.
+    // flush all of them, then reset the counter which tracks memory
+    // consumption.
     if (Counter_Get_Value(ivars->counter) > ivars->mem_thresh) {
         for (uint32_t i = 0; i < VA_Get_Size(ivars->field_writers); i++) {
             SortFieldWriter *const field_writer
                 = (SortFieldWriter*)VA_Fetch(ivars->field_writers, i);
             if (field_writer) { SortFieldWriter_Flush(field_writer); }
         }
-        MemPool_Release_All(ivars->mem_pool);
         Counter_Reset(ivars->counter);
         ivars->flush_at_finish = true;
     }

http://git-wip-us.apache.org/repos/asf/lucy/blob/d0910e43/core/Lucy/Index/SortWriter.cfh
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/SortWriter.cfh b/core/Lucy/Index/SortWriter.cfh
index ba9bd35..6f4a242 100644
--- a/core/Lucy/Index/SortWriter.cfh
+++ b/core/Lucy/Index/SortWriter.cfh
@@ -35,7 +35,6 @@ class Lucy::Index::SortWriter inherits Lucy::Index::DataWriter {
     OutStream  *temp_ord_out;
     OutStream  *temp_ix_out;
     OutStream  *temp_dat_out;
-    MemoryPool *mem_pool;
     Counter    *counter;
     size_t      mem_thresh;
     bool        flush_at_finish;


[2/5] git commit: refs/heads/264_dont_subclass_hash - Use an ordinary Hash for uniquing values.

Posted by ma...@apache.org.
Use an ordinary Hash for uniquing values.

Stop using a ZombieKeyedHash, with values allocated from a MemoryPool,
and instead use ordinary String, etc.


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

Branch: refs/heads/264_dont_subclass_hash
Commit: 2f32088b4277d1fb205753573b86fc29e32ab214
Parents: 3ca75ec
Author: Marvin Humphrey <ma...@rectangular.com>
Authored: Wed Jul 2 17:11:04 2014 -0700
Committer: Marvin Humphrey <ma...@rectangular.com>
Committed: Wed Jul 2 18:08:44 2014 -0700

----------------------------------------------------------------------
 core/Lucy/Index/SortFieldWriter.c | 36 ++++++++++++----------------------
 1 file changed, 12 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucy/blob/2f32088b/core/Lucy/Index/SortFieldWriter.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/SortFieldWriter.c b/core/Lucy/Index/SortFieldWriter.c
index 0183ceb..40c896a 100644
--- a/core/Lucy/Index/SortFieldWriter.c
+++ b/core/Lucy/Index/SortFieldWriter.c
@@ -30,7 +30,6 @@
 #include "Lucy/Index/SortCache/TextSortCache.h"
 #include "Lucy/Index/SortReader.h"
 #include "Lucy/Index/SortWriter.h"
-#include "Lucy/Index/ZombieKeyedHash.h"
 #include "Lucy/Plan/FieldType.h"
 #include "Lucy/Plan/Schema.h"
 #include "Lucy/Store/Folder.h"
@@ -51,9 +50,7 @@ static int32_t
 S_write_files(SortFieldWriter *self, OutStream *ord_out, OutStream *ix_out,
               OutStream *dat_out);
 
-// Create an element for the sort pool.  Both the `value` and the object
-// allocation itself will come from the MemoryPool, so the the element will be
-// deallocated via MemPool_Release_All().
+// Constructor for an element in the sort pool.
 static SFWriterElem*
 S_SFWriterElem_create(Obj *value, int32_t doc_id);
 
@@ -105,7 +102,6 @@ SortFieldWriter_init(SortFieldWriter *self, Schema *schema,
     ivars->sorted_ids      = NULL;
     ivars->run_tick        = 1;
     ivars->ord_width       = 0;
-    ivars->last_val        = NULL;
 
     // Assign.
     ivars->field        = Str_Clone(field);
@@ -132,7 +128,7 @@ SortFieldWriter_init(SortFieldWriter *self, Schema *schema,
     else {
         ivars->var_width = false;
     }
-    ivars->uniq_vals = (Hash*)ZKHash_new(memory_pool, ivars->prim_id);
+    ivars->uniq_vals = Hash_new(0);
 
     return self;
 }
@@ -141,11 +137,6 @@ void
 SortFieldWriter_Clear_Buffer_IMP(SortFieldWriter *self) {
     SortFieldWriterIVARS *const ivars = SortFieldWriter_IVARS(self);
     if (ivars->uniq_vals) {
-        if (ivars->last_val) {
-            Obj *clone = Obj_Clone(ivars->last_val);
-            DECREF(ivars->last_val);
-            ivars->last_val = clone;
-        }
         Hash_Clear(ivars->uniq_vals);
     }
     SortFieldWriter_Clear_Buffer_t super_clear_buffer
@@ -585,30 +576,27 @@ S_write_files(SortFieldWriter *self, OutStream *ord_out, OutStream *ix_out,
     ords[0] = 0;
 
     // Build array of ords, write non-NULL sorted values.
-    ivars->last_val = INCREF(elem_ivars->value);
-    Obj *last_val_address = elem_ivars->value;
+    Obj *last_val = INCREF(elem_ivars->value);
     S_write_val(elem_ivars->value, prim_id, ix_out, dat_out, dat_start);
     DECREF(elem);
     while (NULL != (elem = (SFWriterElem*)SortFieldWriter_Fetch(self))) {
         elem_ivars = SFWriterElem_IVARS(elem);
-        if (elem_ivars->value != last_val_address) {
+        if (elem_ivars->value != last_val) {
             int32_t comparison
                 = FType_Compare_Values(ivars->type, elem_ivars->value,
-                                       ivars->last_val);
+                                       last_val);
             if (comparison != 0) {
                 ord++;
                 S_write_val(elem_ivars->value, prim_id, ix_out, dat_out,
                             dat_start);
-                DECREF(ivars->last_val);
-                ivars->last_val = INCREF(elem_ivars->value);
             }
-            last_val_address = elem_ivars->value;
+            DECREF(last_val);
+            last_val = INCREF(elem_ivars->value);
         }
         ords[elem_ivars->doc_id] = ord;
         DECREF(elem);
     }
-    DECREF(ivars->last_val);
-    ivars->last_val = NULL;
+    DECREF(last_val);
 
     // If there are NULL values, write one now and record the NULL ord.
     if (has_nulls) {
@@ -694,13 +682,11 @@ S_flip_run(SortFieldWriter *run, size_t sub_thresh, InStream *ord_in,
     if (run_ivars->flipped) { THROW(ERR, "Can't Flip twice"); }
     run_ivars->flipped = true;
 
-    // Get our own MemoryPool, ZombieKeyedHash, and slice of mem_thresh.
-    DECREF(run_ivars->uniq_vals);
+    // Get our own slice of mem_thresh.
     DECREF(run_ivars->mem_pool);
     DECREF(run_ivars->counter);
     run_ivars->mem_pool   = MemPool_new(0);
     run_ivars->counter    = Counter_new();
-    run_ivars->uniq_vals  = (Hash*)ZKHash_new(run_ivars->mem_pool, run_ivars->prim_id);
     run_ivars->mem_thresh = sub_thresh;
 
     // Done if we already have a SortCache to read from.
@@ -782,13 +768,15 @@ static SFWriterElem*
 S_SFWriterElem_create(Obj *value, int32_t doc_id) {
     SFWriterElem *self = (SFWriterElem*)VTable_Make_Obj(SFWRITERELEM);
     SFWriterElemIVARS *ivars = SFWriterElem_IVARS(self);
-    ivars->value  = value;
+    ivars->value  = INCREF(value);
     ivars->doc_id = doc_id;
     return self;
 }
 
 void
 SFWriterElem_Destroy_IMP(SFWriterElem *self) {
+    SFWriterElemIVARS *ivars = SFWriterElem_IVARS(self);
+    DECREF(ivars->value);
     SUPER_DESTROY(self, SFWRITERELEM);
 }
 


[3/5] git commit: refs/heads/264_dont_subclass_hash - Make SFWriterElem an ordinary object.

Posted by ma...@apache.org.
Make SFWriterElem an ordinary object.

Stop allocating from MemoryPool.


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

Branch: refs/heads/264_dont_subclass_hash
Commit: 3ca75ec05ce9da0ab7f6634adbd3e04bec12b502
Parents: 1f15932
Author: Marvin Humphrey <ma...@rectangular.com>
Authored: Wed Jul 2 15:13:28 2014 -0700
Committer: Marvin Humphrey <ma...@rectangular.com>
Committed: Wed Jul 2 18:08:44 2014 -0700

----------------------------------------------------------------------
 core/Lucy/Index/SortFieldWriter.c   | 33 +++++++-------------------------
 core/Lucy/Index/SortFieldWriter.cfh | 11 -----------
 2 files changed, 7 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucy/blob/3ca75ec0/core/Lucy/Index/SortFieldWriter.c
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/SortFieldWriter.c b/core/Lucy/Index/SortFieldWriter.c
index d0c2291..0183ceb 100644
--- a/core/Lucy/Index/SortFieldWriter.c
+++ b/core/Lucy/Index/SortFieldWriter.c
@@ -55,7 +55,7 @@ S_write_files(SortFieldWriter *self, OutStream *ord_out, OutStream *ix_out,
 // allocation itself will come from the MemoryPool, so the the element will be
 // deallocated via MemPool_Release_All().
 static SFWriterElem*
-S_SFWriterElem_create(MemoryPool *mem_pool, Counter *counter, Obj *value, int32_t doc_id);
+S_SFWriterElem_create(Obj *value, int32_t doc_id);
 
 static int64_t
 SI_increase_to_word_multiple(int64_t amount) {
@@ -217,7 +217,7 @@ SortFieldWriter_Add_IMP(SortFieldWriter *self, int32_t doc_id, Obj *value) {
 
     // Uniq-ify the value, and record it for this document.
     Obj *copy = S_find_unique_value(ivars->uniq_vals, ivars->counter, value);
-    SFWriterElem *elem = S_SFWriterElem_create(ivars->mem_pool, ivars->counter, copy, doc_id);
+    SFWriterElem *elem = S_SFWriterElem_create(copy, doc_id);
     SortFieldWriter_Feed(self, (Obj*)elem);
     ivars->count++;
 }
@@ -588,6 +588,7 @@ S_write_files(SortFieldWriter *self, OutStream *ord_out, OutStream *ix_out,
     ivars->last_val = INCREF(elem_ivars->value);
     Obj *last_val_address = elem_ivars->value;
     S_write_val(elem_ivars->value, prim_id, ix_out, dat_out, dat_start);
+    DECREF(elem);
     while (NULL != (elem = (SFWriterElem*)SortFieldWriter_Fetch(self))) {
         elem_ivars = SFWriterElem_IVARS(elem);
         if (elem_ivars->value != last_val_address) {
@@ -604,6 +605,7 @@ S_write_files(SortFieldWriter *self, OutStream *ord_out, OutStream *ix_out,
             last_val_address = elem_ivars->value;
         }
         ords[elem_ivars->doc_id] = ord;
+        DECREF(elem);
     }
     DECREF(ivars->last_val);
     ivars->last_val = NULL;
@@ -777,11 +779,8 @@ S_flip_run(SortFieldWriter *run, size_t sub_thresh, InStream *ord_in,
 /***************************************************************************/
 
 static SFWriterElem*
-S_SFWriterElem_create(MemoryPool *mem_pool, Counter *counter, Obj *value, int32_t doc_id) {
-    size_t size = VTable_Get_Obj_Alloc_Size(SFWRITERELEM);
-    Counter_Add(counter, size);
-    SFWriterElem *self = (SFWriterElem*)MemPool_Grab(mem_pool, size);
-    VTable_Init_Obj(SFWRITERELEM, (Obj*)self);
+S_SFWriterElem_create(Obj *value, int32_t doc_id) {
+    SFWriterElem *self = (SFWriterElem*)VTable_Make_Obj(SFWRITERELEM);
     SFWriterElemIVARS *ivars = SFWriterElem_IVARS(self);
     ivars->value  = value;
     ivars->doc_id = doc_id;
@@ -790,24 +789,6 @@ S_SFWriterElem_create(MemoryPool *mem_pool, Counter *counter, Obj *value, int32_
 
 void
 SFWriterElem_Destroy_IMP(SFWriterElem *self) {
-    UNUSED_VAR(self);
-    THROW(ERR, "Illegal attempt to destroy SFWriterElem object");
-}
-
-uint32_t
-SFWriterElem_Get_RefCount_IMP(SFWriterElem* self) {
-    UNUSED_VAR(self);
-    return 1;
-}
-
-SFWriterElem*
-SFWriterElem_Inc_RefCount_IMP(SFWriterElem* self) {
-    return self;
-}
-
-uint32_t
-SFWriterElem_Dec_RefCount_IMP(SFWriterElem* self) {
-    UNUSED_VAR(self);
-    return 1;
+    SUPER_DESTROY(self, SFWRITERELEM);
 }
 

http://git-wip-us.apache.org/repos/asf/lucy/blob/3ca75ec0/core/Lucy/Index/SortFieldWriter.cfh
----------------------------------------------------------------------
diff --git a/core/Lucy/Index/SortFieldWriter.cfh b/core/Lucy/Index/SortFieldWriter.cfh
index 7ca6b5a..df82067 100644
--- a/core/Lucy/Index/SortFieldWriter.cfh
+++ b/core/Lucy/Index/SortFieldWriter.cfh
@@ -104,17 +104,6 @@ class Lucy::Index::SortFieldWriter::SFWriterElem inherits Clownfish::Obj {
     int32_t doc_id;
     Obj *value;
 
-    uint32_t
-    Get_RefCount(SFWriterElem* self);
-
-    incremented SFWriterElem*
-    Inc_RefCount(SFWriterElem* self);
-
-    uint32_t
-    Dec_RefCount(SFWriterElem* self);
-
-    /** Throws an error.
-     */
     public void
     Destroy(SFWriterElem *self);
 }