You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@datasketches.apache.org by GitBox <gi...@apache.org> on 2020/08/25 22:36:17 UTC

[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov opened a new pull request #170: Tuple sketch

AlexanderSaydakov opened a new pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170


   This is an implementation of generic tuple sketch, specialized array of doubles sketch (with serialization compatible with Java equivalent) and experimental theta sketch on top of the same base layer (it should eventually replace the existing theta sketch). Perhaps some reorganization might help to separate this large set of files into several groups (for instance, moving the common base to a separate directory).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r483852356



##########
File path: tuple/test/tuple_sketch_allocation_test.cpp
##########
@@ -0,0 +1,100 @@
+/*
+ * 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_sketch.hpp>
+#include <test_allocator.hpp>
+#include <test_type.hpp>
+
+namespace datasketches {
+
+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;
+  }
+};
+
+using update_tuple_sketch_test =
+    update_tuple_sketch<test_type, test_type, test_type_replace_policy, test_allocator<test_type>>;
+using compact_tuple_sketch_test =
+    compact_tuple_sketch<test_type, test_allocator<test_type>>;
+
+TEST_CASE("tuple sketch with test allocator: estimation mode", "[tuple_sketch]") {
+  test_allocator_total_bytes = 0;
+  test_allocator_net_allocations = 0;
+  {
+    auto update_sketch = update_tuple_sketch_test::builder().build();
+    for (int i = 0; i < 10000; ++i) update_sketch.update(i, 1);
+    for (int i = 0; i < 10000; ++i) update_sketch.update(i, 2);
+    REQUIRE(!update_sketch.is_empty());
+    REQUIRE(update_sketch.is_estimation_mode());
+    unsigned count = 0;
+    for (const auto& entry: update_sketch) {
+      REQUIRE(entry.second.get_value() == 2);
+      ++count;
+    }
+    REQUIRE(count == update_sketch.get_num_retained());
+
+    update_sketch.trim();
+    REQUIRE(update_sketch.get_num_retained() == (1 << update_sketch.get_lg_k()));
+
+    auto compact_sketch = update_sketch.compact();
+    REQUIRE(!compact_sketch.is_empty());
+    REQUIRE(compact_sketch.is_estimation_mode());
+    count = 0;
+    for (const auto& entry: compact_sketch) {
+      REQUIRE(entry.second.get_value() == 2);
+      ++count;
+    }
+    REQUIRE(count == update_sketch.get_num_retained());
+
+    auto bytes = compact_sketch.serialize(0, test_type_serde());
+    auto deserialized_sketch = compact_tuple_sketch_test::deserialize(bytes.data(), bytes.size(), DEFAULT_SEED, test_type_serde());
+    REQUIRE(deserialized_sketch.get_estimate() == compact_sketch.get_estimate());
+
+    // update sketch copy
+//    std::cout << update_sketch.to_string();

Review comment:
       We can clean this up, but I don't think it is important in tests. I see your point, and I prefer not to keep commented-out lines in the main code




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r501936795



##########
File path: tuple/include/theta_update_sketch_base_impl.hpp
##########
@@ -0,0 +1,384 @@
+/*
+ * 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, uint64_t theta, uint64_t seed, const A& allocator, bool is_empty):
+allocator_(allocator),
+is_empty_(is_empty),
+lg_cur_size_(lg_cur_size),
+lg_nom_size_(lg_nom_size),
+rf_(rf),
+num_entries_(0),
+theta_(theta),
+seed_(seed),
+entries_(nullptr)
+{
+  if (lg_cur_size > 0) {
+    const size_t size = 1 << lg_cur_size;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(const theta_update_sketch_base& other):
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(nullptr)
+{
+  if (other.entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(other.entries_[i]) != 0) {
+        new (&entries_[i]) EN(other.entries_[i]);
+      } else {
+        EK()(entries_[i]) = 0;
+      }
+    }
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(theta_update_sketch_base&& other) noexcept:
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(other.entries_)
+{
+  other.entries_ = nullptr;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::~theta_update_sketch_base()
+{
+  if (entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(entries_[i]) != 0) entries_[i].~EN();
+    }
+    allocator_.deallocate(entries_, size);
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(const theta_update_sketch_base& other) {
+  theta_update_sketch_base<EN, EK, A> copy(other);
+  std::swap(allocator_, copy.allocator_);
+  std::swap(is_empty_, copy.is_empty_);
+  std::swap(lg_cur_size_, copy.lg_cur_size_);
+  std::swap(lg_nom_size_, copy.lg_nom_size_);
+  std::swap(rf_, copy.rf_);
+  std::swap(num_entries_, copy.num_entries_);
+  std::swap(theta_, copy.theta_);
+  std::swap(seed_, copy.seed_);
+  std::swap(entries_, copy.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(theta_update_sketch_base&& other) {
+  std::swap(allocator_, other.allocator_);
+  std::swap(is_empty_, other.is_empty_);
+  std::swap(lg_cur_size_, other.lg_cur_size_);
+  std::swap(lg_nom_size_, other.lg_nom_size_);
+  std::swap(rf_, other.rf_);
+  std::swap(num_entries_, other.num_entries_);
+  std::swap(theta_, other.theta_);
+  std::swap(seed_, other.seed_);
+  std::swap(entries_, other.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+uint64_t theta_update_sketch_base<EN, EK, A>::hash_and_screen(const void* data, size_t length) {
+  is_empty_ = false;
+  const uint64_t hash = compute_hash(data, length, seed_);
+  if (hash >= theta_) return 0; // hash == 0 is reserved to mark empty slots in the table
+  return hash;
+}
+
+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));
+  ++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>
+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_;
+  EN* old_entries = entries_;
+  entries_ = allocator_.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();
+    }
+  }
+  allocator_.deallocate(old_entries, old_size);
+}
+
+// assumes number of entries > nominal size
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::rebuild() {
+  const size_t size = 1 << lg_cur_size_;
+  const uint32_t nominal_size = 1 << lg_nom_size_;
+
+  // empty entries have uninitialized payloads
+  // TODO: avoid this for empty or trivial payloads (arithmetic types)
+  consolidate_non_empty(entries_, size, num_entries_);
+
+  std::nth_element(entries_, entries_ + nominal_size, entries_ + num_entries_, comparator());
+  this->theta_ = EK()(entries_[nominal_size]);
+  EN* old_entries = entries_;
+  entries_ = allocator_.allocate(size);
+  for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  num_entries_ = 0;
+  // relies on consolidating non-empty entries to the front
+  for (size_t i = 0; i < nominal_size; ++i) {
+    insert(find(EK()(old_entries[i])).first, std::move(old_entries[i])); // consider a special insert with no comparison
+    old_entries[i].~EN();
+  }
+  allocator_.deallocate(old_entries, size);
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::trim() {
+  if (num_entries_ > static_cast<uint32_t>(1 << lg_nom_size_)) rebuild();
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::consolidate_non_empty(EN* entries, size_t size, size_t num) {
+  // find the first empty slot
+  size_t i = 0;
+  while (i < size) {
+    if (EK()(entries[i]) == 0) break;
+    ++i;
+  }
+  // scan the rest and move non-empty entries to the front
+  for (size_t j = i + 1; j < size; ++j) {
+    if (EK()(entries[j]) != 0) {
+      new (&entries[i]) EN(std::move(entries[j]));
+      entries[j].~EN();
+      EK()(entries[j]) = 0;
+      ++i;
+      if (i == num) break;
+    }
+  }
+}
+
+// builder
+
+template<typename Derived, typename Allocator>
+theta_base_builder<Derived, Allocator>::theta_base_builder(const Allocator& allocator):
+allocator_(allocator), lg_k_(DEFAULT_LG_K), rf_(DEFAULT_RESIZE_FACTOR), p_(1), seed_(DEFAULT_SEED) {}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_lg_k(uint8_t lg_k) {
+  if (lg_k < MIN_LG_K) {

Review comment:
       I think so. I will add that




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] jmalkin commented on pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
jmalkin commented on pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#issuecomment-705931667


   The last few edits look good. Merge away!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r483798767



##########
File path: tuple/include/array_of_doubles_intersection.hpp
##########
@@ -0,0 +1,52 @@
+/*
+ * 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 ARRAY_OF_DOUBLES_INTERSECTION_HPP_
+#define ARRAY_OF_DOUBLES_INTERSECTION_HPP_
+
+#include <vector>
+#include <memory>
+
+#include "array_of_doubles_sketch.hpp"
+#include "tuple_intersection.hpp"
+
+namespace datasketches {
+
+template<
+  typename Policy,

Review comment:
       We try to be consistent if it makes sense. I think in this case the implementation is substantially different. A corresponding thing in Java ArrayOfDoublesIntersection is called Combiner. And there is no way currently to override similar combining in sketch update and in union. Here in C++ I decided to call this concept a Policy, and it can be user-defined for update, union and intersection. It defaults to sum in update and union, but there is no default for the intersection since it is not quite clear what to do with values for the same keys. The naming is debatable, of course.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] davecromberge commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r485529205



##########
File path: tuple/include/theta_set_difference_base_impl.hpp
##########
@@ -0,0 +1,80 @@
+/*
+ * 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 <algorithm>
+
+#include "conditional_back_inserter.hpp"
+#include "conditional_forward.hpp"
+
+namespace datasketches {
+
+template<typename EN, typename EK, typename CS, typename A>
+theta_set_difference_base<EN, EK, CS, A>::theta_set_difference_base(uint64_t seed, const A& allocator):
+allocator_(allocator),
+seed_hash_(compute_seed_hash(seed))
+{}
+
+template<typename EN, typename EK, typename CS, typename A>
+template<typename FwdSketch, typename Sketch>
+CS theta_set_difference_base<EN, EK, CS, A>::compute(FwdSketch&& a, const Sketch& b, bool ordered) const {
+  if (a.is_empty() || a.get_num_retained() == 0 || b.is_empty()) return CS(a, ordered);
+  if (a.get_seed_hash() != seed_hash_) throw std::invalid_argument("A seed hash mismatch");
+  if (b.get_seed_hash() != seed_hash_) throw std::invalid_argument("B seed hash mismatch");
+
+  const uint64_t theta = std::min(a.get_theta64(), b.get_theta64());
+  std::vector<EN, A> entries(allocator_);
+  bool is_empty = a.is_empty();
+
+  if (b.get_num_retained() == 0) {
+    std::copy_if(forward_begin(std::forward<FwdSketch>(a)), forward_end(std::forward<FwdSketch>(a)), std::back_inserter(entries),
+        key_less_than<uint64_t, EN, EK>(theta));
+  } else {
+    if (a.is_ordered() && b.is_ordered()) { // sort-based
+      std::set_difference(forward_begin(std::forward<FwdSketch>(a)), forward_end(std::forward<FwdSketch>(a)), b.begin(), b.end(),
+          conditional_back_inserter(entries, key_less_than<uint64_t, EN, EK>(theta)), comparator());
+    } else { // hash-based
+      const uint8_t lg_size = lg_size_from_count(b.get_num_retained(), hash_table::REBUILD_THRESHOLD);
+      hash_table table(lg_size, lg_size, hash_table::resize_factor::X1, 0, 0, allocator_); // theta and seed are not used here
+      for (const auto& entry: b) {
+        const uint64_t hash = EK()(entry);
+        if (hash < theta) {
+          table.insert(table.find(hash).first, hash);
+        } else if (b.is_ordered()) {
+          break; // early stop
+        }
+      }
+
+      // scan A lookup B

Review comment:
       You are correct, a different algorithm came to mind when I saw the comment on line 63 without me sufficiently appreciating the context.  I recall the java algorithm for intersections having an optimisation about scan order in the data tables, which is not applicable here.
   I agree with you about avoiding the special treatment for the update sketch.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov merged pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov merged pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r501937411



##########
File path: tuple/include/theta_update_sketch_base_impl.hpp
##########
@@ -0,0 +1,384 @@
+/*
+ * 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, uint64_t theta, uint64_t seed, const A& allocator, bool is_empty):
+allocator_(allocator),
+is_empty_(is_empty),
+lg_cur_size_(lg_cur_size),
+lg_nom_size_(lg_nom_size),
+rf_(rf),
+num_entries_(0),
+theta_(theta),
+seed_(seed),
+entries_(nullptr)
+{
+  if (lg_cur_size > 0) {
+    const size_t size = 1 << lg_cur_size;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(const theta_update_sketch_base& other):
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(nullptr)
+{
+  if (other.entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(other.entries_[i]) != 0) {
+        new (&entries_[i]) EN(other.entries_[i]);
+      } else {
+        EK()(entries_[i]) = 0;
+      }
+    }
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(theta_update_sketch_base&& other) noexcept:
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(other.entries_)
+{
+  other.entries_ = nullptr;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::~theta_update_sketch_base()
+{
+  if (entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(entries_[i]) != 0) entries_[i].~EN();
+    }
+    allocator_.deallocate(entries_, size);
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(const theta_update_sketch_base& other) {
+  theta_update_sketch_base<EN, EK, A> copy(other);
+  std::swap(allocator_, copy.allocator_);
+  std::swap(is_empty_, copy.is_empty_);
+  std::swap(lg_cur_size_, copy.lg_cur_size_);
+  std::swap(lg_nom_size_, copy.lg_nom_size_);
+  std::swap(rf_, copy.rf_);
+  std::swap(num_entries_, copy.num_entries_);
+  std::swap(theta_, copy.theta_);
+  std::swap(seed_, copy.seed_);
+  std::swap(entries_, copy.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(theta_update_sketch_base&& other) {
+  std::swap(allocator_, other.allocator_);
+  std::swap(is_empty_, other.is_empty_);
+  std::swap(lg_cur_size_, other.lg_cur_size_);
+  std::swap(lg_nom_size_, other.lg_nom_size_);
+  std::swap(rf_, other.rf_);
+  std::swap(num_entries_, other.num_entries_);
+  std::swap(theta_, other.theta_);
+  std::swap(seed_, other.seed_);
+  std::swap(entries_, other.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+uint64_t theta_update_sketch_base<EN, EK, A>::hash_and_screen(const void* data, size_t length) {
+  is_empty_ = false;
+  const uint64_t hash = compute_hash(data, length, seed_);
+  if (hash >= theta_) return 0; // hash == 0 is reserved to mark empty slots in the table
+  return hash;
+}
+
+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));
+  ++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>
+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_;
+  EN* old_entries = entries_;
+  entries_ = allocator_.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();
+    }
+  }
+  allocator_.deallocate(old_entries, old_size);
+}
+
+// assumes number of entries > nominal size
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::rebuild() {
+  const size_t size = 1 << lg_cur_size_;
+  const uint32_t nominal_size = 1 << lg_nom_size_;
+
+  // empty entries have uninitialized payloads
+  // TODO: avoid this for empty or trivial payloads (arithmetic types)
+  consolidate_non_empty(entries_, size, num_entries_);
+
+  std::nth_element(entries_, entries_ + nominal_size, entries_ + num_entries_, comparator());
+  this->theta_ = EK()(entries_[nominal_size]);
+  EN* old_entries = entries_;
+  entries_ = allocator_.allocate(size);
+  for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  num_entries_ = 0;
+  // relies on consolidating non-empty entries to the front
+  for (size_t i = 0; i < nominal_size; ++i) {
+    insert(find(EK()(old_entries[i])).first, std::move(old_entries[i])); // consider a special insert with no comparison
+    old_entries[i].~EN();
+  }
+  allocator_.deallocate(old_entries, size);
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::trim() {
+  if (num_entries_ > static_cast<uint32_t>(1 << lg_nom_size_)) rebuild();
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::consolidate_non_empty(EN* entries, size_t size, size_t num) {
+  // find the first empty slot
+  size_t i = 0;
+  while (i < size) {
+    if (EK()(entries[i]) == 0) break;
+    ++i;
+  }
+  // scan the rest and move non-empty entries to the front
+  for (size_t j = i + 1; j < size; ++j) {
+    if (EK()(entries[j]) != 0) {
+      new (&entries[i]) EN(std::move(entries[j]));
+      entries[j].~EN();
+      EK()(entries[j]) = 0;
+      ++i;
+      if (i == num) break;
+    }
+  }
+}
+
+// builder
+
+template<typename Derived, typename Allocator>
+theta_base_builder<Derived, Allocator>::theta_base_builder(const Allocator& allocator):
+allocator_(allocator), lg_k_(DEFAULT_LG_K), rf_(DEFAULT_RESIZE_FACTOR), p_(1), seed_(DEFAULT_SEED) {}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_lg_k(uint8_t lg_k) {
+  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 static_cast<Derived&>(*this);
+}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_resize_factor(resize_factor rf) {
+  rf_ = rf;
+  return static_cast<Derived&>(*this);
+}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_p(float p) {
+  if (p < 0 || p > 1) throw std::invalid_argument("sampling probability must be between 0 and 1");

Review comment:
       I am not sure it makes much sense. Sampling with probability of 0?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] davecromberge commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r482494067



##########
File path: tuple/CMakeLists.txt
##########
@@ -0,0 +1,94 @@
+# 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/tuple_intersection.hpp;include/tuple_intersection_impl.hpp")
+list(APPEND tuple_HEADERS "include/tuple_a_not_b.hpp;include/tuple_a_not_b_impl.hpp")
+list(APPEND tuple_HEADERS "include/array_of_doubles_sketch.hpp;include/array_of_doubles_sketch_impl.hpp")
+list(APPEND tuple_HEADERS "include/array_of_doubles_union.hpp;include/array_of_doulbes_union_impl.hpp")
+list(APPEND tuple_HEADERS "include/array_of_doubles_intersection.hpp;include/array_of_doulbes_intersection_impl.hpp")
+list(APPEND tuple_HEADERS "include/array_of_doubles_a_not_b.hpp;include/array_of_doulbes__ot_b_impl.hpp")

Review comment:
       There is a minor spelling error in the names of the header files above `doulbes`.

##########
File path: tuple/include/array_of_doubles_sketch_impl.hpp
##########
@@ -0,0 +1,238 @@
+/*
+ * 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>
+update_array_of_doubles_sketch_alloc<A>::update_array_of_doubles_sketch_alloc(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf,
+    uint64_t theta, uint64_t seed, const array_of_doubles_update_policy<A>& policy, const A& allocator):
+Base(lg_cur_size, lg_nom_size, rf, theta, seed, policy, allocator) {}
+
+
+template<typename A>
+uint8_t update_array_of_doubles_sketch_alloc<A>::get_num_values() const {
+  return this->policy_.get_num_values();
+}
+
+template<typename A>
+compact_array_of_doubles_sketch_alloc<A> update_array_of_doubles_sketch_alloc<A>::compact(bool ordered) const {
+  return compact_array_of_doubles_sketch_alloc<A>(*this, ordered);
+}
+
+// builder
+
+template<typename A>
+update_array_of_doubles_sketch_alloc<A>::builder::builder(const array_of_doubles_update_policy<A>& policy, const A& allocator):
+tuple_base_builder<builder, array_of_doubles_update_policy<A>, A>(policy, allocator) {}
+
+template<typename A>
+update_array_of_doubles_sketch_alloc<A> update_array_of_doubles_sketch_alloc<A>::builder::build() const {
+  return update_array_of_doubles_sketch_alloc<A>(this->starting_lg_size(), this->lg_k_, this->rf_, this->starting_theta(), this->seed_, this->policy_, this->allocator_);
+}
+
+// compact sketch
+
+template<typename A>
+template<typename S>
+compact_array_of_doubles_sketch_alloc<A>::compact_array_of_doubles_sketch_alloc(const S& other, bool ordered):
+Base(other, ordered), num_values_(other.get_num_values()) {}
+
+template<typename A>
+compact_array_of_doubles_sketch_alloc<A>::compact_array_of_doubles_sketch_alloc(bool is_empty, bool is_ordered,
+    uint16_t seed_hash, uint64_t theta, std::vector<Entry, AllocEntry>&& entries, uint8_t num_values):
+Base(is_empty, is_ordered, seed_hash, theta, std::move(entries)), num_values_(num_values) {}
+
+template<typename A>
+compact_array_of_doubles_sketch_alloc<A>::compact_array_of_doubles_sketch_alloc(uint8_t num_values, Base&& base):
+Base(std::move(base)), num_values_(num_values) {}
+
+template<typename A>
+uint8_t compact_array_of_doubles_sketch_alloc<A>::get_num_values() const {
+  return num_values_;
+}
+
+template<typename A>
+void compact_array_of_doubles_sketch_alloc<A>::serialize(std::ostream& os) const {
+  const uint8_t preamble_longs = 1;
+  os.write(reinterpret_cast<const char*>(&preamble_longs), sizeof(preamble_longs));
+  const uint8_t serial_version = SERIAL_VERSION;
+  os.write(reinterpret_cast<const char*>(&serial_version), sizeof(serial_version));
+  const uint8_t family = SKETCH_FAMILY;
+  os.write(reinterpret_cast<const char*>(&family), sizeof(family));
+  const uint8_t type = SKETCH_TYPE;
+  os.write(reinterpret_cast<const char*>(&type), sizeof(type));
+  const uint8_t flags_byte(
+    (this->is_empty() ? 1 << flags::IS_EMPTY : 0) |
+    (this->get_num_retained() > 0 ? 1 << flags::HAS_ENTRIES : 0) |
+    (this->is_ordered() ? 1 << flags::IS_ORDERED : 0)
+  );
+  os.write(reinterpret_cast<const char*>(&flags_byte), sizeof(flags_byte));
+  os.write(reinterpret_cast<const char*>(&num_values_), sizeof(num_values_));
+  const uint16_t seed_hash = this->get_seed_hash();
+  os.write(reinterpret_cast<const char*>(&seed_hash), sizeof(seed_hash));
+  os.write(reinterpret_cast<const char*>(&(this->theta_)), sizeof(uint64_t));
+  if (this->get_num_retained() > 0) {
+    const uint32_t num_entries = this->entries_.size();
+    os.write(reinterpret_cast<const char*>(&num_entries), sizeof(num_entries));
+    const uint32_t unused32 = 0;
+    os.write(reinterpret_cast<const char*>(&unused32), sizeof(unused32));
+    for (const auto& it: this->entries_) {
+      os.write(reinterpret_cast<const char*>(&it.first), sizeof(uint64_t));
+    }
+    for (const auto& it: this->entries_) {
+      os.write(reinterpret_cast<const char*>(it.second.data()), it.second.size() * sizeof(double));
+    }
+  }
+}

Review comment:
       Assuming sketches are compatible across implementations, do we test that the raw bytes that have been serialised are compatible with the java counterpart?  

##########
File path: tuple/include/array_of_doubles_intersection.hpp
##########
@@ -0,0 +1,52 @@
+/*
+ * 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 ARRAY_OF_DOUBLES_INTERSECTION_HPP_
+#define ARRAY_OF_DOUBLES_INTERSECTION_HPP_
+
+#include <vector>
+#include <memory>
+
+#include "array_of_doubles_sketch.hpp"
+#include "tuple_intersection.hpp"
+
+namespace datasketches {
+
+template<
+  typename Policy,

Review comment:
       For consistency purposes, is it better to use Mode here, or use Policy in the java version?  The same conceptual idea has different names in both implementations.  Do we strive for this or is this a non-issue?

##########
File path: tuple/include/theta_intersection_base_impl.hpp
##########
@@ -0,0 +1,121 @@
+/*
+ * 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>
+
+#include "conditional_forward.hpp"
+
+namespace datasketches {
+
+template<typename EN, typename EK, typename P, typename S, typename CS, typename A>
+theta_intersection_base<EN, EK, P, S, CS, A>::theta_intersection_base(uint64_t seed, const P& policy, const A& allocator):
+policy_(policy),
+is_valid_(false),
+table_(0, 0, resize_factor::X1, theta_constants::MAX_THETA, seed, allocator, false)
+{}
+
+template<typename EN, typename EK, typename P, typename S, typename CS, typename A>
+template<typename SS>
+void theta_intersection_base<EN, EK, P, S, CS, A>::update(SS&& sketch) {
+  if (table_.is_empty_) return;
+  if (!sketch.is_empty() && sketch.get_seed_hash() != compute_seed_hash(table_.seed_)) throw std::invalid_argument("seed hash mismatch");
+  table_.is_empty_ |= sketch.is_empty();
+  table_.theta_ = std::min(table_.theta_, sketch.get_theta64());
+  if (is_valid_ && table_.num_entries_ == 0) return;
+  if (sketch.get_num_retained() == 0) {
+    is_valid_ = true;
+    table_ = hash_table(0, 0, resize_factor::X1, table_.theta_, table_.seed_, table_.allocator_, table_.is_empty_);
+    return;
+  }
+  if (!is_valid_) { // first update, copy or move incoming sketch
+    is_valid_ = true;
+    const uint8_t lg_size = lg_size_from_count(sketch.get_num_retained(), theta_update_sketch_base<EN, EK, A>::REBUILD_THRESHOLD);
+    table_ = hash_table(lg_size, lg_size, resize_factor::X1, table_.theta_, table_.seed_, table_.allocator_, table_.is_empty_);
+    for (auto& entry: sketch) {
+      auto result = table_.find(EK()(entry));
+      if (result.second) {
+        throw std::invalid_argument("duplicate key, possibly corrupted input sketch");
+      }
+      table_.insert(result.first, conditional_forward<SS>(entry));
+    }
+    if (table_.num_entries_ != sketch.get_num_retained()) throw std::invalid_argument("num entries mismatch, possibly corrupted input sketch");
+  } else { // intersection
+    const uint32_t max_matches = std::min(table_.num_entries_, sketch.get_num_retained());
+    std::vector<EN, A> matched_entries(table_.allocator_);
+    matched_entries.reserve(max_matches);
+    uint32_t match_count = 0;
+    uint32_t count = 0;
+    for (auto& entry: sketch) {
+      if (EK()(entry) < table_.theta_) {
+        auto result = table_.find(EK()(entry));
+        if (result.second) {
+          if (match_count == max_matches) throw std::invalid_argument("max matches exceeded, possibly corrupted input sketch");
+          policy_(*result.first, conditional_forward<SS>(entry));
+          matched_entries.push_back(std::move(*result.first));
+          ++match_count;
+        }
+      } else if (sketch.is_ordered()) {
+        break; // early stop
+      }
+      ++count;
+    }
+    if (count > sketch.get_num_retained()) {
+      throw std::invalid_argument(" more keys then expected, possibly corrupted input sketch");
+    } else if (!sketch.is_ordered() && count < sketch.get_num_retained()) {
+      throw std::invalid_argument(" fewer keys then expected, possibly corrupted input sketch");

Review comment:
       There is a minor typo in the exception message `then` should be `than`

##########
File path: tuple/include/theta_set_difference_base_impl.hpp
##########
@@ -0,0 +1,80 @@
+/*
+ * 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 <algorithm>
+
+#include "conditional_back_inserter.hpp"
+#include "conditional_forward.hpp"
+
+namespace datasketches {
+
+template<typename EN, typename EK, typename CS, typename A>
+theta_set_difference_base<EN, EK, CS, A>::theta_set_difference_base(uint64_t seed, const A& allocator):
+allocator_(allocator),
+seed_hash_(compute_seed_hash(seed))
+{}
+
+template<typename EN, typename EK, typename CS, typename A>
+template<typename FwdSketch, typename Sketch>
+CS theta_set_difference_base<EN, EK, CS, A>::compute(FwdSketch&& a, const Sketch& b, bool ordered) const {
+  if (a.is_empty() || a.get_num_retained() == 0 || b.is_empty()) return CS(a, ordered);
+  if (a.get_seed_hash() != seed_hash_) throw std::invalid_argument("A seed hash mismatch");
+  if (b.get_seed_hash() != seed_hash_) throw std::invalid_argument("B seed hash mismatch");
+
+  const uint64_t theta = std::min(a.get_theta64(), b.get_theta64());
+  std::vector<EN, A> entries(allocator_);
+  bool is_empty = a.is_empty();
+
+  if (b.get_num_retained() == 0) {
+    std::copy_if(forward_begin(std::forward<FwdSketch>(a)), forward_end(std::forward<FwdSketch>(a)), std::back_inserter(entries),
+        key_less_than<uint64_t, EN, EK>(theta));
+  } else {
+    if (a.is_ordered() && b.is_ordered()) { // sort-based
+      std::set_difference(forward_begin(std::forward<FwdSketch>(a)), forward_end(std::forward<FwdSketch>(a)), b.begin(), b.end(),
+          conditional_back_inserter(entries, key_less_than<uint64_t, EN, EK>(theta)), comparator());
+    } else { // hash-based
+      const uint8_t lg_size = lg_size_from_count(b.get_num_retained(), hash_table::REBUILD_THRESHOLD);
+      hash_table table(lg_size, lg_size, hash_table::resize_factor::X1, 0, 0, allocator_); // theta and seed are not used here
+      for (const auto& entry: b) {
+        const uint64_t hash = EK()(entry);
+        if (hash < theta) {
+          table.insert(table.find(hash).first, hash);
+        } else if (b.is_ordered()) {
+          break; // early stop
+        }
+      }
+
+      // scan A lookup B

Review comment:
       Is the scan optimised to choose the "smaller" of the two, assuming its cheaper to scan smaller sets?

##########
File path: tuple/test/tuple_sketch_allocation_test.cpp
##########
@@ -0,0 +1,100 @@
+/*
+ * 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_sketch.hpp>
+#include <test_allocator.hpp>
+#include <test_type.hpp>
+
+namespace datasketches {
+
+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;
+  }
+};
+
+using update_tuple_sketch_test =
+    update_tuple_sketch<test_type, test_type, test_type_replace_policy, test_allocator<test_type>>;
+using compact_tuple_sketch_test =
+    compact_tuple_sketch<test_type, test_allocator<test_type>>;
+
+TEST_CASE("tuple sketch with test allocator: estimation mode", "[tuple_sketch]") {
+  test_allocator_total_bytes = 0;
+  test_allocator_net_allocations = 0;
+  {
+    auto update_sketch = update_tuple_sketch_test::builder().build();
+    for (int i = 0; i < 10000; ++i) update_sketch.update(i, 1);
+    for (int i = 0; i < 10000; ++i) update_sketch.update(i, 2);
+    REQUIRE(!update_sketch.is_empty());
+    REQUIRE(update_sketch.is_estimation_mode());
+    unsigned count = 0;
+    for (const auto& entry: update_sketch) {
+      REQUIRE(entry.second.get_value() == 2);
+      ++count;
+    }
+    REQUIRE(count == update_sketch.get_num_retained());
+
+    update_sketch.trim();
+    REQUIRE(update_sketch.get_num_retained() == (1 << update_sketch.get_lg_k()));
+
+    auto compact_sketch = update_sketch.compact();
+    REQUIRE(!compact_sketch.is_empty());
+    REQUIRE(compact_sketch.is_estimation_mode());
+    count = 0;
+    for (const auto& entry: compact_sketch) {
+      REQUIRE(entry.second.get_value() == 2);
+      ++count;
+    }
+    REQUIRE(count == update_sketch.get_num_retained());
+
+    auto bytes = compact_sketch.serialize(0, test_type_serde());
+    auto deserialized_sketch = compact_tuple_sketch_test::deserialize(bytes.data(), bytes.size(), DEFAULT_SEED, test_type_serde());
+    REQUIRE(deserialized_sketch.get_estimate() == compact_sketch.get_estimate());
+
+    // update sketch copy
+//    std::cout << update_sketch.to_string();

Review comment:
       Did you mean to this debugging commented out for future convenience?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r483789214



##########
File path: tuple/CMakeLists.txt
##########
@@ -0,0 +1,94 @@
+# 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/tuple_intersection.hpp;include/tuple_intersection_impl.hpp")
+list(APPEND tuple_HEADERS "include/tuple_a_not_b.hpp;include/tuple_a_not_b_impl.hpp")
+list(APPEND tuple_HEADERS "include/array_of_doubles_sketch.hpp;include/array_of_doubles_sketch_impl.hpp")
+list(APPEND tuple_HEADERS "include/array_of_doubles_union.hpp;include/array_of_doulbes_union_impl.hpp")
+list(APPEND tuple_HEADERS "include/array_of_doubles_intersection.hpp;include/array_of_doulbes_intersection_impl.hpp")
+list(APPEND tuple_HEADERS "include/array_of_doubles_a_not_b.hpp;include/array_of_doulbes__ot_b_impl.hpp")

Review comment:
       fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r501936795



##########
File path: tuple/include/theta_update_sketch_base_impl.hpp
##########
@@ -0,0 +1,384 @@
+/*
+ * 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, uint64_t theta, uint64_t seed, const A& allocator, bool is_empty):
+allocator_(allocator),
+is_empty_(is_empty),
+lg_cur_size_(lg_cur_size),
+lg_nom_size_(lg_nom_size),
+rf_(rf),
+num_entries_(0),
+theta_(theta),
+seed_(seed),
+entries_(nullptr)
+{
+  if (lg_cur_size > 0) {
+    const size_t size = 1 << lg_cur_size;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(const theta_update_sketch_base& other):
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(nullptr)
+{
+  if (other.entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(other.entries_[i]) != 0) {
+        new (&entries_[i]) EN(other.entries_[i]);
+      } else {
+        EK()(entries_[i]) = 0;
+      }
+    }
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(theta_update_sketch_base&& other) noexcept:
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(other.entries_)
+{
+  other.entries_ = nullptr;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::~theta_update_sketch_base()
+{
+  if (entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(entries_[i]) != 0) entries_[i].~EN();
+    }
+    allocator_.deallocate(entries_, size);
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(const theta_update_sketch_base& other) {
+  theta_update_sketch_base<EN, EK, A> copy(other);
+  std::swap(allocator_, copy.allocator_);
+  std::swap(is_empty_, copy.is_empty_);
+  std::swap(lg_cur_size_, copy.lg_cur_size_);
+  std::swap(lg_nom_size_, copy.lg_nom_size_);
+  std::swap(rf_, copy.rf_);
+  std::swap(num_entries_, copy.num_entries_);
+  std::swap(theta_, copy.theta_);
+  std::swap(seed_, copy.seed_);
+  std::swap(entries_, copy.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(theta_update_sketch_base&& other) {
+  std::swap(allocator_, other.allocator_);
+  std::swap(is_empty_, other.is_empty_);
+  std::swap(lg_cur_size_, other.lg_cur_size_);
+  std::swap(lg_nom_size_, other.lg_nom_size_);
+  std::swap(rf_, other.rf_);
+  std::swap(num_entries_, other.num_entries_);
+  std::swap(theta_, other.theta_);
+  std::swap(seed_, other.seed_);
+  std::swap(entries_, other.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+uint64_t theta_update_sketch_base<EN, EK, A>::hash_and_screen(const void* data, size_t length) {
+  is_empty_ = false;
+  const uint64_t hash = compute_hash(data, length, seed_);
+  if (hash >= theta_) return 0; // hash == 0 is reserved to mark empty slots in the table
+  return hash;
+}
+
+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));
+  ++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>
+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_;
+  EN* old_entries = entries_;
+  entries_ = allocator_.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();
+    }
+  }
+  allocator_.deallocate(old_entries, old_size);
+}
+
+// assumes number of entries > nominal size
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::rebuild() {
+  const size_t size = 1 << lg_cur_size_;
+  const uint32_t nominal_size = 1 << lg_nom_size_;
+
+  // empty entries have uninitialized payloads
+  // TODO: avoid this for empty or trivial payloads (arithmetic types)
+  consolidate_non_empty(entries_, size, num_entries_);
+
+  std::nth_element(entries_, entries_ + nominal_size, entries_ + num_entries_, comparator());
+  this->theta_ = EK()(entries_[nominal_size]);
+  EN* old_entries = entries_;
+  entries_ = allocator_.allocate(size);
+  for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  num_entries_ = 0;
+  // relies on consolidating non-empty entries to the front
+  for (size_t i = 0; i < nominal_size; ++i) {
+    insert(find(EK()(old_entries[i])).first, std::move(old_entries[i])); // consider a special insert with no comparison
+    old_entries[i].~EN();
+  }
+  allocator_.deallocate(old_entries, size);
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::trim() {
+  if (num_entries_ > static_cast<uint32_t>(1 << lg_nom_size_)) rebuild();
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::consolidate_non_empty(EN* entries, size_t size, size_t num) {
+  // find the first empty slot
+  size_t i = 0;
+  while (i < size) {
+    if (EK()(entries[i]) == 0) break;
+    ++i;
+  }
+  // scan the rest and move non-empty entries to the front
+  for (size_t j = i + 1; j < size; ++j) {
+    if (EK()(entries[j]) != 0) {
+      new (&entries[i]) EN(std::move(entries[j]));
+      entries[j].~EN();
+      EK()(entries[j]) = 0;
+      ++i;
+      if (i == num) break;
+    }
+  }
+}
+
+// builder
+
+template<typename Derived, typename Allocator>
+theta_base_builder<Derived, Allocator>::theta_base_builder(const Allocator& allocator):
+allocator_(allocator), lg_k_(DEFAULT_LG_K), rf_(DEFAULT_RESIZE_FACTOR), p_(1), seed_(DEFAULT_SEED) {}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_lg_k(uint8_t lg_k) {
+  if (lg_k < MIN_LG_K) {

Review comment:
       I think so. I will add that

##########
File path: tuple/include/theta_update_sketch_base_impl.hpp
##########
@@ -0,0 +1,384 @@
+/*
+ * 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, uint64_t theta, uint64_t seed, const A& allocator, bool is_empty):
+allocator_(allocator),
+is_empty_(is_empty),
+lg_cur_size_(lg_cur_size),
+lg_nom_size_(lg_nom_size),
+rf_(rf),
+num_entries_(0),
+theta_(theta),
+seed_(seed),
+entries_(nullptr)
+{
+  if (lg_cur_size > 0) {
+    const size_t size = 1 << lg_cur_size;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(const theta_update_sketch_base& other):
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(nullptr)
+{
+  if (other.entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(other.entries_[i]) != 0) {
+        new (&entries_[i]) EN(other.entries_[i]);
+      } else {
+        EK()(entries_[i]) = 0;
+      }
+    }
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(theta_update_sketch_base&& other) noexcept:
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(other.entries_)
+{
+  other.entries_ = nullptr;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::~theta_update_sketch_base()
+{
+  if (entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(entries_[i]) != 0) entries_[i].~EN();
+    }
+    allocator_.deallocate(entries_, size);
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(const theta_update_sketch_base& other) {
+  theta_update_sketch_base<EN, EK, A> copy(other);
+  std::swap(allocator_, copy.allocator_);
+  std::swap(is_empty_, copy.is_empty_);
+  std::swap(lg_cur_size_, copy.lg_cur_size_);
+  std::swap(lg_nom_size_, copy.lg_nom_size_);
+  std::swap(rf_, copy.rf_);
+  std::swap(num_entries_, copy.num_entries_);
+  std::swap(theta_, copy.theta_);
+  std::swap(seed_, copy.seed_);
+  std::swap(entries_, copy.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(theta_update_sketch_base&& other) {
+  std::swap(allocator_, other.allocator_);
+  std::swap(is_empty_, other.is_empty_);
+  std::swap(lg_cur_size_, other.lg_cur_size_);
+  std::swap(lg_nom_size_, other.lg_nom_size_);
+  std::swap(rf_, other.rf_);
+  std::swap(num_entries_, other.num_entries_);
+  std::swap(theta_, other.theta_);
+  std::swap(seed_, other.seed_);
+  std::swap(entries_, other.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+uint64_t theta_update_sketch_base<EN, EK, A>::hash_and_screen(const void* data, size_t length) {
+  is_empty_ = false;
+  const uint64_t hash = compute_hash(data, length, seed_);
+  if (hash >= theta_) return 0; // hash == 0 is reserved to mark empty slots in the table
+  return hash;
+}
+
+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));
+  ++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>
+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_;
+  EN* old_entries = entries_;
+  entries_ = allocator_.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();
+    }
+  }
+  allocator_.deallocate(old_entries, old_size);
+}
+
+// assumes number of entries > nominal size
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::rebuild() {
+  const size_t size = 1 << lg_cur_size_;
+  const uint32_t nominal_size = 1 << lg_nom_size_;
+
+  // empty entries have uninitialized payloads
+  // TODO: avoid this for empty or trivial payloads (arithmetic types)
+  consolidate_non_empty(entries_, size, num_entries_);
+
+  std::nth_element(entries_, entries_ + nominal_size, entries_ + num_entries_, comparator());
+  this->theta_ = EK()(entries_[nominal_size]);
+  EN* old_entries = entries_;
+  entries_ = allocator_.allocate(size);
+  for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  num_entries_ = 0;
+  // relies on consolidating non-empty entries to the front
+  for (size_t i = 0; i < nominal_size; ++i) {
+    insert(find(EK()(old_entries[i])).first, std::move(old_entries[i])); // consider a special insert with no comparison
+    old_entries[i].~EN();
+  }
+  allocator_.deallocate(old_entries, size);
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::trim() {
+  if (num_entries_ > static_cast<uint32_t>(1 << lg_nom_size_)) rebuild();
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::consolidate_non_empty(EN* entries, size_t size, size_t num) {
+  // find the first empty slot
+  size_t i = 0;
+  while (i < size) {
+    if (EK()(entries[i]) == 0) break;
+    ++i;
+  }
+  // scan the rest and move non-empty entries to the front
+  for (size_t j = i + 1; j < size; ++j) {
+    if (EK()(entries[j]) != 0) {
+      new (&entries[i]) EN(std::move(entries[j]));
+      entries[j].~EN();
+      EK()(entries[j]) = 0;
+      ++i;
+      if (i == num) break;
+    }
+  }
+}
+
+// builder
+
+template<typename Derived, typename Allocator>
+theta_base_builder<Derived, Allocator>::theta_base_builder(const Allocator& allocator):
+allocator_(allocator), lg_k_(DEFAULT_LG_K), rf_(DEFAULT_RESIZE_FACTOR), p_(1), seed_(DEFAULT_SEED) {}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_lg_k(uint8_t lg_k) {
+  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 static_cast<Derived&>(*this);
+}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_resize_factor(resize_factor rf) {
+  rf_ = rf;
+  return static_cast<Derived&>(*this);
+}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_p(float p) {
+  if (p < 0 || p > 1) throw std::invalid_argument("sampling probability must be between 0 and 1");

Review comment:
       I am not sure it makes much sense. Sampling with probability of 0?

##########
File path: tuple/include/array_of_doubles_sketch.hpp
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 ARRAY_OF_DOUBLES_SKETCH_HPP_
+#define ARRAY_OF_DOUBLES_SKETCH_HPP_
+
+#include <vector>
+#include <memory>
+
+#include "serde.hpp"
+#include "tuple_sketch.hpp"
+
+namespace datasketches {
+
+// This sketch is equivalent of ArrayOfDoublesSketch in Java
+
+// This simple array of double is faster than std::vector and should be sufficient for this application
+template<typename Allocator = std::allocator<double>>
+class aod {
+public:
+  explicit aod(uint8_t size, const Allocator& allocator = Allocator()):
+  allocator_(allocator), size_(size), array_(allocator_.allocate(size_)) {
+    std::fill(array_, array_ + size_, 0);
+  }
+  aod(const aod& other):
+    allocator_(other.allocator_),
+    size_(other.size_),
+    array_(allocator_.allocate(size_))
+  {
+    std::copy(other.array_, other.array_ + size_, array_);
+  }
+  aod(aod&& other) noexcept:
+    allocator_(std::move(other.allocator_)),
+    size_(other.size_),
+    array_(other.array_)
+  {
+    other.array_ = nullptr;
+  }
+  ~aod() {
+    if (array_ != nullptr) allocator_.deallocate(array_, size_);
+  }
+  aod& operator=(const aod& other) {
+    aod copy(other);
+    std::swap(allocator_, copy.allocator_);
+    std::swap(size_, copy.size_);
+    std::swap(array_, copy.array_);
+    return *this;
+  }
+  aod& operator=(aod&& other) {
+    std::swap(allocator_, other.allocator_);
+    std::swap(size_, other.size_);
+    std::swap(array_, other.array_);
+    return *this;
+  }
+  double& operator[](size_t index) { return array_[index]; }
+  double operator[](size_t index) const { return array_[index]; }
+  uint8_t size() const { return size_; }
+  double* data() { return array_; }
+  const double* data() const { return array_; }
+  bool operator==(const aod& other) const {
+    for (uint8_t i = 0; i < size_; ++i) if (array_[i] != other.array_[i]) return false;
+    return true;
+  }
+private:
+  Allocator allocator_;
+  uint8_t size_;
+  double* array_;
+};
+
+template<typename A = std::allocator<double>>
+class array_of_doubles_update_policy {
+public:
+  array_of_doubles_update_policy(uint8_t num_values = 1, const A& allocator = A()):
+    allocator_(allocator), num_values_(num_values) {}
+  aod<A> create() const {
+    return aod<A>(num_values_, allocator_);
+  }
+  template<typename InputVector> // to allow any type with indexed access (such as double*)
+  void update(aod<A>& summary, const InputVector& update) const {
+    for (uint8_t i = 0; i < num_values_; ++i) summary[i] += update[i];
+  }
+  uint8_t get_num_values() const {
+    return num_values_;
+  }
+
+private:
+  A allocator_;
+  uint8_t num_values_;
+};
+
+// forward declaration
+template<typename A> class compact_array_of_doubles_sketch_alloc;
+
+template<typename A> using AllocAOD = typename std::allocator_traits<A>::template rebind_alloc<aod<A>>;
+
+template<typename A = std::allocator<double>>
+class update_array_of_doubles_sketch_alloc: public update_tuple_sketch<aod<A>, aod<A>, array_of_doubles_update_policy<A>, AllocAOD<A>> {
+public:
+  using Base = update_tuple_sketch<aod<A>, aod<A>, array_of_doubles_update_policy<A>, AllocAOD<A>>;
+  using resize_factor = typename Base::resize_factor;
+
+  class builder;
+
+  compact_array_of_doubles_sketch_alloc<A> compact(bool ordered = true) const;
+  uint8_t get_num_values() const;
+
+private:
+  // for builder
+  update_array_of_doubles_sketch_alloc(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, uint64_t theta,
+      uint64_t seed, const array_of_doubles_update_policy<A>& policy, const A& allocator);
+};
+
+// alias with the default allocator for convenience
+using update_array_of_doubles_sketch = update_array_of_doubles_sketch_alloc<>;
+
+template<typename A>
+class update_array_of_doubles_sketch_alloc<A>::builder: public tuple_base_builder<builder, array_of_doubles_update_policy<A>, A> {
+public:
+  builder(const array_of_doubles_update_policy<A>& policy = array_of_doubles_update_policy<A>(), const A& allocator = A());
+  update_array_of_doubles_sketch_alloc<A> build() const;
+};
+
+template<typename A = std::allocator<double>>
+class compact_array_of_doubles_sketch_alloc: public compact_tuple_sketch<aod<A>, AllocAOD<A>> {
+public:
+  using Base = compact_tuple_sketch<aod<A>, AllocAOD<A>>;
+  using Entry = typename Base::Entry;
+  using AllocEntry = typename Base::AllocEntry;
+  using AllocU64 = typename Base::AllocU64;
+  using vector_bytes = typename Base::vector_bytes;
+
+  static const uint8_t SERIAL_VERSION = 1;
+  static const uint8_t SKETCH_FAMILY = 9;

Review comment:
       This is a different family and different serial format. I don't really like this concept, but it serves as a compatibility check.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] davecromberge commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r483824281



##########
File path: tuple/include/array_of_doubles_intersection.hpp
##########
@@ -0,0 +1,52 @@
+/*
+ * 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 ARRAY_OF_DOUBLES_INTERSECTION_HPP_
+#define ARRAY_OF_DOUBLES_INTERSECTION_HPP_
+
+#include <vector>
+#include <memory>
+
+#include "array_of_doubles_sketch.hpp"
+#include "tuple_intersection.hpp"
+
+namespace datasketches {
+
+template<
+  typename Policy,

Review comment:
       I agree with your reasons for keeping it as it is, seeing as the implementations are different. The default policy that you have chosen for the different operations also sensible. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r483849318



##########
File path: tuple/include/theta_set_difference_base_impl.hpp
##########
@@ -0,0 +1,80 @@
+/*
+ * 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 <algorithm>
+
+#include "conditional_back_inserter.hpp"
+#include "conditional_forward.hpp"
+
+namespace datasketches {
+
+template<typename EN, typename EK, typename CS, typename A>
+theta_set_difference_base<EN, EK, CS, A>::theta_set_difference_base(uint64_t seed, const A& allocator):
+allocator_(allocator),
+seed_hash_(compute_seed_hash(seed))
+{}
+
+template<typename EN, typename EK, typename CS, typename A>
+template<typename FwdSketch, typename Sketch>
+CS theta_set_difference_base<EN, EK, CS, A>::compute(FwdSketch&& a, const Sketch& b, bool ordered) const {
+  if (a.is_empty() || a.get_num_retained() == 0 || b.is_empty()) return CS(a, ordered);
+  if (a.get_seed_hash() != seed_hash_) throw std::invalid_argument("A seed hash mismatch");
+  if (b.get_seed_hash() != seed_hash_) throw std::invalid_argument("B seed hash mismatch");
+
+  const uint64_t theta = std::min(a.get_theta64(), b.get_theta64());
+  std::vector<EN, A> entries(allocator_);
+  bool is_empty = a.is_empty();
+
+  if (b.get_num_retained() == 0) {
+    std::copy_if(forward_begin(std::forward<FwdSketch>(a)), forward_end(std::forward<FwdSketch>(a)), std::back_inserter(entries),
+        key_less_than<uint64_t, EN, EK>(theta));
+  } else {
+    if (a.is_ordered() && b.is_ordered()) { // sort-based
+      std::set_difference(forward_begin(std::forward<FwdSketch>(a)), forward_end(std::forward<FwdSketch>(a)), b.begin(), b.end(),
+          conditional_back_inserter(entries, key_less_than<uint64_t, EN, EK>(theta)), comparator());
+    } else { // hash-based
+      const uint8_t lg_size = lg_size_from_count(b.get_num_retained(), hash_table::REBUILD_THRESHOLD);
+      hash_table table(lg_size, lg_size, hash_table::resize_factor::X1, 0, 0, allocator_); // theta and seed are not used here
+      for (const auto& entry: b) {
+        const uint64_t hash = EK()(entry);
+        if (hash < theta) {
+          table.insert(table.find(hash).first, hash);
+        } else if (b.is_ordered()) {
+          break; // early stop
+        }
+      }
+
+      // scan A lookup B

Review comment:
       I am not sure what algorithm you have in mind. We are computing a set difference here: we need to copy all entries from A except those also contained in B. So we convert B into a hash table for fast lookup, and then scan A. There is one potential optimization I see: avoid building a hash table from B if it happens to be a hash table already (an update sketch). I did not want to introduce a special treatment and somehow penetrate into B to figure out what it is for a limited (in my view) use case. I believe that the most typical scenario is that sketches are converted to compact form right after building them.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r483789403



##########
File path: tuple/include/array_of_doubles_sketch_impl.hpp
##########
@@ -0,0 +1,238 @@
+/*
+ * 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>
+update_array_of_doubles_sketch_alloc<A>::update_array_of_doubles_sketch_alloc(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf,
+    uint64_t theta, uint64_t seed, const array_of_doubles_update_policy<A>& policy, const A& allocator):
+Base(lg_cur_size, lg_nom_size, rf, theta, seed, policy, allocator) {}
+
+
+template<typename A>
+uint8_t update_array_of_doubles_sketch_alloc<A>::get_num_values() const {
+  return this->policy_.get_num_values();
+}
+
+template<typename A>
+compact_array_of_doubles_sketch_alloc<A> update_array_of_doubles_sketch_alloc<A>::compact(bool ordered) const {
+  return compact_array_of_doubles_sketch_alloc<A>(*this, ordered);
+}
+
+// builder
+
+template<typename A>
+update_array_of_doubles_sketch_alloc<A>::builder::builder(const array_of_doubles_update_policy<A>& policy, const A& allocator):
+tuple_base_builder<builder, array_of_doubles_update_policy<A>, A>(policy, allocator) {}
+
+template<typename A>
+update_array_of_doubles_sketch_alloc<A> update_array_of_doubles_sketch_alloc<A>::builder::build() const {
+  return update_array_of_doubles_sketch_alloc<A>(this->starting_lg_size(), this->lg_k_, this->rf_, this->starting_theta(), this->seed_, this->policy_, this->allocator_);
+}
+
+// compact sketch
+
+template<typename A>
+template<typename S>
+compact_array_of_doubles_sketch_alloc<A>::compact_array_of_doubles_sketch_alloc(const S& other, bool ordered):
+Base(other, ordered), num_values_(other.get_num_values()) {}
+
+template<typename A>
+compact_array_of_doubles_sketch_alloc<A>::compact_array_of_doubles_sketch_alloc(bool is_empty, bool is_ordered,
+    uint16_t seed_hash, uint64_t theta, std::vector<Entry, AllocEntry>&& entries, uint8_t num_values):
+Base(is_empty, is_ordered, seed_hash, theta, std::move(entries)), num_values_(num_values) {}
+
+template<typename A>
+compact_array_of_doubles_sketch_alloc<A>::compact_array_of_doubles_sketch_alloc(uint8_t num_values, Base&& base):
+Base(std::move(base)), num_values_(num_values) {}
+
+template<typename A>
+uint8_t compact_array_of_doubles_sketch_alloc<A>::get_num_values() const {
+  return num_values_;
+}
+
+template<typename A>
+void compact_array_of_doubles_sketch_alloc<A>::serialize(std::ostream& os) const {
+  const uint8_t preamble_longs = 1;
+  os.write(reinterpret_cast<const char*>(&preamble_longs), sizeof(preamble_longs));
+  const uint8_t serial_version = SERIAL_VERSION;
+  os.write(reinterpret_cast<const char*>(&serial_version), sizeof(serial_version));
+  const uint8_t family = SKETCH_FAMILY;
+  os.write(reinterpret_cast<const char*>(&family), sizeof(family));
+  const uint8_t type = SKETCH_TYPE;
+  os.write(reinterpret_cast<const char*>(&type), sizeof(type));
+  const uint8_t flags_byte(
+    (this->is_empty() ? 1 << flags::IS_EMPTY : 0) |
+    (this->get_num_retained() > 0 ? 1 << flags::HAS_ENTRIES : 0) |
+    (this->is_ordered() ? 1 << flags::IS_ORDERED : 0)
+  );
+  os.write(reinterpret_cast<const char*>(&flags_byte), sizeof(flags_byte));
+  os.write(reinterpret_cast<const char*>(&num_values_), sizeof(num_values_));
+  const uint16_t seed_hash = this->get_seed_hash();
+  os.write(reinterpret_cast<const char*>(&seed_hash), sizeof(seed_hash));
+  os.write(reinterpret_cast<const char*>(&(this->theta_)), sizeof(uint64_t));
+  if (this->get_num_retained() > 0) {
+    const uint32_t num_entries = this->entries_.size();
+    os.write(reinterpret_cast<const char*>(&num_entries), sizeof(num_entries));
+    const uint32_t unused32 = 0;
+    os.write(reinterpret_cast<const char*>(&unused32), sizeof(unused32));
+    for (const auto& it: this->entries_) {
+      os.write(reinterpret_cast<const char*>(&it.first), sizeof(uint64_t));
+    }
+    for (const auto& it: this->entries_) {
+      os.write(reinterpret_cast<const char*>(it.second.data()), it.second.size() * sizeof(double));
+    }
+  }
+}

Review comment:
       Yes, there are unit tests for this




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] jmalkin commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
jmalkin commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r499043827



##########
File path: tuple/include/theta_update_sketch_base_impl.hpp
##########
@@ -0,0 +1,384 @@
+/*
+ * 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, uint64_t theta, uint64_t seed, const A& allocator, bool is_empty):
+allocator_(allocator),
+is_empty_(is_empty),
+lg_cur_size_(lg_cur_size),
+lg_nom_size_(lg_nom_size),
+rf_(rf),
+num_entries_(0),
+theta_(theta),
+seed_(seed),
+entries_(nullptr)
+{
+  if (lg_cur_size > 0) {
+    const size_t size = 1 << lg_cur_size;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(const theta_update_sketch_base& other):
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(nullptr)
+{
+  if (other.entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(other.entries_[i]) != 0) {
+        new (&entries_[i]) EN(other.entries_[i]);
+      } else {
+        EK()(entries_[i]) = 0;
+      }
+    }
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(theta_update_sketch_base&& other) noexcept:
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(other.entries_)
+{
+  other.entries_ = nullptr;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::~theta_update_sketch_base()
+{
+  if (entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(entries_[i]) != 0) entries_[i].~EN();
+    }
+    allocator_.deallocate(entries_, size);
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(const theta_update_sketch_base& other) {
+  theta_update_sketch_base<EN, EK, A> copy(other);
+  std::swap(allocator_, copy.allocator_);
+  std::swap(is_empty_, copy.is_empty_);
+  std::swap(lg_cur_size_, copy.lg_cur_size_);
+  std::swap(lg_nom_size_, copy.lg_nom_size_);
+  std::swap(rf_, copy.rf_);
+  std::swap(num_entries_, copy.num_entries_);
+  std::swap(theta_, copy.theta_);
+  std::swap(seed_, copy.seed_);
+  std::swap(entries_, copy.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(theta_update_sketch_base&& other) {
+  std::swap(allocator_, other.allocator_);
+  std::swap(is_empty_, other.is_empty_);
+  std::swap(lg_cur_size_, other.lg_cur_size_);
+  std::swap(lg_nom_size_, other.lg_nom_size_);
+  std::swap(rf_, other.rf_);
+  std::swap(num_entries_, other.num_entries_);
+  std::swap(theta_, other.theta_);
+  std::swap(seed_, other.seed_);
+  std::swap(entries_, other.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+uint64_t theta_update_sketch_base<EN, EK, A>::hash_and_screen(const void* data, size_t length) {
+  is_empty_ = false;
+  const uint64_t hash = compute_hash(data, length, seed_);
+  if (hash >= theta_) return 0; // hash == 0 is reserved to mark empty slots in the table
+  return hash;
+}
+
+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));
+  ++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>
+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_;
+  EN* old_entries = entries_;
+  entries_ = allocator_.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();
+    }
+  }
+  allocator_.deallocate(old_entries, old_size);
+}
+
+// assumes number of entries > nominal size
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::rebuild() {
+  const size_t size = 1 << lg_cur_size_;
+  const uint32_t nominal_size = 1 << lg_nom_size_;
+
+  // empty entries have uninitialized payloads
+  // TODO: avoid this for empty or trivial payloads (arithmetic types)
+  consolidate_non_empty(entries_, size, num_entries_);
+
+  std::nth_element(entries_, entries_ + nominal_size, entries_ + num_entries_, comparator());
+  this->theta_ = EK()(entries_[nominal_size]);
+  EN* old_entries = entries_;
+  entries_ = allocator_.allocate(size);
+  for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  num_entries_ = 0;
+  // relies on consolidating non-empty entries to the front
+  for (size_t i = 0; i < nominal_size; ++i) {
+    insert(find(EK()(old_entries[i])).first, std::move(old_entries[i])); // consider a special insert with no comparison
+    old_entries[i].~EN();
+  }
+  allocator_.deallocate(old_entries, size);
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::trim() {
+  if (num_entries_ > static_cast<uint32_t>(1 << lg_nom_size_)) rebuild();
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::consolidate_non_empty(EN* entries, size_t size, size_t num) {
+  // find the first empty slot
+  size_t i = 0;
+  while (i < size) {
+    if (EK()(entries[i]) == 0) break;
+    ++i;
+  }
+  // scan the rest and move non-empty entries to the front
+  for (size_t j = i + 1; j < size; ++j) {
+    if (EK()(entries[j]) != 0) {
+      new (&entries[i]) EN(std::move(entries[j]));
+      entries[j].~EN();
+      EK()(entries[j]) = 0;
+      ++i;
+      if (i == num) break;
+    }
+  }
+}
+
+// builder
+
+template<typename Derived, typename Allocator>
+theta_base_builder<Derived, Allocator>::theta_base_builder(const Allocator& allocator):
+allocator_(allocator), lg_k_(DEFAULT_LG_K), rf_(DEFAULT_RESIZE_FACTOR), p_(1), seed_(DEFAULT_SEED) {}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_lg_k(uint8_t lg_k) {
+  if (lg_k < MIN_LG_K) {

Review comment:
       do we want to enforce a max?

##########
File path: tuple/include/theta_update_sketch_base_impl.hpp
##########
@@ -0,0 +1,384 @@
+/*
+ * 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, uint64_t theta, uint64_t seed, const A& allocator, bool is_empty):
+allocator_(allocator),
+is_empty_(is_empty),
+lg_cur_size_(lg_cur_size),
+lg_nom_size_(lg_nom_size),
+rf_(rf),
+num_entries_(0),
+theta_(theta),
+seed_(seed),
+entries_(nullptr)
+{
+  if (lg_cur_size > 0) {
+    const size_t size = 1 << lg_cur_size;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(const theta_update_sketch_base& other):
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(nullptr)
+{
+  if (other.entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    entries_ = allocator_.allocate(size);
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(other.entries_[i]) != 0) {
+        new (&entries_[i]) EN(other.entries_[i]);
+      } else {
+        EK()(entries_[i]) = 0;
+      }
+    }
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::theta_update_sketch_base(theta_update_sketch_base&& other) noexcept:
+allocator_(other.allocator_),
+is_empty_(other.is_empty_),
+lg_cur_size_(other.lg_cur_size_),
+lg_nom_size_(other.lg_nom_size_),
+rf_(other.rf_),
+num_entries_(other.num_entries_),
+theta_(other.theta_),
+seed_(other.seed_),
+entries_(other.entries_)
+{
+  other.entries_ = nullptr;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>::~theta_update_sketch_base()
+{
+  if (entries_ != nullptr) {
+    const size_t size = 1 << lg_cur_size_;
+    for (size_t i = 0; i < size; ++i) {
+      if (EK()(entries_[i]) != 0) entries_[i].~EN();
+    }
+    allocator_.deallocate(entries_, size);
+  }
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(const theta_update_sketch_base& other) {
+  theta_update_sketch_base<EN, EK, A> copy(other);
+  std::swap(allocator_, copy.allocator_);
+  std::swap(is_empty_, copy.is_empty_);
+  std::swap(lg_cur_size_, copy.lg_cur_size_);
+  std::swap(lg_nom_size_, copy.lg_nom_size_);
+  std::swap(rf_, copy.rf_);
+  std::swap(num_entries_, copy.num_entries_);
+  std::swap(theta_, copy.theta_);
+  std::swap(seed_, copy.seed_);
+  std::swap(entries_, copy.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+theta_update_sketch_base<EN, EK, A>& theta_update_sketch_base<EN, EK, A>::operator=(theta_update_sketch_base&& other) {
+  std::swap(allocator_, other.allocator_);
+  std::swap(is_empty_, other.is_empty_);
+  std::swap(lg_cur_size_, other.lg_cur_size_);
+  std::swap(lg_nom_size_, other.lg_nom_size_);
+  std::swap(rf_, other.rf_);
+  std::swap(num_entries_, other.num_entries_);
+  std::swap(theta_, other.theta_);
+  std::swap(seed_, other.seed_);
+  std::swap(entries_, other.entries_);
+  return *this;
+}
+
+template<typename EN, typename EK, typename A>
+uint64_t theta_update_sketch_base<EN, EK, A>::hash_and_screen(const void* data, size_t length) {
+  is_empty_ = false;
+  const uint64_t hash = compute_hash(data, length, seed_);
+  if (hash >= theta_) return 0; // hash == 0 is reserved to mark empty slots in the table
+  return hash;
+}
+
+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));
+  ++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>
+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_;
+  EN* old_entries = entries_;
+  entries_ = allocator_.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();
+    }
+  }
+  allocator_.deallocate(old_entries, old_size);
+}
+
+// assumes number of entries > nominal size
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::rebuild() {
+  const size_t size = 1 << lg_cur_size_;
+  const uint32_t nominal_size = 1 << lg_nom_size_;
+
+  // empty entries have uninitialized payloads
+  // TODO: avoid this for empty or trivial payloads (arithmetic types)
+  consolidate_non_empty(entries_, size, num_entries_);
+
+  std::nth_element(entries_, entries_ + nominal_size, entries_ + num_entries_, comparator());
+  this->theta_ = EK()(entries_[nominal_size]);
+  EN* old_entries = entries_;
+  entries_ = allocator_.allocate(size);
+  for (size_t i = 0; i < size; ++i) EK()(entries_[i]) = 0;
+  num_entries_ = 0;
+  // relies on consolidating non-empty entries to the front
+  for (size_t i = 0; i < nominal_size; ++i) {
+    insert(find(EK()(old_entries[i])).first, std::move(old_entries[i])); // consider a special insert with no comparison
+    old_entries[i].~EN();
+  }
+  allocator_.deallocate(old_entries, size);
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::trim() {
+  if (num_entries_ > static_cast<uint32_t>(1 << lg_nom_size_)) rebuild();
+}
+
+template<typename EN, typename EK, typename A>
+void theta_update_sketch_base<EN, EK, A>::consolidate_non_empty(EN* entries, size_t size, size_t num) {
+  // find the first empty slot
+  size_t i = 0;
+  while (i < size) {
+    if (EK()(entries[i]) == 0) break;
+    ++i;
+  }
+  // scan the rest and move non-empty entries to the front
+  for (size_t j = i + 1; j < size; ++j) {
+    if (EK()(entries[j]) != 0) {
+      new (&entries[i]) EN(std::move(entries[j]));
+      entries[j].~EN();
+      EK()(entries[j]) = 0;
+      ++i;
+      if (i == num) break;
+    }
+  }
+}
+
+// builder
+
+template<typename Derived, typename Allocator>
+theta_base_builder<Derived, Allocator>::theta_base_builder(const Allocator& allocator):
+allocator_(allocator), lg_k_(DEFAULT_LG_K), rf_(DEFAULT_RESIZE_FACTOR), p_(1), seed_(DEFAULT_SEED) {}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_lg_k(uint8_t lg_k) {
+  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 static_cast<Derived&>(*this);
+}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_resize_factor(resize_factor rf) {
+  rf_ = rf;
+  return static_cast<Derived&>(*this);
+}
+
+template<typename Derived, typename Allocator>
+Derived& theta_base_builder<Derived, Allocator>::set_p(float p) {
+  if (p < 0 || p > 1) throw std::invalid_argument("sampling probability must be between 0 and 1");

Review comment:
       i think we can check for p <= 0 -- if p == 0 then we can never insert anything since we ignore if hash >= theta

##########
File path: tuple/include/array_of_doubles_sketch.hpp
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 ARRAY_OF_DOUBLES_SKETCH_HPP_
+#define ARRAY_OF_DOUBLES_SKETCH_HPP_
+
+#include <vector>
+#include <memory>
+
+#include "serde.hpp"
+#include "tuple_sketch.hpp"
+
+namespace datasketches {
+
+// This sketch is equivalent of ArrayOfDoublesSketch in Java
+
+// This simple array of double is faster than std::vector and should be sufficient for this application
+template<typename Allocator = std::allocator<double>>
+class aod {
+public:
+  explicit aod(uint8_t size, const Allocator& allocator = Allocator()):
+  allocator_(allocator), size_(size), array_(allocator_.allocate(size_)) {
+    std::fill(array_, array_ + size_, 0);
+  }
+  aod(const aod& other):
+    allocator_(other.allocator_),
+    size_(other.size_),
+    array_(allocator_.allocate(size_))
+  {
+    std::copy(other.array_, other.array_ + size_, array_);
+  }
+  aod(aod&& other) noexcept:
+    allocator_(std::move(other.allocator_)),
+    size_(other.size_),
+    array_(other.array_)
+  {
+    other.array_ = nullptr;
+  }
+  ~aod() {
+    if (array_ != nullptr) allocator_.deallocate(array_, size_);
+  }
+  aod& operator=(const aod& other) {
+    aod copy(other);
+    std::swap(allocator_, copy.allocator_);
+    std::swap(size_, copy.size_);
+    std::swap(array_, copy.array_);
+    return *this;
+  }
+  aod& operator=(aod&& other) {
+    std::swap(allocator_, other.allocator_);
+    std::swap(size_, other.size_);
+    std::swap(array_, other.array_);
+    return *this;
+  }
+  double& operator[](size_t index) { return array_[index]; }
+  double operator[](size_t index) const { return array_[index]; }
+  uint8_t size() const { return size_; }
+  double* data() { return array_; }
+  const double* data() const { return array_; }
+  bool operator==(const aod& other) const {
+    for (uint8_t i = 0; i < size_; ++i) if (array_[i] != other.array_[i]) return false;
+    return true;
+  }
+private:
+  Allocator allocator_;
+  uint8_t size_;
+  double* array_;
+};
+
+template<typename A = std::allocator<double>>
+class array_of_doubles_update_policy {
+public:
+  array_of_doubles_update_policy(uint8_t num_values = 1, const A& allocator = A()):
+    allocator_(allocator), num_values_(num_values) {}
+  aod<A> create() const {
+    return aod<A>(num_values_, allocator_);
+  }
+  template<typename InputVector> // to allow any type with indexed access (such as double*)
+  void update(aod<A>& summary, const InputVector& update) const {
+    for (uint8_t i = 0; i < num_values_; ++i) summary[i] += update[i];
+  }
+  uint8_t get_num_values() const {
+    return num_values_;
+  }
+
+private:
+  A allocator_;
+  uint8_t num_values_;
+};
+
+// forward declaration
+template<typename A> class compact_array_of_doubles_sketch_alloc;
+
+template<typename A> using AllocAOD = typename std::allocator_traits<A>::template rebind_alloc<aod<A>>;
+
+template<typename A = std::allocator<double>>
+class update_array_of_doubles_sketch_alloc: public update_tuple_sketch<aod<A>, aod<A>, array_of_doubles_update_policy<A>, AllocAOD<A>> {
+public:
+  using Base = update_tuple_sketch<aod<A>, aod<A>, array_of_doubles_update_policy<A>, AllocAOD<A>>;
+  using resize_factor = typename Base::resize_factor;
+
+  class builder;
+
+  compact_array_of_doubles_sketch_alloc<A> compact(bool ordered = true) const;
+  uint8_t get_num_values() const;
+
+private:
+  // for builder
+  update_array_of_doubles_sketch_alloc(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, uint64_t theta,
+      uint64_t seed, const array_of_doubles_update_policy<A>& policy, const A& allocator);
+};
+
+// alias with the default allocator for convenience
+using update_array_of_doubles_sketch = update_array_of_doubles_sketch_alloc<>;
+
+template<typename A>
+class update_array_of_doubles_sketch_alloc<A>::builder: public tuple_base_builder<builder, array_of_doubles_update_policy<A>, A> {
+public:
+  builder(const array_of_doubles_update_policy<A>& policy = array_of_doubles_update_policy<A>(), const A& allocator = A());
+  update_array_of_doubles_sketch_alloc<A> build() const;
+};
+
+template<typename A = std::allocator<double>>
+class compact_array_of_doubles_sketch_alloc: public compact_tuple_sketch<aod<A>, AllocAOD<A>> {
+public:
+  using Base = compact_tuple_sketch<aod<A>, AllocAOD<A>>;
+  using Entry = typename Base::Entry;
+  using AllocEntry = typename Base::AllocEntry;
+  using AllocU64 = typename Base::AllocU64;
+  using vector_bytes = typename Base::vector_bytes;
+
+  static const uint8_t SERIAL_VERSION = 1;
+  static const uint8_t SKETCH_FAMILY = 9;

Review comment:
       Not thrilled about these being redefined here when they're already in tuple_sketch.  Might be even better if we could have some of these defined in a common file shared across sketches. But not a blocker.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r483800607



##########
File path: tuple/include/theta_intersection_base_impl.hpp
##########
@@ -0,0 +1,121 @@
+/*
+ * 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>
+
+#include "conditional_forward.hpp"
+
+namespace datasketches {
+
+template<typename EN, typename EK, typename P, typename S, typename CS, typename A>
+theta_intersection_base<EN, EK, P, S, CS, A>::theta_intersection_base(uint64_t seed, const P& policy, const A& allocator):
+policy_(policy),
+is_valid_(false),
+table_(0, 0, resize_factor::X1, theta_constants::MAX_THETA, seed, allocator, false)
+{}
+
+template<typename EN, typename EK, typename P, typename S, typename CS, typename A>
+template<typename SS>
+void theta_intersection_base<EN, EK, P, S, CS, A>::update(SS&& sketch) {
+  if (table_.is_empty_) return;
+  if (!sketch.is_empty() && sketch.get_seed_hash() != compute_seed_hash(table_.seed_)) throw std::invalid_argument("seed hash mismatch");
+  table_.is_empty_ |= sketch.is_empty();
+  table_.theta_ = std::min(table_.theta_, sketch.get_theta64());
+  if (is_valid_ && table_.num_entries_ == 0) return;
+  if (sketch.get_num_retained() == 0) {
+    is_valid_ = true;
+    table_ = hash_table(0, 0, resize_factor::X1, table_.theta_, table_.seed_, table_.allocator_, table_.is_empty_);
+    return;
+  }
+  if (!is_valid_) { // first update, copy or move incoming sketch
+    is_valid_ = true;
+    const uint8_t lg_size = lg_size_from_count(sketch.get_num_retained(), theta_update_sketch_base<EN, EK, A>::REBUILD_THRESHOLD);
+    table_ = hash_table(lg_size, lg_size, resize_factor::X1, table_.theta_, table_.seed_, table_.allocator_, table_.is_empty_);
+    for (auto& entry: sketch) {
+      auto result = table_.find(EK()(entry));
+      if (result.second) {
+        throw std::invalid_argument("duplicate key, possibly corrupted input sketch");
+      }
+      table_.insert(result.first, conditional_forward<SS>(entry));
+    }
+    if (table_.num_entries_ != sketch.get_num_retained()) throw std::invalid_argument("num entries mismatch, possibly corrupted input sketch");
+  } else { // intersection
+    const uint32_t max_matches = std::min(table_.num_entries_, sketch.get_num_retained());
+    std::vector<EN, A> matched_entries(table_.allocator_);
+    matched_entries.reserve(max_matches);
+    uint32_t match_count = 0;
+    uint32_t count = 0;
+    for (auto& entry: sketch) {
+      if (EK()(entry) < table_.theta_) {
+        auto result = table_.find(EK()(entry));
+        if (result.second) {
+          if (match_count == max_matches) throw std::invalid_argument("max matches exceeded, possibly corrupted input sketch");
+          policy_(*result.first, conditional_forward<SS>(entry));
+          matched_entries.push_back(std::move(*result.first));
+          ++match_count;
+        }
+      } else if (sketch.is_ordered()) {
+        break; // early stop
+      }
+      ++count;
+    }
+    if (count > sketch.get_num_retained()) {
+      throw std::invalid_argument(" more keys then expected, possibly corrupted input sketch");
+    } else if (!sketch.is_ordered() && count < sketch.get_num_retained()) {
+      throw std::invalid_argument(" fewer keys then expected, possibly corrupted input sketch");

Review comment:
       fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] jmalkin commented on pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
jmalkin commented on pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#issuecomment-705931667


   The last few edits look good. Merge away!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov merged pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov merged pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [incubator-datasketches-cpp] AlexanderSaydakov commented on a change in pull request #170: Tuple sketch

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #170:
URL: https://github.com/apache/incubator-datasketches-cpp/pull/170#discussion_r501938725



##########
File path: tuple/include/array_of_doubles_sketch.hpp
##########
@@ -0,0 +1,179 @@
+/*
+ * 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 ARRAY_OF_DOUBLES_SKETCH_HPP_
+#define ARRAY_OF_DOUBLES_SKETCH_HPP_
+
+#include <vector>
+#include <memory>
+
+#include "serde.hpp"
+#include "tuple_sketch.hpp"
+
+namespace datasketches {
+
+// This sketch is equivalent of ArrayOfDoublesSketch in Java
+
+// This simple array of double is faster than std::vector and should be sufficient for this application
+template<typename Allocator = std::allocator<double>>
+class aod {
+public:
+  explicit aod(uint8_t size, const Allocator& allocator = Allocator()):
+  allocator_(allocator), size_(size), array_(allocator_.allocate(size_)) {
+    std::fill(array_, array_ + size_, 0);
+  }
+  aod(const aod& other):
+    allocator_(other.allocator_),
+    size_(other.size_),
+    array_(allocator_.allocate(size_))
+  {
+    std::copy(other.array_, other.array_ + size_, array_);
+  }
+  aod(aod&& other) noexcept:
+    allocator_(std::move(other.allocator_)),
+    size_(other.size_),
+    array_(other.array_)
+  {
+    other.array_ = nullptr;
+  }
+  ~aod() {
+    if (array_ != nullptr) allocator_.deallocate(array_, size_);
+  }
+  aod& operator=(const aod& other) {
+    aod copy(other);
+    std::swap(allocator_, copy.allocator_);
+    std::swap(size_, copy.size_);
+    std::swap(array_, copy.array_);
+    return *this;
+  }
+  aod& operator=(aod&& other) {
+    std::swap(allocator_, other.allocator_);
+    std::swap(size_, other.size_);
+    std::swap(array_, other.array_);
+    return *this;
+  }
+  double& operator[](size_t index) { return array_[index]; }
+  double operator[](size_t index) const { return array_[index]; }
+  uint8_t size() const { return size_; }
+  double* data() { return array_; }
+  const double* data() const { return array_; }
+  bool operator==(const aod& other) const {
+    for (uint8_t i = 0; i < size_; ++i) if (array_[i] != other.array_[i]) return false;
+    return true;
+  }
+private:
+  Allocator allocator_;
+  uint8_t size_;
+  double* array_;
+};
+
+template<typename A = std::allocator<double>>
+class array_of_doubles_update_policy {
+public:
+  array_of_doubles_update_policy(uint8_t num_values = 1, const A& allocator = A()):
+    allocator_(allocator), num_values_(num_values) {}
+  aod<A> create() const {
+    return aod<A>(num_values_, allocator_);
+  }
+  template<typename InputVector> // to allow any type with indexed access (such as double*)
+  void update(aod<A>& summary, const InputVector& update) const {
+    for (uint8_t i = 0; i < num_values_; ++i) summary[i] += update[i];
+  }
+  uint8_t get_num_values() const {
+    return num_values_;
+  }
+
+private:
+  A allocator_;
+  uint8_t num_values_;
+};
+
+// forward declaration
+template<typename A> class compact_array_of_doubles_sketch_alloc;
+
+template<typename A> using AllocAOD = typename std::allocator_traits<A>::template rebind_alloc<aod<A>>;
+
+template<typename A = std::allocator<double>>
+class update_array_of_doubles_sketch_alloc: public update_tuple_sketch<aod<A>, aod<A>, array_of_doubles_update_policy<A>, AllocAOD<A>> {
+public:
+  using Base = update_tuple_sketch<aod<A>, aod<A>, array_of_doubles_update_policy<A>, AllocAOD<A>>;
+  using resize_factor = typename Base::resize_factor;
+
+  class builder;
+
+  compact_array_of_doubles_sketch_alloc<A> compact(bool ordered = true) const;
+  uint8_t get_num_values() const;
+
+private:
+  // for builder
+  update_array_of_doubles_sketch_alloc(uint8_t lg_cur_size, uint8_t lg_nom_size, resize_factor rf, uint64_t theta,
+      uint64_t seed, const array_of_doubles_update_policy<A>& policy, const A& allocator);
+};
+
+// alias with the default allocator for convenience
+using update_array_of_doubles_sketch = update_array_of_doubles_sketch_alloc<>;
+
+template<typename A>
+class update_array_of_doubles_sketch_alloc<A>::builder: public tuple_base_builder<builder, array_of_doubles_update_policy<A>, A> {
+public:
+  builder(const array_of_doubles_update_policy<A>& policy = array_of_doubles_update_policy<A>(), const A& allocator = A());
+  update_array_of_doubles_sketch_alloc<A> build() const;
+};
+
+template<typename A = std::allocator<double>>
+class compact_array_of_doubles_sketch_alloc: public compact_tuple_sketch<aod<A>, AllocAOD<A>> {
+public:
+  using Base = compact_tuple_sketch<aod<A>, AllocAOD<A>>;
+  using Entry = typename Base::Entry;
+  using AllocEntry = typename Base::AllocEntry;
+  using AllocU64 = typename Base::AllocU64;
+  using vector_bytes = typename Base::vector_bytes;
+
+  static const uint8_t SERIAL_VERSION = 1;
+  static const uint8_t SKETCH_FAMILY = 9;

Review comment:
       This is a different family and different serial format. I don't really like this concept, but it serves as a compatibility check.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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