You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jo...@apache.org on 2020/04/24 16:45:11 UTC

[impala] 01/02: IMPALA-9631: Import HLL functionality from DataSketches

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

joemcdonnell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 9a3108709bf9cc11f00b97e966d1514bd59b34c8
Author: Gabor Kaszab <ga...@cloudera.com>
AuthorDate: Fri Apr 17 13:48:24 2020 +0200

    IMPALA-9631: Import HLL functionality from DataSketches
    
    This patch imports the functionality needed for HLL approximate
    algorithm from Apache DataSketches. I decided to copy the necessary
    files into be/src/thirdparty/datasketches. Note, that the original
    structure of files was changed during this process as originally hll/
    and common/ libraries were both affected but I copied these into the
    same directory so that Impala can compile them without rewriting the
    include paths in the files themselves. Also note, that not the whole
    common/ directory was copied just the files needed for HLL.
    
    The git hash of the snapshot I used as a source for the files:
    a6265b307a03085abe26c20413fdbf7d7a5eaf29
    
    Browse the source files here:
    https://github.com/apache/incubator-datasketches-cpp
    
    Change-Id: I8ca8e77dcbb6b6c3b1e3bca7ab57cb7d3c018bbf
    Reviewed-on: http://gerrit.cloudera.org:8080/15746
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exprs/CMakeLists.txt                        |   2 +
 be/src/exprs/datasketches-test.cc                  |  74 ++
 .../datasketches/AuxHashMap-internal.hpp           | 301 ++++++++
 be/src/thirdparty/datasketches/AuxHashMap.hpp      |  83 +++
 be/src/thirdparty/datasketches/CommonUtil.hpp      |  71 ++
 .../CompositeInterpolationXTable-internal.hpp      | 811 +++++++++++++++++++++
 .../datasketches/CompositeInterpolationXTable.hpp  |  40 +
 .../datasketches/CouponHashSet-internal.hpp        | 290 ++++++++
 be/src/thirdparty/datasketches/CouponHashSet.hpp   |  59 ++
 .../datasketches/CouponList-internal.hpp           | 413 +++++++++++
 be/src/thirdparty/datasketches/CouponList.hpp      |  91 +++
 .../datasketches/CubicInterpolation-internal.hpp   | 233 ++++++
 .../thirdparty/datasketches/CubicInterpolation.hpp |  43 ++
 .../datasketches/HarmonicNumbers-internal.hpp      |  90 +++
 be/src/thirdparty/datasketches/HarmonicNumbers.hpp |  48 ++
 .../thirdparty/datasketches/Hll4Array-internal.hpp | 335 +++++++++
 be/src/thirdparty/datasketches/Hll4Array.hpp       |  69 ++
 .../thirdparty/datasketches/Hll6Array-internal.hpp | 124 ++++
 be/src/thirdparty/datasketches/Hll6Array.hpp       |  55 ++
 .../thirdparty/datasketches/Hll8Array-internal.hpp | 158 ++++
 be/src/thirdparty/datasketches/Hll8Array.hpp       |  56 ++
 .../thirdparty/datasketches/HllArray-internal.hpp  | 698 ++++++++++++++++++
 be/src/thirdparty/datasketches/HllArray.hpp        | 136 ++++
 .../thirdparty/datasketches/HllSketch-internal.hpp | 472 ++++++++++++
 .../datasketches/HllSketchImpl-internal.hpp        | 149 ++++
 be/src/thirdparty/datasketches/HllSketchImpl.hpp   |  85 +++
 .../datasketches/HllSketchImplFactory.hpp          | 171 +++++
 .../thirdparty/datasketches/HllUnion-internal.hpp  | 351 +++++++++
 be/src/thirdparty/datasketches/HllUtil.hpp         | 289 ++++++++
 be/src/thirdparty/datasketches/LICENSE             | 230 ++++++
 be/src/thirdparty/datasketches/MurmurHash3.h       | 178 +++++
 be/src/thirdparty/datasketches/README.md           |  15 +
 .../datasketches/RelativeErrorTables-internal.hpp  | 112 +++
 .../datasketches/RelativeErrorTables.hpp           |  46 ++
 .../datasketches/coupon_iterator-internal.hpp      |  56 ++
 be/src/thirdparty/datasketches/coupon_iterator.hpp |  43 ++
 be/src/thirdparty/datasketches/hll.hpp             | 761 +++++++++++++++++++
 be/src/thirdparty/datasketches/hll.private.hpp     |  32 +
 be/src/thirdparty/datasketches/inv_pow2_table.hpp  | 107 +++
 bin/rat_exclude_files.txt                          |   1 +
 bin/run_clang_tidy.sh                              |   5 +-
 41 files changed, 7381 insertions(+), 2 deletions(-)

diff --git a/be/src/exprs/CMakeLists.txt b/be/src/exprs/CMakeLists.txt
index b80d77c..92b1fc6 100644
--- a/be/src/exprs/CMakeLists.txt
+++ b/be/src/exprs/CMakeLists.txt
@@ -72,11 +72,13 @@ add_library(Exprs
 add_dependencies(Exprs gen-deps gen_ir_descriptions)
 
 add_library(ExprsTests STATIC
+  datasketches-test.cc
   expr-test.cc
   timezone_db-test.cc
 )
 add_dependencies(ExprsTests gen-deps)
 
+ADD_UNIFIED_BE_LSAN_TEST(datasketches-test "TestDataSketchesHll.*")
 ADD_UNIFIED_BE_LSAN_TEST(expr-test "Instantiations/ExprTest.*")
 # Exception to unified be tests: custom main initiailizes LLVM
 ADD_BE_LSAN_TEST(expr-codegen-test)
diff --git a/be/src/exprs/datasketches-test.cc b/be/src/exprs/datasketches-test.cc
new file mode 100644
index 0000000..bafeb8a
--- /dev/null
+++ b/be/src/exprs/datasketches-test.cc
@@ -0,0 +1,74 @@
+// 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 "thirdparty/datasketches/hll.hpp"
+
+#include <fstream>
+#include <iostream>
+
+#include "testutil/gtest-util.h"
+
+namespace impala {
+
+// This test is meant to cover that the HLL algorithm from the DataSketches library can
+// be imported into Impala, builds without errors and the basic functionality is
+// available to use.
+// The below code is mostly a copy-paste from the example code found on the official
+// DataSketches web page: https://datasketches.apache.org/docs/HLL/HllCppExample.html
+// The purpose is to create 2 HLL sketches that have overlap in their data, serialize
+// these sketches into files, deserialize them and give a cardinality estimate combining
+// the 2 sketches.
+TEST(TestDataSketchesHll, UseDataSketchesInterface) {
+  const int lg_k = 11;
+  const auto type = datasketches::HLL_4;
+  std::stringstream sketch_stream1;
+  std::stringstream sketch_stream2;
+  // This section generates two sketches with some overlap and serializes them into files
+  {
+    // 100000 distinct keys
+    datasketches::hll_sketch sketch1(lg_k, type);
+    for (int key = 0; key < 100000; key++) sketch1.update(key);
+    sketch1.serialize_compact(sketch_stream1);
+
+    // 100000 distinct keys where 50000 overlaps with sketch1
+    datasketches::hll_sketch sketch2(lg_k, type);
+    for (int key = 50000; key < 150000; key++) sketch2.update(key);
+    sketch2.serialize_compact(sketch_stream2);
+  }
+
+  // This section deserializes the sketches and produces union
+  {
+    datasketches::hll_sketch sketch1 =
+        datasketches::hll_sketch::deserialize(sketch_stream1);
+    datasketches::hll_sketch sketch2 =
+        datasketches::hll_sketch::deserialize(sketch_stream2);
+
+    datasketches::hll_union union_sketch(lg_k);
+    union_sketch.update(sketch1);
+    union_sketch.update(sketch2);
+    datasketches::hll_sketch sketch = union_sketch.get_result(type);
+
+    // Approximate result should be in the range of 1.5% to the accurate number. Picked
+    // this threshold to be on the safe side and to make sure that this test won't start
+    // failing once in a while.
+    int accurate_result = 150000;
+    int error_range = accurate_result * 0.015;
+    EXPECT_LE(std::abs(sketch.get_estimate() - accurate_result), error_range);
+  }
+}
+
+}
diff --git a/be/src/thirdparty/datasketches/AuxHashMap-internal.hpp b/be/src/thirdparty/datasketches/AuxHashMap-internal.hpp
new file mode 100644
index 0000000..0d7db7a
--- /dev/null
+++ b/be/src/thirdparty/datasketches/AuxHashMap-internal.hpp
@@ -0,0 +1,301 @@
+/*
+ * 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 _AUXHASHMAP_INTERNAL_HPP_
+#define _AUXHASHMAP_INTERNAL_HPP_
+
+#include "HllUtil.hpp"
+#include "AuxHashMap.hpp"
+
+namespace datasketches {
+
+template<typename A>
+AuxHashMap<A>::AuxHashMap(int lgAuxArrInts, int lgConfigK)
+  : lgConfigK(lgConfigK),
+    lgAuxArrInts(lgAuxArrInts),
+    auxCount(0) {
+  typedef typename std::allocator_traits<A>::template rebind_alloc<int> intAlloc;
+  const int numItems = 1 << lgAuxArrInts;
+  auxIntArr = intAlloc().allocate(numItems);
+  std::fill(auxIntArr, auxIntArr + numItems, 0);
+}
+
+template<typename A>
+AuxHashMap<A>* AuxHashMap<A>::newAuxHashMap(int lgAuxArrInts, int lgConfigK) {
+  return new (ahmAlloc().allocate(1)) AuxHashMap<A>(lgAuxArrInts, lgConfigK);
+}
+
+template<typename A>
+AuxHashMap<A>::AuxHashMap(const AuxHashMap& that)
+  : lgConfigK(that.lgConfigK),
+    lgAuxArrInts(that.lgAuxArrInts),
+    auxCount(that.auxCount) {
+  typedef typename std::allocator_traits<A>::template rebind_alloc<int> intAlloc;
+  const int numItems = 1 << lgAuxArrInts;
+  auxIntArr = intAlloc().allocate(numItems);
+  std::copy(that.auxIntArr, that.auxIntArr + numItems, auxIntArr);
+}
+
+template<typename A>
+AuxHashMap<A>* AuxHashMap<A>::newAuxHashMap(const AuxHashMap& that) {
+  return new (ahmAlloc().allocate(1)) AuxHashMap<A>(that);
+}
+
+template<typename A>
+AuxHashMap<A>* AuxHashMap<A>::deserialize(const void* bytes, size_t len,
+                                          int lgConfigK,
+                                          int auxCount, int lgAuxArrInts,
+                                          bool srcCompact) {
+  int lgArrInts = lgAuxArrInts;
+  if (srcCompact) { // early compact versions didn't use LgArr byte field so ignore input
+    lgArrInts = HllUtil<A>::computeLgArrInts(HLL, auxCount, lgConfigK);
+  } else { // updatable
+    lgArrInts = lgAuxArrInts;
+  }
+  
+  int configKmask = (1 << lgConfigK) - 1;
+
+  AuxHashMap<A>* auxHashMap;
+  const int* auxPtr = static_cast<const int*>(bytes);
+  if (srcCompact) {
+    if (len < auxCount * sizeof(int)) {
+      throw std::invalid_argument("Input array too small to hold AuxHashMap image");
+    }
+    auxHashMap = new (ahmAlloc().allocate(1)) AuxHashMap<A>(lgArrInts, lgConfigK);
+    for (int i = 0; i < auxCount; ++i) {
+      int pair = auxPtr[i];
+      int slotNo = HllUtil<A>::getLow26(pair) & configKmask;
+      int value = HllUtil<A>::getValue(pair);
+      auxHashMap->mustAdd(slotNo, value);
+    }
+  } else { // updatable
+    int itemsToRead = 1 << lgAuxArrInts;
+    if (len < itemsToRead * sizeof(int)) {
+      throw std::invalid_argument("Input array too small to hold AuxHashMap image");
+    }
+    auxHashMap = new (ahmAlloc().allocate(1)) AuxHashMap<A>(lgArrInts, lgConfigK);
+    for (int i = 0; i < itemsToRead; ++i) {
+      int pair = auxPtr[i];
+      if (pair == HllUtil<A>::EMPTY) { continue; }
+      int slotNo = HllUtil<A>::getLow26(pair) & configKmask;
+      int value = HllUtil<A>::getValue(pair);
+      auxHashMap->mustAdd(slotNo, value);
+    }
+  }
+
+  if (auxHashMap->getAuxCount() != auxCount) {
+    make_deleter()(auxHashMap);
+    throw std::invalid_argument("Deserialized AuxHashMap has wrong number of entries");
+  }
+
+  return auxHashMap;                                    
+}
+
+template<typename A>
+AuxHashMap<A>* AuxHashMap<A>::deserialize(std::istream& is, const int lgConfigK,
+                                          const int auxCount, const int lgAuxArrInts,
+                                          const bool srcCompact) {
+  int lgArrInts = lgAuxArrInts;
+  if (srcCompact) { // early compact versions didn't use LgArr byte field so ignore input
+    lgArrInts = HllUtil<A>::computeLgArrInts(HLL, auxCount, lgConfigK);
+  } else { // updatable
+    lgArrInts = lgAuxArrInts;
+  }
+
+  // TODO: truncated stream will throw exception without freeing memory  
+  AuxHashMap<A>* auxHashMap = new (ahmAlloc().allocate(1)) AuxHashMap<A>(lgArrInts, lgConfigK);
+  int configKmask = (1 << lgConfigK) - 1;
+
+  if (srcCompact) {
+    int pair;
+    for (int i = 0; i < auxCount; ++i) {
+      is.read((char*)&pair, sizeof(pair));
+      int slotNo = HllUtil<A>::getLow26(pair) & configKmask;
+      int value = HllUtil<A>::getValue(pair);
+      auxHashMap->mustAdd(slotNo, value);
+    }
+  } else { // updatable
+    int itemsToRead = 1 << lgAuxArrInts;
+    int pair;
+    for (int i = 0; i < itemsToRead; ++i) {
+      is.read((char*)&pair, sizeof(pair));
+      if (pair == HllUtil<A>::EMPTY) { continue; }
+      int slotNo = HllUtil<A>::getLow26(pair) & configKmask;
+      int value = HllUtil<A>::getValue(pair);
+      auxHashMap->mustAdd(slotNo, value);
+    }
+  }
+
+  if (auxHashMap->getAuxCount() != auxCount) {
+    make_deleter()(auxHashMap);
+    throw std::invalid_argument("Deserialized AuxHashMap has wrong number of entries");
+  }
+
+  return auxHashMap;
+}
+
+template<typename A>
+AuxHashMap<A>::~AuxHashMap<A>() {
+  // should be no way to have an object without a valid array
+  typedef typename std::allocator_traits<A>::template rebind_alloc<int> intAlloc;
+  intAlloc().deallocate(auxIntArr, 1 << lgAuxArrInts);
+}
+
+template<typename A>
+std::function<void(AuxHashMap<A>*)> AuxHashMap<A>::make_deleter() {
+  return [](AuxHashMap<A>* ptr) {    
+    ptr->~AuxHashMap();
+    ahmAlloc().deallocate(ptr, 1);
+  };
+}
+
+template<typename A>
+AuxHashMap<A>* AuxHashMap<A>::copy() const {
+  return new (ahmAlloc().allocate(1)) AuxHashMap<A>(*this);
+}
+
+template<typename A>
+int AuxHashMap<A>::getAuxCount() const {
+  return auxCount;
+}
+
+template<typename A>
+int* AuxHashMap<A>::getAuxIntArr(){
+  return auxIntArr;
+}
+
+template<typename A>
+int AuxHashMap<A>::getLgAuxArrInts() const {
+  return lgAuxArrInts;
+}
+
+template<typename A>
+int AuxHashMap<A>::getCompactSizeBytes() const {
+  return auxCount << 2;
+}
+
+template<typename A>
+int AuxHashMap<A>::getUpdatableSizeBytes() const {
+  return 4 << lgAuxArrInts;
+}
+
+template<typename A>
+void AuxHashMap<A>::mustAdd(const int slotNo, const int value) {
+  const int index = find(auxIntArr, lgAuxArrInts, lgConfigK, slotNo);
+  const int entry_pair = HllUtil<A>::pair(slotNo, value);
+  if (index >= 0) {
+    throw std::invalid_argument("Found a slotNo that should not be there: SlotNo: "
+                                + std::to_string(slotNo) + ", Value: " + std::to_string(value));
+  }
+
+  // found empty entry
+  auxIntArr[~index] = entry_pair;
+  ++auxCount;
+  checkGrow();
+}
+
+template<typename A>
+int AuxHashMap<A>::mustFindValueFor(const int slotNo) const {
+  const int index = find(auxIntArr, lgAuxArrInts, lgConfigK, slotNo);
+  if (index >= 0) {
+    return HllUtil<A>::getValue(auxIntArr[index]);
+  }
+
+  throw std::invalid_argument("slotNo not found: " + std::to_string(slotNo));
+}
+
+template<typename A>
+void AuxHashMap<A>::mustReplace(const int slotNo, const int value) {
+  const int idx = find(auxIntArr, lgAuxArrInts, lgConfigK, slotNo);
+  if (idx >= 0) {
+    auxIntArr[idx] = HllUtil<A>::pair(slotNo, value);
+    return;
+  }
+
+  throw std::invalid_argument("Pair not found: SlotNo: " + std::to_string(slotNo)
+                              + ", Value: " + std::to_string(value));
+}
+
+template<typename A>
+void AuxHashMap<A>::checkGrow() {
+  if ((HllUtil<A>::RESIZE_DENOM * auxCount) > (HllUtil<A>::RESIZE_NUMER * (1 << lgAuxArrInts))) {
+    growAuxSpace();
+  }
+}
+
+template<typename A>
+void AuxHashMap<A>::growAuxSpace() {
+  int* oldArray = auxIntArr;
+  const int oldArrLen = 1 << lgAuxArrInts;
+  const int configKmask = (1 << lgConfigK) - 1;
+  const int newArrLen = 1 << ++lgAuxArrInts;
+  typedef typename std::allocator_traits<A>::template rebind_alloc<int> intAlloc;
+  auxIntArr = intAlloc().allocate(newArrLen);
+  std::fill(auxIntArr, auxIntArr + newArrLen, 0);
+  for (int i = 0; i < oldArrLen; ++i) {
+    const int fetched = oldArray[i];
+    if (fetched != HllUtil<A>::EMPTY) {
+      // find empty in new array
+      const int idx = find(auxIntArr, lgAuxArrInts, lgConfigK, fetched & configKmask);
+      auxIntArr[~idx] = fetched;
+    }
+  }
+
+  intAlloc().deallocate(oldArray, oldArrLen);
+}
+
+//Searches the Aux arr hash table for an empty or a matching slotNo depending on the context.
+//If entire entry is empty, returns one's complement of index = found empty.
+//If entry contains given slotNo, returns its index = found slotNo.
+//Continues searching.
+//If the probe comes back to original index, throws an exception.
+template<typename A>
+int AuxHashMap<A>::find(const int* auxArr, const int lgAuxArrInts, const int lgConfigK,
+                        const int slotNo) {
+  const int auxArrMask = (1 << lgAuxArrInts) - 1;
+  const int configKmask = (1 << lgConfigK) - 1;
+  int probe = slotNo & auxArrMask;
+  const  int loopIndex = probe;
+  do {
+    const int arrVal = auxArr[probe];
+    if (arrVal == HllUtil<A>::EMPTY) { //Compares on entire entry
+      return ~probe; //empty
+    }
+    else if (slotNo == (arrVal & configKmask)) { //Compares only on slotNo
+      return probe; //found given slotNo, return probe = index into aux array
+    }
+    const int stride = (slotNo >> lgAuxArrInts) | 1;
+    probe = (probe + stride) & auxArrMask;
+  } while (probe != loopIndex);
+  throw std::runtime_error("Key not found and no empty slots!");
+}
+
+template<typename A>
+coupon_iterator<A> AuxHashMap<A>::begin(bool all) const {
+  return coupon_iterator<A>(auxIntArr, 1 << lgAuxArrInts, 0, all);
+}
+
+template<typename A>
+coupon_iterator<A> AuxHashMap<A>::end() const {
+  return coupon_iterator<A>(auxIntArr, 1 << lgAuxArrInts, 1 << lgAuxArrInts, false);
+}
+
+}
+
+#endif // _AUXHASHMAP_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/AuxHashMap.hpp b/be/src/thirdparty/datasketches/AuxHashMap.hpp
new file mode 100644
index 0000000..b37e85c
--- /dev/null
+++ b/be/src/thirdparty/datasketches/AuxHashMap.hpp
@@ -0,0 +1,83 @@
+/*
+ * 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 _AUXHASHMAP_HPP_
+#define _AUXHASHMAP_HPP_
+
+#include <iostream>
+#include <memory>
+#include <functional>
+
+#include "coupon_iterator.hpp"
+
+namespace datasketches {
+
+template<typename A = std::allocator<char>>
+class AuxHashMap final {
+  public:
+    explicit AuxHashMap(int lgAuxArrInts, int lgConfigK);
+    explicit AuxHashMap(const AuxHashMap<A>& that);
+    static AuxHashMap* newAuxHashMap(int lgAuxArrInts, int lgConfigK);
+    static AuxHashMap* newAuxHashMap(const AuxHashMap<A>& that);
+
+    static AuxHashMap* deserialize(const void* bytes, size_t len,
+                                   int lgConfigK,
+                                   int auxCount, int lgAuxArrInts,
+                                   bool srcCompact);
+    static AuxHashMap* deserialize(std::istream& is, int lgConfigK,
+                                   int auxCount, int lgAuxArrInts,
+                                   bool srcCompact);
+    virtual ~AuxHashMap();
+    static std::function<void(AuxHashMap<A>*)> make_deleter();
+    
+    AuxHashMap* copy() const;
+    int getUpdatableSizeBytes() const;
+    int getCompactSizeBytes() const;
+
+    int getAuxCount() const;
+    int* getAuxIntArr();
+    int getLgAuxArrInts() const;
+
+    coupon_iterator<A> begin(bool all = false) const;
+    coupon_iterator<A> end() const;
+
+    void mustAdd(int slotNo, int value);
+    int mustFindValueFor(int slotNo) const;
+    void mustReplace(int slotNo, int value);
+
+  private:
+    typedef typename std::allocator_traits<A>::template rebind_alloc<AuxHashMap<A>> ahmAlloc;
+
+    // static so it can be used when resizing
+    static int find(const int* auxArr, int lgAuxArrInts, int lgConfigK, int slotNo);
+
+    void checkGrow();
+    void growAuxSpace();
+
+    const int lgConfigK;
+    int lgAuxArrInts;
+    int auxCount;
+    int* auxIntArr;
+};
+
+}
+
+#include "AuxHashMap-internal.hpp"
+
+#endif /* _AUXHASHMAP_HPP_ */
diff --git a/be/src/thirdparty/datasketches/CommonUtil.hpp b/be/src/thirdparty/datasketches/CommonUtil.hpp
new file mode 100644
index 0000000..55e7dfc
--- /dev/null
+++ b/be/src/thirdparty/datasketches/CommonUtil.hpp
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+
+// author Kevin Lang, Yahoo Research
+// author Jon Malkin, Yahoo Research
+
+#ifndef _COMMONUTIL_HPP_
+#define _COMMONUTIL_HPP_
+
+#include <cstdint>
+
+namespace datasketches {
+
+class CommonUtil final {
+  public:
+    static unsigned int getNumberOfLeadingZeros(uint64_t x);
+};
+
+#define FCLZ_MASK_56 ((uint64_t) 0x00ffffffffffffff)
+#define FCLZ_MASK_48 ((uint64_t) 0x0000ffffffffffff)
+#define FCLZ_MASK_40 ((uint64_t) 0x000000ffffffffff)
+#define FCLZ_MASK_32 ((uint64_t) 0x00000000ffffffff)
+#define FCLZ_MASK_24 ((uint64_t) 0x0000000000ffffff)
+#define FCLZ_MASK_16 ((uint64_t) 0x000000000000ffff)
+#define FCLZ_MASK_08 ((uint64_t) 0x00000000000000ff)
+
+inline unsigned int CommonUtil::getNumberOfLeadingZeros(const uint64_t x) {
+  if (x == 0)
+    return 64;
+
+  static const uint8_t clzByteCount[256] = {8,7,6,6,5,5,5,5,4,4,4,4,4,4,4,4,3,3,3,3,3,3,3,3,3,3,3,3,3,3,3,3,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, [...]
+
+  if (x > FCLZ_MASK_56)
+    return ((unsigned int) ( 0 + clzByteCount[(x >> 56) & FCLZ_MASK_08]));
+  if (x > FCLZ_MASK_48)
+    return ((unsigned int) ( 8 + clzByteCount[(x >> 48) & FCLZ_MASK_08]));
+  if (x > FCLZ_MASK_40)
+    return ((unsigned int) (16 + clzByteCount[(x >> 40) & FCLZ_MASK_08]));
+  if (x > FCLZ_MASK_32)
+    return ((unsigned int) (24 + clzByteCount[(x >> 32) & FCLZ_MASK_08]));
+  if (x > FCLZ_MASK_24)
+    return ((unsigned int) (32 + clzByteCount[(x >> 24) & FCLZ_MASK_08]));
+  if (x > FCLZ_MASK_16)
+    return ((unsigned int) (40 + clzByteCount[(x >> 16) & FCLZ_MASK_08]));
+  if (x > FCLZ_MASK_08)
+    return ((unsigned int) (48 + clzByteCount[(x >>  8) & FCLZ_MASK_08]));
+  if (1)
+    return ((unsigned int) (56 + clzByteCount[(x >>  0) & FCLZ_MASK_08]));
+
+}
+
+
+}
+
+#endif // _COMMONUTIL_HPP_
\ No newline at end of file
diff --git a/be/src/thirdparty/datasketches/CompositeInterpolationXTable-internal.hpp b/be/src/thirdparty/datasketches/CompositeInterpolationXTable-internal.hpp
new file mode 100644
index 0000000..a3d0302
--- /dev/null
+++ b/be/src/thirdparty/datasketches/CompositeInterpolationXTable-internal.hpp
@@ -0,0 +1,811 @@
+/*
+ * 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 _COMPOSITEINTERPOLATIONXTABLE_INTERNAL_HPP_
+#define _COMPOSITEINTERPOLATIONXTABLE_INTERNAL_HPP_
+
+#include "HllUtil.hpp"
+#include "CompositeInterpolationXTable.hpp"
+
+#include <exception>
+
+namespace datasketches {
+
+static const int numXArrValues = 257;
+
+/**
+ * 18 Values, index 0 is LgK = 4, index 17 is LgK = 21.
+ */
+static const int yStrides[] =
+  {1, 2, 3, 5, 10, 20, 40, 80, 160, 320, 640, 1280, 2560, 5120, 10240, 20480, 40960, 81920};
+
+template<typename A>
+const int CompositeInterpolationXTable<A>::get_y_stride(const int logK) {
+  if (logK < HllUtil<A>::MIN_LOG_K || logK > HllUtil<A>::MAX_LOG_K) {
+    throw std::invalid_argument("logK must be in range [" + std::to_string(HllUtil<A>::MIN_LOG_K)
+                                + ", " + std::to_string(HllUtil<A>::MAX_LOG_K) + "]. Found: "
+                                + std::to_string(logK));
+  }
+  return yStrides[logK - HllUtil<A>::MIN_LOG_K];
+}
+
+template<typename A>
+const int CompositeInterpolationXTable<A>::get_x_arr_length(const int logK) {
+  return numXArrValues;
+}
+
+static const double xArr[18][numXArrValues] = {
+{
+  10.767999803534, 11.237701481774, 11.722738717438, 12.223246391222,
+  12.739366773787, 13.271184824495, 13.818759686650, 14.382159835785,
+  14.961390904922, 15.556414447178, 16.167227058768, 16.793705840034,
+  17.435831011559, 18.093368824077, 18.766214022468, 19.454114555153,
+  20.156877376380, 20.874309118151, 21.606085061388, 22.351926834624,
+  23.111406651437, 23.884222842419, 24.669953053285, 25.468324695415,
+  26.278740385032, 27.100807701976, 27.934264623663, 28.778498012717,
+  29.633064579479, 30.497559091830, 31.371415928956, 32.254248998304,
+  33.145649456017, 34.045084140394, 34.952069032314, 35.866210502243,
+  36.787128772113, 37.714316067007, 38.647432342061, 39.586020708848,
+  40.529738186256, 41.478377527525, 42.431435504179, 43.388619067416,
+  44.349650659292, 45.314111448463, 46.281805382722, 47.252472240229,
+  48.225969481651, 49.202042512440, 50.180350422249, 51.160761480664,
+  52.143032445587, 53.127233235412, 54.112986437616, 55.100182392694,
+  56.088558030701, 57.078156023607, 58.068788167275, 59.060493030644,
+  60.053033784417, 61.046284072738, 62.040286486621, 63.034929032992,
+  64.030027371399, 65.025588917110, 66.021614945599, 67.018177418440,
+  68.015186786528, 69.012467972526, 70.010014322660, 71.007638150468,
+  72.005472777573, 73.003558069130, 74.001925061314, 75.000323809106,
+  75.998875046818, 76.997735309722, 77.996356820987, 78.995236661037,
+  79.994185446564, 80.993179953574, 81.991936172868, 82.991277726952,
+  83.990564675011, 84.990016215330, 85.989354175963, 86.988664031183,
+  87.988038383374, 88.987487931153, 89.987011275632, 90.986376635228,
+  91.985802667485, 92.985125933965, 93.984894049744, 94.984597185523,
+  95.984374633793, 96.983987641198, 97.983728001025, 98.983151114601,
+  99.982917278162, 100.982752268263, 101.982610588914, 102.982425129077,
+  103.982425883214, 104.982193814993, 105.982188025268, 106.982005912713,
+  107.981949640254, 108.981777381902, 109.981767708570, 110.981785341504,
+  111.981686149156, 112.981635012054, 113.981557303839, 114.981412817204,
+  115.981636136290, 116.981451847734, 117.981619278961, 118.981656205677,
+  119.981462628717, 120.981648197593, 121.981672839888, 122.981536601882,
+  123.981424780957, 124.981245233627, 125.981297203177, 126.981469882453,
+  127.981420558510, 128.981470958251, 129.981803124940, 130.981454077005,
+  131.981307772677, 132.981136552712, 133.981060440468, 134.980764897921,
+  135.980250123519, 136.979947889981, 137.979778713731, 138.979651075854,
+  139.979521006237, 140.979619490143, 141.979418214464, 142.979378742690,
+  143.979006714929, 144.978789085341, 145.978940618746, 146.978699584620,
+  147.978509489139, 148.978375065557, 149.978382007245, 150.978320879037,
+  151.978431501891, 152.978357310224, 153.978079953131, 154.977867908315,
+  155.977897282267, 156.977732141540, 157.977285841428, 158.977381755450,
+  159.977107808858, 160.976898092463, 161.976945065052, 162.976415653211,
+  163.976063254486, 164.976168718176, 165.976009634854, 166.975746910144,
+  167.975365498336, 168.975302031315, 169.975145815395, 170.975048401084,
+  171.974904933990, 172.974962856090, 173.974307743427, 174.974505556696,
+  175.974404122880, 176.974154794798, 177.973936891006, 178.973909768514,
+  179.973629325330, 180.973396428571, 181.973041798012, 182.972922327014,
+  183.972453839631, 184.972389341638, 185.971978290400, 186.971721489488,
+  187.971610992921, 188.971432541974, 189.970886522529, 190.970636748690,
+  191.970543849249, 192.970222044828, 193.969945219163, 194.969816174441,
+  195.969660267113, 196.969306217626, 197.969283141670, 198.968952697752,
+  199.968650726389, 200.968771498143, 201.969063388798, 202.968624157712,
+  203.968385122471, 204.968459873029, 205.968253426841, 206.968128284823,
+  207.968002441937, 208.967981887571, 209.967853843352, 210.967653026858,
+  211.967601511385, 212.967479077297, 213.967293002672, 214.967265475369,
+  215.967098197208, 216.966915139758, 217.966680984700, 218.966156974161,
+  219.965788928685, 220.965578199561, 221.965550145047, 222.965055562297,
+  223.965041228123, 224.965088576052, 225.964951649635, 226.964837079061,
+  227.964949815142, 228.964753278386, 229.964358666956, 230.964158288467,
+  231.964118659643, 232.964105599719, 233.964013041408, 234.963810178150,
+  235.963847138572, 236.963634735645, 237.963596910032, 238.963659218725,
+  239.963498440155, 240.963445597658, 241.963141291971, 242.962684055746,
+  243.962664757583, 244.962762174588, 245.962862664373, 246.962595237647,
+  247.962493432533, 248.962292518484, 249.962622934132, 250.962187495200,
+  251.962038306920, 252.961926735684, 253.961867248643, 254.961699581530,
+  255.961371028199
+},
+// log K = 5
+{
+  22.304000309384, 23.261989079230, 24.249656493833, 25.267226901899,
+  26.314781734430, 27.392414068382, 28.500177845081, 29.638044883897,
+  30.805888087516, 32.003714838418, 33.231360682863, 34.488477823483,
+  35.774924122967, 37.090293268866, 38.434292452116, 39.806594197334,
+  41.206691028159, 42.634002763091, 44.088065403269, 45.568286128520,
+  47.074066947685, 48.604745105475, 50.159729872619, 51.738339282229,
+  53.339805618020, 54.963435280656, 56.608464119077, 58.274038434156,
+  59.959403900602, 61.663926660982, 63.386737477574, 65.127093833107,
+  66.884091606686, 68.656959698762, 70.445052178435, 72.247581860591,
+  74.063607663774, 75.892531705810, 77.733879994604, 79.586788509372,
+  81.450593478752, 83.324645303244, 85.208482496209, 87.101116685537,
+  89.002152917959, 90.911090999940, 92.827275073114, 94.750224301260,
+  96.679773715696, 98.615214511699, 100.556211548644, 102.502148428334,
+  104.452899214344, 106.407767504404, 108.366625832228, 110.329564692634,
+  112.295481742874, 114.264715056639, 116.236769512562, 118.211214593179,
+  120.188376370012, 122.167746134232, 124.148833002551, 126.131612799989,
+  128.116072530208, 130.102155735854, 132.089449244974, 134.077816832908,
+  136.067404409248, 138.057703527432, 140.049350564796, 142.041756984355,
+  144.034780624078, 146.028584459531, 148.022894428003, 150.017952482368,
+  152.013358245376, 154.009157360372, 156.005076955043, 158.001501688538,
+  159.998043891849, 161.995141096448, 163.992507770330, 165.990019741406,
+  167.987379363144, 169.985323168231, 171.983371903769, 173.981134136378,
+  175.979306888599, 177.977458741010, 179.975994045101, 181.974478508488,
+  183.973329438036, 185.972061903815, 187.971014174496, 189.969582278901,
+  191.968450510061, 193.967607022497, 195.966726931677, 197.965915341472,
+  199.964910588293, 201.964427852336, 203.964308576556, 205.963778587196,
+  207.963244179225, 209.962423156859, 211.961859295409, 213.961171237132,
+  215.960624048009, 217.959980906604, 219.959156277574, 221.958692298143,
+  223.958335010302, 225.958329061602, 227.957722024027, 229.957134954397,
+  231.956897929205, 233.956901572571, 235.956650770311, 237.956593290745,
+  239.956700470895, 241.956192761863, 243.956003327600, 245.955794901574,
+  247.955501305446, 249.954960233132, 251.954510187803, 253.954252376340,
+  255.954011943050, 257.953388008439, 259.953139524693, 261.952918466928,
+  263.952432910213, 265.952485387126, 267.952763908808, 269.952510701682,
+  271.952383356830, 273.951921008979, 275.951766755515, 277.951697578478,
+  279.951700548755, 281.951805228218, 283.951934023417, 285.952140722678,
+  287.952063941231, 289.951717207670, 291.951294241955, 293.950937111819,
+  295.950900506385, 297.950320403388, 299.949546547128, 301.949474819234,
+  303.949155377395, 305.948725010033, 307.948408553357, 309.948120512731,
+  311.947897212148, 313.947738181662, 315.947255247579, 317.946631669691,
+  319.946302193308, 321.945855881155, 323.945603079048, 325.944930145876,
+  327.945048538923, 329.944795565500, 331.944227346318, 333.943705422222,
+  335.943231918970, 337.942333069602, 339.942354997227, 341.941890952026,
+  343.941346276351, 345.940653575739, 347.940019001109, 349.940026817951,
+  351.938944534275, 353.938172407151, 355.937584534690, 357.937041224148,
+  359.936673631279, 361.936179855216, 363.936105840596, 365.935367434940,
+  367.935295315770, 369.934793960370, 371.934314961518, 373.933949277377,
+  375.933420587284, 377.933148188654, 379.932379363314, 381.931900192237,
+  383.930951130857, 385.930667541762, 387.930025971711, 389.929529183966,
+  391.929116798157, 393.928361940635, 395.927673611921, 397.927111110912,
+  399.926483516685, 401.925799510661, 403.924918492428, 405.924540208915,
+  407.924014467935, 409.923779779206, 411.922979548363, 413.922515268061,
+  415.922148908645, 417.921643066153, 419.921126587639, 421.920766647684,
+  423.920642824255, 425.920411492484, 427.920603679205, 429.920445437023,
+  431.920286509746, 433.919789058392, 435.919408587813, 437.918873200300,
+  439.918417416574, 441.918192491069, 443.918376503567, 445.917692006023,
+  447.917132642691, 449.917217997363, 451.916924115487, 453.916662151474,
+  455.916485288107, 457.915879851633, 459.915842501592, 461.915547354915,
+  463.914995141359, 465.914585759187, 467.914144928428, 469.913956069337,
+  471.913046082896, 473.912468624617, 475.912322983182, 477.912118100240,
+  479.911923664773, 481.911518944025, 483.911023761757, 485.910386539830,
+  487.910092345284, 489.910153786770, 491.909575245360, 493.908973823502,
+  495.908577061168, 497.908550221055, 499.908127020659, 501.907710605296,
+  503.907806972566, 505.907336571662, 507.907081330259, 509.906648846180,
+  511.906201574892
+},
+// log K = 6
+{
+  45.375999854524, 46.820948326168, 48.298703712049, 49.809402685314,
+  51.353165756275, 52.929933710602, 54.539893242468, 56.183033892089,
+  57.859217068495, 59.568519998303, 61.310801009963, 63.085886926328,
+  64.893798324934, 66.734230191052, 68.607173306372, 70.512210309984,
+  72.449265953981, 74.418007054647, 76.417910710559, 78.448929349152,
+  80.510585932256, 82.602514696500, 84.724061528721, 86.875166381330,
+  89.055217833675, 91.263852803721, 93.500342620773, 95.764451518395,
+  98.055566319691, 100.373234086043, 102.716794393386, 105.085599217566,
+  107.479363461004, 109.897282858060, 112.338907676111, 114.803401103124,
+  117.290259931022, 119.799265623507, 122.329323908482, 124.879681624600,
+  127.450299022993, 130.040239267142, 132.648756733593, 135.275366525219,
+  137.919720233989, 140.580778017562, 143.258128898839, 145.951552244700,
+  148.660145472460, 151.383257444852, 154.120434694643, 156.871202914301,
+  159.634634288797, 162.410377522877, 165.198249005575, 167.997420145534,
+  170.807579918337, 173.628020587905, 176.458633641294, 179.298282098581,
+  182.147449731726, 185.005094444186, 187.870899328937, 190.744709255561,
+  193.625572419776, 196.513740460772, 199.408599560293, 202.309940032246,
+  205.217132295894, 208.129828601313, 211.047995735211, 213.970919426104,
+  216.899109829571, 219.832024990852, 222.769136305806, 225.710258431247,
+  228.655217514772, 231.603984417526, 234.555642012353, 237.510635882749,
+  240.468820458907, 243.429903849174, 246.393192036307, 249.359251612075,
+  252.327416912482, 255.297598046767, 258.269777722144, 261.244337060733,
+  264.219958654591, 267.197643910088, 270.177202282129, 273.157339433786,
+  276.139287129066, 279.122276848731, 282.107226383752, 285.092136000673,
+  288.078230550129, 291.065256815999, 294.053160971001, 297.042401030971,
+  300.031784889467, 303.022502412860, 306.013581138287, 309.005124942334,
+  311.996910583050, 314.989635544162, 317.982189300799, 320.975762085573,
+  323.969719809182, 326.963672522149, 329.958419674913, 332.953046205989,
+  335.948338966110, 338.943201657934, 341.938798787180, 344.935321901459,
+  347.931089897905, 350.927355153871, 353.923731760523, 356.920383399325,
+  359.916863372939, 362.914245241722, 365.911216314347, 368.908636150615,
+  371.905028432797, 374.901981729273, 377.899718967915, 380.897391911925,
+  383.894945428721, 386.892974193389, 389.890517424427, 392.889050804732,
+  395.886893560179, 398.884852345719, 401.883235933616, 404.881805247652,
+  407.880049402262, 410.878410789472, 413.876755787854, 416.875140105536,
+  419.873758096660, 422.872739431833, 425.871627898739, 428.870663138631,
+  431.869475472357, 434.868378059774, 437.867537661532, 440.866274620049,
+  443.865086570094, 446.864282236320, 449.863630871851, 452.862772062678,
+  455.861877807413, 458.860680461949, 461.859728874700, 464.858478829514,
+  467.857314559298, 470.856707830759, 473.855641079729, 476.854880576501,
+  479.853812351976, 482.852666825246, 485.851471230749, 488.850997731659,
+  491.850420535006, 494.849118717584, 497.848645341646, 500.848230362352,
+  503.847546143645, 506.847285358235, 509.846275998715, 512.845599444922,
+  515.845441684852, 518.844522015985, 521.843923976006, 524.843281639782,
+  527.842399994830, 530.841152548296, 533.840788827465, 536.840058715879,
+  539.839578267778, 542.839076801367, 545.838441853502, 548.837925005355,
+  551.837456112430, 554.837041819817, 557.836916936797, 560.836116071833,
+  563.834832988710, 566.834107840572, 569.832763471368, 572.831700689757,
+  575.832050680783, 578.831190216307, 581.831249402778, 584.830325442080,
+  587.828993643112, 590.827962761832, 593.827454067745, 596.825833598259,
+  599.824193205007, 602.822806340644, 605.822276069352, 608.821423024191,
+  611.820006110523, 614.819016304854, 617.818220240160, 620.817311827644,
+  623.816286470744, 626.815211383752, 629.815077535810, 632.813925717940,
+  635.813392704099, 638.812424935595, 641.811289306882, 644.809932622788,
+  647.808609610738, 650.807771758015, 653.806851303878, 656.805452367508,
+  659.804133900742, 662.803103749985, 665.801910649522, 668.800969589926,
+  671.798796938405, 674.798110719805, 677.797843657187, 680.797711918663,
+  683.796449689643, 686.794998006462, 689.794205385109, 692.792836542916,
+  695.792195299304, 698.790778974184, 701.790704644168, 704.790168225574,
+  707.789261680438, 710.788674453657, 713.787207421297, 716.786043303658,
+  719.784740017888, 722.783523691482, 725.782739693893, 728.781695938105,
+  731.780295310459, 734.778719103047, 737.778056368816, 740.777014963309,
+  743.776470087777, 746.774710218241, 749.773897180146, 752.772555359445,
+  755.771799417720, 758.770328749569, 761.768940347768, 764.768630836408,
+  767.767399121425
+},
+// log K = 7
+{
+  91.554623, 93.970317, 96.431113, 98.937264, 101.488909, 104.086010, 106.728780, 109.417062,
+  112.150914, 114.930279, 117.755065, 120.625143, 123.540431, 126.500927, 129.506371, 132.556551,
+  135.651450, 138.790477, 141.973357, 145.200176, 148.470504, 151.783478, 155.139376, 158.537455,
+  161.977357, 165.458477, 168.980630, 172.543258, 176.145951, 179.788203, 183.469266, 187.188166,
+  190.945533, 194.739618, 198.570460, 202.437447, 206.339379, 210.276276, 214.247092, 218.251243,
+  222.288169, 226.357728, 230.458810, 234.590263, 238.751826, 242.942859, 247.162518, 251.410098,
+  255.685517, 259.987202, 264.314569, 268.667458, 273.044873, 277.445418, 281.869296, 286.316796,
+  290.785471, 295.276055, 299.788143, 304.319743, 308.869691, 313.438842, 318.026761, 322.632499,
+  327.254588, 331.892954, 336.547483, 341.217702, 345.901880, 350.601214, 355.313731, 360.038868,
+  364.777348, 369.526476, 374.288795, 379.061811, 383.846122, 388.640850, 393.446218, 398.260691,
+  403.084626, 407.916053, 412.755659, 417.604951, 422.461448, 427.324128, 432.195028, 437.072269,
+  441.956190, 446.846167, 451.741463, 456.641868, 461.549478, 466.461280, 471.377098, 476.297446,
+  481.222717, 486.151860, 491.085033, 496.022133, 500.962856, 505.906586, 510.853868, 515.802981,
+  520.755460, 525.711123, 530.668833, 535.629910, 540.592334, 545.557764, 550.525455, 555.494082,
+  560.463996, 565.436164, 570.410847, 575.385933, 580.361996, 585.339352, 590.318151, 595.300216,
+  600.281543, 605.264482, 610.250114, 615.236608, 620.222025, 625.208433, 630.195367, 635.183997,
+  640.172271, 645.162589, 650.153647, 655.144276, 660.133923, 665.123818, 670.115789, 675.109169,
+  680.103270, 685.097349, 690.091960, 695.086028, 700.080326, 705.075670, 710.069697, 715.065265,
+  720.062426, 725.058529, 730.054916, 735.050560, 740.048508, 745.044790, 750.041067, 755.037833,
+  760.035090, 765.033371, 770.029865, 775.025946, 780.022408, 785.021534, 790.018699, 795.016742,
+  800.013947, 805.012680, 810.009483, 815.007921, 820.008589, 825.007658, 830.007411, 835.006892,
+  840.006917, 845.006362, 850.004606, 855.003367, 860.001219, 864.999491, 869.999131, 874.998190,
+  879.998460, 884.997482, 889.995952, 894.995130, 899.994555, 904.993474, 909.991434, 914.990519,
+  919.991564, 924.992234, 929.993901, 934.992310, 939.992013, 944.993341, 949.993576, 954.992981,
+  959.993548, 964.993781, 969.993129, 974.994050, 979.994706, 984.994372, 989.994395, 994.995339,
+  999.995544, 1004.995498, 1009.996864, 1014.997829, 1019.996531, 1024.996724, 1029.997574, 1034.998591,
+  1039.997807, 1044.998954, 1050.000113, 1055.000633, 1060.000845, 1065.000760, 1070.000907, 1075.001248,
+  1080.001187, 1084.999775, 1089.999105, 1094.999647, 1100.001983, 1105.003468, 1110.004542, 1115.005170,
+  1120.006517, 1125.006493, 1130.006837, 1135.008369, 1140.006966, 1145.009225, 1150.008497, 1155.009589,
+  1160.009558, 1165.011395, 1170.009739, 1175.011131, 1180.013303, 1185.012525, 1190.014190, 1195.016399,
+  1200.016087, 1205.016715, 1210.018008, 1215.018108, 1220.019677, 1225.020269, 1230.019468, 1235.020515,
+  1240.021850, 1245.020961, 1250.019785, 1255.020253, 1260.020623, 1265.020156, 1270.020625, 1275.019028,
+  1280.019972
+},
+// log K = 8
+{
+  183.877784, 188.720097, 193.652407, 198.675131, 203.788397, 208.992238, 214.286620, 219.671580,
+  225.147277, 230.713559, 236.370208, 242.117078, 247.953569, 253.880078, 259.895869, 266.000405,
+  272.193694, 278.474457, 284.843102, 291.298228, 297.839692, 304.466434, 311.177471, 317.972674,
+  324.850719, 331.811076, 338.852902, 345.974709, 353.175880, 360.455237, 367.811613, 375.244484,
+  382.752472, 390.334218, 397.988777, 405.714596, 413.510671, 421.376147, 429.308973, 437.308197,
+  445.372302, 453.500121, 461.690796, 469.942249, 478.253652, 486.623402, 495.049662, 503.532193,
+  512.068670, 520.658371, 529.299186, 537.990886, 546.732097, 555.520093, 564.355935, 573.235689,
+  582.159345, 591.126063, 600.133321, 609.181348, 618.267848, 627.392384, 636.552822, 645.748152,
+  654.977601, 664.239902, 673.534497, 682.859448, 692.213983, 701.598606, 711.009376, 720.446781,
+  729.910995, 739.397888, 748.908546, 758.443812, 768.000572, 777.577584, 787.174815, 796.791250,
+  806.427107, 816.079660, 825.748493, 835.435992, 845.136828, 854.854141, 864.583886, 874.328396,
+  884.085720, 893.855568, 903.637568, 913.431080, 923.234553, 933.049830, 942.873895, 952.707793,
+  962.551743, 972.402421, 982.263001, 992.131464, 1002.006941, 1011.888460, 1021.777462, 1031.671376,
+  1041.569600, 1051.476632, 1061.386996, 1071.302834, 1081.224251, 1091.148847, 1101.080297, 1111.013589,
+  1120.951962, 1130.893360, 1140.838011, 1150.786112, 1160.737100, 1170.689817, 1180.646908, 1190.604979,
+  1200.565150, 1210.527894, 1220.492169, 1230.460128, 1240.429764, 1250.401749, 1260.373274, 1270.349608,
+  1280.325544, 1290.302087, 1300.279074, 1310.257850, 1320.238948, 1330.221273, 1340.203554, 1350.186669,
+  1360.171345, 1370.156233, 1380.142561, 1390.129535, 1400.119177, 1410.107938, 1420.097171, 1430.087586,
+  1440.077831, 1450.067996, 1460.060299, 1470.050059, 1480.042995, 1490.035101, 1500.026878, 1510.021128,
+  1520.014262, 1530.008487, 1540.002895, 1549.996342, 1559.991075, 1569.986665, 1579.981202, 1589.976021,
+  1599.969512, 1609.964051, 1619.959210, 1629.957092, 1639.953516, 1649.948113, 1659.946715, 1669.945253,
+  1679.941736, 1689.938651, 1699.937600, 1709.936396, 1719.933608, 1729.931040, 1739.930316, 1749.927141,
+  1759.922600, 1769.920847, 1779.918614, 1789.915707, 1799.913000, 1809.910958, 1819.908532, 1829.906905,
+  1839.903410, 1849.904217, 1859.901705, 1869.898815, 1879.898869, 1889.899032, 1899.899113, 1909.896164,
+  1919.893199, 1929.894515, 1939.894965, 1949.893184, 1959.892164, 1969.890794, 1979.889035, 1989.888005,
+  1999.888538, 2009.888252, 2019.888976, 2029.890242, 2039.890655, 2049.891065, 2059.891068, 2069.889204,
+  2079.891120, 2089.892462, 2099.892479, 2109.891814, 2119.893049, 2129.893705, 2139.893471, 2149.894162,
+  2159.894213, 2169.895041, 2179.895987, 2189.894721, 2199.894028, 2209.896005, 2219.898683, 2229.898673,
+  2239.898012, 2249.898018, 2259.898215, 2269.899380, 2279.902590, 2289.903144, 2299.902047, 2309.904214,
+  2319.903812, 2329.904180, 2339.902528, 2349.903057, 2359.903089, 2369.904512, 2379.902563, 2389.905522,
+  2399.903653, 2409.902313, 2419.904452, 2429.905949, 2439.905059, 2449.904954, 2459.904977, 2469.905111,
+  2479.904049, 2489.903762, 2499.902022, 2509.904352, 2519.903272, 2529.901643, 2539.900789, 2549.903045,
+  2559.904334
+},
+// log K = 9
+{
+  368.528954, 378.224619, 388.100156, 398.155846, 408.391912, 418.808631, 429.406377, 440.185048,
+  451.144613, 462.284693, 473.604932, 485.105306, 496.784844, 508.643147, 520.679581, 532.893098,
+  545.282616, 557.848295, 570.587645, 583.500268, 596.584345, 609.838697, 623.261582, 636.851681,
+  650.607666, 664.527320, 678.608336, 692.849674, 707.248088, 721.803089, 736.511331, 751.372034,
+  766.381798, 781.538178, 796.839808, 812.283881, 827.867995, 843.588520, 859.444318, 875.432109,
+  891.550599, 907.795630, 924.165943, 940.657393, 957.268601, 973.995685, 990.837235, 1007.789667,
+  1024.850025, 1042.016602, 1059.286426, 1076.658199, 1094.126523, 1111.689346, 1129.345589, 1147.092119,
+  1164.926795, 1182.844471, 1200.846880, 1218.929006, 1237.088281, 1255.322930, 1273.630393, 1292.007269,
+  1310.452948, 1328.963430, 1347.538768, 1366.176566, 1384.871299, 1403.626249, 1422.434094, 1441.295238,
+  1460.208340, 1479.169853, 1498.179265, 1517.234855, 1536.335403, 1555.477305, 1574.659180, 1593.881592,
+  1613.139378, 1632.433516, 1651.764587, 1671.127255, 1690.519577, 1709.942250, 1729.393153, 1748.871689,
+  1768.377142, 1787.907209, 1807.464755, 1827.043146, 1846.641976, 1866.263022, 1885.903553, 1905.564908,
+  1925.240581, 1944.936723, 1964.648221, 1984.375489, 2004.117246, 2023.875053, 2043.644075, 2063.423514,
+  2083.218689, 2103.026148, 2122.840475, 2142.669046, 2162.505619, 2182.351758, 2202.205963, 2222.068658,
+  2241.939010, 2261.814547, 2281.701803, 2301.591894, 2321.490728, 2341.393208, 2361.298412, 2381.211528,
+  2401.132345, 2421.057721, 2440.985510, 2460.916691, 2480.851468, 2500.792645, 2520.738446, 2540.683395,
+  2560.634603, 2580.586307, 2600.540530, 2620.497170, 2640.458137, 2660.421783, 2680.386186, 2700.352664,
+  2720.321411, 2740.291944, 2760.264068, 2780.238910, 2800.212546, 2820.185829, 2840.163952, 2860.142145,
+  2880.120568, 2900.102466, 2920.081981, 2940.062964, 2960.046262, 2980.029101, 3000.012591, 3019.998424,
+  3039.983637, 3059.969573, 3079.955927, 3099.942114, 3119.930348, 3139.919614, 3159.909130, 3179.899816,
+  3199.892231, 3219.885316, 3239.878917, 3259.871675, 3279.865195, 3299.856210, 3319.848871, 3339.840385,
+  3359.834158, 3379.830660, 3399.826361, 3419.822855, 3439.814714, 3459.813765, 3479.808673, 3499.804339,
+  3519.797785, 3539.791012, 3559.789885, 3579.785152, 3599.780237, 3619.775480, 3639.772724, 3659.768920,
+  3679.765735, 3699.766361, 3719.765048, 3739.762845, 3759.761526, 3779.757598, 3799.755719, 3819.751149,
+  3839.750182, 3859.749673, 3879.749344, 3899.749939, 3919.749060, 3939.748385, 3959.750520, 3979.750055,
+  3999.750104, 4019.749963, 4039.748453, 4059.746737, 4079.744522, 4099.744430, 4119.741804, 4139.740704,
+  4159.742056, 4179.740063, 4199.739155, 4219.741718, 4239.743433, 4259.743115, 4279.743265, 4299.745518,
+  4319.746682, 4339.750491, 4359.748549, 4379.747226, 4399.750319, 4419.750379, 4439.752339, 4459.750561,
+  4479.750612, 4499.747862, 4519.750693, 4539.747476, 4559.749051, 4579.750603, 4599.750089, 4619.754781,
+  4639.755536, 4659.758879, 4679.762145, 4699.762199, 4719.763886, 4739.765081, 4759.765681, 4779.765427,
+  4799.767582, 4819.769915, 4839.770107, 4859.769398, 4879.768976, 4899.768288, 4919.768786, 4939.768607,
+  4959.772822, 4979.772833, 4999.775808, 5019.773468, 5039.776181, 5059.770860, 5079.773006, 5099.773136,
+  5119.772474
+},
+// log K = 10
+{
+  737.833738, 757.236338, 776.997875, 797.119465, 817.602058, 838.445349, 859.650320, 881.216437,
+  903.143414, 925.430957, 948.079220, 971.086231, 994.451449, 1018.173846, 1042.251928, 1066.683804,
+  1091.467648, 1116.601416, 1142.083017, 1167.909918, 1194.079097, 1220.588887, 1247.435099, 1274.614860,
+  1302.125360, 1329.962942, 1358.123177, 1386.601609, 1415.396299, 1444.501692, 1473.913256, 1503.628515,
+  1533.642054, 1563.948888, 1594.544067, 1625.424305, 1656.582169, 1688.016543, 1719.718553, 1751.685990,
+  1783.912130, 1816.393610, 1849.122123, 1882.095124, 1915.305647, 1948.748605, 1982.419843, 2016.313904,
+  2050.422277, 2084.742618, 2119.270601, 2153.998538, 2188.920843, 2224.034736, 2259.333960, 2294.812594,
+  2330.467059, 2366.291844, 2402.281356, 2438.428954, 2474.734330, 2511.187839, 2547.786319, 2584.527356,
+  2621.406814, 2658.418551, 2695.554270, 2732.812564, 2770.190157, 2807.682744, 2845.287138, 2882.995251,
+  2920.807982, 2958.719996, 2996.728771, 3034.826885, 3073.014082, 3111.287116, 3149.642093, 3188.073159,
+  3226.577352, 3265.156836, 3303.803738, 3342.515111, 3381.291514, 3420.130671, 3459.024315, 3497.974631,
+  3536.978221, 3576.034737, 3615.138610, 3654.285859, 3693.480455, 3732.715296, 3771.991792, 3811.305300,
+  3850.654870, 3890.041152, 3929.457167, 3968.905021, 4008.381466, 4047.886976, 4087.419414, 4126.978997,
+  4166.562624, 4206.166726, 4245.793284, 4285.445151, 4325.113012, 4364.802795, 4404.506274, 4444.228475,
+  4483.965191, 4523.719569, 4563.487708, 4603.267460, 4643.060387, 4682.865430, 4722.680718, 4762.503248,
+  4802.341326, 4842.185529, 4882.039088, 4921.900432, 4961.772625, 5001.649566, 5041.534380, 5081.425809,
+  5121.323013, 5161.224908, 5201.130560, 5241.044625, 5280.962456, 5320.882373, 5360.810007, 5400.740058,
+  5440.673843, 5480.606748, 5520.547394, 5560.493520, 5600.438100, 5640.383853, 5680.335386, 5720.292400,
+  5760.250576, 5800.212705, 5840.172984, 5880.134991, 5920.097923, 5960.060294, 6000.031177, 6040.003398,
+  6079.981696, 6119.954349, 6159.929074, 6199.908569, 6239.886163, 6279.863659, 6319.838261, 6359.815405,
+  6399.795170, 6439.777225, 6479.755885, 6519.738857, 6559.723169, 6599.708549, 6639.695186, 6679.679788,
+  6719.663289, 6759.651052, 6799.634129, 6839.624763, 6879.615013, 6919.605584, 6959.591310, 6999.586006,
+  7039.577399, 7079.565236, 7119.557222, 7159.548481, 7199.533025, 7239.519648, 7279.511781, 7319.507639,
+  7359.506105, 7399.497488, 7439.494944, 7479.491545, 7519.488415, 7559.480534, 7599.475646, 7639.470315,
+  7679.468730, 7719.460928, 7759.455834, 7799.455686, 7839.451758, 7879.446841, 7919.443884, 7959.444138,
+  7999.443141, 8039.446339, 8079.445285, 8119.444120, 8159.443157, 8199.446480, 8239.438893, 8279.437597,
+  8319.439714, 8359.441566, 8399.440754, 8439.439156, 8479.440112, 8519.442365, 8559.444795, 8599.444497,
+  8639.446714, 8679.449177, 8719.447620, 8759.443423, 8799.445995, 8839.446170, 8879.449622, 8919.450486,
+  8959.450372, 8999.449731, 9039.448449, 9079.452579, 9119.451596, 9159.454020, 9199.456651, 9239.460902,
+  9279.460419, 9319.461322, 9359.464611, 9399.467790, 9439.471448, 9479.468426, 9519.462982, 9559.464881,
+  9599.462983, 9639.461706, 9679.457284, 9719.457023, 9759.455648, 9799.458786, 9839.456980, 9879.457411,
+  9919.454971, 9959.451735, 9999.451916, 10039.443939, 10079.435325, 10119.432142, 10159.431313, 10199.431306,
+  10239.429671
+},
+// log K = 11
+{
+  1476.444530, 1515.260638, 1554.794859, 1595.048761, 1636.023652, 1677.720065, 1720.138513, 1763.279158,
+  1807.141823, 1851.724811, 1897.027530, 1943.048153, 1989.784818, 2037.234835, 2085.395149, 2134.263580,
+  2183.834818, 2234.105548, 2285.071077, 2336.726064, 2389.065940, 2442.084276, 2495.776521, 2550.134177,
+  2605.153465, 2660.825522, 2717.142881, 2774.098631, 2831.683790, 2889.889911, 2948.711233, 3008.137464,
+  3068.158605, 3128.766926, 3189.951984, 3251.704129, 3314.013826, 3376.873018, 3440.270569, 3504.194851,
+  3568.636295, 3633.586039, 3699.031630, 3764.962925, 3831.371336, 3898.245744, 3965.574384, 4033.345701,
+  4101.550888, 4170.180686, 4239.221393, 4308.664143, 4378.497100, 4448.712735, 4519.296212, 4590.243535,
+  4661.538747, 4733.174738, 4805.142002, 4877.427081, 4950.019513, 5022.915001, 5096.098561, 5169.567185,
+  5243.312012, 5317.314578, 5391.572253, 5466.077564, 5540.821377, 5615.792025, 5690.985894, 5766.392174,
+  5842.007403, 5917.816734, 5993.815607, 6069.999957, 6146.362662, 6222.895187, 6299.592875, 6376.440778,
+  6453.443173, 6530.588182, 6607.874560, 6685.288536, 6762.829840, 6840.490840, 6918.272948, 6996.164556,
+  7074.161694, 7152.259404, 7230.451098, 7308.737035, 7387.113244, 7465.575350, 7544.120230, 7622.733447,
+  7701.427750, 7780.188044, 7859.016102, 7937.904850, 8016.853624, 8095.860379, 8174.919884, 8254.034842,
+  8333.197794, 8412.405307, 8491.657429, 8570.946804, 8650.278289, 8729.648657, 8809.056098, 8888.496764,
+  8967.966663, 9047.466051, 9126.993636, 9206.547289, 9286.127523, 9365.733732, 9445.361201, 9525.008810,
+  9604.680107, 9684.363532, 9764.066435, 9843.787987, 9923.523567, 10003.272886, 10083.037524, 10162.811803,
+  10242.597755, 10322.402415, 10402.216522, 10482.042584, 10561.877077, 10641.722187, 10721.570740, 10801.435848,
+  10881.301249, 10961.178474, 11041.059765, 11120.951444, 11200.845806, 11280.746075, 11360.652145, 11440.567398,
+  11520.476709, 11600.395391, 11680.316855, 11760.241900, 11840.170134, 11920.103548, 12000.038206, 12079.975425,
+  12159.912459, 12239.857191, 12319.802744, 12399.754385, 12479.707055, 12559.658981, 12639.614004, 12719.566502,
+  12799.526877, 12879.488833, 12959.451231, 13039.416480, 13119.388158, 13199.360789, 13279.334923, 13359.298927,
+  13439.261641, 13519.235871, 13599.204919, 13679.181082, 13759.157391, 13839.137425, 13919.112545, 13999.096522,
+  14079.082789, 14159.060487, 14239.040581, 14319.025236, 14399.006995, 14478.994078, 14558.980576, 14638.962774,
+  14718.952192, 14798.938522, 14878.931501, 14958.919062, 15038.907891, 15118.897247, 15198.889129, 15278.881057,
+  15358.873272, 15438.867627, 15518.861931, 15598.858206, 15678.850014, 15758.849382, 15838.847872, 15918.837658,
+  15998.837920, 16078.831305, 16158.827345, 16238.815546, 16318.817302, 16398.814389, 16478.815820, 16558.811634,
+  16638.808126, 16718.807170, 16798.805652, 16878.813389, 16958.806543, 17038.807885, 17118.813146, 17198.813752,
+  17278.817574, 17358.820511, 17438.819443, 17518.814262, 17598.815511, 17678.814160, 17758.816726, 17838.817584,
+  17918.814197, 17998.821276, 18078.822342, 18158.828474, 18238.822064, 18318.826732, 18398.827758, 18478.827319,
+  18558.834153, 18638.835651, 18718.839105, 18798.840903, 18878.836463, 18958.836015, 19038.836351, 19118.832583,
+  19198.829856, 19278.822953, 19358.824866, 19438.823697, 19518.826538, 19598.826954, 19678.831393, 19758.822279,
+  19838.818757, 19918.808247, 19998.804516, 20078.809201, 20158.808900, 20238.806398, 20318.798516, 20398.802224,
+  20478.803179
+},
+// log K = 12
+{
+  2953.666722, 3031.310137, 3110.389526, 3190.908699, 3272.865410, 3356.267166, 3441.114376, 3527.404284,
+  3615.136846, 3704.310976, 3794.924283, 3886.973846, 3980.451903, 4075.355844, 4171.681168, 4269.423939,
+  4368.569738, 4469.111760, 4571.043603, 4674.356370, 4779.038117, 4885.076002, 4992.460659, 5101.176875,
+  5211.214098, 5322.553631, 5435.184633, 5549.088369, 5664.258175, 5780.673987, 5898.312682, 6017.158593,
+  6137.195815, 6258.397951, 6380.761679, 6504.261630, 6628.870413, 6754.585867, 6881.374325, 7009.206926,
+  7138.080376, 7267.979175, 7398.866805, 7530.722851, 7663.531580, 7797.269872, 7931.907340, 8067.451667,
+  8203.855692, 8341.091306, 8479.155031, 8618.026003, 8757.682272, 8898.095612, 9039.253787, 9181.124777,
+  9323.694329, 9466.944202, 9610.861899, 9755.403639, 9900.578346, 10046.356477, 10192.718770, 10339.635914,
+  10487.081762, 10635.066232, 10783.560130, 10932.549371, 11082.024055, 11231.949071, 11382.332885, 11533.122831,
+  11684.327555, 11835.935533, 11987.926999, 12140.281619, 12292.994375, 12446.046449, 12599.411083, 12753.088314,
+  12907.079745, 13061.355923, 13215.895621, 13370.722085, 13525.795736, 13681.095971, 13836.645692, 13992.405906,
+  14148.380716, 14304.566860, 14460.959382, 14617.539754, 14774.296125, 14931.214508, 15088.271748, 15245.493870,
+  15402.867309, 15560.380994, 15718.035423, 15875.801492, 16033.687891, 16191.701231, 16349.808763, 16508.026001,
+  16666.336068, 16824.745049, 16983.259302, 17141.848044, 17300.504517, 17459.240272, 17618.060296, 17776.914688,
+  17935.848010, 18094.852645, 18253.914288, 18413.023567, 18572.168358, 18731.376016, 18890.631505, 19049.948196,
+  19209.279260, 19368.638871, 19528.043860, 19687.498901, 19846.941765, 20006.443084, 20165.973535, 20325.525957,
+  20485.109520, 20644.729917, 20804.363876, 20964.021778, 21123.698687, 21283.379344, 21443.080536, 21602.821439,
+  21762.568135, 21922.305338, 22082.065878, 22241.856145, 22401.629436, 22561.435266, 22721.262644, 22881.080478,
+  23040.916036, 23200.734053, 23360.583255, 23520.433188, 23680.275065, 23840.132678, 24000.012847, 24159.886135,
+  24319.788338, 24479.654024, 24639.540316, 24799.466449, 24959.369335, 25119.276072, 25279.186502, 25439.091982,
+  25599.020083, 25758.930671, 25918.847522, 26078.773060, 26238.704021, 26398.647566, 26558.603366, 26718.539855,
+  26878.464425, 27038.402892, 27198.344989, 27358.270906, 27518.203448, 27678.169545, 27838.133625, 27998.111630,
+  28158.064264, 28318.032491, 28477.994153, 28637.950838, 28797.910362, 28957.884994, 29117.861910, 29277.814384,
+  29437.774223, 29597.762256, 29757.729819, 29917.688497, 30077.675371, 30237.667941, 30397.639431, 30557.644527,
+  30717.622991, 30877.622641, 31037.598941, 31197.601527, 31357.599118, 31517.585221, 31677.569983, 31837.556138,
+  31997.527880, 32157.500622, 32317.491709, 32477.504936, 32637.501123, 32797.516330, 32957.547336, 33117.541765,
+  33277.561207, 33437.554332, 33597.564988, 33757.565321, 33917.561180, 34077.580598, 34237.597132, 34397.598246,
+  34557.617536, 34717.623318, 34877.649722, 35037.645029, 35197.681143, 35357.700478, 35517.701804, 35677.734383,
+  35837.736494, 35997.737103, 36157.732045, 36317.741539, 36477.735602, 36637.761491, 36797.742508, 36957.734137,
+  37117.742952, 37277.789218, 37437.793798, 37597.779066, 37757.796449, 37917.794116, 38077.804399, 38237.797507,
+  38397.788049, 38557.768583, 38717.768279, 38877.752553, 39037.747349, 39197.737646, 39357.774395, 39517.752643,
+  39677.753347, 39837.740038, 39997.719486, 40157.709578, 40317.708657, 40477.719261, 40637.726978, 40797.711139,
+  40957.707784
+},
+// log K = 13
+{
+  5908.111420, 6063.410343, 6221.578483, 6382.623803, 6546.553966, 6713.369171, 6883.070010, 7055.656719,
+  7231.129758, 7409.486672, 7590.720089, 7774.822139, 7961.786850, 8151.601206, 8344.261218, 8539.750086,
+  8738.048786, 8939.141702, 9143.011205, 9349.645220, 9559.003557, 9771.078383, 9985.846791, 10203.277103,
+  10423.349219, 10646.031323, 10871.293001, 11099.111735, 11329.434411, 11562.255804, 11797.522109, 12035.204106,
+  12275.263649, 12517.670237, 12762.388671, 13009.375491, 13258.587519, 13510.006825, 13763.564399, 14019.227089,
+  14276.975295, 14536.745792, 14798.511796, 15062.215057, 15327.806593, 15595.257499, 15864.529774, 16135.586770,
+  16408.362074, 16682.849583, 16958.977019, 17236.708221, 17515.990972, 17796.798914, 18079.096585, 18362.834517,
+  18647.959152, 18934.452073, 19222.251998, 19511.366053, 19801.717928, 20093.269020, 20385.960319, 20679.772736,
+  20974.698498, 21270.684814, 21567.671010, 21865.649193, 22164.581341, 22464.423293, 22765.172894, 23066.785452,
+  23369.201089, 23672.387494, 23976.354000, 24281.072155, 24586.465328, 24892.557323, 25199.303712, 25506.654130,
+  25814.626835, 26123.148009, 26432.247046, 26741.889610, 27052.042135, 27362.685181, 27673.761784, 27985.292246,
+  28297.219809, 28609.572368, 28922.317633, 29235.448741, 29548.944253, 29862.766175, 30176.910610, 30491.363107,
+  30806.103787, 31121.126024, 31436.412153, 31751.975134, 32067.737444, 32383.768560, 32700.017044, 33016.458300,
+  33333.101934, 33649.904492, 33966.923435, 34284.105041, 34601.429152, 34918.876076, 35236.481998, 35554.235511,
+  35872.093847, 36190.101572, 36508.190892, 36826.420800, 37144.739624, 37463.138566, 37781.598984, 38100.191087,
+  38418.846511, 38737.587005, 39056.391321, 39375.255819, 39694.182864, 40013.173503, 40332.221146, 40651.311460,
+  40970.465534, 41289.697002, 41608.913381, 41928.242461, 42247.585089, 42566.988678, 42886.409594, 43205.849396,
+  43525.322840, 43844.784209, 44164.327769, 44483.869861, 44803.383648, 45122.975593, 45442.593765, 45762.258110,
+  46081.937012, 46401.605265, 46721.295022, 47041.004034, 47360.695978, 47680.435451, 48000.192231, 48319.953839,
+  48639.726508, 48959.506239, 49279.318744, 49599.117846, 49918.951926, 50238.772730, 50558.620076, 50878.455501,
+  51198.285377, 51518.135316, 51837.976134, 52157.809260, 52477.635805, 52797.492063, 53117.368983, 53437.262930,
+  53757.142921, 54077.026859, 54396.907345, 54716.835955, 55036.713062, 55356.640827, 55676.537369, 55996.488865,
+  56316.420754, 56636.355217, 56956.329593, 57276.245282, 57596.202473, 57916.159021, 58236.077854, 58556.010303,
+  58875.938110, 59195.924442, 59515.871947, 59835.838035, 60155.809948, 60475.808167, 60795.765545, 61115.766398,
+  61435.725825, 61755.677959, 62075.688445, 62395.647263, 62715.609853, 63035.591787, 63355.588060, 63675.579841,
+  63995.560980, 64315.552298, 64635.582956, 64955.538779, 65275.512154, 65595.490993, 65915.477332, 66235.470693,
+  66555.483426, 66875.455452, 67195.455039, 67515.435090, 67835.418300, 68155.401870, 68475.404599, 68795.452423,
+  69115.474466, 69435.450344, 69755.439668, 70075.479966, 70395.468204, 70715.475957, 71035.472807, 71355.513734,
+  71675.486047, 71995.450475, 72315.443410, 72635.425138, 72955.481520, 73275.482426, 73595.476688, 73915.469540,
+  74235.426693, 74555.381683, 74875.368981, 75195.429157, 75515.480428, 75835.517265, 76155.557945, 76475.556134,
+  76795.579451, 77115.605924, 77435.604178, 77755.572770, 78075.576305, 78395.562526, 78715.547599, 79035.551674,
+  79355.540615, 79675.575726, 79995.547586, 80315.544278, 80635.546227, 80955.555617, 81275.540591, 81595.549538,
+  81915.550079
+},
+// log K = 14
+{
+  11817.000969, 12127.609164, 12443.958867, 12766.059919, 13093.926347, 13427.563687, 13766.972021, 14112.158775,
+  14463.118632, 14819.833221, 15182.310838, 15550.514164, 15924.452624, 16304.097682, 16689.414687, 17080.390156,
+  17476.985675, 17879.171048, 18286.913146, 18700.171223, 19118.909859, 19543.074108, 19972.606221, 20407.466793,
+  20847.600290, 21292.965548, 21743.489087, 22199.109276, 22659.760749, 23125.379711, 23595.914325, 24071.277989,
+  24551.407611, 25036.233071, 25525.671155, 26019.641791, 26518.074136, 27020.885189, 27528.004081, 28039.347478,
+  28554.818908, 29074.343878, 29597.869471, 30125.254970, 30656.449891, 31191.351228, 31729.908970, 32272.003587,
+  32817.586623, 33366.531359, 33918.769390, 34474.232727, 35032.798601, 35594.423192, 36159.024745, 36726.469786,
+  37296.712851, 37869.686035, 38445.278953, 39023.442285, 39604.084062, 40187.147209, 40772.503600, 41360.148758,
+  41949.978377, 42541.929077, 43135.892496, 43731.836459, 44329.664552, 44929.334224, 45530.772542, 46133.919152,
+  46738.746702, 47345.120215, 47953.051656, 48562.469011, 49173.281212, 49785.453501, 50398.908717, 51013.645932,
+  51629.563526, 52246.635759, 52864.820434, 53484.057447, 54104.303719, 54725.504675, 55347.680019, 55970.744363,
+  56594.688890, 57219.458967, 57844.949821, 58471.197998, 59098.180259, 59725.774793, 60354.058215, 60982.962558,
+  61612.427469, 62242.448368, 62873.013426, 63504.106743, 64135.672742, 64767.665213, 65400.082326, 66032.949331,
+  66666.219558, 67299.817447, 67933.801373, 68568.092593, 69202.717636, 69837.643507, 70472.852133, 71108.305774,
+  71744.023189, 72380.016332, 73016.168118, 73652.560876, 74289.210741, 74926.041959, 75563.048434, 76200.207724,
+  76837.505542, 77474.999283, 78112.608284, 78750.375633, 79388.254423, 80026.267074, 80664.419197, 81302.602941,
+  81940.909776, 82579.321050, 83217.801329, 83856.420605, 84495.098999, 85133.832545, 85772.588001, 86411.448347,
+  87050.399262, 87689.358100, 88328.463714, 88967.516990, 89606.618839, 90245.788934, 90884.986443, 91524.243962,
+  92163.523370, 92802.835394, 93442.198790, 94081.633840, 94721.043838, 95360.482304, 96000.002974, 96639.518339,
+  97279.069377, 97918.584802, 98558.189888, 99197.773242, 99837.390001, 100476.994827, 101116.655652, 101756.293703,
+  102395.977437, 103035.662337, 103675.360919, 104315.063845, 104954.783621, 105594.504004, 106234.229823, 106874.056079,
+  107513.846129, 108153.562857, 108793.393622, 109433.202056, 110072.966808, 110712.777026, 111352.577741, 111992.441538,
+  112632.256702, 113272.083743, 113911.943484, 114551.818432, 115191.687411, 115831.584194, 116471.522453, 117111.338979,
+  117751.192777, 118391.070556, 119030.991702, 119670.888488, 120310.801509, 120950.761000, 121590.691826, 122230.618394,
+  122870.618097, 123510.616056, 124150.552363, 124790.488946, 125430.445483, 126070.428297, 126710.409643, 127350.398562,
+  127990.397846, 128630.413222, 129270.354937, 129910.365912, 130550.332174, 131190.259646, 131830.280970, 132470.304278,
+  133110.295205, 133750.312426, 134390.364847, 135030.321000, 135670.319926, 136310.335570, 136950.324790, 137590.338289,
+  138230.368657, 138870.374084, 139510.447896, 140150.468353, 140790.441338, 141430.438424, 142070.440787, 142710.431348,
+  143350.433770, 143990.493350, 144630.548409, 145270.549935, 145910.530486, 146550.488808, 147190.510290, 147830.509426,
+  148470.575641, 149110.541845, 149750.592427, 150390.634921, 151030.706500, 151670.646574, 152310.624482, 152950.686127,
+  153590.659153, 154230.646236, 154870.720079, 155510.614028, 156150.596142, 156790.659641, 157430.633386, 158070.649047,
+  158710.591350, 159350.585980, 159990.557711, 160630.530183, 161270.499427, 161910.521323, 162550.516603, 163190.482792,
+  163830.422631
+},
+// log K = 15
+{
+  23634.780143, 24256.008404, 24888.717596, 25532.934303, 26188.680791, 26855.965949, 27534.798395, 28225.179199,
+  28927.099457, 29640.545847, 30365.501013, 31101.936533, 31849.825685, 32609.111850, 33379.755958, 34161.706474,
+  34954.912131, 35759.291030, 36574.778310, 37401.280667, 38238.742234, 39087.060449, 39946.133936, 40815.861824,
+  41696.151280, 42586.877221, 43487.927203, 44399.169293, 45320.460825, 46251.693279, 47192.741847, 48143.464216,
+  49103.716679, 50073.349113, 51052.208336, 52040.138165, 53036.993192, 54042.604241, 55056.838449, 56079.501709,
+  57110.434199, 58149.484623, 59196.455710, 60251.226284, 61313.593108, 62383.385277, 63460.484960, 64544.666337,
+  65635.763495, 66733.629361, 67838.096489, 68948.987037, 70066.117363, 71189.338133, 72318.464573, 73453.423815,
+  74593.923438, 75739.864335, 76891.058098, 78047.391508, 79208.654009, 80374.753680, 81545.514022, 82720.817799,
+  83900.405263, 85084.273208, 86272.263471, 87464.134340, 88659.829437, 89859.204804, 91062.115479, 92268.425863,
+  93478.041907, 94690.804589, 95906.647693, 97125.398400, 98347.018145, 99571.305992, 100798.274503, 102027.737913,
+  103259.574749, 104493.727918, 105730.035415, 106968.465700, 108208.988306, 109451.445144, 110695.748927, 111941.827269,
+  113189.666448, 114439.093665, 115690.116530, 116942.608958, 118196.501294, 119451.722652, 120708.276299, 121966.046247,
+  123225.027480, 124485.082349, 125746.201609, 127008.357433, 128271.416054, 129535.417336, 130800.297278, 132066.008820,
+  133332.483576, 134599.733640, 135867.681429, 137136.303792, 138405.538969, 139675.413251, 140945.839375, 142216.787412,
+  143488.232199, 144760.234976, 146032.650196, 147305.459198, 148578.703708, 149852.356585, 151126.347770, 152400.641475,
+  153675.275168, 154950.179904, 156225.390849, 157500.864126, 158776.585926, 160052.504602, 161328.716398, 162605.171897,
+  163881.825931, 165158.677102, 166435.646121, 167712.814846, 168990.136047, 170267.619890, 171545.238277, 172823.011460,
+  174100.890250, 175378.821293, 176656.920467, 177935.093547, 179213.318173, 180491.692642, 181770.130424, 183048.727206,
+  184327.217835, 185605.921891, 186884.657773, 188163.418059, 189442.221838, 190721.122201, 192000.068271, 193279.087882,
+  194558.139554, 195837.195523, 197116.400538, 198395.612565, 199674.831001, 200954.101970, 202233.365695, 203512.639968,
+  204792.003319, 206071.379608, 207350.730914, 208630.080321, 209909.544216, 211189.014497, 212468.508049, 213748.055445,
+  215027.584964, 216307.216078, 217586.896296, 218866.477232, 220146.109506, 221425.767458, 222705.465633, 223985.129542,
+  225264.797531, 226544.544335, 227824.304232, 229104.031191, 230383.739676, 231663.424684, 232943.198829, 234222.906969,
+  235502.669185, 236782.475696, 238062.262723, 239342.090234, 240622.002668, 241901.911022, 243181.742902, 244461.690551,
+  245741.531359, 247021.432644, 248301.277604, 249581.168404, 250861.094078, 252141.038818, 253421.065594, 254701.020169,
+  255980.887518, 257260.848328, 258540.873565, 259820.905666, 261100.908594, 262380.854763, 263660.862699, 264940.834842,
+  266220.775656, 267500.745402, 268780.761926, 270060.823540, 271340.798715, 272620.823081, 273900.881190, 275180.928531,
+  276460.914606, 277740.919081, 279020.967358, 280301.002010, 281581.054314, 282861.097638, 284141.109728, 285421.181335,
+  286701.182750, 287981.231437, 289261.342952, 290541.328315, 291821.320901, 293101.395969, 294381.402286, 295661.374812,
+  296941.357112, 298221.462446, 299501.443884, 300781.429735, 302061.512800, 303341.459364, 304621.530042, 305901.507763,
+  307181.541613, 308461.506346, 309741.474814, 311021.458944, 312301.373851, 313581.258984, 314861.222655, 316141.192042,
+  317421.179698, 318701.162346, 319981.143104, 321261.185706, 322541.120379, 323821.065555, 325100.961639, 326380.920540,
+  327660.786948
+},
+// log K = 16
+{
+  47270.338530, 48512.804588, 49778.235006, 51066.679846, 52378.178287, 53712.756179, 55070.430016, 56451.198432,
+  57855.044557, 59281.941569, 60731.855114, 62204.727445, 63700.498656, 65219.087806, 66760.381754, 68324.278827,
+  69910.681844, 71519.442760, 73150.426782, 74803.469853, 76478.400456, 78175.038414, 79893.184924, 81632.639705,
+  83393.214587, 85174.662642, 86976.739912, 88799.210643, 90641.829434, 92504.355605, 94386.478165, 96287.925119,
+  98208.422967, 100147.694840, 102105.387259, 104081.226364, 106074.916897, 108086.136683, 110114.560800, 112159.857579,
+  114221.724675, 116299.785348, 118393.731853, 120503.258946, 122627.958417, 124767.540962, 126921.668848, 129090.003274,
+  131272.204811, 133467.918004, 135676.870426, 137898.595565, 140132.829694, 142379.269230, 144637.536478, 146907.321805,
+  149188.299797, 151480.151490, 153782.571459, 156095.292246, 158417.834407, 160749.983694, 163091.465145, 165441.984077,
+  167801.236994, 170168.903670, 172544.767987, 174928.494171, 177319.918928, 179718.624717, 182124.426670, 184537.056818,
+  186956.260737, 189381.869001, 191813.551321, 194251.107949, 196694.280093, 199142.910671, 201596.663557, 204055.453662,
+  206519.090402, 208987.322132, 211460.079506, 213936.962995, 216417.945039, 218902.907481, 221391.594168, 223883.857806,
+  226379.519239, 228878.478995, 231380.515508, 233885.422831, 236393.251620, 238903.734661, 241416.885791, 243932.462444,
+  246450.361381, 248970.497360, 251492.768859, 254017.059221, 256543.244066, 259071.366198, 261601.098326, 264132.472390,
+  266665.477130, 269200.003838, 271735.898262, 274273.128446, 276811.699617, 279351.334667, 281892.176977, 284434.126935,
+  286977.141855, 289520.993664, 292065.816286, 294611.452586, 297157.986206, 299705.252956, 302253.226506, 304801.859484,
+  307351.170990, 309901.148395, 312451.594894, 315002.530644, 317553.961670, 320105.951895, 322658.553440, 325211.299379,
+  327764.626147, 330318.311300, 332872.331048, 335426.696106, 337981.358498, 340536.275072, 343091.446562, 345646.972942,
+  348202.757347, 350758.693296, 353314.808579, 355871.203754, 358427.702337, 360984.491079, 363541.449032, 366098.493281,
+  368655.687887, 371213.030472, 373770.561193, 376328.233445, 378886.001706, 381443.937538, 384001.856504, 386559.857503,
+  389117.926075, 391676.219835, 394234.530250, 396792.812601, 399351.291660, 401909.794770, 404468.330377, 407026.964781,
+  409585.614496, 412144.407471, 414703.175858, 417261.980659, 419820.958264, 422379.863083, 424938.751186, 427497.927620,
+  430056.993719, 432616.066348, 435175.266190, 437734.382772, 440293.531277, 442852.711332, 445412.016423, 447971.502143,
+  450530.798765, 453090.167199, 455649.502729, 458208.951251, 460768.459694, 463328.053373, 465887.570627, 468447.239271,
+  471006.792818, 473566.544068, 476126.194227, 478685.887605, 481245.577573, 483805.348711, 486365.028302, 488924.844568,
+  491484.612666, 494044.321666, 496604.011864, 499163.799789, 501723.629793, 504283.493711, 506843.290195, 509403.157856,
+  511962.990320, 514522.941350, 517082.791953, 519642.848656, 522202.833761, 524762.686522, 527322.632758, 529882.529037,
+  532442.420851, 535002.422317, 537562.386309, 540122.357425, 542682.434171, 545242.468829, 547802.466694, 550362.492718,
+  552922.461624, 555482.539495, 558042.531431, 560602.467437, 563162.648952, 565722.750610, 568282.702877, 570842.810092,
+  573402.917604, 575962.999235, 578523.081937, 581083.162584, 583643.265455, 586203.287883, 588763.260931, 591323.292027,
+  593883.252456, 596443.273935, 599003.353442, 601563.492606, 604123.421135, 606683.438234, 609243.605174, 611803.638038,
+  614363.628593, 616923.690580, 619483.675256, 622043.744171, 624603.673246, 627163.577236, 629723.677960, 632283.670157,
+  634843.692958, 637403.576097, 639963.511752, 642523.482942, 645083.419327, 647643.330711, 650203.272467, 652763.290376,
+  655323.267522
+},
+// log K = 17
+{
+  94541.455324, 97026.403245, 99557.280165, 102134.178452, 104757.187033, 107426.362674, 110141.726029, 112903.278005,
+  115710.975458, 118564.785344, 121464.629896, 124410.377345, 127401.910279, 130439.087070, 133521.683723, 136649.499031,
+  139822.321627, 143039.848320, 146301.785984, 149607.844773, 152957.695333, 156350.945326, 159787.233040, 163266.157476,
+  166787.301371, 170350.201319, 173954.353438, 177599.302692, 181284.526367, 185009.507089, 188773.757457, 192576.638207,
+  196417.619925, 200296.089540, 204211.494530, 208163.221220, 212150.613086, 216173.056150, 220229.879308, 224320.462264,
+  228444.146459, 232600.284426, 236788.187347, 241007.264352, 245256.725247, 249535.938618, 253844.217048, 258180.876528,
+  262545.282090, 266936.686787, 271354.453768, 275797.958112, 280266.429404, 284759.298770, 289275.861948, 293815.386243,
+  298377.390708, 302961.087179, 307565.895452, 312191.173759, 316836.362522, 321500.721532, 326183.652758, 330884.723656,
+  335603.197219, 340338.617624, 345090.316855, 349857.860719, 354640.538362, 359438.001710, 364249.638745, 369074.940533,
+  373913.351331, 378764.438864, 383627.767868, 388502.854215, 393389.259584, 398286.483436, 403194.238263, 408111.948151,
+  413039.292303, 417975.909674, 422921.304084, 427875.195485, 432837.200959, 437807.062429, 442784.399966, 447768.726662,
+  452760.032525, 457757.794185, 462761.810279, 467771.737196, 472787.226777, 477808.283249, 482834.411114, 487865.521166,
+  492901.275392, 497941.616150, 502986.164281, 508034.749030, 513087.210697, 518143.298172, 523202.692981, 528265.454984,
+  533331.384435, 538400.437943, 543472.278711, 548546.790172, 553623.737338, 558703.047575, 563784.666463, 568868.524932,
+  573954.251207, 579041.935016, 584131.557779, 589222.910016, 594315.786547, 599410.314112, 604506.227555, 609603.610432,
+  614702.124262, 619801.918486, 624902.903258, 630005.085347, 635107.990499, 640211.847431, 645316.755403, 650422.424613,
+  655529.009211, 660636.166872, 665744.170495, 670852.806947, 675962.099161, 681072.042796, 686182.458370, 691293.269596,
+  696404.763335, 701516.635253, 706628.965299, 711741.575313, 716854.769083, 721968.220605, 727082.027784, 732196.114131,
+  737310.594653, 742425.339665, 747540.295793, 752655.505647, 757770.968129, 762886.636891, 768002.527720, 773118.559142,
+  778234.871441, 783351.273881, 788467.944809, 793584.599211, 798701.497594, 803818.484285, 808935.619316, 814053.067243,
+  819170.390229, 824287.946609, 829405.450988, 834523.054071, 839640.856519, 844758.800102, 849876.719924, 854994.831607,
+  860112.896788, 865231.116455, 870349.453898, 875467.910756, 880586.444976, 885705.102070, 890823.822978, 895942.513436,
+  901061.120048, 906179.716020, 911298.587548, 916417.583647, 921536.507373, 926655.570709, 931774.690027, 936893.879961,
+  942012.956398, 947132.142141, 952251.503479, 957370.815448, 962490.012685, 967609.521437, 972729.002837, 977848.339983,
+  982967.824448, 988087.392216, 993206.992691, 998326.635354, 1003446.304388, 1008565.985024, 1013685.706678, 1018805.581548,
+  1023925.239073, 1029045.157502, 1034165.035124, 1039284.843524, 1044404.676526, 1049524.620186, 1054644.441170, 1059764.409794,
+  1064884.318434, 1070004.338968, 1075124.268749, 1080244.471055, 1085364.528193, 1090484.511927, 1095604.595114, 1100724.598478,
+  1105844.666500, 1110964.804168, 1116084.864159, 1121204.874577, 1126325.062599, 1131445.152321, 1136565.301592, 1141685.638897,
+  1146805.806800, 1151925.790859, 1157045.932640, 1162166.112443, 1167286.221698, 1172406.273611, 1177526.581875, 1182646.832239,
+  1187766.882661, 1192886.884388, 1198006.962839, 1203127.084377, 1208246.959060, 1213366.976076, 1218486.763464, 1223606.758568,
+  1228726.721761, 1233846.593404, 1238966.815228, 1244086.865449, 1249206.854850, 1254326.996667, 1259447.092239, 1264567.198081,
+  1269687.499091, 1274807.474194, 1279927.244161, 1285047.273317, 1290167.090035, 1295287.035620, 1300406.888764, 1305526.667946,
+  1310646.540340
+},
+// log K = 18
+{
+  189083.688921, 194053.597973, 199115.342781, 204269.157017, 209515.195566, 214853.553887, 220284.295855, 225807.399998,
+  231422.826900, 237130.469824, 242930.139690, 248821.634191, 254804.718978, 260879.073467, 267044.289594, 273299.936993,
+  279645.549742, 286080.586199, 292604.521050, 299216.651567, 305916.376160, 312702.908374, 319575.536730, 326533.420485,
+  333575.639430, 340701.388339, 347909.713977, 355199.642852, 362570.108280, 370020.154080, 377548.652175, 385154.413997,
+  392836.374125, 400593.371525, 408424.172521, 416327.671123, 424302.406749, 432347.292674, 440461.014310, 448642.241683,
+  456889.603709, 465201.946081, 473577.749277, 482015.802141, 490514.770444, 499073.243760, 507689.794308, 516363.149230,
+  525091.902823, 533874.850510, 542710.434311, 551597.433198, 560534.241273, 569520.019389, 578553.034265, 587632.180111,
+  596756.084542, 605923.440522, 615133.037840, 624383.626242, 633673.841133, 643002.674316, 652368.578283, 661770.556501,
+  671207.605795, 680678.417768, 690181.818697, 699716.780471, 709282.084973, 718876.864175, 728499.951143, 738150.280049,
+  747827.053968, 757529.236597, 767255.784062, 777006.100829, 786779.065927, 796573.507429, 806388.828153, 816223.976258,
+  826078.608350, 835951.751956, 845842.493689, 855750.225870, 865674.190527, 875613.809349, 885568.425527, 895537.272219,
+  905519.763307, 915515.310346, 925523.353103, 935543.172648, 945574.292664, 955616.280042, 965668.428231, 975730.535479,
+  985802.055447, 995882.596628, 1005971.583108, 1016068.664992, 1026173.649961, 1036285.774891, 1046404.955770, 1056530.588186,
+  1066662.648243, 1076800.535542, 1086944.269881, 1097093.058652, 1107247.076183, 1117405.990557, 1127569.310917, 1137737.011345,
+  1147908.717700, 1158084.362020, 1168263.501029, 1178446.221907, 1188632.206842, 1198821.131431, 1209013.020373, 1219207.570786,
+  1229404.854893, 1239604.423801, 1249806.433263, 1260010.619993, 1270216.819018, 1280424.774173, 1290634.560600, 1300846.097203,
+  1311059.128630, 1321273.628402, 1331489.832488, 1341707.300774, 1351925.947953, 1362145.802925, 1372366.802414, 1382588.547896,
+  1392811.264776, 1403035.227449, 1413259.922280, 1423485.322819, 1433711.916106, 1443938.981360, 1454166.596984, 1464394.761679,
+  1474623.684212, 1484853.156422, 1495083.020146, 1505313.735890, 1515544.731764, 1525776.086634, 1536007.742073, 1546239.679012,
+  1556472.212370, 1566704.988763, 1576937.965495, 1587171.446007, 1597405.292186, 1607639.395274, 1617873.660775, 1628108.037273,
+  1638342.741799, 1648577.832828, 1658813.325573, 1669048.803784, 1679284.679963, 1689520.927615, 1699757.050010, 1709993.263665,
+  1720229.557663, 1730466.102090, 1740702.604009, 1750939.618504, 1761176.930182, 1771414.210528, 1781651.713742, 1791889.218503,
+  1802126.805234, 1812364.257333, 1822602.054007, 1832839.687792, 1843077.871973, 1853315.944762, 1863554.202066, 1873792.633125,
+  1884030.883830, 1894269.182711, 1904507.712050, 1914746.469507, 1924985.484753, 1935224.245956, 1945463.225215, 1955702.268883,
+  1965941.321722, 1976180.037370, 1986419.052784, 1996658.429579, 2006897.450283, 2017136.904971, 2027376.240039, 2037615.549295,
+  2047855.197425, 2058094.765906, 2068334.279787, 2078573.820075, 2088813.469730, 2099053.348620, 2109293.174176, 2119533.012704,
+  2129772.820132, 2140012.744709, 2150252.607739, 2160492.533017, 2170732.406422, 2180972.400110, 2191212.436319, 2201452.637894,
+  2211692.339292, 2221932.460057, 2232172.746036, 2242412.892705, 2252653.174824, 2262893.432271, 2273134.002270, 2283374.311008,
+  2293614.537881, 2303854.703732, 2314095.191374, 2324335.236186, 2334575.624526, 2344816.313269, 2355056.224171, 2365296.361285,
+  2375537.048184, 2385777.280284, 2396017.222502, 2406257.145546, 2416497.272190, 2426737.212210, 2436977.500856, 2447217.897426,
+  2457458.423605, 2467698.536697, 2477938.448096, 2488178.544726, 2498418.640441, 2508658.160374, 2518898.274852, 2529138.509345,
+  2539378.310538, 2549617.989497, 2559857.504886, 2570097.363128, 2580337.195261, 2590576.878833, 2600816.572159, 2611055.930087,
+  2621295.625004
+},
+// log K = 19
+{
+  378168.156126, 388107.986863, 398231.517433, 408539.178530, 419031.252008, 429707.954086, 440569.365328, 451615.642722,
+  462846.490174, 474261.790112, 485861.190310, 497644.259253, 509610.438730, 521759.095095, 534089.573442, 546600.981190,
+  559292.272697, 572162.409622, 585210.331052, 598434.669889, 611834.157416, 625407.271721, 639152.543961, 653068.262294,
+  667152.777393, 681404.241581, 695820.778241, 710400.501586, 725141.425404, 740041.558661, 755098.503731, 770310.107475,
+  785674.192935, 801188.297422, 816849.950298, 832656.623960, 848606.198026, 864696.151343, 880923.487841, 897285.841319,
+  913780.530043, 930405.131973, 947156.655545, 964032.974599, 981030.762278, 998147.499138, 1015380.659035, 1032727.149847,
+  1050184.616751, 1067750.330789, 1085421.716664, 1103195.771853, 1121069.796650, 1139041.324454, 1157107.373802, 1175265.377022,
+  1193513.201704, 1211847.872855, 1230267.133114, 1248768.293478, 1267348.634472, 1286005.987451, 1304738.187318, 1323542.421988,
+  1342416.458954, 1361357.807871, 1380364.530293, 1399434.095493, 1418564.587724, 1437754.116682, 1457000.254889, 1476301.275902,
+  1495654.977143, 1515059.332212, 1534512.821906, 1554013.130304, 1573558.585534, 1593147.244450, 1612777.978405, 1632448.600136,
+  1652157.704100, 1671904.308928, 1691686.012962, 1711501.716352, 1731349.849417, 1751229.053668, 1771138.423208, 1791076.196899,
+  1811041.338260, 1831032.177526, 1851048.091817, 1871088.096683, 1891150.317142, 1911234.619162, 1931339.423184, 1951463.947205,
+  1971606.994543, 1991767.606441, 2011945.852451, 2032140.098239, 2052349.709954, 2072574.274980, 2092812.387905, 2113063.537955,
+  2133327.750182, 2153603.787963, 2173890.933053, 2194188.225246, 2214496.333997, 2234814.120065, 2255140.753372, 2275476.675949,
+  2295820.689602, 2316172.220323, 2336530.258616, 2356894.932256, 2377266.633368, 2397645.173978, 2418028.419639, 2438417.586475,
+  2458812.422593, 2479211.306360, 2499615.253217, 2520023.375080, 2540436.051898, 2560851.967345, 2581271.569076, 2601694.318944,
+  2622120.693963, 2642549.418269, 2662981.070023, 2683415.520909, 2703852.500003, 2724291.876933, 2744733.213126, 2765177.477758,
+  2785623.320019, 2806070.985142, 2826520.234343, 2846971.203929, 2867423.303616, 2887876.396540, 2908331.739769, 2928788.290672,
+  2949245.276635, 2969704.620226, 2990164.657583, 3010625.841710, 3031087.855626, 3051550.619090, 3072014.642350, 3092479.163042,
+  3112944.278087, 3133410.101603, 3153876.719100, 3174343.414117, 3194810.517970, 3215278.762802, 3235747.174277, 3256216.390486,
+  3276685.652553, 3297155.394593, 3317625.911419, 3338096.547090, 3358567.567862, 3379038.637659, 3399510.649493, 3419983.382105,
+  3440456.231853, 3460929.496047, 3481403.703904, 3501877.874941, 3522351.898403, 3542825.527074, 3563300.038829, 3583775.102749,
+  3604249.896903, 3624724.967423, 3645201.027293, 3665676.534460, 3686152.061202, 3706628.308333, 3727104.271827, 3747580.803335,
+  3768057.710876, 3788534.660048, 3809011.720864, 3829489.164523, 3849967.204462, 3870445.043912, 3890923.080446, 3911400.767281,
+  3931879.061622, 3952357.218696, 3972836.284463, 3993314.516935, 4013793.278282, 4034272.661280, 4054751.814018, 4075230.503334,
+  4095708.557968, 4116187.919814, 4136667.714093, 4157147.152355, 4177627.302553, 4198107.383728, 4218587.174918, 4239066.631662,
+  4259545.994167, 4280026.100110, 4300506.380609, 4320986.617208, 4341466.696312, 4361945.995732, 4382426.527708, 4402907.058309,
+  4423386.556967, 4443867.498542, 4464348.284664, 4484828.344391, 4505309.026083, 4525789.774619, 4546270.396129, 4566750.818685,
+  4587231.362126, 4607711.356124, 4628191.883250, 4648673.308942, 4669153.953372, 4689634.118530, 4710114.637510, 4730595.217429,
+  4751075.358693, 4771555.492921, 4792035.313408, 4812516.057037, 4832995.761283, 4853475.977619, 4873956.732160, 4894436.879314,
+  4914917.149285, 4935397.069776, 4955877.550351, 4976357.852728, 4996838.246842, 5017318.023169, 5037798.008344, 5058278.290403,
+  5078757.968956, 5099237.419344, 5119716.759591, 5140195.935830, 5160675.636320, 5181154.980843, 5201633.780713, 5222113.031789,
+  5242591.881577
+},
+// log K = 20
+{
+  756337.090537, 776216.744995, 796463.785646, 817079.035290, 838063.221300, 859416.678362, 881139.576270, 903232.151726,
+  925693.940877, 948524.600346, 971723.442348, 995289.632784, 1019222.059448, 1043519.402588, 1068180.163485, 1093202.854610,
+  1118585.515820, 1144326.031845, 1170421.828997, 1196870.617641, 1223669.453959, 1250815.679160, 1278306.298944, 1306137.726017,
+  1334306.782451, 1362809.672011, 1391643.048887, 1420802.665486, 1450284.419390, 1480084.556504, 1510198.362531, 1540621.709038,
+  1571349.988582, 1602377.885766, 1633700.853485, 1665314.443974, 1697213.607484, 1729393.242718, 1761847.799154, 1794572.464804,
+  1827562.002973, 1860811.213417, 1894314.858503, 1928067.079352, 1962062.774535, 1996296.400592, 2030762.354026, 2065455.981656,
+  2100370.818573, 2135502.224529, 2170844.722018, 2206392.452100, 2242140.795895, 2278083.565225, 2314215.470175, 2350532.238358,
+  2387028.028531, 2423697.739702, 2460535.617283, 2497537.908190, 2534699.627071, 2572014.508248, 2609478.240813, 2647086.878671,
+  2684834.738503, 2722717.495987, 2760730.659130, 2798869.445379, 2837130.937688, 2875509.782410, 2914002.086936, 2952603.395700,
+  2991310.971904, 3030119.629799, 3069026.891217, 3108027.543585, 3147118.485322, 3186296.724369, 3225557.893026, 3264899.405158,
+  3304317.806643, 3343809.532357, 3383372.875494, 3423003.239113, 3462699.855290, 3502458.588120, 3542276.984489, 3582151.461666,
+  3622080.455509, 3662062.087297, 3702093.774268, 3742173.351913, 3782298.542025, 3822466.599295, 3862676.059061, 3902925.301466,
+  3943212.313188, 3983534.655139, 4023890.682932, 4064278.948819, 4104698.644146, 4145147.867936, 4185624.200116, 4226127.607823,
+  4266655.471909, 4307207.261933, 4347781.577688, 4388376.075665, 4428991.084860, 4469625.512791, 4510278.322018, 4550949.135583,
+  4591635.061142, 4632337.290243, 4673054.034167, 4713783.999697, 4754527.016825, 4795282.199016, 4836049.304645, 4876828.217063,
+  4917617.432795, 4958416.033283, 4999223.332994, 5040040.200620, 5080865.836319, 5121698.006953, 5162537.663489, 5203383.861562,
+  5244235.368756, 5285093.542054, 5325957.172545, 5366826.006832, 5407700.602295, 5448579.444589, 5489462.237514, 5530348.767163,
+  5571240.286944, 5612135.490023, 5653033.590430, 5693935.090249, 5734840.012969, 5775747.413811, 5816658.159197, 5857571.727542,
+  5898486.810748, 5939403.918361, 5980324.159335, 6021245.922584, 6062167.462275, 6103093.061724, 6144019.021028, 6184948.477708,
+  6225878.111073, 6266809.926593, 6307741.380274, 6348674.587241, 6389609.653586, 6430546.024311, 6471482.661256, 6512419.658468,
+  6553358.313587, 6594299.462793, 6635239.780670, 6676180.809115, 6717122.614963, 6758064.640997, 6799009.935719, 6839954.567726,
+  6880900.950276, 6921847.965357, 6962794.580261, 7003742.234176, 7044691.155209, 7085639.223984, 7126587.984374, 7167538.597843,
+  7208487.508674, 7249437.307369, 7290389.448350, 7331341.752887, 7372292.872774, 7413245.082344, 7454198.512431, 7495151.878502,
+  7536105.815461, 7577059.755263, 7618013.175400, 7658967.579066, 7699922.475951, 7740878.015144, 7781833.511721, 7822789.986990,
+  7863745.987999, 7904702.500036, 7945659.757785, 7986615.385020, 8027573.168319, 8068531.917672, 8109489.438412, 8150447.742963,
+  8191403.350221, 8232362.807587, 8273321.255913, 8314281.043410, 8355240.292724, 8396199.214301, 8437156.471990, 8478114.862372,
+  8519073.457296, 8560034.832965, 8600993.512814, 8641954.368755, 8682914.667061, 8723875.362809, 8764837.809961, 8805797.690078,
+  8846758.789178, 8887720.671501, 8928680.126482, 8969640.467726, 9010601.160006, 9051561.769905, 9092522.487783, 9133484.594542,
+  9174446.122191, 9215406.809237, 9256366.837569, 9297328.747584, 9338288.901510, 9379250.948160, 9420212.503729, 9461173.228802,
+  9502133.928845, 9543094.489131, 9584055.513956, 9625015.618922, 9665975.737758, 9706937.403885, 9747896.035874, 9788856.165941,
+  9829816.994627, 9870777.228642, 9911737.327683, 9952698.937964, 9993660.635143, 10034620.747904, 10075579.526207, 10116539.739394,
+  10157498.625700, 10198458.468474, 10239418.486051, 10280376.431741, 10321335.494132, 10362295.312860, 10403254.402980, 10444214.519042,
+  10485173.027410
+},
+// log K = 21
+{
+  1512674.959317, 1552434.197368, 1592928.155075, 1634158.593696, 1676126.959176, 1718834.017325, 1762279.726817, 1806464.610827,
+  1851388.162884, 1897049.337397, 1943446.849678, 1990579.312415, 2038444.063365, 2087038.766553, 2136360.248426, 2186405.470599,
+  2237170.640174, 2288651.015887, 2340842.752937, 2393739.986364, 2447337.958039, 2501630.515180, 2556611.361814, 2612274.289376,
+  2668612.210872, 2725618.064986, 2783284.549430, 2841603.566294, 2900566.895724, 2960167.093438, 3020394.610851, 3081240.447614,
+  3142696.078921, 3204751.105703, 3267397.808234, 3330624.834407, 3394422.872136, 3458781.966184, 3523690.805899, 3589140.856689,
+  3655119.540256, 3721617.907853, 3788624.486645, 3856129.175227, 3924119.940026, 3992586.658818, 4061519.544440, 4130906.734075,
+  4200737.500257, 4271000.909057, 4341685.842431, 4412781.306685, 4484277.199691, 4556162.287552, 4628426.495554, 4701059.642462,
+  4774050.909933, 4847389.117100, 4921065.536469, 4995068.178578, 5069389.529141, 5144019.928509, 5218947.440841, 5294164.568050,
+  5369660.364896, 5445425.915267, 5521454.727645, 5597734.642433, 5674257.411718, 5751014.463556, 5827999.822090, 5905203.759793,
+  5982617.522812, 6060234.561845, 6138048.162353, 6216049.451496, 6294232.713331, 6372588.108087, 6451109.903764, 6529792.611180,
+  6608629.912527, 6687615.097432, 6766740.086064, 6846000.116450, 6925391.839995, 7004908.093083, 7084544.127544, 7164293.822313,
+  7244153.392582, 7324115.814384, 7404179.031163, 7484336.380737, 7564586.128001, 7644922.247694, 7725340.819662, 7805840.541693,
+  7886412.416887, 7967054.301986, 8047764.087248, 8128539.447460, 8209376.477465, 8290273.766928, 8371225.884580, 8452230.183873,
+  8533283.959811, 8614388.863382, 8695539.350564, 8776730.375553, 8857961.046732, 8939230.757845, 9020535.369667, 9101877.322129,
+  9183251.420601, 9264654.743697, 9346087.109813, 9427548.372707, 9509034.441574, 9590547.753266, 9672081.682514, 9753639.600816,
+  9835216.139507, 9916813.989591, 9998428.682824, 10080058.312759, 10161706.673718, 10243372.018016, 10325050.507034, 10406744.529470,
+  10488447.296300, 10570163.109725, 10651893.512546, 10733632.712879, 10815382.384056, 10897140.573621, 10978907.646650, 11060681.383125,
+  11142464.138730, 11224254.549571, 11306053.275368, 11387858.162207, 11469666.329635, 11551480.634292, 11633302.025569, 11715128.710512,
+  11796961.403328, 11878795.670820, 11960634.652330, 12042477.348124, 12124323.950451, 12206172.128053, 12288028.811447, 12369886.306570,
+  12451744.123236, 12533607.183832, 12615473.707341, 12697338.733240, 12779207.987968, 12861080.507718, 12942957.260823, 13024834.007606,
+  13106715.252745, 13188597.806270, 13270478.933580, 13352362.482281, 13434247.263704, 13516133.477076, 13598020.834213, 13679909.264273,
+  13761799.592878, 13843689.299819, 13925583.582108, 14007479.173806, 14089371.780210, 14171268.621526, 14253167.561983, 14335070.527750,
+  14416971.620797, 14498873.784867, 14580778.789301, 14662681.662290, 14744586.311912, 14826493.286700, 14908400.702409, 14990305.605573,
+  15072214.379125, 15154123.120545, 15236031.901296, 15317939.866920, 15399849.993076, 15481761.939120, 15563672.325616, 15645585.312945,
+  15727497.406318, 15809411.780271, 15891325.875461, 15973239.145687, 16055155.936424, 16137069.783844, 16218984.663574, 16300899.732679,
+  16382815.849682, 16464730.053474, 16546645.675799, 16628563.444017, 16710481.533611, 16792400.824636, 16874319.061886, 16956235.859045,
+  17038151.394583, 17120072.629220, 17201992.200096, 17283914.365655, 17365836.161356, 17447756.880495, 17529681.545942, 17611601.675003,
+  17693522.079185, 17775441.939609, 17857361.777214, 17939284.144858, 18021205.445833, 18103125.675947, 18185051.132189, 18266974.391463,
+  18348894.689783, 18430817.526979, 18512736.992850, 18594657.001222, 18676577.636496, 18758497.698999, 18840421.426602, 18922345.360722,
+  19004266.906157, 19086186.184618, 19168105.320223, 19250029.336282, 19331950.350200, 19413874.580826, 19495794.431194, 19577715.745558,
+  19659638.264924, 19741557.363469, 19823478.431258, 19905400.862401, 19987318.682409, 20069238.585823, 20151157.549449, 20233076.181910,
+  20314994.521260, 20396915.069140, 20478836.108717, 20560753.915833, 20642672.570035, 20724588.867440, 20806507.038063, 20888426.370670,
+  20970344.006053
+}
+};
+
+template<typename A>
+const double* const CompositeInterpolationXTable<A>::get_x_arr(const int logK) {
+  if (logK < HllUtil<A>::MIN_LOG_K || logK > HllUtil<A>::MAX_LOG_K) {
+    throw std::invalid_argument("logK must be in range [" + std::to_string(HllUtil<A>::MIN_LOG_K)
+                                + ", " + std::to_string(HllUtil<A>::MAX_LOG_K) + "]. Found: "
+                                + std::to_string(logK));
+  }
+  return xArr[logK - HllUtil<A>::MIN_LOG_K];
+}
+
+}
+
+#endif // _COMPOSITEINTERPOLATIONXTABLE_INTERNAL_HPP_
\ No newline at end of file
diff --git a/be/src/thirdparty/datasketches/CompositeInterpolationXTable.hpp b/be/src/thirdparty/datasketches/CompositeInterpolationXTable.hpp
new file mode 100644
index 0000000..67563c5
--- /dev/null
+++ b/be/src/thirdparty/datasketches/CompositeInterpolationXTable.hpp
@@ -0,0 +1,40 @@
+/*
+ * 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 _COMPOSITEINTERPOLATIONXTABLE_HPP_
+#define _COMPOSITEINTERPOLATIONXTABLE_HPP_
+
+#include <memory>
+
+namespace datasketches {
+
+template<typename A = std::allocator<char>>
+class CompositeInterpolationXTable {
+  public:
+    static const int get_y_stride(int logK);
+
+    static const double* const get_x_arr(int logK);
+    static const int get_x_arr_length(int logK);
+};
+
+}
+
+#include "CompositeInterpolationXTable-internal.hpp"
+
+#endif /* _COMPOSITEINTERPOLATIONXTABLE_HPP_ */
\ No newline at end of file
diff --git a/be/src/thirdparty/datasketches/CouponHashSet-internal.hpp b/be/src/thirdparty/datasketches/CouponHashSet-internal.hpp
new file mode 100644
index 0000000..84c8199
--- /dev/null
+++ b/be/src/thirdparty/datasketches/CouponHashSet-internal.hpp
@@ -0,0 +1,290 @@
+/*
+ * 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 _COUPONHASHSET_INTERNAL_HPP_
+#define _COUPONHASHSET_INTERNAL_HPP_
+
+#include "CouponHashSet.hpp"
+
+#include <cstring>
+#include <exception>
+
+namespace datasketches {
+
+template<typename A>
+static int find(const int* array, const int lgArrInts, const int coupon);
+
+template<typename A>
+CouponHashSet<A>::CouponHashSet(const int lgConfigK, const target_hll_type tgtHllType)
+  : CouponList<A>(lgConfigK, tgtHllType, hll_mode::SET)
+{
+  if (lgConfigK <= 7) {
+    throw std::invalid_argument("CouponHashSet must be initialized with lgConfigK > 7. Found: "
+                                + std::to_string(lgConfigK));
+  }
+}
+
+template<typename A>
+CouponHashSet<A>::CouponHashSet(const CouponHashSet<A>& that)
+  : CouponList<A>(that) {}
+
+template<typename A>
+CouponHashSet<A>::CouponHashSet(const CouponHashSet<A>& that, const target_hll_type tgtHllType)
+  : CouponList<A>(that, tgtHllType) {}
+
+template<typename A>
+CouponHashSet<A>::~CouponHashSet() {}
+
+template<typename A>
+std::function<void(HllSketchImpl<A>*)> CouponHashSet<A>::get_deleter() const {
+  return [](HllSketchImpl<A>* ptr) {
+    CouponHashSet<A>* chs = static_cast<CouponHashSet<A>*>(ptr);
+    chs->~CouponHashSet();
+    chsAlloc().deallocate(chs, 1);
+  };
+}
+
+template<typename A>
+CouponHashSet<A>* CouponHashSet<A>::newSet(const void* bytes, size_t len) {
+  if (len < HllUtil<A>::HASH_SET_INT_ARR_START) { // hard-coded 
+    throw std::invalid_argument("Input data length insufficient to hold CouponHashSet");
+  }
+
+  const uint8_t* data = static_cast<const uint8_t*>(bytes);
+  if (data[HllUtil<A>::PREAMBLE_INTS_BYTE] != HllUtil<A>::HASH_SET_PREINTS) {
+    throw std::invalid_argument("Incorrect number of preInts in input stream");
+  }
+  if (data[HllUtil<A>::SER_VER_BYTE] != HllUtil<A>::SER_VER) {
+    throw std::invalid_argument("Wrong ser ver in input stream");
+  }
+  if (data[HllUtil<A>::FAMILY_BYTE] != HllUtil<A>::FAMILY_ID) {
+    throw std::invalid_argument("Input stream is not an HLL sketch");
+  }
+
+  const hll_mode mode = HllSketchImpl<A>::extractCurMode(data[HllUtil<A>::MODE_BYTE]);
+  if (mode != SET) {
+    throw std::invalid_argument("Calling set construtor with non-set mode data");
+  }
+
+  const target_hll_type tgtHllType = HllSketchImpl<A>::extractTgtHllType(data[HllUtil<A>::MODE_BYTE]);
+
+  const int lgK = data[HllUtil<A>::LG_K_BYTE];
+  if (lgK <= 7) {
+    throw std::invalid_argument("Attempt to deserialize invalid CouponHashSet with lgConfigK <= 7. Found: "
+                                + std::to_string(lgK));
+  }   
+  int lgArrInts = data[HllUtil<A>::LG_ARR_BYTE];
+  const bool compactFlag = ((data[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::COMPACT_FLAG_MASK) ? true : false);
+
+  int couponCount;
+  std::memcpy(&couponCount, data + HllUtil<A>::HASH_SET_COUNT_INT, sizeof(couponCount));
+  if (lgArrInts < HllUtil<A>::LG_INIT_SET_SIZE) { 
+    lgArrInts = HllUtil<A>::computeLgArrInts(SET, couponCount, lgK);
+  }
+  // Don't set couponCount in sketch here;
+  // we'll set later if updatable, and increment with updates if compact
+  const int couponsInArray = (compactFlag ? couponCount : (1 << lgArrInts));
+  const size_t expectedLength = HllUtil<A>::HASH_SET_INT_ARR_START + (couponsInArray * sizeof(int));
+  if (len < expectedLength) {
+    throw std::invalid_argument("Byte array too short for sketch. Expected " + std::to_string(expectedLength)
+                                + ", found: " + std::to_string(len));
+  }
+
+  CouponHashSet<A>* sketch = new (chsAlloc().allocate(1)) CouponHashSet<A>(lgK, tgtHllType);
+  sketch->putOutOfOrderFlag(true);
+
+  if (compactFlag) {
+    const uint8_t* curPos = data + HllUtil<A>::HASH_SET_INT_ARR_START;
+    int coupon;
+    for (int i = 0; i < couponCount; ++i, curPos += sizeof(coupon)) {
+      std::memcpy(&coupon, curPos, sizeof(coupon));
+      sketch->couponUpdate(coupon);
+    }
+  } else {
+    int* oldArr = sketch->couponIntArr;
+    const size_t oldArrLen = 1 << sketch->lgCouponArrInts;
+    sketch->lgCouponArrInts = lgArrInts;
+    typedef typename std::allocator_traits<A>::template rebind_alloc<int> intAlloc;
+    sketch->couponIntArr = intAlloc().allocate(1 << lgArrInts);
+    sketch->couponCount = couponCount;
+    // only need to read valid coupons, unlike in stream case
+    std::memcpy(sketch->couponIntArr,
+                data + HllUtil<A>::HASH_SET_INT_ARR_START,
+                couponCount * sizeof(int));
+    intAlloc().deallocate(oldArr, oldArrLen);
+  }
+
+  return sketch;
+}
+
+template<typename A>
+CouponHashSet<A>* CouponHashSet<A>::newSet(std::istream& is) {
+  uint8_t listHeader[8];
+  is.read((char*)listHeader, 8 * sizeof(uint8_t));
+
+  if (listHeader[HllUtil<A>::PREAMBLE_INTS_BYTE] != HllUtil<A>::HASH_SET_PREINTS) {
+    throw std::invalid_argument("Incorrect number of preInts in input stream");
+  }
+  if (listHeader[HllUtil<A>::SER_VER_BYTE] != HllUtil<A>::SER_VER) {
+    throw std::invalid_argument("Wrong ser ver in input stream");
+  }
+  if (listHeader[HllUtil<A>::FAMILY_BYTE] != HllUtil<A>::FAMILY_ID) {
+    throw std::invalid_argument("Input stream is not an HLL sketch");
+  }
+
+  hll_mode mode = HllSketchImpl<A>::extractCurMode(listHeader[HllUtil<A>::MODE_BYTE]);
+  if (mode != SET) {
+    throw std::invalid_argument("Calling set construtor with non-set mode data");
+  }
+
+  target_hll_type tgtHllType = HllSketchImpl<A>::extractTgtHllType(listHeader[HllUtil<A>::MODE_BYTE]);
+
+  const int lgK = listHeader[HllUtil<A>::LG_K_BYTE];
+  if (lgK <= 7) {
+    throw std::invalid_argument("Attempt to deserialize invalid CouponHashSet with lgConfigK <= 7. Found: "
+                                + std::to_string(lgK));
+  }
+  int lgArrInts = listHeader[HllUtil<A>::LG_ARR_BYTE];
+  const bool compactFlag = ((listHeader[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::COMPACT_FLAG_MASK) ? true : false);
+
+  int couponCount;
+  is.read((char*)&couponCount, sizeof(couponCount));
+  if (lgArrInts < HllUtil<A>::LG_INIT_SET_SIZE) { 
+    lgArrInts = HllUtil<A>::computeLgArrInts(SET, couponCount, lgK);
+  }
+
+  CouponHashSet<A>* sketch = new (chsAlloc().allocate(1)) CouponHashSet<A>(lgK, tgtHllType);
+  sketch->putOutOfOrderFlag(true);
+
+  // Don't set couponCount here;
+  // we'll set later if updatable, and increment with updates if compact
+  if (compactFlag) {
+    for (int i = 0; i < couponCount; ++i) {
+      int coupon;
+      is.read((char*)&coupon, sizeof(coupon));
+      sketch->couponUpdate(coupon);
+    }
+  } else {
+    int* oldArr = sketch->couponIntArr;
+    const size_t oldArrLen = 1 << sketch->lgCouponArrInts;
+    sketch->lgCouponArrInts = lgArrInts;
+    typedef typename std::allocator_traits<A>::template rebind_alloc<int> intAlloc;
+    sketch->couponIntArr = intAlloc().allocate(1 << lgArrInts);
+    sketch->couponCount = couponCount;
+    // for stream processing, read entire list so read pointer ends up set correctly
+    is.read((char*)sketch->couponIntArr, (1 << sketch->lgCouponArrInts) * sizeof(int));
+    intAlloc().deallocate(oldArr, oldArrLen);
+  } 
+
+  return sketch;
+}
+
+template<typename A>
+CouponHashSet<A>* CouponHashSet<A>::copy() const {
+  return new (chsAlloc().allocate(1)) CouponHashSet<A>(*this);
+}
+
+template<typename A>
+CouponHashSet<A>* CouponHashSet<A>::copyAs(const target_hll_type tgtHllType) const {
+  return new (chsAlloc().allocate(1)) CouponHashSet<A>(*this, tgtHllType);
+}
+
+template<typename A>
+HllSketchImpl<A>* CouponHashSet<A>::couponUpdate(int coupon) {
+  const int index = find<A>(this->couponIntArr, this->lgCouponArrInts, coupon);
+  if (index >= 0) {
+    return this; // found duplicate, ignore
+  }
+  this->couponIntArr[~index] = coupon; // found empty
+  ++this->couponCount;
+  if (checkGrowOrPromote()) {
+    return this->promoteHeapListOrSetToHll(*this);
+  }
+  return this;
+}
+
+template<typename A>
+int CouponHashSet<A>::getMemDataStart() const {
+  return HllUtil<A>::HASH_SET_INT_ARR_START;
+}
+
+template<typename A>
+int CouponHashSet<A>::getPreInts() const {
+  return HllUtil<A>::HASH_SET_PREINTS;
+}
+
+template<typename A>
+bool CouponHashSet<A>::checkGrowOrPromote() {
+  if ((HllUtil<A>::RESIZE_DENOM * this->couponCount) > (HllUtil<A>::RESIZE_NUMER * (1 << this->lgCouponArrInts))) {
+    if (this->lgCouponArrInts == (this->lgConfigK - 3)) { // at max size
+      return true; // promote to HLL
+    }
+    int tgtLgCoupArrSize = this->lgCouponArrInts + 1;
+    growHashSet(this->lgCouponArrInts, tgtLgCoupArrSize);
+  }
+  return false;
+}
+
+template<typename A>
+void CouponHashSet<A>::growHashSet(const int srcLgCoupArrSize, const int tgtLgCoupArrSize) {
+  const int tgtLen = 1 << tgtLgCoupArrSize;
+  typedef typename std::allocator_traits<A>::template rebind_alloc<int> intAlloc;
+  int* tgtCouponIntArr = intAlloc().allocate(tgtLen);
+  std::fill(tgtCouponIntArr, tgtCouponIntArr + tgtLen, 0);
+
+  const int srcLen = 1 << srcLgCoupArrSize;
+  for (int i = 0; i < srcLen; ++i) { // scan existing array for non-zero values
+    const int fetched = this->couponIntArr[i];
+    if (fetched != HllUtil<A>::EMPTY) {
+      const int idx = find<A>(tgtCouponIntArr, tgtLgCoupArrSize, fetched); // search TGT array
+      if (idx < 0) { // found EMPTY
+        tgtCouponIntArr[~idx] = fetched; // insert
+        continue;
+      }
+      throw std::runtime_error("Error: Found duplicate coupon");
+    }
+  }
+
+  intAlloc().deallocate(this->couponIntArr, 1 << this->lgCouponArrInts);
+  this->couponIntArr = tgtCouponIntArr;
+  this->lgCouponArrInts = tgtLgCoupArrSize;
+}
+
+template<typename A>
+static int find(const int* array, const int lgArrInts, const int coupon) {
+  const int arrMask = (1 << lgArrInts) - 1;
+  int probe = coupon & arrMask;
+  const int loopIndex = probe;
+  do {
+    const int couponAtIdx = array[probe];
+    if (couponAtIdx == HllUtil<A>::EMPTY) {
+      return ~probe; //empty
+    }
+    else if (coupon == couponAtIdx) {
+      return probe; //duplicate
+    }
+    const int stride = ((coupon & HllUtil<A>::KEY_MASK_26) >> lgArrInts) | 1;
+    probe = (probe + stride) & arrMask;
+  } while (probe != loopIndex);
+  throw std::invalid_argument("Key not found and no empty slots!");
+}
+
+}
+
+#endif // _COUPONHASHSET_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/CouponHashSet.hpp b/be/src/thirdparty/datasketches/CouponHashSet.hpp
new file mode 100644
index 0000000..7aaffc3
--- /dev/null
+++ b/be/src/thirdparty/datasketches/CouponHashSet.hpp
@@ -0,0 +1,59 @@
+/*
+ * 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 _COUPONHASHSET_HPP_
+#define _COUPONHASHSET_HPP_
+
+#include "CouponList.hpp"
+
+namespace datasketches {
+
+template<typename A = std::allocator<char>>
+class CouponHashSet : public CouponList<A> {
+  public:
+    static CouponHashSet* newSet(const void* bytes, size_t len);
+    static CouponHashSet* newSet(std::istream& is);
+    explicit CouponHashSet(int lgConfigK, target_hll_type tgtHllType);
+    explicit CouponHashSet(const CouponHashSet& that, target_hll_type tgtHllType);
+    explicit CouponHashSet(const CouponHashSet& that);
+
+    virtual ~CouponHashSet();
+    virtual std::function<void(HllSketchImpl<A>*)> get_deleter() const;
+
+  protected:
+    
+    virtual CouponHashSet* copy() const;
+    virtual CouponHashSet* copyAs(target_hll_type tgtHllType) const;
+
+    virtual HllSketchImpl<A>* couponUpdate(int coupon);
+
+    virtual int getMemDataStart() const;
+    virtual int getPreInts() const;
+
+    friend class HllSketchImplFactory<A>;
+
+  private:
+    typedef typename std::allocator_traits<A>::template rebind_alloc<CouponHashSet<A>> chsAlloc;
+    bool checkGrowOrPromote();
+    void growHashSet(int srcLgCoupArrSize, int tgtLgCoupArrSize);
+};
+
+}
+
+#endif /* _COUPONHASHSET_HPP_ */
diff --git a/be/src/thirdparty/datasketches/CouponList-internal.hpp b/be/src/thirdparty/datasketches/CouponList-internal.hpp
new file mode 100644
index 0000000..f9d3ca0
--- /dev/null
+++ b/be/src/thirdparty/datasketches/CouponList-internal.hpp
@@ -0,0 +1,413 @@
+/*
+ * 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 _COUPONLIST_INTERNAL_HPP_
+#define _COUPONLIST_INTERNAL_HPP_
+
+#include "CouponList.hpp"
+#include "CubicInterpolation.hpp"
+#include "HllUtil.hpp"
+
+#include <algorithm>
+#include <cmath>
+
+namespace datasketches {
+
+template<typename A>
+CouponList<A>::CouponList(const int lgConfigK, const target_hll_type tgtHllType, const hll_mode mode)
+  : HllSketchImpl<A>(lgConfigK, tgtHllType, mode, false) {
+    if (mode == hll_mode::LIST) {
+      lgCouponArrInts = HllUtil<A>::LG_INIT_LIST_SIZE;
+      oooFlag = false;
+    } else { // mode == SET
+      lgCouponArrInts = HllUtil<A>::LG_INIT_SET_SIZE;
+      oooFlag = true;
+    }
+    const int arrayLen = 1 << lgCouponArrInts;
+    typedef typename std::allocator_traits<A>::template rebind_alloc<int> intAlloc;
+    couponIntArr = intAlloc().allocate(arrayLen);
+    std::fill(couponIntArr, couponIntArr + arrayLen, 0);
+    couponCount = 0;
+}
+
+template<typename A>
+CouponList<A>::CouponList(const CouponList& that)
+  : HllSketchImpl<A>(that.lgConfigK, that.tgtHllType, that.mode, false),
+    lgCouponArrInts(that.lgCouponArrInts),
+    couponCount(that.couponCount),
+    oooFlag(that.oooFlag) {
+
+  const int numItems = 1 << lgCouponArrInts;
+  typedef typename std::allocator_traits<A>::template rebind_alloc<int> intAlloc;
+  couponIntArr = intAlloc().allocate(numItems);
+  std::copy(that.couponIntArr, that.couponIntArr + numItems, couponIntArr);
+}
+
+template<typename A>
+CouponList<A>::CouponList(const CouponList& that, const target_hll_type tgtHllType)
+  : HllSketchImpl<A>(that.lgConfigK, tgtHllType, that.mode, false),
+    lgCouponArrInts(that.lgCouponArrInts),
+    couponCount(that.couponCount),
+    oooFlag(that.oooFlag) {
+
+  const int numItems = 1 << lgCouponArrInts;
+  typedef typename std::allocator_traits<A>::template rebind_alloc<int> intAlloc;
+  couponIntArr = intAlloc().allocate(numItems);
+  std::copy(that.couponIntArr, that.couponIntArr + numItems, couponIntArr);
+}
+
+template<typename A>
+CouponList<A>::~CouponList() {
+  typedef typename std::allocator_traits<A>::template rebind_alloc<int> intAlloc;
+  intAlloc().deallocate(couponIntArr, 1 << lgCouponArrInts);
+}
+
+template<typename A>
+std::function<void(HllSketchImpl<A>*)> CouponList<A>::get_deleter() const {
+  return [](HllSketchImpl<A>* ptr) {
+    CouponList<A>* cl = static_cast<CouponList<A>*>(ptr);
+    cl->~CouponList();
+    clAlloc().deallocate(cl, 1);
+  };
+}
+
+template<typename A>
+CouponList<A>* CouponList<A>::copy() const {
+  return new (clAlloc().allocate(1)) CouponList<A>(*this);
+}
+
+template<typename A>
+CouponList<A>* CouponList<A>::copyAs(target_hll_type tgtHllType) const {
+  return new (clAlloc().allocate(1)) CouponList<A>(*this, tgtHllType);
+}
+
+template<typename A>
+CouponList<A>* CouponList<A>::newList(const void* bytes, size_t len) {
+  if (len < HllUtil<A>::LIST_INT_ARR_START) {
+    throw std::invalid_argument("Input data length insufficient to hold CouponHashSet");
+  }
+
+  const uint8_t* data = static_cast<const uint8_t*>(bytes);
+  if (data[HllUtil<A>::PREAMBLE_INTS_BYTE] != HllUtil<A>::LIST_PREINTS) {
+    throw std::invalid_argument("Incorrect number of preInts in input stream");
+  }
+  if (data[HllUtil<A>::SER_VER_BYTE] != HllUtil<A>::SER_VER) {
+    throw std::invalid_argument("Wrong ser ver in input stream");
+  }
+  if (data[HllUtil<A>::FAMILY_BYTE] != HllUtil<A>::FAMILY_ID) {
+    throw std::invalid_argument("Input stream is not an HLL sketch");
+  }
+
+  hll_mode mode = HllSketchImpl<A>::extractCurMode(data[HllUtil<A>::MODE_BYTE]);
+  if (mode != LIST) {
+    throw std::invalid_argument("Calling set construtor with non-list mode data");
+  }
+
+  target_hll_type tgtHllType = HllSketchImpl<A>::extractTgtHllType(data[HllUtil<A>::MODE_BYTE]);
+
+  const int lgK = data[HllUtil<A>::LG_K_BYTE];
+  const bool compact = ((data[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::COMPACT_FLAG_MASK) ? true : false);
+  const bool oooFlag = ((data[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::OUT_OF_ORDER_FLAG_MASK) ? true : false);
+  const bool emptyFlag = ((data[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::EMPTY_FLAG_MASK) ? true : false);
+
+  const int couponCount = data[HllUtil<A>::LIST_COUNT_BYTE];
+  const int couponsInArray = (compact ? couponCount : (1 << HllUtil<A>::computeLgArrInts(LIST, couponCount, lgK)));
+  const size_t expectedLength = HllUtil<A>::LIST_INT_ARR_START + (couponsInArray * sizeof(int));
+  if (len < expectedLength) {
+    throw std::invalid_argument("Byte array too short for sketch. Expected " + std::to_string(expectedLength)
+                                + ", found: " + std::to_string(len));
+  }
+
+  CouponList<A>* sketch = new (clAlloc().allocate(1)) CouponList<A>(lgK, tgtHllType, mode);
+  sketch->couponCount = couponCount;
+  sketch->putOutOfOrderFlag(oooFlag); // should always be false for LIST
+
+  if (!emptyFlag) {
+    // only need to read valid coupons, unlike in stream case
+    std::memcpy(sketch->couponIntArr, data + HllUtil<A>::LIST_INT_ARR_START, couponCount * sizeof(int));
+  }
+  
+  return sketch;
+}
+
+template<typename A>
+CouponList<A>* CouponList<A>::newList(std::istream& is) {
+  uint8_t listHeader[8];
+  is.read((char*)listHeader, 8 * sizeof(uint8_t));
+
+  if (listHeader[HllUtil<A>::PREAMBLE_INTS_BYTE] != HllUtil<A>::LIST_PREINTS) {
+    throw std::invalid_argument("Incorrect number of preInts in input stream");
+  }
+  if (listHeader[HllUtil<A>::SER_VER_BYTE] != HllUtil<A>::SER_VER) {
+    throw std::invalid_argument("Wrong ser ver in input stream");
+  }
+  if (listHeader[HllUtil<A>::FAMILY_BYTE] != HllUtil<A>::FAMILY_ID) {
+    throw std::invalid_argument("Input stream is not an HLL sketch");
+  }
+
+  hll_mode mode = HllSketchImpl<A>::extractCurMode(listHeader[HllUtil<A>::MODE_BYTE]);
+  if (mode != LIST) {
+    throw std::invalid_argument("Calling list construtor with non-list mode data");
+  }
+
+  const target_hll_type tgtHllType = HllSketchImpl<A>::extractTgtHllType(listHeader[HllUtil<A>::MODE_BYTE]);
+
+  const int lgK = (int) listHeader[HllUtil<A>::LG_K_BYTE];
+  const bool compact = ((listHeader[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::COMPACT_FLAG_MASK) ? true : false);
+  const bool oooFlag = ((listHeader[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::OUT_OF_ORDER_FLAG_MASK) ? true : false);
+  const bool emptyFlag = ((listHeader[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::EMPTY_FLAG_MASK) ? true : false);
+
+  CouponList<A>* sketch = new (clAlloc().allocate(1)) CouponList<A>(lgK, tgtHllType, mode);
+  const int couponCount = listHeader[HllUtil<A>::LIST_COUNT_BYTE];
+  sketch->couponCount = couponCount;
+  sketch->putOutOfOrderFlag(oooFlag); // should always be false for LIST
+
+  if (!emptyFlag) {
+    // For stream processing, need to read entire number written to stream so read
+    // pointer ends up set correctly.
+    // If not compact, still need to read empty items even though in order.
+    const int numToRead = (compact ? couponCount : (1 << sketch->lgCouponArrInts));
+    is.read((char*)sketch->couponIntArr, numToRead * sizeof(int));
+  }
+
+  return sketch;
+}
+
+template<typename A>
+vector_u8<A> CouponList<A>::serialize(bool compact, unsigned header_size_bytes) const {
+  const size_t sketchSizeBytes = (compact ? getCompactSerializationBytes() : getUpdatableSerializationBytes()) + header_size_bytes;
+  vector_u8<A> byteArr(sketchSizeBytes);
+  uint8_t* bytes = byteArr.data() + header_size_bytes;
+
+  bytes[HllUtil<A>::PREAMBLE_INTS_BYTE] = static_cast<uint8_t>(getPreInts());
+  bytes[HllUtil<A>::SER_VER_BYTE] = static_cast<uint8_t>(HllUtil<A>::SER_VER);
+  bytes[HllUtil<A>::FAMILY_BYTE] = static_cast<uint8_t>(HllUtil<A>::FAMILY_ID);
+  bytes[HllUtil<A>::LG_K_BYTE] = static_cast<uint8_t>(this->lgConfigK);
+  bytes[HllUtil<A>::LG_ARR_BYTE] = static_cast<uint8_t>(lgCouponArrInts);
+  bytes[HllUtil<A>::FLAGS_BYTE] = this->makeFlagsByte(compact);
+  bytes[HllUtil<A>::LIST_COUNT_BYTE] = static_cast<uint8_t>(this->mode == LIST ? couponCount : 0);
+  bytes[HllUtil<A>::MODE_BYTE] = this->makeModeByte();
+
+  if (this->mode == SET) {
+    std::memcpy(bytes + HllUtil<A>::HASH_SET_COUNT_INT, &couponCount, sizeof(couponCount));
+  }
+
+  // coupons
+  // isCompact() is always false for now
+  const int sw = (isCompact() ? 2 : 0) | (compact ? 1 : 0);
+  switch (sw) {
+    case 0: { // src updatable, dst updatable
+      std::memcpy(bytes + getMemDataStart(), getCouponIntArr(), (1 << lgCouponArrInts) * sizeof(int));
+      break;
+    }
+    case 1: { // src updatable, dst compact
+      bytes += getMemDataStart(); // reusing pointer for incremental writes
+      for (uint32_t coupon: *this) {
+        std::memcpy(bytes, &coupon, sizeof(coupon));
+        bytes += sizeof(coupon);
+      }
+      break;
+    }
+
+    default:
+      throw std::runtime_error("Impossible condition when serializing");
+  }
+
+  return byteArr;
+}
+
+template<typename A>
+void CouponList<A>::serialize(std::ostream& os, const bool compact) const {
+  // header
+  const uint8_t preInts(getPreInts());
+  os.write((char*)&preInts, sizeof(preInts));
+  const uint8_t serialVersion(HllUtil<A>::SER_VER);
+  os.write((char*)&serialVersion, sizeof(serialVersion));
+  const uint8_t familyId(HllUtil<A>::FAMILY_ID);
+  os.write((char*)&familyId, sizeof(familyId));
+  const uint8_t lgKByte((uint8_t) this->lgConfigK);
+  os.write((char*)&lgKByte, sizeof(lgKByte));
+  const uint8_t lgArrIntsByte((uint8_t) lgCouponArrInts);
+  os.write((char*)&lgArrIntsByte, sizeof(lgArrIntsByte));
+  const uint8_t flagsByte(this->makeFlagsByte(compact));
+  os.write((char*)&flagsByte, sizeof(flagsByte));
+
+  if (this->mode == LIST) {
+    const uint8_t listCount((uint8_t) couponCount);
+    os.write((char*)&listCount, sizeof(listCount));
+  } else { // mode == SET
+    const uint8_t unused(0);
+    os.write((char*)&unused, sizeof(unused));
+  }
+
+  const uint8_t modeByte(this->makeModeByte());
+  os.write((char*)&modeByte, sizeof(modeByte));
+
+  if (this->mode == SET) {
+    // writing as int, already stored as int
+    os.write((char*)&couponCount, sizeof(couponCount));
+  }
+
+  // coupons
+  // isCompact() is always false for now
+  const int sw = (isCompact() ? 2 : 0) | (compact ? 1 : 0);
+  switch (sw) {
+    case 0: { // src updatable, dst updatable
+      os.write((char*)getCouponIntArr(), (1 << lgCouponArrInts) * sizeof(int));
+      break;
+    }
+    case 1: { // src updatable, dst compact
+      for (uint32_t coupon: *this) {
+        os.write((char*)&coupon, sizeof(coupon));
+      }
+      break;
+    }
+
+    default:
+      throw std::runtime_error("Impossible condition when serializing");
+  }
+  
+  return;
+}
+
+template<typename A>
+HllSketchImpl<A>* CouponList<A>::couponUpdate(int coupon) {
+  const int len = 1 << lgCouponArrInts;
+  for (int i = 0; i < len; ++i) { // search for empty slot
+    const int couponAtIdx = couponIntArr[i];
+    if (couponAtIdx == HllUtil<A>::EMPTY) {
+      couponIntArr[i] = coupon; // the actual update
+      ++couponCount;
+      if (couponCount >= len) { // array full
+        if (this->lgConfigK < 8) {
+          return promoteHeapListOrSetToHll(*this); // oooFlag = false
+        }
+        return promoteHeapListToSet(*this); // oooFlag = true;
+      }
+      return this;
+    }
+    // cell not empty
+    if (couponAtIdx == coupon) {
+      return this; // duplicate
+    }
+    // cell not empty and not a duplicate, continue
+  }
+  throw std::runtime_error("Array invalid: no empties and no duplicates");
+}
+
+template<typename A>
+double CouponList<A>::getCompositeEstimate() const { return getEstimate(); }
+
+template<typename A>
+double CouponList<A>::getEstimate() const {
+  const int couponCount = getCouponCount();
+  const double est = CubicInterpolation<A>::usingXAndYTables(couponCount);
+  return fmax(est, couponCount);
+}
+
+template<typename A>
+double CouponList<A>::getLowerBound(const int numStdDev) const {
+  HllUtil<A>::checkNumStdDev(numStdDev);
+  const int couponCount = getCouponCount();
+  const double est = CubicInterpolation<A>::usingXAndYTables(couponCount);
+  const double tmp = est / (1.0 + (numStdDev * HllUtil<A>::COUPON_RSE));
+  return fmax(tmp, couponCount);
+}
+
+template<typename A>
+double CouponList<A>::getUpperBound(const int numStdDev) const {
+  HllUtil<A>::checkNumStdDev(numStdDev);
+  const int couponCount = getCouponCount();
+  const double est = CubicInterpolation<A>::usingXAndYTables(couponCount);
+  const double tmp = est / (1.0 - (numStdDev * HllUtil<A>::COUPON_RSE));
+  return fmax(tmp, couponCount);
+}
+
+template<typename A>
+bool CouponList<A>::isEmpty() const { return getCouponCount() == 0; }
+
+template<typename A>
+int CouponList<A>::getUpdatableSerializationBytes() const {
+  return getMemDataStart() + (4 << getLgCouponArrInts());
+}
+
+template<typename A>
+int CouponList<A>::getCouponCount() const {
+  return couponCount;
+}
+
+template<typename A>
+int CouponList<A>::getCompactSerializationBytes() const {
+  return getMemDataStart() + (couponCount << 2);
+}
+
+template<typename A>
+int CouponList<A>::getMemDataStart() const {
+  return HllUtil<A>::LIST_INT_ARR_START;
+}
+
+template<typename A>
+int CouponList<A>::getPreInts() const {
+  return HllUtil<A>::LIST_PREINTS;
+}
+
+template<typename A>
+bool CouponList<A>::isCompact() const { return false; }
+
+template<typename A>
+bool CouponList<A>::isOutOfOrderFlag() const { return oooFlag; }
+
+template<typename A>
+void CouponList<A>::putOutOfOrderFlag(bool oooFlag) {
+  this->oooFlag = oooFlag;
+}
+
+template<typename A>
+int CouponList<A>::getLgCouponArrInts() const {
+  return lgCouponArrInts;
+}
+
+template<typename A>
+int* CouponList<A>::getCouponIntArr() const {
+  return couponIntArr;
+}
+
+template<typename A>
+HllSketchImpl<A>* CouponList<A>::promoteHeapListToSet(CouponList& list) {
+  return HllSketchImplFactory<A>::promoteListToSet(list);
+}
+
+template<typename A>
+HllSketchImpl<A>* CouponList<A>::promoteHeapListOrSetToHll(CouponList& src) {
+  return HllSketchImplFactory<A>::promoteListOrSetToHll(src);
+}
+
+template<typename A>
+coupon_iterator<A> CouponList<A>::begin(bool all) const {
+  return coupon_iterator<A>(couponIntArr, 1 << lgCouponArrInts, 0, all);
+}
+
+template<typename A>
+coupon_iterator<A> CouponList<A>::end() const {
+  return coupon_iterator<A>(couponIntArr, 1 << lgCouponArrInts, 1 << lgCouponArrInts, false);
+}
+
+}
+
+#endif // _COUPONLIST_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/CouponList.hpp b/be/src/thirdparty/datasketches/CouponList.hpp
new file mode 100644
index 0000000..063805b
--- /dev/null
+++ b/be/src/thirdparty/datasketches/CouponList.hpp
@@ -0,0 +1,91 @@
+/*
+ * 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 _COUPONLIST_HPP_
+#define _COUPONLIST_HPP_
+
+#include "HllSketchImpl.hpp"
+#include "coupon_iterator.hpp"
+
+#include <iostream>
+
+namespace datasketches {
+
+template<typename A>
+class HllSketchImplFactory;
+
+template<typename A = std::allocator<char>>
+class CouponList : public HllSketchImpl<A> {
+  public:
+    explicit CouponList(int lgConfigK, target_hll_type tgtHllType, hll_mode mode);
+    explicit CouponList(const CouponList& that);
+    explicit CouponList(const CouponList& that, target_hll_type tgtHllType);
+
+    static CouponList* newList(const void* bytes, size_t len);
+    static CouponList* newList(std::istream& is);
+    virtual vector_u8<A> serialize(bool compact, unsigned header_size_bytes) const;
+    virtual void serialize(std::ostream& os, bool compact) const;
+
+    virtual ~CouponList();
+    virtual std::function<void(HllSketchImpl<A>*)> get_deleter() const;
+
+    virtual CouponList* copy() const;
+    virtual CouponList* copyAs(target_hll_type tgtHllType) const;
+
+    virtual HllSketchImpl<A>* couponUpdate(int coupon);
+
+    virtual double getEstimate() const;
+    virtual double getCompositeEstimate() const;
+    virtual double getUpperBound(int numStdDev) const;
+    virtual double getLowerBound(int numStdDev) const;
+
+    virtual bool isEmpty() const;
+    virtual int getCouponCount() const;
+
+    coupon_iterator<A> begin(bool all = false) const;
+    coupon_iterator<A> end() const;
+
+  protected:
+    typedef typename std::allocator_traits<A>::template rebind_alloc<CouponList<A>> clAlloc;
+
+    HllSketchImpl<A>* promoteHeapListToSet(CouponList& list);
+    HllSketchImpl<A>* promoteHeapListOrSetToHll(CouponList& src);
+
+    virtual int getUpdatableSerializationBytes() const;
+    virtual int getCompactSerializationBytes() const;
+    virtual int getMemDataStart() const;
+    virtual int getPreInts() const;
+    virtual bool isCompact() const;
+    virtual bool isOutOfOrderFlag() const;
+    virtual void putOutOfOrderFlag(bool oooFlag);
+
+    virtual int getLgCouponArrInts() const;
+    virtual int* getCouponIntArr() const;
+
+    int lgCouponArrInts;
+    int couponCount;
+    bool oooFlag;
+    int* couponIntArr;
+
+    friend class HllSketchImplFactory<A>;
+};
+
+}
+
+#endif /* _COUPONLIST_HPP_ */
diff --git a/be/src/thirdparty/datasketches/CubicInterpolation-internal.hpp b/be/src/thirdparty/datasketches/CubicInterpolation-internal.hpp
new file mode 100644
index 0000000..be03e80
--- /dev/null
+++ b/be/src/thirdparty/datasketches/CubicInterpolation-internal.hpp
@@ -0,0 +1,233 @@
+/*
+ * 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 _CUBICINTERPOLATION_INTERNAL_HPP_
+#define _CUBICINTERPOLATION_INTERNAL_HPP_
+
+#include "CubicInterpolation.hpp"
+
+#include <string>
+#include <stdexcept>
+
+namespace datasketches {
+
+template<typename A>
+static double interpolateUsingXAndYTables(const double xArr[], const double yArr[], const int offset, const double x);
+
+template<typename A>
+static double cubicInterpolate(const double x0, const double y0, const double x1, const double y1,
+                               const double x2, const double y2, const double x3, const double y3, const double x);
+
+template<typename A>
+static int findStraddle(const double xArr[], const int len, const double x);
+
+template<typename A>
+static int recursiveFindStraddle(const double xArr[], const int l, const int r, const double x);
+
+template<typename A>
+static double interpolateUsingXArrAndYStride(const double xArr[], const double yStride,
+                                             const int offset, const double x);
+
+const int numEntries = 40;
+
+//Computed for Coupon lgK = 26 ONLY. Designed for the cubic interpolator function.
+const double xArrComputed[numEntries] = {
+    0.0, 1.0, 20.0, 400.0,
+    8000.0, 160000.0, 300000.0, 600000.0,
+    900000.0, 1200000.0, 1500000.0, 1800000.0,
+    2100000.0, 2400000.0, 2700000.0, 3000000.0,
+    3300000.0, 3600000.0, 3900000.0, 4200000.0,
+    4500000.0, 4800000.0, 5100000.0, 5400000.0,
+    5700000.0, 6000000.0, 6300000.0, 6600000.0,
+    6900000.0, 7200000.0, 7500000.0, 7800000.0,
+    8100000.0, 8400000.0, 8700000.0, 9000000.0,
+    9300000.0, 9600000.0, 9900000.0, 10200000.0
+};
+
+//Computed for Coupon lgK = 26 ONLY. Designed for the cubic interpolator function.
+const double yArrComputed[numEntries] =  {
+    0.0000000000000000, 1.0000000000000000, 20.0000009437402611, 400.0003963713384110,
+    8000.1589294602090376, 160063.6067763759638183, 300223.7071597663452849, 600895.5933856170158833,
+    902016.8065120954997838, 1203588.4983199508860707, 1505611.8245524743106216, 1808087.9449319066479802,
+    2111018.0231759352609515, 2414403.2270142501220107, 2718244.7282051891088486, 3022543.7025524540804327,
+    3327301.3299219091422856, 3632518.7942584538832307, 3938197.2836029687896371, 4244337.9901093561202288,
+    4550942.1100616492331028, 4858010.8438911894336343, 5165545.3961938973516226, 5473546.9757476449012756,
+    5782016.7955296505242586, 6090956.0727340159937739, 6400366.0287892958149314, 6710247.8893762007355690,
+    7020602.8844453142955899, 7331432.2482349723577499, 7642737.2192891482263803, 7954519.0404754765331745,
+    8266778.9590033423155546, 8579518.2264420464634895, 8892738.0987390466034412, 9206439.8362383283674717,
+    9520624.7036988288164139, 9835293.9703129194676876, 10150448.9097250290215015, 10466090.8000503256917000
+};
+
+template<typename A>
+double CubicInterpolation<A>::usingXAndYTables(const double x) {
+  return usingXAndYTables(xArrComputed, yArrComputed, numEntries, x);
+}
+
+template<typename A>
+double CubicInterpolation<A>::usingXAndYTables(const double xArr[], const double yArr[],
+                                            const int len, const double x) {
+  int offset;
+  if (x < xArr[0] || x > xArr[len-1]) {
+    throw std::invalid_argument("x value out of range: " + std::to_string(x));
+  }
+
+  if (x ==  xArr[len-1]) { // corner case
+    return (yArr[len-1]);
+  }
+
+  offset = findStraddle<A>(xArr, len, x);
+  if (offset < 0 && offset > len-2) {
+    throw std::logic_error("offset must be >= 0 and <= " + std::to_string(len) + "-2");
+  }
+
+  if (offset == 0) { // corner case
+    return (interpolateUsingXAndYTables<A>(xArr, yArr, (offset-0), x));
+  }
+  else if (offset == numEntries-2) { // corner case
+    return (interpolateUsingXAndYTables<A>(xArr, yArr, (offset-2), x));
+  }
+  else { // main case
+    return (interpolateUsingXAndYTables<A>(xArr, yArr, (offset-1), x));
+  }
+  throw std::logic_error("Exception should be unreachable");
+}
+
+// In C: again-two-registers cubic_interpolate_aux L1368
+template<typename A>
+static double interpolateUsingXAndYTables(const double xArr[], const double yArr[],
+                                          const int offset, const double x) {
+    return (cubicInterpolate<A>(xArr[offset+0], yArr[offset+0],
+                        xArr[offset+1], yArr[offset+1],
+                        xArr[offset+2], yArr[offset+2],
+                        xArr[offset+3], yArr[offset+3],
+                        x) );
+}
+
+template<typename A>
+static inline double cubicInterpolate(const double x0, const double y0,
+                                      const double x1, const double y1,
+                                      const double x2, const double y2,
+                                      const double x3, const double y3,
+                                      const double x)
+{
+  double l0_numer = (x - x1) * (x - x2) * (x - x3);
+  double l1_numer = (x - x0) * (x - x2) * (x - x3);
+  double l2_numer = (x - x0) * (x - x1) * (x - x3);
+  double l3_numer = (x - x0) * (x - x1) * (x - x2);
+
+  double l0_denom = (x0 - x1) * (x0 - x2) * (x0 - x3);
+  double l1_denom = (x1 - x0) * (x1 - x2) * (x1 - x3);
+  double l2_denom = (x2 - x0) * (x2 - x1) * (x2 - x3);
+  double l3_denom = (x3 - x0) * (x3 - x1) * (x3 - x2);
+
+  double term0 = y0 * l0_numer / l0_denom;
+  double term1 = y1 * l1_numer / l1_denom;
+  double term2 = y2 * l2_numer / l2_denom;
+  double term3 = y3 * l3_numer / l3_denom;
+
+  return (term0 + term1 + term2 + term3);
+}
+
+/* returns j such that xArr[j] <= x and x < xArr[j+1] */
+template<typename A>
+static int findStraddle(const double xArr[], const int len, const double x)
+{
+  if ((len < 2) || (x < xArr[0]) || (x > xArr[len-1])) {
+    throw std::logic_error("invariant violated during interpolation");
+  }
+  return(recursiveFindStraddle<A>(xArr, 0, len-1, x));
+}
+
+
+/* the invariant here is that xArr[l] <= x && x < xArr[r] */
+template<typename A>
+static int recursiveFindStraddle(const double xArr[], const int l, const int r, const double x)
+{
+  int m;
+  if (l >= r) {
+    throw std::logic_error("lower bound not less than upper bound in search");
+  }
+  if ((xArr[l] > x) || (x >= xArr[r])) { // the invariant
+    throw std::logic_error("target value invariant violated in search");
+  }
+
+  if (l+1 == r) return (l);
+  m = l + ((r-l)/2);
+  if (xArr[m] <= x) return (recursiveFindStraddle<A>(xArr, m, r, x));
+  else              return (recursiveFindStraddle<A>(xArr, l, m, x));
+}
+
+
+//Interpolate using X table and Y stride
+
+/**
+ * Cubic interpolation using interpolation X table and Y stride.
+ *
+ * @param xArr The x array
+ * @param yStride The y stride
+ * @param xArrLen the length of xArr
+ * @param x The value x
+ * @return cubic interpolation
+ */
+//In C: again-two-registers cubic_interpolate_with_x_arr_and_y_stride L1411
+// Used by HllEstimators
+template<typename A>
+double CubicInterpolation<A>::usingXArrAndYStride(const double xArr[], const int xArrLen,
+                                                  const double yStride, const double x) {
+  const int xArrLenM1 = xArrLen - 1;
+
+  if ((xArrLen < 4) || (x < xArr[0]) || (x > xArr[xArrLenM1])) {
+    throw std::logic_error("impossible values during interpolaiton");
+  }
+
+  if (x ==  xArr[xArrLenM1]) { /* corner case */
+    return (yStride * (xArrLenM1));
+  }
+
+  const int offset = findStraddle<A>(xArr, xArrLen, x); //uses recursion
+  const int xArrLenM2 = xArrLen - 2;
+  if ((offset < 0) || (offset > xArrLenM2)) {
+    throw std::logic_error("invalid offset during interpolation");
+  }
+
+  if (offset == 0) { /* corner case */
+    return (interpolateUsingXArrAndYStride<A>(xArr, yStride, (offset - 0), x));
+  }
+  else if (offset == xArrLenM2) { /* corner case */
+    return (interpolateUsingXArrAndYStride<A>(xArr, yStride, (offset - 2), x));
+  }
+  /* main case */
+  return (interpolateUsingXArrAndYStride<A>(xArr, yStride, (offset - 1), x));
+}
+
+//In C: again-two-registers cubic_interpolate_with_x_arr_and_y_stride_aux L1402
+template<typename A>
+static double interpolateUsingXArrAndYStride(const double xArr[], const double yStride,
+                                             const int offset, const double x) {
+  return cubicInterpolate<A>(
+      xArr[offset + 0], yStride * (offset + 0),
+      xArr[offset + 1], yStride * (offset + 1),
+      xArr[offset + 2], yStride * (offset + 2),
+      xArr[offset + 3], yStride * (offset + 3),
+      x);
+}
+
+}
+
+#endif // _CUBICINTERPOLATION_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/CubicInterpolation.hpp b/be/src/thirdparty/datasketches/CubicInterpolation.hpp
new file mode 100644
index 0000000..b9cdfe7
--- /dev/null
+++ b/be/src/thirdparty/datasketches/CubicInterpolation.hpp
@@ -0,0 +1,43 @@
+/*
+ * 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 _CUBICINTERPOLATION_HPP_
+#define _CUBICINTERPOLATION_HPP_
+
+#include <memory>
+
+namespace datasketches {
+
+template<typename A = std::allocator<char>>
+class CubicInterpolation {
+  public:
+    static double usingXAndYTables(const double xArr[], const double yArr[],
+                                   int len, double x);
+
+    static double usingXAndYTables(double x);
+
+    static double usingXArrAndYStride(const double xArr[], const int xArrLen,
+                                      double yStride, double x);
+};
+
+}
+
+#include "CubicInterpolation-internal.hpp"
+
+#endif /* _CUBICINTERPOLATION_HPP_ */
\ No newline at end of file
diff --git a/be/src/thirdparty/datasketches/HarmonicNumbers-internal.hpp b/be/src/thirdparty/datasketches/HarmonicNumbers-internal.hpp
new file mode 100644
index 0000000..db73b86
--- /dev/null
+++ b/be/src/thirdparty/datasketches/HarmonicNumbers-internal.hpp
@@ -0,0 +1,90 @@
+/*
+ * 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 _HARMONICNUMBERS_INTERNAL_HPP_
+#define _HARMONICNUMBERS_INTERNAL_HPP_
+
+#include "HarmonicNumbers.hpp"
+
+#include <cmath>
+
+namespace datasketches {
+
+template<typename A>
+double HarmonicNumbers<A>::getBitMapEstimate(const int bitVectorLength, const int numBitsSet) {
+  return (bitVectorLength * (harmonicNumber(bitVectorLength) - harmonicNumber(bitVectorLength - numBitsSet)));
+}
+
+static const int NUM_EXACT_HARMONIC_NUMBERS = 25;
+
+static double tableOfExactHarmonicNumbers[] = {
+    0.0, // 0
+    1.0, // 1
+    1.5, // 2
+    11.0 / 6.0, // 3
+    25.0 / 12.0, // 4
+    137.0 / 60.0, // 5
+    49.0 / 20.0, // 6
+    363.0 / 140.0, // 7
+    761.0 / 280.0, // 8
+    7129.0 / 2520.0, // 9
+    7381.0 / 2520.0, // 10
+    83711.0 / 27720.0, // 11
+    86021.0 / 27720.0, // 12
+    1145993.0 / 360360.0, // 13
+    1171733.0 / 360360.0, // 14
+    1195757.0 / 360360.0, // 15
+    2436559.0 / 720720.0, // 16
+    42142223.0 / 12252240.0, // 17
+    14274301.0 / 4084080.0, // 18
+    275295799.0 / 77597520.0, // 19
+    55835135.0 / 15519504.0, // 20
+    18858053.0 / 5173168.0, // 21
+    19093197.0 / 5173168.0, // 22
+    444316699.0 / 118982864.0, // 23
+    1347822955.0 / 356948592.0 // 24
+  };
+
+static const double EULER_MASCHERONI_CONSTANT = 0.577215664901532860606512090082;
+
+template<typename A>
+double HarmonicNumbers<A>::harmonicNumber(const uint64_t x_i) {
+  if (x_i < NUM_EXACT_HARMONIC_NUMBERS) {
+    return tableOfExactHarmonicNumbers[x_i];
+  } else {
+    double x = x_i;
+    double invSq = 1.0 / (x * x);
+    double sum = log(x) + EULER_MASCHERONI_CONSTANT + (1.0 / (2.0 * x));
+    /* note: the number of terms included from this series expansion is appropriate
+       for the size of the exact table (25) and the precision of doubles */
+    double pow = invSq; // now n^-2
+    sum -= pow * (1.0 / 12.0);
+    pow *= invSq; // now n^-4
+    sum += pow * (1.0 / 120.0);
+    pow *= invSq; /* now n^-6 */
+    sum -= pow * (1.0 / 252.0);
+    pow *= invSq; /* now n^-8 */
+    sum += pow * (1.0 / 240.0);
+    return sum;
+  }
+}
+
+}
+
+#endif // _HARMONICNUMBERS_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/HarmonicNumbers.hpp b/be/src/thirdparty/datasketches/HarmonicNumbers.hpp
new file mode 100644
index 0000000..501ce0c
--- /dev/null
+++ b/be/src/thirdparty/datasketches/HarmonicNumbers.hpp
@@ -0,0 +1,48 @@
+/*
+ * 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 _HARMONICNUMBERS_HPP_
+#define _HARMONICNUMBERS_HPP_
+
+#include <cstdint>
+#include <memory>
+
+namespace datasketches {
+
+template<typename A = std::allocator<char>>
+class HarmonicNumbers {
+  public:
+    /**
+     * This is the estimator you would use for flat bit map random accessed, similar to a Bloom filter.
+     * @param bitVectorLength the length of the bit vector in bits. Must be &gt; 0.
+     * @param numBitsSet the number of bits set in this bit vector. Must be &ge; 0 and &le;
+     * bitVectorLength.
+     * @return the estimate.
+     */
+    static double getBitMapEstimate(int bitVectorLength, int numBitsSet);
+
+  private:
+    static double harmonicNumber(uint64_t x_i);
+};
+
+}
+
+#include "HarmonicNumbers-internal.hpp"
+
+#endif /* _HARMONICNUMBERS_HPP_ */
\ No newline at end of file
diff --git a/be/src/thirdparty/datasketches/Hll4Array-internal.hpp b/be/src/thirdparty/datasketches/Hll4Array-internal.hpp
new file mode 100644
index 0000000..30b24f7
--- /dev/null
+++ b/be/src/thirdparty/datasketches/Hll4Array-internal.hpp
@@ -0,0 +1,335 @@
+/*
+ * 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 _HLL4ARRAY_INTERNAL_HPP_
+#define _HLL4ARRAY_INTERNAL_HPP_
+
+#include "Hll4Array.hpp"
+
+#include <cstring>
+#include <memory>
+#include <stdexcept>
+#include <string>
+
+namespace datasketches {
+
+template<typename A>
+Hll4Array<A>::Hll4Array(const int lgConfigK, const bool startFullSize) :
+    HllArray<A>(lgConfigK, target_hll_type::HLL_4, startFullSize) {
+  const int numBytes = this->hll4ArrBytes(lgConfigK);
+  typedef typename std::allocator_traits<A>::template rebind_alloc<uint8_t> uint8Alloc;
+  this->hllByteArr = uint8Alloc().allocate(numBytes);
+  std::fill(this->hllByteArr, this->hllByteArr + numBytes, 0);
+  auxHashMap = nullptr;
+}
+
+template<typename A>
+Hll4Array<A>::Hll4Array(const Hll4Array<A>& that) :
+  HllArray<A>(that)
+{
+  // can determine hllByteArr size in parent class, no need to allocate here
+  // but parent class doesn't handle the auxHashMap
+  if (that.auxHashMap != nullptr) {
+    auxHashMap = that.auxHashMap->copy();
+  } else {
+    auxHashMap = nullptr;
+  }
+}
+
+template<typename A>
+Hll4Array<A>::~Hll4Array() {
+  // hllByteArr deleted in parent
+  if (auxHashMap != nullptr) {
+    AuxHashMap<A>::make_deleter()(auxHashMap);
+  }
+}
+
+template<typename A>
+std::function<void(HllSketchImpl<A>*)> Hll4Array<A>::get_deleter() const {
+  return [](HllSketchImpl<A>* ptr) {
+    typedef typename std::allocator_traits<A>::template rebind_alloc<Hll4Array<A>> hll4Alloc;
+    Hll4Array<A>* hll = static_cast<Hll4Array<A>*>(ptr);
+    hll->~Hll4Array();
+    hll4Alloc().deallocate(hll, 1);
+  };
+}
+
+template<typename A>
+Hll4Array<A>* Hll4Array<A>::copy() const {
+  typedef typename std::allocator_traits<A>::template rebind_alloc<Hll4Array<A>> hll4Alloc;
+  return new (hll4Alloc().allocate(1)) Hll4Array<A>(*this);
+}
+
+template<typename A>
+int Hll4Array<A>::getUpdatableSerializationBytes() const {
+  AuxHashMap<A>* auxHashMap = getAuxHashMap();
+  int auxBytes;
+  if (auxHashMap == nullptr) {
+    auxBytes = 4 << HllUtil<A>::LG_AUX_ARR_INTS[this->lgConfigK];
+  } else {
+    auxBytes = 4 << auxHashMap->getLgAuxArrInts();
+  }
+  return HllUtil<A>::HLL_BYTE_ARR_START + getHllByteArrBytes() + auxBytes;
+}
+
+template<typename A>
+int Hll4Array<A>::getHllByteArrBytes() const {
+  return this->hll4ArrBytes(this->lgConfigK);
+}
+
+template<typename A>
+AuxHashMap<A>* Hll4Array<A>::getAuxHashMap() const {
+  return auxHashMap;
+}
+
+template<typename A>
+void Hll4Array<A>::putAuxHashMap(AuxHashMap<A>* auxHashMap) {
+  this->auxHashMap = auxHashMap;
+}
+
+template<typename A>
+uint8_t Hll4Array<A>::getSlot(int slotNo) const {
+  const uint8_t byte = this->hllByteArr[slotNo >> 1];
+  if ((slotNo & 1) > 0) { // odd?
+    return byte >> 4;
+  }
+  return byte & HllUtil<A>::loNibbleMask;
+}
+
+template<typename A>
+uint8_t Hll4Array<A>::get_value(uint32_t index) const {
+  const uint8_t value = getSlot(index);
+  if (value != HllUtil<A>::AUX_TOKEN) return value + this->curMin;
+  return auxHashMap->mustFindValueFor(index);
+}
+
+template<typename A>
+HllSketchImpl<A>* Hll4Array<A>::couponUpdate(const int coupon) {
+  internalCouponUpdate(coupon);
+  return this;
+}
+
+template<typename A>
+void Hll4Array<A>::internalCouponUpdate(const int coupon) {
+  const int newValue = HllUtil<A>::getValue(coupon);
+  if (newValue <= this->curMin) {
+    return; // quick rejection, but only works for large N
+  }
+  const int configKmask = (1 << this->lgConfigK) - 1;
+  const int slotNo = HllUtil<A>::getLow26(coupon) & configKmask;
+  internalHll4Update(slotNo, newValue);
+}
+
+template<typename A>
+void Hll4Array<A>::putSlot(int slotNo, uint8_t newValue) {
+  const int byteno = slotNo >> 1;
+  const uint8_t oldValue = this->hllByteArr[byteno];
+  if ((slotNo & 1) == 0) { // set low nibble
+    this->hllByteArr[byteno]
+      = ((oldValue & HllUtil<A>::hiNibbleMask) | (newValue & HllUtil<A>::loNibbleMask));
+  } else { // set high nibble
+    this->hllByteArr[byteno]
+      = ((oldValue & HllUtil<A>::loNibbleMask) | ((newValue << 4) & HllUtil<A>::hiNibbleMask));
+  }
+}
+
+//In C: two-registers.c Line 836 in "hhb_abstract_set_slot_if_new_value_bigger" non-sparse
+template<typename A>
+void Hll4Array<A>::internalHll4Update(const int slotNo, const int newVal) {
+
+  const int rawStoredOldValue = getSlot(slotNo); // could be a 0
+  // this is provably a LB:
+  const int lbOnOldValue = rawStoredOldValue + this->curMin; // lower bound, could be 0
+
+  if (newVal > lbOnOldValue) { // 842
+    // Note: if an AUX_TOKEN exists, then auxHashMap must already exist
+    // 846: rawStoredOldValue == AUX_TOKEN
+    const int actualOldValue = (rawStoredOldValue < HllUtil<A>::AUX_TOKEN)
+       ? (lbOnOldValue) : (auxHashMap->mustFindValueFor(slotNo));
+
+    if (newVal > actualOldValue) { // 848: actualOldValue could still be 0; newValue > 0
+      // we know that the array will change, but we haven't actually updated yet
+      this->hipAndKxQIncrementalUpdate(actualOldValue, newVal);
+
+      // newVal >= curMin
+
+      const int shiftedNewValue = newVal - this->curMin; // 874
+      // redundant since we know newVal >= curMin,
+      // and lgConfigK bounds do not allow overflowing an int
+      //assert(shiftedNewValue >= 0);
+
+      if (rawStoredOldValue == HllUtil<A>::AUX_TOKEN) { // 879
+        // Given that we have an AUX_TOKEN, tehre are 4 cases for how to
+        // actually modify the data structure
+
+        if (shiftedNewValue >= HllUtil<A>::AUX_TOKEN) { // case 1: 881
+          // the byte array already contains aux token
+          // This is the case where old and new values are both exceptions.
+          // The 4-bit array already is AUX_TOKEN, only need to update auxHashMap
+          auxHashMap->mustReplace(slotNo, newVal);
+        }
+        else { // case 2: 885
+          // This is the hypothetical case where the old value is an exception and the new one is not,
+          // which is impossible given that curMin has not changed here and newVal > oldValue
+        }
+      } else { // rawStoredOldValue != AUX_TOKEN
+        if (shiftedNewValue >= HllUtil<A>::AUX_TOKEN) { // case 3: 892
+          // This is the case where the old value is not an exception and the new value is.
+          // The AUX_TOKEN must be stored in the 4-bit array and the new value
+          // added to the exception table
+          putSlot(slotNo, HllUtil<A>::AUX_TOKEN);
+          if (auxHashMap == nullptr) {
+            auxHashMap = AuxHashMap<A>::newAuxHashMap(HllUtil<A>::LG_AUX_ARR_INTS[this->lgConfigK], this->lgConfigK);
+          }
+          auxHashMap->mustAdd(slotNo, newVal);
+        }
+        else { // case 4: 897
+          // This is the case where neither the old value nor the new value is an exception.
+          // We just overwrite the 4-bit array with the shifted new value.
+          putSlot(slotNo, shiftedNewValue);
+        }
+      }
+
+      // we just increased a pair value, so it might be time to change curMin
+      if (actualOldValue == this->curMin) { // 908
+        this->decNumAtCurMin();
+        while (this->numAtCurMin == 0) {
+          shiftToBiggerCurMin(); // increases curMin by 1, builds a new aux table
+          // shifts values in 4-bit table and recounts curMin
+        }
+      }
+    } // end newVal <= actualOldValue
+  } // end newValue <= lbOnOldValue -> return, no need to update array
+}
+
+// This scheme only works with two double registers (2 kxq values).
+//   HipAccum, kxq0 and kxq1 remain untouched.
+//   This changes curMin, numAtCurMin, hllByteArr and auxMap.
+// Entering this routine assumes that all slots have valid values > 0 and <= 15.
+// An AuxHashMap must exist if any values in the current hllByteArray are already 15.
+// In C: again-two-registers.c Lines 710 "hhb_shift_to_bigger_curmin"
+template<typename A>
+void Hll4Array<A>::shiftToBiggerCurMin() {
+  const int newCurMin = this->curMin + 1;
+  const int configK = 1 << this->lgConfigK;
+  const int configKmask = configK - 1;
+
+  int numAtNewCurMin = 0;
+  int numAuxTokens = 0;
+
+  // Walk through the slots of 4-bit array decrementing stored values by one unless it
+  // equals AUX_TOKEN, where it is left alone but counted to be checked later.
+  // If oldStoredValue is 0 it is an error.
+  // If the decremented value is 0, we increment numAtNewCurMin.
+  // Because getNibble is masked to 4 bits oldStoredValue can never be > 15 or negative
+  for (int i = 0; i < configK; i++) { //724
+    int oldStoredValue = getSlot(i);
+    if (oldStoredValue == 0) {
+      throw std::runtime_error("Array slots cannot be 0 at this point.");
+    }
+    if (oldStoredValue < HllUtil<A>::AUX_TOKEN) {
+      putSlot(i, --oldStoredValue);
+      if (oldStoredValue == 0) { numAtNewCurMin++; }
+    } else { //oldStoredValue == AUX_TOKEN
+      numAuxTokens++;
+      if (auxHashMap == nullptr) {
+        throw std::logic_error("auxHashMap cannot be null at this point");
+      }
+    }
+  }
+
+  // If old AuxHashMap exists, walk through it updating some slots and build a new AuxHashMap
+  // if needed.
+  AuxHashMap<A>* newAuxMap = nullptr;
+  if (auxHashMap != nullptr) {
+    int slotNum;
+    int oldActualVal;
+    int newShiftedVal;
+
+    for (auto coupon: *auxHashMap) {
+      slotNum = HllUtil<A>::getLow26(coupon) & configKmask;
+      oldActualVal = HllUtil<A>::getValue(coupon);
+      newShiftedVal = oldActualVal - newCurMin;
+      if (newShiftedVal < 0) {
+        throw std::logic_error("oldActualVal < newCurMin when incrementing curMin");
+      }
+
+      if (getSlot(slotNum) != HllUtil<A>::AUX_TOKEN) {
+        throw std::logic_error("getSlot(slotNum) != AUX_TOKEN for item in auxiliary hash map");
+      }
+      // Array slot != AUX_TOKEN at getSlot(slotNum);
+      if (newShiftedVal < HllUtil<A>::AUX_TOKEN) { // 756
+        if (newShiftedVal != 14) {
+          throw std::logic_error("newShiftedVal != 14 for item in old auxHashMap despite curMin increment");
+        }
+        // The former exception value isn't one anymore, so it stays out of new AuxHashMap.
+        // Correct the AUX_TOKEN value in the HLL array to the newShiftedVal (14).
+        putSlot(slotNum, newShiftedVal);
+        numAuxTokens--;
+      } else { //newShiftedVal >= AUX_TOKEN
+        // the former exception remains an exception, so must be added to the newAuxMap
+        if (newAuxMap == nullptr) {
+          newAuxMap = AuxHashMap<A>::newAuxHashMap(HllUtil<A>::LG_AUX_ARR_INTS[this->lgConfigK], this->lgConfigK);
+        }
+        newAuxMap->mustAdd(slotNum, oldActualVal);
+      }
+    } //end scan of oldAuxMap
+  } //end if (auxHashMap != null)
+  else { // oldAuxMap == null
+    if (numAuxTokens != 0) {
+      throw std::logic_error("No auxiliary hash map, but numAuxTokens != 0");
+    }
+  }
+
+  if (newAuxMap != nullptr) {
+    if (newAuxMap->getAuxCount() != numAuxTokens) {
+      throw std::runtime_error("Inconsistent counts: auxCount: " + std::to_string(newAuxMap->getAuxCount())
+                               + ", HLL tokesn: " + std::to_string(numAuxTokens));
+    }
+  }
+
+  if (auxHashMap != nullptr) {
+    AuxHashMap<A>::make_deleter()(auxHashMap);
+  }
+  auxHashMap = newAuxMap;
+
+  this->curMin = newCurMin;
+  this->numAtCurMin = numAtNewCurMin;
+}
+
+template<typename A>
+typename HllArray<A>::const_iterator Hll4Array<A>::begin(bool all) const {
+  return typename HllArray<A>::const_iterator(this->hllByteArr, 1 << this->lgConfigK, 0, this->tgtHllType, auxHashMap, this->curMin, all);
+}
+
+template<typename A>
+typename HllArray<A>::const_iterator Hll4Array<A>::end() const {
+  return typename HllArray<A>::const_iterator(this->hllByteArr, 1 << this->lgConfigK, 1 << this->lgConfigK, this->tgtHllType, auxHashMap, this->curMin, false);
+}
+
+template<typename A>
+void Hll4Array<A>::mergeHll(const HllArray<A>& src) {
+  for (auto coupon: src) {
+    internalCouponUpdate(coupon);
+  }
+}
+
+}
+
+#endif // _HLL4ARRAY_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/Hll4Array.hpp b/be/src/thirdparty/datasketches/Hll4Array.hpp
new file mode 100644
index 0000000..ff56c86
--- /dev/null
+++ b/be/src/thirdparty/datasketches/Hll4Array.hpp
@@ -0,0 +1,69 @@
+/*
+ * 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 _HLL4ARRAY_HPP_
+#define _HLL4ARRAY_HPP_
+
+#include "AuxHashMap.hpp"
+#include "HllArray.hpp"
+
+namespace datasketches {
+
+template<typename A>
+class Hll4Iterator;
+
+template<typename A>
+class Hll4Array final : public HllArray<A> {
+  public:
+    explicit Hll4Array(int lgConfigK, bool startFullSize);
+    explicit Hll4Array(const Hll4Array<A>& that);
+
+    virtual ~Hll4Array();
+    virtual std::function<void(HllSketchImpl<A>*)> get_deleter() const;
+
+    virtual Hll4Array* copy() const;
+
+    inline uint8_t getSlot(int slotNo) const;
+    inline void putSlot(int slotNo, uint8_t value);
+    inline uint8_t get_value(uint32_t index) const;
+
+    virtual int getUpdatableSerializationBytes() const;
+    virtual int getHllByteArrBytes() const;
+
+    virtual HllSketchImpl<A>* couponUpdate(int coupon) final;
+    void mergeHll(const HllArray<A>& src);
+
+    virtual AuxHashMap<A>* getAuxHashMap() const;
+    // does *not* delete old map if overwriting
+    void putAuxHashMap(AuxHashMap<A>* auxHashMap);
+
+    virtual typename HllArray<A>::const_iterator begin(bool all = false) const;
+    virtual typename HllArray<A>::const_iterator end() const;
+
+  private:
+    void internalCouponUpdate(int coupon);
+    void internalHll4Update(int slotNo, int newVal);
+    void shiftToBiggerCurMin();
+
+    AuxHashMap<A>* auxHashMap;
+};
+
+}
+
+#endif /* _HLL4ARRAY_HPP_ */
diff --git a/be/src/thirdparty/datasketches/Hll6Array-internal.hpp b/be/src/thirdparty/datasketches/Hll6Array-internal.hpp
new file mode 100644
index 0000000..a318564
--- /dev/null
+++ b/be/src/thirdparty/datasketches/Hll6Array-internal.hpp
@@ -0,0 +1,124 @@
+/*
+ * 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 _HLL6ARRAY_INTERNAL_HPP_
+#define _HLL6ARRAY_INTERNAL_HPP_
+
+#include <cstring>
+
+#include "Hll6Array.hpp"
+
+namespace datasketches {
+
+template<typename A>
+Hll6Array<A>::Hll6Array(const int lgConfigK, const bool startFullSize) :
+    HllArray<A>(lgConfigK, target_hll_type::HLL_6, startFullSize) {
+  const int numBytes = this->hll6ArrBytes(lgConfigK);
+  typedef typename std::allocator_traits<A>::template rebind_alloc<uint8_t> uint8Alloc;
+  this->hllByteArr = uint8Alloc().allocate(numBytes);
+  std::fill(this->hllByteArr, this->hllByteArr + numBytes, 0);
+}
+
+template<typename A>
+Hll6Array<A>::Hll6Array(const Hll6Array<A>& that) :
+  HllArray<A>(that)
+{
+  // can determine hllByteArr size in parent class, no need to allocate here
+}
+
+template<typename A>
+Hll6Array<A>::~Hll6Array() {
+  // hllByteArr deleted in parent
+}
+
+template<typename A>
+std::function<void(HllSketchImpl<A>*)> Hll6Array<A>::get_deleter() const {
+  return [](HllSketchImpl<A>* ptr) {
+    typedef typename std::allocator_traits<A>::template rebind_alloc<Hll6Array<A>> hll6Alloc;
+    Hll6Array<A>* hll = static_cast<Hll6Array<A>*>(ptr);
+    hll->~Hll6Array();
+    hll6Alloc().deallocate(hll, 1);
+  };
+}
+
+template<typename A>
+Hll6Array<A>* Hll6Array<A>::copy() const {
+  typedef typename std::allocator_traits<A>::template rebind_alloc<Hll6Array<A>> hll6Alloc;
+  return new (hll6Alloc().allocate(1)) Hll6Array<A>(*this);
+}
+
+template<typename A>
+uint8_t Hll6Array<A>::getSlot(int slotNo) const {
+  const int startBit = slotNo * 6;
+  const int shift = startBit & 0x7;
+  const int byteIdx = startBit >> 3;  
+  const uint16_t twoByteVal = (this->hllByteArr[byteIdx + 1] << 8) | this->hllByteArr[byteIdx];
+  return (twoByteVal >> shift) & HllUtil<A>::VAL_MASK_6;
+}
+
+template<typename A>
+void Hll6Array<A>::putSlot(int slotNo, uint8_t value) {
+  const int startBit = slotNo * 6;
+  const int shift = startBit & 0x7;
+  const int byteIdx = startBit >> 3;
+  const uint16_t valShifted = (value & 0x3F) << shift;
+  uint16_t curMasked = (this->hllByteArr[byteIdx + 1] << 8) | this->hllByteArr[byteIdx];
+  curMasked &= (~(HllUtil<A>::VAL_MASK_6 << shift));
+  const uint16_t insert = curMasked | valShifted;
+  this->hllByteArr[byteIdx]     = insert & 0xFF;
+  this->hllByteArr[byteIdx + 1] = (insert & 0xFF00) >> 8;
+}
+
+template<typename A>
+int Hll6Array<A>::getHllByteArrBytes() const {
+  return this->hll6ArrBytes(this->lgConfigK);
+}
+
+template<typename A>
+HllSketchImpl<A>* Hll6Array<A>::couponUpdate(const int coupon) {
+  internalCouponUpdate(coupon);
+  return this;
+}
+
+template<typename A>
+void Hll6Array<A>::internalCouponUpdate(const int coupon) {
+  const int configKmask = (1 << this->lgConfigK) - 1;
+  const int slotNo = HllUtil<A>::getLow26(coupon) & configKmask;
+  const int newVal = HllUtil<A>::getValue(coupon);
+
+  const int curVal = getSlot(slotNo);
+  if (newVal > curVal) {
+    putSlot(slotNo, newVal);
+    this->hipAndKxQIncrementalUpdate(curVal, newVal);
+    if (curVal == 0) {
+      this->numAtCurMin--; // interpret numAtCurMin as num zeros
+    }
+  }
+}
+
+template<typename A>
+void Hll6Array<A>::mergeHll(const HllArray<A>& src) {
+  for (auto coupon: src) {
+    internalCouponUpdate(coupon);
+  }
+}
+
+}
+
+#endif // _HLL6ARRAY_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/Hll6Array.hpp b/be/src/thirdparty/datasketches/Hll6Array.hpp
new file mode 100644
index 0000000..5178de8
--- /dev/null
+++ b/be/src/thirdparty/datasketches/Hll6Array.hpp
@@ -0,0 +1,55 @@
+/*
+ * 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 _HLL6ARRAY_HPP_
+#define _HLL6ARRAY_HPP_
+
+#include "HllArray.hpp"
+
+namespace datasketches {
+
+template<typename A>
+class Hll6Iterator;
+
+template<typename A>
+class Hll6Array final : public HllArray<A> {
+  public:
+    explicit Hll6Array(int lgConfigK, bool startFullSize);
+    explicit Hll6Array(const Hll6Array<A>& that);
+
+    virtual ~Hll6Array();
+    virtual std::function<void(HllSketchImpl<A>*)> get_deleter() const;
+
+    virtual Hll6Array* copy() const;
+
+    inline uint8_t getSlot(int slotNo) const;
+    inline void putSlot(int slotNo, uint8_t value);
+
+    virtual HllSketchImpl<A>* couponUpdate(int coupon) final;
+    void mergeHll(const HllArray<A>& src);
+
+    virtual int getHllByteArrBytes() const;
+
+  private:
+    void internalCouponUpdate(int coupon);
+};
+
+}
+
+#endif /* _HLL6ARRAY_HPP_ */
diff --git a/be/src/thirdparty/datasketches/Hll8Array-internal.hpp b/be/src/thirdparty/datasketches/Hll8Array-internal.hpp
new file mode 100644
index 0000000..cb14a0f
--- /dev/null
+++ b/be/src/thirdparty/datasketches/Hll8Array-internal.hpp
@@ -0,0 +1,158 @@
+/*
+ * 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 _HLL8ARRAY_INTERNAL_HPP_
+#define _HLL8ARRAY_INTERNAL_HPP_
+
+#include "Hll8Array.hpp"
+
+namespace datasketches {
+
+template<typename A>
+Hll8Array<A>::Hll8Array(const int lgConfigK, const bool startFullSize) :
+    HllArray<A>(lgConfigK, target_hll_type::HLL_8, startFullSize) {
+  const int numBytes = this->hll8ArrBytes(lgConfigK);
+  typedef typename std::allocator_traits<A>::template rebind_alloc<uint8_t> uint8Alloc;
+  this->hllByteArr = uint8Alloc().allocate(numBytes);
+  std::fill(this->hllByteArr, this->hllByteArr + numBytes, 0);
+}
+
+template<typename A>
+Hll8Array<A>::Hll8Array(const Hll8Array<A>& that) :
+  HllArray<A>(that)
+{
+  // can determine hllByteArr size in parent class, no need to allocate here
+}
+
+template<typename A>
+Hll8Array<A>::~Hll8Array() {
+  // hllByteArr deleted in parent
+}
+
+template<typename A>
+std::function<void(HllSketchImpl<A>*)> Hll8Array<A>::get_deleter() const {
+  return [](HllSketchImpl<A>* ptr) {
+    typedef typename std::allocator_traits<A>::template rebind_alloc<Hll8Array<A>> hll8Alloc;
+    Hll8Array<A>* hll = static_cast<Hll8Array<A>*>(ptr);
+    hll->~Hll8Array();
+    hll8Alloc().deallocate(hll, 1);
+  };
+}
+
+template<typename A>
+Hll8Array<A>* Hll8Array<A>::copy() const {
+  typedef typename std::allocator_traits<A>::template rebind_alloc<Hll8Array<A>> hll8Alloc;
+  return new (hll8Alloc().allocate(1)) Hll8Array<A>(*this);
+}
+
+template<typename A>
+uint8_t Hll8Array<A>::getSlot(const int slotNo) const {
+  return this->hllByteArr[slotNo];
+}
+
+template<typename A>
+void Hll8Array<A>::putSlot(const int slotNo, uint8_t value) {
+  this->hllByteArr[slotNo] = value;
+}
+
+template<typename A>
+int Hll8Array<A>::getHllByteArrBytes() const {
+  return this->hll8ArrBytes(this->lgConfigK);
+}
+
+template<typename A>
+HllSketchImpl<A>* Hll8Array<A>::couponUpdate(int coupon) {
+  internalCouponUpdate(coupon);
+  return this;
+}
+
+template<typename A>
+void Hll8Array<A>::internalCouponUpdate(int coupon) {
+  const int configKmask = (1 << this->lgConfigK) - 1;
+  const int slotNo = HllUtil<A>::getLow26(coupon) & configKmask;
+  const int newVal = HllUtil<A>::getValue(coupon);
+
+  const int curVal = getSlot(slotNo);
+  if (newVal > curVal) {
+    putSlot(slotNo, newVal);
+    this->hipAndKxQIncrementalUpdate(curVal, newVal);
+    if (curVal == 0) {
+      this->numAtCurMin--; // interpret numAtCurMin as num zeros
+    }
+  }
+}
+
+template<typename A>
+void Hll8Array<A>::mergeList(const CouponList<A>& src) {
+  for (auto coupon: src) {
+    internalCouponUpdate(coupon);
+  }
+}
+
+template<typename A>
+void Hll8Array<A>::mergeHll(const HllArray<A>& src) {
+  // at this point src_k >= dst_k
+  const int src_k = 1 << src.getLgConfigK();
+  const  int dst_mask = (1 << this->getLgConfigK()) - 1;
+  // duplication below is to avoid a virtual method call in a loop
+  if (src.getTgtHllType() == target_hll_type::HLL_8) {
+    for (int i = 0; i < src_k; i++) {
+      const uint8_t new_v = static_cast<const Hll8Array<A>&>(src).getSlot(i);
+      const int j = i & dst_mask;
+      const uint8_t old_v = this->hllByteArr[j];
+      if (new_v > old_v) {
+        this->hllByteArr[j] = new_v;
+        this->hipAndKxQIncrementalUpdate(old_v, new_v);
+        if (old_v == 0) {
+          this->numAtCurMin--;
+        }
+      }
+    }
+  } else if (src.getTgtHllType() == target_hll_type::HLL_6) {
+    for (int i = 0; i < src_k; i++) {
+      const uint8_t new_v = static_cast<const Hll6Array<A>&>(src).getSlot(i);
+      const int j = i & dst_mask;
+      const uint8_t old_v = this->hllByteArr[j];
+      if (new_v > old_v) {
+        this->hllByteArr[j] = new_v;
+        this->hipAndKxQIncrementalUpdate(old_v, new_v);
+        if (old_v == 0) {
+          this->numAtCurMin--;
+        }
+      }
+    }
+  } else { // HLL_4
+    for (int i = 0; i < src_k; i++) {
+      const uint8_t new_v = static_cast<const Hll4Array<A>&>(src).get_value(i);
+      const int j = i & dst_mask;
+      const uint8_t old_v = this->hllByteArr[j];
+      if (new_v > old_v) {
+        this->hllByteArr[j] = new_v;
+        this->hipAndKxQIncrementalUpdate(old_v, new_v);
+        if (old_v == 0) {
+          this->numAtCurMin--;
+        }
+      }
+    }
+  }
+}
+
+}
+
+#endif // _HLL8ARRAY_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/Hll8Array.hpp b/be/src/thirdparty/datasketches/Hll8Array.hpp
new file mode 100644
index 0000000..2b0aefc
--- /dev/null
+++ b/be/src/thirdparty/datasketches/Hll8Array.hpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _HLL8ARRAY_HPP_
+#define _HLL8ARRAY_HPP_
+
+#include "HllArray.hpp"
+
+namespace datasketches {
+
+template<typename A>
+class Hll8Iterator;
+
+template<typename A>
+class Hll8Array final : public HllArray<A> {
+  public:
+    explicit Hll8Array(int lgConfigK, bool startFullSize);
+    explicit Hll8Array(const Hll8Array& that);
+
+    virtual ~Hll8Array();
+    virtual std::function<void(HllSketchImpl<A>*)> get_deleter() const;
+
+    virtual Hll8Array<A>* copy() const;
+
+    inline uint8_t getSlot(int slotNo) const;
+    inline void putSlot(int slotNo, uint8_t value);
+
+    virtual HllSketchImpl<A>* couponUpdate(int coupon) final;
+    void mergeList(const CouponList<A>& src);
+    void mergeHll(const HllArray<A>& src);
+
+    virtual int getHllByteArrBytes() const;
+
+  private:
+    inline void internalCouponUpdate(int coupon);
+};
+
+}
+
+#endif /* _HLL8ARRAY_HPP_ */
diff --git a/be/src/thirdparty/datasketches/HllArray-internal.hpp b/be/src/thirdparty/datasketches/HllArray-internal.hpp
new file mode 100644
index 0000000..b8c0a57
--- /dev/null
+++ b/be/src/thirdparty/datasketches/HllArray-internal.hpp
@@ -0,0 +1,698 @@
+/*
+ * 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 _HLLARRAY_INTERNAL_HPP_
+#define _HLLARRAY_INTERNAL_HPP_
+
+#include "HllArray.hpp"
+#include "HllUtil.hpp"
+#include "HarmonicNumbers.hpp"
+#include "CubicInterpolation.hpp"
+#include "CompositeInterpolationXTable.hpp"
+#include "CouponList.hpp"
+#include "inv_pow2_table.hpp"
+#include <cstring>
+#include <cmath>
+#include <stdexcept>
+#include <string>
+
+namespace datasketches {
+
+template<typename A>
+HllArray<A>::HllArray(const int lgConfigK, const target_hll_type tgtHllType, bool startFullSize)
+  : HllSketchImpl<A>(lgConfigK, tgtHllType, hll_mode::HLL, startFullSize) {
+  hipAccum = 0.0;
+  kxq0 = 1 << lgConfigK;
+  kxq1 = 0.0;
+  curMin = 0;
+  numAtCurMin = 1 << lgConfigK;
+  oooFlag = false;
+  hllByteArr = nullptr; // allocated in derived class
+}
+
+template<typename A>
+HllArray<A>::HllArray(const HllArray<A>& that):
+HllSketchImpl<A>(that.lgConfigK, that.tgtHllType, hll_mode::HLL, that.startFullSize),
+hipAccum(that.hipAccum),
+kxq0(that.kxq0),
+kxq1(that.kxq1),
+hllByteArr(nullptr),
+curMin(that.curMin),
+numAtCurMin(that.numAtCurMin),
+oooFlag(that.oooFlag)
+{
+  const int arrayLen = that.getHllByteArrBytes();
+  typedef typename std::allocator_traits<A>::template rebind_alloc<uint8_t> uint8Alloc;
+  hllByteArr = uint8Alloc().allocate(arrayLen);
+  std::copy(that.hllByteArr, that.hllByteArr + arrayLen, hllByteArr);
+}
+
+template<typename A>
+HllArray<A>::~HllArray() {
+  // need to determine number of bytes to deallocate
+  int hllArrBytes = 0;
+  if (this->tgtHllType == target_hll_type::HLL_4) {
+    hllArrBytes = hll4ArrBytes(this->lgConfigK);
+  } else if (this->tgtHllType == target_hll_type::HLL_6) {
+    hllArrBytes = hll6ArrBytes(this->lgConfigK);
+  } else { // tgtHllType == HLL_8
+    hllArrBytes = hll8ArrBytes(this->lgConfigK);
+  }
+  typedef typename std::allocator_traits<A>::template rebind_alloc<uint8_t> uint8Alloc;
+  uint8Alloc().deallocate(hllByteArr, hllArrBytes);
+}
+
+template<typename A>
+HllArray<A>* HllArray<A>::copyAs(const target_hll_type tgtHllType) const {
+  if (tgtHllType == this->getTgtHllType()) {
+    return static_cast<HllArray*>(copy());
+  }
+  if (tgtHllType == target_hll_type::HLL_4) {
+    return HllSketchImplFactory<A>::convertToHll4(*this);
+  } else if (tgtHllType == target_hll_type::HLL_6) {
+    return HllSketchImplFactory<A>::convertToHll6(*this);
+  } else { // tgtHllType == HLL_8
+    return HllSketchImplFactory<A>::convertToHll8(*this);
+  }
+}
+
+template<typename A>
+HllArray<A>* HllArray<A>::newHll(const void* bytes, size_t len) {
+  if (len < HllUtil<A>::HLL_BYTE_ARR_START) {
+    throw std::invalid_argument("Input data length insufficient to hold HLL array");
+  }
+
+  const uint8_t* data = static_cast<const uint8_t*>(bytes);
+  if (data[HllUtil<A>::PREAMBLE_INTS_BYTE] != HllUtil<A>::HLL_PREINTS) {
+    throw std::invalid_argument("Incorrect number of preInts in input stream");
+  }
+  if (data[HllUtil<A>::SER_VER_BYTE] != HllUtil<A>::SER_VER) {
+    throw std::invalid_argument("Wrong ser ver in input stream");
+  }
+  if (data[HllUtil<A>::FAMILY_BYTE] != HllUtil<A>::FAMILY_ID) {
+    throw std::invalid_argument("Input array is not an HLL sketch");
+  }
+
+  const hll_mode mode = HllSketchImpl<A>::extractCurMode(data[HllUtil<A>::MODE_BYTE]);
+  if (mode != HLL) {
+    throw std::invalid_argument("Calling HLL array construtor with non-HLL mode data");
+  }
+
+  const target_hll_type tgtHllType = HllSketchImpl<A>::extractTgtHllType(data[HllUtil<A>::MODE_BYTE]);
+  const bool oooFlag = ((data[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::OUT_OF_ORDER_FLAG_MASK) ? true : false);
+  const bool comapctFlag = ((data[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::COMPACT_FLAG_MASK) ? true : false);
+  const bool startFullSizeFlag = ((data[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::FULL_SIZE_FLAG_MASK) ? true : false);
+
+  const int lgK = (int) data[HllUtil<A>::LG_K_BYTE];
+  const int curMin = (int) data[HllUtil<A>::HLL_CUR_MIN_BYTE];
+
+  const int arrayBytes = hllArrBytes(tgtHllType, lgK);
+  if (len < static_cast<size_t>(HllUtil<A>::HLL_BYTE_ARR_START + arrayBytes)) {
+    throw std::invalid_argument("Input array too small to hold sketch image");
+  }
+
+  double hip, kxq0, kxq1;
+  std::memcpy(&hip, data + HllUtil<A>::HIP_ACCUM_DOUBLE, sizeof(double));
+  std::memcpy(&kxq0, data + HllUtil<A>::KXQ0_DOUBLE, sizeof(double));
+  std::memcpy(&kxq1, data + HllUtil<A>::KXQ1_DOUBLE, sizeof(double));
+
+  int numAtCurMin, auxCount;
+  std::memcpy(&numAtCurMin, data + HllUtil<A>::CUR_MIN_COUNT_INT, sizeof(int));
+  std::memcpy(&auxCount, data + HllUtil<A>::AUX_COUNT_INT, sizeof(int));
+
+  AuxHashMap<A>* auxHashMap = nullptr;
+  if (auxCount > 0) { // necessarily TgtHllType == HLL_4
+    int auxLgIntArrSize = (int) data[4];
+    const size_t offset = HllUtil<A>::HLL_BYTE_ARR_START + arrayBytes;
+    const uint8_t* auxDataStart = data + offset;
+    auxHashMap = AuxHashMap<A>::deserialize(auxDataStart, len - offset, lgK, auxCount, auxLgIntArrSize, comapctFlag);
+  }
+
+  HllArray<A>* sketch = HllSketchImplFactory<A>::newHll(lgK, tgtHllType, startFullSizeFlag);
+  sketch->putCurMin(curMin);
+  sketch->putOutOfOrderFlag(oooFlag);
+  sketch->putHipAccum(hip);
+  sketch->putKxQ0(kxq0);
+  sketch->putKxQ1(kxq1);
+  sketch->putNumAtCurMin(numAtCurMin);
+
+  std::memcpy(sketch->hllByteArr, data + HllUtil<A>::HLL_BYTE_ARR_START, arrayBytes);
+
+  if (auxHashMap != nullptr)
+    ((Hll4Array<A>*)sketch)->putAuxHashMap(auxHashMap);
+
+  return sketch;
+}
+
+template<typename A>
+HllArray<A>* HllArray<A>::newHll(std::istream& is) {
+  uint8_t listHeader[8];
+  is.read((char*)listHeader, 8 * sizeof(uint8_t));
+
+  if (listHeader[HllUtil<A>::PREAMBLE_INTS_BYTE] != HllUtil<A>::HLL_PREINTS) {
+    throw std::invalid_argument("Incorrect number of preInts in input stream");
+  }
+  if (listHeader[HllUtil<A>::SER_VER_BYTE] != HllUtil<A>::SER_VER) {
+    throw std::invalid_argument("Wrong ser ver in input stream");
+  }
+  if (listHeader[HllUtil<A>::FAMILY_BYTE] != HllUtil<A>::FAMILY_ID) {
+    throw std::invalid_argument("Input stream is not an HLL sketch");
+  }
+
+  hll_mode mode = HllSketchImpl<A>::extractCurMode(listHeader[HllUtil<A>::MODE_BYTE]);
+  if (mode != HLL) {
+    throw std::invalid_argument("Calling HLL construtor with non-HLL mode data");
+  }
+
+  const target_hll_type tgtHllType = HllSketchImpl<A>::extractTgtHllType(listHeader[HllUtil<A>::MODE_BYTE]);
+  const bool oooFlag = ((listHeader[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::OUT_OF_ORDER_FLAG_MASK) ? true : false);
+  const bool comapctFlag = ((listHeader[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::COMPACT_FLAG_MASK) ? true : false);
+  const bool startFullSizeFlag = ((listHeader[HllUtil<A>::FLAGS_BYTE] & HllUtil<A>::FULL_SIZE_FLAG_MASK) ? true : false);
+
+  const int lgK = (int) listHeader[HllUtil<A>::LG_K_BYTE];
+  const int curMin = (int) listHeader[HllUtil<A>::HLL_CUR_MIN_BYTE];
+
+  // TODO: truncated stream will throw exception without freeing memory
+  HllArray* sketch = HllSketchImplFactory<A>::newHll(lgK, tgtHllType, startFullSizeFlag);
+  sketch->putCurMin(curMin);
+  sketch->putOutOfOrderFlag(oooFlag);
+
+  double hip, kxq0, kxq1;
+  is.read((char*)&hip, sizeof(hip));
+  is.read((char*)&kxq0, sizeof(kxq0));
+  is.read((char*)&kxq1, sizeof(kxq1));
+  sketch->putHipAccum(hip);
+  sketch->putKxQ0(kxq0);
+  sketch->putKxQ1(kxq1);
+
+  int numAtCurMin, auxCount;
+  is.read((char*)&numAtCurMin, sizeof(numAtCurMin));
+  is.read((char*)&auxCount, sizeof(auxCount));
+  sketch->putNumAtCurMin(numAtCurMin);
+  
+  is.read((char*)sketch->hllByteArr, sketch->getHllByteArrBytes());
+  
+  if (auxCount > 0) { // necessarily TgtHllType == HLL_4
+    int auxLgIntArrSize = listHeader[4];
+    AuxHashMap<A>* auxHashMap = AuxHashMap<A>::deserialize(is, lgK, auxCount, auxLgIntArrSize, comapctFlag);
+    ((Hll4Array<A>*)sketch)->putAuxHashMap(auxHashMap);
+  }
+
+  return sketch;
+}
+
+template<typename A>
+vector_u8<A> HllArray<A>::serialize(bool compact, unsigned header_size_bytes) const {
+  const size_t sketchSizeBytes = (compact ? getCompactSerializationBytes() : getUpdatableSerializationBytes()) + header_size_bytes;
+  vector_u8<A> byteArr(sketchSizeBytes);
+  uint8_t* bytes = byteArr.data() + header_size_bytes;
+  AuxHashMap<A>* auxHashMap = getAuxHashMap();
+
+  bytes[HllUtil<A>::PREAMBLE_INTS_BYTE] = static_cast<uint8_t>(getPreInts());
+  bytes[HllUtil<A>::SER_VER_BYTE] = static_cast<uint8_t>(HllUtil<A>::SER_VER);
+  bytes[HllUtil<A>::FAMILY_BYTE] = static_cast<uint8_t>(HllUtil<A>::FAMILY_ID);
+  bytes[HllUtil<A>::LG_K_BYTE] = static_cast<uint8_t>(this->lgConfigK);
+  bytes[HllUtil<A>::LG_ARR_BYTE] = static_cast<uint8_t>(auxHashMap == nullptr ? 0 : auxHashMap->getLgAuxArrInts());
+  bytes[HllUtil<A>::FLAGS_BYTE] = this->makeFlagsByte(compact);
+  bytes[HllUtil<A>::HLL_CUR_MIN_BYTE] = static_cast<uint8_t>(curMin);
+  bytes[HllUtil<A>::MODE_BYTE] = this->makeModeByte();
+
+  std::memcpy(bytes + HllUtil<A>::HIP_ACCUM_DOUBLE, &hipAccum, sizeof(double));
+  std::memcpy(bytes + HllUtil<A>::KXQ0_DOUBLE, &kxq0, sizeof(double));
+  std::memcpy(bytes + HllUtil<A>::KXQ1_DOUBLE, &kxq1, sizeof(double));
+  std::memcpy(bytes + HllUtil<A>::CUR_MIN_COUNT_INT, &numAtCurMin, sizeof(int));
+  const int auxCount = (auxHashMap == nullptr ? 0 : auxHashMap->getAuxCount());
+  std::memcpy(bytes + HllUtil<A>::AUX_COUNT_INT, &auxCount, sizeof(int));
+
+  const int hllByteArrBytes = getHllByteArrBytes();
+  std::memcpy(bytes + getMemDataStart(), hllByteArr, hllByteArrBytes);
+
+  // aux map if HLL_4
+  if (this->tgtHllType == HLL_4) {
+    bytes += getMemDataStart() + hllByteArrBytes; // start of auxHashMap
+    if (auxHashMap != nullptr) {
+      if (compact) {
+        for (uint32_t coupon: *auxHashMap) {
+          std::memcpy(bytes, &coupon, sizeof(coupon));
+          bytes += sizeof(coupon);
+        }
+      } else {
+        std::memcpy(bytes, auxHashMap->getAuxIntArr(), auxHashMap->getUpdatableSizeBytes());
+      }
+    } else if (!compact) {
+      // if updatable, we write even if currently unused so the binary can be wrapped
+      int auxBytes = 4 << HllUtil<A>::LG_AUX_ARR_INTS[this->lgConfigK];
+      std::fill_n(bytes, auxBytes, 0);
+    }
+  }
+
+  return byteArr;
+}
+
+template<typename A>
+void HllArray<A>::serialize(std::ostream& os, const bool compact) const {
+  // header
+  const uint8_t preInts(getPreInts());
+  os.write((char*)&preInts, sizeof(preInts));
+  const uint8_t serialVersion(HllUtil<A>::SER_VER);
+  os.write((char*)&serialVersion, sizeof(serialVersion));
+  const uint8_t familyId(HllUtil<A>::FAMILY_ID);
+  os.write((char*)&familyId, sizeof(familyId));
+  const uint8_t lgKByte((uint8_t) this->lgConfigK);
+  os.write((char*)&lgKByte, sizeof(lgKByte));
+
+  AuxHashMap<A>* auxHashMap = getAuxHashMap();
+  uint8_t lgArrByte(0);
+  if (auxHashMap != nullptr) {
+    lgArrByte = auxHashMap->getLgAuxArrInts();
+  }
+  os.write((char*)&lgArrByte, sizeof(lgArrByte));
+
+  const uint8_t flagsByte(this->makeFlagsByte(compact));
+  os.write((char*)&flagsByte, sizeof(flagsByte));
+  const uint8_t curMinByte((uint8_t) curMin);
+  os.write((char*)&curMinByte, sizeof(curMinByte));
+  const uint8_t modeByte(this->makeModeByte());
+  os.write((char*)&modeByte, sizeof(modeByte));
+
+  // estimator data
+  os.write((char*)&hipAccum, sizeof(hipAccum));
+  os.write((char*)&kxq0, sizeof(kxq0));
+  os.write((char*)&kxq1, sizeof(kxq1));
+
+  // array data
+  os.write((char*)&numAtCurMin, sizeof(numAtCurMin));
+
+  const int auxCount = (auxHashMap == nullptr ? 0 : auxHashMap->getAuxCount());
+  os.write((char*)&auxCount, sizeof(auxCount));
+  os.write((char*)hllByteArr, getHllByteArrBytes());
+
+  // aux map if HLL_4
+  if (this->tgtHllType == HLL_4) {
+    if (auxHashMap != nullptr) {
+      if (compact) {
+        for (uint32_t coupon: *auxHashMap) {
+          os.write((char*)&coupon, sizeof(coupon));
+        }
+      } else {
+        os.write((char*)auxHashMap->getAuxIntArr(), auxHashMap->getUpdatableSizeBytes());
+      }
+    } else if (!compact) {
+      // if updatable, we write even if currently unused so the binary can be wrapped      
+      int auxBytes = 4 << HllUtil<A>::LG_AUX_ARR_INTS[this->lgConfigK];
+      std::fill_n(std::ostreambuf_iterator<char>(os), auxBytes, 0);
+    }
+  }
+}
+
+template<typename A>
+double HllArray<A>::getEstimate() const {
+  if (oooFlag) {
+    return getCompositeEstimate();
+  }
+  return getHipAccum();
+}
+
+// HLL UPPER AND LOWER BOUNDS
+
+/*
+ * The upper and lower bounds are not symmetric and thus are treated slightly differently.
+ * For the lower bound, when the unique count is <= k, LB >= numNonZeros, where
+ * numNonZeros = k - numAtCurMin AND curMin == 0.
+ *
+ * For HLL6 and HLL8, curMin is always 0 and numAtCurMin is initialized to k and is decremented
+ * down for each valid update until it reaches 0, where it stays. Thus, for these two
+ * isomorphs, when numAtCurMin = 0, means the true curMin is > 0 and the unique count must be
+ * greater than k.
+ *
+ * HLL4 always maintains both curMin and numAtCurMin dynamically. Nonetheless, the rules for
+ * the very small values <= k where curMin = 0 still apply.
+ */
+template<typename A>
+double HllArray<A>::getLowerBound(const int numStdDev) const {
+  HllUtil<A>::checkNumStdDev(numStdDev);
+  const int configK = 1 << this->lgConfigK;
+  const double numNonZeros = ((curMin == 0) ? (configK - numAtCurMin) : configK);
+
+  double estimate;
+  double rseFactor;
+  if (oooFlag) {
+    estimate = getCompositeEstimate();
+    rseFactor = HllUtil<A>::HLL_NON_HIP_RSE_FACTOR;
+  } else {
+    estimate = hipAccum;
+    rseFactor = HllUtil<A>::HLL_HIP_RSE_FACTOR;
+  }
+
+  double relErr;
+  if (this->lgConfigK > 12) {
+    relErr = (numStdDev * rseFactor) / sqrt(configK);
+  } else {
+    relErr = HllUtil<A>::getRelErr(false, oooFlag, this->lgConfigK, numStdDev);
+  }
+  return fmax(estimate / (1.0 + relErr), numNonZeros);
+}
+
+template<typename A>
+double HllArray<A>::getUpperBound(const int numStdDev) const {
+  HllUtil<A>::checkNumStdDev(numStdDev);
+  const int configK = 1 << this->lgConfigK;
+
+  double estimate;
+  double rseFactor;
+  if (oooFlag) {
+    estimate = getCompositeEstimate();
+    rseFactor = HllUtil<A>::HLL_NON_HIP_RSE_FACTOR;
+  } else {
+    estimate = hipAccum;
+    rseFactor = HllUtil<A>::HLL_HIP_RSE_FACTOR;
+  }
+
+  double relErr;
+  if (this->lgConfigK > 12) {
+    relErr = (-1.0) * (numStdDev * rseFactor) / sqrt(configK);
+  } else {
+    relErr = HllUtil<A>::getRelErr(true, oooFlag, this->lgConfigK, numStdDev);
+  }
+  return estimate / (1.0 + relErr);
+}
+
+/**
+ * This is the (non-HIP) estimator.
+ * It is called "composite" because multiple estimators are pasted together.
+ * @param absHllArr an instance of the AbstractHllArray class.
+ * @return the composite estimate
+ */
+// Original C: again-two-registers.c hhb_get_composite_estimate L1489
+template<typename A>
+double HllArray<A>::getCompositeEstimate() const {
+  const double rawEst = getHllRawEstimate(this->lgConfigK, kxq0 + kxq1);
+
+  const double* xArr = CompositeInterpolationXTable<A>::get_x_arr(this->lgConfigK);
+  const int xArrLen = CompositeInterpolationXTable<A>::get_x_arr_length(this->lgConfigK);
+  const double yStride = CompositeInterpolationXTable<A>::get_y_stride(this->lgConfigK);
+
+  if (rawEst < xArr[0]) {
+    return 0;
+  }
+
+  const int xArrLenM1 = xArrLen - 1;
+
+  if (rawEst > xArr[xArrLenM1]) {
+    double finalY = yStride * xArrLenM1;
+    double factor = finalY / xArr[xArrLenM1];
+    return rawEst * factor;
+  }
+
+  double adjEst = CubicInterpolation<A>::usingXArrAndYStride(xArr, xArrLen, yStride, rawEst);
+
+  // We need to completely avoid the linear_counting estimator if it might have a crazy value.
+  // Empirical evidence suggests that the threshold 3*k will keep us safe if 2^4 <= k <= 2^21.
+
+  if (adjEst > (3 << this->lgConfigK)) { return adjEst; }
+
+  const double linEst =
+      getHllBitMapEstimate(this->lgConfigK, curMin, numAtCurMin);
+
+  // Bias is created when the value of an estimator is compared with a threshold to decide whether
+  // to use that estimator or a different one.
+  // We conjecture that less bias is created when the average of the two estimators
+  // is compared with the threshold. Empirical measurements support this conjecture.
+
+  const double avgEst = (adjEst + linEst) / 2.0;
+
+  // The following constants comes from empirical measurements of the crossover point
+  // between the average error of the linear estimator and the adjusted hll estimator
+  double crossOver = 0.64;
+  if (this->lgConfigK == 4)      { crossOver = 0.718; }
+  else if (this->lgConfigK == 5) { crossOver = 0.672; }
+
+  return (avgEst > (crossOver * (1 << this->lgConfigK))) ? adjEst : linEst;
+}
+
+template<typename A>
+double HllArray<A>::getKxQ0() const {
+  return kxq0;
+}
+
+template<typename A>
+double HllArray<A>::getKxQ1() const {
+  return kxq1;
+}
+
+template<typename A>
+double HllArray<A>::getHipAccum() const {
+  return hipAccum;
+}
+
+template<typename A>
+int HllArray<A>::getCurMin() const {
+  return curMin;
+}
+
+template<typename A>
+int HllArray<A>::getNumAtCurMin() const {
+  return numAtCurMin;
+}
+
+template<typename A>
+void HllArray<A>::putKxQ0(const double kxq0) {
+  this->kxq0 = kxq0;
+}
+
+template<typename A>
+void HllArray<A>::putKxQ1(const double kxq1) {
+  this->kxq1 = kxq1;
+}
+
+template<typename A>
+void HllArray<A>::putHipAccum(const double hipAccum) {
+  this->hipAccum = hipAccum;
+}
+
+template<typename A>
+void HllArray<A>::putCurMin(const int curMin) {
+  this->curMin = curMin;
+}
+
+template<typename A>
+void HllArray<A>::putNumAtCurMin(const int numAtCurMin) {
+  this->numAtCurMin = numAtCurMin;
+}
+
+template<typename A>
+void HllArray<A>::decNumAtCurMin() {
+  --numAtCurMin;
+}
+
+template<typename A>
+void HllArray<A>::addToHipAccum(const double delta) {
+  hipAccum += delta;
+}
+
+template<typename A>
+bool HllArray<A>::isCompact() const {
+  return false;
+}
+
+template<typename A>
+bool HllArray<A>::isEmpty() const {
+  const int configK = 1 << this->lgConfigK;
+  return (getCurMin() == 0) && (getNumAtCurMin() == configK);
+}
+
+template<typename A>
+void HllArray<A>::putOutOfOrderFlag(bool flag) {
+  oooFlag = flag;
+}
+
+template<typename A>
+bool HllArray<A>::isOutOfOrderFlag() const {
+  return oooFlag;
+}
+
+template<typename A>
+int HllArray<A>::hllArrBytes(target_hll_type tgtHllType, int lgConfigK) {
+  switch (tgtHllType) {
+  case HLL_4:
+    return hll4ArrBytes(lgConfigK);
+  case HLL_6:
+    return hll6ArrBytes(lgConfigK);
+  case HLL_8:
+    return hll8ArrBytes(lgConfigK);
+  default:
+    throw std::invalid_argument("Invalid target HLL type"); 
+  }
+}
+
+template<typename A>
+int HllArray<A>::hll4ArrBytes(const int lgConfigK) {
+  return 1 << (lgConfigK - 1);
+}
+
+template<typename A>
+int HllArray<A>::hll6ArrBytes(const int lgConfigK) {
+  const int numSlots = 1 << lgConfigK;
+  return ((numSlots * 3) >> 2) + 1;
+}
+
+template<typename A>
+int HllArray<A>::hll8ArrBytes(const int lgConfigK) {
+  return 1 << lgConfigK;
+}
+
+template<typename A>
+int HllArray<A>::getMemDataStart() const {
+  return HllUtil<A>::HLL_BYTE_ARR_START;
+}
+
+template<typename A>
+int HllArray<A>::getUpdatableSerializationBytes() const {
+  return HllUtil<A>::HLL_BYTE_ARR_START + getHllByteArrBytes();
+}
+
+template<typename A>
+int HllArray<A>::getCompactSerializationBytes() const {
+  AuxHashMap<A>* auxHashMap = getAuxHashMap();
+  const int auxCountBytes = ((auxHashMap == nullptr) ? 0 : auxHashMap->getCompactSizeBytes());
+  return HllUtil<A>::HLL_BYTE_ARR_START + getHllByteArrBytes() + auxCountBytes;
+}
+
+template<typename A>
+int HllArray<A>::getPreInts() const {
+  return HllUtil<A>::HLL_PREINTS;
+}
+
+template<typename A>
+AuxHashMap<A>* HllArray<A>::getAuxHashMap() const {
+  return nullptr;
+}
+
+template<typename A>
+void HllArray<A>::hipAndKxQIncrementalUpdate(uint8_t oldValue, uint8_t newValue) {
+  const int configK = 1 << this->getLgConfigK();
+  // update hip BEFORE updating kxq
+  hipAccum += configK / (kxq0 + kxq1);
+  // update kxq0 and kxq1; subtract first, then add
+  if (oldValue < 32) { kxq0 -= INVERSE_POWERS_OF_2[oldValue]; }
+  else               { kxq1 -= INVERSE_POWERS_OF_2[oldValue]; }
+  if (newValue < 32) { kxq0 += INVERSE_POWERS_OF_2[newValue]; }
+  else               { kxq1 += INVERSE_POWERS_OF_2[newValue]; }
+}
+
+/**
+ * Estimator when N is small, roughly less than k log(k).
+ * Refer to Wikipedia: Coupon Collector Problem
+ * @return the very low range estimate
+ */
+//In C: again-two-registers.c hhb_get_improved_linear_counting_estimate L1274
+template<typename A>
+double HllArray<A>::getHllBitMapEstimate(const int lgConfigK, const int curMin, const int numAtCurMin) const {
+  const  int configK = 1 << lgConfigK;
+  const  int numUnhitBuckets =  ((curMin == 0) ? numAtCurMin : 0);
+
+  //This will eventually go away.
+  if (numUnhitBuckets == 0) {
+    return configK * log(configK / 0.5);
+  }
+
+  const int numHitBuckets = configK - numUnhitBuckets;
+  return HarmonicNumbers<A>::getBitMapEstimate(configK, numHitBuckets);
+}
+
+//In C: again-two-registers.c hhb_get_raw_estimate L1167
+template<typename A>
+double HllArray<A>::getHllRawEstimate(const int lgConfigK, const double kxqSum) const {
+  const int configK = 1 << lgConfigK;
+  double correctionFactor;
+  if (lgConfigK == 4) { correctionFactor = 0.673; }
+  else if (lgConfigK == 5) { correctionFactor = 0.697; }
+  else if (lgConfigK == 6) { correctionFactor = 0.709; }
+  else { correctionFactor = 0.7213 / (1.0 + (1.079 / configK)); }
+  const double hyperEst = (correctionFactor * configK * configK) / kxqSum;
+  return hyperEst;
+}
+
+template<typename A>
+typename HllArray<A>::const_iterator HllArray<A>::begin(bool all) const {
+  return const_iterator(hllByteArr, 1 << this->lgConfigK, 0, this->tgtHllType, nullptr, 0, all);
+}
+
+template<typename A>
+typename HllArray<A>::const_iterator HllArray<A>::end() const {
+  return const_iterator(hllByteArr, 1 << this->lgConfigK, 1 << this->lgConfigK, this->tgtHllType, nullptr, 0, false);
+}
+
+template<typename A>
+HllArray<A>::const_iterator::const_iterator(const uint8_t* array, size_t array_size, size_t index, target_hll_type hll_type, const AuxHashMap<A>* exceptions, uint8_t offset, bool all):
+array(array), array_size(array_size), index(index), hll_type(hll_type), exceptions(exceptions), offset(offset), all(all)
+{
+  while (this->index < array_size) {
+    value = get_value(array, this->index, hll_type, exceptions, offset);
+    if (all || value != HllUtil<A>::EMPTY) break;
+    this->index++;
+  }
+}
+
+template<typename A>
+typename HllArray<A>::const_iterator& HllArray<A>::const_iterator::operator++() {
+  while (++index < array_size) {
+    value = get_value(array, index, hll_type, exceptions, offset);
+    if (all || value != HllUtil<A>::EMPTY) break;
+  }
+  return *this;
+}
+
+template<typename A>
+bool HllArray<A>::const_iterator::operator!=(const const_iterator& other) const {
+  return index != other.index;
+}
+
+template<typename A>
+uint32_t HllArray<A>::const_iterator::operator*() const {
+  return HllUtil<A>::pair(index, value);
+}
+
+template<typename A>
+uint8_t HllArray<A>::const_iterator::get_value(const uint8_t* array, size_t index, target_hll_type hll_type, const AuxHashMap<A>* exceptions, uint8_t offset) {
+  if (hll_type == target_hll_type::HLL_4) {
+    uint8_t value = array[index >> 1];
+    if ((index & 1) > 0) { // odd
+        value >>= 4;
+    } else {
+      value &= HllUtil<A>::loNibbleMask;
+    }
+    if (value == HllUtil<A>::AUX_TOKEN) { // exception
+      return exceptions->mustFindValueFor(index);
+    }
+    return value + offset;
+  } else if (hll_type == target_hll_type::HLL_6) {
+    const int start_bit = index * 6;
+    const int shift = start_bit & 0x7;
+    const int byte_idx = start_bit >> 3;
+    const uint16_t two_byte_val = (array[byte_idx + 1] << 8) | array[byte_idx];
+    return (two_byte_val >> shift) & HllUtil<A>::VAL_MASK_6;
+  }
+  // HLL_8
+  return array[index];
+}
+
+}
+
+#endif // _HLLARRAY_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/HllArray.hpp b/be/src/thirdparty/datasketches/HllArray.hpp
new file mode 100644
index 0000000..1cc64ea
--- /dev/null
+++ b/be/src/thirdparty/datasketches/HllArray.hpp
@@ -0,0 +1,136 @@
+/*
+ * 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 _HLLARRAY_HPP_
+#define _HLLARRAY_HPP_
+
+#include "HllSketchImpl.hpp"
+#include "HllUtil.hpp"
+
+namespace datasketches {
+
+template<typename A>
+class AuxHashMap;
+
+template<typename A = std::allocator<char>>
+class HllArray : public HllSketchImpl<A> {
+  public:
+    explicit HllArray(int lgConfigK, target_hll_type tgtHllType, bool startFullSize);
+    explicit HllArray(const HllArray<A>& that);
+
+    static HllArray* newHll(const void* bytes, size_t len);
+    static HllArray* newHll(std::istream& is);
+
+    virtual vector_u8<A> serialize(bool compact, unsigned header_size_bytes) const;
+    virtual void serialize(std::ostream& os, bool compact) const;
+
+    virtual ~HllArray();
+    virtual std::function<void(HllSketchImpl<A>*)> get_deleter() const = 0;
+
+    virtual HllArray* copy() const = 0;
+    virtual HllArray* copyAs(target_hll_type tgtHllType) const;
+
+    virtual HllSketchImpl<A>* couponUpdate(int coupon) = 0;
+
+    virtual double getEstimate() const;
+    virtual double getCompositeEstimate() const;
+    virtual double getLowerBound(int numStdDev) const;
+    virtual double getUpperBound(int numStdDev) const;
+
+    inline void addToHipAccum(double delta);
+
+    inline void decNumAtCurMin();
+
+    inline int getCurMin() const;
+    inline int getNumAtCurMin() const;
+    inline double getHipAccum() const;
+
+    virtual int getHllByteArrBytes() const = 0;
+
+    virtual int getUpdatableSerializationBytes() const;
+    virtual int getCompactSerializationBytes() const;
+
+    virtual bool isOutOfOrderFlag() const;
+    virtual bool isEmpty() const;
+    virtual bool isCompact() const;
+
+    virtual void putOutOfOrderFlag(bool flag);
+
+    inline double getKxQ0() const;
+    inline double getKxQ1() const;
+
+    virtual int getMemDataStart() const;
+    virtual int getPreInts() const;
+
+    void putCurMin(int curMin);
+    void putHipAccum(double hipAccum);
+    inline void putKxQ0(double kxq0);
+    inline void putKxQ1(double kxq1);
+    void putNumAtCurMin(int numAtCurMin);
+
+    static int hllArrBytes(target_hll_type tgtHllType, int lgConfigK);
+    static int hll4ArrBytes(int lgConfigK);
+    static int hll6ArrBytes(int lgConfigK);
+    static int hll8ArrBytes(int lgConfigK);
+
+    virtual AuxHashMap<A>* getAuxHashMap() const;
+
+    class const_iterator;
+    virtual const_iterator begin(bool all = false) const;
+    virtual const_iterator end() const;
+
+  protected:
+    void hipAndKxQIncrementalUpdate(uint8_t oldValue, uint8_t newValue);
+    double getHllBitMapEstimate(int lgConfigK, int curMin, int numAtCurMin) const;
+    double getHllRawEstimate(int lgConfigK, double kxqSum) const;
+
+    double hipAccum;
+    double kxq0;
+    double kxq1;
+    uint8_t* hllByteArr; //init by sub-classes
+    int curMin; //always zero for Hll6 and Hll8, only tracked by Hll4Array
+    int numAtCurMin; //interpreted as num zeros when curMin == 0
+    bool oooFlag; //Out-Of-Order Flag
+
+    friend class HllSketchImplFactory<A>;
+};
+
+template<typename A>
+class HllArray<A>::const_iterator: public std::iterator<std::input_iterator_tag, uint32_t> {
+public:
+  const_iterator(const uint8_t* array, size_t array_slze, size_t index, target_hll_type hll_type, const AuxHashMap<A>* exceptions, uint8_t offset, bool all);
+  //const_iterator(const uint8_t* array, size_t array_slze, size_t index, target_hll_type hll_type, const AuxHashMap<A>* exceptions, uint8_t offset);
+  const_iterator& operator++();
+  bool operator!=(const const_iterator& other) const;
+  uint32_t operator*() const;
+private:
+  const uint8_t* array;
+  size_t array_size;
+  size_t index;
+  target_hll_type hll_type;
+  const AuxHashMap<A>* exceptions;
+  uint8_t offset;
+  bool all;
+  uint8_t value; // cached value to avoid computing in operator++ and in operator*()
+  static inline uint8_t get_value(const uint8_t* array, size_t index, target_hll_type hll_type, const AuxHashMap<A>* exceptions, uint8_t offset);
+};
+
+}
+
+#endif /* _HLLARRAY_HPP_ */
diff --git a/be/src/thirdparty/datasketches/HllSketch-internal.hpp b/be/src/thirdparty/datasketches/HllSketch-internal.hpp
new file mode 100644
index 0000000..6587fe8
--- /dev/null
+++ b/be/src/thirdparty/datasketches/HllSketch-internal.hpp
@@ -0,0 +1,472 @@
+/*
+ * 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 _HLLSKETCH_INTERNAL_HPP_
+#define _HLLSKETCH_INTERNAL_HPP_
+
+#include "hll.hpp"
+#include "HllUtil.hpp"
+#include "HllSketchImplFactory.hpp"
+#include "CouponList.hpp"
+#include "HllArray.hpp"
+
+#include <cstdio>
+#include <cstdlib>
+#include <string>
+#include <iostream>
+#include <sstream>
+#include <iomanip>
+
+namespace datasketches {
+
+typedef union {
+  int64_t longBytes;
+  double doubleBytes;
+} longDoubleUnion;
+
+template<typename A>
+hll_sketch_alloc<A>::hll_sketch_alloc(int lg_config_k, target_hll_type tgt_type, bool start_full_size) {
+  HllUtil<A>::checkLgK(lg_config_k);
+  if (start_full_size) {
+    sketch_impl = HllSketchImplFactory<A>::newHll(lg_config_k, tgt_type, start_full_size);
+  } else {
+    typedef typename std::allocator_traits<A>::template rebind_alloc<CouponList<A>> clAlloc;
+    sketch_impl = new (clAlloc().allocate(1)) CouponList<A>(lg_config_k, tgt_type, hll_mode::LIST);
+  }
+}
+
+template<typename A>
+hll_sketch_alloc<A> hll_sketch_alloc<A>::deserialize(std::istream& is) {
+  HllSketchImpl<A>* impl = HllSketchImplFactory<A>::deserialize(is);
+  hll_sketch_alloc<A> sketch(impl);
+  return sketch;
+}
+
+template<typename A>
+hll_sketch_alloc<A> hll_sketch_alloc<A>::deserialize(const void* bytes, size_t len) {
+  HllSketchImpl<A>* impl = HllSketchImplFactory<A>::deserialize(bytes, len);
+  hll_sketch_alloc<A> sketch(impl);
+  return sketch;
+}
+
+template<typename A>
+hll_sketch_alloc<A>::~hll_sketch_alloc() {
+  if (sketch_impl != nullptr) {
+    sketch_impl->get_deleter()(sketch_impl);
+  }
+}
+
+template<typename A>
+std::ostream& operator<<(std::ostream& os, const hll_sketch_alloc<A>& sketch) {
+  return sketch.to_string(os, true, true, false, false);
+}
+
+template<typename A>
+hll_sketch_alloc<A>::hll_sketch_alloc(const hll_sketch_alloc<A>& that) :
+  sketch_impl(that.sketch_impl->copy())
+{}
+
+template<typename A>
+hll_sketch_alloc<A>::hll_sketch_alloc(const hll_sketch_alloc<A>& that, target_hll_type tgt_type) :
+  sketch_impl(that.sketch_impl->copyAs(tgt_type))
+{}
+
+template<typename A>
+hll_sketch_alloc<A>::hll_sketch_alloc(hll_sketch_alloc<A>&& that) noexcept :
+  sketch_impl(nullptr)
+{
+  std::swap(sketch_impl, that.sketch_impl);
+}
+
+template<typename A>
+hll_sketch_alloc<A>::hll_sketch_alloc(HllSketchImpl<A>* that) :
+  sketch_impl(that)
+{}
+
+template<typename A>
+hll_sketch_alloc<A> hll_sketch_alloc<A>::operator=(const hll_sketch_alloc<A>& other) {
+  sketch_impl->get_deleter()(sketch_impl);
+  sketch_impl = other.sketch_impl->copy();
+  return *this;
+}
+
+template<typename A>
+hll_sketch_alloc<A> hll_sketch_alloc<A>::operator=(hll_sketch_alloc<A>&& other) {
+  std::swap(sketch_impl, other.sketch_impl);
+  return *this;
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::reset() {
+  // TODO: need to allow starting from a full-sized sketch
+  //       (either here or in other implementation)
+  sketch_impl = sketch_impl->reset();
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const std::string& datum) {
+  if (datum.empty()) { return; }
+  HashState hashResult;
+  HllUtil<A>::hash(datum.c_str(), datum.length(), HllUtil<A>::DEFAULT_UPDATE_SEED, hashResult);
+  coupon_update(HllUtil<A>::coupon(hashResult));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const uint64_t datum) {
+  // no sign extension with 64 bits so no need to cast to signed value
+  HashState hashResult;
+  HllUtil<A>::hash(&datum, sizeof(uint64_t), HllUtil<A>::DEFAULT_UPDATE_SEED, hashResult);
+  coupon_update(HllUtil<A>::coupon(hashResult));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const uint32_t datum) {
+  update(static_cast<int32_t>(datum));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const uint16_t datum) {
+  update(static_cast<int16_t>(datum));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const uint8_t datum) {
+  update(static_cast<int8_t>(datum));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const int64_t datum) {
+  HashState hashResult;
+  HllUtil<A>::hash(&datum, sizeof(int64_t), HllUtil<A>::DEFAULT_UPDATE_SEED, hashResult);
+  coupon_update(HllUtil<A>::coupon(hashResult));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const int32_t datum) {
+  int64_t val = static_cast<int64_t>(datum);
+  HashState hashResult;
+  HllUtil<A>::hash(&val, sizeof(int64_t), HllUtil<A>::DEFAULT_UPDATE_SEED, hashResult);
+  coupon_update(HllUtil<A>::coupon(hashResult));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const int16_t datum) {
+  int64_t val = static_cast<int64_t>(datum);
+  HashState hashResult;
+  HllUtil<A>::hash(&val, sizeof(int64_t), HllUtil<A>::DEFAULT_UPDATE_SEED, hashResult);
+  coupon_update(HllUtil<A>::coupon(hashResult));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const int8_t datum) {
+  int64_t val = static_cast<int64_t>(datum);
+  HashState hashResult;
+  HllUtil<A>::hash(&val, sizeof(int64_t), HllUtil<A>::DEFAULT_UPDATE_SEED, hashResult);
+  coupon_update(HllUtil<A>::coupon(hashResult));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const double datum) {
+  longDoubleUnion d;
+  d.doubleBytes = static_cast<double>(datum);
+  if (datum == 0.0) {
+    d.doubleBytes = 0.0; // canonicalize -0.0 to 0.0
+  } else if (std::isnan(d.doubleBytes)) {
+    d.longBytes = 0x7ff8000000000000L; // canonicalize NaN using value from Java's Double.doubleToLongBits()
+  }
+  HashState hashResult;
+  HllUtil<A>::hash(&d, sizeof(double), HllUtil<A>::DEFAULT_UPDATE_SEED, hashResult);
+  coupon_update(HllUtil<A>::coupon(hashResult));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const float datum) {
+  longDoubleUnion d;
+  d.doubleBytes = static_cast<double>(datum);
+  if (datum == 0.0) {
+    d.doubleBytes = 0.0; // canonicalize -0.0 to 0.0
+  } else if (std::isnan(d.doubleBytes)) {
+    d.longBytes = 0x7ff8000000000000L; // canonicalize NaN using value from Java's Double.doubleToLongBits()
+  }
+  HashState hashResult;
+  HllUtil<A>::hash(&d, sizeof(double), HllUtil<A>::DEFAULT_UPDATE_SEED, hashResult);
+  coupon_update(HllUtil<A>::coupon(hashResult));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::update(const void* data, const size_t lengthBytes) {
+  if (data == nullptr) { return; }
+  HashState hashResult;
+  HllUtil<A>::hash(data, lengthBytes, HllUtil<A>::DEFAULT_UPDATE_SEED, hashResult);
+  coupon_update(HllUtil<A>::coupon(hashResult));
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::coupon_update(int coupon) {
+  if (coupon == HllUtil<A>::EMPTY) { return; }
+  HllSketchImpl<A>* result = this->sketch_impl->couponUpdate(coupon);
+  if (result != this->sketch_impl) {
+    this->sketch_impl->get_deleter()(this->sketch_impl);
+    this->sketch_impl = result;
+  }
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::serialize_compact(std::ostream& os) const {
+  return sketch_impl->serialize(os, true);
+}
+
+template<typename A>
+void hll_sketch_alloc<A>::serialize_updatable(std::ostream& os) const {
+  return sketch_impl->serialize(os, false);
+}
+
+template<typename A>
+vector_u8<A> hll_sketch_alloc<A>::serialize_compact(unsigned header_size_bytes) const {
+  return sketch_impl->serialize(true, header_size_bytes);
+}
+
+template<typename A>
+vector_u8<A> hll_sketch_alloc<A>::serialize_updatable() const {
+  return sketch_impl->serialize(false, 0);
+}
+
+template<typename A>
+std::string hll_sketch_alloc<A>::to_string(const bool summary,
+                                    const bool detail,
+                                    const bool aux_detail,
+                                    const bool all) const {
+  std::ostringstream oss;
+  to_string(oss, summary, detail, aux_detail, all);
+  return oss.str();
+}
+
+template<typename A>
+std::ostream& hll_sketch_alloc<A>::to_string(std::ostream& os,
+                                      const bool summary,
+                                      const bool detail,
+                                      const bool aux_detail,
+                                      const bool all) const {
+  if (summary) {
+    os << "### HLL sketch summary:" << std::endl
+       << "  Log Config K   : " << get_lg_config_k() << std::endl
+       << "  Hll Target     : " << type_as_string() << std::endl
+       << "  Current Mode   : " << mode_as_string() << std::endl
+       << "  LB             : " << get_lower_bound(1) << std::endl
+       << "  Estimate       : " << get_estimate() << std::endl
+       << "  UB             : " << get_upper_bound(1) << std::endl
+       << "  OutOfOrder flag: " << (is_out_of_order_flag() ? "true" : "false") << std::endl;
+    if (get_current_mode() == HLL) {
+      HllArray<A>* hllArray = (HllArray<A>*) sketch_impl;
+      os << "  CurMin         : " << hllArray->getCurMin() << std::endl
+         << "  NumAtCurMin    : " << hllArray->getNumAtCurMin() << std::endl
+         << "  HipAccum       : " << hllArray->getHipAccum() << std::endl
+         << "  KxQ0           : " << hllArray->getKxQ0() << std::endl
+         << "  KxQ1           : " << hllArray->getKxQ1() << std::endl;
+    } else {
+      os << "  Coupon count   : "
+         << std::to_string(((CouponList<A>*) sketch_impl)->getCouponCount()) << std::endl;
+    }
+    os << "### End HLL sketch summary" << std::endl;
+  }
+
+  if (detail) {
+    os << "### HLL sketch data detail:" << std::endl;
+    if (get_current_mode() == HLL) {
+      const HllArray<A>* hll_ptr = static_cast<const HllArray<A>*>(sketch_impl);
+      os << std::left << std::setw(10) << "Slot" << std::setw(6) << "Value" << std::endl;
+      auto it = hll_ptr->begin(all);
+      while (it != hll_ptr->end()) {
+        os << std::setw(10) << HllUtil<A>::getLow26(*it);
+        os << std::setw(6) << HllUtil<A>::getValue(*it);
+        os << std::endl;
+        ++it;
+      }
+    } else {
+      const CouponList<A>* list_ptr = static_cast<const CouponList<A>*>(sketch_impl);
+      os << std::left;
+      os << std::setw(10) << "Index";
+      os << std::setw(10) << "Key";
+      os << std::setw(10) << "Slot";
+      os << std::setw(6) << "Value";
+      os << std::endl;
+      auto it = list_ptr->begin(all);
+      int i = 0;
+      int mask = (1 << get_lg_config_k()) - 1;
+      while (it != list_ptr->end()) {
+        os << std::setw(10) << i;
+        os << std::setw(10) << HllUtil<A>::getLow26(*it);
+        os << std::setw(10) << (HllUtil<A>::getLow26(*it) & mask);
+        os << std::setw(6) << HllUtil<A>::getValue(*it);
+        os << std::endl;
+        ++it;
+        ++i;
+      }
+    }
+    os << "### End HLL sketch data detail" << std::endl;
+  }
+  if (aux_detail) {
+    if ((get_current_mode() == HLL) && (get_target_type() == HLL_4)) {
+      const Hll4Array<A>* hll4_ptr = static_cast<const Hll4Array<A>*>(sketch_impl);
+      const AuxHashMap<A>* aux_ptr = hll4_ptr->getAuxHashMap();
+      if (aux_ptr != nullptr) {
+        os << "### HLL sketch aux detail:" << std::endl;
+        os << std::left;
+        os << std::setw(10) << "Index";
+        os << std::setw(10) << "Key";
+        os << std::setw(10) << "Slot";
+        os << std::setw(6) << "Value";
+        os << std::endl;
+        auto it = aux_ptr->begin(all);
+        int i = 0;
+        int mask = (1 << get_lg_config_k()) - 1;
+        while (it != aux_ptr->end()) {
+          os << std::setw(10) << i;
+          os << std::setw(10) << HllUtil<A>::getLow26(*it);
+          os << std::setw(10) << (HllUtil<A>::getLow26(*it) & mask);
+          os << std::setw(6) << HllUtil<A>::getValue(*it);
+          os << std::endl;
+          ++it;
+          ++i;
+        }
+        os << "### End HLL sketch aux detail" << std::endl;
+      }
+    }
+  }
+
+  return os;
+}
+
+template<typename A>
+double hll_sketch_alloc<A>::get_estimate() const {
+  return sketch_impl->getEstimate();
+}
+
+template<typename A>
+double hll_sketch_alloc<A>::get_composite_estimate() const {
+  return sketch_impl->getCompositeEstimate();
+}
+
+template<typename A>
+double hll_sketch_alloc<A>::get_lower_bound(int numStdDev) const {
+  return sketch_impl->getLowerBound(numStdDev);
+}
+
+template<typename A>
+double hll_sketch_alloc<A>::get_upper_bound(int numStdDev) const {
+  return sketch_impl->getUpperBound(numStdDev);
+}
+
+template<typename A>
+hll_mode hll_sketch_alloc<A>::get_current_mode() const {
+  return sketch_impl->getCurMode();
+}
+
+template<typename A>
+int hll_sketch_alloc<A>::get_lg_config_k() const {
+  return sketch_impl->getLgConfigK();
+}
+
+template<typename A>
+target_hll_type hll_sketch_alloc<A>::get_target_type() const {
+  return sketch_impl->getTgtHllType();
+}
+
+template<typename A>
+bool hll_sketch_alloc<A>::is_out_of_order_flag() const {
+  return sketch_impl->isOutOfOrderFlag();
+}
+
+template<typename A>
+bool hll_sketch_alloc<A>::is_estimation_mode() const {
+  return true;
+}
+
+template<typename A>
+int hll_sketch_alloc<A>::get_updatable_serialization_bytes() const {
+  return sketch_impl->getUpdatableSerializationBytes();
+}
+
+template<typename A>
+int hll_sketch_alloc<A>::get_compact_serialization_bytes() const {
+  return sketch_impl->getCompactSerializationBytes();
+}
+
+template<typename A>
+bool hll_sketch_alloc<A>::is_compact() const {
+  return sketch_impl->isCompact();
+}
+
+template<typename A>
+bool hll_sketch_alloc<A>::is_empty() const {
+  return sketch_impl->isEmpty();
+}
+
+template<typename A>
+std::string hll_sketch_alloc<A>::type_as_string() const {
+  switch (sketch_impl->getTgtHllType()) {
+    case target_hll_type::HLL_4:
+      return std::string("HLL_4");
+    case target_hll_type::HLL_6:
+      return std::string("HLL_6");
+    case target_hll_type::HLL_8:
+      return std::string("HLL_8");
+    default:
+      throw std::runtime_error("Sketch state error: Invalid target_hll_type");
+  }
+}
+
+template<typename A>
+std::string hll_sketch_alloc<A>::mode_as_string() const {
+  switch (sketch_impl->getCurMode()) {
+    case LIST:
+      return std::string("LIST");
+    case SET:
+      return std::string("SET");
+    case HLL:
+      return std::string("HLL");
+    default:
+      throw std::runtime_error("Sketch state error: Invalid hll_mode");
+  }
+}
+
+template<typename A>
+int hll_sketch_alloc<A>::get_max_updatable_serialization_bytes(const int lg_config_k,
+    const target_hll_type tgtHllType) {
+  int arrBytes;
+  if (tgtHllType == target_hll_type::HLL_4) {
+    const int auxBytes = 4 << HllUtil<A>::LG_AUX_ARR_INTS[lg_config_k];
+    arrBytes = HllArray<A>::hll4ArrBytes(lg_config_k) + auxBytes;
+  } else if (tgtHllType == target_hll_type::HLL_6) {
+    arrBytes = HllArray<A>::hll6ArrBytes(lg_config_k);
+  } else { //HLL_8
+    arrBytes = HllArray<A>::hll8ArrBytes(lg_config_k);
+  }
+  return HllUtil<A>::HLL_BYTE_ARR_START + arrBytes;
+}
+
+template<typename A>
+double hll_sketch_alloc<A>::get_rel_err(const bool upperBound, const bool unioned,
+                           const int lg_config_k, const int numStdDev) {
+  return HllUtil<A>::getRelErr(upperBound, unioned, lg_config_k, numStdDev);
+}
+
+}
+
+#endif // _HLLSKETCH_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/HllSketchImpl-internal.hpp b/be/src/thirdparty/datasketches/HllSketchImpl-internal.hpp
new file mode 100644
index 0000000..a280fd0
--- /dev/null
+++ b/be/src/thirdparty/datasketches/HllSketchImpl-internal.hpp
@@ -0,0 +1,149 @@
+/*
+ * 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 _HLLSKETCHIMPL_INTERNAL_HPP_
+#define _HLLSKETCHIMPL_INTERNAL_HPP_
+
+#include "HllSketchImpl.hpp"
+#include "HllSketchImplFactory.hpp"
+
+namespace datasketches {
+
+template<typename A>
+HllSketchImpl<A>::HllSketchImpl(const int lgConfigK, const target_hll_type tgtHllType,
+                                const hll_mode mode, const bool startFullSize)
+  : lgConfigK(lgConfigK),
+    tgtHllType(tgtHllType),
+    mode(mode),
+    startFullSize(startFullSize)
+{
+}
+
+template<typename A>
+HllSketchImpl<A>::~HllSketchImpl() {
+}
+
+template<typename A>
+target_hll_type HllSketchImpl<A>::extractTgtHllType(const uint8_t modeByte) {
+  switch ((modeByte >> 2) & 0x3) {
+  case 0:
+    return target_hll_type::HLL_4;
+  case 1:
+    return target_hll_type::HLL_6;
+  case 2:
+    return target_hll_type::HLL_8;
+  default:
+    throw std::invalid_argument("Invalid target HLL type");
+  }
+}
+
+template<typename A>
+hll_mode HllSketchImpl<A>::extractCurMode(const uint8_t modeByte) {
+  switch (modeByte & 0x3) {
+  case 0:
+    return hll_mode::LIST;
+  case 1:
+    return hll_mode::SET;
+  case 2:
+    return hll_mode::HLL;
+  default:
+    throw std::invalid_argument("Invalid current sketch mode");
+  }
+}
+
+template<typename A>
+uint8_t HllSketchImpl<A>::makeFlagsByte(const bool compact) const {
+  uint8_t flags(0);
+  flags |= (isEmpty() ? HllUtil<A>::EMPTY_FLAG_MASK : 0);
+  flags |= (compact ? HllUtil<A>::COMPACT_FLAG_MASK : 0);
+  flags |= (isOutOfOrderFlag() ? HllUtil<A>::OUT_OF_ORDER_FLAG_MASK : 0);
+  flags |= (startFullSize ? HllUtil<A>::FULL_SIZE_FLAG_MASK : 0);
+  return flags;
+}
+
+// lo2bits = curMode, next 2 bits = tgtHllType
+// Dec  Lo4Bits TgtHllType, CurMode
+//   0     0000      HLL_4,    LIST
+//   1     0001      HLL_4,     SET
+//   2     0010      HLL_4,     HLL
+//   4     0100      HLL_6,    LIST
+//   5     0101      HLL_6,     SET
+//   6     0110      HLL_6,     HLL
+//   8     1000      HLL_8,    LIST
+//   9     1001      HLL_8,     SET
+//  10     1010      HLL_8,     HLL
+template<typename A>
+uint8_t HllSketchImpl<A>::makeModeByte() const {
+  uint8_t byte = 0;
+
+  switch (mode) {
+  case LIST:
+    byte = 0;
+    break;
+  case SET:
+    byte = 1;
+    break;
+  case HLL:
+    byte = 2;
+    break;
+  }
+
+  switch (tgtHllType) {
+  case HLL_4:
+    byte |= (0 << 2);  // for completeness
+    break;
+  case HLL_6:
+    byte |= (1 << 2);
+    break;
+  case HLL_8:
+    byte |= (2 << 2); 
+    break;
+  }
+
+  return byte;
+}
+
+template<typename A>
+HllSketchImpl<A>* HllSketchImpl<A>::reset() {
+  return HllSketchImplFactory<A>::reset(this, startFullSize);
+}
+
+template<typename A>
+target_hll_type HllSketchImpl<A>::getTgtHllType() const {
+  return tgtHllType;
+}
+
+template<typename A>
+int HllSketchImpl<A>::getLgConfigK() const {
+  return lgConfigK;
+}
+
+template<typename A>
+hll_mode HllSketchImpl<A>::getCurMode() const {
+  return mode;
+}
+
+template<typename A>
+bool HllSketchImpl<A>::isStartFullSize() const {
+  return startFullSize;
+}
+
+}
+
+#endif // _HLLSKETCHIMPL_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/HllSketchImpl.hpp b/be/src/thirdparty/datasketches/HllSketchImpl.hpp
new file mode 100644
index 0000000..82180b4
--- /dev/null
+++ b/be/src/thirdparty/datasketches/HllSketchImpl.hpp
@@ -0,0 +1,85 @@
+/*
+ * 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 _HLLSKETCHIMPL_HPP_
+#define _HLLSKETCHIMPL_HPP_
+
+#include "HllUtil.hpp"
+#include "hll.hpp" // for TgtHllType
+
+#include <memory>
+
+namespace datasketches {
+
+template<typename A = std::allocator<char>>
+class HllSketchImpl {
+  public:
+    HllSketchImpl(int lgConfigK, target_hll_type tgtHllType, hll_mode mode, bool startFullSize);
+    virtual ~HllSketchImpl();
+
+    virtual void serialize(std::ostream& os, bool compact) const = 0;
+    virtual vector_u8<A> serialize(bool compact, unsigned header_size_bytes) const = 0;
+
+    virtual HllSketchImpl* copy() const = 0;
+    virtual HllSketchImpl* copyAs(target_hll_type tgtHllType) const = 0;
+    HllSketchImpl<A>* reset();
+
+    virtual std::function<void(HllSketchImpl<A>*)> get_deleter() const = 0;
+
+    virtual HllSketchImpl* couponUpdate(int coupon) = 0;
+
+    hll_mode getCurMode() const;
+
+    virtual double getEstimate() const = 0;
+    virtual double getCompositeEstimate() const = 0;
+    virtual double getUpperBound(int numStdDev) const = 0;
+    virtual double getLowerBound(int numStdDev) const = 0;
+
+    inline int getLgConfigK() const;
+
+    virtual int getMemDataStart() const = 0;
+
+    virtual int getPreInts() const = 0;
+
+    target_hll_type getTgtHllType() const;
+
+    virtual int getUpdatableSerializationBytes() const = 0;
+    virtual int getCompactSerializationBytes() const = 0;
+
+    virtual bool isCompact() const = 0;
+    virtual bool isEmpty() const = 0;
+    virtual bool isOutOfOrderFlag() const = 0;
+    virtual void putOutOfOrderFlag(bool oooFlag) = 0;
+    bool isStartFullSize() const;
+
+  protected:
+    static target_hll_type extractTgtHllType(uint8_t modeByte);
+    static hll_mode extractCurMode(uint8_t modeByte);
+    uint8_t makeFlagsByte(bool compact) const;
+    uint8_t makeModeByte() const;
+
+    const int lgConfigK;
+    const target_hll_type tgtHllType;
+    const hll_mode mode;
+    const bool startFullSize;
+};
+
+}
+
+#endif // _HLLSKETCHIMPL_HPP_
diff --git a/be/src/thirdparty/datasketches/HllSketchImplFactory.hpp b/be/src/thirdparty/datasketches/HllSketchImplFactory.hpp
new file mode 100644
index 0000000..eae6f75
--- /dev/null
+++ b/be/src/thirdparty/datasketches/HllSketchImplFactory.hpp
@@ -0,0 +1,171 @@
+/*
+ * 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 _HLLSKETCHIMPLFACTORY_HPP_
+#define _HLLSKETCHIMPLFACTORY_HPP_
+
+#include "HllUtil.hpp"
+#include "HllSketchImpl.hpp"
+#include "CouponList.hpp"
+#include "CouponHashSet.hpp"
+#include "HllArray.hpp"
+#include "Hll4Array.hpp"
+#include "Hll6Array.hpp"
+#include "Hll8Array.hpp"
+
+namespace datasketches {
+
+template<typename A = std::allocator<char>>
+class HllSketchImplFactory final {
+public:
+  static HllSketchImpl<A>* deserialize(std::istream& os);
+  static HllSketchImpl<A>* deserialize(const void* bytes, size_t len);
+
+  static CouponHashSet<A>* promoteListToSet(const CouponList<A>& list);
+  static HllArray<A>* promoteListOrSetToHll(const CouponList<A>& list);
+  static HllArray<A>* newHll(int lgConfigK, target_hll_type tgtHllType, bool startFullSize = false);
+  
+  // resets the input impl, deleting the input pointer and returning a new pointer
+  static HllSketchImpl<A>* reset(HllSketchImpl<A>* impl, bool startFullSize);
+
+  static Hll4Array<A>* convertToHll4(const HllArray<A>& srcHllArr);
+  static Hll6Array<A>* convertToHll6(const HllArray<A>& srcHllArr);
+  static Hll8Array<A>* convertToHll8(const HllArray<A>& srcHllArr);
+};
+
+template<typename A>
+CouponHashSet<A>* HllSketchImplFactory<A>::promoteListToSet(const CouponList<A>& list) {
+  typedef typename std::allocator_traits<A>::template rebind_alloc<CouponHashSet<A>> chsAlloc;
+  CouponHashSet<A>* chSet = new (chsAlloc().allocate(1)) CouponHashSet<A>(list.getLgConfigK(), list.getTgtHllType());
+  for (auto coupon: list) {
+    chSet->couponUpdate(coupon);
+  }
+  chSet->putOutOfOrderFlag(true);
+  return chSet;
+}
+
+template<typename A>
+HllArray<A>* HllSketchImplFactory<A>::promoteListOrSetToHll(const CouponList<A>& src) {
+  HllArray<A>* tgtHllArr = HllSketchImplFactory<A>::newHll(src.getLgConfigK(), src.getTgtHllType());
+  tgtHllArr->putKxQ0(1 << src.getLgConfigK());
+  for (auto coupon: src) {
+    tgtHllArr->couponUpdate(coupon);
+  }
+  tgtHllArr->putHipAccum(src.getEstimate());
+  tgtHllArr->putOutOfOrderFlag(false);
+  return tgtHllArr;
+}
+
+template<typename A>
+HllSketchImpl<A>* HllSketchImplFactory<A>::deserialize(std::istream& is) {
+  // we'll hand off the sketch based on PreInts so we don't need
+  // to move the stream pointer back and forth -- perhaps somewhat fragile?
+  const int preInts = is.peek();
+  if (preInts == HllUtil<A>::HLL_PREINTS) {
+    return HllArray<A>::newHll(is);
+  } else if (preInts == HllUtil<A>::HASH_SET_PREINTS) {
+    return CouponHashSet<A>::newSet(is);
+  } else if (preInts == HllUtil<A>::LIST_PREINTS) {
+    return CouponList<A>::newList(is);
+  } else {
+    throw std::invalid_argument("Attempt to deserialize unknown object type");
+  }
+}
+
+template<typename A>
+HllSketchImpl<A>* HllSketchImplFactory<A>::deserialize(const void* bytes, size_t len) {
+  // read current mode directly
+  const int preInts = static_cast<const uint8_t*>(bytes)[0];
+  if (preInts == HllUtil<A>::HLL_PREINTS) {
+    return HllArray<A>::newHll(bytes, len);
+  } else if (preInts == HllUtil<A>::HASH_SET_PREINTS) {
+    return CouponHashSet<A>::newSet(bytes, len);
+  } else if (preInts == HllUtil<A>::LIST_PREINTS) {
+    return CouponList<A>::newList(bytes, len);
+  } else {
+    throw std::invalid_argument("Attempt to deserialize unknown object type");
+  }
+}
+
+template<typename A>
+HllArray<A>* HllSketchImplFactory<A>::newHll(int lgConfigK, target_hll_type tgtHllType, bool startFullSize) {
+  switch (tgtHllType) {
+    case HLL_8:
+      typedef typename std::allocator_traits<A>::template rebind_alloc<Hll8Array<A>> hll8Alloc;
+      return new (hll8Alloc().allocate(1)) Hll8Array<A>(lgConfigK, startFullSize);
+    case HLL_6:
+      typedef typename std::allocator_traits<A>::template rebind_alloc<Hll6Array<A>> hll6Alloc;
+      return new (hll6Alloc().allocate(1)) Hll6Array<A>(lgConfigK, startFullSize);
+    case HLL_4:
+      typedef typename std::allocator_traits<A>::template rebind_alloc<Hll4Array<A>> hll4Alloc;
+      return new (hll4Alloc().allocate(1)) Hll4Array<A>(lgConfigK, startFullSize);
+  }
+  throw std::logic_error("Invalid target_hll_type");
+}
+
+template<typename A>
+HllSketchImpl<A>* HllSketchImplFactory<A>::reset(HllSketchImpl<A>* impl, bool startFullSize) {
+  if (startFullSize) {
+    HllArray<A>* hll = newHll(impl->getLgConfigK(), impl->getTgtHllType(), startFullSize);
+    impl->get_deleter()(impl);
+    return hll;
+  } else {
+    typedef typename std::allocator_traits<A>::template rebind_alloc<CouponList<A>> clAlloc;
+    CouponList<A>* cl = new (clAlloc().allocate(1)) CouponList<A>(impl->getLgConfigK(), impl->getTgtHllType(), hll_mode::LIST);
+    impl->get_deleter()(impl);
+    return cl;
+  }
+}
+
+template<typename A>
+Hll4Array<A>* HllSketchImplFactory<A>::convertToHll4(const HllArray<A>& srcHllArr) {
+  const int lgConfigK = srcHllArr.getLgConfigK();
+  typedef typename std::allocator_traits<A>::template rebind_alloc<Hll4Array<A>> hll4Alloc;
+  Hll4Array<A>* hll4Array = new (hll4Alloc().allocate(1)) Hll4Array<A>(lgConfigK, srcHllArr.isStartFullSize());
+  hll4Array->putOutOfOrderFlag(srcHllArr.isOutOfOrderFlag());
+  hll4Array->mergeHll(srcHllArr);
+  hll4Array->putHipAccum(srcHllArr.getHipAccum());
+  return hll4Array;
+}
+
+template<typename A>
+Hll6Array<A>* HllSketchImplFactory<A>::convertToHll6(const HllArray<A>& srcHllArr) {
+  const int lgConfigK = srcHllArr.getLgConfigK();
+  typedef typename std::allocator_traits<A>::template rebind_alloc<Hll6Array<A>> hll6Alloc;
+  Hll6Array<A>* hll6Array = new (hll6Alloc().allocate(1)) Hll6Array<A>(lgConfigK, srcHllArr.isStartFullSize());
+  hll6Array->putOutOfOrderFlag(srcHllArr.isOutOfOrderFlag());
+  hll6Array->mergeHll(srcHllArr);
+  hll6Array->putHipAccum(srcHllArr.getHipAccum());
+  return hll6Array;
+}
+
+template<typename A>
+Hll8Array<A>* HllSketchImplFactory<A>::convertToHll8(const HllArray<A>& srcHllArr) {
+  const int lgConfigK = srcHllArr.getLgConfigK();
+  typedef typename std::allocator_traits<A>::template rebind_alloc<Hll8Array<A>> hll8Alloc;
+  Hll8Array<A>* hll8Array = new (hll8Alloc().allocate(1)) Hll8Array<A>(lgConfigK, srcHllArr.isStartFullSize());
+  hll8Array->putOutOfOrderFlag(srcHllArr.isOutOfOrderFlag());
+  hll8Array->mergeHll(srcHllArr);
+  hll8Array->putHipAccum(srcHllArr.getHipAccum());
+  return hll8Array;
+}
+
+}
+
+#endif /* _HLLSKETCHIMPLFACTORY_HPP_ */
diff --git a/be/src/thirdparty/datasketches/HllUnion-internal.hpp b/be/src/thirdparty/datasketches/HllUnion-internal.hpp
new file mode 100644
index 0000000..0d039f2
--- /dev/null
+++ b/be/src/thirdparty/datasketches/HllUnion-internal.hpp
@@ -0,0 +1,351 @@
+/*
+ * 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 _HLLUNION_INTERNAL_HPP_
+#define _HLLUNION_INTERNAL_HPP_
+
+#include "hll.hpp"
+
+#include "HllSketchImpl.hpp"
+#include "HllArray.hpp"
+#include "HllUtil.hpp"
+
+#include <stdexcept>
+#include <string>
+
+namespace datasketches {
+
+template<typename A>
+hll_union_alloc<A>::hll_union_alloc(const int lg_max_k):
+  lg_max_k(HllUtil<A>::checkLgK(lg_max_k)),
+  gadget(lg_max_k, target_hll_type::HLL_8)
+{}
+
+template<typename A>
+hll_union_alloc<A> hll_union_alloc<A>::deserialize(const void* bytes, size_t len) {
+  hll_sketch_alloc<A> sk(hll_sketch_alloc<A>::deserialize(bytes, len));
+  // we're using the sketch's lg_config_k to initialize the union so
+  // we can initialize the Union with it as long as it's HLL_8.
+  hll_union_alloc<A> hllUnion(sk.get_lg_config_k());
+  if (sk.get_target_type() == HLL_8) {
+    std::swap(hllUnion.gadget.sketch_impl, sk.sketch_impl);
+  } else {
+    hllUnion.update(sk);
+  }
+  return hllUnion;
+}
+
+template<typename A>
+hll_union_alloc<A> hll_union_alloc<A>::deserialize(std::istream& is) {
+  hll_sketch_alloc<A> sk(hll_sketch_alloc<A>::deserialize(is));
+  // we're using the sketch's lg_config_k to initialize the union so
+  // we can initialize the Union with it as long as it's HLL_8.
+  hll_union_alloc<A> hllUnion(sk.get_lg_config_k());
+  if (sk.get_target_type() == HLL_8) {    
+    std::swap(hllUnion.gadget.sketch_impl, sk.sketch_impl);
+  } else {
+    hllUnion.update(sk);
+  }
+  return hllUnion;
+}
+
+template<typename A>
+static std::ostream& operator<<(std::ostream& os, const hll_union_alloc<A>& hllUnion) {
+  return hllUnion.to_string(os, true, true, false, false);
+}
+
+template<typename A>
+hll_sketch_alloc<A> hll_union_alloc<A>::get_result(target_hll_type target_type) const {
+  return hll_sketch_alloc<A>(gadget, target_type);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const hll_sketch_alloc<A>& sketch) {
+  if (sketch.is_empty()) return;
+  union_impl(sketch, lg_max_k);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(hll_sketch_alloc<A>&& sketch) {
+  if (sketch.is_empty()) return;
+  if (gadget.is_empty() && sketch.get_target_type() == HLL_8 && sketch.get_lg_config_k() <= lg_max_k) {
+    if (sketch.get_current_mode() == HLL || sketch.get_lg_config_k() == lg_max_k) {
+      gadget = std::move(sketch);
+    }
+  }
+  union_impl(sketch, lg_max_k);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const std::string& datum) {
+  gadget.update(datum);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const uint64_t datum) {
+  gadget.update(datum);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const uint32_t datum) {
+  gadget.update(datum);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const uint16_t datum) {
+  gadget.update(datum);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const uint8_t datum) {
+  gadget.update(datum);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const int64_t datum) {
+  gadget.update(datum);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const int32_t datum) {
+  gadget.update(datum);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const int16_t datum) {
+  gadget.update(datum);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const int8_t datum) {
+  gadget.update(datum);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const double datum) {
+  gadget.update(datum);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const float datum) {
+  gadget.update(datum);
+}
+
+template<typename A>
+void hll_union_alloc<A>::update(const void* data, const size_t length_bytes) {
+  gadget.update(data, length_bytes);
+}
+
+template<typename A>
+void hll_union_alloc<A>::coupon_update(const int coupon) {
+  if (coupon == HllUtil<A>::EMPTY) { return; }
+  HllSketchImpl<A>* result = gadget.sketch_impl->coupon_update(coupon);
+  if (result != gadget.sketch_impl) {
+    if (gadget.sketch_impl != nullptr) { gadget.sketch_impl->get_deleter()(gadget.sketch_impl); }
+    gadget.sketch_impl = result;
+  }
+}
+
+template<typename A>
+vector_u8<A> hll_union_alloc<A>::serialize_compact() const {
+  return gadget.serialize_compact();
+}
+
+template<typename A>
+vector_u8<A> hll_union_alloc<A>::serialize_updatable() const {
+  return gadget.serialize_updatable();
+}
+
+template<typename A>
+void hll_union_alloc<A>::serialize_compact(std::ostream& os) const {
+  return gadget.serialize_compact(os);
+}
+
+template<typename A>
+void hll_union_alloc<A>::serialize_updatable(std::ostream& os) const {
+  return gadget.serialize_updatable(os);
+}
+
+template<typename A>
+std::ostream& hll_union_alloc<A>::to_string(std::ostream& os, const bool summary,
+                                  const bool detail, const bool aux_detail, const bool all) const {
+  return gadget.to_string(os, summary, detail, aux_detail, all);
+}
+
+template<typename A>
+std::string hll_union_alloc<A>::to_string(const bool summary, const bool detail,
+                                   const bool aux_detail, const bool all) const {
+  return gadget.to_string(summary, detail, aux_detail, all);
+}
+
+template<typename A>
+double hll_union_alloc<A>::get_estimate() const {
+  return gadget.get_estimate();
+}
+
+template<typename A>
+double hll_union_alloc<A>::get_composite_estimate() const {
+  return gadget.get_composite_estimate();
+}
+
+template<typename A>
+double hll_union_alloc<A>::get_lower_bound(const int num_std_dev) const {
+  return gadget.get_lower_bound(num_std_dev);
+}
+
+template<typename A>
+double hll_union_alloc<A>::get_upper_bound(const int num_std_dev) const {
+  return gadget.get_upper_bound(num_std_dev);
+}
+
+template<typename A>
+int hll_union_alloc<A>::get_compact_serialization_bytes() const {
+  return gadget.get_compact_serialization_bytes();
+}
+
+template<typename A>
+int hll_union_alloc<A>::get_updatable_serialization_bytes() const {
+  return gadget.get_updatable_serialization_bytes();
+}
+
+template<typename A>
+int hll_union_alloc<A>::get_lg_config_k() const {
+  return gadget.get_lg_config_k();
+}
+
+template<typename A>
+void hll_union_alloc<A>::reset() {
+  gadget.reset();
+}
+
+template<typename A>
+bool hll_union_alloc<A>::is_compact() const {
+  return gadget.is_compact();
+}
+
+template<typename A>
+bool hll_union_alloc<A>::is_empty() const {
+  return gadget.is_empty();
+}
+
+template<typename A>
+bool hll_union_alloc<A>::is_out_of_order_flag() const {
+  return gadget.is_out_of_order_flag();
+}
+
+template<typename A>
+hll_mode hll_union_alloc<A>::get_current_mode() const {
+  return gadget.get_current_mode();
+}
+
+template<typename A>
+bool hll_union_alloc<A>::is_estimation_mode() const {
+  return gadget.is_estimation_mode();
+}
+
+template<typename A>
+int hll_union_alloc<A>::get_serialization_version() const {
+  return HllUtil<A>::SER_VER;
+}
+
+template<typename A>
+target_hll_type hll_union_alloc<A>::get_target_type() const {
+  return target_hll_type::HLL_8;
+}
+
+template<typename A>
+int hll_union_alloc<A>::get_max_serialization_bytes(const int lg_k) {
+  return hll_sketch_alloc<A>::get_max_updatable_serialization_bytes(lg_k, target_hll_type::HLL_8);
+}
+
+template<typename A>
+double hll_union_alloc<A>::get_rel_err(const bool upper_bound, const bool unioned,
+                           const int lg_config_k, const int num_std_dev) {
+  return HllUtil<A>::getRelErr(upper_bound, unioned, lg_config_k, num_std_dev);
+}
+
+template<typename A>
+HllSketchImpl<A>* hll_union_alloc<A>::copy_or_downsample(const HllSketchImpl<A>* src_impl, const int tgt_lg_k) {
+  if (src_impl->getCurMode() != HLL) {
+    throw std::logic_error("Attempt to downsample non-HLL sketch");
+  }
+  const HllArray<A>* src = static_cast<const HllArray<A>*>(src_impl);
+  const int src_lg_k = src->getLgConfigK();
+  if (src_lg_k <= tgt_lg_k) {
+    return src->copyAs(HLL_8);
+  }
+  typedef typename std::allocator_traits<A>::template rebind_alloc<Hll8Array<A>> hll8Alloc;
+  Hll8Array<A>* tgtHllArr = new (hll8Alloc().allocate(1)) Hll8Array<A>(tgt_lg_k, false);
+  tgtHllArr->mergeHll(*src);
+  //both of these are required for isomorphism
+  tgtHllArr->putHipAccum(src->getHipAccum());
+  tgtHllArr->putOutOfOrderFlag(src->isOutOfOrderFlag());
+  return tgtHllArr;
+}
+
+template<typename A>
+inline HllSketchImpl<A>* hll_union_alloc<A>::leak_free_coupon_update(HllSketchImpl<A>* impl, const int coupon) {
+  HllSketchImpl<A>* result = impl->couponUpdate(coupon);
+  if (result != impl) {
+    impl->get_deleter()(impl);
+  }
+  return result;
+}
+
+template<typename A>
+void hll_union_alloc<A>::union_impl(const hll_sketch_alloc<A>& sketch, const int lg_max_k) {
+  const HllSketchImpl<A>* src_impl = sketch.sketch_impl; //default
+  HllSketchImpl<A>* dst_impl = gadget.sketch_impl; //default
+  if (src_impl->getCurMode() == LIST || src_impl->getCurMode() == SET) {
+    if (dst_impl->isEmpty() && src_impl->getLgConfigK() == dst_impl->getLgConfigK()) {
+      dst_impl = src_impl->copyAs(HLL_8);
+      gadget.sketch_impl->get_deleter()(gadget.sketch_impl); // gadget replaced
+    } else {
+      const CouponList<A>* src = static_cast<const CouponList<A>*>(src_impl);
+      for (auto coupon: *src) {
+        dst_impl = leak_free_coupon_update(dst_impl, coupon); //assignment required
+      }
+    }
+  } else if (!dst_impl->isEmpty()) { // src is HLL
+    if (dst_impl->getCurMode() == LIST || dst_impl->getCurMode() == SET) {
+      // swap so that src is LIST or SET, tgt is HLL
+      // use lg_max_k because LIST has effective K of 2^26
+      const CouponList<A>* src = static_cast<const CouponList<A>*>(dst_impl);
+      dst_impl = copy_or_downsample(src_impl, lg_max_k);
+      static_cast<Hll8Array<A>*>(dst_impl)->mergeList(*src);
+      gadget.sketch_impl->get_deleter()(gadget.sketch_impl); // gadget replaced
+    } else { // gadget is HLL
+      if (src_impl->getLgConfigK() < dst_impl->getLgConfigK()) {
+        dst_impl = copy_or_downsample(dst_impl, sketch.get_lg_config_k());
+        gadget.sketch_impl->get_deleter()(gadget.sketch_impl); // gadget replaced
+      }
+      const HllArray<A>* src = static_cast<const HllArray<A>*>(src_impl);
+      static_cast<Hll8Array<A>*>(dst_impl)->mergeHll(*src);
+      dst_impl->putOutOfOrderFlag(true);
+    }
+  } else { // src is HLL, gadget is empty
+    dst_impl = copy_or_downsample(src_impl, lg_max_k);
+    gadget.sketch_impl->get_deleter()(gadget.sketch_impl); // gadget replaced
+  }
+  gadget.sketch_impl = dst_impl;
+}
+
+}
+
+#endif // _HLLUNION_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/HllUtil.hpp b/be/src/thirdparty/datasketches/HllUtil.hpp
new file mode 100644
index 0000000..eb956bc
--- /dev/null
+++ b/be/src/thirdparty/datasketches/HllUtil.hpp
@@ -0,0 +1,289 @@
+/*
+ * 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 _HLLUTIL_HPP_
+#define _HLLUTIL_HPP_
+
+#include "MurmurHash3.h"
+#include "RelativeErrorTables.hpp"
+#include "CommonUtil.hpp"
+
+#include <cmath>
+#include <stdexcept>
+#include <string>
+
+namespace datasketches {
+
+enum hll_mode { LIST = 0, SET, HLL };
+
+// template provides internal consistency and allows static float values
+// but we don't use the template parameter anywhere
+template<typename A = std::allocator<char> >
+class HllUtil final {
+public:
+  // preamble stuff
+  static const int SER_VER = 1;
+  static const int FAMILY_ID = 7;
+
+  static const int EMPTY_FLAG_MASK          = 4;
+  static const int COMPACT_FLAG_MASK        = 8;
+  static const int OUT_OF_ORDER_FLAG_MASK   = 16;
+  static const int FULL_SIZE_FLAG_MASK      = 32;
+
+  static const int PREAMBLE_INTS_BYTE = 0;
+  static const int SER_VER_BYTE       = 1;
+  static const int FAMILY_BYTE        = 2;
+  static const int LG_K_BYTE          = 3;
+  static const int LG_ARR_BYTE        = 4;
+  static const int FLAGS_BYTE         = 5;
+  static const int LIST_COUNT_BYTE    = 6;
+  static const int HLL_CUR_MIN_BYTE   = 6;
+  static const int MODE_BYTE          = 7; // lo2bits = curMode, next 2 bits = tgtHllMode
+
+  // Coupon List
+  static const int LIST_INT_ARR_START = 8;
+  static const int LIST_PREINTS = 2;
+  // Coupon Hash Set
+  static const int HASH_SET_COUNT_INT             = 8;
+  static const int HASH_SET_INT_ARR_START         = 12;
+  static const int HASH_SET_PREINTS         = 3;
+  // HLL
+  static const int HLL_PREINTS = 10;
+  static const int HLL_BYTE_ARR_START = 40;
+  static const int HIP_ACCUM_DOUBLE = 8;
+  static const int KXQ0_DOUBLE = 16;
+  static const int KXQ1_DOUBLE = 24;
+  static const int CUR_MIN_COUNT_INT = 32;
+  static const int AUX_COUNT_INT = 36;
+  
+  static const int EMPTY_SKETCH_SIZE_BYTES = 8;
+
+  // other HllUtil stuff
+  static const int KEY_BITS_26 = 26;
+  static const int VAL_BITS_6 = 6;
+  static const int KEY_MASK_26 = (1 << KEY_BITS_26) - 1;
+  static const int VAL_MASK_6 = (1 << VAL_BITS_6) - 1;
+  static const int EMPTY = 0;
+  static const int MIN_LOG_K = 4;
+  static const int MAX_LOG_K = 21;
+
+  static const uint64_t DEFAULT_UPDATE_SEED = 9001L;
+
+  static const double HLL_HIP_RSE_FACTOR; // sqrt(log(2.0)) = 0.8325546
+  static const double HLL_NON_HIP_RSE_FACTOR; // sqrt((3.0 * log(2.0)) - 1.0) = 1.03896
+  static const double COUPON_RSE_FACTOR; // 0.409 at transition point not the asymptote
+  static const double COUPON_RSE; // COUPON_RSE_FACTOR / (1 << 13);
+
+  static const int LG_INIT_LIST_SIZE = 3;
+  static const int LG_INIT_SET_SIZE = 5;
+  static const int RESIZE_NUMER = 3;
+  static const int RESIZE_DENOM = 4;
+
+  static const int loNibbleMask = 0x0f;
+  static const int hiNibbleMask = 0xf0;
+  static const int AUX_TOKEN = 0xf;
+
+  /**
+  * Log2 table sizes for exceptions based on lgK from 0 to 26.
+  * However, only lgK from 4 to 21 are used.
+  */
+  static const int LG_AUX_ARR_INTS[];
+
+  static int coupon(const uint64_t hash[]);
+  static int coupon(const HashState& hashState);
+  static void hash(const void* key, int keyLen, uint64_t seed, HashState& result);
+
+  static int checkLgK(int lgK);
+  static void checkMemSize(uint64_t minBytes, uint64_t capBytes);
+  static inline void checkNumStdDev(int numStdDev);
+  static int pair(int slotNo, int value);
+  static int getLow26(unsigned int coupon);
+  static int getValue(unsigned int coupon);
+  static double invPow2(int e);
+  static unsigned int ceilingPowerOf2(unsigned int n);
+  static unsigned int simpleIntLog2(unsigned int n); // n must be power of 2
+  static unsigned int getNumberOfLeadingZeros(uint64_t x);
+  static unsigned int numberOfTrailingZeros(uint32_t n);
+  static int computeLgArrInts(hll_mode mode, int count, int lgConfigK);
+  static double getRelErr(bool upperBound, bool unioned,
+                          int lgConfigK, int numStdDev);
+};
+
+template<typename A>
+const double HllUtil<A>::HLL_HIP_RSE_FACTOR = sqrt(log(2.0)); // 0.8325546
+template<typename A>
+const double HllUtil<A>::HLL_NON_HIP_RSE_FACTOR = sqrt((3.0 * log(2.0)) - 1.0); // 1.03896
+template<typename A>
+const double HllUtil<A>::COUPON_RSE_FACTOR = 0.409;
+template<typename A>
+const double HllUtil<A>::COUPON_RSE = COUPON_RSE_FACTOR / (1 << 13);
+
+template<typename A>
+const int HllUtil<A>::LG_AUX_ARR_INTS[] = {
+      0, 2, 2, 2, 2, 2, 2, 3, 3, 3,   // 0 - 9
+      4, 4, 5, 5, 6, 7, 8, 9, 10, 11, // 10-19
+      12, 13, 14, 15, 16, 17, 18      // 20-26
+      };
+
+template<typename A>
+inline int HllUtil<A>::coupon(const uint64_t hash[]) {
+  int addr26 = (int) (hash[0] & KEY_MASK_26);
+  int lz = CommonUtil::getNumberOfLeadingZeros(hash[1]);
+  int value = ((lz > 62 ? 62 : lz) + 1); 
+  return (value << KEY_BITS_26) | addr26;
+}
+
+template<typename A>
+inline int HllUtil<A>::coupon(const HashState& hashState) {
+  int addr26 = (int) (hashState.h1 & KEY_MASK_26);
+  int lz = CommonUtil::getNumberOfLeadingZeros(hashState.h2);  
+  int value = ((lz > 62 ? 62 : lz) + 1); 
+  return (value << KEY_BITS_26) | addr26;
+}
+
+template<typename A>
+inline void HllUtil<A>::hash(const void* key, const int keyLen, const uint64_t seed, HashState& result) {
+  MurmurHash3_x64_128(key, keyLen, DEFAULT_UPDATE_SEED, result);
+}
+
+template<typename A>
+inline double HllUtil<A>::getRelErr(const bool upperBound, const bool unioned,
+                                    const int lgConfigK, const int numStdDev) {
+  return RelativeErrorTables<A>::getRelErr(upperBound, unioned, lgConfigK, numStdDev);
+}
+
+template<typename A>
+inline int HllUtil<A>::checkLgK(const int lgK) {
+  if ((lgK >= HllUtil<A>::MIN_LOG_K) && (lgK <= HllUtil<A>::MAX_LOG_K)) {
+    return lgK;
+  } else {
+    throw std::invalid_argument("Invalid value of k: " + std::to_string(lgK));
+  }
+}
+
+template<typename A>
+inline void HllUtil<A>::checkMemSize(const uint64_t minBytes, const uint64_t capBytes) {
+  if (capBytes < minBytes) {
+    throw std::invalid_argument("Given destination array is not large enough: " + std::to_string(capBytes));
+  }
+}
+
+template<typename A>
+inline void HllUtil<A>::checkNumStdDev(const int numStdDev) {
+  if ((numStdDev < 1) || (numStdDev > 3)) {
+    throw std::invalid_argument("NumStdDev may not be less than 1 or greater than 3.");
+  }
+}
+
+template<typename A>
+inline int HllUtil<A>::pair(const int slotNo, const int value) {
+  return (value << HllUtil<A>::KEY_BITS_26) | (slotNo & HllUtil<A>::KEY_MASK_26);
+}
+
+template<typename A>
+inline int HllUtil<A>::getLow26(const unsigned int coupon) {
+  return coupon & HllUtil<A>::KEY_MASK_26;
+}
+
+template<typename A>
+inline int HllUtil<A>::getValue(const unsigned int coupon) {
+  return coupon >> HllUtil<A>::KEY_BITS_26;
+}
+
+template<typename A>
+inline double HllUtil<A>::invPow2(const int e) {
+  union {
+    long long longVal;
+    double doubleVal;
+  } conv;
+  conv.longVal = (1023L - e) << 52;
+  return conv.doubleVal;
+}
+
+// compute the next highest power of 2 of 32-bit n
+// taken from https://graphics.stanford.edu/~seander/bithacks.html
+template<typename A>
+inline unsigned int HllUtil<A>::ceilingPowerOf2(unsigned int n) {
+  --n;
+  n |= n >> 1;
+  n |= n >> 2;
+  n |= n >> 4;
+  n |= n >> 8;
+  n |= n >> 16;
+  return ++n;
+}
+
+template<typename A>
+inline unsigned int HllUtil<A>::simpleIntLog2(unsigned int n) {
+  if (n == 0) {
+    throw std::logic_error("cannot take log of 0");
+  }
+  const unsigned int e = numberOfTrailingZeros(n);
+  return e;
+}
+
+// taken from https://graphics.stanford.edu/~seander/bithacks.html
+// input is 32-bit word to count zero bits on right
+template<typename A>
+inline unsigned int HllUtil<A>::numberOfTrailingZeros(uint32_t v) {
+  unsigned int c;     // c will be the number of zero bits on the right,
+                      // so if v is 1101000 (base 2), then c will be 3
+  // NOTE: if 0 == v, then c = 31.
+  if (v & 0x1) {
+    // special case for odd v (assumed to happen half of the time)
+    c = 0;
+  } else {
+    c = 1;
+    if ((v & 0xffff) == 0) {  
+      v >>= 16;  
+      c += 16;
+    }
+    if ((v & 0xff) == 0) {  
+      v >>= 8;  
+      c += 8;
+    }
+    if ((v & 0xf) == 0) {  
+      v >>= 4;
+      c += 4;
+    }
+    if ((v & 0x3) == 0) {  
+      v >>= 2;
+      c += 2;
+    }
+    c -= v & 0x1;
+  }
+  return c;	
+}
+
+template<typename A>
+inline int HllUtil<A>::computeLgArrInts(hll_mode mode, int count, int lgConfigK) {
+  // assume value missing and recompute
+  if (mode == LIST) { return HllUtil<A>::LG_INIT_LIST_SIZE; }
+  int ceilPwr2 = HllUtil<A>::ceilingPowerOf2(count);
+  if ((HllUtil<A>::RESIZE_DENOM * count) > (HllUtil<A>::RESIZE_NUMER * ceilPwr2)) { ceilPwr2 <<= 1;}
+  if (mode == SET) {
+    return fmax(HllUtil<A>::LG_INIT_SET_SIZE, HllUtil<A>::simpleIntLog2(ceilPwr2));
+  }
+  //only used for HLL4
+  return fmax(HllUtil<A>::LG_AUX_ARR_INTS[lgConfigK], HllUtil<A>::simpleIntLog2(ceilPwr2));
+}
+
+}
+
+#endif /* _HLLUTIL_HPP_ */
\ No newline at end of file
diff --git a/be/src/thirdparty/datasketches/LICENSE b/be/src/thirdparty/datasketches/LICENSE
new file mode 100644
index 0000000..937b0f0
--- /dev/null
+++ b/be/src/thirdparty/datasketches/LICENSE
@@ -0,0 +1,230 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
+
+  Additional licenses relevant to this work.
+
+    Apache DataSketches-cpp (incubating) includes a number of source 
+    files with separate copyright notices and license terms. Your use of 
+    the source code for these subcomponents is subject to the terms and
+    conditions of the following licenses. See the NOTICE file for details
+    of the licence dependencies that refer to the following licences.
+
+MIT License
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
diff --git a/be/src/thirdparty/datasketches/MurmurHash3.h b/be/src/thirdparty/datasketches/MurmurHash3.h
new file mode 100644
index 0000000..45a64c6
--- /dev/null
+++ b/be/src/thirdparty/datasketches/MurmurHash3.h
@@ -0,0 +1,178 @@
+// Minimally modified from Austin Applebee's code:
+//  * Removed MurmurHash3_x86_32 and MurmurHash3_x86_128
+//  * Changed input seed in MurmurHash3_x64_128 to uint64_t
+//  * Define and use HashState reference to return result
+//  * Made entire hash function defined inline
+//-----------------------------------------------------------------------------
+// MurmurHash3 was written by Austin Appleby, and is placed in the public
+// domain. The author hereby disclaims copyright to this source code.
+
+// Note - The x86 and x64 versions do _not_ produce the same results, as the
+// algorithms are optimized for their respective platforms. You can still
+// compile and run any of them on any platform, but your performance with the
+// non-native version will be less than optimal.
+
+#ifndef _MURMURHASH3_H_
+#define _MURMURHASH3_H_
+
+//-----------------------------------------------------------------------------
+// Platform-specific functions and macros
+
+// Microsoft Visual Studio
+
+#if defined(_MSC_VER)
+
+typedef unsigned char uint8_t;
+typedef unsigned int uint32_t;
+typedef unsigned __int64 uint64_t;
+
+#define FORCE_INLINE	__forceinline
+
+#include <stdlib.h>
+
+#define ROTL32(x,y)	_rotl(x,y)
+#define ROTL64(x,y)	_rotl64(x,y)
+
+#define BIG_CONSTANT(x) (x)
+
+// Other compilers
+
+#else   // defined(_MSC_VER)
+
+#include <stdint.h>
+
+#define	FORCE_INLINE inline __attribute__((always_inline))
+
+inline uint32_t rotl32 ( uint32_t x, int8_t r )
+{
+  return (x << r) | (x >> (32 - r));
+}
+
+inline uint64_t rotl64 ( uint64_t x, int8_t r )
+{
+  return (x << r) | (x >> (64 - r));
+}
+
+#define	ROTL32(x,y)	rotl32(x,y)
+#define ROTL64(x,y)	rotl64(x,y)
+
+#define BIG_CONSTANT(x) (x##LLU)
+
+#endif // !defined(_MSC_VER)
+
+//-----------------------------------------------------------------------------
+
+//-----------------------------------------------------------------------------
+// Return type - Using C++ reference for return type which should allow better
+// compiler optimization than a void* pointer
+typedef struct {
+  uint64_t h1;
+  uint64_t h2;
+} HashState;
+
+
+//-----------------------------------------------------------------------------
+// Block read - if your platform needs to do endian-swapping or can only
+// handle aligned reads, do the conversion here
+
+FORCE_INLINE uint64_t getblock64 ( const uint64_t * p, int i )
+{
+  return p[i];
+}
+
+//-----------------------------------------------------------------------------
+// Finalization mix - force all bits of a hash block to avalanche
+
+FORCE_INLINE uint64_t fmix64 ( uint64_t k )
+{
+  k ^= k >> 33;
+  k *= BIG_CONSTANT(0xff51afd7ed558ccd);
+  k ^= k >> 33;
+  k *= BIG_CONSTANT(0xc4ceb9fe1a85ec53);
+  k ^= k >> 33;
+
+  return k;
+}
+
+FORCE_INLINE void MurmurHash3_x64_128(const void* key, int lenBytes, uint64_t seed, HashState& out) {
+  static const uint64_t c1 = BIG_CONSTANT(0x87c37b91114253d5);
+  static const uint64_t c2 = BIG_CONSTANT(0x4cf5ad432745937f);
+
+  const uint8_t* data = (const uint8_t*)key;
+
+  out.h1 = seed;
+  out.h2 = seed;
+
+  // Number of full 128-bit blocks of 16 bytes.
+  // Possible exclusion fo a remainder of up to 15 bytes.
+  const int nblocks = lenBytes >> 4; // bytes / 16 
+
+  // Process the 128-bit blocks (the body) into teh hash
+  const uint64_t* blocks = (const uint64_t*)(data);
+  for (int i = 0; i < nblocks; ++i) { // 16 bytes per block
+    //uint64_t k1 = getblock64(blocks, 0);
+    //uint64_t k2 = getblock64(blocks, 1);
+    uint64_t k1 = getblock64(blocks,i*2+0);
+    uint64_t k2 = getblock64(blocks,i*2+1);
+
+    k1 *= c1; k1  = ROTL64(k1,31); k1 *= c2; out.h1 ^= k1;
+    out.h1 = ROTL64(out.h1,27);
+    out.h1 += out.h2;
+    out.h1 = out.h1*5+0x52dce729;
+
+    k2 *= c2; k2  = ROTL64(k2,33); k2 *= c1; out.h2 ^= k2;
+    out.h2 = ROTL64(out.h2,31);
+    out.h2 += out.h1;
+    out.h2 = out.h2*5+0x38495ab5;
+
+    blocks += 2;
+  }
+
+  // tail
+  //const uint8_t * tail = (const uint8_t*)blocks;
+  const uint8_t * tail = (const uint8_t*)(data + (nblocks << 4));
+
+  uint64_t k1 = 0;
+  uint64_t k2 = 0;
+
+  switch(lenBytes & 15)
+  {
+  case 15: k2 ^= ((uint64_t)tail[14]) << 48; //@suppress("No break at end of case")
+  case 14: k2 ^= ((uint64_t)tail[13]) << 40; //@suppress("No break at end of case")
+  case 13: k2 ^= ((uint64_t)tail[12]) << 32; //@suppress("No break at end of case")
+  case 12: k2 ^= ((uint64_t)tail[11]) << 24; //@suppress("No break at end of case")
+  case 11: k2 ^= ((uint64_t)tail[10]) << 16; //@suppress("No break at end of case")
+  case 10: k2 ^= ((uint64_t)tail[ 9]) << 8;  //@suppress("No break at end of case")
+  case  9: k2 ^= ((uint64_t)tail[ 8]) << 0;
+           k2 *= c2; k2  = ROTL64(k2,33); k2 *= c1; out.h2 ^= k2;
+           //@suppress("No break at end of case")
+  case  8: k1 ^= ((uint64_t)tail[ 7]) << 56; //@suppress("No break at end of case")
+  case  7: k1 ^= ((uint64_t)tail[ 6]) << 48; //@suppress("No break at end of case")
+  case  6: k1 ^= ((uint64_t)tail[ 5]) << 40; //@suppress("No break at end of case")
+  case  5: k1 ^= ((uint64_t)tail[ 4]) << 32; //@suppress("No break at end of case")
+  case  4: k1 ^= ((uint64_t)tail[ 3]) << 24; //@suppress("No break at end of case")
+  case  3: k1 ^= ((uint64_t)tail[ 2]) << 16; //@suppress("No break at end of case")
+  case  2: k1 ^= ((uint64_t)tail[ 1]) << 8; //@suppress("No break at end of case")
+  case  1: k1 ^= ((uint64_t)tail[ 0]) << 0;
+           k1 *= c1; k1  = ROTL64(k1,31); k1 *= c2; out.h1 ^= k1;
+  };
+
+  //----------
+  // finalization
+
+  out.h1 ^= lenBytes;
+  out.h2 ^= lenBytes;
+
+  out.h1 += out.h2;
+  out.h2 += out.h1;
+
+  out.h1 = fmix64(out.h1);
+  out.h2 = fmix64(out.h2);
+
+  out.h1 += out.h2;
+  out.h2 += out.h1;
+}
+
+//-----------------------------------------------------------------------------
+
+#endif // _MURMURHASH3_H_
\ No newline at end of file
diff --git a/be/src/thirdparty/datasketches/README.md b/be/src/thirdparty/datasketches/README.md
new file mode 100644
index 0000000..51e1d57
--- /dev/null
+++ b/be/src/thirdparty/datasketches/README.md
@@ -0,0 +1,15 @@
+The content of this folder imports the functionality needed for HLL approximate
+algorithm from Apache DataSketches by copying the necessary files from that
+project into this folder. Note, that the original structure of files was
+changed during this process as originally hll/ and common/ libraries were
+both affected but I copied these into the same directory so that Impala can
+compile them without rewriting the include paths in the files themselves. Also
+note, that not the whole common/ directory was copied just the files needed for
+HLL.
+
+The git hash of the snapshot I used as a source for the files:
+a6265b307a03085abe26c20413fdbf7d7a5eaf29
+
+Browse the source files here:
+https://github.com/apache/incubator-datasketches-cpp
+
diff --git a/be/src/thirdparty/datasketches/RelativeErrorTables-internal.hpp b/be/src/thirdparty/datasketches/RelativeErrorTables-internal.hpp
new file mode 100644
index 0000000..d5d235e
--- /dev/null
+++ b/be/src/thirdparty/datasketches/RelativeErrorTables-internal.hpp
@@ -0,0 +1,112 @@
+/*
+ * 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 _RELATIVEERRORTABLES_INTERNAL_HPP_
+#define _RELATIVEERRORTABLES_INTERNAL_HPP_
+
+#include "RelativeErrorTables.hpp"
+
+namespace datasketches {
+
+//case 0
+static double HIP_LB[] = //sd 1, 2, 3
+  { //Q(.84134), Q(.97725), Q(.99865) respectively
+    0.207316195, 0.502865572, 0.882303765, //4
+    0.146981579, 0.335426881, 0.557052,    //5
+    0.104026721, 0.227683872, 0.365888317, //6
+    0.073614601, 0.156781585, 0.245740374, //7
+    0.05205248,  0.108783763, 0.168030442, //8
+    0.036770852, 0.075727545, 0.11593785,  //9
+    0.025990219, 0.053145536, 0.080772263, //10
+    0.018373987, 0.037266176, 0.056271814, //11
+    0.012936253, 0.02613829,  0.039387631, //12
+  };
+
+//case 1
+static double HIP_UB[] = //sd 1, 2, 3
+  { //Q(.15866), Q(.02275), Q(.00135) respectively
+    -0.207805347, -0.355574279, -0.475535095, //4
+    -0.146988328, -0.262390832, -0.360864026, //5
+    -0.103877775, -0.191503663, -0.269311582, //6
+    -0.073452978, -0.138513438, -0.198487447, //7
+    -0.051982806, -0.099703123, -0.144128618, //8
+    -0.036768609, -0.07138158,  -0.104430324, //9
+    -0.025991325, -0.050854296, -0.0748143,   //10
+    -0.01834533,  -0.036121138, -0.05327616,  //11
+    -0.012920332, -0.025572893, -0.037896952, //12
+  };
+
+//case 2
+static double NON_HIP_LB[] = //sd 1, 2, 3`
+  { //Q(.84134), Q(.97725), Q(.99865) respectively
+    0.254409839, 0.682266712, 1.304022158, //4
+    0.181817353, 0.443389054, 0.778776219, //5
+    0.129432281, 0.295782195, 0.49252279,  //6
+    0.091640655, 0.201175925, 0.323664385, //7
+    0.064858051, 0.138523393, 0.218805328, //8
+    0.045851855, 0.095925072, 0.148635751, //9
+    0.032454144, 0.067009668, 0.102660669, //10
+    0.022921382, 0.046868565, 0.071307398, //11
+    0.016155679, 0.032825719, 0.049677541  //12
+  };
+
+//case 3
+static double NON_HIP_UB[] = //sd 1, 2, 3
+  { //Q(.15866), Q(.02275), Q(.00135) respectively
+    -0.256980172, -0.411905944, -0.52651057,  //4
+    -0.182332109, -0.310275547, -0.412660505, //5
+    -0.129314228, -0.230142294, -0.315636197, //6
+    -0.091584836, -0.16834013,  -0.236346847, //7
+    -0.06487411,  -0.122045231, -0.174112107, //8
+    -0.04591465,  -0.08784505,  -0.126917615, //9
+    -0.032433119, -0.062897613, -0.091862929, //10
+    -0.022960633, -0.044875401, -0.065736049, //11
+    -0.016186662, -0.031827816, -0.046973459  //12
+  };
+
+template<typename A>
+double RelativeErrorTables<A>::getRelErr(const bool upperBound, const bool oooFlag,
+                                         const int lgK, const int stdDev) {
+  const int idx = ((lgK - 4) * 3) + (stdDev - 1);
+  const int sw = (oooFlag ? 2 : 0) | (upperBound ? 1 : 0);
+  double f = 0;
+  switch (sw) {
+    case 0 : { // HIP, LB
+      f = HIP_LB[idx];
+      break;
+    }
+    case 1 : { // HIP, UB
+      f = HIP_UB[idx];
+      break;
+    }
+    case 2 : { // NON_HIP, LB
+      f = NON_HIP_LB[idx];
+      break;
+    }
+    case 3 : { // NON_HIP, UB
+      f = NON_HIP_UB[idx];
+      break;
+    }
+  }
+  return f;
+}
+
+}
+
+#endif // _RELATIVEERRORTABLES_INTERNAL_HPP_
\ No newline at end of file
diff --git a/be/src/thirdparty/datasketches/RelativeErrorTables.hpp b/be/src/thirdparty/datasketches/RelativeErrorTables.hpp
new file mode 100644
index 0000000..da8bebf
--- /dev/null
+++ b/be/src/thirdparty/datasketches/RelativeErrorTables.hpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _RELATIVEERRORTABLES_HPP_
+#define _RELATIVEERRORTABLES_HPP_
+
+#include <memory>
+
+namespace datasketches {
+
+template<typename A = std::allocator<char>>
+class RelativeErrorTables {
+  public:
+    /**
+     * Return Relative Error for UB or LB for HIP or Non-HIP as a function of numStdDev.
+     * @param upperBound true if for upper bound
+     * @param oooFlag true if for Non-HIP
+     * @param lgK must be between 4 and 12 inclusive
+     * @param stdDev must be between 1 and 3 inclusive
+     * @return Relative Error for UB or LB for HIP or Non-HIP as a function of numStdDev.
+     */
+    static double getRelErr(bool upperBound, bool oooFlag,
+                            int lgK, int stdDev);
+};
+
+}
+
+#include "RelativeErrorTables-internal.hpp"
+
+#endif /* _RELATIVEERRORTABLES_HPP_ */
diff --git a/be/src/thirdparty/datasketches/coupon_iterator-internal.hpp b/be/src/thirdparty/datasketches/coupon_iterator-internal.hpp
new file mode 100644
index 0000000..35d0e0b
--- /dev/null
+++ b/be/src/thirdparty/datasketches/coupon_iterator-internal.hpp
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#ifndef _INTARRAYPAIRITERATOR_INTERNAL_HPP_
+#define _INTARRAYPAIRITERATOR_INTERNAL_HPP_
+
+#include "HllUtil.hpp"
+
+namespace datasketches {
+
+template<typename A>
+coupon_iterator<A>::coupon_iterator(const int* array, size_t array_size, size_t index, bool all):
+array(array), array_size(array_size), index(index), all(all) {
+  while (this->index < array_size) {
+    if (all || array[this->index] != HllUtil<A>::EMPTY) break;
+    this->index++;
+  }
+}
+
+template<typename A>
+coupon_iterator<A>& coupon_iterator<A>::operator++() {
+  while (++index < array_size) {
+    if (all || array[index] != HllUtil<A>::EMPTY) break;
+  }
+  return *this;
+}
+
+template<typename A>
+bool coupon_iterator<A>::operator!=(const coupon_iterator& other) const {
+  return index != other.index;
+}
+
+template<typename A>
+uint32_t coupon_iterator<A>::operator*() const {
+  return array[index];
+}
+
+}
+
+#endif // _INTARRAYPAIRITERATOR_INTERNAL_HPP_
diff --git a/be/src/thirdparty/datasketches/coupon_iterator.hpp b/be/src/thirdparty/datasketches/coupon_iterator.hpp
new file mode 100644
index 0000000..9f373cf
--- /dev/null
+++ b/be/src/thirdparty/datasketches/coupon_iterator.hpp
@@ -0,0 +1,43 @@
+/*
+ * 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 _INTARRAYPAIRITERATOR_HPP_
+#define _INTARRAYPAIRITERATOR_HPP_
+
+namespace datasketches {
+
+template<typename A>
+class coupon_iterator: public std::iterator<std::input_iterator_tag, uint32_t> {
+public:
+  coupon_iterator(const int* array, size_t array_slze, size_t index, bool all);
+  coupon_iterator& operator++();
+  bool operator!=(const coupon_iterator& other) const;
+  uint32_t operator*() const;
+private:
+  const int* array;
+  size_t array_size;
+  size_t index;
+  bool all;
+};
+
+}
+
+#include "coupon_iterator-internal.hpp"
+
+#endif /* _INTARRAYPAIRITERATOR_HPP_ */
diff --git a/be/src/thirdparty/datasketches/hll.hpp b/be/src/thirdparty/datasketches/hll.hpp
new file mode 100644
index 0000000..31db794
--- /dev/null
+++ b/be/src/thirdparty/datasketches/hll.hpp
@@ -0,0 +1,761 @@
+/*
+ * 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 _HLL_HPP_
+#define _HLL_HPP_
+
+#include "HllUtil.hpp"
+
+#include <memory>
+#include <iostream>
+#include <vector>
+
+namespace datasketches {
+
+  /**
+ * This is a high performance implementation of Phillipe Flajolet&#8217;s HLL sketch but with
+ * significantly improved error behavior.  If the ONLY use case for sketching is counting
+ * uniques and merging, the HLL sketch is a reasonable choice, although the highest performing in terms of accuracy for
+ * storage space consumed is CPC (Compressed Probabilistic Counting). For large enough counts, this HLL version (with HLL_4) can be 2 to
+ * 16 times smaller than the Theta sketch family for the same accuracy.
+ *
+ * <p>This implementation offers three different types of HLL sketch, each with different
+ * trade-offs with accuracy, space and performance. These types are specified with the
+ * {@link TgtHllType} parameter.
+ *
+ * <p>In terms of accuracy, all three types, for the same <i>lg_config_k</i>, have the same error
+ * distribution as a function of <i>n</i>, the number of unique values fed to the sketch.
+ * The configuration parameter <i>lg_config_k</i> is the log-base-2 of <i>K</i>,
+ * where <i>K</i> is the number of buckets or slots for the sketch.
+ *
+ * <p>During warmup, when the sketch has only received a small number of unique items
+ * (up to about 10% of <i>K</i>), this implementation leverages a new class of estimator
+ * algorithms with significantly better accuracy.
+ *
+ * <p>This sketch also offers the capability of operating off-heap. Given a WritableMemory object
+ * created by the user, the sketch will perform all of its updates and internal phase transitions
+ * in that object, which can actually reside either on-heap or off-heap based on how it is
+ * configured. In large systems that must update and merge many millions of sketches, having the
+ * sketch operate off-heap avoids the serialization and deserialization costs of moving sketches
+ * to and from off-heap memory-mapped files, for example, and eliminates big garbage collection
+ * delays.
+ *
+ * author Jon Malkin
+ * author Lee Rhodes
+ * author Kevin Lang
+ */
+
+  
+/**
+ * Specifies the target type of HLL sketch to be created. It is a target in that the actual
+ * allocation of the HLL array is deferred until sufficient number of items have been received by
+ * the warm-up phases.
+ *
+ * <p>These three target types are isomorphic representations of the same underlying HLL algorithm.
+ * Thus, given the same value of <i>lg_config_k</i> and the same input, all three HLL target types
+ * will produce identical estimates and have identical error distributions.</p>
+ *
+ * <p>The memory (and also the serialization) of the sketch during this early warmup phase starts
+ * out very small (8 bytes, when empty) and then grows in increments of 4 bytes as required
+ * until the full HLL array is allocated.  This transition point occurs at about 10% of K for
+ * sketches where lg_config_k is &gt; 8.</p>
+ *
+ * <ul>
+ * <li><b>HLL_8</b> This uses an 8-bit byte per HLL bucket. It is generally the
+ * fastest in terms of update time, but has the largest storage footprint of about
+ * <i>K</i> bytes.</li>
+ *
+ * <li><b>HLL_6</b> This uses a 6-bit field per HLL bucket. It is the generally the next fastest
+ * in terms of update time with a storage footprint of about <i>3/4 * K</i> bytes.</li>
+ *
+ * <li><b>HLL_4</b> This uses a 4-bit field per HLL bucket and for large counts may require
+ * the use of a small internal auxiliary array for storing statistical exceptions, which are rare.
+ * For the values of <i>lg_config_k &gt; 13</i> (<i>K</i> = 8192),
+ * this additional array adds about 3% to the overall storage. It is generally the slowest in
+ * terms of update time, but has the smallest storage footprint of about
+ * <i>K/2 * 1.03</i> bytes.</li>
+ * </ul>
+ */
+enum target_hll_type {
+    HLL_4, ///< 4 bits per entry (most compact, size may vary)
+    HLL_6, ///< 6 bits per entry (fixed size)
+    HLL_8  ///< 8 bits per entry (fastest, fixed size)
+};
+
+template<typename A>
+class HllSketchImpl;
+
+template<typename A>
+class hll_union_alloc;
+
+template<typename A> using AllocU8 = typename std::allocator_traits<A>::template rebind_alloc<uint8_t>;
+template<typename A> using vector_u8 = std::vector<uint8_t, AllocU8<A>>;
+
+template<typename A = std::allocator<char> >
+class hll_sketch_alloc final {
+  public:
+    /**
+     * Constructs a new HLL sketch.
+     * @param lg_config_k Sketch can hold 2^lg_config_k rows
+     * @param tgt_type The HLL mode to use, if/when the sketch reaches that state
+     * @param start_full_size Indicates whether to start in HLL mode,
+     *        keeping memory use constant (if HLL_6 or HLL_8) at the cost of
+     *        starting out using much more memory
+     */
+    explicit hll_sketch_alloc(int lg_config_k, target_hll_type tgt_type = HLL_4, bool start_full_size = false);
+
+    /**
+     * Copy constructor
+     */
+    hll_sketch_alloc(const hll_sketch_alloc<A>& that);
+
+    /**
+     * Copy constructor to a new target type
+     */
+    hll_sketch_alloc(const hll_sketch_alloc<A>& that, target_hll_type tgt_type);
+
+    /**
+     * Move constructor
+     */
+    hll_sketch_alloc(hll_sketch_alloc<A>&& that) noexcept;
+
+    /**
+     * Reconstructs a sketch from a serialized image on a stream.
+     * @param is An input stream with a binary image of a sketch
+     */
+    static hll_sketch_alloc deserialize(std::istream& is);
+
+    /**
+     * Reconstructs a sketch from a serialized image in a byte array.
+     * @param is bytes An input array with a binary image of a sketch
+     * @param len Length of the input array, in bytes
+     */
+    static hll_sketch_alloc deserialize(const void* bytes, size_t len);
+
+    //! Class destructor
+    virtual ~hll_sketch_alloc();
+
+    //! Copy assignment operator
+    hll_sketch_alloc operator=(const hll_sketch_alloc<A>& other);
+
+    //! Move assignment operator
+    hll_sketch_alloc operator=(hll_sketch_alloc<A>&& other);
+
+    /**
+     * Resets the sketch to an empty state in coupon collection mode.
+     * Does not re-use existing internal objects.
+     */
+    void reset();
+
+    typedef vector_u8<A> vector_bytes; // alias for users
+
+    /**
+     * Serializes the sketch to a byte array, compacting data structures
+     * where feasible to eliminate unused storage in the serialized image.
+     * @param header_size_bytes Allows for PostgreSQL integration
+     */
+    vector_bytes serialize_compact(unsigned header_size_bytes = 0) const;
+
+    /**
+     * Serializes the sketch to a byte array, retaining all internal 
+     * data structures in their current form.
+     */
+    vector_bytes serialize_updatable() const;
+
+    /**
+     * Serializes the sketch to an ostream, compacting data structures
+     * where feasible to eliminate unused storage in the serialized image.
+     * @param os std::ostream to use for output.
+     */
+    void serialize_compact(std::ostream& os) const;
+
+    /**
+     * Serializes the sketch to an ostream, retaining all internal data
+     * structures in their current form.
+     * @param os std::ostream to use for output.
+     */
+    void serialize_updatable(std::ostream& os) const;
+
+    /**
+     * Human readable summary with optional detail
+     * @param os std::ostram to which the summary is written
+     * @param summary if true, output the sketch summary
+     * @param detail if true, output the internal data array
+     * @param auxDetail if true, output the internal Aux array, if it exists.
+     * @param all if true, outputs all entries including empty ones
+     * @return human readable string with optional detail.
+     */
+    std::ostream& to_string(std::ostream& os,
+                            bool summary = true,
+                            bool detail = false,
+                            bool aux_detail = false,
+                            bool all = false) const;
+
+    /**
+     * Human readable summary with optional detail
+     * @param summary if true, output the sketch summary
+     * @param detail if true, output the internal data array
+     * @param auxDetail if true, output the internal Aux array, if it exists.
+     * @param all if true, outputs all entries including empty ones
+     * @return human readable string with optional detail.
+     */
+    std::string to_string(bool summary = true,
+                          bool detail = false,
+                          bool aux_detail = false,
+                          bool all = false) const;
+
+    /**
+     * Present the given std::string as a potential unique item.
+     * The string is converted to a byte array using UTF8 encoding.
+     * If the string is null or empty no update attempt is made and the method returns.
+     * @param datum The given string.
+     */
+    void update(const std::string& datum);
+
+    /**
+     * Present the given unsigned 64-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(uint64_t datum);
+
+    /**
+     * Present the given unsigned 32-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(uint32_t datum);
+
+    /**
+     * Present the given unsigned 16-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(uint16_t datum);
+
+    /**
+     * Present the given unsigned 8-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(uint8_t datum);
+
+    /**
+     * Present the given signed 64-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(int64_t datum);
+
+    /**
+     * Present the given signed 32-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(int32_t datum);
+
+    /**
+     * Present the given signed 16-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(int16_t datum);
+
+    /**
+     * Present the given signed 8-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(int8_t datum);
+
+    /**
+     * Present the given 64-bit floating point value as a potential unique item.
+     * @param datum The given double.
+     */
+    void update(double datum);
+
+    /**
+     * Present the given 32-bit floating point value as a potential unique item.
+     * @param datum The given float.
+     */
+    void update(float datum);
+
+    /**
+     * Present the given data array as a potential unique item.
+     * @param data The given array.
+     * @param length_bytes The array length in bytes.
+     */
+    void update(const void* data, size_t length_bytes);
+
+    /**
+     * Returns the current cardinality estimate
+     * @return the cardinality estimate
+     */
+    double get_estimate() const;
+
+    /**
+     * This is less accurate than the getEstimate() method
+     * and is automatically used when the sketch has gone through
+     * union operations where the more accurate HIP estimator cannot
+     * be used.
+     *
+     * This is made public only for error characterization software
+     * that exists in separate packages and is not intended for normal
+     * use.
+     * @return the composite cardinality estimate
+     */
+    double get_composite_estimate() const;
+
+    /**
+     * Returns the approximate lower error bound given the specified
+     * number of standard deviations.
+     * @param num_std_dev Number of standard deviations, an integer from the set  {1, 2, 3}.
+     * @return The approximate lower bound.
+     */
+    double get_lower_bound(int num_std_dev) const;
+
+    /**
+     * Returns the approximate upper error bound given the specified
+     * number of standard deviations.
+     * @param num_std_dev Number of standard deviations, an integer from the set  {1, 2, 3}.
+     * @return The approximate upper bound.
+     */
+    double get_upper_bound(int num_std_dev) const;
+
+    /**
+     * Returns sketch's configured lg_k value.
+     * @return Configured lg_k value.
+     */
+    int get_lg_config_k() const;
+
+    /**
+     * Returns the sketch's target HLL mode (from #target_hll_type).
+     * @return The sketch's target HLL mode.
+     */
+    target_hll_type get_target_type() const;
+
+    /**
+     * Indicates if the sketch is currently stored compacted.
+     * @return True if the sketch is stored in compact form.
+     */
+    bool is_compact() const;
+
+    /**
+     * Indicates if the sketch is currently empty.
+     * @return True if the sketch is empty.
+     */
+    bool is_empty() const;
+
+    /**
+     * Returns the size of the sketch serialized in compact form.
+     * @return Size of the sketch serialized in compact form, in bytes.
+     */
+    int get_compact_serialization_bytes() const;
+
+    /**
+     * Returns the size of the sketch serialized without compaction.
+     * @return Size of the sketch serialized without compaction, in bytes.
+     */
+    int get_updatable_serialization_bytes() const;
+
+    /**
+     * Returns the maximum size in bytes that this sketch can grow to
+     * given lg_config_k.  However, for the HLL_4 sketch type, this
+     * value can be exceeded in extremely rare cases.  If exceeded, it
+     * will be larger by only a few percent.
+     *
+     * @param lg_config_k The Log2 of K for the target HLL sketch. This value must be
+     *        between 4 and 21 inclusively.
+     * @param tgt_type the desired Hll type
+     * @return the maximum size in bytes that this sketch can grow to.
+     */
+    static int get_max_updatable_serialization_bytes(int lg_k, target_hll_type tgt_type);
+  
+    /**
+     * Gets the current (approximate) Relative Error (RE) asymptotic values given several
+     * parameters. This is used primarily for testing.
+     * @param upper_bound return the RE for the Upper Bound, otherwise for the Lower Bound.
+     * @param unioned set true if the sketch is the result of a union operation.
+     * @param lg_config_k the configured value for the sketch.
+     * @param num_std_dev the given number of Standard Deviations. This must be an integer between
+     * 1 and 3, inclusive.
+     * @return the current (approximate) RelativeError
+     */
+    static double get_rel_err(bool upper_bound, bool unioned,
+                              int lg_config_k, int num_std_dev);
+
+  private:
+    explicit hll_sketch_alloc(HllSketchImpl<A>* that);
+
+    void coupon_update(int coupon);
+
+    std::string type_as_string() const;
+    std::string mode_as_string() const;
+
+    hll_mode get_current_mode() const;
+    int get_serialization_version() const;
+    bool is_out_of_order_flag() const;
+    bool is_estimation_mode() const;
+
+    typedef typename std::allocator_traits<A>::template rebind_alloc<hll_sketch_alloc> AllocHllSketch;
+
+    HllSketchImpl<A>* sketch_impl;
+    friend hll_union_alloc<A>;
+};
+
+/**
+ * This performs union operations for HLL sketches. This union operator is configured with a
+ * <i>lgMaxK</i> instead of the normal <i>lg_config_k</i>.
+ *
+ * <p>This union operator does permit the unioning of sketches with different values of
+ * <i>lg_config_k</i>.  The user should be aware that the resulting accuracy of a sketch returned
+ * at the end of the unioning process will be a function of the smallest of <i>lg_max_k</i> and
+ * <i>lg_config_k</i> that the union operator has seen.
+ *
+ * <p>This union operator also permits unioning of any of the three different target hll_sketch
+ * types.
+ *
+ * <p>Although the API for this union operator parallels many of the methods of the
+ * <i>HllSketch</i>, the behavior of the union operator has some fundamental differences.
+ *
+ * <p>First, the user cannot specify the #tgt_hll_type as an input parameter.
+ * Instead, it is specified for the sketch returned with #get_result(tgt_hll_tyope).
+ *
+ * <p>Second, the internal effective value of log-base-2 of <i>k</i> for the union operation can
+ * change dynamically based on the smallest <i>lg_config_k</i> that the union operation has seen.
+ *
+ * author Jon Malkin
+ * author Lee Rhodes
+ * author Kevin Lang
+ */
+ 
+template<typename A = std::allocator<char> >
+class hll_union_alloc {
+  public:
+    /**
+     * Construct an hll_union operator with the given maximum log2 of k.
+     * @param lg_max_k The maximum size, in log2, of k. The value must
+     * be between 7 and 21, inclusive.
+     */
+    explicit hll_union_alloc(int lg_max_k);
+
+    /**
+     * Construct an hll_union operator from the given std::istream, which
+     * must be a valid serialized image of an hll_union.
+     * @param is The input stream from which to read.
+     */
+    static hll_union_alloc deserialize(std::istream& is);
+
+  /**
+     * Construct an hll_union operator from the given byte array, which
+     * must be a valid serialized image of an hll_union.
+     * @param bytes The byte array to read.
+     * @param len Byte array length in bytes.
+     */
+    static hll_union_alloc deserialize(const void* bytes, size_t len);
+
+    /**
+     * Returns the current cardinality estimate
+     * @return the cardinality estimate
+     */
+    double get_estimate() const;
+
+    /**
+     * This is less accurate than the get_estimate() method
+     * and is automatically used when the union has gone through
+     * union operations where the more accurate HIP estimator cannot
+     * be used.
+     *
+     * This is made public only for error characterization software
+     * that exists in separate packages and is not intended for normal
+     * use.
+     * @return the composite cardinality estimate
+     */
+    double get_composite_estimate() const;
+
+    /**
+     * Returns the approximate lower error bound given the specified
+     * number of standard deviations.
+     * @param num_std_dev Number of standard deviations, an integer from the set  {1, 2, 3}.
+     * @return The approximate lower bound.
+     */
+    double get_lower_bound(int num_std_dev) const;
+
+    /**
+     * Returns the approximate upper error bound given the specified
+     * number of standard deviations.
+     * @param num_std_dev Number of standard deviations, an integer from the set  {1, 2, 3}.
+     * @return The approximate upper bound.
+     */
+    double get_upper_bound(int num_std_dev) const;
+
+    /**
+     * Returns the size of the union serialized in compact form.
+     * @return Size of the union serialized in compact form, in bytes.
+     */
+    int get_compact_serialization_bytes() const;
+
+    /**
+     * Returns the size of the union serialized without compaction.
+     * @return Size of the union serialized without compaction, in bytes.
+     */
+    int get_updatable_serialization_bytes() const;
+
+    /**
+     * Returns union's configured lg_k value.
+     * @return Configured lg_k value.
+     */
+    int get_lg_config_k() const;
+
+    /**
+     * Returns the union's target HLL mode (from #target_hll_type).
+     * @return The union's target HLL mode.
+     */
+    target_hll_type get_target_type() const;
+
+    /**
+     * Indicates if the union is currently stored compacted.
+     * @return True if the union is stored in compact form.
+     */
+    bool is_compact() const;
+
+    /**
+     * Indicates if the union is currently empty.
+     * @return True if the union is empty.
+     */
+    bool is_empty() const;
+
+    /**
+     * Resets the union to an empty state in coupon collection mode.
+     * Does not re-use existing internal objects.
+     */
+    void reset();
+
+    /**
+     * Returns the result of this union operator with the specified
+     * #tgt_hll_type.
+     * @param The tgt_hll_type enum value of the desired result (Default: HLL_4)
+     * @return The result of this union with the specified tgt_hll_type
+     */
+    hll_sketch_alloc<A> get_result(target_hll_type tgt_type = HLL_4) const;
+
+    typedef vector_u8<A> vector_bytes; // alias for users
+
+    /**
+     * Serializes the sketch to a byte array, compacting data structures
+     * where feasible to eliminate unused storage in the serialized image.
+     * @param header_size_bytes Allows for PostgreSQL integration
+     */
+    vector_bytes serialize_compact() const;
+  
+    /**
+     * Serializes the sketch to a byte array, retaining all internal 
+     * data structures in their current form.
+     */
+    vector_bytes serialize_updatable() const;
+
+    /**
+     * Serializes the sketch to an ostream, compacting data structures
+     * where feasible to eliminate unused storage in the serialized image.
+     * @param os std::ostream to use for output.
+     */
+    void serialize_compact(std::ostream& os) const;
+
+    /**
+     * Serializes the sketch to an ostream, retaining all internal data
+     * structures in their current form.
+     * @param os std::ostream to use for output.
+     */
+    void serialize_updatable(std::ostream& os) const;
+
+    /**
+     * Human readable summary with optional detail
+     * @param os std::ostram to which the summary is written
+     * @param summary if true, output the sketch summary
+     * @param detail if true, output the internal data array
+     * @param auxDetail if true, output the internal Aux array, if it exists.
+     * @param all if true, outputs all entries including empty ones
+     * @return human readable string with optional detail.
+     */
+    std::ostream& to_string(std::ostream& os,
+                            bool summary = true,
+                            bool detail = false,
+                            bool aux_Detail = false,
+                            bool all = false) const;
+  
+    /**
+     * Human readable summary with optional detail
+     * @param summary if true, output the sketch summary
+     * @param detail if true, output the internal data array
+     * @param auxDetail if true, output the internal Aux array, if it exists.
+     * @param all if true, outputs all entries including empty ones
+     * @return human readable string with optional detail.
+     */
+    std::string to_string(bool summary = true,
+                          bool detail = false,
+                          bool aux_detail = false,
+                          bool all = false) const;
+
+    /**
+     * Update this union operator with the given sketch.
+     * @param The given sketch.
+     */
+    void update(const hll_sketch_alloc<A>& sketch);
+
+    /**
+     * Update this union operator with the given temporary sketch.
+     * @param The given sketch.
+     */
+    void update(hll_sketch_alloc<A>&& sketch);
+  
+    /**
+     * Present the given std::string as a potential unique item.
+     * The string is converted to a byte array using UTF8 encoding.
+     * If the string is null or empty no update attempt is made and the method returns.
+     * @param datum The given string.
+     */
+    void update(const std::string& datum);
+
+    /**
+     * Present the given unsigned 64-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(uint64_t datum);
+
+    /**
+     * Present the given unsigned 32-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(uint32_t datum);
+
+    /**
+     * Present the given unsigned 16-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(uint16_t datum);
+
+    /**
+     * Present the given unsigned 8-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(uint8_t datum);
+
+    /**
+     * Present the given signed 64-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(int64_t datum);
+
+    /**
+     * Present the given signed 32-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(int32_t datum);
+
+    /**
+     * Present the given signed 16-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(int16_t datum);
+
+    /**
+     * Present the given signed 8-bit integer as a potential unique item.
+     * @param datum The given integer.
+     */
+    void update(int8_t datum);
+
+    /**
+     * Present the given 64-bit floating point value as a potential unique item.
+     * @param datum The given double.
+     */
+    void update(double datum);
+
+    /**
+     * Present the given 32-bit floating point value as a potential unique item.
+     * @param datum The given float.
+     */
+    void update(float datum);
+
+    /**
+     * Present the given data array as a potential unique item.
+     * @param data The given array.
+     * @param length_bytes The array length in bytes.
+     */
+    void update(const void* data, size_t length_bytes);
+
+    /**
+     * Returns the maximum size in bytes that this union operator can grow to given a lg_k.
+     *
+     * @param lg_k The maximum Log2 of k for this union operator. This value must be
+     * between 4 and 21 inclusively.
+     * @return the maximum size in bytes that this union operator can grow to.
+     */
+    static int get_max_serialization_bytes(int lg_k);
+
+    /**
+     * Gets the current (approximate) Relative Error (RE) asymptotic values given several
+     * parameters. This is used primarily for testing.
+     * @param upper_bound return the RE for the Upper Bound, otherwise for the Lower Bound.
+     * @param unioned set true if the sketch is the result of a union operation.
+     * @param lg_config_k the configured value for the sketch.
+     * @param num_std_dev the given number of Standard Deviations. This must be an integer between
+     * 1 and 3, inclusive.
+     * @return the current (approximate) RelativeError
+     */
+    static double get_rel_err(bool upper_bound, bool unioned,
+                              int lg_config_k, int num_std_dev);
+
+  private:
+
+   /**
+    * Union the given source and destination sketches. This method examines the state of
+    * the current internal gadget and the incoming sketch and determines the optimal way to
+    * perform the union. This may involve swapping, down-sampling, transforming, and / or
+    * copying one of the arguments and may completely replace the internals of the union.
+    *
+    * @param incoming_impl the given incoming sketch, which may not be modified.
+    * @param lg_max_k the maximum value of log2 K for this union.
+    */
+    inline void union_impl(const hll_sketch_alloc<A>& sketch, int lg_max_k);
+
+    static HllSketchImpl<A>* copy_or_downsample(const HllSketchImpl<A>* src_impl, int tgt_lg_k);
+
+    void coupon_update(int coupon);
+
+    hll_mode get_current_mode() const;
+    int get_serialization_version() const;
+    bool is_out_of_order_flag() const;
+    bool is_estimation_mode() const;
+
+    // calls couponUpdate on sketch, freeing the old sketch upon changes in hll_mode
+    static HllSketchImpl<A>* leak_free_coupon_update(HllSketchImpl<A>* impl, int coupon);
+
+    int lg_max_k;
+    hll_sketch_alloc<A> gadget;
+};
+
+template<typename A>
+static std::ostream& operator<<(std::ostream& os, const hll_sketch_alloc<A>& sketch);
+
+template<typename A>
+static std::ostream& operator<<(std::ostream& os, const hll_union_alloc<A>& union_in);
+
+/// convenience alias for hll_sketch with default allocator
+typedef hll_sketch_alloc<> hll_sketch;
+
+/// convenience alias for hll_union with default allocator
+typedef hll_union_alloc<> hll_union;
+
+} // namespace datasketches
+
+#include "hll.private.hpp"
+
+#endif // _HLL_HPP_
diff --git a/be/src/thirdparty/datasketches/hll.private.hpp b/be/src/thirdparty/datasketches/hll.private.hpp
new file mode 100644
index 0000000..980558e
--- /dev/null
+++ b/be/src/thirdparty/datasketches/hll.private.hpp
@@ -0,0 +1,32 @@
+#ifndef _HLL_PRIVATE_HPP_
+#define _HLL_PRIVATE_HPP_
+
+#include "AuxHashMap.hpp"
+#include "CompositeInterpolationXTable.hpp"
+#include "CouponHashSet.hpp"
+#include "CouponList.hpp"
+#include "CubicInterpolation.hpp"
+#include "HarmonicNumbers.hpp"
+#include "Hll4Array.hpp"
+#include "Hll6Array.hpp"
+#include "Hll8Array.hpp"
+#include "HllArray.hpp"
+#include "HllSketchImpl.hpp"
+#include "HllSketchImplFactory.hpp"
+#include "HllUtil.hpp"
+#include "RelativeErrorTables.hpp"
+
+#include "AuxHashMap-internal.hpp"
+#include "coupon_iterator.hpp"
+#include "CouponHashSet-internal.hpp"
+#include "CouponList-internal.hpp"
+#include "Hll4Array-internal.hpp"
+#include "Hll6Array-internal.hpp"
+#include "Hll8Array-internal.hpp"
+#include "HllArray-internal.hpp"
+#include "HllSketch-internal.hpp"
+#include "HllSketchImpl-internal.hpp"
+#include "HllUnion-internal.hpp"
+#include "coupon_iterator-internal.hpp"
+
+#endif // _HLL_PRIVATE_HPP_
diff --git a/be/src/thirdparty/datasketches/inv_pow2_table.hpp b/be/src/thirdparty/datasketches/inv_pow2_table.hpp
new file mode 100644
index 0000000..c904bfe
--- /dev/null
+++ b/be/src/thirdparty/datasketches/inv_pow2_table.hpp
@@ -0,0 +1,107 @@
+/*
+ * 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 INV_POW_2_TAB_HPP_
+#define INV_POW_2_TAB_HPP_
+
+#include <cmath>
+
+namespace datasketches {
+
+// the table was created by the following procedure:
+
+//void fill_inverse_power_of_2_table() {
+//  for (int i = 0; i < 256; i++) {
+//    inverse_powers_of_2_table[i] = pow(2.0, (-1.0 * ((double) i)));
+//    printf("%.17g", inverse_powers_of_2_table[i]);
+//    if (i != 255) printf(", ");
+//    if ((i + 1) % 4 == 0) printf("\n");
+//  }
+//}
+
+static const double INVERSE_POWERS_OF_2[256] = {
+    1, 0.5, 0.25, 0.125,
+    0.0625, 0.03125, 0.015625, 0.0078125,
+    0.00390625, 0.001953125, 0.0009765625, 0.00048828125,
+    0.000244140625, 0.0001220703125, 6.103515625e-05, 3.0517578125e-05,
+    1.52587890625e-05, 7.62939453125e-06, 3.814697265625e-06, 1.9073486328125e-06,
+    9.5367431640625e-07, 4.76837158203125e-07, 2.384185791015625e-07, 1.1920928955078125e-07,
+    5.9604644775390625e-08, 2.9802322387695312e-08, 1.4901161193847656e-08, 7.4505805969238281e-09,
+    3.7252902984619141e-09, 1.862645149230957e-09, 9.3132257461547852e-10, 4.6566128730773926e-10,
+    2.3283064365386963e-10, 1.1641532182693481e-10, 5.8207660913467407e-11, 2.9103830456733704e-11,
+    1.4551915228366852e-11, 7.2759576141834259e-12, 3.637978807091713e-12, 1.8189894035458565e-12,
+    9.0949470177292824e-13, 4.5474735088646412e-13, 2.2737367544323206e-13, 1.1368683772161603e-13,
+    5.6843418860808015e-14, 2.8421709430404007e-14, 1.4210854715202004e-14, 7.1054273576010019e-15,
+    3.5527136788005009e-15, 1.7763568394002505e-15, 8.8817841970012523e-16, 4.4408920985006262e-16,
+    2.2204460492503131e-16, 1.1102230246251565e-16, 5.5511151231257827e-17, 2.7755575615628914e-17,
+    1.3877787807814457e-17, 6.9388939039072284e-18, 3.4694469519536142e-18, 1.7347234759768071e-18,
+    8.6736173798840355e-19, 4.3368086899420177e-19, 2.1684043449710089e-19, 1.0842021724855044e-19,
+    5.4210108624275222e-20, 2.7105054312137611e-20, 1.3552527156068805e-20, 6.7762635780344027e-21,
+    3.3881317890172014e-21, 1.6940658945086007e-21, 8.4703294725430034e-22, 4.2351647362715017e-22,
+    2.1175823681357508e-22, 1.0587911840678754e-22, 5.2939559203393771e-23, 2.6469779601696886e-23,
+    1.3234889800848443e-23, 6.6174449004242214e-24, 3.3087224502121107e-24, 1.6543612251060553e-24,
+    8.2718061255302767e-25, 4.1359030627651384e-25, 2.0679515313825692e-25, 1.0339757656912846e-25,
+    5.169878828456423e-26, 2.5849394142282115e-26, 1.2924697071141057e-26, 6.4623485355705287e-27,
+    3.2311742677852644e-27, 1.6155871338926322e-27, 8.0779356694631609e-28, 4.0389678347315804e-28,
+    2.0194839173657902e-28, 1.0097419586828951e-28, 5.0487097934144756e-29, 2.5243548967072378e-29,
+    1.2621774483536189e-29, 6.3108872417680944e-30, 3.1554436208840472e-30, 1.5777218104420236e-30,
+    7.8886090522101181e-31, 3.944304526105059e-31, 1.9721522630525295e-31, 9.8607613152626476e-32,
+    4.9303806576313238e-32, 2.4651903288156619e-32, 1.2325951644078309e-32, 6.1629758220391547e-33,
+    3.0814879110195774e-33, 1.5407439555097887e-33, 7.7037197775489434e-34, 3.8518598887744717e-34,
+    1.9259299443872359e-34, 9.6296497219361793e-35, 4.8148248609680896e-35, 2.4074124304840448e-35,
+    1.2037062152420224e-35, 6.018531076210112e-36, 3.009265538105056e-36, 1.504632769052528e-36,
+    7.5231638452626401e-37, 3.76158192263132e-37, 1.88079096131566e-37, 9.4039548065783001e-38,
+    4.70197740328915e-38, 2.350988701644575e-38, 1.1754943508222875e-38, 5.8774717541114375e-39,
+    2.9387358770557188e-39, 1.4693679385278594e-39, 7.3468396926392969e-40, 3.6734198463196485e-40,
+    1.8367099231598242e-40, 9.1835496157991212e-41, 4.5917748078995606e-41, 2.2958874039497803e-41,
+    1.1479437019748901e-41, 5.7397185098744507e-42, 2.8698592549372254e-42, 1.4349296274686127e-42,
+    7.1746481373430634e-43, 3.5873240686715317e-43, 1.7936620343357659e-43, 8.9683101716788293e-44,
+    4.4841550858394146e-44, 2.2420775429197073e-44, 1.1210387714598537e-44, 5.6051938572992683e-45,
+    2.8025969286496341e-45, 1.4012984643248171e-45, 7.0064923216240854e-46, 3.5032461608120427e-46,
+    1.7516230804060213e-46, 8.7581154020301067e-47, 4.3790577010150533e-47, 2.1895288505075267e-47,
+    1.0947644252537633e-47, 5.4738221262688167e-48, 2.7369110631344083e-48, 1.3684555315672042e-48,
+    6.8422776578360209e-49, 3.4211388289180104e-49, 1.7105694144590052e-49, 8.5528470722950261e-50,
+    4.276423536147513e-50, 2.1382117680737565e-50, 1.0691058840368783e-50, 5.3455294201843913e-51,
+    2.6727647100921956e-51, 1.3363823550460978e-51, 6.6819117752304891e-52, 3.3409558876152446e-52,
+    1.6704779438076223e-52, 8.3523897190381114e-53, 4.1761948595190557e-53, 2.0880974297595278e-53,
+    1.0440487148797639e-53, 5.2202435743988196e-54, 2.6101217871994098e-54, 1.3050608935997049e-54,
+    6.5253044679985245e-55, 3.2626522339992623e-55, 1.6313261169996311e-55, 8.1566305849981557e-56,
+    4.0783152924990778e-56, 2.0391576462495389e-56, 1.0195788231247695e-56, 5.0978941156238473e-57,
+    2.5489470578119236e-57, 1.2744735289059618e-57, 6.3723676445298091e-58, 3.1861838222649046e-58,
+    1.5930919111324523e-58, 7.9654595556622614e-59, 3.9827297778311307e-59, 1.9913648889155653e-59,
+    9.9568244445778267e-60, 4.9784122222889134e-60, 2.4892061111444567e-60, 1.2446030555722283e-60,
+    6.2230152778611417e-61, 3.1115076389305709e-61, 1.5557538194652854e-61, 7.7787690973264271e-62,
+    3.8893845486632136e-62, 1.9446922743316068e-62, 9.7234613716580339e-63, 4.861730685829017e-63,
+    2.4308653429145085e-63, 1.2154326714572542e-63, 6.0771633572862712e-64, 3.0385816786431356e-64,
+    1.5192908393215678e-64, 7.596454196607839e-65, 3.7982270983039195e-65, 1.8991135491519597e-65,
+    9.4955677457597987e-66, 4.7477838728798994e-66, 2.3738919364399497e-66, 1.1869459682199748e-66,
+    5.9347298410998742e-67, 2.9673649205499371e-67, 1.4836824602749686e-67, 7.4184123013748428e-68,
+    3.7092061506874214e-68, 1.8546030753437107e-68, 9.2730153767185535e-69, 4.6365076883592767e-69,
+    2.3182538441796384e-69, 1.1591269220898192e-69, 5.7956346104490959e-70, 2.897817305224548e-70,
+    1.448908652612274e-70, 7.2445432630613699e-71, 3.6222716315306849e-71, 1.8111358157653425e-71,
+    9.0556790788267124e-72, 4.5278395394133562e-72, 2.2639197697066781e-72, 1.131959884853339e-72,
+    5.6597994242666952e-73, 2.8298997121333476e-73, 1.4149498560666738e-73, 7.074749280333369e-74,
+    3.5373746401666845e-74, 1.7686873200833423e-74, 8.8434366004167113e-75, 4.4217183002083556e-75,
+    2.2108591501041778e-75, 1.1054295750520889e-75, 5.5271478752604446e-76, 2.7635739376302223e-76,
+    1.3817869688151111e-76, 6.9089348440755557e-77, 3.4544674220377779e-77, 1.7272337110188889e-77
+};
+
+} /* namespace datasketches */
+
+#endif
diff --git a/bin/rat_exclude_files.txt b/bin/rat_exclude_files.txt
index 04d73d0..d0c4968 100644
--- a/bin/rat_exclude_files.txt
+++ b/bin/rat_exclude_files.txt
@@ -30,6 +30,7 @@ testdata/cluster/node_templates/cdh7/etc/init.d/kms
 
 # See $IMPALA_HOME/LICENSE.txt
 be/src/gutil/*
+be/src/thirdparty/datasketches/*
 be/src/thirdparty/mpfit/*
 be/src/kudu/gutil
 www/highlight/*
diff --git a/bin/run_clang_tidy.sh b/bin/run_clang_tidy.sh
index 22a80de..4b8eb6b 100755
--- a/bin/run_clang_tidy.sh
+++ b/bin/run_clang_tidy.sh
@@ -39,9 +39,10 @@ then
 fi
 
 DIRS=$(ls -d "${IMPALA_HOME}/be/src/"*/ | grep -v gutil | grep -v kudu |\
-  grep -v thirdparty | tr '\n' ' ')
+    grep -v thirdparty | tr '\n' ' ')
 # Include/exclude select thirdparty dirs.
-DIRS=$DIRS$(ls -d "${IMPALA_HOME}/be/src/thirdparty/"*/ | grep -v mpfit | tr '\n' ' ')
+DIRS=$DIRS$(ls -d "${IMPALA_HOME}/be/src/thirdparty/"*/ | grep -v mpfit |\
+    grep -v datasketches | tr '\n' ' ')
 PIPE_DIRS=$(echo "${DIRS}" | tr ' ' '|')
 
 # Reduce the concurrency to one less than the number of cores in the system. Note than