You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@datasketches.apache.org by al...@apache.org on 2019/09/12 23:32:43 UTC

[incubator-datasketches-cpp] 01/02: converted sketch and union to templates (allocator is not used yet), moved implementation out, return sketch by value

This is an automated email from the ASF dual-hosted git repository.

alsay pushed a commit to branch cpc_template
in repository https://gitbox.apache.org/repos/asf/incubator-datasketches-cpp.git

commit 7d4d833bcba05133dbd01c84f1ab8ad112f8638f
Author: AlexanderSaydakov <Al...@users.noreply.github.com>
AuthorDate: Thu Sep 12 16:31:38 2019 -0700

    converted sketch and union to templates (allocator is not used yet),
    moved implementation out, return sketch by value
---
 cpc/include/cpc_sketch.hpp      | 547 ++++----------------------------------
 cpc/include/cpc_sketch_impl.hpp | 566 ++++++++++++++++++++++++++++++++++++++++
 cpc/include/cpc_union.hpp       |  63 +----
 cpc/include/cpc_union_impl.hpp  |  70 +++++
 cpc/include/fm85Merging.h       |   2 +
 cpc/src/cpc_sketch.cpp          |  27 --
 cpc/src/fm85Merging.cpp         |  12 +
 cpc/test/cpc_sketch_test.cpp    | 142 +++++-----
 cpc/test/cpc_union_test.cpp     |  24 +-
 9 files changed, 799 insertions(+), 654 deletions(-)

diff --git a/cpc/include/cpc_sketch.hpp b/cpc/include/cpc_sketch.hpp
index 06a2ded..203d608 100644
--- a/cpc/include/cpc_sketch.hpp
+++ b/cpc/include/cpc_sketch.hpp
@@ -21,10 +21,7 @@
 #define CPC_SKETCH_HPP_
 
 #include <iostream>
-#include <memory>
 #include <functional>
-#include <stdexcept>
-#include <cmath>
 #include <string>
 
 #if defined(_MSC_VER)
@@ -50,8 +47,9 @@ namespace datasketches {
 
 typedef std::unique_ptr<void, std::function<void(void*)>> ptr_with_deleter;
 
-class cpc_sketch;
-typedef std::unique_ptr<cpc_sketch, void(*)(cpc_sketch*)> cpc_sketch_unique_ptr;
+// forward-declarations
+template<typename A> class cpc_sketch_alloc;
+template<typename A> class cpc_union_alloc;
 
 // allocation and initialization of global compression tables
 // call this before anything else if you want to control the initialization time
@@ -64,460 +62,59 @@ void cpc_init(void* (*alloc)(size_t) = &malloc, void (*dealloc)(void*) = &free);
 // optional deallocation of globally allocated compression tables
 void cpc_cleanup();
 
-class cpc_sketch {
+template<typename A>
+class cpc_sketch_alloc {
   public:
 
-    explicit cpc_sketch(uint8_t lg_k = CPC_DEFAULT_LG_K, uint64_t seed = DEFAULT_SEED) : seed(seed) {
-      fm85Init();
-      if (lg_k < CPC_MIN_LG_K or lg_k > CPC_MAX_LG_K) {
-        throw std::invalid_argument("lg_k must be >= " + std::to_string(CPC_MIN_LG_K) + " and <= " + std::to_string(CPC_MAX_LG_K) + ": " + std::to_string(lg_k));
-      }
-      state = fm85Make(lg_k);
-    }
+    explicit cpc_sketch_alloc(uint8_t lg_k = CPC_DEFAULT_LG_K, uint64_t seed = DEFAULT_SEED);
+    cpc_sketch_alloc(const cpc_sketch_alloc<A>& other);
+    cpc_sketch_alloc<A>& operator=(cpc_sketch_alloc<A> other);
+    ~cpc_sketch_alloc();
 
-    cpc_sketch(const cpc_sketch& other) : state(fm85Copy(other.state)), seed(other.seed) {}
+    bool is_empty() const;
+    double get_estimate() const;
+    double get_lower_bound(unsigned kappa) const;
+    double get_upper_bound(unsigned kappa) const;
 
-    cpc_sketch& operator=(cpc_sketch other) {
-      seed = other.seed;
-      std::swap(state, other.state); // @suppress("Invalid arguments")
-      return *this;
-    }
-
-    ~cpc_sketch() {
-      fm85Free(state);
-    }
-
-    bool is_empty() const {
-      return state->numCoupons == 0;
-    }
-
-    double get_estimate() const {
-      if (!state->mergeFlag) return getHIPEstimate(state);
-      return getIconEstimate(state->lgK, state->numCoupons);
-    }
-
-    double get_lower_bound(unsigned kappa) const {
-      if (kappa > 3) {
-        throw std::invalid_argument("kappa must be 1, 2 or 3");
-      }
-      if (!state->mergeFlag) return getHIPConfidenceLB(state, kappa);
-      return getIconConfidenceLB(state, kappa);
-    }
-
-    double get_upper_bound(unsigned kappa) const {
-      if (kappa > 3) {
-        throw std::invalid_argument("kappa must be 1, 2 or 3");
-      }
-      if (!state->mergeFlag) return getHIPConfidenceUB(state, kappa);
-      return getIconConfidenceUB(state, kappa);
-    }
-
-    void update(const std::string& value) {
-      if (value.empty()) return;
-      update(value.c_str(), value.length());
-    }
-
-    void update(uint64_t value) {
-      update(&value, sizeof(value));
-    }
-
-    void update(int64_t value) {
-      update(&value, sizeof(value));
-    }
-
-    // for compatibility with Java implementation
-    void update(uint32_t value) {
-      update(static_cast<int32_t>(value));
-    }
-
-    // for compatibility with Java implementation
-    void update(int32_t value) {
-      update(static_cast<int64_t>(value));
-    }
-
-    // for compatibility with Java implementation
-    void update(uint16_t value) {
-      update(static_cast<int16_t>(value));
-    }
-
-    // for compatibility with Java implementation
-    void update(int16_t value) {
-      update(static_cast<int64_t>(value));
-    }
-
-    // for compatibility with Java implementation
-    void update(uint8_t value) {
-      update(static_cast<int8_t>(value));
-    }
+    void update(const std::string& value);
+    void update(uint64_t value);
+    void update(int64_t value);
 
     // for compatibility with Java implementation
-    void update(int8_t value) {
-      update(static_cast<int64_t>(value));
-    }
-
-    typedef union {
-      int64_t long_value;
-      double double_value;
-    } long_double_union;
-
-    // for compatibility with Java implementation
-    void update(double value) {
-      long_double_union ldu;
-      if (value == 0.0) {
-        ldu.double_value = 0.0; // canonicalize -0.0 to 0.0
-      } else if (std::isnan(value)) {
-        ldu.long_value = 0x7ff8000000000000L; // canonicalize NaN using value from Java's Double.doubleToLongBits()
-      } else {
-        ldu.double_value = value;
-      }
-      update(&ldu, sizeof(ldu));
-    }
-
-    // for compatibility with Java implementation
-    void update(float value) {
-      update(static_cast<double>(value));
-    }
-
-    // Be very careful to hash input values consistently using the same approach
-    // either over time or on different platforms
-    // or while passing sketches from Java environment or to Java environment
+    void update(uint32_t value);
+    void update(int32_t value);
+    void update(uint16_t value);
+    void update(int16_t value);
+    void update(uint8_t value);
+    void update(int8_t value);
+    void update(double value);
+    void update(float value);
+
+    // This is a "universal" update that covers all cases above, but may produce different hashes
+    // Be very careful to hash input values consistently using the same approach over time,
+    // on different platforms and while passing sketches from or to Java environment
     // Otherwise two sketches that should represent overlapping sets will be disjoint
     // For instance, for signed 32-bit values call update(int32_t) method above,
     // which does widening conversion to int64_t, if compatibility with Java is expected
-    void update(const void* value, int size) {
-      HashState hashes;
-      MurmurHash3_x64_128(value, size, seed, hashes);
-      fm85Update(state, hashes.h1, hashes.h2);
-    }
-
-    void serialize(std::ostream& os) const {
-      FM85* compressed = fm85Compress(state);
-      const uint8_t preamble_ints(get_preamble_ints(compressed));
-      os.write((char*)&preamble_ints, sizeof(preamble_ints));
-      const uint8_t serial_version(SERIAL_VERSION);
-      os.write((char*)&serial_version, sizeof(serial_version));
-      const uint8_t family(FAMILY);
-      os.write((char*)&family, sizeof(family));
-      const uint8_t lg_k(compressed->lgK);
-      os.write((char*)&lg_k, sizeof(lg_k));
-      const uint8_t first_interesting_column(compressed->firstInterestingColumn);
-      os.write((char*)&first_interesting_column, sizeof(first_interesting_column));
-      const bool has_hip(!compressed->mergeFlag);
-      const bool has_table(compressed->compressedSurprisingValues != nullptr);
-      const bool has_window(compressed->compressedWindow != nullptr);
-      const uint8_t flags_byte(
-        (1 << flags::IS_COMPRESSED)
-        | (has_hip ? 1 << flags::HAS_HIP : 0)
-        | (has_table ? 1 << flags::HAS_TABLE : 0)
-        | (has_window ? 1 << flags::HAS_WINDOW : 0)
-      );
-      os.write((char*)&flags_byte, sizeof(flags_byte));
-      const uint16_t seed_hash(compute_seed_hash(seed));
-      os.write((char*)&seed_hash, sizeof(seed_hash));
-      if (!is_empty()) {
-        const uint32_t num_coupons(compressed->numCoupons);
-        os.write((char*)&num_coupons, sizeof(num_coupons));
-        if (has_table && has_window) {
-          // if there is no window it is the same as number of coupons
-          const uint32_t num_values(compressed->numCompressedSurprisingValues);
-          os.write((char*)&num_values, sizeof(num_values));
-          // HIP values are at the same offset because of alignment, which can be in two different places in the sequence of fields
-          // this is the first HIP decision point
-          if (has_hip) write_hip(compressed, os);
-        }
-        if (has_table) {
-          const uint32_t csv_length(compressed->csvLength);
-          os.write((char*)&csv_length, sizeof(csv_length));
-        }
-        if (has_window) {
-          const uint32_t cw_length(compressed->cwLength);
-          os.write((char*)&cw_length, sizeof(cw_length));
-        }
-        // this is the second HIP decision point
-        if (has_hip && !(has_table && has_window)) write_hip(compressed, os);
-        if (has_window) {
-          os.write((char*)compressed->compressedWindow, compressed->cwLength * sizeof(uint32_t));
-        }
-        if (has_table) {
-          os.write((char*)compressed->compressedSurprisingValues, compressed->csvLength * sizeof(uint32_t));
-        }
-      }
-      fm85Free(compressed);
-    }
-
-    std::pair<ptr_with_deleter, const size_t> serialize(unsigned header_size_bytes = 0) const {
-      FM85* compressed = fm85Compress(state);
-      const uint8_t preamble_ints(get_preamble_ints(compressed));
-      const size_t size = header_size_bytes + (preamble_ints + compressed->csvLength + compressed->cwLength) * sizeof(uint32_t);
-      ptr_with_deleter data_ptr(
-          fm85alloc(size),
-          [](void* ptr) { fm85free(ptr); }
-      );
-      char* ptr = static_cast<char*>(data_ptr.get()) + header_size_bytes;
-      ptr += copy_to_mem(ptr, &preamble_ints, sizeof(preamble_ints));
-      const uint8_t serial_version(SERIAL_VERSION);
-      ptr += copy_to_mem(ptr, &serial_version, sizeof(serial_version));
-      const uint8_t family(FAMILY);
-      ptr += copy_to_mem(ptr, &family, sizeof(family));
-      const uint8_t lg_k(compressed->lgK);
-      ptr += copy_to_mem(ptr, &lg_k, sizeof(lg_k));
-      const uint8_t first_interesting_column(compressed->firstInterestingColumn);
-      ptr += copy_to_mem(ptr, &first_interesting_column, sizeof(first_interesting_column));
-      const bool has_hip(!compressed->mergeFlag);
-      const bool has_table(compressed->compressedSurprisingValues != nullptr);
-      const bool has_window(compressed->compressedWindow != nullptr);
-      const uint8_t flags_byte(
-        (1 << flags::IS_COMPRESSED)
-        | (has_hip ? 1 << flags::HAS_HIP : 0)
-        | (has_table ? 1 << flags::HAS_TABLE : 0)
-        | (has_window ? 1 << flags::HAS_WINDOW : 0)
-      );
-      ptr += copy_to_mem(ptr, &flags_byte, sizeof(flags_byte));
-      const uint16_t seed_hash(compute_seed_hash(seed));
-      ptr += copy_to_mem(ptr, &seed_hash, sizeof(seed_hash));
-      if (!is_empty()) {
-        const uint32_t num_coupons(compressed->numCoupons);
-        ptr += copy_to_mem(ptr, &num_coupons, sizeof(num_coupons));
-        if (has_table && has_window) {
-          // if there is no window it is the same as number of coupons
-          const uint32_t num_values(compressed->numCompressedSurprisingValues);
-          ptr += copy_to_mem(ptr, &num_values, sizeof(num_values));
-          // HIP values are at the same offset because of alignment, which can be in two different places in the sequence of fields
-          // this is the first HIP decision point
-          if (has_hip) ptr += copy_hip_to_mem(compressed, ptr);
-        }
-        if (has_table) {
-          const uint32_t csv_length(compressed->csvLength);
-          ptr += copy_to_mem(ptr, &csv_length, sizeof(csv_length));
-        }
-        if (has_window) {
-          const uint32_t cw_length(compressed->cwLength);
-          ptr += copy_to_mem(ptr, &cw_length, sizeof(cw_length));
-        }
-        // this is the second HIP decision point
-        if (has_hip && !(has_table && has_window)) ptr += copy_hip_to_mem(compressed, ptr);
-        if (has_window) {
-          ptr += copy_to_mem(ptr, compressed->compressedWindow, compressed->cwLength * sizeof(uint32_t));
-        }
-        if (has_table) {
-          ptr += copy_to_mem(ptr, compressed->compressedSurprisingValues, compressed->csvLength * sizeof(uint32_t));
-        }
-      }
-      if (ptr != static_cast<char*>(data_ptr.get()) + size) throw std::logic_error("serialized size mismatch");
-      fm85Free(compressed);
-      return std::make_pair(std::move(data_ptr), size);
-    }
-
-    static cpc_sketch_unique_ptr
-    deserialize(std::istream& is, uint64_t seed = DEFAULT_SEED) {
-      fm85Init();
-      uint8_t preamble_ints;
-      is.read((char*)&preamble_ints, sizeof(preamble_ints));
-      uint8_t serial_version;
-      is.read((char*)&serial_version, sizeof(serial_version));
-      uint8_t family_id;
-      is.read((char*)&family_id, sizeof(family_id));
-      uint8_t lg_k;
-      is.read((char*)&lg_k, sizeof(lg_k));
-      uint8_t first_interesting_column;
-      is.read((char*)&first_interesting_column, sizeof(first_interesting_column));
-      uint8_t flags_byte;
-      is.read((char*)&flags_byte, sizeof(flags_byte));
-      uint16_t seed_hash;
-      is.read((char*)&seed_hash, sizeof(seed_hash));
-      const bool has_hip = flags_byte & (1 << flags::HAS_HIP);
-      const bool has_table = flags_byte & (1 << flags::HAS_TABLE);
-      const bool has_window = flags_byte & (1 << flags::HAS_WINDOW);
-      FM85 compressed;
-      compressed.isCompressed = true;
-      compressed.mergeFlag = !has_hip;
-      compressed.lgK = lg_k;
-      compressed.firstInterestingColumn = first_interesting_column;
-      compressed.numCoupons = 0;
-      compressed.numCompressedSurprisingValues = 0;
-      compressed.kxp = 1 << lg_k;
-      compressed.hipEstAccum = 0;
-      compressed.hipErrAccum = 0;
-      compressed.csvLength = 0;
-      compressed.cwLength = 0;
-      compressed.compressedSurprisingValues = nullptr;
-      compressed.compressedWindow = nullptr;
-      compressed.surprisingValueTable = nullptr;
-      compressed.slidingWindow = nullptr;
-      if (has_table || has_window) {
-        uint32_t num_coupons;
-        is.read((char*)&num_coupons, sizeof(num_coupons));
-        compressed.numCoupons = num_coupons;
-        if (has_table && has_window) {
-          uint32_t num_values;
-          is.read((char*)&num_values, sizeof(num_values));
-          compressed.numCompressedSurprisingValues = num_values;
-          if (has_hip) read_hip(&compressed, is);
-        }
-        if (has_table) {
-          uint32_t csv_length;
-          is.read((char*)&csv_length, sizeof(csv_length));
-          compressed.csvLength = csv_length;
-        }
-        if (has_window) {
-          uint32_t cw_length;
-          is.read((char*)&cw_length, sizeof(cw_length));
-          compressed.cwLength = cw_length;
-        }
-        if (has_hip && !(has_table && has_window)) read_hip(&compressed, is);
-        if (has_window) {
-          compressed.compressedWindow = new uint32_t[compressed.cwLength];
-          is.read((char*)compressed.compressedWindow, compressed.cwLength * sizeof(uint32_t));
-        }
-        if (has_table) {
-          compressed.compressedSurprisingValues = new uint32_t[compressed.csvLength];
-          is.read((char*)compressed.compressedSurprisingValues, compressed.csvLength * sizeof(uint32_t));
-        }
-        if (!has_window) compressed.numCompressedSurprisingValues = compressed.numCoupons;
-      }
-      compressed.windowOffset = determineCorrectOffset(compressed.lgK, compressed.numCoupons);
+    void update(const void* value, int size);
 
-      uint8_t expected_preamble_ints(get_preamble_ints(&compressed));
-      if (preamble_ints != expected_preamble_ints) {
-        throw std::invalid_argument("Possible corruption: preamble ints: expected "
-            + std::to_string(expected_preamble_ints) + ", got " + std::to_string(preamble_ints));
-      }
-      if (serial_version != SERIAL_VERSION) {
-        throw std::invalid_argument("Possible corruption: serial version: expected "
-            + std::to_string(SERIAL_VERSION) + ", got " + std::to_string(serial_version));
-      }
-      if (family_id != FAMILY) {
-        throw std::invalid_argument("Possible corruption: family: expected "
-            + std::to_string(FAMILY) + ", got " + std::to_string(family_id));
-      }
-      if (seed_hash != compute_seed_hash(seed)) {
-        throw std::invalid_argument("Incompatible seed hashes: " + std::to_string(seed_hash) + ", "
-            + std::to_string(compute_seed_hash(seed)));
-      }
-      FM85* uncompressed = fm85Uncompress(&compressed);
-      delete [] compressed.compressedSurprisingValues;
-      delete [] compressed.compressedWindow;
-      cpc_sketch_unique_ptr sketch_ptr(
-          new (fm85alloc(sizeof(cpc_sketch))) cpc_sketch(uncompressed, seed),
-          [](cpc_sketch* s) { s->~cpc_sketch(); fm85free(s); }
-      );
-      return sketch_ptr;
-    }
+    // prints a sketch summary to a given stream
+    void to_stream(std::ostream& os) const;
 
-    static cpc_sketch_unique_ptr
-    deserialize(const void* bytes, size_t size, uint64_t seed = DEFAULT_SEED) {
-      fm85Init();
-      const char* ptr = static_cast<const char*>(bytes);
-      uint8_t preamble_ints;
-      ptr += copy_from_mem(ptr, &preamble_ints, sizeof(preamble_ints));
-      uint8_t serial_version;
-      ptr += copy_from_mem(ptr, &serial_version, sizeof(serial_version));
-      uint8_t family_id;
-      ptr += copy_from_mem(ptr, &family_id, sizeof(family_id));
-      uint8_t lg_k;
-      ptr += copy_from_mem(ptr, &lg_k, sizeof(lg_k));
-      uint8_t first_interesting_column;
-      ptr += copy_from_mem(ptr, &first_interesting_column, sizeof(first_interesting_column));
-      uint8_t flags_byte;
-      ptr += copy_from_mem(ptr, &flags_byte, sizeof(flags_byte));
-      uint16_t seed_hash;
-      ptr += copy_from_mem(ptr, &seed_hash, sizeof(seed_hash));
-      const bool has_hip = flags_byte & (1 << flags::HAS_HIP);
-      const bool has_table = flags_byte & (1 << flags::HAS_TABLE);
-      const bool has_window = flags_byte & (1 << flags::HAS_WINDOW);
-      FM85 compressed;
-      compressed.isCompressed = true;
-      compressed.mergeFlag = !has_hip;
-      compressed.lgK = lg_k;
-      compressed.firstInterestingColumn = first_interesting_column;
-      compressed.numCoupons = 0;
-      compressed.numCompressedSurprisingValues = 0;
-      compressed.kxp = 1 << lg_k;
-      compressed.hipEstAccum = 0;
-      compressed.hipErrAccum = 0;
-      compressed.csvLength = 0;
-      compressed.cwLength = 0;
-      compressed.compressedSurprisingValues = nullptr;
-      compressed.compressedWindow = nullptr;
-      compressed.surprisingValueTable = nullptr;
-      compressed.slidingWindow = nullptr;
-      if (has_table || has_window) {
-        uint32_t num_coupons;
-        ptr += copy_from_mem(ptr, &num_coupons, sizeof(num_coupons));
-        compressed.numCoupons = num_coupons;
-        if (has_table && has_window) {
-          uint32_t num_values;
-          ptr += copy_from_mem(ptr, &num_values, sizeof(num_values));
-          compressed.numCompressedSurprisingValues = num_values;
-          if (has_hip) ptr += copy_hip_from_mem(&compressed, ptr);
-        }
-        if (has_table) {
-          uint32_t csv_length;
-          ptr += copy_from_mem(ptr, &csv_length, sizeof(csv_length));
-          compressed.csvLength = csv_length;
-        }
-        if (has_window) {
-          uint32_t cw_length;
-          ptr += copy_from_mem(ptr, &cw_length, sizeof(cw_length));
-          compressed.cwLength = cw_length;
-        }
-        if (has_hip && !(has_table && has_window)) ptr += copy_hip_from_mem(&compressed, ptr);
-        if (has_window) {
-          compressed.compressedWindow = new uint32_t[compressed.cwLength];
-          ptr += copy_from_mem(ptr, compressed.compressedWindow, compressed.cwLength * sizeof(uint32_t));
-        }
-        if (has_table) {
-          compressed.compressedSurprisingValues = new uint32_t[compressed.csvLength];
-          ptr += copy_from_mem(ptr, compressed.compressedSurprisingValues, compressed.csvLength * sizeof(uint32_t));
-        }
-        if (!has_window) compressed.numCompressedSurprisingValues = compressed.numCoupons;
-      }
-      if (ptr != static_cast<const char*>(bytes) + size) throw std::logic_error("deserialized size mismatch");
-      compressed.windowOffset = determineCorrectOffset(compressed.lgK, compressed.numCoupons);
+    void serialize(std::ostream& os) const;
+    std::pair<ptr_with_deleter, const size_t> serialize(unsigned header_size_bytes = 0) const;
 
-      uint8_t expected_preamble_ints(get_preamble_ints(&compressed));
-      if (preamble_ints != expected_preamble_ints) {
-        throw std::invalid_argument("Possible corruption: preamble ints: expected "
-            + std::to_string(expected_preamble_ints) + ", got " + std::to_string(preamble_ints));
-      }
-      if (serial_version != SERIAL_VERSION) {
-        throw std::invalid_argument("Possible corruption: serial version: expected "
-            + std::to_string(SERIAL_VERSION) + ", got " + std::to_string(serial_version));
-      }
-      if (family_id != FAMILY) {
-        throw std::invalid_argument("Possible corruption: family: expected "
-            + std::to_string(FAMILY) + ", got " + std::to_string(family_id));
-      }
-      if (seed_hash != compute_seed_hash(seed)) {
-        throw std::invalid_argument("Incompatible seed hashes: " + std::to_string(seed_hash) + ", "
-            + std::to_string(compute_seed_hash(seed)));
-      }
-      FM85* uncompressed = fm85Uncompress(&compressed);
-      delete [] compressed.compressedSurprisingValues;
-      delete [] compressed.compressedWindow;
-      cpc_sketch_unique_ptr sketch_ptr(
-          new (fm85alloc(sizeof(cpc_sketch))) cpc_sketch(uncompressed, seed),
-          [](cpc_sketch* s) { s->~cpc_sketch(); fm85free(s); }
-      );
-      return sketch_ptr;
-    }
+    static cpc_sketch_alloc<A> deserialize(std::istream& is, uint64_t seed = DEFAULT_SEED);
+    static cpc_sketch_alloc<A> deserialize(const void* bytes, size_t size, uint64_t seed = DEFAULT_SEED);
 
     // for debugging
-    uint64_t get_num_coupons() const {
-      return state->numCoupons;
-    }
+    uint64_t get_num_coupons() const;
 
     // for debugging
     // this should catch some forms of corruption during serialization-deserialization
-    bool validate() const {
-      U64* bit_matrix = bitMatrixOfSketch(state);
-      const long long num_bits_set = countBitsSetInMatrix(bit_matrix, 1LL << state->lgK);
-      fm85free(bit_matrix);
-      return num_bits_set == state->numCoupons;
-    }
+    bool validate() const;
 
-    friend std::ostream& operator<<(std::ostream& os, cpc_sketch const& sketch);
-    friend class cpc_union;
+    friend cpc_union_alloc<A>;
 
   private:
     static const uint8_t SERIAL_VERSION = 1;
@@ -529,62 +126,22 @@ class cpc_sketch {
     uint64_t seed;
 
     // for deserialization and cpc_union::get_result()
-    cpc_sketch(FM85* state, uint64_t seed = DEFAULT_SEED) : state(state), seed(seed) {}
-
-    static uint8_t get_preamble_ints(const FM85* state) {
-      uint8_t preamble_ints(2);
-      if (state->numCoupons > 0) {
-        preamble_ints += 1; // number of coupons
-        if (!state->mergeFlag) {
-          preamble_ints += 4; // HIP
-        }
-        if (state->compressedSurprisingValues != nullptr) {
-          preamble_ints += 1; // table length
-          // number of values (if there is no window it is the same as number of coupons)
-          if (state->compressedWindow != nullptr) {
-            preamble_ints += 1;
-          }
-        }
-        if (state->compressedWindow != nullptr) {
-          preamble_ints += 1; // window length
-        }
-      }
-      return preamble_ints;
-    }
-
-    static inline void write_hip(const FM85* state, std::ostream& os) {
-      os.write((char*)&state->kxp, sizeof(FM85::kxp));
-      os.write((char*)&state->hipEstAccum, sizeof(FM85::hipEstAccum));
-    }
-
-    static inline void read_hip(FM85* state, std::istream& is) {
-      is.read((char*)&state->kxp, sizeof(FM85::kxp));
-      is.read((char*)&state->hipEstAccum, sizeof(FM85::hipEstAccum));
-    }
-
-    static inline size_t copy_hip_to_mem(const FM85* state, void* dst) {
-      memcpy(dst, &state->kxp, sizeof(FM85::kxp));
-      memcpy(static_cast<char*>(dst) + sizeof(FM85::kxp), &state->hipEstAccum, sizeof(FM85::hipEstAccum));
-      return sizeof(FM85::kxp) + sizeof(FM85::hipEstAccum);
-    }
-
-    static inline size_t copy_hip_from_mem(FM85* state, const void* src) {
-      memcpy(&state->kxp, src, sizeof(FM85::kxp));
-      memcpy(&state->hipEstAccum, static_cast<const char*>(src) + sizeof(FM85::kxp), sizeof(FM85::hipEstAccum));
-      return sizeof(FM85::kxp) + sizeof(FM85::hipEstAccum);
-    }
-
-    static inline size_t copy_to_mem(void* dst, const void* src, size_t size) {
-      memcpy(dst, src, size);
-      return size;
-    }
-
-    static inline size_t copy_from_mem(const void* src, void* dst, size_t size) {
-      memcpy(dst, src, size);
-      return size;
-    }
+    cpc_sketch_alloc(FM85* state, uint64_t seed = DEFAULT_SEED);
+
+    static uint8_t get_preamble_ints(const FM85* state);
+    static inline void write_hip(const FM85* state, std::ostream& os);
+    static inline void read_hip(FM85* state, std::istream& is);
+    static inline size_t copy_hip_to_mem(const FM85* state, void* dst);
+    static inline size_t copy_hip_from_mem(FM85* state, const void* src);
+    static inline size_t copy_to_mem(void* dst, const void* src, size_t size);
+    static inline size_t copy_from_mem(const void* src, void* dst, size_t size);
 };
 
+// alias with default allocator for convenience
+typedef cpc_sketch_alloc<std::allocator<void>> cpc_sketch;
+
 } /* namespace datasketches */
 
+#include "cpc_sketch_impl.hpp"
+
 #endif
diff --git a/cpc/include/cpc_sketch_impl.hpp b/cpc/include/cpc_sketch_impl.hpp
new file mode 100644
index 0000000..2a36fd9
--- /dev/null
+++ b/cpc/include/cpc_sketch_impl.hpp
@@ -0,0 +1,566 @@
+/*
+ * 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 CPC_SKETCH_IMPL_HPP_
+#define CPC_SKETCH_IMPL_HPP_
+
+#include <stdexcept>
+#include <cmath>
+
+namespace datasketches {
+
+template<typename A>
+cpc_sketch_alloc<A>::cpc_sketch_alloc(uint8_t lg_k, uint64_t seed): seed(seed) {
+  fm85Init();
+  if (lg_k < CPC_MIN_LG_K or lg_k > CPC_MAX_LG_K) {
+    throw std::invalid_argument("lg_k must be >= " + std::to_string(CPC_MIN_LG_K) + " and <= " + std::to_string(CPC_MAX_LG_K) + ": " + std::to_string(lg_k));
+  }
+  state = fm85Make(lg_k);
+}
+
+template<typename A>
+cpc_sketch_alloc<A>::cpc_sketch_alloc(const cpc_sketch_alloc<A>& other): state(fm85Copy(other.state)), seed(other.seed) {}
+
+template<typename A>
+cpc_sketch_alloc<A>& cpc_sketch_alloc<A>::operator=(cpc_sketch_alloc<A> other) {
+  seed = other.seed;
+  std::swap(state, other.state); // @suppress("Invalid arguments")
+  return *this;
+}
+
+template<typename A>
+cpc_sketch_alloc<A>::~cpc_sketch_alloc() {
+  fm85Free(state);
+}
+
+template<typename A>
+bool cpc_sketch_alloc<A>::is_empty() const {
+  return state->numCoupons == 0;
+}
+
+template<typename A>
+double cpc_sketch_alloc<A>::get_estimate() const {
+  if (!state->mergeFlag) return getHIPEstimate(state);
+  return getIconEstimate(state->lgK, state->numCoupons);
+}
+
+template<typename A>
+double cpc_sketch_alloc<A>::get_lower_bound(unsigned kappa) const {
+  if (kappa > 3) {
+    throw std::invalid_argument("kappa must be 1, 2 or 3");
+  }
+  if (!state->mergeFlag) return getHIPConfidenceLB(state, kappa);
+  return getIconConfidenceLB(state, kappa);
+}
+
+template<typename A>
+double cpc_sketch_alloc<A>::get_upper_bound(unsigned kappa) const {
+  if (kappa > 3) {
+    throw std::invalid_argument("kappa must be 1, 2 or 3");
+  }
+  if (!state->mergeFlag) return getHIPConfidenceUB(state, kappa);
+  return getIconConfidenceUB(state, kappa);
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(const std::string& value) {
+  if (value.empty()) return;
+  update(value.c_str(), value.length());
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(uint64_t value) {
+  update(&value, sizeof(value));
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(int64_t value) {
+  update(&value, sizeof(value));
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(uint32_t value) {
+  update(static_cast<int32_t>(value));
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(int32_t value) {
+  update(static_cast<int64_t>(value));
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(uint16_t value) {
+  update(static_cast<int16_t>(value));
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(int16_t value) {
+  update(static_cast<int64_t>(value));
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(uint8_t value) {
+  update(static_cast<int8_t>(value));
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(int8_t value) {
+  update(static_cast<int64_t>(value));
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(double value) {
+  union {
+    int64_t long_value;
+    double double_value;
+  } ldu;
+  if (value == 0.0) {
+    ldu.double_value = 0.0; // canonicalize -0.0 to 0.0
+  } else if (std::isnan(value)) {
+    ldu.long_value = 0x7ff8000000000000L; // canonicalize NaN using value from Java's Double.doubleToLongBits()
+  } else {
+    ldu.double_value = value;
+  }
+  update(&ldu, sizeof(ldu));
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(float value) {
+  update(static_cast<double>(value));
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::update(const void* value, int size) {
+  HashState hashes;
+  MurmurHash3_x64_128(value, size, seed, hashes);
+  fm85Update(state, hashes.h1, hashes.h2);
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::to_stream(std::ostream& os) const {
+  os << "### CPC sketch summary:" << std::endl;
+  os << "   lgK            : " << state->lgK << std::endl;
+  os << "   seed hash      : " << std::hex << compute_seed_hash(seed) << std::dec << std::endl;
+  os << "   C              : " << state->numCoupons << std::endl;
+  os << "   flavor         : " << determineFlavor(state->lgK, state->numCoupons) << std::endl;
+  os << "   merged         : " << (state->mergeFlag ? "true" : "false") << std::endl;
+  os << "   compressed     : " << (state->isCompressed ? "true" : "false") << std::endl;
+  os << "   intresting col : " << state->firstInterestingColumn << std::endl;
+  os << "   HIP estimate   : " << state->hipEstAccum << std::endl;
+  os << "   kxp            : " << state->kxp << std::endl;
+  if (state->isCompressed) {
+    os << "   num CSV        : " << state->numCompressedSurprisingValues << std::endl;
+    os << "   CSV length     : " << state->csvLength << std::endl;
+    os << "   CW length      : " << state->cwLength << std::endl;
+  } else {
+    os << "   offset         : " << state->windowOffset << std::endl;
+    os << "   table          : " << (state->surprisingValueTable == nullptr ? "not " : "") <<  "allocated" << std::endl;
+    if (state->surprisingValueTable != nullptr) {
+      os << "   num SV         : " << state->surprisingValueTable->numItems << std::endl;
+    }
+    os << "   window         : " << (state->slidingWindow == nullptr ? "not " : "") <<  "allocated" << std::endl;
+  }
+  os << "### End sketch summary" << std::endl;
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::serialize(std::ostream& os) const {
+  FM85* compressed = fm85Compress(state);
+  const uint8_t preamble_ints(get_preamble_ints(compressed));
+  os.write((char*)&preamble_ints, sizeof(preamble_ints));
+  const uint8_t serial_version(SERIAL_VERSION);
+  os.write((char*)&serial_version, sizeof(serial_version));
+  const uint8_t family(FAMILY);
+  os.write((char*)&family, sizeof(family));
+  const uint8_t lg_k(compressed->lgK);
+  os.write((char*)&lg_k, sizeof(lg_k));
+  const uint8_t first_interesting_column(compressed->firstInterestingColumn);
+  os.write((char*)&first_interesting_column, sizeof(first_interesting_column));
+  const bool has_hip(!compressed->mergeFlag);
+  const bool has_table(compressed->compressedSurprisingValues != nullptr);
+  const bool has_window(compressed->compressedWindow != nullptr);
+  const uint8_t flags_byte(
+    (1 << flags::IS_COMPRESSED)
+    | (has_hip ? 1 << flags::HAS_HIP : 0)
+    | (has_table ? 1 << flags::HAS_TABLE : 0)
+    | (has_window ? 1 << flags::HAS_WINDOW : 0)
+  );
+  os.write((char*)&flags_byte, sizeof(flags_byte));
+  const uint16_t seed_hash(compute_seed_hash(seed));
+  os.write((char*)&seed_hash, sizeof(seed_hash));
+  if (!is_empty()) {
+    const uint32_t num_coupons(compressed->numCoupons);
+    os.write((char*)&num_coupons, sizeof(num_coupons));
+    if (has_table && has_window) {
+      // if there is no window it is the same as number of coupons
+      const uint32_t num_values(compressed->numCompressedSurprisingValues);
+      os.write((char*)&num_values, sizeof(num_values));
+      // HIP values are at the same offset because of alignment, which can be in two different places in the sequence of fields
+      // this is the first HIP decision point
+      if (has_hip) write_hip(compressed, os);
+    }
+    if (has_table) {
+      const uint32_t csv_length(compressed->csvLength);
+      os.write((char*)&csv_length, sizeof(csv_length));
+    }
+    if (has_window) {
+      const uint32_t cw_length(compressed->cwLength);
+      os.write((char*)&cw_length, sizeof(cw_length));
+    }
+    // this is the second HIP decision point
+    if (has_hip && !(has_table && has_window)) write_hip(compressed, os);
+    if (has_window) {
+      os.write((char*)compressed->compressedWindow, compressed->cwLength * sizeof(uint32_t));
+    }
+    if (has_table) {
+      os.write((char*)compressed->compressedSurprisingValues, compressed->csvLength * sizeof(uint32_t));
+    }
+  }
+  fm85Free(compressed);
+}
+
+template<typename A>
+std::pair<ptr_with_deleter, const size_t> cpc_sketch_alloc<A>::serialize(unsigned header_size_bytes) const {
+  FM85* compressed = fm85Compress(state);
+  const uint8_t preamble_ints(get_preamble_ints(compressed));
+  const size_t size = header_size_bytes + (preamble_ints + compressed->csvLength + compressed->cwLength) * sizeof(uint32_t);
+  ptr_with_deleter data_ptr(
+      fm85alloc(size),
+      [](void* ptr) { fm85free(ptr); }
+  );
+  char* ptr = static_cast<char*>(data_ptr.get()) + header_size_bytes;
+  ptr += copy_to_mem(ptr, &preamble_ints, sizeof(preamble_ints));
+  const uint8_t serial_version(SERIAL_VERSION);
+  ptr += copy_to_mem(ptr, &serial_version, sizeof(serial_version));
+  const uint8_t family(FAMILY);
+  ptr += copy_to_mem(ptr, &family, sizeof(family));
+  const uint8_t lg_k(compressed->lgK);
+  ptr += copy_to_mem(ptr, &lg_k, sizeof(lg_k));
+  const uint8_t first_interesting_column(compressed->firstInterestingColumn);
+  ptr += copy_to_mem(ptr, &first_interesting_column, sizeof(first_interesting_column));
+  const bool has_hip(!compressed->mergeFlag);
+  const bool has_table(compressed->compressedSurprisingValues != nullptr);
+  const bool has_window(compressed->compressedWindow != nullptr);
+  const uint8_t flags_byte(
+    (1 << flags::IS_COMPRESSED)
+    | (has_hip ? 1 << flags::HAS_HIP : 0)
+    | (has_table ? 1 << flags::HAS_TABLE : 0)
+    | (has_window ? 1 << flags::HAS_WINDOW : 0)
+  );
+  ptr += copy_to_mem(ptr, &flags_byte, sizeof(flags_byte));
+  const uint16_t seed_hash(compute_seed_hash(seed));
+  ptr += copy_to_mem(ptr, &seed_hash, sizeof(seed_hash));
+  if (!is_empty()) {
+    const uint32_t num_coupons(compressed->numCoupons);
+    ptr += copy_to_mem(ptr, &num_coupons, sizeof(num_coupons));
+    if (has_table && has_window) {
+      // if there is no window it is the same as number of coupons
+      const uint32_t num_values(compressed->numCompressedSurprisingValues);
+      ptr += copy_to_mem(ptr, &num_values, sizeof(num_values));
+      // HIP values are at the same offset because of alignment, which can be in two different places in the sequence of fields
+      // this is the first HIP decision point
+      if (has_hip) ptr += copy_hip_to_mem(compressed, ptr);
+    }
+    if (has_table) {
+      const uint32_t csv_length(compressed->csvLength);
+      ptr += copy_to_mem(ptr, &csv_length, sizeof(csv_length));
+    }
+    if (has_window) {
+      const uint32_t cw_length(compressed->cwLength);
+      ptr += copy_to_mem(ptr, &cw_length, sizeof(cw_length));
+    }
+    // this is the second HIP decision point
+    if (has_hip && !(has_table && has_window)) ptr += copy_hip_to_mem(compressed, ptr);
+    if (has_window) {
+      ptr += copy_to_mem(ptr, compressed->compressedWindow, compressed->cwLength * sizeof(uint32_t));
+    }
+    if (has_table) {
+      ptr += copy_to_mem(ptr, compressed->compressedSurprisingValues, compressed->csvLength * sizeof(uint32_t));
+    }
+  }
+  if (ptr != static_cast<char*>(data_ptr.get()) + size) throw std::logic_error("serialized size mismatch");
+  fm85Free(compressed);
+  return std::make_pair(std::move(data_ptr), size);
+}
+
+template<typename A>
+cpc_sketch_alloc<A> cpc_sketch_alloc<A>::deserialize(std::istream& is, uint64_t seed) {
+  fm85Init();
+  uint8_t preamble_ints;
+  is.read((char*)&preamble_ints, sizeof(preamble_ints));
+  uint8_t serial_version;
+  is.read((char*)&serial_version, sizeof(serial_version));
+  uint8_t family_id;
+  is.read((char*)&family_id, sizeof(family_id));
+  uint8_t lg_k;
+  is.read((char*)&lg_k, sizeof(lg_k));
+  uint8_t first_interesting_column;
+  is.read((char*)&first_interesting_column, sizeof(first_interesting_column));
+  uint8_t flags_byte;
+  is.read((char*)&flags_byte, sizeof(flags_byte));
+  uint16_t seed_hash;
+  is.read((char*)&seed_hash, sizeof(seed_hash));
+  const bool has_hip = flags_byte & (1 << flags::HAS_HIP);
+  const bool has_table = flags_byte & (1 << flags::HAS_TABLE);
+  const bool has_window = flags_byte & (1 << flags::HAS_WINDOW);
+  FM85 compressed;
+  compressed.isCompressed = true;
+  compressed.mergeFlag = !has_hip;
+  compressed.lgK = lg_k;
+  compressed.firstInterestingColumn = first_interesting_column;
+  compressed.numCoupons = 0;
+  compressed.numCompressedSurprisingValues = 0;
+  compressed.kxp = 1 << lg_k;
+  compressed.hipEstAccum = 0;
+  compressed.hipErrAccum = 0;
+  compressed.csvLength = 0;
+  compressed.cwLength = 0;
+  compressed.compressedSurprisingValues = nullptr;
+  compressed.compressedWindow = nullptr;
+  compressed.surprisingValueTable = nullptr;
+  compressed.slidingWindow = nullptr;
+  if (has_table || has_window) {
+    uint32_t num_coupons;
+    is.read((char*)&num_coupons, sizeof(num_coupons));
+    compressed.numCoupons = num_coupons;
+    if (has_table && has_window) {
+      uint32_t num_values;
+      is.read((char*)&num_values, sizeof(num_values));
+      compressed.numCompressedSurprisingValues = num_values;
+      if (has_hip) read_hip(&compressed, is);
+    }
+    if (has_table) {
+      uint32_t csv_length;
+      is.read((char*)&csv_length, sizeof(csv_length));
+      compressed.csvLength = csv_length;
+    }
+    if (has_window) {
+      uint32_t cw_length;
+      is.read((char*)&cw_length, sizeof(cw_length));
+      compressed.cwLength = cw_length;
+    }
+    if (has_hip && !(has_table && has_window)) read_hip(&compressed, is);
+    if (has_window) {
+      compressed.compressedWindow = new uint32_t[compressed.cwLength];
+      is.read((char*)compressed.compressedWindow, compressed.cwLength * sizeof(uint32_t));
+    }
+    if (has_table) {
+      compressed.compressedSurprisingValues = new uint32_t[compressed.csvLength];
+      is.read((char*)compressed.compressedSurprisingValues, compressed.csvLength * sizeof(uint32_t));
+    }
+    if (!has_window) compressed.numCompressedSurprisingValues = compressed.numCoupons;
+  }
+  compressed.windowOffset = determineCorrectOffset(compressed.lgK, compressed.numCoupons);
+
+  uint8_t expected_preamble_ints(get_preamble_ints(&compressed));
+  if (preamble_ints != expected_preamble_ints) {
+    throw std::invalid_argument("Possible corruption: preamble ints: expected "
+        + std::to_string(expected_preamble_ints) + ", got " + std::to_string(preamble_ints));
+  }
+  if (serial_version != SERIAL_VERSION) {
+    throw std::invalid_argument("Possible corruption: serial version: expected "
+        + std::to_string(SERIAL_VERSION) + ", got " + std::to_string(serial_version));
+  }
+  if (family_id != FAMILY) {
+    throw std::invalid_argument("Possible corruption: family: expected "
+        + std::to_string(FAMILY) + ", got " + std::to_string(family_id));
+  }
+  if (seed_hash != compute_seed_hash(seed)) {
+    throw std::invalid_argument("Incompatible seed hashes: " + std::to_string(seed_hash) + ", "
+        + std::to_string(compute_seed_hash(seed)));
+  }
+  FM85* uncompressed = fm85Uncompress(&compressed);
+  delete [] compressed.compressedSurprisingValues;
+  delete [] compressed.compressedWindow;
+  return cpc_sketch_alloc(uncompressed, seed);
+}
+
+template<typename A>
+cpc_sketch_alloc<A> cpc_sketch_alloc<A>::deserialize(const void* bytes, size_t size, uint64_t seed) {
+  fm85Init();
+  const char* ptr = static_cast<const char*>(bytes);
+  uint8_t preamble_ints;
+  ptr += copy_from_mem(ptr, &preamble_ints, sizeof(preamble_ints));
+  uint8_t serial_version;
+  ptr += copy_from_mem(ptr, &serial_version, sizeof(serial_version));
+  uint8_t family_id;
+  ptr += copy_from_mem(ptr, &family_id, sizeof(family_id));
+  uint8_t lg_k;
+  ptr += copy_from_mem(ptr, &lg_k, sizeof(lg_k));
+  uint8_t first_interesting_column;
+  ptr += copy_from_mem(ptr, &first_interesting_column, sizeof(first_interesting_column));
+  uint8_t flags_byte;
+  ptr += copy_from_mem(ptr, &flags_byte, sizeof(flags_byte));
+  uint16_t seed_hash;
+  ptr += copy_from_mem(ptr, &seed_hash, sizeof(seed_hash));
+  const bool has_hip = flags_byte & (1 << flags::HAS_HIP);
+  const bool has_table = flags_byte & (1 << flags::HAS_TABLE);
+  const bool has_window = flags_byte & (1 << flags::HAS_WINDOW);
+  FM85 compressed;
+  compressed.isCompressed = true;
+  compressed.mergeFlag = !has_hip;
+  compressed.lgK = lg_k;
+  compressed.firstInterestingColumn = first_interesting_column;
+  compressed.numCoupons = 0;
+  compressed.numCompressedSurprisingValues = 0;
+  compressed.kxp = 1 << lg_k;
+  compressed.hipEstAccum = 0;
+  compressed.hipErrAccum = 0;
+  compressed.csvLength = 0;
+  compressed.cwLength = 0;
+  compressed.compressedSurprisingValues = nullptr;
+  compressed.compressedWindow = nullptr;
+  compressed.surprisingValueTable = nullptr;
+  compressed.slidingWindow = nullptr;
+  if (has_table || has_window) {
+    uint32_t num_coupons;
+    ptr += copy_from_mem(ptr, &num_coupons, sizeof(num_coupons));
+    compressed.numCoupons = num_coupons;
+    if (has_table && has_window) {
+      uint32_t num_values;
+      ptr += copy_from_mem(ptr, &num_values, sizeof(num_values));
+      compressed.numCompressedSurprisingValues = num_values;
+      if (has_hip) ptr += copy_hip_from_mem(&compressed, ptr);
+    }
+    if (has_table) {
+      uint32_t csv_length;
+      ptr += copy_from_mem(ptr, &csv_length, sizeof(csv_length));
+      compressed.csvLength = csv_length;
+    }
+    if (has_window) {
+      uint32_t cw_length;
+      ptr += copy_from_mem(ptr, &cw_length, sizeof(cw_length));
+      compressed.cwLength = cw_length;
+    }
+    if (has_hip && !(has_table && has_window)) ptr += copy_hip_from_mem(&compressed, ptr);
+    if (has_window) {
+      compressed.compressedWindow = new uint32_t[compressed.cwLength];
+      ptr += copy_from_mem(ptr, compressed.compressedWindow, compressed.cwLength * sizeof(uint32_t));
+    }
+    if (has_table) {
+      compressed.compressedSurprisingValues = new uint32_t[compressed.csvLength];
+      ptr += copy_from_mem(ptr, compressed.compressedSurprisingValues, compressed.csvLength * sizeof(uint32_t));
+    }
+    if (!has_window) compressed.numCompressedSurprisingValues = compressed.numCoupons;
+  }
+  if (ptr != static_cast<const char*>(bytes) + size) throw std::logic_error("deserialized size mismatch");
+  compressed.windowOffset = determineCorrectOffset(compressed.lgK, compressed.numCoupons);
+
+  uint8_t expected_preamble_ints(get_preamble_ints(&compressed));
+  if (preamble_ints != expected_preamble_ints) {
+    throw std::invalid_argument("Possible corruption: preamble ints: expected "
+        + std::to_string(expected_preamble_ints) + ", got " + std::to_string(preamble_ints));
+  }
+  if (serial_version != SERIAL_VERSION) {
+    throw std::invalid_argument("Possible corruption: serial version: expected "
+        + std::to_string(SERIAL_VERSION) + ", got " + std::to_string(serial_version));
+  }
+  if (family_id != FAMILY) {
+    throw std::invalid_argument("Possible corruption: family: expected "
+        + std::to_string(FAMILY) + ", got " + std::to_string(family_id));
+  }
+  if (seed_hash != compute_seed_hash(seed)) {
+    throw std::invalid_argument("Incompatible seed hashes: " + std::to_string(seed_hash) + ", "
+        + std::to_string(compute_seed_hash(seed)));
+  }
+  FM85* uncompressed = fm85Uncompress(&compressed);
+  delete [] compressed.compressedSurprisingValues;
+  delete [] compressed.compressedWindow;
+  return cpc_sketch_alloc(uncompressed, seed);
+}
+
+template<typename A>
+uint64_t cpc_sketch_alloc<A>::get_num_coupons() const {
+  return state->numCoupons;
+}
+
+template<typename A>
+bool cpc_sketch_alloc<A>::validate() const {
+  U64* bit_matrix = bitMatrixOfSketch(state);
+  const long long num_bits_set = countBitsSetInMatrix(bit_matrix, 1LL << state->lgK);
+  fm85free(bit_matrix);
+  return num_bits_set == state->numCoupons;
+}
+
+template<typename A>
+cpc_sketch_alloc<A>::cpc_sketch_alloc(FM85* state, uint64_t seed): state(state), seed(seed) {}
+
+template<typename A>
+uint8_t cpc_sketch_alloc<A>::get_preamble_ints(const FM85* state) {
+  uint8_t preamble_ints(2);
+  if (state->numCoupons > 0) {
+    preamble_ints += 1; // number of coupons
+    if (!state->mergeFlag) {
+      preamble_ints += 4; // HIP
+    }
+    if (state->compressedSurprisingValues != nullptr) {
+      preamble_ints += 1; // table length
+      // number of values (if there is no window it is the same as number of coupons)
+      if (state->compressedWindow != nullptr) {
+        preamble_ints += 1;
+      }
+    }
+    if (state->compressedWindow != nullptr) {
+      preamble_ints += 1; // window length
+    }
+  }
+  return preamble_ints;
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::write_hip(const FM85* state, std::ostream& os) {
+  os.write((char*)&state->kxp, sizeof(FM85::kxp));
+  os.write((char*)&state->hipEstAccum, sizeof(FM85::hipEstAccum));
+}
+
+template<typename A>
+void cpc_sketch_alloc<A>::read_hip(FM85* state, std::istream& is) {
+  is.read((char*)&state->kxp, sizeof(FM85::kxp));
+  is.read((char*)&state->hipEstAccum, sizeof(FM85::hipEstAccum));
+}
+
+template<typename A>
+size_t cpc_sketch_alloc<A>::copy_hip_to_mem(const FM85* state, void* dst) {
+  memcpy(dst, &state->kxp, sizeof(FM85::kxp));
+  memcpy(static_cast<char*>(dst) + sizeof(FM85::kxp), &state->hipEstAccum, sizeof(FM85::hipEstAccum));
+  return sizeof(FM85::kxp) + sizeof(FM85::hipEstAccum);
+}
+
+template<typename A>
+size_t cpc_sketch_alloc<A>::copy_hip_from_mem(FM85* state, const void* src) {
+  memcpy(&state->kxp, src, sizeof(FM85::kxp));
+  memcpy(&state->hipEstAccum, static_cast<const char*>(src) + sizeof(FM85::kxp), sizeof(FM85::hipEstAccum));
+  return sizeof(FM85::kxp) + sizeof(FM85::hipEstAccum);
+}
+
+template<typename A>
+size_t cpc_sketch_alloc<A>::copy_to_mem(void* dst, const void* src, size_t size) {
+  memcpy(dst, src, size);
+  return size;
+}
+
+template<typename A>
+size_t cpc_sketch_alloc<A>::copy_from_mem(const void* src, void* dst, size_t size) {
+  memcpy(dst, src, size);
+  return size;
+}
+
+} /* namespace datasketches */
+
+#endif
diff --git a/cpc/include/cpc_union.hpp b/cpc/include/cpc_union.hpp
index fc2f274..251bf8a 100644
--- a/cpc/include/cpc_union.hpp
+++ b/cpc/include/cpc_union.hpp
@@ -38,65 +38,28 @@ namespace datasketches {
  * author Alexander Saydakov
  */
 
-UG85* ug85Copy(UG85* other) {
-  UG85* copy(new UG85(*other));
-  if (other->accumulator != nullptr) copy->accumulator = fm85Copy(other->accumulator);
-  if (other->bitMatrix != nullptr) {
-    uint32_t k = 1 << copy->lgK;
-    copy->bitMatrix = (U64 *) malloc ((size_t) (k * sizeof(U64)));
-    std::copy(&other->bitMatrix[0], &other->bitMatrix[k], copy->bitMatrix);
-  }
-  return copy;
-}
-
-class cpc_union {
+template<typename A>
+class cpc_union_alloc {
   public:
-    explicit cpc_union(uint8_t lg_k = CPC_DEFAULT_LG_K, uint64_t seed = DEFAULT_SEED) : seed(seed) {
-      fm85Init();
-      if (lg_k < CPC_MIN_LG_K or lg_k > CPC_MAX_LG_K) {
-        throw std::invalid_argument("lg_k must be >= " + std::to_string(CPC_MIN_LG_K) + " and <= " + std::to_string(CPC_MAX_LG_K) + ": " + std::to_string(lg_k));
-      }
-      state = ug85Make(lg_k);
-    }
-
-    cpc_union(const cpc_union& other) {
-      seed = other.seed;
-      state = ug85Copy(other.state);
-    }
-
-    cpc_union& operator=(cpc_union other) {
-      seed = other.seed;
-      std::swap(state, other.state); // @suppress("Invalid arguments")
-      return *this;
-    }
 
-    ~cpc_union() {
-      ug85Free(state);
-    }
+    explicit cpc_union_alloc(uint8_t lg_k = CPC_DEFAULT_LG_K, uint64_t seed = DEFAULT_SEED);
+    cpc_union_alloc(const cpc_union_alloc<A>& other);
+    cpc_union_alloc<A>& operator=(cpc_union_alloc<A> other);
+    ~cpc_union_alloc();
 
-    void update(const cpc_sketch& sketch) {
-      const uint16_t seed_hash_union = compute_seed_hash(seed);
-      const uint16_t seed_hash_sketch = compute_seed_hash(sketch.seed);
-      if (seed_hash_union != seed_hash_sketch) {
-        throw std::invalid_argument("Incompatible seed hashes: " + std::to_string(seed_hash_union) + ", "
-            + std::to_string(seed_hash_sketch));
-      }
-      ug85MergeInto(state, sketch.state);
-    }
-
-    cpc_sketch_unique_ptr get_result() const {
-      cpc_sketch_unique_ptr sketch_ptr(
-          new (fm85alloc(sizeof(cpc_sketch))) cpc_sketch(ug85GetResult(state), seed),
-          [](cpc_sketch* s) { s->~cpc_sketch(); fm85free(s); }
-      );
-      return sketch_ptr;
-    }
+    void update(const cpc_sketch_alloc<A>& sketch);
+    cpc_sketch_alloc<A> get_result() const;
 
   private:
     UG85* state;
     uint64_t seed;
 };
 
+// alias with default allocator for convenience
+typedef cpc_union_alloc<std::allocator<void>> cpc_union;
+
 } /* namespace datasketches */
 
+#include "cpc_union_impl.hpp"
+
 #endif
diff --git a/cpc/include/cpc_union_impl.hpp b/cpc/include/cpc_union_impl.hpp
new file mode 100644
index 0000000..21829ff
--- /dev/null
+++ b/cpc/include/cpc_union_impl.hpp
@@ -0,0 +1,70 @@
+/*
+ * 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 CPC_UNION_IMPL_HPP_
+#define CPC_UNION_IMPL_HPP_
+
+namespace datasketches {
+
+template<typename A>
+cpc_union_alloc<A>::cpc_union_alloc(uint8_t lg_k, uint64_t seed) : seed(seed) {
+  fm85Init();
+  if (lg_k < CPC_MIN_LG_K or lg_k > CPC_MAX_LG_K) {
+    throw std::invalid_argument("lg_k must be >= " + std::to_string(CPC_MIN_LG_K) + " and <= " + std::to_string(CPC_MAX_LG_K) + ": " + std::to_string(lg_k));
+  }
+  state = ug85Make(lg_k);
+}
+
+template<typename A>
+cpc_union_alloc<A>::cpc_union_alloc(const cpc_union_alloc<A>& other) {
+  seed = other.seed;
+  state = ug85Copy(other.state);
+}
+
+template<typename A>
+cpc_union_alloc<A>& cpc_union_alloc<A>::operator=(cpc_union_alloc<A> other) {
+  seed = other.seed;
+  std::swap(state, other.state); // @suppress("Invalid arguments")
+  return *this;
+}
+
+template<typename A>
+cpc_union_alloc<A>::~cpc_union_alloc() {
+  ug85Free(state);
+}
+
+template<typename A>
+void cpc_union_alloc<A>::update(const cpc_sketch_alloc<A>& sketch) {
+  const uint16_t seed_hash_union = compute_seed_hash(seed);
+  const uint16_t seed_hash_sketch = compute_seed_hash(sketch.seed);
+  if (seed_hash_union != seed_hash_sketch) {
+    throw std::invalid_argument("Incompatible seed hashes: " + std::to_string(seed_hash_union) + ", "
+        + std::to_string(seed_hash_sketch));
+  }
+  ug85MergeInto(state, sketch.state);
+}
+
+template<typename A>
+cpc_sketch_alloc<A> cpc_union_alloc<A>::get_result() const {
+  return cpc_sketch_alloc<A>(ug85GetResult(state), seed);
+}
+
+} /* namespace datasketches */
+
+#endif
diff --git a/cpc/include/fm85Merging.h b/cpc/include/fm85Merging.h
index 0114e06..244d596 100644
--- a/cpc/include/fm85Merging.h
+++ b/cpc/include/fm85Merging.h
@@ -42,6 +42,8 @@ UG85* ug85Make(Short lgK);
 
 void ug85Free(UG85* unioner);
 
+UG85* ug85Copy(UG85* other);
+
 void ug85MergeInto(UG85* unioner, const FM85* sourceSketch);
 
 FM85* ug85GetResult(const UG85* unioner);
diff --git a/cpc/src/cpc_sketch.cpp b/cpc/src/cpc_sketch.cpp
index 0630580..4e42709 100644
--- a/cpc/src/cpc_sketch.cpp
+++ b/cpc/src/cpc_sketch.cpp
@@ -32,31 +32,4 @@ void cpc_cleanup() {
   fm85Clean();
 }
 
-std::ostream& operator<<(std::ostream& os, cpc_sketch const& sketch) {
-  os << "### CPC sketch summary:" << std::endl;
-  os << "   lgK            : " << sketch.state->lgK << std::endl;
-  os << "   seed hash      : " << std::hex << compute_seed_hash(sketch.seed) << std::dec << std::endl;
-  os << "   C              : " << sketch.state->numCoupons << std::endl;
-  os << "   flavor         : " << determineFlavor(sketch.state->lgK, sketch.state->numCoupons) << std::endl;
-  os << "   merged         : " << (sketch.state->mergeFlag ? "true" : "false") << std::endl;
-  os << "   compressed     : " << (sketch.state->isCompressed ? "true" : "false") << std::endl;
-  os << "   intresting col : " << sketch.state->firstInterestingColumn << std::endl;
-  os << "   HIP estimate   : " << sketch.state->hipEstAccum << std::endl;
-  os << "   kxp            : " << sketch.state->kxp << std::endl;
-  if (sketch.state->isCompressed) {
-    os << "   num CSV        : " << sketch.state->numCompressedSurprisingValues << std::endl;
-    os << "   CSV length     : " << sketch.state->csvLength << std::endl;
-    os << "   CW length      : " << sketch.state->cwLength << std::endl;
-  } else {
-    os << "   offset         : " << sketch.state->windowOffset << std::endl;
-    os << "   table          : " << (sketch.state->surprisingValueTable == nullptr ? "not " : "") <<  "allocated" << std::endl;
-    if (sketch.state->surprisingValueTable != nullptr) {
-      os << "   num SV         : " << sketch.state->surprisingValueTable->numItems << std::endl;
-    }
-    os << "   window         : " << (sketch.state->slidingWindow == nullptr ? "not " : "") <<  "allocated" << std::endl;
-  }
-  os << "### End sketch summary" << std::endl;
-  return os;
-}
-
 } /* namespace datasketches */
diff --git a/cpc/src/fm85Merging.cpp b/cpc/src/fm85Merging.cpp
index 0a718d8..4725c0c 100644
--- a/cpc/src/fm85Merging.cpp
+++ b/cpc/src/fm85Merging.cpp
@@ -21,6 +21,7 @@
 
 #include "fm85Merging.h"
 
+#include <algorithm>
 #include <stdexcept>
 #include <new>
 
@@ -44,6 +45,17 @@ void ug85Free(UG85* self) {
   }
 }
 
+UG85* ug85Copy(UG85* other) {
+  UG85* copy(new UG85(*other));
+  if (other->accumulator != nullptr) copy->accumulator = fm85Copy(other->accumulator);
+  if (other->bitMatrix != nullptr) {
+    uint32_t k = 1 << copy->lgK;
+    copy->bitMatrix = (U64 *) malloc ((size_t) (k * sizeof(U64)));
+    std::copy(&other->bitMatrix[0], &other->bitMatrix[k], copy->bitMatrix);
+  }
+  return copy;
+}
+
 // This is used for testing purposes only.
 U64* bitMatrixOfUG85(const UG85* self, bool* needToFreePtr) {
   if (self->bitMatrix != NULL) { // return the matrix
diff --git a/cpc/test/cpc_sketch_test.cpp b/cpc/test/cpc_sketch_test.cpp
index 84390e7..2daf073 100644
--- a/cpc/test/cpc_sketch_test.cpp
+++ b/cpc/test/cpc_sketch_test.cpp
@@ -96,10 +96,10 @@ class cpc_sketch_test: public CppUnit::TestFixture {
     cpc_sketch sketch(11);
     std::stringstream s(std::ios::in | std::ios::out | std::ios::binary);
     sketch.serialize(s);
-    auto sketch_ptr(cpc_sketch::deserialize(s));
-    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(s);
+    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), deserialized.is_empty());
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     std::ofstream os("cpc-empty.bin");
     sketch.serialize(os);
@@ -111,15 +111,15 @@ class cpc_sketch_test: public CppUnit::TestFixture {
     for (int i = 0; i < n; i++) sketch.update(i);
     std::stringstream s(std::ios::in | std::ios::out | std::ios::binary);
     sketch.serialize(s);
-    auto sketch_ptr(cpc_sketch::deserialize(s));
-    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(s);
+    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), deserialized.is_empty());
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     // updating again with the same values should not change the sketch
-    for (int i = 0; i < n; i++) sketch_ptr->update(i);
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    for (int i = 0; i < n; i++) deserialized.update(i);
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     std::ofstream os("cpc-sparse.bin");
     sketch.serialize(os);
@@ -131,15 +131,15 @@ class cpc_sketch_test: public CppUnit::TestFixture {
     for (int i = 0; i < n; i++) sketch.update(i);
     std::stringstream s(std::ios::in | std::ios::out | std::ios::binary);
     sketch.serialize(s);
-    auto sketch_ptr(cpc_sketch::deserialize(s));
-    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(s);
+    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), deserialized.is_empty());
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     // updating again with the same values should not change the sketch
-    for (int i = 0; i < n; i++) sketch_ptr->update(i);
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    for (int i = 0; i < n; i++) deserialized.update(i);
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     std::ofstream os("cpc-hybrid.bin");
     sketch.serialize(os);
@@ -151,15 +151,15 @@ class cpc_sketch_test: public CppUnit::TestFixture {
     for (int i = 0; i < n; i++) sketch.update(i);
     std::stringstream s(std::ios::in | std::ios::out | std::ios::binary);
     sketch.serialize(s);
-    auto sketch_ptr(cpc_sketch::deserialize(s));
-    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(s);
+    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), deserialized.is_empty());
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     // updating again with the same values should not change the sketch
-    for (int i = 0; i < n; i++) sketch_ptr->update(i);
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    for (int i = 0; i < n; i++) deserialized.update(i);
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     std::ofstream os("cpc-pinned.bin");
     sketch.serialize(os);
@@ -171,15 +171,15 @@ class cpc_sketch_test: public CppUnit::TestFixture {
     for (int i = 0; i < n; i++) sketch.update(i);
     std::stringstream s(std::ios::in | std::ios::out | std::ios::binary);
     sketch.serialize(s);
-    auto sketch_ptr(cpc_sketch::deserialize(s));
-    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(s);
+    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), deserialized.is_empty());
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     // updating again with the same values should not change the sketch
-    for (int i = 0; i < n; i++) sketch_ptr->update(i);
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    for (int i = 0; i < n; i++) deserialized.update(i);
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     std::ofstream os("cpc-sliding.bin");
     sketch.serialize(os);
@@ -188,10 +188,10 @@ class cpc_sketch_test: public CppUnit::TestFixture {
   void serialize_deserialize_empty_bytes() {
     cpc_sketch sketch(11);
     auto data = sketch.serialize();
-    auto sketch_ptr(cpc_sketch::deserialize(data.first.get(), data.second));
-    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(data.first.get(), data.second);
+    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), deserialized.is_empty());
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     std::ofstream os("cpc-empty.bin");
     sketch.serialize(os);
@@ -202,15 +202,15 @@ class cpc_sketch_test: public CppUnit::TestFixture {
     const int n(200);
     for (int i = 0; i < n; i++) sketch.update(i);
     auto data = sketch.serialize();
-    auto sketch_ptr(cpc_sketch::deserialize(data.first.get(), data.second));
-    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(data.first.get(), data.second);
+    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), deserialized.is_empty());
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     // updating again with the same values should not change the sketch
-    for (int i = 0; i < n; i++) sketch_ptr->update(i);
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    for (int i = 0; i < n; i++) deserialized.update(i);
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
   }
 
   void serialize_deserialize_sparse_bytes() {
@@ -218,15 +218,15 @@ class cpc_sketch_test: public CppUnit::TestFixture {
     const int n(100);
     for (int i = 0; i < n; i++) sketch.update(i);
     auto data = sketch.serialize();
-    auto sketch_ptr(cpc_sketch::deserialize(data.first.get(), data.second));
-    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(data.first.get(), data.second);
+    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), deserialized.is_empty());
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     // updating again with the same values should not change the sketch
-    for (int i = 0; i < n; i++) sketch_ptr->update(i);
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    for (int i = 0; i < n; i++) deserialized.update(i);
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
   }
 
   void serialize_deserialize_pinned_bytes() {
@@ -234,15 +234,17 @@ class cpc_sketch_test: public CppUnit::TestFixture {
     const int n(2000);
     for (int i = 0; i < n; i++) sketch.update(i);
     auto data = sketch.serialize();
-    auto sketch_ptr(cpc_sketch::deserialize(data.first.get(), data.second));
-    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(data.first.get(), data.second);
+    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), deserialized.is_empty());
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     // updating again with the same values should not change the sketch
-    for (int i = 0; i < n; i++) sketch_ptr->update(i);
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    for (int i = 0; i < n; i++) deserialized.update(i);
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
+
+    sketch.to_stream(std::cout);
   }
 
   void serialize_deserialize_sliding_bytes() {
@@ -250,15 +252,15 @@ class cpc_sketch_test: public CppUnit::TestFixture {
     const int n(20000);
     for (int i = 0; i < n; i++) sketch.update(i);
     auto data = sketch.serialize();
-    auto sketch_ptr(cpc_sketch::deserialize(data.first.get(), data.second));
-    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(data.first.get(), data.second);
+    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), deserialized.is_empty());
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     // updating again with the same values should not change the sketch
-    for (int i = 0; i < n; i++) sketch_ptr->update(i);
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    for (int i = 0; i < n; i++) deserialized.update(i);
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
   }
 
   void copy() {
@@ -276,10 +278,10 @@ class cpc_sketch_test: public CppUnit::TestFixture {
     cpc_sketch sketch(11, 123);
     std::stringstream s(std::ios::in | std::ios::out | std::ios::binary);
     sketch.serialize(s);
-    auto sketch_ptr(cpc_sketch::deserialize(s, 123));
-    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), sketch_ptr->get_estimate());
-    CPPUNIT_ASSERT(sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(s, 123);
+    CPPUNIT_ASSERT_EQUAL(sketch.is_empty(), deserialized.is_empty());
+    CPPUNIT_ASSERT_EQUAL(sketch.get_estimate(), deserialized.get_estimate());
+    CPPUNIT_ASSERT(deserialized.validate());
 
     // incompatible seed
     s.seekg(0); // rewind the stream to read the same sketch again
@@ -306,8 +308,8 @@ class cpc_sketch_test: public CppUnit::TestFixture {
     sketch.serialize(s);
     s.seekp(700); // the stream should be 856 bytes long. corrupt it somewhere before the end
     s << "corrupt data";
-    auto sketch_ptr(cpc_sketch::deserialize(s));
-    CPPUNIT_ASSERT(!sketch_ptr->validate());
+    cpc_sketch deserialized = cpc_sketch::deserialize(s);
+    CPPUNIT_ASSERT(!deserialized.validate());
   }
 
   void serialize_both_ways() {
diff --git a/cpc/test/cpc_union_test.cpp b/cpc/test/cpc_union_test.cpp
index e8b12e8..b7f2ced 100644
--- a/cpc/test/cpc_union_test.cpp
+++ b/cpc/test/cpc_union_test.cpp
@@ -44,9 +44,9 @@ class cpc_union_test: public CppUnit::TestFixture {
 
   void empty() {
     cpc_union u(11);
-    auto sketch_ptr(u.get_result());
-    CPPUNIT_ASSERT(sketch_ptr->is_empty());
-    CPPUNIT_ASSERT_EQUAL(0.0, sketch_ptr->get_estimate());
+    auto s = u.get_result();
+    CPPUNIT_ASSERT(s.is_empty());
+    CPPUNIT_ASSERT_EQUAL(0.0, s.get_estimate());
   }
 
   void copy() {
@@ -56,15 +56,15 @@ class cpc_union_test: public CppUnit::TestFixture {
     u1.update(s);
 
     cpc_union u2 = u1; // copy constructor
-    auto sp1(u2.get_result());
-    CPPUNIT_ASSERT(!sp1->is_empty());
-    CPPUNIT_ASSERT_DOUBLES_EQUAL(1, sp1->get_estimate(), RELATIVE_ERROR_FOR_LG_K_11);
+    auto s1 = u2.get_result();
+    CPPUNIT_ASSERT(!s1.is_empty());
+    CPPUNIT_ASSERT_DOUBLES_EQUAL(1, s1.get_estimate(), RELATIVE_ERROR_FOR_LG_K_11);
     s.update(2);
     u2.update(s);
     u1 = u2; // operator=
-    auto sp2(u1.get_result());
-    CPPUNIT_ASSERT(!sp2->is_empty());
-    CPPUNIT_ASSERT_DOUBLES_EQUAL(2, sp2->get_estimate(), RELATIVE_ERROR_FOR_LG_K_11);
+    auto s2 = u1.get_result();
+    CPPUNIT_ASSERT(!s2.is_empty());
+    CPPUNIT_ASSERT_DOUBLES_EQUAL(2, s2.get_estimate(), RELATIVE_ERROR_FOR_LG_K_11);
   }
 
   void custom_seed() {
@@ -76,9 +76,9 @@ class cpc_union_test: public CppUnit::TestFixture {
 
     cpc_union u1(11, 123);
     u1.update(s);
-    auto sp(u1.get_result());
-    CPPUNIT_ASSERT(!sp->is_empty());
-    CPPUNIT_ASSERT_DOUBLES_EQUAL(3, sp->get_estimate(), RELATIVE_ERROR_FOR_LG_K_11);
+    auto r = u1.get_result();
+    CPPUNIT_ASSERT(!r.is_empty());
+    CPPUNIT_ASSERT_DOUBLES_EQUAL(3, r.get_estimate(), RELATIVE_ERROR_FOR_LG_K_11);
 
     // incompatible seed
     cpc_union u2(11, 234);


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org