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 2020/06/19 20:51:40 UTC

[incubator-datasketches-cpp] 01/01: partially implemented working prototype

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

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

commit 2ed9399d83162e9a11f1b44389a3b37ac3984160
Author: AlexanderSaydakov <Al...@users.noreply.github.com>
AuthorDate: Fri Jun 19 13:51:23 2020 -0700

    partially implemented working prototype
---
 CMakeLists.txt                                   |   1 +
 {theta => common}/include/binomial_bounds.hpp    |   0
 common/test/test_type.hpp                        |  20 +-
 tuple/CMakeLists.txt                             |  64 +++
 tuple/include/theta_sketch_experimental.hpp      |  95 +++++
 tuple/include/theta_sketch_experimental_impl.hpp | 152 +++++++
 tuple/include/theta_union_base.hpp               |  56 +++
 tuple/include/theta_union_base_impl.hpp          |  61 +++
 tuple/include/theta_union_experimental.hpp       |  87 ++++
 tuple/include/theta_union_experimental_impl.hpp  |  42 ++
 tuple/include/theta_update_sketch_base.hpp       | 227 ++++++++++
 tuple/include/theta_update_sketch_base_impl.hpp  | 255 ++++++++++++
 tuple/include/tuple_sketch.hpp                   | 502 +++++++++++++++++++++++
 tuple/include/tuple_sketch_impl.hpp              | 313 ++++++++++++++
 tuple/include/tuple_union.hpp                    | 109 +++++
 tuple/include/tuple_union_impl.hpp               |  46 +++
 tuple/test/CMakeLists.txt                        |  45 ++
 tuple/test/theta_sketch_experimental_test.cpp    |  56 +++
 tuple/test/theta_union_experimental_test.cpp     |  47 +++
 tuple/test/tuple_sketch_test.cpp                 | 189 +++++++++
 tuple/test/tuple_union_test.cpp                  |  45 ++
 21 files changed, 2411 insertions(+), 1 deletion(-)

diff --git a/CMakeLists.txt b/CMakeLists.txt
index c98c4a2..ff1d4a5 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -95,6 +95,7 @@ add_subdirectory(kll)
 add_subdirectory(fi)
 add_subdirectory(theta)
 add_subdirectory(sampling)
+add_subdirectory(tuple)
 
 if (WITH_PYTHON)
   add_subdirectory(python)
diff --git a/theta/include/binomial_bounds.hpp b/common/include/binomial_bounds.hpp
similarity index 100%
rename from theta/include/binomial_bounds.hpp
rename to common/include/binomial_bounds.hpp
diff --git a/common/test/test_type.hpp b/common/test/test_type.hpp
index abbd8f3..1651c9a 100644
--- a/common/test/test_type.hpp
+++ b/common/test/test_type.hpp
@@ -25,7 +25,7 @@
 namespace datasketches {
 
 class test_type {
-  static const bool DEBUG = false;
+  static const bool DEBUG = true;
 public:
   // no default constructor should be required
   test_type(int value): value(value) {
@@ -91,6 +91,24 @@ struct test_type_serde {
   size_t size_of_item(const test_type&) {
     return sizeof(int);
   }
+  size_t serialize(void* ptr, size_t capacity, const test_type* items, unsigned num) {
+    const size_t bytes_written = sizeof(int) * num;
+    check_memory_size(bytes_written, capacity);
+    for (unsigned i = 0; i < num; ++i) {
+      memcpy(ptr, &items[i], sizeof(int));
+      ptr = static_cast<char*>(ptr) + sizeof(int);
+    }
+    return bytes_written;
+  }
+  size_t deserialize(const void* ptr, size_t capacity, test_type* items, unsigned num) {
+    const size_t bytes_read = sizeof(int) * num;
+    check_memory_size(bytes_read, capacity);
+    for (unsigned i = 0; i < num; ++i) {
+      memcpy(&items[i], ptr, sizeof(int));
+      ptr = static_cast<const char*>(ptr) + sizeof(int);
+    }
+    return bytes_read;
+  }
 };
 
 std::ostream& operator<<(std::ostream& os, const test_type& a) {
diff --git a/tuple/CMakeLists.txt b/tuple/CMakeLists.txt
new file mode 100644
index 0000000..17fc2b1
--- /dev/null
+++ b/tuple/CMakeLists.txt
@@ -0,0 +1,64 @@
+# 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.
+
+add_library(tuple INTERFACE)
+
+add_library(${PROJECT_NAME}::TUPLE ALIAS tuple)
+
+if (BUILD_TESTS)
+  add_subdirectory(test)
+endif()
+
+target_include_directories(tuple
+  INTERFACE
+    $<BUILD_INTERFACE:${CMAKE_CURRENT_SOURCE_DIR}/include>
+    $<INSTALL_INTERFACE:$<INSTALL_PREFIX>/include>
+)
+
+target_link_libraries(tuple INTERFACE common)
+target_compile_features(tuple INTERFACE cxx_std_11)
+
+set(tuple_HEADERS "")
+list(APPEND tuple_HEADERS "include/tuple_sketch.hpp;include/tuple_sketch_impl.hpp")
+list(APPEND tuple_HEADERS "include/tuple_union.hpp;include/tuple_union_impl.hpp")
+list(APPEND tuple_HEADERS "include/theta_update_sketch_base.hpp;include/theta_update_sketch_base_impl.hpp")
+list(APPEND tuple_HEADERS "include/theta_union_base.hpp;include/theta_union_base_impl.hpp")
+list(APPEND tuple_HEADERS "include/theta_sketch_experimental.hpp;include/theta_sketch_experimental_impl.hpp")
+list(APPEND tuple_HEADERS "include/theta_union_experimental.hpp;include/theta_union_experimental_impl.hpp")
+
+install(TARGETS tuple
+  EXPORT ${PROJECT_NAME}
+)
+
+install(FILES ${tuple_HEADERS}
+  DESTINATION "${CMAKE_INSTALL_INCLUDEDIR}/DataSketches")
+
+target_sources(tuple
+  INTERFACE
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/tuple_sketch.hpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/tuple_sketch_impl.hpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/tuple_union.hpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/tuple_union_impl.hpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/theta_update_sketch_base.hpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/theta_update_sketch_base_impl.hpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/theta_union_base.hpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/theta_union_base_impl.hpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/theta_sketch_experimental.hpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/theta_sketch_experimental_impl.hpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/theta_union_experimental.hpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/include/theta_union_experimental_impl.hpp
+)
diff --git a/tuple/include/theta_sketch_experimental.hpp b/tuple/include/theta_sketch_experimental.hpp
new file mode 100644
index 0000000..9c8bd3d
--- /dev/null
+++ b/tuple/include/theta_sketch_experimental.hpp
@@ -0,0 +1,95 @@
+/*
+ * 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 THETA_SKETCH_EXPERIMENTAL_HPP_
+#define THETA_SKETCH_EXPERIMENTAL_HPP_
+
+#include "serde.hpp"
+#include "theta_update_sketch_base.hpp"
+
+namespace datasketches {
+
+// experimental theta sketch derived from the same base as tuple sketch
+
+template<typename A> class compact_theta_sketch_experimental;
+
+template<typename A = std::allocator<uint64_t>>
+class theta_sketch_experimental {
+public:
+  using resize_factor = theta_constants::resize_factor;
+  using AllocBytes = typename std::allocator_traits<A>::template rebind_alloc<uint8_t>;
+  using vector_bytes = std::vector<uint8_t, AllocBytes>;
+
+  class builder: public theta_base_builder<true> {
+  public:
+      theta_sketch_experimental build() const;
+  };
+
+  bool is_empty() const { return table_.is_empty_; }
+  bool is_ordered() const { return false; }
+  uint16_t get_seed_hash() const { return compute_seed_hash(DEFAULT_SEED); }
+  uint64_t get_theta64() const { return table_.theta_; }
+  uint32_t get_num_retained() const { return table_.num_entries_; }
+
+  void update(uint64_t key);
+  void update(const void* key, size_t length);
+
+  string<A> to_string(bool detail = false) const;
+
+  vector_bytes serialize(unsigned header_size_bytes = 0) const;
+
+  using const_iterator = theta_const_iterator<uint64_t, trivial_extract_key<uint64_t>>;
+  const_iterator begin() const;
+  const_iterator end() const;
+
+  compact_theta_sketch_experimental<A> compact(bool ordered = true) const;
+
+private:
+  enum flags { IS_BIG_ENDIAN, IS_READ_ONLY, IS_EMPTY, IS_COMPACT, IS_ORDERED };
+  typedef theta_update_sketch_base<uint64_t, trivial_extract_key<uint64_t>, A> theta_table;
+  theta_table table_;
+
+  theta_sketch_experimental(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed);
+};
+
+template<typename A = std::allocator<uint64_t>>
+class compact_theta_sketch_experimental {
+public:
+  compact_theta_sketch_experimental(const theta_sketch_experimental<A>& other, bool ordered);
+
+  template<typename InputIt>
+  compact_theta_sketch_experimental(bool is_empty, bool is_ordered, uint16_t seed_hash, uint64_t theta, InputIt first, InputIt last);
+
+  uint32_t get_num_retained() const { return entries_.size(); }
+
+  string<A> to_string(bool detail = false) const;
+
+private:
+  bool is_empty_;
+  bool is_ordered_;
+  uint16_t seed_hash_;
+  uint64_t theta_;
+  std::vector<uint64_t, A> entries_;
+};
+
+} /* namespace datasketches */
+
+#include "theta_sketch_experimental_impl.hpp"
+
+#endif
diff --git a/tuple/include/theta_sketch_experimental_impl.hpp b/tuple/include/theta_sketch_experimental_impl.hpp
new file mode 100644
index 0000000..347022f
--- /dev/null
+++ b/tuple/include/theta_sketch_experimental_impl.hpp
@@ -0,0 +1,152 @@
+/*
+ * 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.
+ */
+
+#include <sstream>
+
+namespace datasketches {
+
+// experimental update theta sketch derived from the same base as tuple sketch
+
+template<typename A>
+theta_sketch_experimental<A>::theta_sketch_experimental(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed):
+table_(lg_cur_size, lg_nom_size, rf, p, seed)
+{}
+
+template<typename A>
+void theta_sketch_experimental<A>::update(uint64_t key) {
+  update(&key, sizeof(key));
+}
+
+template<typename A>
+void theta_sketch_experimental<A>::update(const void* key, size_t length) {
+  const uint64_t hash = compute_hash(key, length, DEFAULT_SEED);
+  if (hash >= table_.theta_ || hash == 0) return; // hash == 0 is reserved to mark empty slots in the table
+  auto result = table_.find(hash);
+  if (!result.second) {
+    table_.insert(result.first, hash);
+  }
+}
+
+template<typename A>
+string<A> theta_sketch_experimental<A>::to_string(bool detail) const {
+  std::basic_ostringstream<char, std::char_traits<char>, AllocChar<A>> os;
+  auto type = typeid(*this).name();
+  os << "sizeof(" << type << ")=" << sizeof(*this) << std::endl;
+  os << table_.to_string();
+  if (detail) {
+    for (const auto& it: table_) {
+      if (it != 0) {
+        os << it << std::endl;
+      }
+    }
+  }
+  return os.str();
+}
+
+template<typename A>
+auto theta_sketch_experimental<A>::serialize(unsigned header_size_bytes) const -> vector_bytes {
+  const uint8_t preamble_longs = 3;
+  const size_t size = header_size_bytes + sizeof(uint64_t) * preamble_longs + sizeof(uint64_t) * (1 << table_.lg_cur_size_);
+  vector_bytes bytes(size);
+  uint8_t* ptr = bytes.data() + header_size_bytes;
+
+  const uint8_t preamble_longs_and_rf = preamble_longs | (table_.rf_ << 6);
+  ptr += copy_to_mem(&preamble_longs_and_rf, ptr, sizeof(preamble_longs_and_rf));
+  const uint8_t serial_version = 0;
+  ptr += copy_to_mem(&serial_version, ptr, sizeof(serial_version));
+  const uint8_t type = 0;
+  ptr += copy_to_mem(&type, ptr, sizeof(type));
+  ptr += copy_to_mem(&table_.lg_nom_size_, ptr, sizeof(table_.lg_nom_size_));
+  ptr += copy_to_mem(&table_.lg_cur_size_, ptr, sizeof(table_.lg_cur_size_));
+  const uint8_t flags_byte(
+    (this->is_empty() ? 1 << flags::IS_EMPTY : 0)
+  );
+  ptr += copy_to_mem(&flags_byte, ptr, sizeof(flags_byte));
+  const uint16_t seed_hash = 0;
+  ptr += copy_to_mem(&seed_hash, ptr, sizeof(seed_hash));
+  ptr += copy_to_mem(&table_.num_entries_, ptr, sizeof(table_.num_entries_));
+  const float p = 1;
+  ptr += copy_to_mem(&p, ptr, sizeof(p));
+  ptr += copy_to_mem(&table_.theta_, ptr, sizeof(table_.theta_));
+  ptr += copy_to_mem(table_.entries_, ptr, sizeof(uint64_t) * (1 << table_.lg_cur_size_));
+
+  return bytes;
+}
+
+template<typename A>
+auto theta_sketch_experimental<A>::begin() const -> const_iterator {
+  return const_iterator(table_.entries_, 1 << table_.lg_cur_size_, 0);
+}
+
+template<typename A>
+auto theta_sketch_experimental<A>::end() const -> const_iterator {
+  return const_iterator(nullptr, 0, 1 << table_.lg_cur_size_);
+}
+
+template<typename A>
+theta_sketch_experimental<A> theta_sketch_experimental<A>::builder::build() const {
+  return theta_sketch_experimental(starting_sub_multiple(lg_k_ + 1, MIN_LG_K, static_cast<uint8_t>(rf_)), lg_k_, rf_, p_, seed_);
+}
+
+template<typename A>
+compact_theta_sketch_experimental<A> theta_sketch_experimental<A>::compact(bool ordered) const {
+  return compact_theta_sketch_experimental<A>(*this, ordered);
+}
+
+// experimental compact theta sketch
+
+template<typename A>
+compact_theta_sketch_experimental<A>::compact_theta_sketch_experimental(const theta_sketch_experimental<A>& other, bool ordered):
+is_empty_(other.is_empty()),
+is_ordered_(other.is_ordered()),
+seed_hash_(other.get_seed_hash()),
+theta_(other.get_theta64()),
+entries_()
+{
+  entries_.reserve(other.get_num_retained());
+  std::copy(other.begin(), other.end(), std::back_inserter(entries_));
+  if (ordered && !other.is_ordered()) std::sort(entries_.begin(), entries_.end());
+}
+
+template<typename A>
+template<typename InputIt>
+compact_theta_sketch_experimental<A>::compact_theta_sketch_experimental(bool is_empty, bool is_ordered, uint16_t seed_hash, uint64_t theta, InputIt first, InputIt last):
+is_empty_(is_empty),
+is_ordered_(is_ordered),
+seed_hash_(seed_hash),
+theta_(theta),
+entries_()
+{
+  std::copy_if(first, last, std::back_inserter(entries_), [theta](uint64_t value) { return value != 0 && value < theta; });
+}
+
+template<typename A>
+string<A> compact_theta_sketch_experimental<A>::to_string(bool detail) const {
+  std::basic_ostringstream<char, std::char_traits<char>, AllocChar<A>> os;
+  auto type = typeid(*this).name();
+  os << "sizeof(" << type << ")=" << sizeof(*this) << std::endl;
+  if (detail) {
+    for (const auto& hash: entries_) {
+      os << hash << std::endl;
+    }
+  }
+  return os.str();
+}
+
+} /* namespace datasketches */
diff --git a/tuple/include/theta_union_base.hpp b/tuple/include/theta_union_base.hpp
new file mode 100644
index 0000000..e62dc79
--- /dev/null
+++ b/tuple/include/theta_union_base.hpp
@@ -0,0 +1,56 @@
+/*
+ * 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 THETA_UNION_BASE_HPP_
+#define THETA_UNION_BASE_HPP_
+
+#include "theta_update_sketch_base.hpp"
+
+namespace datasketches {
+
+template<
+  typename Entry,
+  typename ExtractKey,
+  typename Policy,
+  typename Sketch,
+  typename CompactSketch,
+  typename Allocator = std::allocator<Entry>
+>
+class theta_union_base {
+public:
+  using hash_table = theta_update_sketch_base<Entry, ExtractKey, Allocator>;
+  using resize_factor = theta_constants::resize_factor;
+
+  theta_union_base(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed, Policy policy);
+
+  void update(const Sketch& sketch);
+
+  CompactSketch get_result(bool ordered = true) const;
+
+private:
+  Policy policy_;
+  hash_table table_;
+  uint64_t union_theta_;
+};
+
+} /* namespace datasketches */
+
+#include "theta_union_base_impl.hpp"
+
+#endif
diff --git a/tuple/include/theta_union_base_impl.hpp b/tuple/include/theta_union_base_impl.hpp
new file mode 100644
index 0000000..2e6def6
--- /dev/null
+++ b/tuple/include/theta_union_base_impl.hpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+#include <iostream>
+#include <sstream>
+#include <algorithm>
+
+namespace datasketches {
+
+template<typename E, typename EK, typename P, typename S, typename CS, typename A>
+theta_union_base<E, EK, P, S, CS, A>::theta_union_base(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed, P policy):
+policy_(policy),
+table_(lg_cur_size, lg_nom_size, rf, p, seed),
+union_theta_(table_.theta_)
+{}
+
+template<typename E, typename EK, typename P, typename S, typename CS, typename A>
+void theta_union_base<E, EK, P, S, CS, A>::update(const S& sketch) {
+  if (sketch.is_empty()) return;
+  if (sketch.get_seed_hash() != compute_seed_hash(table_.seed_)) throw std::invalid_argument("seed hash mismatch");
+  table_.is_empty_ = false;
+  if (sketch.get_theta64() < union_theta_) union_theta_ = sketch.get_theta64();
+  for (const auto& entry: sketch) {
+    const uint64_t hash = EK()(entry);
+    if (hash < union_theta_) {
+      auto result = table_.find(hash);
+      if (!result.second) {
+        table_.insert(result.first, entry);
+      } else {
+        *result.first = policy_(*result.first, entry);
+      }
+    } else {
+      if (sketch.is_ordered()) break; // early stop
+    }
+  }
+  if (table_.theta_ < union_theta_) union_theta_ = table_.theta_;
+}
+
+template<typename E, typename EK, typename P, typename S, typename CS, typename A>
+CS theta_union_base<E, EK, P, S, CS, A>::get_result(bool ordered) const {
+  // relies on the constructor to skip holes and hashes < theta
+  return CS(table_.is_empty_, ordered, DEFAULT_SEED, theta_constants::MAX_THETA, table_.begin(), table_.end());
+}
+
+} /* namespace datasketches */
diff --git a/tuple/include/theta_union_experimental.hpp b/tuple/include/theta_union_experimental.hpp
new file mode 100644
index 0000000..585c5fe
--- /dev/null
+++ b/tuple/include/theta_union_experimental.hpp
@@ -0,0 +1,87 @@
+/*
+ * 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 THETA_UNION_EXPERIMENTAL_HPP_
+#define THETA_UNION_EXPERIMENTAL_HPP_
+
+#include "serde.hpp"
+#include "tuple_sketch.hpp"
+#include "theta_union_base.hpp"
+#include "theta_sketch_experimental.hpp"
+
+namespace datasketches {
+
+// experimental theta union derived from the same base as tuple union
+
+template<typename Allocator = std::allocator<uint64_t>>
+class theta_union_experimental {
+public:
+  using Entry = uint64_t;
+  using ExtractKey = trivial_extract_key<uint64_t>;
+  using Sketch = theta_sketch_experimental<Allocator>;
+  using CompactSketch = compact_theta_sketch_experimental<Allocator>;
+  using resize_factor = theta_constants::resize_factor;
+
+  struct pass_through_policy {
+    uint64_t operator()(uint64_t internal_entry, uint64_t incoming_entry) const {
+      return internal_entry;
+    }
+  };
+
+  using State = theta_union_base<Entry, ExtractKey, pass_through_policy, Sketch, CompactSketch, Allocator>;
+
+  // No constructor here. Use builder instead.
+  class builder;
+
+  /**
+   * This method is to update the union with a given sketch
+   * @param sketch to update the union with
+   */
+  void update(const Sketch& sketch);
+
+  /**
+   * This method produces a copy of the current state of the union as a compact sketch.
+   * @param ordered optional flag to specify if ordered sketch should be produced
+   * @return the result of the union
+   */
+  CompactSketch get_result(bool ordered = true) const;
+
+private:
+  State state_;
+
+  // for builder
+  theta_union_experimental(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed);
+};
+
+template<typename A>
+class theta_union_experimental<A>::builder: public theta_base_builder<true> {
+public:
+
+  /**
+   * This is to create an instance of the union with predefined parameters.
+   * @return an instance of the union
+   */
+  theta_union_experimental<A> build() const;
+};
+
+} /* namespace datasketches */
+
+#include "theta_union_experimental_impl.hpp"
+
+#endif
diff --git a/tuple/include/theta_union_experimental_impl.hpp b/tuple/include/theta_union_experimental_impl.hpp
new file mode 100644
index 0000000..be75024
--- /dev/null
+++ b/tuple/include/theta_union_experimental_impl.hpp
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+namespace datasketches {
+
+template<typename A>
+theta_union_experimental<A>::theta_union_experimental(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed):
+state_(lg_cur_size, lg_nom_size, rf, p, seed, pass_through_policy())
+{}
+
+template<typename A>
+void theta_union_experimental<A>::update(const Sketch& sketch) {
+  state_.update(sketch);
+}
+
+template<typename A>
+auto theta_union_experimental<A>::get_result(bool ordered) const -> CompactSketch {
+  return state_.get_result(ordered);
+}
+
+template<typename A>
+auto theta_union_experimental<A>::builder::build() const -> theta_union_experimental {
+  return theta_union_experimental(starting_sub_multiple(lg_k_ + 1, MIN_LG_K, static_cast<uint8_t>(rf_)), lg_k_, rf_, p_, seed_);
+}
+
+} /* namespace datasketches */
diff --git a/tuple/include/theta_update_sketch_base.hpp b/tuple/include/theta_update_sketch_base.hpp
new file mode 100644
index 0000000..e90091f
--- /dev/null
+++ b/tuple/include/theta_update_sketch_base.hpp
@@ -0,0 +1,227 @@
+/*
+ * 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 THETA_UPDATE_SKETCH_BASE_HPP_
+#define THETA_UPDATE_SKETCH_BASE_HPP_
+
+#include <vector>
+#include <climits>
+
+#include "common_defs.hpp"
+#include "MurmurHash3.h"
+
+namespace datasketches {
+
+namespace theta_constants {
+  enum resize_factor { X1, X2, X4, X8 };
+  static const uint64_t MAX_THETA = LLONG_MAX; // signed max for compatibility with Java
+}
+
+template<
+  typename Entry,
+  typename ExtractKey,
+  typename Allocator = std::allocator<Entry>
+>
+struct theta_update_sketch_base {
+  using resize_factor = theta_constants::resize_factor;
+
+  theta_update_sketch_base(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed);
+  // TODO: copy and move
+  ~theta_update_sketch_base();
+
+  typedef Entry* iterator;
+
+  std::pair<iterator, bool> find(uint64_t key) const;
+
+  template<typename FwdEntry>
+  void insert(iterator it, FwdEntry&& entry);
+
+  iterator begin() const;
+  iterator end() const;
+
+  string<Allocator> to_string() const;
+
+  // resize threshold = 0.5 tuned for speed
+  static constexpr double RESIZE_THRESHOLD = 0.5;
+  // hash table rebuild threshold = 15/16
+  static constexpr double REBUILD_THRESHOLD = 15.0 / 16.0;
+
+  static constexpr uint8_t STRIDE_HASH_BITS = 7;
+  static constexpr uint32_t STRIDE_MASK = (1 << STRIDE_HASH_BITS) - 1;
+
+  bool is_empty_;
+  uint8_t lg_cur_size_;
+  uint8_t lg_nom_size_;
+  resize_factor rf_;
+  float p_;
+  uint32_t num_entries_;
+  uint64_t theta_;
+  uint64_t seed_;
+  Entry* entries_;
+
+  void resize();
+  void rebuild();
+
+  static inline uint32_t get_capacity(uint8_t lg_cur_size, uint8_t lg_nom_size);
+  static inline uint32_t get_stride(uint64_t key, uint8_t lg_size);
+
+  struct comparator {
+    bool operator()(Entry& a, Entry& b) const {
+      return ExtractKey()(a) < ExtractKey()(b);
+    }
+  };
+};
+
+// builder
+
+template<bool dummy>
+class theta_base_builder {
+public:
+  using resize_factor = theta_constants::resize_factor;
+  static const uint8_t MIN_LG_K = 5;
+  static const uint8_t DEFAULT_LG_K = 12;
+  static const resize_factor DEFAULT_RESIZE_FACTOR = resize_factor::X8;
+
+  /**
+   * Creates and instance of the builder with default parameters.
+   */
+  theta_base_builder();
+
+  /**
+   * Set log2(k), where k is a nominal number of entries in the sketch
+   * @param lg_k base 2 logarithm of nominal number of entries
+   * @return this builder
+   */
+  theta_base_builder& set_lg_k(uint8_t lg_k);
+
+  /**
+   * Set resize factor for the internal hash table (defaults to 8)
+   * @param rf resize factor
+   * @return this builder
+   */
+  theta_base_builder& set_resize_factor(resize_factor rf);
+
+  /**
+   * Set sampling probability (initial theta). The default is 1, so the sketch retains
+   * all entries until it reaches the limit, at which point it goes into the estimation mode
+   * and reduces the effective sampling probability (theta) as necessary.
+   * @param p sampling probability
+   * @return this builder
+   */
+  theta_base_builder& set_p(float p);
+
+  /**
+   * Set the seed for the hash function. Should be used carefully if needed.
+   * Sketches produced with different seed are not compatible
+   * and cannot be mixed in set operations.
+   * @param seed hash seed
+   * @return this builder
+   */
+  theta_base_builder& set_seed(uint64_t seed);
+
+protected:
+  uint8_t lg_k_;
+  resize_factor rf_;
+  float p_;
+  uint64_t seed_;
+
+  static uint8_t starting_sub_multiple(uint8_t lg_tgt, uint8_t lg_min, uint8_t lg_rf);
+};
+
+// key extractors
+
+template<typename T>
+struct trivial_extract_key {
+  T& operator()(T& entry) const {
+    return entry;
+  }
+  const T& operator()(const T& entry) const {
+    return entry;
+  }
+};
+
+template<typename K, typename V>
+struct pair_extract_key {
+  K& operator()(std::pair<K, V>& entry) const {
+    return entry.first;
+  }
+  const K& operator()(const std::pair<K, V>& entry) const {
+    return entry.first;
+  }
+};
+
+// less than
+
+template<typename T>
+class less_than {
+public:
+  explicit less_than(const T& value): value(value) {}
+  bool operator()(const T& value) const { return value < this->value; }
+private:
+  T value;
+};
+
+template<typename Key, typename Entry, typename ExtractKey>
+class key_less_than {
+public:
+  explicit key_less_than(const Key& key): key(key) {}
+  bool operator()(const Entry& entry) const {
+    return ExtractKey()(entry) < this->key;
+  }
+private:
+  Key key;
+};
+
+// MurMur3 hash functions
+
+static inline uint64_t compute_hash(const void* data, size_t length, uint64_t seed) {
+  HashState hashes;
+  MurmurHash3_x64_128(data, length, seed, hashes);
+  return (hashes.h1 >> 1); // Java implementation does unsigned shift >>> to make values positive
+}
+
+static inline uint16_t compute_seed_hash(uint64_t seed) {
+  HashState hashes;
+  MurmurHash3_x64_128(&seed, sizeof(seed), 0, hashes);
+  return hashes.h1;
+}
+
+// iterator
+
+template<typename Entry, typename ExtractKey>
+class theta_const_iterator: public std::iterator<std::input_iterator_tag, Entry> {
+public:
+  theta_const_iterator(const Entry* entries, uint32_t size, uint32_t index);
+  theta_const_iterator& operator++();
+  theta_const_iterator operator++(int);
+  bool operator==(const theta_const_iterator& other) const;
+  bool operator!=(const theta_const_iterator& other) const;
+  const Entry& operator*() const;
+
+private:
+  const Entry* entries_;
+  uint32_t size_;
+  uint32_t index_;
+};
+
+} /* namespace datasketches */
+
+#include "theta_update_sketch_base_impl.hpp"
+
+#endif
diff --git a/tuple/include/theta_update_sketch_base_impl.hpp b/tuple/include/theta_update_sketch_base_impl.hpp
new file mode 100644
index 0000000..f65f3e9
--- /dev/null
+++ b/tuple/include/theta_update_sketch_base_impl.hpp
@@ -0,0 +1,255 @@
+/*
+ * 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.
+ */
+
+#include <iostream>
+#include <sstream>
+#include <algorithm>
+
+namespace datasketches {
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed):
+is_empty_(true),
+lg_cur_size_(lg_cur_size),
+lg_nom_size_(lg_nom_size),
+rf_(rf),
+p_(p),
+num_entries_(0),
+theta_(theta_constants::MAX_THETA),
+seed_(seed),
+entries_(nullptr)
+{
+  const size_t size = 1 << (lg_cur_size + 1);
+  entries_ = A().allocate(size);
+  for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  if (p < 1) this->theta_ *= p;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::~theta_update_sketch_base()
+{
+  const size_t size = 1 << lg_cur_size_;
+  for (size_t i = 0; i < size; ++i) {
+    if (EK()(entries_[i]) != 0) entries_[i].~EN();
+  }
+  A().deallocate(entries_, size);
+}
+
+//template<typename EN, typename EK, typename A>
+//void theta_base<EN, EK, A>::update(EN&& entry) {
+//  const uint64_t hash = EK()(entry);
+//  if (hash >= this->theta_ || hash == 0) return; // hash == 0 is reserved to mark empty slots in the table
+//  auto result = find(hash);
+//  if (!result.second) {
+//    insert(result.first, std::forward<EN>(entry));
+//  }
+//}
+
+template<typename EN, typename EK, typename A>
+auto theta_update_sketch_base<EN, EK, A>::find(uint64_t key) const -> std::pair<iterator, bool> {
+  const size_t size = 1 << lg_cur_size_;
+  const size_t mask = size - 1;
+  const uint32_t stride = get_stride(key, lg_cur_size_);
+  uint32_t index = static_cast<uint32_t>(key) & mask;
+  // search for duplicate or zero
+  const uint32_t loop_index = index;
+  do {
+    const uint64_t probe = EK()(entries_[index]);
+    if (probe == 0) {
+      return std::pair<iterator, bool>(&entries_[index], false);
+    } else if (probe == key) {
+      return std::pair<iterator, bool>(&entries_[index], true);
+    }
+    index = (index + stride) & mask;
+  } while (index != loop_index);
+  throw std::logic_error("key not found and no empty slots!");
+}
+
+template<typename EN, typename EK, typename A>
+template<typename Fwd>
+void theta_update_sketch_base<EN, EK, A>::insert(iterator it, Fwd&& entry) {
+  new (it) EN(std::forward<Fwd>(entry));
+  is_empty_ = false;
+  ++num_entries_;
+  if (num_entries_ > get_capacity(lg_cur_size_, lg_nom_size_)) {
+    if (lg_cur_size_ <= lg_nom_size_) {
+      resize();
+    } else {
+      rebuild();
+    }
+  }
+}
+
+template<typename EN, typename EK, typename A>
+auto theta_update_sketch_base<EN, EK, A>::begin() const -> iterator {
+  return entries_;
+}
+
+template<typename EN, typename EK, typename A>
+auto theta_update_sketch_base<EN, EK, A>::end() const -> iterator {
+  return &entries_[1 << lg_cur_size_];
+}
+
+template<typename EN, typename EK, typename A>
+string<A> theta_update_sketch_base<EN, EK, A>::to_string() const {
+  std::basic_ostringstream<char, std::char_traits<char>, AllocChar<A>> os;
+  auto type = typeid(*this).name();
+  os << "type: " << type << std::endl;
+  os << "sizeof: " << sizeof(*this) << std::endl;
+  os << "is_empty:    " << (is_empty_ ? "true" : "false") << std::endl;
+  os << "lg_cur_size: " << std::to_string(lg_cur_size_) << std::endl;
+  os << "lg_nom_size: " << std::to_string(lg_nom_size_) << std::endl;
+  os << "num_entries: " << num_entries_ << std::endl;
+  os << "theta (as long): " << theta_ << std::endl;
+  os << "theta (as fraction): " << static_cast<double>(theta_) / theta_constants::MAX_THETA << std::endl;
+  return os.str();
+}
+
+template<typename EN, typename EK, typename A>
+uint32_t theta_update_sketch_base<EN, EK, A>::get_capacity(uint8_t lg_cur_size, uint8_t lg_nom_size) {
+  const double fraction = (lg_cur_size <= lg_nom_size) ? RESIZE_THRESHOLD : REBUILD_THRESHOLD;
+  return std::floor(fraction * (1 << lg_cur_size));
+}
+
+template<typename EN, typename EK, typename A>
+uint32_t theta_update_sketch_base<EN, EK, A>::get_stride(uint64_t key, uint8_t lg_size) {
+  // odd and independent of index assuming lg_size lowest bits of the key were used for the index
+  return (2 * static_cast<uint32_t>((key >> lg_size) & STRIDE_MASK)) + 1;
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::resize() {
+  const size_t old_size = 1 << lg_cur_size_;
+  const uint8_t lg_tgt_size = lg_nom_size_ + 1;
+  const uint8_t factor = std::max(1, std::min(static_cast<int>(rf_), lg_tgt_size - lg_cur_size_));
+  lg_cur_size_ += factor;
+  const size_t new_size = 1 << lg_cur_size_;
+//  std::cout << "resizing from " << old_size << " to " << new_size << std::endl;
+  EN* old_entries = entries_;
+  entries_ = A().allocate(new_size);
+  for (size_t i = 0; i < new_size; ++i) EK()(entries_[i]) = 0;
+  num_entries_ = 0;
+  for (size_t i = 0; i < old_size; ++i) {
+    const uint64_t key = EK()(old_entries[i]);
+    if (key != 0) {
+      insert(find(key).first, std::move(old_entries[i])); // consider a special insert with no comparison
+      old_entries[i].~EN();
+    }
+  }
+  A().deallocate(old_entries, old_size);
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::rebuild() {
+//  std::cout << "rebuilding" << std::endl;
+  const size_t size = 1 << lg_cur_size_;
+  const uint32_t pivot = (1 << lg_nom_size_) + size - num_entries_;
+  std::nth_element(&entries_[0], &entries_[pivot], &entries_[size], comparator());
+  this->theta_ = EK()(entries_[pivot]);
+  EN* old_entries = entries_;
+  entries_ = A().allocate(size);
+  for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  num_entries_ = 0;
+  for (size_t i = 0; i < size; ++i) {
+    const uint64_t key = EK()(old_entries[i]);
+    if (key != 0 && key < this->theta_) {
+      insert(find(key).first, std::move(old_entries[i])); // consider a special insert with no comparison
+      old_entries[i].~EN();
+    }
+  }
+  A().deallocate(old_entries, size);
+}
+
+// builder
+
+template<bool dummy>
+theta_base_builder<dummy>::theta_base_builder():
+lg_k_(DEFAULT_LG_K), rf_(DEFAULT_RESIZE_FACTOR), p_(1), seed_(DEFAULT_SEED) {}
+
+template<bool dummy>
+auto theta_base_builder<dummy>::set_lg_k(uint8_t lg_k) -> theta_base_builder& {
+  if (lg_k < MIN_LG_K) {
+    throw std::invalid_argument("lg_k must not be less than " + std::to_string(MIN_LG_K) + ": " + std::to_string(lg_k));
+  }
+  lg_k_ = lg_k;
+  return *this;
+}
+
+template<bool dummy>
+auto theta_base_builder<dummy>::set_resize_factor(resize_factor rf) -> theta_base_builder& {
+  rf_ = rf;
+  return *this;
+}
+
+template<bool dummy>
+auto theta_base_builder<dummy>::set_p(float p) -> theta_base_builder& {
+  if (p < 0 || p > 1) throw std::invalid_argument("sampling probability must be between 0 and 1");
+  p_ = p;
+  return *this;
+}
+
+template<bool dummy>
+auto theta_base_builder<dummy>::set_seed(uint64_t seed) -> theta_base_builder& {
+  seed_ = seed;
+  return *this;
+}
+
+template<bool dummy>
+uint8_t theta_base_builder<dummy>::starting_sub_multiple(uint8_t lg_tgt, uint8_t lg_min, uint8_t lg_rf) {
+  return (lg_tgt <= lg_min) ? lg_min : (lg_rf == 0) ? lg_tgt : ((lg_tgt - lg_min) % lg_rf) + lg_min;
+}
+
+// iterator
+
+template<typename Entry, typename ExtractKey>
+theta_const_iterator<Entry, ExtractKey>::theta_const_iterator(const Entry* entries, uint32_t size, uint32_t index):
+entries_(entries), size_(size), index_(index) {
+  while (index_ < size_ && ExtractKey()(entries_[index_]) == 0) ++index_;
+}
+
+template<typename Entry, typename ExtractKey>
+auto theta_const_iterator<Entry, ExtractKey>::operator++() -> theta_const_iterator& {
+  ++index_;
+  while (index_ < size_ && ExtractKey()(entries_[index_]) == 0) ++index_;
+  return *this;
+}
+
+template<typename Entry, typename ExtractKey>
+auto theta_const_iterator<Entry, ExtractKey>::operator++(int) -> theta_const_iterator {
+  theta_const_iterator tmp(*this);
+  operator++();
+  return tmp;
+}
+
+template<typename Entry, typename ExtractKey>
+bool theta_const_iterator<Entry, ExtractKey>::operator!=(const theta_const_iterator& other) const {
+  return index_ != other.index_;
+}
+
+template<typename Entry, typename ExtractKey>
+bool theta_const_iterator<Entry, ExtractKey>::operator==(const theta_const_iterator& other) const {
+  return index_ == other.index_;
+}
+
+template<typename Entry, typename ExtractKey>
+auto theta_const_iterator<Entry, ExtractKey>::operator*() const -> const Entry& {
+  return entries_[index_];
+}
+
+} /* namespace datasketches */
diff --git a/tuple/include/tuple_sketch.hpp b/tuple/include/tuple_sketch.hpp
new file mode 100644
index 0000000..55dda48
--- /dev/null
+++ b/tuple/include/tuple_sketch.hpp
@@ -0,0 +1,502 @@
+/*
+ * 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 TUPLE_SKETCH_HPP_
+#define TUPLE_SKETCH_HPP_
+
+#include <string>
+
+#include "serde.hpp"
+#include "theta_update_sketch_base.hpp"
+
+namespace datasketches {
+
+// forward-declarations
+template<typename S, typename SD, typename A> class tuple_sketch;
+template<typename S, typename U, typename P, typename SD, typename A> class update_tuple_sketch;
+template<typename S, typename SD, typename A> class compact_tuple_sketch;
+
+template<
+  typename Summary,
+  typename SerDe = serde<Summary>,
+  typename Allocator = std::allocator<Summary>
+>
+class tuple_sketch {
+public:
+  static const uint8_t SERIAL_VERSION = 3;
+
+  virtual ~tuple_sketch() = default;
+
+  /**
+   * @return true if this sketch represents an empty set (not the same as no retained entries!)
+   */
+  virtual bool is_empty() const = 0;
+
+  /**
+   * @return estimate of the distinct count of the input stream
+   */
+  double get_estimate() const;
+
+  /**
+   * Returns the approximate lower error bound given a number of standard deviations.
+   * This parameter is similar to the number of standard deviations of the normal distribution
+   * and corresponds to approximately 67%, 95% and 99% confidence intervals.
+   * @param num_std_devs number of Standard Deviations (1, 2 or 3)
+   * @return the lower bound
+   */
+  double get_lower_bound(uint8_t num_std_devs) const;
+
+  /**
+   * Returns the approximate upper error bound given a number of standard deviations.
+   * This parameter is similar to the number of standard deviations of the normal distribution
+   * and corresponds to approximately 67%, 95% and 99% confidence intervals.
+   * @param num_std_devs number of Standard Deviations (1, 2 or 3)
+   * @return the upper bound
+   */
+  double get_upper_bound(uint8_t num_std_devs) const;
+
+  /**
+   * @return true if the sketch is in estimation mode (as opposed to exact mode)
+   */
+  bool is_estimation_mode() const;
+
+  /**
+   * @return theta as a fraction from 0 to 1 (effective sampling rate)
+   */
+  double get_theta() const;
+
+  /**
+   * @return theta as a positive integer between 0 and LLONG_MAX
+   */
+  virtual uint64_t get_theta64() const = 0;
+
+  /**
+   * @return the number of retained entries in the sketch
+   */
+  virtual uint32_t get_num_retained() const = 0;
+
+  virtual uint16_t get_seed_hash() const = 0;
+
+  /**
+   * @return true if retained entries are ordered
+   */
+  virtual bool is_ordered() const = 0;
+
+  /**
+   * Writes a human-readable summary of this sketch to a given stream
+   * @param print_items if true include the list of items retained by the sketch
+   */
+  virtual string<Allocator> to_string(bool print_items = false) const = 0;
+
+  /**
+   * This method serializes the sketch into a given stream in a binary form
+   * @param os output stream
+   */
+//  virtual void serialize(std::ostream& os) const = 0;
+
+  // This is a convenience alias for users
+  // The type returned by the following serialize method
+//  typedef vector_u8<Allocator> vector_bytes;
+
+  /**
+   * This method serializes the sketch as a vector of bytes.
+   * An optional header can be reserved in front of the sketch.
+   * It is an uninitialized space of a given size.
+   * This header is used in Datasketches PostgreSQL extension.
+   * @param header_size_bytes space to reserve in front of the sketch
+   */
+//  virtual vector_bytes serialize(unsigned header_size_bytes = 0) const = 0;
+
+  // This is a convenience alias for users
+  // The type returned by the following deserialize methods
+  // It is not possible to return instances of an abstract type, so this has to be a pointer
+//  typedef std::unique_ptr<tuple_sketch, std::function<void(tuple_sketch*)>> unique_ptr;
+
+  /**
+   * This method deserializes a sketch from a given stream.
+   * @param is input stream
+   * @param seed the seed for the hash function that was used to create the sketch
+   * @return an instance of a sketch as a unique_ptr
+   */
+//  static unique_ptr deserialize(std::istream& is, uint64_t seed = DEFAULT_SEED);
+
+  /**
+   * This method deserializes a sketch from a given array of bytes.
+   * @param bytes pointer to the array of bytes
+   * @param size the size of the array
+   * @param seed the seed for the hash function that was used to create the sketch
+   * @return an instance of the sketch
+   */
+//  static unique_ptr deserialize(const void* bytes, size_t size, uint64_t seed = DEFAULT_SEED);
+
+  using Entry = std::pair<uint64_t, Summary>;
+  using const_iterator = theta_const_iterator<Entry, pair_extract_key<uint64_t, Summary>>;
+
+  /**
+   * Iterator over entries in this sketch.
+   * @return begin iterator
+   */
+  virtual const_iterator begin() const = 0;
+
+  /**
+   * Iterator pointing past the valid range.
+   * Not to be incremented or dereferenced.
+   * @return end iterator
+   */
+  virtual const_iterator end() const = 0;
+
+protected:
+  enum flags { IS_BIG_ENDIAN, IS_READ_ONLY, IS_EMPTY, IS_COMPACT, IS_ORDERED };
+
+  static uint16_t get_seed_hash(uint64_t seed);
+
+  static void check_sketch_type(uint8_t actual, uint8_t expected);
+  static void check_serial_version(uint8_t actual, uint8_t expected);
+  static void check_seed_hash(uint16_t actual, uint16_t expected);
+};
+
+// update sketch
+
+// for types with defined default constructor and + operation
+template<typename Summary, typename Update>
+struct default_update_policy {
+  Summary create() const {
+    return Summary();
+  }
+  void update(Summary& summary, const Update& update) const {
+    summary += update;
+  }
+};
+
+template<
+  typename Summary,
+  typename Update = Summary,
+  typename Policy = default_update_policy<Summary, Update>,
+  typename SerDe = serde<Summary>,
+  typename Allocator = std::allocator<Summary>
+>
+class update_tuple_sketch: public tuple_sketch<Summary, SerDe, Allocator> {
+public:
+  using Base = tuple_sketch<Summary, SerDe, Allocator>;
+  using Entry = typename Base::Entry;
+  using AllocEntry = typename std::allocator_traits<Allocator>::template rebind_alloc<Entry>;
+  using tuple_map = theta_update_sketch_base<Entry, pair_extract_key<uint64_t, Summary>, AllocEntry>;
+  using resize_factor = typename tuple_map::resize_factor;
+  using const_iterator = typename Base::const_iterator;
+
+  static const uint8_t SKETCH_TYPE = 2;
+
+  // No constructor here. Use builder instead.
+  class builder;
+
+  virtual ~update_tuple_sketch() = default;
+
+  virtual bool is_empty() const;
+  virtual bool is_ordered() const;
+  virtual uint64_t get_theta64() const;
+  virtual uint32_t get_num_retained() const;
+  virtual uint16_t get_seed_hash() const;
+  virtual string<Allocator> to_string(bool print_items = false) const;
+
+//  virtual void serialize(std::ostream& os) const;
+//  typedef vector_u8<Allocator> vector_bytes; // alias for users
+//  virtual vector_bytes serialize(unsigned header_size_bytes = 0) const;
+
+  /**
+   * @return configured nominal number of entries in the sketch
+   */
+  uint8_t get_lg_k() const;
+
+  /**
+   * @return configured resize factor of the sketch
+   */
+  resize_factor get_rf() const;
+
+  /**
+   * Update this sketch with a given string.
+   * @param value string to update the sketch with
+   */
+//  template<typename FwdUpdate>
+//  void update(const std::string& key, FwdUpdate&& value);
+
+  /**
+   * Update this sketch with a given unsigned 64-bit integer.
+   * @param value uint64_t to update the sketch with
+   */
+  template<typename FwdUpdate>
+  void update(uint64_t key, FwdUpdate&& value);
+
+  /**
+   * Update this sketch with a given signed 64-bit integer.
+   * @param value int64_t to update the sketch with
+   */
+//  template<typename FwdUpdate>
+//  void update(int64_t key, FwdUpdate&& value);
+//
+//  /**
+//   * Update this sketch with a given unsigned 32-bit integer.
+//   * For compatibility with Java implementation.
+//   * @param value uint32_t to update the sketch with
+//   */
+//  template<typename FwdUpdate>
+//  void update(uint32_t key, FwdUpdate&& value);
+//
+//  /**
+//   * Update this sketch with a given signed 32-bit integer.
+//   * For compatibility with Java implementation.
+//   * @param value int32_t to update the sketch with
+//   */
+//  template<typename FwdUpdate>
+//  void update(int32_t key, FwdUpdate&& value);
+//
+//  /**
+//   * Update this sketch with a given unsigned 16-bit integer.
+//   * For compatibility with Java implementation.
+//   * @param value uint16_t to update the sketch with
+//   */
+//  template<typename FwdUpdate>
+//  void update(uint16_t key, FwdUpdate&& value);
+//
+//  /**
+//   * Update this sketch with a given signed 16-bit integer.
+//   * For compatibility with Java implementation.
+//   * @param value int16_t to update the sketch with
+//   */
+//  template<typename FwdUpdate>
+//  void update(int16_t key, FwdUpdate&& value);
+//
+//  /**
+//   * Update this sketch with a given unsigned 8-bit integer.
+//   * For compatibility with Java implementation.
+//   * @param value uint8_t to update the sketch with
+//   */
+//  template<typename FwdUpdate>
+//  void update(uint8_t key, FwdUpdate&& value);
+//
+//  /**
+//   * Update this sketch with a given signed 8-bit integer.
+//   * For compatibility with Java implementation.
+//   * @param value int8_t to update the sketch with
+//   */
+//  template<typename FwdUpdate>
+//  void update(int8_t key, FwdUpdate&& value);
+//
+//  /**
+//   * Update this sketch with a given double-precision floating point value.
+//   * For compatibility with Java implementation.
+//   * @param value double to update the sketch with
+//   */
+//  template<typename FwdUpdate>
+//  void update(double key, FwdUpdate&& value);
+//
+//  /**
+//   * Update this sketch with a given floating point value.
+//   * For compatibility with Java implementation.
+//   * @param value float to update the sketch with
+//   */
+//  template<typename FwdUpdate>
+//  void update(float key, FwdUpdate&& value);
+
+  /**
+   * Update this sketch with given data of any type.
+   * 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
+   * both over time and on different platforms
+   * and while passing sketches between C++ environment and 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
+   * @param data pointer to the data
+   * @param length of the data in bytes
+   */
+  template<typename FwdUpdate>
+  void update(const void* key, size_t length, FwdUpdate&& value);
+
+  /**
+   * Remove retained entries in excess of the nominal size k (if any)
+   */
+  void trim();
+
+  /**
+   * Converts this sketch to a compact sketch (ordered or unordered).
+   * @param ordered optional flag to specify if ordered sketch should be produced
+   * @return compact sketch
+   */
+  compact_tuple_sketch<Summary, SerDe, Allocator> compact(bool ordered = true) const;
+
+  virtual const_iterator begin() const;
+  virtual const_iterator end() const;
+
+  /**
+   * This method deserializes a sketch from a given stream.
+   * @param is input stream
+   * @param seed the seed for the hash function that was used to create the sketch
+   * @return an instance of a sketch
+   */
+//  static update_tuple_sketch deserialize(std::istream& is, uint64_t seed = DEFAULT_SEED);
+
+  /**
+   * This method deserializes a sketch from a given array of bytes.
+   * @param bytes pointer to the array of bytes
+   * @param size the size of the array
+   * @param seed the seed for the hash function that was used to create the sketch
+   * @return an instance of the sketch
+   */
+//  static update_tuple_sketch deserialize(const void* bytes, size_t size, uint64_t seed = DEFAULT_SEED);
+
+  /**
+   * Computes size needed to serialize the current state of the sketch.
+   * This version is for fixed-size arithmetic types (integral and floating point).
+   * @return size in bytes needed to serialize this sketch
+   */
+//  template<typename SS = Summary, typename std::enable_if<std::is_arithmetic<SS>::value, int>::type = 0>
+//  size_t get_serialized_size_bytes() const;
+
+  /**
+   * Computes size needed to serialize the current state of the sketch.
+   * This version is for all other types and can be expensive since every item needs to be looked at.
+   * @return size in bytes needed to serialize this sketch
+   */
+//  template<typename SS = Summary, typename std::enable_if<!std::is_arithmetic<SS>::value, int>::type = 0>
+//  size_t get_serialized_size_bytes() const;
+
+private:
+  Policy policy_;
+  tuple_map map_;
+
+  // for builder
+  update_tuple_sketch(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed, const Policy& policy);
+};
+
+// compact sketch
+
+template<
+  typename Summary,
+  typename SerDe = serde<Summary>,
+  typename Allocator = std::allocator<Summary>
+>
+class compact_tuple_sketch: public tuple_sketch<Summary, SerDe, Allocator> {
+public:
+  using Base = tuple_sketch<Summary, SerDe, Allocator>;
+  using Entry = typename Base::Entry;
+  using const_iterator = typename Base::const_iterator;
+  using AllocBytes = typename std::allocator_traits<Allocator>::template rebind_alloc<uint8_t>;
+  using vector_bytes = std::vector<uint8_t, AllocBytes>;
+
+  static const uint8_t SKETCH_TYPE = 3;
+
+  // Instances of this type can be obtained:
+  // - by compacting an update_tuple_sketch
+  // - as a result of a set operation
+  // - by deserializing a previously serialized compact sketch
+
+  compact_tuple_sketch(const Base& other, bool ordered);
+  virtual ~compact_tuple_sketch() = default;
+
+  virtual bool is_empty() const;
+  virtual bool is_ordered() const;
+  virtual uint64_t get_theta64() const;
+  virtual uint32_t get_num_retained() const;
+  virtual uint16_t get_seed_hash() const;
+  virtual string<Allocator> to_string(bool print_items = false) const;
+
+  void serialize(std::ostream& os) const;
+  vector_bytes serialize(unsigned header_size_bytes = 0) const;
+
+  virtual const_iterator begin() const;
+  virtual const_iterator end() const;
+
+  /**
+   * This method deserializes a sketch from a given stream.
+   * @param is input stream
+   * @param seed the seed for the hash function that was used to create the sketch
+   * @return an instance of a sketch
+   */
+  static compact_tuple_sketch deserialize(std::istream& is, uint64_t seed = DEFAULT_SEED);
+
+  /**
+   * This method deserializes a sketch from a given array of bytes.
+   * @param bytes pointer to the array of bytes
+   * @param size the size of the array
+   * @param seed the seed for the hash function that was used to create the sketch
+   * @return an instance of the sketch
+   */
+  static compact_tuple_sketch deserialize(const void* bytes, size_t size, uint64_t seed = DEFAULT_SEED);
+
+  // TODO: try to hide this
+  template<typename InputIt>
+  compact_tuple_sketch(bool is_empty, bool is_ordered, uint16_t seed_hash, uint64_t theta, InputIt first, InputIt last);
+
+private:
+  bool is_empty_;
+  bool is_ordered_;
+  uint16_t seed_hash_;
+  uint64_t theta_;
+
+  using AllocEntry = typename std::allocator_traits<Allocator>::template rebind_alloc<Entry>;
+  std::vector<Entry, AllocEntry> entries_;
+
+  //compact_tuple_sketch(bool is_empty, bool is_ordered, uint16_t seed_hash, uint64_t theta, std::vector<Entry, AllocEntry>&& entries);
+
+  /**
+   * Computes size needed to serialize summaries in the sketch.
+   * This version is for fixed-size arithmetic types (integral and floating point).
+   * @return size in bytes needed to serialize summaries in this sketch
+   */
+  template<typename SS = Summary, typename std::enable_if<std::is_arithmetic<SS>::value, int>::type = 0>
+  size_t get_serialized_size_summaries_bytes() const;
+
+  /**
+   * Computes size needed to serialize summaries in the sketch.
+   * This version is for all other types and can be expensive since every item needs to be looked at.
+   * @return size in bytes needed to serialize summaries in this sketch
+   */
+  template<typename SS = Summary, typename std::enable_if<!std::is_arithmetic<SS>::value, int>::type = 0>
+  size_t get_serialized_size_summaries_bytes() const;
+
+//  static compact_tuple_sketch<Summary, SerDe, Allocator> internal_deserialize(std::istream& is, uint8_t preamble_longs, uint8_t flags_byte, uint16_t seed_hash);
+//  static compact_tuple_sketch<Summary, SerDe, Allocator> internal_deserialize(const void* bytes, size_t size, uint8_t preamble_longs, uint8_t flags_byte, uint16_t seed_hash);
+};
+
+// builder
+
+template<typename S, typename U, typename P, typename SD, typename A>
+class update_tuple_sketch<S, U, P, SD, A>::builder: public theta_base_builder<true> {
+public:
+  /**
+   * Creates and instance of the builder with default parameters.
+   */
+  builder(const P& policy = P());
+
+  /**
+   * This is to create an instance of the sketch with predefined parameters.
+   * @return an instance of the sketch
+   */
+  update_tuple_sketch<S, U, P, SD, A> build() const;
+
+private:
+  P policy_;
+};
+
+} /* namespace datasketches */
+
+#include "tuple_sketch_impl.hpp"
+
+#endif
diff --git a/tuple/include/tuple_sketch_impl.hpp b/tuple/include/tuple_sketch_impl.hpp
new file mode 100644
index 0000000..c6bb323
--- /dev/null
+++ b/tuple/include/tuple_sketch_impl.hpp
@@ -0,0 +1,313 @@
+/*
+ * 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.
+ */
+
+#include <sstream>
+
+#include "binomial_bounds.hpp"
+
+namespace datasketches {
+
+template<typename S, typename SD, typename A>
+bool tuple_sketch<S, SD, A>::is_estimation_mode() const {
+  return get_theta64() < theta_constants::MAX_THETA && !is_empty();
+}
+
+template<typename S, typename SD, typename A>
+double tuple_sketch<S, SD, A>::get_theta() const {
+  return static_cast<double>(get_theta64()) / theta_constants::MAX_THETA;
+}
+
+template<typename S, typename SD, typename A>
+double tuple_sketch<S, SD, A>::get_estimate() const {
+  return get_num_retained() / get_theta();
+}
+
+template<typename S, typename SD, typename A>
+double tuple_sketch<S, SD, A>::get_lower_bound(uint8_t num_std_devs) const {
+  if (!is_estimation_mode()) return get_num_retained();
+  return binomial_bounds::get_lower_bound(get_num_retained(), get_theta(), num_std_devs);
+}
+
+template<typename S, typename SD, typename A>
+double tuple_sketch<S, SD, A>::get_upper_bound(uint8_t num_std_devs) const {
+  if (!is_estimation_mode()) return get_num_retained();
+  return binomial_bounds::get_upper_bound(get_num_retained(), get_theta(), num_std_devs);
+}
+
+// update sketch
+
+template<typename S, typename U, typename P, typename SD, typename A>
+update_tuple_sketch<S, U, P, SD, A>::update_tuple_sketch(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed, const P& policy):
+policy_(policy),
+map_(lg_cur_size, lg_nom_size, rf, p, seed)
+{}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+bool update_tuple_sketch<S, U, P, SD, A>::is_empty() const {
+  return map_.is_empty_;
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+bool update_tuple_sketch<S, U, P, SD, A>::is_ordered() const {
+  return false;
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+uint64_t update_tuple_sketch<S, U, P, SD, A>::get_theta64() const {
+  return map_.theta_;
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+uint32_t update_tuple_sketch<S, U, P, SD, A>::get_num_retained() const {
+  return map_.num_entries_;
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+uint16_t update_tuple_sketch<S, U, P, SD, A>::get_seed_hash() const {
+  return compute_seed_hash(map_.seed_);
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+uint8_t update_tuple_sketch<S, U, P, SD, A>::get_lg_k() const {
+  return map_.lg_nom_size_;
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+auto update_tuple_sketch<S, U, P, SD, A>::get_rf() const -> resize_factor {
+  return map_.rf_;
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+template<typename UU>
+void update_tuple_sketch<S, U, P, SD, A>::update(uint64_t key, UU&& value) {
+  update(&key, sizeof(key), std::forward<UU>(value));
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+template<typename UU>
+void update_tuple_sketch<S, U, P, SD, A>::update(const void* key, size_t length, UU&& value) {
+  const uint64_t hash = compute_hash(key, length, map_.seed_);
+  if (hash >= map_.theta_ || hash == 0) return; // hash == 0 is reserved to mark empty slots in the table
+  auto result = map_.find(hash);
+  if (!result.second) {
+    map_.insert(result.first, Entry(hash, policy_.create()));
+  }
+  policy_.update((*result.first).second, std::forward<UU>(value));
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+string<A> update_tuple_sketch<S, U, P, SD, A>::to_string(bool detail) const {
+  std::basic_ostringstream<char, std::char_traits<char>, AllocChar<A>> os;
+  auto type = typeid(*this).name();
+  os << "sizeof(" << type << ")=" << sizeof(*this) << std::endl;
+  os << "sizeof(entry)=" << sizeof(Entry) << std::endl;
+  os << map_.to_string();
+  if (detail) {
+    for (const auto& it: map_) {
+      if (it.first != 0) {
+        os << it.first << ": " << it.second << std::endl;
+      }
+    }
+  }
+  return os.str();
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+auto update_tuple_sketch<S, U, P, SD, A>::begin() const -> const_iterator {
+  return const_iterator(map_.entries_, 1 << map_.lg_cur_size_, 0);
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+auto update_tuple_sketch<S, U, P, SD, A>::end() const -> const_iterator {
+  return const_iterator(nullptr, 0, 1 << map_.lg_cur_size_);
+}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+compact_tuple_sketch<S, SD, A> update_tuple_sketch<S, U, P, SD, A>::compact(bool ordered) const {
+  return compact_tuple_sketch<S, SD, A>(*this, ordered);
+}
+
+// compact sketch
+
+//template<typename S, typename SD, typename A>
+//compact_tuple_sketch<S, SD, A>::compact_tuple_sketch(bool is_empty, bool is_ordered, uint16_t seed_hash, uint64_t theta, std::vector<Entry, AllocEntry>&& entries):
+//is_empty_(is_empty),
+//is_ordered_(is_ordered),
+//seed_hash_(seed_hash),
+//theta_(theta),
+//entries_(std::move(entries))
+//{}
+
+template<typename S, typename SD, typename A>
+template<typename InputIt>
+compact_tuple_sketch<S, SD, A>::compact_tuple_sketch(bool is_empty, bool is_ordered, uint16_t seed_hash, uint64_t theta, InputIt first, InputIt last):
+is_empty_(is_empty),
+is_ordered_(is_ordered),
+seed_hash_(seed_hash),
+theta_(theta),
+entries_()
+{
+  std::copy_if(first, last, back_inserter(entries_), key_less_than<uint64_t, Entry, pair_extract_key<uint64_t, S>>(theta));
+}
+
+template<typename S, typename SD, typename A>
+compact_tuple_sketch<S, SD, A>::compact_tuple_sketch(const Base& other, bool ordered):
+is_empty_(other.is_empty()),
+is_ordered_(other.is_ordered() || ordered),
+seed_hash_(other.get_seed_hash()),
+theta_(other.get_theta64()),
+entries_()
+{
+  entries_.reserve(other.get_num_retained());
+  std::copy(other.begin(), other.end(), std::back_inserter(entries_));
+  if (ordered && !other.is_ordered()) std::sort(entries_.begin(), entries_.end());
+}
+
+template<typename S, typename SD, typename A>
+bool compact_tuple_sketch<S, SD, A>::is_empty() const {
+  return is_empty_;
+}
+
+template<typename S, typename SD, typename A>
+bool compact_tuple_sketch<S, SD, A>::is_ordered() const {
+  return is_ordered_;
+}
+
+template<typename S, typename SD, typename A>
+uint64_t compact_tuple_sketch<S, SD, A>::get_theta64() const {
+  return theta_;
+}
+
+template<typename S, typename SD, typename A>
+uint32_t compact_tuple_sketch<S, SD, A>::get_num_retained() const {
+  return entries_.size();
+}
+
+template<typename S, typename SD, typename A>
+uint16_t compact_tuple_sketch<S, SD, A>::get_seed_hash() const {
+  return seed_hash_;
+}
+
+template<typename S, typename SD, typename A>
+string<A> compact_tuple_sketch<S, SD, A>::to_string(bool detail) const {
+  std::basic_ostringstream<char, std::char_traits<char>, AllocChar<A>> os;
+  auto type = typeid(*this).name();
+  os << "sizeof(" << type << ")=" << sizeof(*this) << std::endl;
+  os << "sizeof(entry)=" << sizeof(Entry) << std::endl;
+  os << "empty? : " << (is_empty_ ? "true" : "false") << std::endl;
+  if (detail) {
+    for (const auto& it: entries_) {
+      if (it.first != 0) {
+        os << it.first << ": " << it.second << std::endl;
+      }
+    }
+  }
+  return os.str();
+}
+
+// implementation for fixed-size arithmetic types (integral and floating point)
+template<typename S, typename SD, typename A>
+template<typename SS, typename std::enable_if<std::is_arithmetic<SS>::value, int>::type>
+size_t compact_tuple_sketch<S, SD, A>::get_serialized_size_summaries_bytes() const {
+  return entries_.size() * sizeof(SS);
+}
+
+// implementation for all other types (non-arithmetic)
+template<typename S, typename SD, typename A>
+template<typename SS, typename std::enable_if<!std::is_arithmetic<SS>::value, int>::type>
+size_t compact_tuple_sketch<S, SD, A>::get_serialized_size_summaries_bytes() const {
+  size_t size = 0;
+  for (const auto& it: entries_) {
+    size += SD().size_of_item(it.second);
+  }
+  return size;
+}
+
+template<typename S, typename SD, typename A>
+void compact_tuple_sketch<S, SD, A>::serialize(std::ostream& os) const {
+}
+
+template<typename S, typename SD, typename A>
+auto compact_tuple_sketch<S, SD, A>::serialize(unsigned header_size_bytes) const -> vector_bytes {
+  const bool is_single_item = entries_.size() == 1 && !this->is_estimation_mode();
+  const uint8_t preamble_longs = this->is_empty() || is_single_item ? 1 : this->is_estimation_mode() ? 3 : 2;
+  const size_t size = header_size_bytes + sizeof(uint64_t) * preamble_longs
+      + sizeof(uint64_t) * entries_.size() + get_serialized_size_summaries_bytes();
+  vector_bytes bytes(size);
+  uint8_t* ptr = bytes.data() + header_size_bytes;
+  const uint8_t* end_ptr = ptr + size;
+
+  ptr += copy_to_mem(&preamble_longs, ptr, sizeof(preamble_longs));
+  const uint8_t serial_version = Base::SERIAL_VERSION;
+  ptr += copy_to_mem(&serial_version, ptr, sizeof(serial_version));
+  const uint8_t type = SKETCH_TYPE;
+  ptr += copy_to_mem(&type, ptr, sizeof(type));
+  const uint16_t unused16 = 0;
+  ptr += copy_to_mem(&unused16, ptr, sizeof(unused16));
+  const uint8_t flags_byte(
+    (1 << Base::flags::IS_COMPACT) |
+    (1 << Base::flags::IS_READ_ONLY) |
+    (this->is_empty() ? 1 << Base::flags::IS_EMPTY : 0) |
+    (this->is_ordered() ? 1 << Base::flags::IS_ORDERED : 0)
+  );
+  ptr += copy_to_mem(&flags_byte, ptr, sizeof(flags_byte));
+  const uint16_t seed_hash = get_seed_hash();
+  ptr += copy_to_mem(&seed_hash, ptr, sizeof(seed_hash));
+  if (!this->is_empty()) {
+    if (!is_single_item) {
+      const uint32_t num_entries = entries_.size();
+      ptr += copy_to_mem(&num_entries, ptr, sizeof(num_entries));
+      const uint32_t unused32 = 0;
+      ptr += copy_to_mem(&unused32, ptr, sizeof(unused32));
+      if (this->is_estimation_mode()) {
+        ptr += copy_to_mem(&theta_, ptr, sizeof(uint64_t));
+      }
+    }
+    for (const auto& it: entries_) {
+      ptr += copy_to_mem(&it.first, ptr, sizeof(uint64_t));
+    }
+    for (const auto& it: entries_) {
+      ptr += SD().serialize(ptr, end_ptr - ptr, &it.second, 1);
+    }
+  }
+  return bytes;
+}
+
+template<typename S, typename SD, typename A>
+auto compact_tuple_sketch<S, SD, A>::begin() const -> const_iterator {
+  return const_iterator(entries_.data(), entries_.size(), 0);
+}
+
+template<typename S, typename SD, typename A>
+auto compact_tuple_sketch<S, SD, A>::end() const -> const_iterator {
+  return const_iterator(nullptr, 0, entries_.size());
+}
+
+// builder
+
+template<typename S, typename U, typename P, typename SD, typename A>
+update_tuple_sketch<S, U, P, SD, A>::builder::builder(const P& policy):
+theta_base_builder<true>(), policy_(policy) {}
+
+template<typename S, typename U, typename P, typename SD, typename A>
+update_tuple_sketch<S, U, P, SD, A> update_tuple_sketch<S, U, P, SD, A>::builder::build() const {
+  return update_tuple_sketch<S, U, P, SD, A>(starting_sub_multiple(lg_k_ + 1, MIN_LG_K, static_cast<uint8_t>(rf_)), lg_k_, rf_, p_, seed_, policy_);
+}
+
+} /* namespace datasketches */
diff --git a/tuple/include/tuple_union.hpp b/tuple/include/tuple_union.hpp
new file mode 100644
index 0000000..a23cade
--- /dev/null
+++ b/tuple/include/tuple_union.hpp
@@ -0,0 +1,109 @@
+/*
+ * 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 TUPLE_UNION_HPP_
+#define TUPLE_UNION_HPP_
+
+#include "serde.hpp"
+#include "tuple_sketch.hpp"
+#include "theta_union_base.hpp"
+#include "theta_sketch_experimental.hpp"
+
+namespace datasketches {
+
+// for types with defined + operation
+template<typename Summary>
+struct default_union_policy {
+  void operator()(Summary& summary, const Summary& other) const {
+    summary += other;
+  }
+};
+
+template<
+  typename Summary,
+  typename Policy = default_union_policy<Summary>,
+  typename SerDe = serde<Summary>,
+  typename Allocator = std::allocator<Summary>
+>
+class tuple_union {
+public:
+  using Entry = std::pair<uint64_t, Summary>;
+  using AllocEntry = typename std::allocator_traits<Allocator>::template rebind_alloc<Entry>;
+  using ExtractKey = pair_extract_key<uint64_t, Summary>;
+  using Sketch = tuple_sketch<Summary, SerDe, Allocator>;
+  using CompactSketch = compact_tuple_sketch<Summary, SerDe, Allocator>;
+  using resize_factor = theta_constants::resize_factor;
+
+  struct internal_policy {
+    internal_policy(const Policy& policy): policy_(policy) {}
+    Entry& operator()(Entry& internal_entry, const Entry& incoming_entry) const {
+      policy_(internal_entry.second, incoming_entry.second);
+      return internal_entry;
+    }
+    Policy policy_;
+  };
+
+  using State = theta_union_base<Entry, ExtractKey, internal_policy, Sketch, CompactSketch, AllocEntry>;
+
+  // No constructor here. Use builder instead.
+  class builder;
+
+  /**
+   * This method is to update the union with a given sketch
+   * @param sketch to update the union with
+   */
+  void update(const Sketch& sketch);
+
+  /**
+   * This method produces a copy of the current state of the union as a compact sketch.
+   * @param ordered optional flag to specify if ordered sketch should be produced
+   * @return the result of the union
+   */
+  CompactSketch get_result(bool ordered = true) const;
+
+private:
+  State state_;
+
+  // for builder
+  tuple_union(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed, const Policy& policy);
+};
+
+template<typename S, typename P, typename SD, typename A>
+class tuple_union<S, P, SD, A>::builder: public theta_base_builder<true> {
+public:
+  /**
+   * Creates and instance of the builder with default parameters.
+   */
+  builder(const P& policy = P());
+
+  /**
+   * This is to create an instance of the union with predefined parameters.
+   * @return an instance of the union
+   */
+  tuple_union<S, P, SD, A> build() const;
+
+private:
+  P policy_;
+};
+
+} /* namespace datasketches */
+
+#include "tuple_union_impl.hpp"
+
+#endif
diff --git a/tuple/include/tuple_union_impl.hpp b/tuple/include/tuple_union_impl.hpp
new file mode 100644
index 0000000..8cfca52
--- /dev/null
+++ b/tuple/include/tuple_union_impl.hpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+namespace datasketches {
+
+template<typename S, typename P, typename SD, typename A>
+tuple_union<S, P, SD, A>::tuple_union(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, float p, uint64_t seed, const P& policy):
+state_(lg_cur_size, lg_nom_size, rf, p, seed, policy)
+{}
+
+template<typename S, typename P, typename SD, typename A>
+void tuple_union<S, P, SD, A>::update(const Sketch& sketch) {
+  state_.update(sketch);
+}
+
+template<typename S, typename P, typename SD, typename A>
+auto tuple_union<S, P, SD, A>::get_result(bool ordered) const -> CompactSketch {
+  return state_.get_result(ordered);
+}
+
+template<typename S, typename P, typename SD, typename A>
+tuple_union<S, P, SD, A>::builder::builder(const P& policy):
+policy_(policy) {}
+
+template<typename S, typename P, typename SD, typename A>
+auto tuple_union<S, P, SD, A>::builder::build() const -> tuple_union {
+  return tuple_union(starting_sub_multiple(lg_k_ + 1, MIN_LG_K, static_cast<uint8_t>(rf_)), lg_k_, rf_, p_, seed_, policy_);
+}
+
+} /* namespace datasketches */
diff --git a/tuple/test/CMakeLists.txt b/tuple/test/CMakeLists.txt
new file mode 100644
index 0000000..29f9c75
--- /dev/null
+++ b/tuple/test/CMakeLists.txt
@@ -0,0 +1,45 @@
+# 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.
+
+add_executable(tuple_test)
+
+target_link_libraries(tuple_test tuple common_test)
+
+set_target_properties(tuple_test PROPERTIES
+  CXX_STANDARD 11
+  CXX_STANDARD_REQUIRED YES
+)
+
+file(TO_CMAKE_PATH "${CMAKE_CURRENT_SOURCE_DIR}" THETA_TEST_BINARY_PATH)
+string(APPEND THETA_TEST_BINARY_PATH "/")
+target_compile_definitions(tuple_test
+  PRIVATE
+    TEST_BINARY_INPUT_PATH="${THETA_TEST_BINARY_PATH}"
+)
+
+add_test(
+  NAME tuple_test
+  COMMAND tuple_test
+)
+
+target_sources(tuple_test
+  PRIVATE
+    tuple_sketch_test.cpp
+    tuple_union_test.cpp
+    theta_sketch_experimental_test.cpp
+    theta_union_experimental_test.cpp
+)
diff --git a/tuple/test/theta_sketch_experimental_test.cpp b/tuple/test/theta_sketch_experimental_test.cpp
new file mode 100644
index 0000000..5407689
--- /dev/null
+++ b/tuple/test/theta_sketch_experimental_test.cpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+#include <iostream>
+
+#include <catch.hpp>
+#include <theta_sketch_experimental.hpp>
+#include <../../theta/include/theta_sketch.hpp>
+
+namespace datasketches {
+
+TEST_CASE("theta_sketch_experimental: basics ", "[theta_sketch]") {
+  auto update_sketch = theta_sketch_experimental<>::builder().build();
+  update_sketch.update(1);
+  update_sketch.update(2);
+  REQUIRE(update_sketch.get_num_retained() == 2);
+  int count = 0;
+  for (const auto& entry: update_sketch) ++count;
+  REQUIRE(count == 2);
+
+  auto compact_sketch = update_sketch.compact();
+  REQUIRE(compact_sketch.get_num_retained() == 2);
+}
+
+//TEST_CASE("theta_sketch_experimental: compare with theta production", "[theta_sketch]") {
+//  auto test = theta_sketch_experimental<>::builder().build();
+//  update_theta_sketch prod = update_theta_sketch::builder().build();
+//
+//  for (int i = 0; i < 1000000; ++i) {
+//    test.update(i);
+//    prod.update(i);
+//  }
+//
+//  std::cout << "--- theta production vs experimental ---" << std::endl;
+//  std::cout << test.to_string(true);
+//  std::cout << "sizeof(update_theta_sketch)=" << sizeof(update_theta_sketch) << std::endl;
+//  std::cout << prod.to_string(true);
+//}
+
+} /* namespace datasketches */
diff --git a/tuple/test/theta_union_experimental_test.cpp b/tuple/test/theta_union_experimental_test.cpp
new file mode 100644
index 0000000..cbe1412
--- /dev/null
+++ b/tuple/test/theta_union_experimental_test.cpp
@@ -0,0 +1,47 @@
+/*
+ * 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.
+ */
+
+#include <iostream>
+
+#include <catch.hpp>
+#include <tuple_union.hpp>
+
+#include <theta_union_experimental.hpp>
+
+namespace datasketches {
+
+TEST_CASE("theta_union_exeperimental") {
+  std::cout << "theta union test begin" << std::endl;
+  auto update_sketch1 = theta_sketch_experimental<>::builder().build();
+  update_sketch1.update(1);
+  update_sketch1.update(2);
+
+  auto update_sketch2 = theta_sketch_experimental<>::builder().build();
+  update_sketch2.update(1);
+  update_sketch2.update(3);
+
+  auto u = theta_union_experimental<>::builder().build();
+  u.update(update_sketch1);
+  u.update(update_sketch2);
+  auto r = u.get_result();
+  std::cout << r.to_string(true);
+  std::cout << "theta union test end" << std::endl;
+}
+
+} /* namespace datasketches */
diff --git a/tuple/test/tuple_sketch_test.cpp b/tuple/test/tuple_sketch_test.cpp
new file mode 100644
index 0000000..730cbb5
--- /dev/null
+++ b/tuple/test/tuple_sketch_test.cpp
@@ -0,0 +1,189 @@
+/*
+ * 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.
+ */
+
+#include <iostream>
+#include <tuple>
+
+// this is needed for a test below, but should be defined here
+std::ostream& operator<<(std::ostream& os, const std::tuple<double, double, double>& tuple) {
+  os << std::get<0>(tuple) << ", " << std::get<1>(tuple) << ", " << std::get<2>(tuple);
+  return os;
+}
+
+#include <catch.hpp>
+#include <tuple_sketch.hpp>
+#include <test_type.hpp>
+
+namespace datasketches {
+
+TEST_CASE("tuple sketch float: builder", "[tuple_sketch]") {
+  auto builder = update_tuple_sketch<float>::builder();
+  builder.set_lg_k(10).set_p(0.5).set_resize_factor(theta_constants::resize_factor::X2).set_seed(123);
+  auto sketch = builder.build();
+  REQUIRE(sketch.get_lg_k() == 10);
+  REQUIRE(sketch.get_theta() == 0.5);
+  REQUIRE(sketch.get_rf() == theta_constants::resize_factor::X2);
+  REQUIRE(sketch.get_seed_hash() == compute_seed_hash(123));
+}
+
+TEST_CASE("tuple sketch float: empty", "[tuple_sketch]") {
+  auto update_sketch = update_tuple_sketch<float>::builder().build();
+  REQUIRE(update_sketch.is_empty());
+  REQUIRE(!update_sketch.is_estimation_mode());
+  REQUIRE(update_sketch.get_estimate() == 0);
+  REQUIRE(update_sketch.get_lower_bound(1) == 0);
+  REQUIRE(update_sketch.get_upper_bound(1) == 0);
+  REQUIRE(update_sketch.get_theta() == 1);
+  REQUIRE(update_sketch.get_num_retained() == 0);
+  REQUIRE(!update_sketch.is_ordered());
+
+  auto compact_sketch = update_sketch.compact();
+  REQUIRE(compact_sketch.is_empty());
+  REQUIRE(!compact_sketch.is_estimation_mode());
+  REQUIRE(compact_sketch.get_estimate() == 0);
+  REQUIRE(compact_sketch.get_lower_bound(1) == 0);
+  REQUIRE(compact_sketch.get_upper_bound(1) == 0);
+  REQUIRE(compact_sketch.get_theta() == 1);
+  REQUIRE(compact_sketch.get_num_retained() == 0);
+  REQUIRE(compact_sketch.is_ordered());
+}
+
+TEST_CASE("tuple sketch float: exact mode", "[tuple_sketch]") {
+  auto update_sketch = update_tuple_sketch<float>::builder().build();
+  update_sketch.update(1, 1);
+  update_sketch.update(2, 2);
+  update_sketch.update(1, 1);
+  std::cout << update_sketch.to_string(true);
+  REQUIRE(!update_sketch.is_empty());
+  REQUIRE(!update_sketch.is_estimation_mode());
+  REQUIRE(update_sketch.get_estimate() == 2);
+  REQUIRE(update_sketch.get_lower_bound(1) == 2);
+  REQUIRE(update_sketch.get_upper_bound(1) == 2);
+  REQUIRE(update_sketch.get_theta() == 1);
+  REQUIRE(update_sketch.get_num_retained() == 2);
+  REQUIRE(!update_sketch.is_ordered());
+  int count = 0;
+  for (const auto& entry: update_sketch) {
+    REQUIRE(entry.second == 2);
+    ++count;
+  }
+  REQUIRE(count == 2);
+
+  auto compact_sketch = update_sketch.compact();
+  std::cout << compact_sketch.to_string(true);
+  REQUIRE(!compact_sketch.is_empty());
+  REQUIRE(!compact_sketch.is_estimation_mode());
+  REQUIRE(compact_sketch.get_estimate() == 2);
+  REQUIRE(compact_sketch.get_lower_bound(1) == 2);
+  REQUIRE(compact_sketch.get_upper_bound(1) == 2);
+  REQUIRE(compact_sketch.get_theta() == 1);
+  REQUIRE(compact_sketch.get_num_retained() == 2);
+  REQUIRE(compact_sketch.is_ordered());
+  count = 0;
+  for (const auto& entry: compact_sketch) {
+    REQUIRE(entry.second == 2);
+    ++count;
+  }
+  REQUIRE(count == 2);
+}
+
+template<typename T>
+class max_value_policy {
+public:
+  max_value_policy(const T& initial_value): initial_value(initial_value) {}
+  T create() const { return initial_value; }
+  void update(T& summary, const T& update) const { summary = std::max(summary, update); }
+private:
+  T initial_value;
+};
+
+typedef update_tuple_sketch<float, float, max_value_policy<float>> max_float_update_tuple_sketch;
+
+TEST_CASE("tuple sketch: float, custom policy", "[tuple_sketch]") {
+  auto update_sketch = max_float_update_tuple_sketch::builder(max_value_policy<float>(5)).build();
+  update_sketch.update(1, 1);
+  update_sketch.update(1, 2);
+  update_sketch.update(2, 10);
+  update_sketch.update(3, 3);
+  update_sketch.update(3, 7);
+  std::cout << update_sketch.to_string(true);
+  int count = 0;
+  float sum = 0;
+  for (const auto& entry: update_sketch) {
+    sum += entry.second;
+    ++count;
+  }
+  REQUIRE(count == 3);
+  REQUIRE(sum == 22); // 5 + 10 + 7
+}
+
+struct test_type_replace_policy {
+  test_type create() const { return test_type(0); }
+  void update(test_type& summary, const test_type& update) const {
+    //std::cerr << "policy::update lvalue begin" << std::endl;
+    summary = update;
+    //std::cerr << "policy::update lvalue end" << std::endl;
+  }
+  void update(test_type& summary, test_type&& update) const {
+    //std::cerr << "policy::update rvalue begin" << std::endl;
+    summary = std::move(update);
+    //std::cerr << "policy::update rvalue end" << std::endl;
+  }
+};
+
+TEST_CASE("tuple sketch: test type with replace policy", "[tuple_sketch]") {
+  auto sketch = update_tuple_sketch<test_type, test_type, test_type_replace_policy, test_type_serde>::builder().build();
+  test_type a(1);
+  sketch.update(1, a); // this should copy
+  sketch.update(2, 2); // this should move
+  sketch.update(1, 2); // this should move
+  std::cout << sketch.to_string(true);
+  REQUIRE(sketch.get_num_retained() == 2);
+  for (const auto& entry: sketch) {
+    REQUIRE(entry.second.get_value() == 2);
+  }
+}
+
+struct three_doubles_update_policy {
+  std::tuple<double, double, double> create() const {
+    return std::tuple<double, double, double>(0, 0, 0);
+  }
+  void update(std::tuple<double, double, double>& summary, const std::tuple<double, double, double>& update) const {
+    std::get<0>(summary) += std::get<0>(update);
+    std::get<1>(summary) += std::get<1>(update);
+    std::get<2>(summary) += std::get<2>(update);
+  }
+};
+
+TEST_CASE("tuple sketch: array of doubles", "[tuple_sketch]") {
+  using three_doubles = std::tuple<double, double, double>;
+  using three_doubles_update_tuple_sketch = update_tuple_sketch<three_doubles, three_doubles, three_doubles_update_policy>;
+  auto update_sketch = three_doubles_update_tuple_sketch::builder().build();
+  update_sketch.update(1, three_doubles(1, 2, 3));
+  std::cout << update_sketch.to_string(true);
+  const auto& entry = *update_sketch.begin();
+  REQUIRE(std::get<0>(entry.second) == 1.0);
+  REQUIRE(std::get<1>(entry.second) == 2.0);
+  REQUIRE(std::get<2>(entry.second) == 3.0);
+
+  auto compact_sketch = update_sketch.compact();
+  std::cout << compact_sketch.to_string(true);
+}
+
+} /* namespace datasketches */
diff --git a/tuple/test/tuple_union_test.cpp b/tuple/test/tuple_union_test.cpp
new file mode 100644
index 0000000..437aacd
--- /dev/null
+++ b/tuple/test/tuple_union_test.cpp
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+
+#include <iostream>
+
+#include <catch.hpp>
+#include <tuple_union.hpp>
+
+namespace datasketches {
+
+TEST_CASE("tuple_union float") {
+  std::cout << "union test begin" << std::endl;
+  auto update_sketch1 = update_tuple_sketch<float>::builder().build();
+  update_sketch1.update(1, 1);
+  update_sketch1.update(2, 1);
+
+  auto update_sketch2 = update_tuple_sketch<float>::builder().build();
+  update_sketch2.update(1, 1);
+  update_sketch2.update(3, 1);
+
+  auto u = tuple_union<float>::builder().build();
+  u.update(update_sketch1);
+  u.update(update_sketch2);
+  auto r = u.get_result();
+  std::cout << r.to_string(true);
+  std::cout << "union test end" << std::endl;
+}
+
+} /* namespace datasketches */


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