You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@datasketches.apache.org by al...@apache.org on 2023/12/19 00:33:40 UTC

(datasketches-go) 02/04: Initial HLL port implementation

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

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

commit ac2092ffe03047d4948bd108e59f8f9013546567
Author: Pierre Lacave <pi...@datadoghq.com>
AuthorDate: Thu Dec 14 12:11:11 2023 +0100

    Initial HLL port implementation
---
 .github/workflows/go.yml               |  30 ++
 common/family.go                       |  22 +
 common/utils.go                        |  63 +++
 common/utils_slice.go                  |  29 ++
 common/utils_test.go                   |  27 ++
 go.mod                                 |  16 +
 go.sum                                 |  12 +
 hll/README.md                          |  46 ++
 hll/aux_hash_map.go                    | 207 +++++++++
 hll/aux_hash_map_test.go               |  72 +++
 hll/composite_interpolation_xtables.go | 769 +++++++++++++++++++++++++++++++++
 hll/coupon.go                          | 145 +++++++
 hll/coupon_hash_set.go                 | 233 ++++++++++
 hll/coupon_list.go                     | 189 ++++++++
 hll/coupon_list_test.go                | 113 +++++
 hll/cross_counting_test.go             |  97 +++++
 hll/cubic_interpolation.go             | 138 ++++++
 hll/cubic_interpolation_test.go        |  38 ++
 hll/harmonic_numbers.go                |  85 ++++
 hll/hll_4array.go                      | 203 +++++++++
 hll/hll_4update.go                     | 197 +++++++++
 hll/hll_6array.go                      | 199 +++++++++
 hll/hll_8array.go                      | 170 ++++++++
 hll/hll_array.go                       | 303 +++++++++++++
 hll/hll_array_test.go                  | 120 +++++
 hll/hll_config.go                      |  44 ++
 hll/hll_estimator.go                   | 153 +++++++
 hll/hll_pair_iterator.go               |  52 +++
 hll/hll_sketch.go                      | 311 +++++++++++++
 hll/hll_sketch_isomomorphism_test.go   | 250 +++++++++++
 hll/hll_sketch_serialization_test.go   | 240 ++++++++++
 hll/hll_sketch_test.go                 | 398 +++++++++++++++++
 hll/hll_utils.go                       | 169 ++++++++
 hll/pair_iterator.go                   |  95 ++++
 hll/preamble_utils.go                  | 287 ++++++++++++
 hll/relative_error_tables.go           |  94 ++++
 hll/to_slice_impl.go                   | 159 +++++++
 hll/union.go                           | 395 +++++++++++++++++
 hll/union_test.go                      | 312 +++++++++++++
 main.go                                |  18 +
 thetacommon/theta_utils.go             |  22 +
 41 files changed, 6522 insertions(+)

diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml
new file mode 100644
index 0000000..fefbe47
--- /dev/null
+++ b/.github/workflows/go.yml
@@ -0,0 +1,30 @@
+# This workflow will build a golang project
+# For more information see: https://docs.github.com/en/actions/automating-builds-and-tests/building-and-testing-go
+
+name: Go
+
+on:
+  push:
+    branches: [ "main" ]
+  pull_request:
+    branches: [ "main" ]
+
+jobs:
+
+  build:
+    runs-on: ubuntu-latest
+    strategy:
+      matrix:
+        go-version: [ '1.21.x' ]
+    steps:
+    - uses: actions/checkout@v3
+
+    - name: Set up Go ${{ matrix.go-version }}
+      uses: actions/setup-go@v4
+      with:
+        go-version: ${{ matrix.go-version }}
+    - name: Build
+      run: go build -v ./...
+
+    - name: Test
+      run: go test -v ./...
diff --git a/common/family.go b/common/family.go
new file mode 100644
index 0000000..8f36886
--- /dev/null
+++ b/common/family.go
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+package common
+
+const (
+	Family_HLL_ID = 7
+)
diff --git a/common/utils.go b/common/utils.go
new file mode 100644
index 0000000..5664645
--- /dev/null
+++ b/common/utils.go
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+
+package common
+
+import (
+	"math"
+	"math/bits"
+	"strconv"
+)
+
+// InvPow2 returns 2^(-e).
+func InvPow2(e int) float64 {
+	if (e | 1024 - e - 1) < 0 {
+		panic("e cannot be negative or greater than 1023: " + strconv.Itoa(e))
+	}
+	return math.Float64frombits((1023 - uint64(e)) << 52)
+}
+
+// CeilPowerOf2 returns the smallest power of 2 greater than or equal to n.
+func CeilPowerOf2(n int) int {
+	if n <= 1 {
+		return 1
+	}
+	topIntPwrOf2 := 1 << 30
+	if n >= topIntPwrOf2 {
+		return topIntPwrOf2
+	}
+	return int(math.Pow(2, math.Ceil(math.Log2(float64(n)))))
+}
+
+func ExactLog2OfLong(powerOf2 uint64) int {
+	if !isLongPowerOf2(powerOf2) {
+		panic("Argument 'powerOf2' must be a positive power of 2.")
+	}
+	return bits.TrailingZeros64(powerOf2)
+}
+
+// isLongPowerOf2 returns true if the given number is a power of 2.
+func isLongPowerOf2(powerOf2 uint64) bool {
+	return powerOf2 > 0 && (powerOf2&(powerOf2-1)) == 0
+}
+
+func BoolToInt(b bool) int {
+	if b {
+		return 1
+	}
+	return 0
+}
diff --git a/common/utils_slice.go b/common/utils_slice.go
new file mode 100644
index 0000000..238fbfc
--- /dev/null
+++ b/common/utils_slice.go
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+package common
+
+// GetShortLE gets a short value from a byte array in little endian format.
+func GetShortLE(array []byte, offset int) int {
+	return int(array[offset]&0xFF) | (int(array[offset+1]&0xFF) << 8)
+}
+
+// PutShortLE puts a short value into a byte array in little endian format.
+func PutShortLE(array []byte, offset int, value int) {
+	array[offset] = byte(value)
+	array[offset+1] = byte(value >> 8)
+}
diff --git a/common/utils_test.go b/common/utils_test.go
new file mode 100644
index 0000000..4ce12e9
--- /dev/null
+++ b/common/utils_test.go
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+package common
+
+import (
+	"fmt"
+	"testing"
+)
+
+func TestInvPow2(t *testing.T) {
+	fmt.Printf("%f", InvPow2(0))
+}
diff --git a/go.mod b/go.mod
new file mode 100644
index 0000000..53eba33
--- /dev/null
+++ b/go.mod
@@ -0,0 +1,16 @@
+module github.com/apache/datasketches-go
+
+go 1.21
+
+toolchain go1.21.1
+
+require (
+	github.com/spaolacci/murmur3 v1.1.0
+	github.com/stretchr/testify v1.8.4
+)
+
+require (
+	github.com/davecgh/go-spew v1.1.1 // indirect
+	github.com/pmezard/go-difflib v1.0.0 // indirect
+	gopkg.in/yaml.v3 v3.0.1 // indirect
+)
diff --git a/go.sum b/go.sum
new file mode 100644
index 0000000..d004ff9
--- /dev/null
+++ b/go.sum
@@ -0,0 +1,12 @@
+github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
+github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
+github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
+github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI=
+github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA=
+github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk=
+github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
+gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
diff --git a/hll/README.md b/hll/README.md
new file mode 100644
index 0000000..4352c5c
--- /dev/null
+++ b/hll/README.md
@@ -0,0 +1,46 @@
+
+
+# HllSketch
+
+HllSketch is actually a collection of compact implementations of Phillipe Flajolet’s HyperLogLog (HLL) sketch but with significantly improved error behavior and excellent speed performance.
+
+If the use case for sketching is primarily counting uniques and merging, the HLL sketch is the 2nd highest
+performing in terms of accuracy for storage space consumed in the DataSketches library (the new CPC sketch developed by Kevin J. Lang now beats HLL in terms of accuracy / space).
+
+
+For large counts, HLL sketches can be 2 to 8 times smaller for the same accuracy than the DataSketches Theta Sketches when serialized, but the Theta sketches can do set intersections and differences while HLL and CPC cannot.
+
+The CPC sketch and HLL share similar use cases, but the CPC sketch is about 30 to 40% smaller than the HLL sketch when serialized and larger than the HLL when active in memory.  Choose your weapons!
+
+
+A new HLL sketch is created with a simple constructor:
+
+    lgK := 12 // This is log-base2 of k, so k = 4096. lgK can be from 4 to 21
+    sketch := NewHllSketch(lgK) // TgtHllType_HLL_4 is the default
+    // OR
+    sketch := NewHllSketch(lgK, TgtHllType_HLL_6)
+    // OR
+    sketch := NewHllSketch(lgK, TgtHllType_HLL_8)
+
+
+All three different sketch types are targets in that the sketches start out in a warm-up mode that is small in size and gradually grows as needed until the full HLL array is allocated. 
+
+The HLL_4, HLL_6 and HLL_8 represent different levels of compression of the final HLL array where the 4, 6 and 8 refer to the number of bits each bucket of the HLL array is compressed down to.
+
+The HLL_4 is the most compressed but generally slower than the other two, especially during union operations.</p>
+
+All three types share the same API. Updating the HllSketch is very simple:
+
+	for i := 0; i < 1000000; i++ {
+	  sketch.UpdateInt64(i);
+	}
+
+Each of the presented integers above are first hashed into 128-bit hash values that are used by the sketch
+HLL algorithm, so the above loop is essentially equivalent to using a random number generator initialized with a
+seed so that the sequence is deterministic and random.
+
+Obtaining the cardinality results from the sketch is also simple:
+
+	estimate := sketch.GetEstimate()
+
+Note, this is a port from the Java version of the HLL sketch of Apache Datasketches, so the API is slightly different from the Java version.
diff --git a/hll/aux_hash_map.go b/hll/aux_hash_map.go
new file mode 100644
index 0000000..9d09534
--- /dev/null
+++ b/hll/aux_hash_map.go
@@ -0,0 +1,207 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"encoding/binary"
+	"fmt"
+)
+
+type auxHashMap interface {
+	mustFindValueFor(slotNo int) int
+	mustReplace(slotNo int, value int)
+	mustAdd(slotNo int, value int)
+	iterator() pairIterator
+	getAuxCount() int
+	getLgAuxArrInts() int
+	copy() auxHashMap
+	getUpdatableSizeBytes() int
+	getCompactSizeBytes() int
+	getAuxIntArr() []int
+}
+
+// auxHashMapImpl is a hash table for the Aux array.
+type auxHashMapImpl struct {
+	lgConfigK    int //required for #slot bits
+	lgAuxArrInts int
+	auxCount     int
+	auxIntArr    []int
+}
+
+func (a *auxHashMapImpl) copy() auxHashMap {
+	newA := a
+	newA.auxIntArr = make([]int, len(a.auxIntArr))
+	copy(newA.auxIntArr, a.auxIntArr)
+	return newA
+}
+
+// newAuxHashMap returns a new auxHashMap.
+func newAuxHashMap(lgAuxArrInts int, lgConfigK int) auxHashMap {
+	return &auxHashMapImpl{
+		lgConfigK:    lgConfigK,
+		lgAuxArrInts: lgAuxArrInts,
+		auxCount:     0,
+		auxIntArr:    make([]int, 1<<lgAuxArrInts),
+	}
+}
+
+// deserializeAuxHashMap returns a new auxHashMap from the given byte array.
+func deserializeAuxHashMap(byteArray []byte, offset int, lgConfigL int, auxCount int, srcCompact bool) auxHashMap {
+	var (
+		lgAuxArrInts int
+	)
+
+	if srcCompact {
+		lgAuxArrInts = computeLgArr(byteArray, auxCount, lgConfigL)
+	} else {
+		lgAuxArrInts = extractLgArr(byteArray)
+	}
+
+	auxMap := newAuxHashMap(lgAuxArrInts, lgConfigL)
+	configKMask := (1 << lgConfigL) - 1
+
+	if srcCompact {
+		for i := 0; i < auxCount; i++ {
+			pair := int(binary.LittleEndian.Uint32(byteArray[offset+(i<<2) : offset+(i<<2)+4]))
+			slotNo := getPairLow26(pair) & configKMask
+			value := getPairValue(pair)
+			auxMap.mustAdd(slotNo, value) //increments count
+		}
+	} else { //updatable
+		auxArrInts := 1 << lgAuxArrInts
+		for i := 0; i < auxArrInts; i++ {
+			pair := int(binary.LittleEndian.Uint32(byteArray[offset+(i<<2) : offset+(i<<2)+4]))
+			if pair == empty {
+				continue
+			}
+			slotNo := getPairLow26(pair) & configKMask
+			value := getPairValue(pair)
+			auxMap.mustAdd(slotNo, value) //increments count
+		}
+	}
+	return auxMap
+}
+
+func (a *auxHashMapImpl) getAuxIntArr() []int {
+	return a.auxIntArr
+}
+
+func (a *auxHashMapImpl) getCompactSizeBytes() int {
+	return a.auxCount << 2
+}
+
+func (a *auxHashMapImpl) getUpdatableSizeBytes() int {
+	return 4 << a.lgAuxArrInts
+}
+
+func (a *auxHashMapImpl) mustFindValueFor(slotNo int) int {
+	index := findAuxHashMap(a.auxIntArr, a.lgAuxArrInts, a.lgConfigK, slotNo)
+	if index < 0 {
+		panic(fmt.Sprintf("SlotNo not found: %d", slotNo))
+	}
+	return getPairValue(a.auxIntArr[index])
+}
+
+func (a *auxHashMapImpl) mustReplace(slotNo int, value int) {
+	index := findAuxHashMap(a.auxIntArr, a.lgAuxArrInts, a.lgConfigK, slotNo)
+	if index < 0 {
+		pairStr := pairString(pair(slotNo, value))
+		panic(fmt.Sprintf("pair not found: %v", pairStr))
+	}
+	a.auxIntArr[index] = pair(slotNo, value)
+}
+
+// mustAdd adds the slotNo and value to the aux array.
+// slotNo the index from the HLL array
+// value the HLL value at the slotNo.
+func (a *auxHashMapImpl) mustAdd(slotNo int, value int) {
+	index := findAuxHashMap(a.auxIntArr, a.lgAuxArrInts, a.lgConfigK, slotNo)
+	pair := pair(slotNo, value)
+	if index >= 0 {
+		pairStr := pairString(pair)
+		panic(fmt.Sprintf("found a slotNo that should not be there: %s", pairStr))
+	}
+	a.auxIntArr[^index] = pair
+	a.auxCount++
+	a.checkGrow()
+}
+
+func (a *auxHashMapImpl) getLgAuxArrInts() int {
+	return a.lgAuxArrInts
+}
+
+// iterator returns an iterator over the Aux array.
+func (a *auxHashMapImpl) iterator() pairIterator {
+	return newIntArrayPairIterator(a.auxIntArr, a.lgConfigK)
+}
+
+// getAuxCount returns the number of entries in the Aux array.
+func (a *auxHashMapImpl) getAuxCount() int {
+	return a.auxCount
+}
+
+// checkGrow checks to see if the aux array should be grown and does so if needed.
+func (a *auxHashMapImpl) checkGrow() {
+	if (resizeDenom * a.auxCount) <= (resizeNumber * len(a.auxIntArr)) {
+		return
+	}
+	a.growAuxSpace()
+}
+
+// growAuxSpace doubles the size of the aux array and reinsert the existing entries.
+func (a *auxHashMapImpl) growAuxSpace() {
+	oldArray := a.auxIntArr
+	configKMask := int((1 << a.lgConfigK) - 1)
+	a.lgAuxArrInts++
+	a.auxIntArr = make([]int, 1<<a.lgAuxArrInts)
+	for _, fetched := range oldArray {
+		if fetched != empty {
+			//find empty in new array
+			idx := findAuxHashMap(a.auxIntArr, a.lgAuxArrInts, a.lgConfigK, fetched&configKMask)
+			a.auxIntArr[^idx] = fetched
+		}
+	}
+}
+
+// findAuxHashMap 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, panic.
+func findAuxHashMap(auxArr []int, lgAuxArrInts int, lgConfigK int, slotNo int) int {
+	if lgAuxArrInts >= lgConfigK {
+		panic("lgAuxArrInts >= lgConfigK")
+	}
+	auxArrMask := (1 << lgAuxArrInts) - 1
+	configKMask := (1 << lgConfigK) - 1
+	probe := slotNo & auxArrMask
+	loopIndex := probe
+	for {
+		arrVal := auxArr[probe]
+		if arrVal == empty {
+			return ^probe
+		} else if slotNo == (arrVal & configKMask) {
+			return probe
+		}
+		stride := (slotNo >> lgAuxArrInts) | 1
+		probe = (probe + stride) & auxArrMask
+		if probe == loopIndex {
+			panic("key not found and no empty slots")
+		}
+	}
+}
diff --git a/hll/aux_hash_map_test.go b/hll/aux_hash_map_test.go
new file mode 100644
index 0000000..8909066
--- /dev/null
+++ b/hll/aux_hash_map_test.go
@@ -0,0 +1,72 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestMustReplace(t *testing.T) {
+	auxMap := newAuxHashMap(3, 7)
+	auxMap.mustAdd(100, 5)
+	val := auxMap.mustFindValueFor(100)
+	assert.Equal(t, 5, val)
+	auxMap.mustReplace(100, 10)
+	val = auxMap.mustFindValueFor(100)
+	assert.Equal(t, 10, val)
+	assert.Panics(t, func() { auxMap.mustReplace(101, 5) }, "pair not found: SlotNo: 101, Value: 5")
+}
+
+func TestGrowAuxSpace(t *testing.T) {
+	auxMap := newAuxHashMap(3, 7)
+	assert.Equal(t, 3, auxMap.getLgAuxArrInts())
+	for i := 1; i <= 7; i++ {
+		auxMap.mustAdd(i, i)
+	}
+	assert.Equal(t, 4, auxMap.getLgAuxArrInts())
+	itr := auxMap.iterator()
+
+	var (
+		count1 = 0
+		count2 = 0
+	)
+
+	for itr.nextAll() {
+		count2++
+		pair := itr.getPair()
+		if pair != 0 {
+			count1++
+		}
+	}
+	assert.Equal(t, 7, count1)
+	assert.Equal(t, 16, count2)
+}
+
+func TestExceptions1(t *testing.T) {
+	auxMap := newAuxHashMap(3, 7)
+	auxMap.mustAdd(100, 5)
+	assert.Panics(t, func() { auxMap.mustFindValueFor(101) }, "SlotNo not found: 101")
+}
+
+func TestExceptions2(t *testing.T) {
+	auxMap := newAuxHashMap(3, 7)
+	auxMap.mustAdd(100, 5)
+	assert.Panics(t, func() { auxMap.mustAdd(100, 6) }, "found a slotNo that should not be there: SlotNo: 100, Value: 6")
+}
diff --git a/hll/composite_interpolation_xtables.go b/hll/composite_interpolation_xtables.go
new file mode 100644
index 0000000..64cbfc5
--- /dev/null
+++ b/hll/composite_interpolation_xtables.go
@@ -0,0 +1,769 @@
+/*
+ * 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.
+ */
+
+package hll
+
+var (
+	// compositeInterpolationYstrides contains 18 Values, index 0 is LgK = 4, index 17 is LgK = 21.
+	compositeInterpolationYstrides = []float64{1, 2, 3, 5, 10, 20, 40, 80, 160, 320, 640, 1280, 2560, 5120, 10240, 20480, 40960, 81920}
+	compositeInterpolationXarrs    = [][]float64{
+		// log K = 4
+		{
+			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,
+		},
+	}
+)
diff --git a/hll/coupon.go b/hll/coupon.go
new file mode 100644
index 0000000..0604aca
--- /dev/null
+++ b/hll/coupon.go
@@ -0,0 +1,145 @@
+/*
+ * 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.
+ */
+
+package hll
+
+type hllCoupon interface {
+	hllSketchBase
+
+	getCouponCount() int
+	getCouponIntArr() []int
+	getLgCouponArrInts() int
+}
+
+type hllCouponState struct {
+	lgCouponArrInts int
+	couponCount     int
+	couponIntArr    []int
+}
+
+func (c hllCouponState) IsEmpty() bool {
+	return c.couponCount == 0
+}
+
+func (c hllCouponState) getCouponIntArr() []int {
+	return c.couponIntArr
+}
+
+func (c hllCouponState) getCouponCount() int {
+	return c.couponCount
+}
+
+func (c hllCouponState) getLgCouponArrInts() int {
+	return c.lgCouponArrInts
+}
+
+func (c hllCouponState) isOutOfOrder() bool {
+	return false
+}
+
+func (c hllCouponState) isRebuildCurMinNumKxQFlag() bool {
+	return false
+}
+
+func (c hllCouponState) putOutOfOrder(bool) {
+	//no-op for coupons
+}
+
+func (c hllCouponState) putRebuildCurMinNumKxQFlag(bool) {
+	//no-op for coupons
+}
+
+func newHllCouponState(lgCouponArrInts int, couponCount int, couponIntArr []int) hllCouponState {
+	return hllCouponState{
+		lgCouponArrInts: lgCouponArrInts,
+		couponCount:     couponCount,
+		couponIntArr:    couponIntArr,
+	}
+}
+
+// GetEstimate returns the estimate of the hllCouponState.
+func getEstimate(c hllCoupon) float64 {
+	couponCount := c.getCouponCount()
+	est := usingXAndYTables(couponMappingXArr, couponMappingYArr, float64(couponCount))
+	return max(est, float64(couponCount))
+}
+
+func getUpperBound(c hllCoupon, numStdDev int) (float64, error) {
+	err := checkNumStdDev(numStdDev)
+	if err != nil {
+		return 0, err
+	}
+	couponCount := c.getCouponCount()
+	est := usingXAndYTables(couponMappingXArr, couponMappingYArr, float64(couponCount))
+	tmp := est / (1.0 - (float64(numStdDev) * couponRSE))
+	return max(tmp, float64(couponCount)), nil
+}
+
+func getLowerBound(c hllCoupon, numStdDev int) (float64, error) {
+	err := checkNumStdDev(numStdDev)
+	if err != nil {
+		return 0, err
+	}
+	couponCount := c.getCouponCount()
+	est := usingXAndYTables(couponMappingXArr, couponMappingYArr, float64(couponCount))
+	tmp := est / (1.0 + (float64(numStdDev) * couponRSE))
+	return max(tmp, float64(couponCount)), nil
+}
+
+func mergeCouponTo(from hllCoupon, dest HllSketch) {
+	intArrFrom := from.getCouponIntArr()
+	arrLen := len(intArrFrom)
+	for i := 0; i < arrLen; i++ {
+		pair := intArrFrom[i]
+		if pair == empty {
+			continue
+		}
+		sk := dest.(*hllSketchImpl).sketch.couponUpdate(pair)
+		dest.(*hllSketchImpl).sketch = sk
+	}
+}
+
+var (
+
+	// couponMappingXArr is computed for Coupon lgK = 26 ONLY. Designed for the cubic interpolator function.
+	couponMappingXArr = []float64{
+		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,
+	}
+
+	//couponMappingYArr is computed for Coupon lgK = 26 ONLY. Designed for the cubic interpolator function.
+	couponMappingYArr = []float64{
+		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,
+	}
+)
diff --git a/hll/coupon_hash_set.go b/hll/coupon_hash_set.go
new file mode 100644
index 0000000..2ca7843
--- /dev/null
+++ b/hll/coupon_hash_set.go
@@ -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.
+ */
+
+package hll
+
+import (
+	"encoding/binary"
+	"fmt"
+)
+
+type couponHashSetImpl struct {
+	hllSketchConfig
+	hllCouponState
+}
+
+func (c *couponHashSetImpl) GetCompositeEstimate() float64 {
+	return getEstimate(c)
+}
+
+func (c *couponHashSetImpl) GetEstimate() float64 {
+	return getEstimate(c)
+}
+
+func (c *couponHashSetImpl) GetHipEstimate() float64 {
+	return getEstimate(c)
+}
+
+func (c *couponHashSetImpl) GetLowerBound(numStdDev int) (float64, error) {
+	return getLowerBound(c, numStdDev)
+}
+
+func (c *couponHashSetImpl) GetUpperBound(numStdDev int) (float64, error) {
+	return getUpperBound(c, numStdDev)
+}
+
+func (c *couponHashSetImpl) GetUpdatableSerializationBytes() int {
+	return c.getMemDataStart() + (4 << c.getLgCouponArrInts())
+}
+
+func (c *couponHashSetImpl) ToCompactSlice() ([]byte, error) {
+	return toCouponSlice(c, true)
+}
+
+func (c *couponHashSetImpl) ToUpdatableSlice() ([]byte, error) {
+	return toCouponSlice(c, false)
+}
+
+// couponUpdate updates the couponHashSetImpl with the given coupon.
+func (c *couponHashSetImpl) couponUpdate(coupon int) hllSketchBase {
+	index := findCoupon(c.couponIntArr, c.lgCouponArrInts, coupon)
+	if index >= 0 {
+		return c //found duplicate, ignore
+	}
+	c.couponIntArr[^index] = coupon
+	c.couponCount++ //found empty
+	t, err := c.checkGrowOrPromote()
+	if err != nil {
+		return nil
+	}
+	if t {
+		return promoteSetToHll(c)
+	}
+	return c
+}
+
+func (c *couponHashSetImpl) iterator() pairIterator {
+	return newIntArrayPairIterator(c.couponIntArr, c.lgConfigK)
+}
+
+func (c *couponHashSetImpl) getMemDataStart() int {
+	return hashSetIntArrStart
+}
+
+func (c *couponHashSetImpl) getPreInts() int {
+	return hashSetPreInts
+}
+
+func (c *couponHashSetImpl) copyAs(tgtHllType TgtHllType) hllSketchBase {
+	newC := &couponHashSetImpl{
+		hllSketchConfig: newHllSketchConfig(c.lgConfigK, tgtHllType, curMode_SET),
+		hllCouponState:  newHllCouponState(c.lgCouponArrInts, c.couponCount, make([]int, len(c.couponIntArr))),
+	}
+
+	copy(newC.couponIntArr, c.couponIntArr)
+	return newC
+}
+
+func (c *couponHashSetImpl) copy() hllSketchBase {
+	newC := &couponHashSetImpl{
+		hllSketchConfig: newHllSketchConfig(c.lgConfigK, c.tgtHllType, curMode_SET),
+		hllCouponState:  newHllCouponState(c.lgCouponArrInts, c.couponCount, make([]int, len(c.couponIntArr))),
+	}
+
+	copy(newC.couponIntArr, c.couponIntArr)
+	return newC
+}
+
+func (c *couponHashSetImpl) mergeTo(dest HllSketch) {
+	mergeCouponTo(c, dest)
+}
+
+// checkGrowOrPromote checks if the couponHashSetImpl should grow or promote to HLL.
+func (c *couponHashSetImpl) checkGrowOrPromote() (bool, error) {
+	if (resizeDenom * c.couponCount) <= (resizeNumber * (1 << c.lgCouponArrInts)) {
+		return false, nil
+	}
+	if c.lgCouponArrInts == (c.lgConfigK - 3) {
+		return true, nil // promote to HLL
+	}
+	c.lgCouponArrInts++
+	arr := growHashSet(c.couponIntArr, c.lgCouponArrInts)
+	c.couponIntArr = arr
+	return false, nil
+}
+
+// growHashSet doubles the size of the given couponHashSetImpl and reinsert the existing entries.
+func growHashSet(couponIntArr []int, tgtLgCoupArrSize int) []int {
+	tgtCouponIntArr := make([]int, 1<<tgtLgCoupArrSize)
+	for _, fetched := range couponIntArr {
+		if fetched != empty {
+			idx := findCoupon(tgtCouponIntArr, tgtLgCoupArrSize, fetched)
+			if idx < 0 {
+				tgtCouponIntArr[^idx] = fetched
+				continue
+			}
+			panic("growHashSet, found duplicate")
+		}
+	}
+	return tgtCouponIntArr
+}
+
+// promoteSetToHll move coupons to an hllArray from couponHashSetImpl
+func promoteSetToHll(src *couponHashSetImpl) hllArray {
+	tgtHllArr, _ := newHllArray(src.lgConfigK, src.tgtHllType)
+	srcIter := src.iterator()
+	tgtHllArr.putKxQ0(float64(uint64(1) << src.lgConfigK))
+
+	for srcIter.nextValid() {
+		p := srcIter.getPair()
+		tgtHllArr.couponUpdate(p)
+	}
+	est := src.GetEstimate()
+	tgtHllArr.putHipAccum(est)
+	tgtHllArr.putOutOfOrder(false)
+	return tgtHllArr
+}
+
+// findCoupon searches the Coupon hash table for an empty slot or a duplicate depending on the context.
+// If entire entry is empty, returns one's complement of index = found empty.
+// If entry equals given coupon, returns its index = found duplicate coupon.
+// Continues searching.
+// If the probe comes back to original index, panic.
+func findCoupon(array []int, lgArrInts int, coupon int) int {
+	arrMask := len(array) - 1
+	probe := coupon & arrMask
+	loopIndex := probe
+
+	for ok := true; ok; ok = probe != loopIndex {
+		couponAtIdx := array[probe]
+		if couponAtIdx == empty {
+			return ^probe //empty
+		} else if coupon == couponAtIdx {
+			return probe //duplicate
+		}
+		stride := ((coupon & keyMask26) >> lgArrInts) | 1
+		probe = (probe + stride) & arrMask
+	}
+	panic("key not found and no empty slots")
+}
+
+// newCouponHashSet returns a new couponHashSetImpl.
+// lgConfigK the configured Lg K
+// tgtHllType the target HLL type
+func newCouponHashSet(lgConfigK int, tgtHllType TgtHllType) (couponHashSetImpl, error) {
+	if lgConfigK <= 7 {
+		return couponHashSetImpl{}, fmt.Errorf("lgConfigK must be > 7 for SET mode")
+	}
+	cl, err := newCouponList(lgConfigK, tgtHllType, curMode_SET)
+	if err != nil {
+		return couponHashSetImpl{}, err
+	}
+	return couponHashSetImpl(cl), nil
+}
+
+// deserializeCouponHashSet returns a new couponHashSetImpl from the given byte array.
+func deserializeCouponHashSet(byteArray []byte) (hllCoupon, error) {
+	lgConfigK := extractLgK(byteArray)
+	tgtHllType := extractTgtHllType(byteArray)
+
+	curMode := extractCurMode(byteArray)
+	memArrStart := listIntArrStart
+	if curMode == curMode_SET {
+		memArrStart = hashSetIntArrStart
+	}
+	set, err := newCouponHashSet(lgConfigK, tgtHllType)
+	if err != nil {
+		return nil, err
+	}
+	memIsCompact := extractCompactFlag(byteArray)
+	couponCount := extractHashSetCount(byteArray)
+	lgCouponArrInts := extractLgArr(byteArray)
+	if lgCouponArrInts < lgInitSetSize {
+		lgCouponArrInts = computeLgArr(byteArray, couponCount, lgConfigK)
+	}
+	if memIsCompact {
+		for it := 0; it < couponCount; it++ {
+			set.couponUpdate(int(binary.LittleEndian.Uint32(byteArray[memArrStart+(it<<2) : memArrStart+(it<<2)+4])))
+		}
+	} else {
+		set.couponCount = couponCount
+		set.lgCouponArrInts = lgCouponArrInts
+		couponArrInts := 1 << lgCouponArrInts
+		set.couponIntArr = make([]int, couponArrInts)
+		for it := 0; it < couponArrInts; it++ {
+			set.couponIntArr[it] = int(binary.LittleEndian.Uint32(byteArray[hashSetIntArrStart+(it<<2) : hashSetIntArrStart+(it<<2)+4]))
+		}
+	}
+	return &set, nil
+}
diff --git a/hll/coupon_list.go b/hll/coupon_list.go
new file mode 100644
index 0000000..7c6d43d
--- /dev/null
+++ b/hll/coupon_list.go
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package hll
+
+import (
+	"encoding/binary"
+	"fmt"
+)
+
+type couponListImpl struct {
+	hllSketchConfig
+	hllCouponState
+}
+
+func (c *couponListImpl) GetCompositeEstimate() float64 {
+	return getEstimate(c)
+}
+
+func (c *couponListImpl) GetEstimate() float64 {
+	return getEstimate(c)
+}
+
+func (c *couponListImpl) GetHipEstimate() float64 {
+	return getEstimate(c)
+}
+
+func (c *couponListImpl) GetLowerBound(numStdDev int) (float64, error) {
+	return getLowerBound(c, numStdDev)
+}
+
+func (c *couponListImpl) GetUpperBound(numStdDev int) (float64, error) {
+	return getUpperBound(c, numStdDev)
+}
+
+func (c *couponListImpl) GetUpdatableSerializationBytes() int {
+	return c.getMemDataStart() + (4 << c.getLgCouponArrInts())
+}
+
+func (c *couponListImpl) ToCompactSlice() ([]byte, error) {
+	return toCouponSlice(c, true)
+}
+
+func (c *couponListImpl) ToUpdatableSlice() ([]byte, error) {
+	return toCouponSlice(c, false)
+}
+
+// couponUpdate updates the couponListImpl with the given coupon.
+// it returns the updated couponListImpl or a newly promoted couponHashSetImpl.
+func (c *couponListImpl) couponUpdate(coupon int) hllSketchBase {
+	length := 1 << c.lgCouponArrInts
+	for i := 0; i < length; i++ {
+		couponAtIdx := c.couponIntArr[i]
+		if couponAtIdx == empty {
+			c.couponIntArr[i] = coupon //update
+			c.couponCount++
+			if c.couponCount >= length {
+				if c.lgConfigK < 8 {
+					return promoteListToHll(c) //oooFlag = false
+				}
+				return promoteListToSet(c) //oooFlag = true
+			}
+			return c
+		}
+		//cell not empty
+		if couponAtIdx == coupon {
+			return c //duplicate
+		}
+		//cell not empty & not a duplicate, continue
+	}
+	panic("array invalid: no empties & no duplicates")
+}
+
+// iterator returns an iterator over the couponListImpl.
+func (c *couponListImpl) iterator() pairIterator {
+	return newIntArrayPairIterator(c.couponIntArr, c.lgConfigK)
+}
+
+func (c *couponListImpl) getMemDataStart() int {
+	return listIntArrStart
+}
+
+func (c *couponListImpl) getPreInts() int {
+	return listPreInts
+}
+
+func (c *couponListImpl) copyAs(tgtHllType TgtHllType) hllSketchBase {
+	newC := &couponListImpl{
+		hllSketchConfig: newHllSketchConfig(c.lgConfigK, tgtHllType, curMode_LIST),
+		hllCouponState:  newHllCouponState(c.lgCouponArrInts, c.couponCount, make([]int, len(c.couponIntArr))),
+	}
+
+	copy(newC.couponIntArr, c.couponIntArr)
+	return newC
+}
+
+func (c *couponListImpl) copy() hllSketchBase {
+	newC := &couponListImpl{
+		hllSketchConfig: newHllSketchConfig(c.lgConfigK, c.tgtHllType, curMode_LIST),
+		hllCouponState:  newHllCouponState(c.lgCouponArrInts, c.couponCount, make([]int, len(c.couponIntArr))),
+	}
+
+	copy(newC.couponIntArr, c.couponIntArr)
+	return newC
+}
+
+func (c *couponListImpl) mergeTo(dest HllSketch) {
+	mergeCouponTo(c, dest)
+}
+
+// promoteListToHll move coupons to an hllArray from couponListImpl
+func promoteListToHll(src *couponListImpl) hllArray {
+	tgtHllArr, _ := newHllArray(src.lgConfigK, src.tgtHllType)
+	srcIter := src.iterator()
+	tgtHllArr.putKxQ0(float64(uint64(1) << src.lgConfigK))
+
+	for srcIter.nextValid() {
+		p := srcIter.getPair()
+		tgtHllArr.couponUpdate(p)
+	}
+	est := src.GetEstimate()
+	tgtHllArr.putHipAccum(est)
+	tgtHllArr.putOutOfOrder(false)
+	return tgtHllArr
+}
+
+// promoteListToSet move coupons to a couponHashSetImpl from couponListImpl
+func promoteListToSet(c *couponListImpl) hllSketchBase {
+	couponCount := c.getCouponCount()
+	arr := c.couponIntArr
+	chSet, _ := newCouponHashSet(c.lgConfigK, c.tgtHllType)
+	for i := 0; i < couponCount; i++ {
+		chSet.couponUpdate(arr[i])
+	}
+	return &chSet
+}
+
+// newCouponList returns a new couponListImpl.
+func newCouponList(lgConfigK int, tgtHllType TgtHllType, curMode curMode) (couponListImpl, error) {
+	var (
+		lgCouponArrInts = lgInitSetSize //SET
+	)
+
+	if curMode == curMode_LIST {
+		lgCouponArrInts = lgInitListSize
+	} else if lgConfigK <= 7 {
+		return couponListImpl{}, fmt.Errorf("lgConfigK must be > 7 for non-HLL mode")
+	}
+
+	couponIntArr := make([]int, 1<<lgCouponArrInts)
+	couponCount := 0
+
+	return couponListImpl{
+		hllSketchConfig: newHllSketchConfig(lgConfigK, tgtHllType, curMode),
+		hllCouponState:  newHllCouponState(lgCouponArrInts, couponCount, couponIntArr),
+	}, nil
+}
+
+// deserializeCouponList returns a new couponListImpl from the given byte slice.
+func deserializeCouponList(byteArray []byte) (hllCoupon, error) {
+	lgConfigK := extractLgK(byteArray)
+	tgtHllType := extractTgtHllType(byteArray)
+
+	list, err := newCouponList(lgConfigK, tgtHllType, curMode_LIST)
+	if err != nil {
+		return nil, err
+	}
+	couponCount := extractListCount(byteArray)
+	// TODO there must be a more efficient to reinterpret the byte array as an int array
+	for it := 0; it < couponCount; it++ {
+		list.couponIntArr[it] = int(binary.LittleEndian.Uint32(byteArray[listIntArrStart+it*4 : listIntArrStart+it*4+4]))
+	}
+	list.couponCount = couponCount
+	return &list, nil
+}
diff --git a/hll/coupon_list_test.go b/hll/coupon_list_test.go
new file mode 100644
index 0000000..4c93f07
--- /dev/null
+++ b/hll/coupon_list_test.go
@@ -0,0 +1,113 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestCouponIterator(t *testing.T) {
+	lgK := 4
+	n := 7
+	sk, err := NewHllSketchDefault(lgK)
+	assert.NoError(t, err)
+	for i := 0; i < n; i++ {
+		sk.UpdateInt64(int64(i))
+	}
+
+	iter := sk.iterator()
+	for iter.nextAll() {
+		if iter.getIndex() >= (1 << lgK) {
+			t.Errorf("Slot %d is out of range", iter.getSlot())
+		}
+	}
+}
+
+func TestCouponDuplicatesAndMisc(t *testing.T) {
+	sk, err := NewHllSketchDefault(8)
+	assert.NoError(t, err)
+	for i := 1; i <= 7; i++ {
+		sk.UpdateInt64(int64(i))
+		sk.UpdateInt64(int64(i))
+	}
+	assert.Equal(t, sk.GetCurMode(), curMode_LIST)
+	est := sk.GetCompositeEstimate()
+	assert.InDelta(t, est, 7.0, 7*.01)
+	est = sk.GetHipEstimate()
+	assert.InDelta(t, est, 7.0, 7*.01)
+	sk.(*hllSketchImpl).putRebuildCurMinNumKxQFlag(false) //dummy
+
+	sk.UpdateInt64(8)
+	sk.UpdateInt64(8)
+	assert.Equal(t, sk.GetCurMode(), curMode_SET)
+	est = sk.GetCompositeEstimate()
+	assert.InDelta(t, est, 8.0, 8*.01)
+	est = sk.GetHipEstimate()
+	assert.InDelta(t, est, 8.0, 8*.01)
+
+	for i := 9; i <= 25; i++ {
+		sk.UpdateInt64(int64(i))
+		sk.UpdateInt64(int64(i))
+	}
+
+	assert.Equal(t, sk.GetCurMode(), curMode_HLL)
+	est = sk.GetCompositeEstimate()
+	assert.InDelta(t, est, 25.0, 25*.1)
+}
+
+func TestToCouponSliceDeserialize(t *testing.T) {
+	toCouponSliceDeserialize(t, 7)
+	toCouponSliceDeserialize(t, 21)
+
+}
+
+func toCouponSliceDeserialize(t *testing.T, lgK int) {
+	sk1, err := NewHllSketchDefault(lgK)
+	assert.NoError(t, err)
+
+	u := 7
+	if lgK >= 8 {
+		u = (1 << (lgK - 3)) * 3 / 4
+	}
+
+	for i := 0; i < u; i++ {
+		sk1.UpdateInt64(int64(i))
+	}
+
+	_, isCoupon := sk1.(*hllSketchImpl).sketch.(hllCoupon)
+	assert.True(t, isCoupon)
+
+	est1 := sk1.GetEstimate()
+	assert.InDelta(t, est1, float64(u), float64(u)*100.0e-6)
+
+	sl1, err := sk1.ToCompactSlice()
+	assert.NoError(t, err)
+	sk2, e := DeserializeHllSketch(sl1, true)
+	assert.NoError(t, e)
+	est2 := sk2.GetEstimate()
+	assert.Equal(t, est2, est1)
+
+	sl1, err = sk1.ToUpdatableSlice()
+	assert.NoError(t, err)
+	sk2, e = DeserializeHllSketch(sl1, true)
+	assert.NoError(t, e)
+	est2 = sk2.GetEstimate()
+	assert.Equal(t, est2, est1)
+}
diff --git a/hll/cross_counting_test.go b/hll/cross_counting_test.go
new file mode 100644
index 0000000..f1bffc8
--- /dev/null
+++ b/hll/cross_counting_test.go
@@ -0,0 +1,97 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestCrossCounting(t *testing.T) {
+	crossCountingCheck(t, 4, 100)
+	crossCountingCheck(t, 4, 10000)
+	crossCountingCheck(t, 12, 7)
+	crossCountingCheck(t, 12, 384)
+	crossCountingCheck(t, 12, 10000)
+}
+
+func crossCountingCheck(t *testing.T, lgK int, n int) {
+	sk4, err := buildSketch(lgK, n, TgtHllType_HLL_4)
+	assert.NoError(t, err)
+	s4csum := computeCheckSum(t, sk4)
+
+	sk6, err := buildSketch(lgK, n, TgtHllType_HLL_6)
+	assert.NoError(t, err)
+	s6csum := computeCheckSum(t, sk6)
+
+	assert.Equal(t, s6csum, s4csum)
+
+	sk8, err := buildSketch(lgK, n, TgtHllType_HLL_8)
+	assert.NoError(t, err)
+	s8csum := computeCheckSum(t, sk8)
+	assert.Equal(t, s8csum, s4csum)
+
+	// Conversions
+	sk6to4 := sk6.CopyAs(TgtHllType_HLL_4)
+	sk6to4csum := computeCheckSum(t, sk6to4)
+	assert.Equal(t, sk6to4csum, s4csum)
+
+	sk8to4 := sk8.CopyAs(TgtHllType_HLL_4)
+	sk8to4csum := computeCheckSum(t, sk8to4)
+	assert.Equal(t, sk8to4csum, s4csum)
+
+	sk4to6 := sk4.CopyAs(TgtHllType_HLL_6)
+	sk4to6csum := computeCheckSum(t, sk4to6)
+	assert.Equal(t, sk4to6csum, s4csum)
+
+	sk8to6 := sk8.CopyAs(TgtHllType_HLL_6)
+	sk8to6csum := computeCheckSum(t, sk8to6)
+	assert.Equal(t, sk8to6csum, s4csum)
+
+	sk4to8 := sk4.CopyAs(TgtHllType_HLL_8)
+	sk4to8csum := computeCheckSum(t, sk4to8)
+	assert.Equal(t, sk4to8csum, s4csum)
+
+	sk6to8 := sk6.CopyAs(TgtHllType_HLL_8)
+	sk6to8csum := computeCheckSum(t, sk6to8)
+	assert.Equal(t, sk6to8csum, s4csum)
+
+}
+
+func buildSketch(lgK int, n int, tgtHllType TgtHllType) (HllSketch, error) {
+	sketch, err := NewHllSketch(lgK, tgtHllType)
+	if err != nil {
+		return nil, err
+	}
+	for i := 0; i < n; i++ {
+		sketch.UpdateInt64(int64(i))
+	}
+	return sketch, nil
+}
+
+func computeCheckSum(t *testing.T, sketch HllSketch) int {
+	itr := sketch.iterator()
+	checksum := 0
+	for itr.nextAll() {
+		p := itr.getPair()
+		checksum += p
+		_ = itr.getKey()
+	}
+	return checksum
+}
diff --git a/hll/cubic_interpolation.go b/hll/cubic_interpolation.go
new file mode 100644
index 0000000..6065f0a
--- /dev/null
+++ b/hll/cubic_interpolation.go
@@ -0,0 +1,138 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import "fmt"
+
+// UsingXAndYTables returns the cubic interpolation using the X and Y tables.
+func usingXAndYTables(xArr []float64, yArr []float64, x float64) float64 {
+	if len(xArr) < 4 || len(xArr) != len(yArr) {
+		panic(fmt.Sprintf("X value out of range: %f", x))
+	}
+
+	if x == xArr[len(xArr)-1] {
+		return yArr[len(yArr)-1] // corer case
+	}
+
+	offset := findStraddle(xArr, x) //uses recursion
+	if (offset < 0) || (offset > (len(xArr) - 2)) {
+		panic(fmt.Sprintf("offset out of range: %d", offset))
+	}
+	if offset == 0 {
+		return interpolateUsingXAndYTables(xArr, yArr, offset, x) // corner case
+	}
+
+	if offset == len(xArr)-2 {
+		return interpolateUsingXAndYTables(xArr, yArr, offset-2, x) // corner case
+	}
+
+	return interpolateUsingXAndYTables(xArr, yArr, offset-1, x)
+}
+
+func interpolateUsingXAndYTables(xArr []float64, yArr []float64, offset int, x float64) float64 {
+	return cubicInterpolate(
+		xArr[offset], yArr[offset],
+		xArr[offset+1], yArr[offset+1],
+		xArr[offset+2], yArr[offset+2],
+		xArr[offset+3], yArr[offset+3],
+		x)
+}
+
+func usingXArrAndYStride(xArr []float64, yStride float64, x float64) float64 {
+	xArrLen := len(xArr)
+	xArrLenM1 := xArrLen - 1
+
+	if xArrLen < 4 || x < xArr[0] || x > xArr[xArrLenM1] {
+		panic(fmt.Sprintf("X value out of range: %f", x))
+	}
+	if x == xArr[xArrLenM1] {
+		return yStride * float64(xArrLenM1) // corner case
+	}
+	offset := findStraddle(xArr, x) //uses recursion
+	xArrLenM2 := xArrLen - 2
+	if (offset < 0) || (offset > xArrLenM2) {
+		panic(fmt.Sprintf("offset out of range: %d", offset))
+	}
+	if offset == 0 {
+		return interpolateUsingXArrAndYStride(xArr, yStride, offset, x) // corner case
+	}
+	if offset == xArrLenM2 {
+		return interpolateUsingXArrAndYStride(xArr, yStride, offset-2, x) // corner case
+	}
+	return interpolateUsingXArrAndYStride(xArr, yStride, offset-1, x)
+}
+
+// interpolateUsingXArrAndYStride interpolates using the X array and the Y stride.
+func interpolateUsingXArrAndYStride(xArr []float64, yStride float64, offset int, x float64) float64 {
+	return cubicInterpolate(xArr[offset+0], yStride*float64(offset+0),
+		xArr[offset+1], yStride*float64(offset+1),
+		xArr[offset+2], yStride*float64(offset+2),
+		xArr[offset+3], yStride*float64(offset+3), x)
+}
+
+// cubicInterpolate interpolates using the cubic curve that passes through the four given points, using the
+// Lagrange interpolation formula.
+func cubicInterpolate(x0 float64, y0 float64, x1 float64, y1 float64, x2 float64, y2 float64, x3 float64, y3 float64, x float64) float64 {
+	l0Numer := (x - x1) * (x - x2) * (x - x3)
+	l1Numer := (x - x0) * (x - x2) * (x - x3)
+	l2Numer := (x - x0) * (x - x1) * (x - x3)
+	l3Numer := (x - x0) * (x - x1) * (x - x2)
+
+	l0Denom := (x0 - x1) * (x0 - x2) * (x0 - x3)
+	l1Denom := (x1 - x0) * (x1 - x2) * (x1 - x3)
+	l2Denom := (x2 - x0) * (x2 - x1) * (x2 - x3)
+	l3Denom := (x3 - x0) * (x3 - x1) * (x3 - x2)
+
+	term0 := (y0 * l0Numer) / l0Denom
+	term1 := (y1 * l1Numer) / l1Denom
+	term2 := (y2 * l2Numer) / l2Denom
+	term3 := (y3 * l3Numer) / l3Denom
+
+	return term0 + term1 + term2 + term3
+}
+
+// findStraddle returns the index of the largest value in the array that is less than or equal to the given value.
+func findStraddle(xArr []float64, x float64) int {
+	if len(xArr) < 2 || x < xArr[0] || x > xArr[len(xArr)-1] {
+		panic(fmt.Sprintf("X value out of range: %f", x))
+	}
+	return recursiveFindStraddle(xArr, 0, len(xArr)-1, x)
+}
+
+// recursiveFindStraddle returns the index of the largest value in the array that is less than or equal to the given value.
+func recursiveFindStraddle(xArr []float64, left int, right int, x float64) int {
+	if left >= right {
+		panic(fmt.Sprintf("left >= right: %d >= %d", left, right))
+	}
+
+	if xArr[left] > x || x >= xArr[right] {
+		panic(fmt.Sprintf("X value out of range: %f", x))
+	}
+
+	if left+1 == right {
+		return left
+	}
+
+	middle := left + ((right - left) / 2)
+
+	if xArr[middle] <= x {
+		return recursiveFindStraddle(xArr, middle, right, x)
+	} else {
+		return recursiveFindStraddle(xArr, left, middle, x)
+	}
+}
diff --git a/hll/cubic_interpolation_test.go b/hll/cubic_interpolation_test.go
new file mode 100644
index 0000000..45de93f
--- /dev/null
+++ b/hll/cubic_interpolation_test.go
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestInterpolationExceptions(t *testing.T) {
+	assert.Panics(t, func() { usingXAndYTables(couponMappingXArr, couponMappingYArr, -1) }, "X value out of range: -1.000000")
+
+	assert.Panics(t, func() { usingXAndYTables(couponMappingXArr, couponMappingYArr, 11000000.0) }, "X value out of range: 11000000.000000")
+}
+
+func TestCornerCases(t *testing.T) {
+	leng := len(couponMappingXArr)
+	x := couponMappingXArr[leng-1]
+	y := usingXAndYTables(couponMappingXArr, couponMappingYArr, x)
+	yExp := couponMappingYArr[leng-1]
+	assert.Equal(t, y, yExp)
+}
diff --git a/hll/harmonic_numbers.go b/hll/harmonic_numbers.go
new file mode 100644
index 0000000..0dbc554
--- /dev/null
+++ b/hll/harmonic_numbers.go
@@ -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.
+ */
+
+package hll
+
+import "math"
+
+const (
+	numExactHarmonicNumbers = 25
+	eulerMascheroniConstant = 0.577215664901532860606512090082
+)
+
+var (
+	// tableOfExactHarmonicNumbers is a table of the first 25 harmonic numbers.
+	tableOfExactHarmonicNumbers = []float64{
+		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
+	}
+)
+
+// getBitMapEstimate is the estimator you would use for flat bit map random accessed, similar to a Bloom filter.
+// bitVectorLength the length of the bit vector in bits. Must be > 0.
+// numBitsSet the number of bits set in this bit vector. Must be >= 0 and <= bitVectorLength.
+func getBitMapEstimate(bitVectorLength int, numBitsSet int) float64 {
+	return float64(bitVectorLength) * (harmonicNumber(bitVectorLength) - harmonicNumber(bitVectorLength-numBitsSet))
+}
+
+// harmonicNumber returns the nth harmonic number.
+func harmonicNumber(n int) float64 {
+	if n < numExactHarmonicNumbers {
+		return tableOfExactHarmonicNumbers[n]
+	} else {
+		x := float64(n)
+		invSq := 1.0 / (x * x)
+		sum := math.Log(x) + eulerMascheroniConstant + (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 */
+		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
+	}
+}
diff --git a/hll/hll_4array.go b/hll/hll_4array.go
new file mode 100644
index 0000000..324c42f
--- /dev/null
+++ b/hll/hll_4array.go
@@ -0,0 +1,203 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"fmt"
+)
+
+// hll4ArrayImpl Uses 4 bits per slot in a packed byte array
+type hll4ArrayImpl struct {
+	hllArrayImpl
+}
+
+func (h *hll4ArrayImpl) getSlotValue(slotNo int) int {
+	nib := h.getNibble(slotNo)
+	if nib == auxToken {
+		auxHashMap := h.getAuxHashMap()
+		return auxHashMap.mustFindValueFor(slotNo)
+	} else {
+		return nib + h.curMin
+	}
+}
+
+type hll4Iterator struct {
+	hllPairIterator
+	hll *hll4ArrayImpl
+}
+
+func (h *hll4ArrayImpl) iterator() pairIterator {
+	a := newHll4Iterator(1<<h.lgConfigK, h)
+	return &a
+}
+
+func (h *hll4ArrayImpl) ToCompactSlice() ([]byte, error) {
+	return toHllByteArr(h, true)
+}
+
+func (h *hll4ArrayImpl) ToUpdatableSlice() ([]byte, error) {
+	return toHllByteArr(h, false)
+}
+
+func (h *hll4ArrayImpl) GetUpdatableSerializationBytes() int {
+	auxHashMap := h.getAuxHashMap()
+	auxBytes := 0
+	if auxHashMap == nil {
+		auxBytes = 4 << lgAuxArrInts[h.lgConfigK]
+	} else {
+		auxBytes = 4 << auxHashMap.getLgAuxArrInts()
+	}
+	return hllByteArrStart + h.getHllByteArrBytes() + auxBytes
+}
+
+func (h *hll4ArrayImpl) copyAs(tgtHllType TgtHllType) hllSketchBase {
+	if tgtHllType == h.tgtHllType {
+		return h.copy()
+	}
+	if tgtHllType == TgtHllType_HLL_6 {
+		return convertToHll6(h)
+	}
+	if tgtHllType == TgtHllType_HLL_8 {
+		return convertToHll8(h)
+	}
+	panic(fmt.Sprintf("Cannot convert to TgtHllType id: %d ", int(tgtHllType)))
+}
+
+func (h *hll4ArrayImpl) copy() hllSketchBase {
+	return &hll4ArrayImpl{
+		hllArrayImpl: h.copyCommon(),
+	}
+}
+
+// newHll4Array returns a new Hll4Array.
+func newHll4Array(lgConfigK int) hllArray {
+	return &hll4ArrayImpl{
+		hllArrayImpl: hllArrayImpl{
+			hllSketchConfig: hllSketchConfig{
+				lgConfigK:  lgConfigK,
+				tgtHllType: TgtHllType_HLL_4,
+				curMode:    curMode_HLL,
+			},
+			curMin:      0,
+			numAtCurMin: 1 << lgConfigK,
+			hipAccum:    0,
+			kxq0:        float64(uint64(1 << lgConfigK)),
+			kxq1:        0,
+			hllByteArr:  make([]byte, 1<<(lgConfigK-1)),
+			auxStart:    hllByteArrStart + 1<<(lgConfigK-1),
+		},
+	}
+}
+
+// deserializeHll4 returns a new Hll4Array from the given byte array.
+func deserializeHll4(byteArray []byte) hllArray {
+	lgConfigK := extractLgK(byteArray)
+	hll4 := newHll4Array(lgConfigK)
+	hll4.extractCommonHll(byteArray)
+
+	auxStart := hll4.getAuxStart()
+	auxCount := extractAuxCount(byteArray)
+	compact := extractCompactFlag(byteArray)
+
+	if auxCount > 0 {
+		auxHashMap := deserializeAuxHashMap(byteArray, auxStart, lgConfigK, auxCount, compact)
+		hll4.putAuxHashMap(auxHashMap, false)
+	}
+
+	return hll4
+}
+
+func convertToHll4(srcAbsHllArr hllArray) hllSketchBase {
+	lgConfigK := srcAbsHllArr.GetLgConfigK()
+	hll4Array := newHll4Array(lgConfigK)
+	hll4Array.putOutOfOrder(srcAbsHllArr.isOutOfOrder())
+
+	// 1st pass: compute starting curMin and numAtCurMin:
+	pair := curMinAndNum(srcAbsHllArr)
+	curMin := getPairValue(pair)
+	numAtCurMin := getPairLow26(pair)
+
+	// 2nd pass: Must know curMin to create auxHashMap.
+	// Populate KxQ registers, build auxHashMap if needed
+	srcItr := srcAbsHllArr.iterator()
+	auxHashMap := hll4Array.getAuxHashMap() //may be null
+	for srcItr.nextValid() {
+		slotNo := srcItr.getIndex()
+		actualValue := srcItr.getValue()
+		hll4Array.hipAndKxQIncrementalUpdate(0, actualValue)
+		if actualValue >= (curMin + 15) {
+			hll4Array.putNibble(slotNo, auxToken)
+			if auxHashMap == nil {
+				auxHashMap = newAuxHashMap(lgAuxArrInts[lgConfigK], lgConfigK)
+				hll4Array.putAuxHashMap(auxHashMap, false)
+			}
+			auxHashMap.mustAdd(slotNo, actualValue)
+		} else {
+			hll4Array.putNibble(slotNo, byte(actualValue-curMin))
+		}
+	}
+	hll4Array.putCurMin(curMin)
+	hll4Array.putNumAtCurMin(numAtCurMin)
+	hll4Array.putHipAccum(srcAbsHllArr.getHipAccum()) //intentional overwrite
+	hll4Array.putRebuildCurMinNumKxQFlag(false)
+	return hll4Array
+}
+
+// couponUpdate updates the Hll4Array with the given coupon and returns the updated Hll4Array.
+func (h *hll4ArrayImpl) couponUpdate(coupon int) hllSketchBase {
+	newValue := coupon >> keyBits26
+	configKmask := (1 << h.lgConfigK) - 1
+	slotNo := coupon & configKmask
+	internalHll4Update(h, slotNo, newValue)
+	return h
+}
+
+func curMinAndNum(absHllArr hllArray) int {
+	curMin := 64
+	numAtCurMin := 0
+	itr := absHllArr.iterator()
+	for itr.nextAll() {
+		v := itr.getValue()
+		if v > curMin {
+			continue
+		}
+		if v < curMin {
+			curMin = v
+			numAtCurMin = 1
+		} else {
+			numAtCurMin++
+		}
+	}
+	return pair(numAtCurMin, curMin)
+}
+
+func newHll4Iterator(lengthPairs int, hll *hll4ArrayImpl) hll4Iterator {
+	return hll4Iterator{
+		hllPairIterator: newHllPairIterator(lengthPairs),
+		hll:             hll,
+	}
+}
+
+func (itr *hll4Iterator) getValue() int {
+	return itr.hll.getSlotValue(itr.getIndex())
+}
+
+func (itr *hll4Iterator) getPair() int {
+	v := itr.getValue()
+	return pair(itr.index, v)
+}
diff --git a/hll/hll_4update.go b/hll/hll_4update.go
new file mode 100644
index 0000000..b5d0c54
--- /dev/null
+++ b/hll/hll_4update.go
@@ -0,0 +1,197 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"fmt"
+)
+
+// internalHll4Update is the internal update method for Hll4Array.
+func internalHll4Update(h *hll4ArrayImpl, slotNo int, newValue int) {
+	curMin := h.curMin
+	rawStoredOldNibble := h.getNibble(slotNo)   // could be 0
+	lb0nOldValue := rawStoredOldNibble + curMin // provable lower bound, could be 0
+
+	if newValue <= lb0nOldValue {
+		return
+	}
+
+	var (
+		actualOldValue  int
+		shiftedNewValue int //value - curMin
+	)
+	// Based on whether we have an AUX_TOKEN and whether the shiftedNewValue is greater than
+	// AUX_TOKEN, we have four cases for how to actually modify the data structure:
+	// 1. (shiftedNewValue >= AUX_TOKEN) && (rawStoredOldNibble = AUX_TOKEN) //881:
+	//    The byte array already contains aux token
+	//    This is the case where old and new values are both exceptions.
+	//    Therefore, the 4-bit array already is AUX_TOKEN. Only need to update auxMap
+	// 2. (shiftedNewValue < AUX_TOKEN) && (rawStoredOldNibble = AUX_TOKEN) //885
+	//    This is the (hypothetical) case where old value is an exception and the new one is not,
+	//    which is impossible given that curMin has not changed here and the newValue > oldValue.
+	// 3. (shiftedNewValue >= AUX_TOKEN) && (rawStoredOldNibble < AUX_TOKEN) //892
+	//    This is the case where the old value is not an exception and the new value is.
+	//    Therefore, the AUX_TOKEN must be stored in the 4-bit array and the new value
+	//    added to the exception table.
+	// 4. (shiftedNewValue < AUX_TOKEN) && (rawStoredOldNibble < AUX_TOKEN) //897
+	//    This is the case where neither the old value nor the new value is an exception.
+	//    Therefore, we just overwrite the 4-bit array with the shifted new value.
+
+	if rawStoredOldNibble == auxToken { //846 Note: This is rare and really hard to test!
+		if h.auxHashMap == nil {
+			panic("auxHashMap must already exist")
+		}
+		actualOldValue = h.auxHashMap.mustFindValueFor(slotNo)
+		if newValue <= actualOldValue {
+			return
+		}
+		// We know that the array will be changed, but we haven't actually updated yet.
+		h.hipAndKxQIncrementalUpdate(actualOldValue, newValue)
+		shiftedNewValue = newValue - curMin
+		if shiftedNewValue < 0 {
+			panic("shifedNewValue < 0")
+		}
+		if shiftedNewValue >= auxToken { //CASE 1:
+			h.auxHashMap.mustReplace(slotNo, newValue)
+		} //else                         //CASE 2: impossible
+	} else { //rawStoredOldNibble < AUX_TOKEN
+		actualOldValue = lb0nOldValue
+		// We know that the array will be changed, but we haven't actually updated yet.
+		h.hipAndKxQIncrementalUpdate(actualOldValue, newValue)
+		shiftedNewValue = newValue - curMin
+		if shiftedNewValue < 0 {
+			panic("shifedNewValue < 0")
+		}
+		if shiftedNewValue >= auxToken { //CASE 3: //892
+			h.putNibble(slotNo, auxToken)
+			if h.auxHashMap == nil {
+				h.auxHashMap = h.getNewAuxHashMap()
+			}
+			h.auxHashMap.mustAdd(slotNo, newValue)
+		} else { // CASE 4: //897
+			h.putNibble(slotNo, byte(shiftedNewValue))
+		}
+	}
+
+	// We just changed the HLL array, so it might be time to change curMin.
+	if actualOldValue == curMin {
+		if h.numAtCurMin < 1 {
+			panic("h.numAtCurMin < 1")
+		}
+		h.numAtCurMin--
+		for h.numAtCurMin == 0 {
+			// Increases curMin by 1, and builds a new aux table,
+			// shifts values in 4-bit table, and recounts curMin.
+			shiftToBiggerCurMin(h)
+		}
+	}
+}
+
+// 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 nibbles > 0 and <= 15.
+// An auxHashMap must exist if any values in the current hllByteArray are already 15.
+func shiftToBiggerCurMin(h *hll4ArrayImpl) {
+	oldCurMin := h.curMin
+	newCurMin := oldCurMin + 1
+	lgConfigK := h.lgConfigK
+	configK := 1 << lgConfigK
+	configKmask := configK - 1
+
+	numAtNewCurMin := 0
+	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 i := 0; i < configK; i++ { //724
+		oldStoredNibble := uint64(h.getNibble(i))
+		if oldStoredNibble == 0 {
+			panic("array slots cannot be 0 at this point")
+		}
+		if oldStoredNibble < auxToken {
+			oldStoredNibble--
+			h.putNibble(i, byte(oldStoredNibble))
+			if oldStoredNibble == 0 {
+				numAtNewCurMin++
+			}
+		} else { //oldStoredNibble == AUX_TOKEN
+			numAuxTokens++
+			if h.auxHashMap == nil {
+				panic("auxHashMap cannot be nil at this point")
+			}
+		}
+	}
+	// If old auxHashMap exists, walk through it updating some slots and build a new auxHashMap
+	// if needed.
+	var newAuxMap auxHashMap
+	oldAuxMap := h.auxHashMap
+
+	if oldAuxMap != nil {
+		var (
+			slotNum       int
+			oldActualVal  int
+			newShiftedVal int
+		)
+
+		itr := oldAuxMap.iterator()
+		for itr.nextValid() {
+			slotNum = itr.getKey() & configKmask
+			oldActualVal = itr.getValue()
+
+			newShiftedVal = oldActualVal - newCurMin
+			if newShiftedVal < 0 {
+				panic("newShiftedVal < 0")
+			}
+			if h.getNibble(slotNum) != auxToken {
+				panic(fmt.Sprintf("Array slot != AUX_TOKEN %d", h.getNibble(slotNum)))
+			}
+			if newShiftedVal < auxToken {
+				if newShiftedVal != 14 {
+					panic("newShiftedVal != 14")
+				}
+				// 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).
+				h.putNibble(slotNum, byte(newShiftedVal))
+				numAuxTokens--
+			} else { // newShiftedVal >= AUX_TOKEN
+				// the former exception remains an exception, so must be added to the newAuxMap
+				if newAuxMap == nil {
+					newAuxMap = h.getNewAuxHashMap()
+				}
+				newAuxMap.mustAdd(slotNum, oldActualVal)
+			}
+		}
+	} else {
+		if numAuxTokens != 0 {
+			panic("numAuxTokens != 0")
+		}
+	}
+	if newAuxMap != nil {
+		if newAuxMap.getAuxCount() != numAuxTokens {
+			panic("newAuxMap.getAuxCount() != numAuxTokens")
+		}
+	}
+	h.auxHashMap = newAuxMap
+	h.curMin = newCurMin
+	h.numAtCurMin = numAtNewCurMin
+}
diff --git a/hll/hll_6array.go b/hll/hll_6array.go
new file mode 100644
index 0000000..657d00d
--- /dev/null
+++ b/hll/hll_6array.go
@@ -0,0 +1,199 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"fmt"
+
+	"github.com/apache/datasketches-go/common"
+)
+
+// hll6ArrayImpl Uses 6 bits per slot in a packed byte array
+type hll6ArrayImpl struct {
+	hllArrayImpl
+}
+
+type hll6Iterator struct {
+	hllPairIterator
+	hll       *hll6ArrayImpl
+	bitOffset int
+}
+
+func (h *hll6ArrayImpl) iterator() pairIterator {
+	a := newHll6Iterator(1<<h.lgConfigK, h)
+	return &a
+}
+
+func (h *hll6ArrayImpl) copyAs(tgtHllType TgtHllType) hllSketchBase {
+	if tgtHllType == h.tgtHllType {
+		return h.copy()
+	}
+	if tgtHllType == TgtHllType_HLL_4 {
+		return convertToHll4(h)
+	}
+	if tgtHllType == TgtHllType_HLL_8 {
+		return convertToHll8(h)
+	}
+	panic(fmt.Sprintf("Cannot convert to TgtHllType id: %d", int(tgtHllType)))
+}
+
+func (h *hll6ArrayImpl) copy() hllSketchBase {
+	return &hll6ArrayImpl{
+		hllArrayImpl: h.copyCommon(),
+	}
+}
+
+func (h *hll6ArrayImpl) ToCompactSlice() ([]byte, error) {
+	return h.ToUpdatableSlice()
+}
+
+func (h *hll6ArrayImpl) ToUpdatableSlice() ([]byte, error) {
+	return toHllByteArr(h, false)
+}
+
+// newHll6Array returns a new Hll4Array.
+func newHll6Array(lgConfigK int) hllArray {
+	return &hll6ArrayImpl{
+		hllArrayImpl: hllArrayImpl{
+			hllSketchConfig: hllSketchConfig{
+				lgConfigK:  lgConfigK,
+				tgtHllType: TgtHllType_HLL_6,
+				curMode:    curMode_HLL,
+			},
+			curMin:      0,
+			numAtCurMin: 1 << lgConfigK,
+			hipAccum:    0,
+			kxq0:        float64(uint64(1 << lgConfigK)),
+			kxq1:        0,
+			hllByteArr:  make([]byte, (((1<<lgConfigK)*3)>>2)+1),
+			auxStart:    hllByteArrStart + 1<<(lgConfigK-1),
+		},
+	}
+}
+
+// deserializeHll6 returns a new Hll6Array from the given byte array.
+func deserializeHll6(byteArray []byte) hllArray {
+	lgConfigK := extractLgK(byteArray)
+	hll6 := newHll6Array(lgConfigK)
+	hll6.extractCommonHll(byteArray)
+	return hll6
+}
+
+func (h *hll6ArrayImpl) couponUpdate(coupon int) hllSketchBase {
+	newValue := coupon >> keyBits26
+	configKmask := (1 << h.lgConfigK) - 1
+	slotNo := coupon & configKmask
+	h.updateSlotWithKxQ(slotNo, newValue)
+	return h
+}
+
+func (h *hll6ArrayImpl) updateSlotWithKxQ(slotNo int, newValue int) {
+	oldValue := h.getSlotValue(slotNo)
+	if newValue > oldValue {
+		put6Bit(h.hllByteArr, 0, slotNo, newValue)
+		h.hipAndKxQIncrementalUpdate(oldValue, newValue)
+		if oldValue == 0 {
+			h.numAtCurMin-- //interpret numAtCurMin as num Zeros
+			if h.numAtCurMin < 0 {
+				panic("numAtCurMin < 0")
+			}
+		}
+	}
+}
+
+func (h *hll6ArrayImpl) getSlotValue(slotNo int) int {
+	return get6Bit(h.hllByteArr, 0, slotNo)
+}
+
+func get6Bit(arr []byte, offsetBytes int, slotNo int) int {
+	startBit := slotNo * 6
+	shift := startBit & 0x7
+	byteIdx := (startBit >> 3) + offsetBytes
+	return (common.GetShortLE(arr, byteIdx) >> shift) & 0x3F
+}
+
+func put6Bit(arr []byte, offsetBytes int, slotNo int, newValue int) {
+	startBit := slotNo * 6
+	shift := startBit & 0x7
+	byteIdx := (startBit >> 3) + offsetBytes
+	valShifted := (newValue & 0x3F) << shift
+	curMasked := common.GetShortLE(arr, byteIdx) & (^(valMask6 << shift))
+	insert := curMasked | valShifted
+	common.PutShortLE(arr, byteIdx, insert)
+}
+
+func convertToHll6(srcAbsHllArr hllArray) hllSketchBase {
+	lgConfigK := srcAbsHllArr.GetLgConfigK()
+	hll6Array := newHll6Array(lgConfigK)
+	hll6Array.putOutOfOrder(srcAbsHllArr.isOutOfOrder())
+	numZeros := 1 << lgConfigK
+	srcItr := srcAbsHllArr.iterator()
+	for srcItr.nextAll() {
+		v := srcItr.getValue()
+		if v != empty {
+			numZeros--
+			p := srcItr.getPair()
+			hll6Array.couponUpdate(p) //couponUpdate creates KxQ registers
+		}
+	}
+	hll6Array.putNumAtCurMin(numZeros)
+	hll6Array.putHipAccum(srcAbsHllArr.getHipAccum()) //intentional overwrite
+	hll6Array.putRebuildCurMinNumKxQFlag(false)
+	return hll6Array
+}
+
+func newHll6Iterator(lengthPairs int, hll *hll6ArrayImpl) hll6Iterator {
+	return hll6Iterator{
+		hllPairIterator: newHllPairIterator(lengthPairs),
+		hll:             hll,
+		bitOffset:       -6,
+	}
+}
+
+func (h *hll6Iterator) nextAll() bool {
+	h.index++
+	if h.index >= h.lengthPairs {
+		return false
+	}
+	h.bitOffset += 6
+	return true
+}
+
+func (h *hll6Iterator) nextValid() bool {
+	for h.index+1 < h.lengthPairs {
+		h.index++
+		h.bitOffset += 6
+		tmp := common.GetShortLE(h.hll.hllByteArr, h.bitOffset/8)
+		h.value = (tmp >> ((h.bitOffset % 8) & 0x7)) & valMask6
+		if h.value != empty {
+			return true
+		}
+	}
+	return false
+}
+
+func (h *hll6Iterator) getValue() int {
+	tmp := common.GetShortLE(h.hll.hllByteArr, h.bitOffset/8)
+	shift := (h.bitOffset % 8) & 0x7
+	return (tmp >> shift) & valMask6
+}
+
+func (h *hll6Iterator) getPair() int {
+	v := h.getValue()
+	return pair(h.index, v)
+}
diff --git a/hll/hll_8array.go b/hll/hll_8array.go
new file mode 100644
index 0000000..c6f69f4
--- /dev/null
+++ b/hll/hll_8array.go
@@ -0,0 +1,170 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"fmt"
+)
+
+// hll8ArrayImpl Uses 6 bits per slot in a packed byte array
+type hll8ArrayImpl struct {
+	hllArrayImpl
+}
+
+type hll8Iterator struct {
+	hllPairIterator
+	hll *hll8ArrayImpl
+}
+
+func (h *hll8ArrayImpl) iterator() pairIterator {
+	a := newHll8Iterator(1<<h.lgConfigK, h)
+	return &a
+}
+
+func (h *hll8ArrayImpl) copyAs(tgtHllType TgtHllType) hllSketchBase {
+	if tgtHllType == h.tgtHllType {
+		return h.copy()
+	}
+	if tgtHllType == TgtHllType_HLL_4 {
+		return convertToHll4(h)
+	}
+	if tgtHllType == TgtHllType_HLL_6 {
+		return convertToHll6(h)
+	}
+	panic(fmt.Sprintf("Cannot convert to TgtHllType id: %d", int(tgtHllType)))
+}
+
+func (h *hll8ArrayImpl) copy() hllSketchBase {
+	return &hll8ArrayImpl{
+		hllArrayImpl: h.copyCommon(),
+	}
+}
+
+func (h *hll8ArrayImpl) ToCompactSlice() ([]byte, error) {
+	return h.ToUpdatableSlice()
+}
+
+func (h *hll8ArrayImpl) ToUpdatableSlice() ([]byte, error) {
+	return toHllByteArr(h, false)
+}
+
+// newHll8Array returns a new Hll8Array.
+func newHll8Array(lgConfigK int) hllArray {
+	return &hll8ArrayImpl{
+		hllArrayImpl: hllArrayImpl{
+			hllSketchConfig: hllSketchConfig{
+				lgConfigK:  lgConfigK,
+				tgtHllType: TgtHllType_HLL_8,
+				curMode:    curMode_HLL,
+			},
+			curMin:      0,
+			numAtCurMin: 1 << lgConfigK,
+			hipAccum:    0,
+			kxq0:        float64(uint64(1 << lgConfigK)),
+			kxq1:        0,
+			hllByteArr:  make([]byte, 1<<lgConfigK),
+			auxStart:    hllByteArrStart + 1<<(lgConfigK-1),
+		},
+	}
+}
+
+// deserializeHll8 returns a new Hll8Array from the given byte array.
+func deserializeHll8(byteArray []byte) hllArray {
+	lgConfigK := extractLgK(byteArray)
+	hll8 := newHll8Array(lgConfigK)
+	hll8.extractCommonHll(byteArray)
+	return hll8
+}
+
+func convertToHll8(srcAbsHllArr hllArray) hllSketchBase {
+	lgConfigK := srcAbsHllArr.GetLgConfigK()
+	hll8Array := newHll8Array(lgConfigK)
+	hll8Array.putOutOfOrder(srcAbsHllArr.isOutOfOrder())
+	numZeros := 1 << lgConfigK
+	itr := srcAbsHllArr.iterator()
+	for itr.nextAll() {
+		v := itr.getValue()
+		if v != empty {
+			numZeros--
+			p := itr.getPair()
+			hll8Array.couponUpdate(p) //creates KxQ registers
+		}
+	}
+	hll8Array.putNumAtCurMin(numZeros)
+	hll8Array.putHipAccum(srcAbsHllArr.getHipAccum()) //intentional overwrite
+	hll8Array.putRebuildCurMinNumKxQFlag(false)
+	return hll8Array
+}
+
+func (h *hll8ArrayImpl) couponUpdate(coupon int) hllSketchBase {
+	newValue := coupon >> keyBits26
+	configKmask := (1 << h.lgConfigK) - 1
+	slotNo := coupon & configKmask
+	h.updateSlotWithKxQ(slotNo, newValue)
+	return h
+}
+
+func (h *hll8ArrayImpl) updateSlotWithKxQ(slotNo int, newValue int) {
+	oldValue := h.getSlotValue(slotNo)
+	if newValue > oldValue {
+		h.hllByteArr[slotNo] = byte(newValue & valMask6)
+		h.hipAndKxQIncrementalUpdate(oldValue, newValue)
+		if oldValue == 0 {
+			h.numAtCurMin-- //interpret numAtCurMin as num Zeros
+			if h.numAtCurMin < 0 {
+				panic("numAtCurMin < 0")
+			}
+		}
+	}
+}
+
+func (h *hll8ArrayImpl) updateSlotNoKxQ(slotNo int, newValue int) {
+	oldValue := h.getSlotValue(slotNo)
+	h.hllByteArr[slotNo] = byte(max(newValue, oldValue))
+}
+
+func (h *hll8ArrayImpl) getSlotValue(slotNo int) int {
+	return int(h.hllByteArr[slotNo] & valMask6)
+}
+
+func newHll8Iterator(lengthPairs int, hll *hll8ArrayImpl) hll8Iterator {
+	return hll8Iterator{
+		hllPairIterator: newHllPairIterator(lengthPairs),
+		hll:             hll,
+	}
+}
+
+func (h *hll8Iterator) nextValid() bool {
+	for h.index+1 < h.lengthPairs {
+		h.index++
+		h.value = int(h.hll.hllByteArr[h.index]) & valMask6
+		if h.value != empty {
+			return true
+		}
+	}
+	return false
+}
+
+func (h *hll8Iterator) getValue() int {
+	return int(h.hll.hllByteArr[h.index]) & valMask6
+}
+
+func (h *hll8Iterator) getPair() int {
+	v := h.getValue()
+	return pair(h.index, v)
+}
diff --git a/hll/hll_array.go b/hll/hll_array.go
new file mode 100644
index 0000000..b72ba5e
--- /dev/null
+++ b/hll/hll_array.go
@@ -0,0 +1,303 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"fmt"
+	"github.com/apache/datasketches-go/common"
+)
+
+type hllArray interface {
+	hllSketchBase
+
+	getAuxHashMap() auxHashMap
+	getAuxStart() int
+	getCurMin() int
+	getHipAccum() float64
+	getHllByteArr() []byte
+	getHllByteArrBytes() int
+	getKxQ0() float64
+	getKxQ1() float64
+	getNumAtCurMin() int
+
+	putAuxHashMap(auxHashMap auxHashMap, compact bool)
+	putCurMin(curMin int)
+	putHipAccum(hipAccum float64)
+	putKxQ0(kxq0 float64)
+	putKxQ1(kxq1 float64)
+	putNibble(slotNo int, value byte)
+	putNumAtCurMin(numAtCurMin int)
+	putOutOfOrder(oooFlag bool)
+
+	extractCommonHll(byteArr []byte)
+	hipAndKxQIncrementalUpdate(oldValue int, newValue int)
+}
+
+type hllArrayImpl struct {
+	hllSketchConfig
+	oooFrag             bool //Out-Of-Order Flag
+	rebuildCurMinNumKxQ bool
+	curMin              int //always zero for Hll6 and Hll8, only used by Hll4Array
+	numAtCurMin         int //# of values at curMin. If curMin = 0, it is # of zeros
+	hipAccum            float64
+	kxq0                float64
+	kxq1                float64
+
+	hllByteArr []byte
+
+	auxHashMap auxHashMap
+	auxStart   int //used for direct HLL4
+}
+
+// newHllArray returns a new hllArray of the given lgConfigK and tgtHllType.
+func newHllArray(lgConfigK int, tgtHllType TgtHllType) (hllArray, error) {
+	switch tgtHllType {
+	case TgtHllType_HLL_4:
+		return newHll4Array(lgConfigK), nil
+	case TgtHllType_HLL_6:
+		return newHll6Array(lgConfigK), nil
+	case TgtHllType_HLL_8:
+		return newHll8Array(lgConfigK), nil
+	}
+	return nil, fmt.Errorf("unknown TgtHllType")
+}
+
+func (a *hllArrayImpl) getPreInts() int {
+	return hllPreInts
+}
+
+func (a *hllArrayImpl) IsEmpty() bool {
+	return false
+}
+
+func (a *hllArrayImpl) GetEstimate() float64 {
+	if a.oooFrag {
+		return a.GetCompositeEstimate()
+	}
+	return a.hipAccum
+}
+
+// GetCompositeEstimate getCompositeEstimate returns the composite estimate.
+func (a *hllArrayImpl) GetCompositeEstimate() float64 {
+	return hllCompositeEstimate(a)
+}
+
+func (a *hllArrayImpl) GetHipEstimate() float64 {
+	return a.hipAccum
+}
+
+func (a *hllArrayImpl) getMemDataStart() int {
+	return hllByteArrStart
+}
+
+func (a *hllArrayImpl) GetUpperBound(numStdDev int) (float64, error) {
+	err := checkNumStdDev(numStdDev)
+	if err != nil {
+		return 0, err
+	}
+	return hllUpperBound(a, numStdDev)
+}
+
+func (a *hllArrayImpl) GetLowerBound(numStdDev int) (float64, error) {
+	err := checkNumStdDev(numStdDev)
+	if err != nil {
+		return 0, err
+	}
+	return hllLowerBound(a, numStdDev)
+}
+
+func (a *hllArrayImpl) GetUpdatableSerializationBytes() int {
+	return hllByteArrStart + a.getHllByteArrBytes()
+}
+
+func (a *hllArrayImpl) getCurMin() int {
+	return a.curMin
+}
+
+func (a *hllArrayImpl) getNumAtCurMin() int {
+	return a.numAtCurMin
+}
+
+func (a *hllArrayImpl) getKxQ1() float64 {
+	return a.kxq1
+}
+
+func (a *hllArrayImpl) getKxQ0() float64 {
+	return a.kxq0
+}
+
+func (a *hllArrayImpl) getHllByteArrBytes() int {
+	return len(a.hllByteArr)
+}
+
+func (a *hllArrayImpl) getHllByteArr() []byte {
+	return a.hllByteArr
+}
+
+// putHipAccum sets the HipAccum.
+func (a *hllArrayImpl) putHipAccum(hipAccum float64) {
+	a.hipAccum = hipAccum
+}
+
+// getHipAccum sets the HipAccum.
+func (a *hllArrayImpl) getHipAccum() float64 {
+	return a.hipAccum
+}
+
+// addToHipAccum adds the given value to the HipAccum.
+func (a *hllArrayImpl) addToHipAccum(value float64) {
+	a.hipAccum += value
+}
+
+// putOutOfOrder sets the Out-Of-Order Flag
+func (a *hllArrayImpl) putOutOfOrder(oooFlag bool) {
+	if oooFlag {
+		a.putHipAccum(0)
+	}
+	a.oooFrag = oooFlag
+}
+
+func (a *hllArrayImpl) isOutOfOrder() bool {
+	return a.oooFrag
+}
+
+func (a *hllArrayImpl) putAuxHashMap(auxHashMap auxHashMap, _ bool) {
+	a.auxHashMap = auxHashMap
+}
+
+func (a *hllArrayImpl) putCurMin(curMin int) {
+	a.curMin = curMin
+}
+
+// putKxQ0 sets the kxq0 value.
+func (a *hllArrayImpl) putKxQ0(kxq0 float64) {
+	a.kxq0 = kxq0
+}
+
+// putKxQ1 sets the kxq1 value.
+func (a *hllArrayImpl) putKxQ1(kxq1 float64) {
+	a.kxq1 = kxq1
+}
+
+func (a *hllArrayImpl) putNumAtCurMin(numAtCurMin int) {
+	a.numAtCurMin = numAtCurMin
+}
+
+func (a *hllArrayImpl) putRebuildCurMinNumKxQFlag(rebuildCurMinNumKxQ bool) {
+	a.rebuildCurMinNumKxQ = rebuildCurMinNumKxQ
+}
+
+// getNewAuxHashMap returns a new auxHashMap.
+func (a *hllArrayImpl) getNewAuxHashMap() auxHashMap {
+	return newAuxHashMap(lgAuxArrInts[a.lgConfigK], a.lgConfigK)
+}
+
+// getAuxHashMap returns the auxHashMap.
+func (a *hllArrayImpl) getAuxHashMap() auxHashMap {
+	return a.auxHashMap
+}
+
+func (a *hllArrayImpl) getAuxStart() int {
+	return a.auxStart
+}
+
+// getNibble returns the value of the nibble at the given slotNo.
+func (a *hllArrayImpl) getNibble(slotNo int) int {
+	theByte := int(a.hllByteArr[slotNo>>1])
+
+	if (slotNo & 1) > 0 { //odd?
+		theByte >>= 4
+	}
+	return theByte & loNibbleMask
+}
+
+// putNibble sets the value of the nibble at the given slotNo.
+func (a *hllArrayImpl) putNibble(slotNo int, value byte) {
+	byteNo := slotNo >> 1
+	oldValue := a.hllByteArr[byteNo]
+	if (slotNo & 1) == 0 {
+		a.hllByteArr[byteNo] = (oldValue & hiNibbleMask) | (value & loNibbleMask)
+	} else {
+		a.hllByteArr[byteNo] = (oldValue & loNibbleMask) | ((value << 4) & hiNibbleMask)
+	}
+}
+
+func (a *hllArrayImpl) mergeTo(HllSketch) {
+	panic("possible Corruption, improper access")
+}
+
+func (a *hllArrayImpl) copyCommon() hllArrayImpl {
+	newH := *a
+	if newH.getAuxHashMap() != nil {
+		newH.putAuxHashMap(a.getAuxHashMap().copy(), false)
+	} else {
+		newH.putAuxHashMap(nil, false)
+	}
+	newH.hllByteArr = make([]byte, len(a.hllByteArr))
+	copy(newH.hllByteArr, a.hllByteArr)
+	return newH
+}
+
+func (a *hllArrayImpl) isRebuildCurMinNumKxQFlag() bool {
+	return a.rebuildCurMinNumKxQ
+}
+
+// hipAndKxQIncrementalUpdate is the HIP and KxQ incremental update for hll.
+// This is used when incrementally updating an existing array with non-zero values.
+func (a *hllArrayImpl) hipAndKxQIncrementalUpdate(oldValue int, newValue int) {
+	if oldValue >= newValue {
+		panic("oldValue >= newValue")
+	}
+	kxq0 := a.kxq0
+	kxq1 := a.kxq1
+	//update hipAccum BEFORE updating kxq0 and kxq1
+	a.addToHipAccum(float64(uint64(1<<a.lgConfigK)) / (kxq0 + kxq1))
+	a.incrementalUpdateKxQ(oldValue, newValue, kxq0, kxq1)
+}
+
+// incrementalUpdateKxQ updates kxq0 and kxq1.
+func (a *hllArrayImpl) incrementalUpdateKxQ(oldValue int, newValue int, kxq0 float64, kxq1 float64) {
+	//update kxq0 and kxq1; subtract first, then add.
+	if oldValue < 32 {
+		kxq0 -= common.InvPow2(oldValue)
+		a.kxq0 = kxq0
+	} else {
+		kxq1 -= common.InvPow2(oldValue)
+		a.kxq1 = kxq1
+	}
+	if newValue < 32 {
+		kxq0 += common.InvPow2(newValue)
+		a.kxq0 = kxq0
+	} else {
+		kxq1 += common.InvPow2(newValue)
+		a.kxq1 = kxq1
+	}
+}
+
+// extractCommonHll extracts the common fields from the given byte array.
+func (a *hllArrayImpl) extractCommonHll(byteArr []byte) {
+	a.putOutOfOrder(extractOooFlag(byteArr))
+	a.putCurMin(extractCurMin(byteArr))
+	a.putHipAccum(extractHipAccum(byteArr))
+	a.putKxQ0(extractKxQ0(byteArr))
+	a.putKxQ1(extractKxQ1(byteArr))
+	a.putNumAtCurMin(extractNumAtCurMin(byteArr))
+	a.putRebuildCurMinNumKxQFlag(extractRebuildCurMinNumKxQFlag(byteArr))
+
+	a.hllByteArr = byteArr[hllByteArrStart : hllByteArrStart+len(a.hllByteArr)]
+}
diff --git a/hll/hll_array_test.go b/hll/hll_array_test.go
new file mode 100644
index 0000000..df4c9bb
--- /dev/null
+++ b/hll/hll_array_test.go
@@ -0,0 +1,120 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestCompositeEst(t *testing.T) {
+	testComposite(t, 4, TgtHllType_HLL_4, 1000)
+	testComposite(t, 5, TgtHllType_HLL_4, 1000)
+	testComposite(t, 6, TgtHllType_HLL_4, 1000)
+	testComposite(t, 13, TgtHllType_HLL_4, 10000)
+
+	testComposite(t, 4, TgtHllType_HLL_6, 1000)
+	testComposite(t, 5, TgtHllType_HLL_6, 1000)
+	testComposite(t, 6, TgtHllType_HLL_6, 1000)
+	testComposite(t, 13, TgtHllType_HLL_6, 10000)
+
+	testComposite(t, 4, TgtHllType_HLL_8, 1000)
+	testComposite(t, 5, TgtHllType_HLL_8, 1000)
+	testComposite(t, 6, TgtHllType_HLL_8, 1000)
+	testComposite(t, 13, TgtHllType_HLL_8, 10000)
+}
+
+func testComposite(t *testing.T, lgK int, tgtHllType TgtHllType, n int) {
+	u, err := NewUnion(lgK)
+	assert.NoError(t, err)
+	sk, err := NewHllSketch(lgK, tgtHllType)
+	assert.NoError(t, err)
+
+	for i := 0; i < n; i++ {
+		u.UpdateInt64(int64(i))
+		sk.UpdateInt64(int64(i))
+	}
+
+	u.UpdateSketch(sk)
+	res, err := u.GetResult(tgtHllType)
+	assert.NoError(t, err)
+	res.GetCompositeEstimate()
+}
+
+func TestBigHipGetRse(t *testing.T) {
+	sk, err := NewHllSketch(13, TgtHllType_HLL_8)
+	assert.NoError(t, err)
+
+	for i := 0; i < 10000; i++ {
+		sk.UpdateInt64(int64(i))
+	}
+}
+
+func TestToArraySliceDeserialize(t *testing.T) {
+	lgK := 4
+	u := 8
+	toArraySliceDeserialize(t, lgK, TgtHllType_HLL_4, u)
+	toArraySliceDeserialize(t, lgK, TgtHllType_HLL_6, u)
+	toArraySliceDeserialize(t, lgK, TgtHllType_HLL_8, u)
+
+	lgK = 16
+	u = (((1 << (lgK - 3)) * 3) / 4) + 100
+	toArraySliceDeserialize(t, lgK, TgtHllType_HLL_4, u)
+	toArraySliceDeserialize(t, lgK, TgtHllType_HLL_6, u)
+	toArraySliceDeserialize(t, lgK, TgtHllType_HLL_8, u)
+
+	lgK = 21
+	u = (((1 << (lgK - 3)) * 3) / 4) + 1000
+	toArraySliceDeserialize(t, lgK, TgtHllType_HLL_4, u)
+	toArraySliceDeserialize(t, lgK, TgtHllType_HLL_6, u)
+	toArraySliceDeserialize(t, lgK, TgtHllType_HLL_8, u)
+}
+
+func toArraySliceDeserialize(t *testing.T, lgK int, tgtHllType TgtHllType, u int) {
+	sk1, err := NewHllSketch(lgK, tgtHllType)
+	assert.NoError(t, err)
+
+	for i := 0; i < u; i++ {
+		sk1.UpdateInt64(int64(i))
+	}
+	_, isArray := sk1.(*hllSketchImpl).sketch.(hllArray)
+	assert.True(t, isArray)
+
+	// Update
+	est1 := sk1.GetEstimate()
+	assert.InDelta(t, est1, u, float64(u)*.03)
+	est := sk1.GetHipEstimate()
+	assert.Equal(t, est, est1, 0.0)
+
+	// misc
+	sk1.(*hllSketchImpl).putRebuildCurMinNumKxQFlag(true)
+	sk1.(*hllSketchImpl).putRebuildCurMinNumKxQFlag(false)
+
+	sl1, err := sk1.ToCompactSlice()
+	assert.NoError(t, err)
+	sk2, e := DeserializeHllSketch(sl1, true)
+	assert.NoError(t, e)
+	est2 := sk2.GetEstimate()
+	assert.Equal(t, est2, est1, 0.0)
+
+	err = sk1.Reset()
+	assert.NoError(t, err)
+	est = sk1.GetEstimate()
+	assert.Equal(t, est, 0.0, 0.0)
+}
diff --git a/hll/hll_config.go b/hll/hll_config.go
new file mode 100644
index 0000000..8a2d514
--- /dev/null
+++ b/hll/hll_config.go
@@ -0,0 +1,44 @@
+/*
+ * 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.
+ */
+
+package hll
+
+type hllSketchConfig struct { // extends hllSketchConfig
+	lgConfigK  int
+	tgtHllType TgtHllType
+	curMode    curMode
+}
+
+func newHllSketchConfig(lgConfigK int, tgtHllType TgtHllType, curMode curMode) hllSketchConfig {
+	return hllSketchConfig{
+		lgConfigK:  lgConfigK,
+		tgtHllType: tgtHllType,
+		curMode:    curMode,
+	}
+}
+
+func (c *hllSketchConfig) GetLgConfigK() int {
+	return c.lgConfigK
+}
+
+func (c *hllSketchConfig) GetTgtHllType() TgtHllType {
+	return c.tgtHllType
+}
+
+func (c *hllSketchConfig) GetCurMode() curMode {
+	return c.curMode
+}
diff --git a/hll/hll_estimator.go b/hll/hll_estimator.go
new file mode 100644
index 0000000..7ac5339
--- /dev/null
+++ b/hll/hll_estimator.go
@@ -0,0 +1,153 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"math"
+)
+
+// hllCompositeEstimate is the (non-HIP) estimator.
+// It is called "composite" because multiple estimators are pasted together.
+func hllCompositeEstimate(hllArray *hllArrayImpl) float64 {
+	lgConfigK := hllArray.lgConfigK
+	rawEst := getHllRawEstimate(lgConfigK, hllArray.kxq0+hllArray.kxq1)
+
+	xArr := compositeInterpolationXarrs[lgConfigK-minLogK]
+	yStride := compositeInterpolationYstrides[lgConfigK-minLogK]
+	xArrLen := len(xArr)
+
+	if rawEst < xArr[0] {
+		return 0
+	}
+
+	xArrLenM1 := xArrLen - 1
+
+	if rawEst > xArr[xArrLenM1] {
+		finalY := yStride * float64(xArrLenM1)
+		factor := finalY / xArr[xArrLenM1]
+		return rawEst * factor
+	}
+	adjEst := usingXArrAndYStride(xArr, 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 > float64(uint64(3<<lgConfigK)) {
+		return adjEst
+	}
+
+	linEst := getHllBitMapEstimate(lgConfigK, hllArray.curMin, hllArray.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.
+	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
+	crossOver := 0.64
+	if lgConfigK == 4 {
+		crossOver = 0.718
+	} else if lgConfigK == 5 {
+		crossOver = 0.672
+	}
+
+	if avgEst > (crossOver * float64(uint64(1<<lgConfigK))) {
+		return adjEst
+	} else {
+		return linEst
+	}
+}
+
+// getHllBitMapEstimate is the estimator when N is small, roughly less than k log(k).
+// Refer to Wikipedia: Coupon Collector Problem
+func getHllBitMapEstimate(lgConfigK int, curMin int, numAtCurMin int) float64 {
+	configK := 1 << lgConfigK
+	numUnhitBuckets := 0
+	if curMin == 0 {
+		numUnhitBuckets = numAtCurMin
+	}
+
+	//This will eventually go away.
+	if numUnhitBuckets == 0 {
+		return float64(configK) * math.Log(float64(configK)/0.5)
+	}
+
+	numHitBuckets := configK - numUnhitBuckets
+	return getBitMapEstimate(configK, numHitBuckets)
+}
+
+// getHllRawEstimate is the algorithm from Flajolet's, et al, 2007 HLL paper, Fig 3.
+func getHllRawEstimate(lgConfigK int, kxqSum float64) float64 {
+	configK := 1 << lgConfigK
+	correctionFactor := 0.0
+
+	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 / float64(configK)))
+	}
+
+	return (correctionFactor * float64(configK) * float64(configK)) / kxqSum
+}
+
+func hllUpperBound(hllArray *hllArrayImpl, numStdDev int) (float64, error) {
+	lgConfigK := hllArray.lgConfigK
+	estimate := hllArray.GetEstimate()
+	oooFlag := hllArray.isOutOfOrder()
+	relErr, err := getRelErrAllK(true, oooFlag, lgConfigK, numStdDev)
+	if err != nil {
+		return 0, err
+	}
+	return estimate / (1.0 - relErr), nil
+}
+
+func hllLowerBound(hllArray *hllArrayImpl, numStdDev int) (float64, error) {
+	lgConfigK := hllArray.lgConfigK
+	configK := 1 << lgConfigK
+	numNonZeros := float64(configK)
+	if hllArray.curMin == 0 {
+		numNonZeros -= float64(hllArray.numAtCurMin)
+	}
+	estimate := hllArray.GetEstimate()
+	oooFlag := hllArray.isOutOfOrder()
+	relErr, err := getRelErrAllK(false, oooFlag, lgConfigK, numStdDev)
+	if err != nil {
+		return 0, err
+	}
+	return math.Max(estimate/(1.0+relErr), numNonZeros), nil
+}
+
+func getRelErrAllK(upperBound bool, oooFlag bool, lgConfigK int, numStdDev int) (float64, error) {
+	lgK, err := checkLgK(lgConfigK)
+	if err != nil {
+		return 0, err
+	}
+	if lgK > 12 {
+		rseFactor := hllHipRSEFActor
+		if oooFlag {
+			rseFactor = hllNonHipRSEFactor
+		}
+		configK := 1 << lgK
+		return (float64(numStdDev) * rseFactor) / math.Sqrt(float64(configK)), nil
+	}
+	return math.Abs(getRelErrKLT12(upperBound, oooFlag, lgK, numStdDev)), nil
+}
diff --git a/hll/hll_pair_iterator.go b/hll/hll_pair_iterator.go
new file mode 100644
index 0000000..cebea49
--- /dev/null
+++ b/hll/hll_pair_iterator.go
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package hll
+
+type hllPairIterator struct {
+	lengthPairs int
+	index       int
+	value       int
+}
+
+func newHllPairIterator(lengthPairs int) hllPairIterator {
+	return hllPairIterator{
+		lengthPairs: lengthPairs,
+		index:       -1,
+	}
+}
+
+func (h *hllPairIterator) nextValid() bool {
+	panic("override me")
+}
+
+func (h *hllPairIterator) nextAll() bool {
+	h.index++
+	return h.index < h.lengthPairs
+}
+
+func (h *hllPairIterator) getIndex() int {
+	return h.index
+}
+
+func (h *hllPairIterator) getSlot() int {
+	return h.index
+}
+
+func (h *hllPairIterator) getKey() int {
+	return h.index
+}
diff --git a/hll/hll_sketch.go b/hll/hll_sketch.go
new file mode 100644
index 0000000..08b0149
--- /dev/null
+++ b/hll/hll_sketch.go
@@ -0,0 +1,311 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"encoding/binary"
+	"fmt"
+	"math/bits"
+	"unsafe"
+
+	"github.com/apache/datasketches-go/thetacommon"
+	"github.com/spaolacci/murmur3"
+)
+
+type HllSketch interface {
+	publiclyUpdatable
+	estimableSketch
+	configuredSketch
+	toSliceSketch
+	privatelyUpdatable
+	iterableSketch
+	CopyAs(tgtHllType TgtHllType) HllSketch
+	Copy() HllSketch
+	IsEstimationMode() bool
+	GetSerializationVersion() int
+}
+
+type publiclyUpdatable interface {
+	UpdateUInt64(datum uint64)
+	UpdateInt64(datum int64)
+	UpdateSlice(datum []byte)
+	UpdateString(datum string)
+	Reset() error
+}
+
+type estimableSketch interface {
+	GetCompositeEstimate() float64
+	GetEstimate() float64
+	GetHipEstimate() float64
+	GetLowerBound(numStdDev int) (float64, error)
+	GetUpperBound(numStdDev int) (float64, error)
+	IsEmpty() bool
+}
+
+type configuredSketch interface {
+	GetLgConfigK() int
+	GetTgtHllType() TgtHllType
+	GetCurMode() curMode
+}
+
+type toSliceSketch interface {
+	GetUpdatableSerializationBytes() int
+	ToCompactSlice() ([]byte, error)
+	ToUpdatableSlice() ([]byte, error)
+}
+
+type privatelyUpdatable interface {
+	couponUpdate(coupon int) hllSketchBase
+}
+
+type iterableSketch interface {
+	iterator() pairIterator
+}
+
+type hllSketchBase interface {
+	estimableSketch
+	configuredSketch
+	toSliceSketch
+	privatelyUpdatable
+	iterableSketch
+
+	getMemDataStart() int
+	getPreInts() int
+	isOutOfOrder() bool
+	isRebuildCurMinNumKxQFlag() bool
+
+	putOutOfOrder(oooFlag bool)
+	putRebuildCurMinNumKxQFlag(rebuildCurMinNumKxQFlag bool)
+	copyAs(tgtHllType TgtHllType) hllSketchBase
+	copy() hllSketchBase
+	mergeTo(dest HllSketch)
+}
+
+type hllSketchImpl struct { // extends BaseHllSketch
+	sketch  hllSketchBase
+	scratch [8]byte
+}
+
+func (h *hllSketchImpl) Reset() error {
+	lgK, err := checkLgK(h.sketch.GetLgConfigK())
+	if err != nil {
+		return err
+	}
+	couponList, err := newCouponList(lgK, h.sketch.GetTgtHllType(), curMode_LIST)
+	if err != nil {
+		return err
+	}
+	h.sketch = &couponList
+	return nil
+}
+
+func NewHllSketch(lgConfigK int, tgtHllType TgtHllType) (HllSketch, error) {
+	lgK := lgConfigK
+	lgK, err := checkLgK(lgK)
+	if err != nil {
+		return nil, err
+	}
+	couponList, err := newCouponList(lgK, tgtHllType, curMode_LIST)
+	if err != nil {
+		return nil, err
+	}
+	return newHllSketchImpl(&couponList), nil
+}
+
+func NewHllSketchDefault(lgConfigK int) (HllSketch, error) {
+	lgK, err := checkLgK(lgConfigK)
+	if err != nil {
+		return nil, err
+	}
+	couponList, err := newCouponList(lgK, TgtHllType_DEFAULT, curMode_LIST)
+	if err != nil {
+		return nil, err
+	}
+	return newHllSketchImpl(&couponList), nil
+}
+
+func DeserializeHllSketch(byteArray []byte, checkRebuild bool) (HllSketch, error) {
+	if len(byteArray) < 8 {
+		return nil, fmt.Errorf("input array too small: %d", len(byteArray))
+	}
+	curMode, err := checkPreamble(byteArray)
+	if err != nil {
+		return nil, err
+	}
+	if curMode == curMode_HLL {
+		tgtHllType := extractTgtHllType(byteArray)
+		if tgtHllType == TgtHllType_HLL_4 {
+			return newHllSketchImpl(deserializeHll4(byteArray)), nil
+		} else if tgtHllType == TgtHllType_HLL_6 {
+			return newHllSketchImpl(deserializeHll6(byteArray)), nil
+		} else {
+			a := newHllSketchImpl(deserializeHll8(byteArray))
+			if checkRebuild {
+				err := checkRebuildCurMinNumKxQ(a)
+				if err != nil {
+					return nil, err
+				}
+			}
+			return a, nil
+		}
+	} else if curMode == curMode_LIST {
+		cp, err := deserializeCouponList(byteArray)
+		if err != nil {
+			return nil, err
+		}
+		return newHllSketchImpl(cp), nil
+	} else {
+		chs, err := deserializeCouponHashSet(byteArray)
+		if err != nil {
+			return nil, err
+		}
+		return newHllSketchImpl(chs), nil
+	}
+}
+
+func newHllSketchImpl(coupon hllSketchBase) HllSketch {
+	return &hllSketchImpl{
+		sketch:  coupon,
+		scratch: [8]byte{},
+	}
+}
+
+func (h *hllSketchImpl) GetEstimate() float64 {
+	return h.sketch.GetEstimate()
+}
+
+func (h *hllSketchImpl) GetCompositeEstimate() float64 {
+	return h.sketch.GetCompositeEstimate()
+}
+
+func (h *hllSketchImpl) GetHipEstimate() float64 {
+	return h.sketch.GetHipEstimate()
+}
+
+func (h *hllSketchImpl) GetUpperBound(numStdDev int) (float64, error) {
+	return h.sketch.GetUpperBound(numStdDev)
+}
+
+func (h *hllSketchImpl) GetLowerBound(numStdDev int) (float64, error) {
+	return h.sketch.GetLowerBound(numStdDev)
+}
+
+func (h *hllSketchImpl) GetUpdatableSerializationBytes() int {
+	return h.sketch.GetUpdatableSerializationBytes()
+}
+
+func (h *hllSketchImpl) UpdateUInt64(datum uint64) {
+	binary.LittleEndian.PutUint64(h.scratch[:], datum)
+	hi, lo := h.hash(h.scratch[:])
+	h.couponUpdate(coupon(hi, lo))
+}
+
+func (h *hllSketchImpl) UpdateInt64(datum int64) {
+	h.UpdateUInt64(uint64(datum))
+}
+
+func (h *hllSketchImpl) UpdateSlice(datum []byte) {
+	if len(datum) == 0 {
+		return
+	}
+	hi, lo := h.hash(datum)
+	h.couponUpdate(coupon(hi, lo))
+}
+
+func (h *hllSketchImpl) UpdateString(datum string) {
+	// get a slice to the string data (avoiding a copy to heap)
+	unsafeSlice := unsafe.Slice(unsafe.StringData(datum), len(datum))
+	h.UpdateSlice(unsafeSlice)
+}
+
+func (h *hllSketchImpl) IsEmpty() bool {
+	return h.sketch.IsEmpty()
+}
+
+func (h *hllSketchImpl) ToCompactSlice() ([]byte, error) {
+	return h.sketch.ToCompactSlice()
+}
+
+func (h *hllSketchImpl) ToUpdatableSlice() ([]byte, error) {
+	return h.sketch.ToUpdatableSlice()
+}
+
+func (h *hllSketchImpl) GetLgConfigK() int {
+	return h.sketch.GetLgConfigK()
+}
+
+func (h *hllSketchImpl) GetTgtHllType() TgtHllType {
+	return h.sketch.GetTgtHllType()
+}
+
+func (h *hllSketchImpl) GetCurMode() curMode {
+	return h.sketch.GetCurMode()
+}
+
+func (h *hllSketchImpl) iterator() pairIterator {
+	return h.sketch.iterator()
+}
+
+func coupon(hashLo uint64, hashHi uint64) int {
+	addr26 := hashLo & keyMask26
+	lz := uint64(bits.LeadingZeros64(hashHi))
+	value := min(lz, 62) + 1
+	return int((value << keyBits26) | addr26)
+}
+
+func (h *hllSketchImpl) couponUpdate(coupon int) hllSketchBase {
+	if (coupon >> keyBits26) == empty {
+		return h.sketch
+	}
+	h.sketch = h.sketch.couponUpdate(coupon)
+	return h.sketch
+}
+
+func (h *hllSketchImpl) putRebuildCurMinNumKxQFlag(rebuildCurMinNumKxQFlag bool) {
+	h.sketch.putRebuildCurMinNumKxQFlag(rebuildCurMinNumKxQFlag)
+}
+
+func (h *hllSketchImpl) mergeTo(dest HllSketch) {
+	h.sketch.mergeTo(dest)
+}
+
+func (h *hllSketchImpl) CopyAs(tgtHllType TgtHllType) HllSketch {
+	a := h.sketch.copyAs(tgtHllType)
+	return newHllSketchImpl(a)
+}
+
+func (h *hllSketchImpl) Copy() HllSketch {
+	a := h.sketch.copy()
+	return newHllSketchImpl(a)
+}
+
+// IsEstimationMode returns true for all sketches in this package.
+// Hll family of sketches and operators is always estimating, even for very small values.
+func (h *hllSketchImpl) IsEstimationMode() bool {
+	return true
+}
+
+// GetSerializationVersion returns the serialization version used by this sketch.
+func (h *hllSketchImpl) GetSerializationVersion() int {
+	return serVer
+}
+
+func (h *hllSketchImpl) hash(bs []byte) (uint64, uint64) {
+	hi, lo := murmur3.Sum128WithSeed(bs, thetacommon.DEFAULT_UPDATE_SEED)
+	return hi, lo
+}
diff --git a/hll/hll_sketch_isomomorphism_test.go b/hll/hll_sketch_isomomorphism_test.go
new file mode 100644
index 0000000..5535f03
--- /dev/null
+++ b/hll/hll_sketch_isomomorphism_test.go
@@ -0,0 +1,250 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"fmt"
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+var v = 0
+
+// Merges a type1 to an empty union (heap, HLL_8), and gets result as type1, checks binary equivalence
+func TestIsomorphicUnionUpdatableHeap(t *testing.T) {
+	for lgK := 4; lgK <= 21; lgK++ { //All LgK
+		for cm := 0; cm <= 2; cm++ { //List, Set, Hll
+			if (lgK < 8) && (cm == 1) { //lgk < 8 list transistions directly to HLL
+				continue
+			}
+			curMode := curMode(cm)
+			for tt := 0; tt <= 2; tt++ { //HLL_4, HLL_6, HLL_8
+				tgtHllType1 := TgtHllType(tt)
+				sk1, err := buildHeapSketch(lgK, tgtHllType1, curMode)
+				assert.NoError(t, err)
+				sk1bytes, err := sk1.ToUpdatableSlice()
+				assert.NoError(t, err) //UPDATABLE
+				union, err := NewUnion(lgK)
+				assert.NoError(t, err)
+				//UNION
+				union.UpdateSketch(sk1)
+				sk2, err := union.GetResult(tgtHllType1)
+				assert.NoError(t, err)
+				sk2bytes, err := sk2.ToUpdatableSlice() //UPDATABLE
+				assert.NoError(t, err)
+				comp := fmt.Sprintf("LgK=%d, curMode=%d, Type:%d", lgK, curMode, tgtHllType1)
+				checkArrays(t, sk1bytes, sk2bytes, comp, false)
+			}
+		}
+	}
+}
+
+func TestIsomorphicUnionCompactHeap(t *testing.T) {
+	for lgK := 4; lgK <= 21; lgK++ { //All LgK
+		for cm := 0; cm <= 2; cm++ { //List, Set, Hll
+			if (lgK < 8) && (cm == 1) { //lgk < 8 list transistions directly to HLL
+				continue
+			}
+			curMode := curMode(cm)
+			for tt := 0; tt <= 2; tt++ { //HLL_4, HLL_6, HLL_8
+				tgtHllType1 := TgtHllType(tt)
+				sk1, err := buildHeapSketch(lgK, tgtHllType1, curMode)
+				assert.NoError(t, err)
+				sk1bytes, err := sk1.ToCompactSlice() //COMPACT
+				assert.NoError(t, err)
+				union, err := NewUnion(lgK) //UNION
+				assert.NoError(t, err)
+				union.UpdateSketch(sk1)
+				sk2, err := union.GetResult(tgtHllType1)
+				assert.NoError(t, err)
+				sk2bytes, err := sk2.ToCompactSlice() //COMPACT
+				assert.NoError(t, err)
+				comp := fmt.Sprintf("LgK=%d, curMode=%d, Type:%d", lgK, curMode, tgtHllType1)
+				checkArrays(t, sk1bytes, sk2bytes, comp, false)
+			}
+		}
+	}
+}
+
+func TestIsomorphicCopyAsUpdatableHeap(t *testing.T) {
+	for lgK := 4; lgK <= 21; lgK++ { //All LgK
+		for cm := 0; cm <= 2; cm++ { //List, Set, Hll
+			if (lgK < 8) && (cm == 1) { //lgk < 8 list transistions directly to HLL
+				continue
+			}
+			curMode := curMode(cm)
+			for t1 := 0; t1 <= 2; t1++ { //HLL_4, HLL_6, HLL_8
+				tgtHllType1 := TgtHllType(t1)
+				sk1, err := buildHeapSketch(lgK, tgtHllType1, curMode)
+				assert.NoError(t, err)
+				sk1bytes, err := sk1.ToUpdatableSlice() //UPDATABLE
+				assert.NoError(t, err)
+				for t2 := 0; t2 <= 2; t2++ { //HLL_4, HLL_6, HLL_8
+					if t2 == t1 {
+						continue
+					}
+					tgtHllType2 := TgtHllType(t2)
+					sk2 := sk1.CopyAs(tgtHllType2)            //COPY AS
+					sk1B := sk2.CopyAs(tgtHllType1)           //COPY AS
+					sk1Bbytes, err := sk1B.ToUpdatableSlice() //UPDATABLE
+					assert.NoError(t, err)
+					comp := fmt.Sprintf("LgK=%d, curMode=%d, Type1:%d, Type2:%d", lgK, curMode, tgtHllType1, tgtHllType2)
+					checkArrays(t, sk1bytes, sk1Bbytes, comp, false)
+				}
+			}
+		}
+	}
+}
+
+func TestIsomorphicHllMerges2(t *testing.T) {
+	for lgK := 4; lgK <= 4; lgK++ { //All LgK
+		u, err := buildHeapUnionHllMode(lgK, 0)
+		assert.NoError(t, err)
+		sk, err := buildHeapSketchHllMode(lgK, TgtHllType_HLL_8, 1<<lgK)
+		assert.NoError(t, err)
+		u.UpdateSketch(sk)
+		resultOut8, err := u.GetResult(TgtHllType_HLL_8) //The reference
+		assert.NoError(t, err)
+		bytesOut8, err := resultOut8.ToUpdatableSlice()
+		assert.NoError(t, err)
+
+		u, err = buildHeapUnionHllMode(lgK, 0)
+		assert.NoError(t, err)
+		sk, err = buildHeapSketchHllMode(lgK, TgtHllType_HLL_6, 1<<lgK)
+		assert.NoError(t, err)
+		u.UpdateSketch(sk)
+		resultOut6, err := u.GetResult(TgtHllType_HLL_8) //should be identical except for HllAccum
+		assert.NoError(t, err)
+		bytesOut6, err := resultOut6.ToUpdatableSlice()
+		assert.NoError(t, err)
+
+		comb := fmt.Sprintf("LgK: %d, SkType: HLL_6, Compared with SkType HLL_8", lgK)
+		checkArrays(t, bytesOut8, bytesOut6, comb, false)
+
+		u, err = buildHeapUnionHllMode(lgK, 0)
+		assert.NoError(t, err)
+		sk, err = buildHeapSketchHllMode(lgK, TgtHllType_HLL_4, 1<<lgK)
+		assert.NoError(t, err)
+		u.UpdateSketch(sk)
+		resultOut4, err := u.GetResult(TgtHllType_HLL_8) //should be identical except for HllAccum
+		assert.NoError(t, err)
+		bytesOut4, err := resultOut4.ToUpdatableSlice()
+		assert.NoError(t, err)
+		comb = fmt.Sprintf("LgK: %d, SkType: HLL_4, Compared with SkType HLL_8", lgK)
+		checkArrays(t, bytesOut8, bytesOut4, comb, false)
+	}
+}
+
+func TestIsomorphicCopyAsCompactHeap(t *testing.T) {
+	for lgK := 4; lgK <= 21; lgK++ { //All LgK
+		for cm := 0; cm <= 2; cm++ { //List, Set, Hll
+			if (lgK < 8) && (cm == 1) { //lgk < 8 list transistions directly to HLL
+				continue
+			}
+			curMode := curMode(cm)
+			for t1 := 0; t1 <= 2; t1++ { //HLL_4, HLL_6, HLL_8
+				tgtHllType1 := TgtHllType(t1)
+				sk1, err := buildHeapSketch(lgK, tgtHllType1, curMode)
+				assert.NoError(t, err)
+				sk1bytes, err := sk1.ToCompactSlice() //COMPACT
+				assert.NoError(t, err)
+				for t2 := 0; t2 <= 2; t2++ { //HLL_4, HLL_6, HLL_8
+					if t2 == t1 {
+						continue
+					}
+					tgtHllType2 := TgtHllType(t2)
+					sk2 := sk1.CopyAs(tgtHllType2)          //COPY AS
+					sk1B := sk2.CopyAs(tgtHllType1)         //COPY AS
+					sk1Bbytes, err := sk1B.ToCompactSlice() //COMPACT
+					assert.NoError(t, err)
+					comp := fmt.Sprintf("LgK=%d, curMode=%d, Type1:%d, Type2:%d", lgK, curMode, tgtHllType1, tgtHllType2)
+					checkArrays(t, sk1bytes, sk1Bbytes, comp, false)
+				}
+			}
+		}
+	}
+}
+
+func checkArrays(t *testing.T, sk1bytes, sk2bytes []byte, comb string, omitHipAccum bool) {
+	leng := len(sk1bytes)
+	if leng != len(sk2bytes) {
+		t.Errorf("Sketch images not the same length: %s", comb)
+		return
+	}
+	for i := 0; i < leng; i++ {
+		if omitHipAccum && (i >= 8) && (i <= 15) {
+			continue
+		}
+		if sk1bytes[i] == sk2bytes[i] {
+			continue
+		}
+		t.Errorf("%s: %d", comb, i)
+	}
+}
+
+func buildHeapUnionHllMode(lgK int, startN int) (Union, error) {
+	u, err := NewUnion(lgK)
+	if err != nil {
+		return nil, err
+	}
+	n := getN(lgK, curMode_HLL)
+	for i := 0; i < n; i++ {
+		u.UpdateUInt64(uint64(i + startN))
+	}
+	return u, nil
+}
+
+func buildHeapSketch(lgK int, tgtHllType TgtHllType, curMode curMode) (HllSketch, error) {
+	sk, err := NewHllSketch(lgK, tgtHllType)
+	if err != nil {
+		return nil, err
+	}
+	n := getN(lgK, curMode)
+	for i := 0; i < n; i++ {
+		sk.UpdateUInt64(uint64(i + v))
+	}
+	v += n
+	return sk, nil
+}
+
+func buildHeapSketchHllMode(lgK int, tgtHllType TgtHllType, startN int) (HllSketch, error) {
+	sk, err := NewHllSketch(lgK, tgtHllType)
+	if err != nil {
+		return nil, err
+	}
+	n := getN(lgK, curMode_HLL)
+	for i := 0; i < n; i++ {
+		sk.UpdateUInt64(uint64(i + startN))
+	}
+	return sk, nil
+}
+
+// if lgK >= 8, curMode != SET!
+func getN(lgK int, curMode curMode) int {
+	if curMode == curMode_LIST {
+		return 4
+	}
+	if curMode == curMode_SET {
+		return 1 << (lgK - 4)
+	}
+	if (lgK < 8) && (curMode == curMode_HLL) {
+		return 1 << lgK
+	}
+	return 1 << (lgK - 3)
+}
diff --git a/hll/hll_sketch_serialization_test.go b/hll/hll_sketch_serialization_test.go
new file mode 100644
index 0000000..a4cefc2
--- /dev/null
+++ b/hll/hll_sketch_serialization_test.go
@@ -0,0 +1,240 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"fmt"
+	"os"
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+const (
+	DSKETCH_TEST_GENERATE_GO = "DSKETCH_TEST_GENERATE_GO"
+	DSKETCH_TEST_CROSS_JAVA  = "DSKETCH_TEST_CROSS_JAVA"
+	DSKETCH_TEST_CROSS_CPP   = "DSKETCH_TEST_CROSS_CPP"
+	DSKETCH_TEST_CROSS_GO    = "DSKETCH_TEST_CROSS_GO"
+)
+
+// Run me manually for generation
+func TestGenerateGoFiles(t *testing.T) {
+	if len(os.Getenv(DSKETCH_TEST_GENERATE_GO)) == 0 {
+		t.Skipf("%s not set", DSKETCH_TEST_GENERATE_GO)
+	}
+
+	nArr := []int{0, 1, 10, 100, 1000, 10000, 100000, 1000000}
+	for _, n := range nArr {
+		hll4, err := NewHllSketch(defaultLgK, TgtHllType_HLL_4)
+		assert.NoError(t, err)
+		hll6, err := NewHllSketch(defaultLgK, TgtHllType_HLL_6)
+		assert.NoError(t, err)
+		hll8, err := NewHllSketch(defaultLgK, TgtHllType_HLL_8)
+		assert.NoError(t, err)
+
+		for i := 0; i < n; i++ {
+			hll4.UpdateUInt64(uint64(i))
+			hll6.UpdateUInt64(uint64(i))
+			hll8.UpdateUInt64(uint64(i))
+		}
+		err = os.MkdirAll(goPath, os.ModePerm)
+		assert.NoError(t, err)
+
+		sl4, err := hll4.ToCompactSlice()
+		assert.NoError(t, err)
+		err = os.WriteFile(fmt.Sprintf("%s/hll4_n%d_go.sk", goPath, n), sl4, 0644)
+		assert.NoError(t, err)
+
+		sl6, err := hll6.ToCompactSlice()
+		assert.NoError(t, err)
+		err = os.WriteFile(fmt.Sprintf("%s/hll6_n%d_go.sk", goPath, n), sl6, 0644)
+		assert.NoError(t, err)
+
+		sl8, err := hll8.ToCompactSlice()
+		assert.NoError(t, err)
+		err = os.WriteFile(fmt.Sprintf("%s/hll8_n%d_go.sk", goPath, n), sl8, 0644)
+		assert.NoError(t, err)
+	}
+}
+
+func TestJavaCompat(t *testing.T) {
+	if len(os.Getenv(DSKETCH_TEST_CROSS_JAVA)) == 0 {
+		t.Skipf("%s not set", DSKETCH_TEST_CROSS_JAVA)
+	}
+
+	t.Run("Java Hll4", func(t *testing.T) {
+		nArr := []int{0, 1, 10, 100, 1000, 10000, 100000, 1000000}
+		for _, n := range nArr {
+			bytes, err := os.ReadFile(fmt.Sprintf("%s/hll4_n%d_java.sk", javaPath, n))
+			assert.NoError(t, err)
+			sketch, err := DeserializeHllSketch(bytes, true)
+			if err != nil {
+				return
+			}
+
+			assert.Equal(t, 12, sketch.GetLgConfigK())
+			est := sketch.GetEstimate()
+			assert.InDelta(t, n, est, float64(n)*0.02)
+		}
+	})
+
+	t.Run("Java Hll6", func(t *testing.T) {
+		nArr := []int{0, 1, 10, 100, 1000, 10000, 100000, 1000000}
+		for _, n := range nArr {
+			bytes, err := os.ReadFile(fmt.Sprintf("%s/hll6_n%d_java.sk", javaPath, n))
+			assert.NoError(t, err)
+
+			sketch, err := DeserializeHllSketch(bytes, true)
+			if err != nil {
+				return
+			}
+
+			assert.Equal(t, 12, sketch.GetLgConfigK())
+			est := sketch.GetEstimate()
+			assert.InDelta(t, n, est, float64(n)*0.02)
+		}
+	})
+
+	t.Run("Java Hll8", func(t *testing.T) {
+		nArr := []int{0, 1, 10, 100, 1000, 10000, 100000, 1000000}
+		for _, n := range nArr {
+			bytes, err := os.ReadFile(fmt.Sprintf("%s/hll8_n%d_java.sk", javaPath, n))
+			assert.NoError(t, err)
+			sketch, err := DeserializeHllSketch(bytes, true)
+			if err != nil {
+				return
+			}
+
+			assert.Equal(t, 12, sketch.GetLgConfigK())
+			est := sketch.GetEstimate()
+			assert.InDelta(t, n, est, float64(n)*0.02)
+		}
+	})
+}
+
+func TestCppCompat(t *testing.T) {
+	if len(os.Getenv(DSKETCH_TEST_CROSS_CPP)) == 0 {
+		t.Skipf("%s not set", DSKETCH_TEST_CROSS_CPP)
+	}
+
+	t.Run("Cpp Hll4", func(t *testing.T) {
+		nArr := []int{0, 1, 10, 100, 1000, 10000, 100000, 1000000}
+		for _, n := range nArr {
+			bytes, err := os.ReadFile(fmt.Sprintf("%s/hll4_n%d_cpp.sk", cppPath, n))
+			assert.NoError(t, err)
+			sketch, err := DeserializeHllSketch(bytes, true)
+			if err != nil {
+				return
+			}
+
+			assert.Equal(t, 12, sketch.GetLgConfigK())
+			est := sketch.GetEstimate()
+			assert.InDelta(t, n, est, float64(n)*0.02)
+		}
+	})
+
+	t.Run("Cpp Hll6", func(t *testing.T) {
+		nArr := []int{0, 1, 10, 100, 1000, 10000, 100000, 1000000}
+		for _, n := range nArr {
+			bytes, err := os.ReadFile(fmt.Sprintf("%s/hll6_n%d_cpp.sk", cppPath, n))
+			assert.NoError(t, err)
+
+			sketch, err := DeserializeHllSketch(bytes, true)
+			if err != nil {
+				return
+			}
+
+			assert.Equal(t, 12, sketch.GetLgConfigK())
+			est := sketch.GetEstimate()
+			assert.InDelta(t, n, est, float64(n)*0.02)
+		}
+	})
+
+	t.Run("Cpp Hll8", func(t *testing.T) {
+		nArr := []int{0, 1, 10, 100, 1000, 10000, 100000, 1000000}
+		for _, n := range nArr {
+			bytes, err := os.ReadFile(fmt.Sprintf("%s/hll8_n%d_cpp.sk", cppPath, n))
+			assert.NoError(t, err)
+			sketch, err := DeserializeHllSketch(bytes, true)
+			if err != nil {
+				return
+			}
+
+			assert.Equal(t, 12, sketch.GetLgConfigK())
+			est := sketch.GetEstimate()
+			assert.InDelta(t, n, est, float64(n)*0.02)
+		}
+	})
+}
+
+func TestGoCompat(t *testing.T) {
+	if len(os.Getenv(DSKETCH_TEST_CROSS_GO)) == 0 {
+		t.Skipf("%s not set", DSKETCH_TEST_CROSS_GO)
+	}
+
+	t.Run("Go Hll4", func(t *testing.T) {
+		nArr := []int{0, 1, 10, 100, 1000, 10000, 100000, 1000000}
+		for _, n := range nArr {
+			bytes, err := os.ReadFile(fmt.Sprintf("%s/hll4_n%d_go.sk", goPath, n))
+			assert.NoError(t, err)
+
+			sketch, err := DeserializeHllSketch(bytes, true)
+			if err != nil {
+				return
+			}
+
+			assert.Equal(t, 12, sketch.GetLgConfigK())
+			est := sketch.GetEstimate()
+			assert.InDelta(t, n, est, float64(n)*0.02)
+		}
+	})
+
+	t.Run("Go Hll6", func(t *testing.T) {
+		nArr := []int{0, 1, 10, 100, 1000, 10000, 100000, 1000000}
+		for _, n := range nArr {
+			bytes, err := os.ReadFile(fmt.Sprintf("%s/hll6_n%d_go.sk", goPath, n))
+			assert.NoError(t, err)
+
+			sketch, err := DeserializeHllSketch(bytes, true)
+			if err != nil {
+				return
+			}
+
+			assert.Equal(t, 12, sketch.GetLgConfigK())
+			est := sketch.GetEstimate()
+			assert.InDelta(t, n, est, float64(n)*0.02)
+		}
+	})
+
+	t.Run("Go Hll8", func(t *testing.T) {
+		nArr := []int{0, 1, 10, 100, 1000, 10000, 100000, 1000000}
+		for _, n := range nArr {
+			bytes, err := os.ReadFile(fmt.Sprintf("%s/hll8_n%d_go.sk", goPath, n))
+			assert.NoError(t, err)
+
+			sketch, err := DeserializeHllSketch(bytes, true)
+			if err != nil {
+				return
+			}
+
+			assert.Equal(t, 12, sketch.GetLgConfigK())
+			est := sketch.GetEstimate()
+			assert.InDelta(t, n, est, float64(n)*0.02)
+		}
+	})
+}
diff --git a/hll/hll_sketch_test.go b/hll/hll_sketch_test.go
new file mode 100644
index 0000000..4b6d9a3
--- /dev/null
+++ b/hll/hll_sketch_test.go
@@ -0,0 +1,398 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"encoding/binary"
+	"fmt"
+	"strconv"
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+const (
+	javaPath = "../serialization_test_data/java_generated_files"
+	cppPath  = "../serialization_test_data/cpp_generated_files"
+	goPath   = "../serialization_test_data/go_generated_files"
+)
+
+func TestMisc(t *testing.T) {
+	hll, err := NewHllSketch(10, TgtHllType_HLL_4)
+	assert.NoError(t, err)
+	assert.True(t, hll.IsEstimationMode())
+	err = hll.Reset()
+	assert.NoError(t, err)
+	assert.Equal(t, serVer, hll.GetSerializationVersion())
+}
+
+func TestUpdateTypes(t *testing.T) {
+	checkUpdateType(t, TgtHllType_HLL_4)
+	checkUpdateType(t, TgtHllType_HLL_6)
+	checkUpdateType(t, TgtHllType_HLL_8)
+}
+
+func checkUpdateType(t *testing.T, tgtHllType TgtHllType) {
+	hll, err := NewHllSketch(11, tgtHllType)
+	assert.NoError(t, err)
+
+	hll.UpdateSlice(nil)
+	hll.UpdateSlice(make([]byte, 0))
+	hll.UpdateSlice([]byte{1, 2, 3})
+	hll.UpdateString("")
+	hll.UpdateString("abc")
+
+	hll.UpdateInt64(0)
+	hll.UpdateInt64(1)
+	hll.UpdateInt64(-1)
+
+	hll.UpdateUInt64(0)
+	hll.UpdateUInt64(1)
+}
+
+func TestCopies(t *testing.T) {
+	checkCopy(t, 14, TgtHllType_HLL_4)
+	checkCopy(t, 8, TgtHllType_HLL_6)
+	checkCopy(t, 8, TgtHllType_HLL_8)
+}
+
+func checkCopy(t *testing.T, lgK int, tgtHllType TgtHllType) {
+	sk, err := NewHllSketch(lgK, tgtHllType)
+	assert.NoError(t, err)
+	for i := 0; i < 7; i++ {
+		sk.UpdateInt64(int64(i))
+	}
+	assert.Equal(t, curMode_LIST, sk.GetCurMode())
+
+	skCopy := sk.Copy()
+	assert.Equal(t, curMode_LIST, skCopy.GetCurMode())
+
+	impl1 := sk.(*hllSketchImpl).sketch
+	impl2 := skCopy.(*hllSketchImpl).sketch
+
+	assert.Equal(t, impl1.(*couponListImpl).couponCount, impl2.(*couponListImpl).couponCount)
+
+	est1 := impl1.GetEstimate()
+	est2 := impl2.GetEstimate()
+	assert.Equal(t, est1, est2)
+	assert.False(t, impl1 == impl2)
+
+	for i := 7; i < 24; i++ {
+		sk.UpdateInt64(int64(i))
+	}
+
+	assert.Equal(t, curMode_SET, sk.GetCurMode())
+	skCopy = sk.Copy()
+	assert.Equal(t, curMode_SET, skCopy.GetCurMode())
+
+	impl1 = sk.(*hllSketchImpl).sketch
+	impl2 = skCopy.(*hllSketchImpl).sketch
+
+	assert.Equal(t, impl1.(*couponHashSetImpl).couponCount, impl2.(*couponHashSetImpl).couponCount)
+	est1 = impl1.GetEstimate()
+	est2 = impl2.GetEstimate()
+	assert.Equal(t, est1, est2)
+	assert.False(t, impl1 == impl2)
+
+	u := 25
+	if tgtHllType == TgtHllType_HLL_4 {
+		u = 100000
+	}
+	for i := 24; i < u; i++ {
+		sk.UpdateInt64(int64(i))
+	}
+
+	assert.Equal(t, curMode_HLL, sk.GetCurMode())
+	skCopy = sk.Copy()
+	assert.Equal(t, curMode_HLL, skCopy.GetCurMode())
+
+	impl1 = sk.(*hllSketchImpl).sketch
+	impl2 = skCopy.(*hllSketchImpl).sketch
+
+	est1 = impl1.GetEstimate()
+	est2 = impl2.GetEstimate()
+	assert.Equal(t, est1, est2)
+	assert.False(t, impl1 == impl2)
+}
+
+func TestCopyAs(t *testing.T) {
+	checkCopyAs(t, TgtHllType_HLL_4, TgtHllType_HLL_4)
+	checkCopyAs(t, TgtHllType_HLL_4, TgtHllType_HLL_6)
+	checkCopyAs(t, TgtHllType_HLL_4, TgtHllType_HLL_8)
+	checkCopyAs(t, TgtHllType_HLL_6, TgtHllType_HLL_4)
+	checkCopyAs(t, TgtHllType_HLL_6, TgtHllType_HLL_6)
+	checkCopyAs(t, TgtHllType_HLL_6, TgtHllType_HLL_8)
+	checkCopyAs(t, TgtHllType_HLL_8, TgtHllType_HLL_4)
+	checkCopyAs(t, TgtHllType_HLL_8, TgtHllType_HLL_6)
+	checkCopyAs(t, TgtHllType_HLL_8, TgtHllType_HLL_8)
+}
+
+func checkCopyAs(t *testing.T, srcType TgtHllType, dstType TgtHllType) {
+	var (
+		lgK  = 8
+		n1   = 7
+		n2   = 24
+		n3   = 1000
+		base = 0
+	)
+
+	src, err := NewHllSketch(lgK, srcType)
+	assert.NoError(t, err)
+	for i := 0; i < n1; i++ {
+		src.UpdateInt64(int64(i + base))
+	}
+	dst := src.CopyAs(dstType)
+	srcEst := src.GetEstimate()
+	dstEst := dst.GetEstimate()
+	assert.Equal(t, srcEst, dstEst)
+
+	for i := n1; i < n2; i++ {
+		src.UpdateInt64(int64(i + base))
+	}
+	dst = src.CopyAs(dstType)
+	srcEst = src.GetEstimate()
+	dstEst = dst.GetEstimate()
+	assert.Equal(t, srcEst, dstEst)
+
+	for i := n2; i < n3; i++ {
+		src.UpdateInt64(int64(i + base))
+	}
+	dst = src.CopyAs(dstType)
+	srcEst = src.GetEstimate()
+	dstEst = dst.GetEstimate()
+	assert.Equal(t, srcEst, dstEst)
+}
+
+func TestNewHLLDataSketchUint(t *testing.T) {
+	tgts := []TgtHllType{TgtHllType_HLL_4, TgtHllType_HLL_6, TgtHllType_HLL_8}
+	ns := []int{1, 10, 100, 1000, 10000, 100000, 1000000}
+	for _, tgt := range tgts {
+		hll, err := NewHllSketch(11, tgt)
+		assert.NoError(t, err)
+		for _, n := range ns {
+			for i := 0; i < n; i++ {
+				hll.UpdateUInt64(uint64(i))
+			}
+			est := hll.GetEstimate()
+			assert.InDelta(t, n, est, float64(n)*0.03)
+		}
+	}
+}
+
+func TestNewHLLDataSketchString(t *testing.T) {
+	tgts := []TgtHllType{TgtHllType_HLL_4, TgtHllType_HLL_6, TgtHllType_HLL_8}
+	ns := []int{1, 10, 100, 1000, 10000, 100000, 1000000}
+	for _, tgt := range tgts {
+		hll, err := NewHllSketch(11, tgt)
+		assert.NoError(t, err)
+		for _, n := range ns {
+			for i := 0; i < n; i++ {
+				hll.UpdateString(strconv.Itoa(i))
+			}
+			est := hll.GetEstimate()
+			assert.InDelta(t, n, est, float64(n)*0.03)
+		}
+	}
+}
+
+func TestHLLDataSketchT(b *testing.T) {
+	hll, err := NewHllSketch(21, TgtHllType_HLL_4)
+	assert.NoError(b, err)
+	for i := 0; i < 1000000; i++ {
+		hll.UpdateUInt64(uint64(i))
+	}
+	est := hll.GetEstimate()
+	assert.InDelta(b, 1000000, est, float64(1000000)*0.03)
+
+}
+
+func BenchmarkHLLDataSketch(b *testing.B) {
+	// HLL uint64 BenchMark
+	b.Run("lgK4 HLL4 uint", func(b *testing.B) {
+		hll, _ := NewHllSketch(4, TgtHllType_HLL_4)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateUInt64(uint64(i))
+		}
+	})
+	b.Run("lgK16 HLL4 uint", func(b *testing.B) {
+		hll, _ := NewHllSketch(16, TgtHllType_HLL_4)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateUInt64(uint64(i))
+		}
+	})
+	b.Run("lgK21 HLL4 uint", func(b *testing.B) {
+		hll, _ := NewHllSketch(21, TgtHllType_HLL_4)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateUInt64(uint64(i))
+		}
+	})
+
+	b.Run("lgK4 HLL6 uint", func(b *testing.B) {
+		hll, _ := NewHllSketch(11, TgtHllType_HLL_6)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateUInt64(uint64(i))
+		}
+	})
+	b.Run("lgK16 HLL6 uint", func(b *testing.B) {
+		hll, _ := NewHllSketch(16, TgtHllType_HLL_6)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateUInt64(uint64(i))
+		}
+	})
+	b.Run("lgK21 HLL6 uint", func(b *testing.B) {
+		hll, _ := NewHllSketch(21, TgtHllType_HLL_6)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateUInt64(uint64(i))
+		}
+	})
+
+	b.Run("lgK4 HLL8 uint", func(b *testing.B) {
+		hll, _ := NewHllSketch(11, TgtHllType_HLL_8)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateUInt64(uint64(i))
+		}
+	})
+	b.Run("lgK16 HLL8 uint", func(b *testing.B) {
+		hll, _ := NewHllSketch(16, TgtHllType_HLL_8)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateUInt64(uint64(i))
+		}
+	})
+	b.Run("lgK21 HLL8 uint", func(b *testing.B) {
+		hll, _ := NewHllSketch(21, TgtHllType_HLL_8)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateUInt64(uint64(i))
+		}
+	})
+
+	// HLL Slice BenchMark
+	bs := make([]byte, 8)
+	b.Run("lgK4 HLL4 slice", func(b *testing.B) {
+		hll, _ := NewHllSketch(11, TgtHllType_HLL_4)
+		for i := 0; i < b.N; i++ {
+			binary.LittleEndian.PutUint64(bs, uint64(i))
+			hll.UpdateSlice(bs)
+		}
+	})
+	b.Run("lgK16 HLL4 slice", func(b *testing.B) {
+		hll, _ := NewHllSketch(16, TgtHllType_HLL_4)
+		for i := 0; i < b.N; i++ {
+			binary.LittleEndian.PutUint64(bs, uint64(i))
+			hll.UpdateSlice(bs)
+		}
+	})
+	b.Run("lgK21 HLL4 slice", func(b *testing.B) {
+		hll, _ := NewHllSketch(21, TgtHllType_HLL_4)
+		for i := 0; i < b.N; i++ {
+			binary.LittleEndian.PutUint64(bs, uint64(i))
+			hll.UpdateSlice(bs)
+		}
+	})
+
+	b.Run("lgK4 HLL6 slice", func(b *testing.B) {
+		hll, _ := NewHllSketch(11, TgtHllType_HLL_6)
+		for i := 0; i < b.N; i++ {
+			binary.LittleEndian.PutUint64(bs, uint64(i))
+			hll.UpdateSlice(bs)
+		}
+	})
+	b.Run("lgK16 HLL6 slice", func(b *testing.B) {
+		hll, _ := NewHllSketch(16, TgtHllType_HLL_6)
+		for i := 0; i < b.N; i++ {
+			binary.LittleEndian.PutUint64(bs, uint64(i))
+			hll.UpdateSlice(bs)
+		}
+	})
+	b.Run("lgK21 HLL6 slice", func(b *testing.B) {
+		hll, _ := NewHllSketch(21, TgtHllType_HLL_6)
+		for i := 0; i < b.N; i++ {
+			binary.LittleEndian.PutUint64(bs, uint64(i))
+			hll.UpdateSlice(bs)
+		}
+	})
+
+	b.Run("lgK4 HLL8 slice", func(b *testing.B) {
+		hll, _ := NewHllSketch(11, TgtHllType_HLL_8)
+		for i := 0; i < b.N; i++ {
+			binary.LittleEndian.PutUint64(bs, uint64(i))
+			hll.UpdateSlice(bs)
+		}
+	})
+	b.Run("lgK16 HLL8 slice", func(b *testing.B) {
+		hll, _ := NewHllSketch(16, TgtHllType_HLL_8)
+		for i := 0; i < b.N; i++ {
+			binary.LittleEndian.PutUint64(bs, uint64(i))
+			hll.UpdateSlice(bs)
+		}
+	})
+	b.Run("lgK21 HLL8 slice", func(b *testing.B) {
+		hll, _ := NewHllSketch(21, TgtHllType_HLL_8)
+		for i := 0; i < b.N; i++ {
+			binary.LittleEndian.PutUint64(bs, uint64(i))
+			hll.UpdateSlice(bs)
+		}
+	})
+
+	// Union benchmark
+	b.Run("lgK4 HLL8 union", func(b *testing.B) {
+		hll, _ := NewHllSketch(4, TgtHllType_HLL_8)
+		union, _ := NewUnion(4)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateSlice(bs)
+			union.UpdateSketch(hll)
+		}
+	})
+	b.Run("lgK16 HLL8 union", func(b *testing.B) {
+		hll, _ := NewHllSketch(16, TgtHllType_HLL_8)
+		union, _ := NewUnion(16)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateSlice(bs)
+			union.UpdateSketch(hll)
+		}
+	})
+	b.Run("lgK21 HLL8 union", func(b *testing.B) {
+		hll, _ := NewHllSketch(21, TgtHllType_HLL_8)
+		union, _ := NewUnion(21)
+		for i := 0; i < b.N; i++ {
+			hll.UpdateSlice(bs)
+			union.UpdateSketch(hll)
+		}
+	})
+
+}
+
+func BenchmarkHLLDataSketchWithEstimate(b *testing.B) {
+	hll, err := NewHllSketch(11, TgtHllType_HLL_8)
+	assert.NoError(b, err)
+	for i := 0; i < b.N; i++ {
+		hll.UpdateString(strconv.Itoa(i))
+	}
+	est := hll.GetEstimate()
+
+	estimate := int64(est)
+	fmt.Printf("Estimated cardinality: %d (true: %d) (error: %f)\n ", estimate, b.N, float64(int64(b.N)-estimate)*100/float64(b.N))
+}
+
+// Test the hard case for (shiftedNewValue >= AUX_TOKEN) && (rawStoredOldNibble = AUX_TOKEN)
+func TestHLL4RawStoredOldNibbleAndShiftedNewValueAuxToken(t *testing.T) {
+	hll, _ := NewHllSketch(21, TgtHllType_HLL_4)
+	for i := uint64(0); i < 29197004; i++ {
+		hll.UpdateUInt64(i)
+	}
+	hll.UpdateUInt64(29197004)
+}
diff --git a/hll/hll_utils.go b/hll/hll_utils.go
new file mode 100644
index 0000000..664130a
--- /dev/null
+++ b/hll/hll_utils.go
@@ -0,0 +1,169 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"fmt"
+	"math"
+
+	"github.com/apache/datasketches-go/common"
+)
+
+const (
+	defaultLgK     = 12
+	lgInitListSize = 3
+	lgInitSetSize  = 5
+)
+
+const (
+	minLogK         = 4
+	maxLogK         = 21
+	empty           = 0
+	keyBits26       = 26
+	valBits6        = 6
+	keyMask26       = (1 << keyBits26) - 1
+	valMask6        = (1 << valBits6) - 1
+	resizeNumber    = 3
+	resizeDenom     = 4
+	couponRSEFactor = .409 //at transition point not the asymptote
+	couponRSE       = couponRSEFactor / (1 << 13)
+	hiNibbleMask    = 0xf0
+	loNibbleMask    = 0x0f
+
+	auxToken = 0xf
+)
+
+var (
+	hllNonHipRSEFactor = math.Sqrt((3.0 * math.Log(2.0)) - 1.0) //1.03896
+	hllHipRSEFActor    = math.Sqrt(math.Log(2.0))               //.8325546
+)
+
+type TgtHllType int
+type curMode int
+
+const (
+	curMode_LIST curMode = 0
+	curMode_SET  curMode = 1
+	curMode_HLL  curMode = 2
+)
+
+const (
+	TgtHllType_HLL_4   TgtHllType = 0
+	TgtHllType_HLL_6   TgtHllType = 1
+	TgtHllType_HLL_8   TgtHllType = 2
+	TgtHllType_DEFAULT            = TgtHllType_HLL_4
+)
+
+var (
+	// lgAuxArrInts is the Log2 table sizes for exceptions based on lgK from 0 to 26.
+	//However, only lgK from 4 to 21 are used.
+	lgAuxArrInts = []int{
+		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
+	}
+)
+
+// CheckLgK checks the given lgK and returns it if it is valid and panics otherwise.
+func checkLgK(lgK int) (int, error) {
+	if lgK >= minLogK && lgK <= maxLogK {
+		return lgK, nil
+	}
+	return 0, fmt.Errorf("log K must be between 4 and 21, inclusive: %d", lgK)
+}
+
+// pair returns a value where the lower 26 bits are the slotNo and the upper 6 bits are the value.
+func pair(slotNo int, value int) int {
+	return (value << keyBits26) | (slotNo & keyMask26)
+}
+
+// pairString returns a string representation of the pair.
+func pairString(pair int) string {
+	return fmt.Sprintf("SlotNo: %d, Value: %d", getPairLow26(pair), getPairValue(pair))
+}
+
+// getPairLow26 returns the pair, the lower 26 bits of the pair.
+func getPairLow26(pair int) int {
+	return pair & keyMask26
+}
+
+// getPairValue returns the value of the pair.
+// The value is the upper 6 bits of the pair.
+func getPairValue(pair int) int {
+	return pair >> keyBits26
+}
+
+func checkNumStdDev(numStdDev int) error {
+	if numStdDev < 1 || numStdDev > 3 {
+		return fmt.Errorf("NumStdDev may not be less than 1 or greater than 3: %d", numStdDev)
+	}
+	return nil
+}
+
+// checkPreamble checks the given preamble and returns the curMode if it is valid and panics otherwise.
+func checkPreamble(preamble []byte) (curMode, error) {
+	if len(preamble) == 0 {
+		return 0, fmt.Errorf("preamble cannot be nil or empty")
+	}
+	preInts := extractPreInts(preamble)
+	if len(preamble) < (preInts * 4) {
+		return 0, fmt.Errorf("preamble length mismatch: %d, %d", len(preamble), preInts)
+	}
+	serVer := extractSerVer(preamble)
+	famId := extractFamilyID(preamble)
+	curMode := extractCurMode(preamble)
+
+	if famId != common.Family_HLL_ID {
+		return 0, fmt.Errorf("possible Corruption: Invalid Family: %d", famId)
+	}
+	if serVer != 1 {
+		return 0, fmt.Errorf("possible Corruption: Invalid Serialization Version: %d", serVer)
+	}
+
+	if preInts != listPreInts && preInts != hashSetPreInts && preInts != hllPreInts {
+		return 0, fmt.Errorf("possible Corruption: Invalid Preamble Ints: %d", preInts)
+	}
+
+	if curMode == curMode_LIST && preInts != listPreInts {
+		return 0, fmt.Errorf("possible Corruption: Invalid Preamble Ints: %d", preInts)
+	}
+
+	if curMode == curMode_SET && preInts != hashSetPreInts {
+		return 0, fmt.Errorf("possible Corruption: Invalid Preamble Ints: %d", preInts)
+	}
+
+	if curMode == curMode_HLL && preInts != hllPreInts {
+		return 0, fmt.Errorf("possible Corruption: Invalid Preamble Ints: %d", preInts)
+	}
+
+	return curMode, nil
+}
+
+func getMaxUpdatableSerializationBytes(lgConfigK int, tgtHllType TgtHllType) int {
+	var arrBytes int
+	if tgtHllType == TgtHllType_HLL_4 {
+		auxBytes := 4 << lgAuxArrInts[lgConfigK]
+		arrBytes = (1 << (lgConfigK - 1)) + auxBytes
+	} else if tgtHllType == TgtHllType_HLL_6 {
+		numSlots := 1 << lgConfigK
+		arrBytes = ((numSlots * 3) >> 2) + 1
+	} else {
+		arrBytes = 1 << lgConfigK
+	}
+	return hllByteArrStart + arrBytes
+}
diff --git a/hll/pair_iterator.go b/hll/pair_iterator.go
new file mode 100644
index 0000000..1f7a903
--- /dev/null
+++ b/hll/pair_iterator.go
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package hll
+
+type pairIterator interface {
+	nextValid() bool
+	nextAll() bool
+	getIndex() int
+	getPair() int
+	getKey() int
+	getValue() int
+	getSlot() int
+}
+
+type intArrayPairIterator struct {
+	array    []int
+	arrLen   int
+	slotMask int
+	index    int
+	pair     int
+}
+
+func (i *intArrayPairIterator) getIndex() int {
+	return i.index
+}
+
+// newIntArrayPairIterator returns a new intArrayPairIterator.
+func newIntArrayPairIterator(array []int, lgConfigK int) pairIterator {
+	return &intArrayPairIterator{
+		array:    array,
+		slotMask: (1 << lgConfigK) - 1,
+		arrLen:   len(array),
+		index:    -1,
+	}
+}
+
+// getPair returns the current key, value pair as a single int where the key is the lower 26 bits
+// and the value is in the upper 6 bits.
+func (i *intArrayPairIterator) getPair() int {
+	return i.pair
+}
+
+// nextValid returns true at the next pair where getKey() and getValue() are valid.
+// If false, the iteration is done.
+func (i *intArrayPairIterator) nextValid() bool {
+	for (i.index + 1) < i.arrLen {
+		i.index++
+		pair := i.array[i.index]
+		if pair != empty {
+			i.pair = pair
+			return true
+		}
+	}
+	return false
+}
+
+// nextAll returns true if there is another pair in the array.
+func (i *intArrayPairIterator) nextAll() bool {
+	i.index++
+	if i.index < i.arrLen {
+		i.pair = i.array[i.index]
+		return true
+	}
+	return false
+}
+
+// getKey returns the key of the pair.
+// the low 26 bits of a pair, and can be up to 26 bits in length.
+func (i *intArrayPairIterator) getKey() int {
+	return getPairLow26(i.pair)
+}
+
+// getValue returns the value of the pair.
+func (i *intArrayPairIterator) getValue() int {
+	return getPairValue(i.pair)
+}
+
+func (i *intArrayPairIterator) getSlot() int {
+	return i.getKey() & i.slotMask
+}
diff --git a/hll/preamble_utils.go b/hll/preamble_utils.go
new file mode 100644
index 0000000..137e918
--- /dev/null
+++ b/hll/preamble_utils.go
@@ -0,0 +1,287 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"encoding/binary"
+	"math"
+
+	"github.com/apache/datasketches-go/common"
+)
+
+const (
+	preambleIntsBytes = 0
+	serVerByte        = 1
+	familyByte        = 2
+	lgKByte           = 3
+	lgArrByte         = 4
+	flagsByte         = 5
+	listCountByte     = 6
+	hllCurMinByte     = 6
+	// modeByte
+	// mode encoding of combined curMode and 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
+	modeByte = 7 //lo2bits = curMode, next 2 bits = tgtHllType
+
+	listIntArrStart = 8
+)
+
+const (
+	//Coupon Hash Set
+	hashSetCountInt    = 8
+	hashSetIntArrStart = 12
+)
+
+const (
+	// HLL
+	hipAccumDouble  = 8
+	kxq0Double      = 16
+	kxq1Double      = 24
+	curMinCountInt  = 32
+	auxCountInt     = 36
+	hllByteArrStart = 40
+)
+
+const (
+	//Flag bit masks
+	emptyFlagMask           = 4
+	compactFlagMask         = 8
+	outOfOrderFlagMask      = 16
+	rebuildCurminNumKxqMask = 32
+)
+
+const (
+	//Mode byte masks
+	curModeMask    = 3
+	tgtHllTypeMask = 12
+)
+
+const (
+	// Other constants
+	serVer         = 1
+	familyId       = 7
+	listPreInts    = 2
+	hashSetPreInts = 3
+	hllPreInts     = 10
+)
+
+func extractPreInts(byteArr []byte) int {
+	return int(byteArr[preambleIntsBytes] & 0x3F)
+}
+
+func extractSerVer(byteArr []byte) int {
+	return int((byteArr[serVerByte]) & 0xFF)
+}
+
+func extractFamilyID(byteArr []byte) int {
+	return int((byteArr[familyByte]) & 0xFF)
+}
+
+func extractCurMode(byteArr []byte) curMode {
+	return curMode(byteArr[modeByte] & curModeMask)
+}
+
+func extractTgtHllType(byteArr []byte) TgtHllType {
+	typeId := byteArr[modeByte] & tgtHllTypeMask
+	return TgtHllType(typeId >> 2)
+}
+
+func extractLgK(byteArr []byte) int {
+	return int(byteArr[lgKByte] & 0xFF)
+}
+
+func extractListCount(byteArr []byte) int {
+	return int(byteArr[listCountByte] & 0xFF)
+}
+
+func extractCompactFlag(byteArr []byte) bool {
+	return (int(byteArr[flagsByte]) & compactFlagMask) > 0
+}
+
+func extractHashSetCount(byteArr []byte) int {
+	return int(binary.LittleEndian.Uint32(byteArr[hashSetCountInt : hashSetCountInt+4]))
+}
+
+func extractLgArr(byteArr []byte) int {
+	return int(byteArr[lgArrByte] & 0xFF)
+}
+
+func extractOooFlag(byteArr []byte) bool {
+	flags := byteArr[flagsByte]
+	return (flags & outOfOrderFlagMask) > 0
+}
+
+func extractCurMin(byteArr []byte) int {
+	return int(byteArr[hllCurMinByte] & 0xFF)
+}
+
+func extractHipAccum(byteArr []byte) float64 {
+	return math.Float64frombits(binary.LittleEndian.Uint64(byteArr[hipAccumDouble : hipAccumDouble+8]))
+}
+
+func extractKxQ0(byteArr []byte) float64 {
+	return math.Float64frombits(binary.LittleEndian.Uint64(byteArr[kxq0Double : kxq0Double+8]))
+}
+
+func extractKxQ1(byteArr []byte) float64 {
+	return math.Float64frombits(binary.LittleEndian.Uint64(byteArr[kxq1Double : kxq1Double+8]))
+}
+
+func extractNumAtCurMin(byteArr []byte) int {
+	return int(binary.LittleEndian.Uint32(byteArr[curMinCountInt : curMinCountInt+4]))
+}
+
+func extractRebuildCurMinNumKxQFlag(byteArr []byte) bool {
+	return (byteArr[flagsByte] & rebuildCurminNumKxqMask) > 0
+}
+
+func extractAuxCount(byteArr []byte) int {
+	return int(binary.LittleEndian.Uint32(byteArr[auxCountInt : auxCountInt+4]))
+}
+
+func computeLgArr(byteArr []byte, couponCount int, lgConfigK int) int {
+	//value is missing, recompute
+	curMode := extractCurMode(byteArr)
+	if curMode == curMode_LIST {
+		return lgInitListSize
+	}
+	ceilPwr2 := common.CeilPowerOf2(couponCount)
+	if (resizeDenom * couponCount) > (resizeNumber * ceilPwr2) {
+		ceilPwr2 <<= 1
+	}
+	if curMode == curMode_SET {
+		return max(lgInitSetSize, common.ExactLog2OfLong(uint64(ceilPwr2)))
+	}
+	//only used for HLL4
+	return max(lgAuxArrInts[lgConfigK], common.ExactLog2OfLong(uint64(ceilPwr2)))
+
+}
+
+func insertAuxCount(byteArr []byte, auxCount int) {
+	binary.LittleEndian.PutUint32(byteArr[auxCountInt:auxCountInt+4], uint32(auxCount))
+}
+
+func insertListCount(byteArr []byte, listCnt int) {
+	byteArr[listCountByte] = byte(listCnt)
+}
+
+func insertHashSetCount(byteArr []byte, hashSetCnt int) {
+	binary.LittleEndian.PutUint32(byteArr[hashSetCountInt:hashSetCountInt+4], uint32(hashSetCnt))
+}
+
+func insertPreInts(byteArr []byte, preInts int) {
+	byteArr[preambleIntsBytes] = byte(preInts & 0x3F)
+}
+
+func insertSerVer(byteArr []byte) {
+	byteArr[serVerByte] = byte(serVer)
+}
+
+func insertFamilyID(byteArr []byte) {
+	byteArr[familyByte] = byte(familyId)
+}
+
+func insertLgK(byteArr []byte, lgK int) {
+	byteArr[lgKByte] = byte(lgK)
+}
+
+func insertLgArr(byteArr []byte, lgArr int) {
+	byteArr[lgArrByte] = byte(lgArr)
+
+}
+
+func insertEmptyFlag(byteArr []byte, emptyFlag bool) {
+	flags := byteArr[flagsByte]
+	if emptyFlag {
+		flags |= emptyFlagMask
+	} else {
+		flags &= ^uint8(emptyFlagMask)
+	}
+	byteArr[flagsByte] = flags
+}
+
+func insertOooFlag(byteArr []byte, oooFlag bool) {
+	flags := byteArr[flagsByte]
+	if oooFlag {
+		flags |= outOfOrderFlagMask
+	} else {
+		flags &= ^uint8(outOfOrderFlagMask)
+	}
+	byteArr[flagsByte] = flags
+}
+
+func insertCurMode(byteArr []byte, curMode curMode) {
+	mode := byteArr[modeByte] & ^uint8(curModeMask)
+	mode |= uint8(curMode) & curModeMask
+	byteArr[modeByte] = mode
+}
+
+func insertTgtHllType(byteArr []byte, tgtHllType TgtHllType) {
+	mode := byteArr[modeByte] & ^uint8(tgtHllTypeMask)
+	mode |= (uint8(tgtHllType) << 2) & tgtHllTypeMask
+	byteArr[modeByte] = mode
+}
+
+func insertCompactFlag(byteArr []byte, compactFlag bool) {
+	flags := byteArr[flagsByte]
+	if compactFlag {
+		flags |= compactFlagMask
+	} else {
+		flags &= ^uint8(compactFlagMask)
+	}
+	byteArr[flagsByte] = flags
+}
+
+func insertCurMin(byteArr []byte, curMin int) {
+	byteArr[hllCurMinByte] = byte(curMin)
+}
+
+func insertHipAccum(byteArr []byte, hipAccum float64) {
+	binary.LittleEndian.PutUint64(byteArr[hipAccumDouble:hipAccumDouble+8], math.Float64bits(hipAccum))
+}
+
+func insertKxQ0(byteArr []byte, kxq0 float64) {
+	binary.LittleEndian.PutUint64(byteArr[kxq0Double:kxq0Double+8], math.Float64bits(kxq0))
+}
+
+func insertKxQ1(byteArr []byte, kxq1 float64) {
+	binary.LittleEndian.PutUint64(byteArr[kxq1Double:kxq1Double+8], math.Float64bits(kxq1))
+}
+
+func insertNumAtCurMin(byteArr []byte, numAtCurMin int) {
+	binary.LittleEndian.PutUint32(byteArr[curMinCountInt:curMinCountInt+4], uint32(numAtCurMin))
+}
+
+func insertRebuildCurMinNumKxQFlag(byteArr []byte, rebuild bool) {
+	flags := byteArr[flagsByte]
+	if rebuild {
+		flags |= rebuildCurminNumKxqMask
+	} else {
+		flags &= ^uint8(rebuildCurminNumKxqMask)
+	}
+	byteArr[flagsByte] = flags
+}
diff --git a/hll/relative_error_tables.go b/hll/relative_error_tables.go
new file mode 100644
index 0000000..5a8acf9
--- /dev/null
+++ b/hll/relative_error_tables.go
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package hll
+
+import "github.com/apache/datasketches-go/common"
+
+var hipLB = //sd 1, 2, 3
+[]float64{  //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
+var hipUB = //sd 1, 2, 3
+[]float64{  //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
+var nonHipLB = //sd 1, 2, 3
+[]float64{     //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
+var nonHipUB = //sd 1, 2, 3
+[]float64{     //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
+}
+
+func getRelErrKLT12(upperBound bool, oooFlag bool, lgK int, stdDev int) float64 {
+	idx := ((lgK - 4) * 3) + (stdDev - 1)
+	sw := (2 * common.BoolToInt(oooFlag)) + common.BoolToInt(upperBound)
+	var f float64
+	switch sw {
+	case 0:
+		f = hipLB[idx]
+	case 1:
+		f = hipUB[idx]
+	case 2:
+		f = nonHipLB[idx]
+	case 3:
+		f = nonHipUB[idx]
+	default:
+		f = 0
+	}
+	return f
+}
diff --git a/hll/to_slice_impl.go b/hll/to_slice_impl.go
new file mode 100644
index 0000000..5f09b62
--- /dev/null
+++ b/hll/to_slice_impl.go
@@ -0,0 +1,159 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"encoding/binary"
+	"fmt"
+)
+
+func toHllByteArr(impl hllArray, compact bool) ([]byte, error) {
+	auxBytes := 0
+	if impl.GetTgtHllType() == TgtHllType_HLL_4 {
+		auxHashMap := impl.getAuxHashMap()
+		if auxHashMap != nil {
+			if compact {
+				auxBytes = auxHashMap.getCompactSizeBytes()
+			} else {
+				auxBytes = auxHashMap.getUpdatableSizeBytes()
+			}
+		} else {
+			if compact {
+				auxBytes = 0
+			} else {
+				auxBytes = 4 << lgAuxArrInts[impl.GetLgConfigK()]
+			}
+		}
+	}
+	totalBytes := hllByteArrStart + impl.getHllByteArrBytes() + auxBytes
+	byteArr := make([]byte, totalBytes)
+	err := insertHll(impl, byteArr, compact)
+	return byteArr, err
+}
+
+func toCouponSlice(impl hllCoupon, dstCompact bool) ([]byte, error) {
+	srcCouponCount := impl.getCouponCount()
+	srcLgCouponArrInts := impl.getLgCouponArrInts()
+	srcCouponArrInts := 1 << srcLgCouponArrInts
+	list := impl.GetCurMode() == curMode_LIST
+	if dstCompact {
+		//Src Heap,   Src Updatable, Dst Compact
+		dataStart := impl.getMemDataStart()
+		bytesOut := dataStart + (srcCouponCount << 2)
+		byteArrOut := make([]byte, bytesOut)
+		copyCommonListAndSet(impl, byteArrOut)
+		insertCompactFlag(byteArrOut, dstCompact)
+		itr := impl.iterator()
+		cnt := 0
+		for itr.nextValid() {
+			p := itr.getPair()
+			binary.LittleEndian.PutUint32(byteArrOut[dataStart+(cnt<<2):dataStart+(cnt<<2)+4], uint32(p))
+			cnt++
+		}
+		if list {
+			insertListCount(byteArrOut, srcCouponCount)
+		} else {
+			insertHashSetCount(byteArrOut, srcCouponCount)
+		}
+		return byteArrOut, nil
+	} else {
+		//Src Heap, Src Updatable, Dst Updatable
+		dataStart := impl.getMemDataStart()
+		bytesOut := dataStart + (srcCouponArrInts << 2)
+		byteArrOut := make([]byte, bytesOut)
+		copyCommonListAndSet(impl, byteArrOut)
+		for _, v := range impl.getCouponIntArr() {
+			binary.LittleEndian.PutUint32(byteArrOut[dataStart:dataStart+4], uint32(v))
+			dataStart += 4
+		}
+		if list {
+			insertListCount(byteArrOut, srcCouponCount)
+		} else {
+			insertHashSetCount(byteArrOut, srcCouponCount)
+		}
+		return byteArrOut, nil
+	}
+}
+
+func copyCommonListAndSet(impl hllCoupon, dst []byte) {
+	insertPreInts(dst, impl.getPreInts())
+	insertSerVer(dst)
+	insertFamilyID(dst)
+	insertLgK(dst, impl.GetLgConfigK())
+	insertLgArr(dst, impl.getLgCouponArrInts())
+	insertEmptyFlag(dst, impl.IsEmpty())
+	insertOooFlag(dst, impl.isOutOfOrder())
+	insertCurMode(dst, impl.GetCurMode())
+	insertTgtHllType(dst, impl.GetTgtHllType())
+}
+
+func insertHll(impl hllArray, dst []byte, compact bool) error {
+	insertCommonHll(impl, dst, compact)
+	hllByteArr := impl.getHllByteArr()
+	copy(dst[hllByteArrStart:], hllByteArr)
+	if impl.getAuxHashMap() != nil {
+		insertAux(impl, dst, compact)
+	} else {
+		insertAuxCount(dst, 0)
+	}
+	return nil
+}
+
+func insertCommonHll(impl hllArray, dst []byte, compact bool) {
+	insertPreInts(dst, impl.getPreInts())
+	insertSerVer(dst)
+	insertFamilyID(dst)
+	insertLgK(dst, impl.GetLgConfigK())
+	insertEmptyFlag(dst, impl.IsEmpty())
+	insertCompactFlag(dst, compact)
+	insertOooFlag(dst, impl.isOutOfOrder())
+	insertCurMin(dst, impl.getCurMin())
+	insertCurMode(dst, impl.GetCurMode())
+	insertTgtHllType(dst, impl.GetTgtHllType())
+	insertHipAccum(dst, impl.getHipAccum())
+	insertKxQ0(dst, impl.getKxQ0())
+	insertKxQ1(dst, impl.getKxQ1())
+	insertNumAtCurMin(dst, impl.getNumAtCurMin())
+	insertRebuildCurMinNumKxQFlag(dst, impl.isRebuildCurMinNumKxQFlag())
+}
+
+func insertAux(impl hllArray, dst []byte, compact bool) {
+	auxHashMap := impl.getAuxHashMap()
+	auxCount := auxHashMap.getAuxCount()
+	insertAuxCount(dst, auxCount)
+	insertLgArr(dst, auxHashMap.getLgAuxArrInts())
+	auxStart := impl.getAuxStart()
+	if compact {
+		itr := auxHashMap.iterator()
+		cnt := 0
+		for itr.nextValid() {
+			p := itr.getPair()
+			binary.LittleEndian.PutUint32(dst[auxStart+(cnt<<2):auxStart+(cnt<<2)+4], uint32(p))
+			cnt++
+		}
+		if cnt != auxCount {
+			panic(fmt.Sprintf("corruption, should not happen: %d != %d", cnt, auxCount))
+		}
+	} else {
+		auxInts := 1 << auxHashMap.getLgAuxArrInts()
+		auxArr := auxHashMap.getAuxIntArr()
+		for i, v := range auxArr[:auxInts] {
+			binary.LittleEndian.PutUint32(dst[auxStart+(i<<2):auxStart+(i<<2)+4], uint32(v))
+		}
+	}
+}
diff --git a/hll/union.go b/hll/union.go
new file mode 100644
index 0000000..c6b1a8d
--- /dev/null
+++ b/hll/union.go
@@ -0,0 +1,395 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"github.com/apache/datasketches-go/common"
+)
+
+type Union interface {
+	publiclyUpdatable
+	estimableSketch
+	configuredSketch
+	toSliceSketch
+	privatelyUpdatable
+	UpdateSketch(sketch HllSketch)
+	GetResult(tgtHllType TgtHllType) (HllSketch, error)
+}
+
+type unionImpl struct {
+	lgMaxK int
+	gadget HllSketch
+}
+
+func (u *unionImpl) GetHipEstimate() float64 {
+	return u.gadget.GetHipEstimate()
+}
+
+func (u *unionImpl) GetUpperBound(numStdDev int) (float64, error) {
+	return u.gadget.GetUpperBound(numStdDev)
+}
+
+func (u *unionImpl) GetLowerBound(numStdDev int) (float64, error) {
+	return u.gadget.GetLowerBound(numStdDev)
+}
+
+func (u *unionImpl) couponUpdate(coupon int) hllSketchBase {
+	if coupon == empty {
+		return u.gadget.(*hllSketchImpl).sketch
+	}
+	sk := u.gadget.couponUpdate(coupon)
+	u.gadget.(*hllSketchImpl).sketch = sk
+	return sk
+}
+
+func (u *unionImpl) GetResult(tgtHllType TgtHllType) (HllSketch, error) {
+	err := checkRebuildCurMinNumKxQ(u.gadget)
+	if err != nil {
+		return nil, err
+	}
+	return u.gadget.CopyAs(tgtHllType), nil
+}
+
+func NewUnionWithDefault() (Union, error) {
+	return NewUnion(defaultLgK)
+}
+
+func NewUnion(lgMaxK int) (Union, error) {
+	sk, err := NewHllSketch(lgMaxK, TgtHllType_HLL_8)
+	if err != nil {
+		return nil, err
+	}
+	return &unionImpl{
+		lgMaxK: lgMaxK,
+		gadget: sk,
+	}, nil
+}
+
+func DeserializeUnion(byteArray []byte) (Union, error) {
+	lgK, err := checkLgK(extractLgK(byteArray))
+	if err != nil {
+		return nil, err
+	}
+	sk, e := DeserializeHllSketch(byteArray, false)
+	if e != nil {
+		return nil, e
+	}
+	union, err := NewUnion(lgK)
+	if err != nil {
+		return nil, err
+	}
+	union.UpdateSketch(sk)
+	return union, nil
+}
+
+func (u *unionImpl) GetCompositeEstimate() float64 {
+	return u.gadget.GetCompositeEstimate()
+}
+
+func (u *unionImpl) GetEstimate() float64 {
+	return u.gadget.GetEstimate()
+}
+
+func (u *unionImpl) UpdateUInt64(datum uint64) {
+	u.gadget.UpdateUInt64(datum)
+}
+
+func (u *unionImpl) UpdateInt64(datum int64) {
+	u.gadget.UpdateInt64(datum)
+}
+
+func (u *unionImpl) UpdateSlice(datum []byte) {
+	u.gadget.UpdateSlice(datum)
+}
+
+func (u *unionImpl) UpdateString(datum string) {
+	u.gadget.UpdateString(datum)
+}
+
+func (u *unionImpl) UpdateSketch(sketch HllSketch) {
+	un := u.unionImpl(sketch)
+	u.gadget.(*hllSketchImpl).sketch = un
+}
+
+func (u *unionImpl) GetLgConfigK() int {
+	return u.gadget.GetLgConfigK()
+}
+
+func (u *unionImpl) GetTgtHllType() TgtHllType {
+	return u.gadget.GetTgtHllType()
+}
+
+func (u *unionImpl) GetCurMode() curMode {
+	return u.gadget.GetCurMode()
+}
+
+func (u *unionImpl) IsEmpty() bool {
+	return u.gadget.IsEmpty()
+}
+
+func (u *unionImpl) ToCompactSlice() ([]byte, error) {
+	err := checkRebuildCurMinNumKxQ(u.gadget)
+	if err != nil {
+		return nil, err
+	}
+	return u.gadget.ToCompactSlice()
+}
+
+func (u *unionImpl) ToUpdatableSlice() ([]byte, error) {
+	err := checkRebuildCurMinNumKxQ(u.gadget)
+	if err != nil {
+		return nil, err
+	}
+	return u.gadget.ToUpdatableSlice()
+}
+
+func (u *unionImpl) GetUpdatableSerializationBytes() int {
+	return u.gadget.GetUpdatableSerializationBytes()
+}
+
+func (u *unionImpl) Reset() error {
+	return u.gadget.Reset()
+}
+
+func (u *unionImpl) unionImpl(source HllSketch) hllSketchBase {
+	if u.gadget.GetTgtHllType() != TgtHllType_HLL_8 {
+		panic("gadget must be HLL_8")
+	}
+	if source == nil || source.IsEmpty() {
+		return u.gadget.(*hllSketchImpl).sketch
+	}
+
+	gadgetC := u.gadget.(*hllSketchImpl)
+	sourceC := source.(*hllSketchImpl)
+
+	srcMode := sourceC.sketch.GetCurMode()
+	if srcMode == curMode_LIST {
+		sourceC.mergeTo(u.gadget)
+		return u.gadget.(*hllSketchImpl).sketch
+	}
+
+	srcLgK := source.GetLgConfigK()
+	gdgtLgK := u.gadget.GetLgConfigK()
+	gdgtEmpty := u.gadget.IsEmpty()
+
+	if srcMode == curMode_SET {
+		if gdgtEmpty && srcLgK == gdgtLgK {
+			un := sourceC.CopyAs(TgtHllType_HLL_8)
+			gadgetC.sketch = un.(*hllSketchImpl).sketch
+			return gadgetC.sketch
+		}
+		sourceC.mergeTo(u.gadget)
+		return gadgetC.sketch
+	}
+
+	// Hereafter, the source is in HLL mode.
+	var (
+		bits12 int
+		bit3   int
+		bit4   int
+	)
+
+	if !gdgtEmpty {
+		bits12 = int(gadgetC.GetCurMode()) << 1
+	} else {
+		bits12 = 3 << 1
+	}
+
+	if srcLgK < gdgtLgK {
+		bit3 = 0
+	}
+
+	if srcLgK > u.lgMaxK {
+		bit4 = 16
+	}
+
+	sw := bit4 | bit3 | bits12
+
+	switch sw {
+	case 0, 8, 2, 10:
+		// case 0: src <= max, src >= gdt, gdtLIST, gdtHeap
+		// case 8: src <= max, src <  gdt, gdtLIST, gdtHeap
+		// case 2: src <= max, src >= gdt, gdtSET,  gdtHeap
+		// case 10: src <= max, src <  gdt, gdtSET,  gdtHeap
+		{
+			// Action: copy src, reverse merge w/autofold, ooof=src
+			srcHll8 := sourceC.CopyAs(TgtHllType_HLL_8)
+			gadgetC.mergeTo(srcHll8.(*hllSketchImpl))
+			return srcHll8.(*hllSketchImpl).sketch
+		}
+	case 16, 18:
+		// case 16: src >  max, src >= gdt, gdtList, gdtHeap
+		// case 18: src >  max, src >= gdt, gdtSet,  gdtHeap
+		{ //Action: downsample src to MaxLgK, reverse merge w/autofold, ooof=src
+			panic("not implemented")
+		}
+	case 4, 20:
+		// case 4: src <= max, src >= gdt, gdtHLL, gdtHeap
+		// case 20: src >  max, src >= gdt, gdtHLL, gdtHeap
+		{ //Action: forward HLL merge w/autofold, ooof=True
+			//merge src(Hll4,6,8,heap/mem,Mode=HLL) -> gdt(Hll8,heap,Mode=HLL)
+			mergeHlltoHLLmode(source, u.gadget, srcLgK, gdgtLgK)
+			u.gadget.(*hllSketchImpl).sketch.putOutOfOrder(true)
+			return u.gadget.(*hllSketchImpl).sketch
+		}
+	case 12: //src <= max, src <  gdt, gdtHLL, gdtHeap
+		{ //Action: downsample gdt to srcLgK, forward HLL merge w/autofold, ooof=True
+			panic("not implemented")
+		}
+	case 6, 14:
+		// case 6: src <= max, src >= gdt, gdtEmpty, gdtHeap
+		// case 14: src <= max, src <  gdt, gdtEmpty, gdtHeap
+		{ //Action: copy src, replace gdt, ooof=src
+			srcHll8 := sourceC.CopyAs(TgtHllType_HLL_8)
+			return srcHll8.(*hllSketchImpl).sketch
+		}
+	case 22: //src >  max, src >= gdt, gdtEmpty, gdtHeap
+		{ //Action: downsample src to lgMaxK, replace gdt, ooof=src
+			panic("not implemented")
+		}
+	default:
+		panic("impossible")
+	}
+}
+
+func checkRebuildCurMinNumKxQ(sketch HllSketch) error {
+	sketchImpl := sketch.(*hllSketchImpl).sketch
+	curMode := sketch.GetCurMode()
+	tgtHllType := sketch.GetTgtHllType()
+	rebuild := sketchImpl.isRebuildCurMinNumKxQFlag()
+	if !rebuild || curMode != curMode_HLL || tgtHllType != TgtHllType_HLL_8 {
+		return nil
+	}
+
+	sketchArrImpl := sketchImpl.(*hll8ArrayImpl)
+	curMin := 64
+	numAtCurMin := 0
+	kxq0 := float64(uint64(1 << sketch.GetLgConfigK()))
+	kxq1 := 0.0
+	itr := sketchArrImpl.iterator()
+	for itr.nextAll() {
+		v := itr.getValue()
+		if v > 0 {
+			if v < 32 {
+				kxq0 += common.InvPow2(v) - 1.0
+			} else {
+				kxq1 += common.InvPow2(v) - 1.0
+			}
+		}
+		if v > curMin {
+			continue
+		}
+		if v < curMin {
+			curMin = v
+			numAtCurMin = 1
+		} else {
+			numAtCurMin++
+		}
+	}
+
+	sketchArrImpl.putKxQ0(kxq0)
+	sketchArrImpl.putKxQ1(kxq1)
+	sketchArrImpl.putCurMin(curMin)
+	sketchArrImpl.putNumAtCurMin(numAtCurMin)
+	sketchArrImpl.putRebuildCurMinNumKxQFlag(false)
+	//HipAccum is not affected
+	return nil
+}
+
+func mergeHlltoHLLmode(src HllSketch, tgt HllSketch, srcLgK int, tgtLgK int) {
+	sw := 0
+	if srcLgK > tgtLgK {
+		sw |= 4
+	}
+	if src.GetTgtHllType() != TgtHllType_HLL_8 {
+		sw |= 8
+	}
+	srcK := 1 << srcLgK
+
+	switch sw {
+	case 0: //HLL_8, srcLgK=tgtLgK, src=heap, tgt=heap
+		{
+			srcArr := src.(*hllSketchImpl).sketch.(*hll8ArrayImpl).hllByteArr
+			tgtArr := tgt.(*hllSketchImpl).sketch.(*hll8ArrayImpl).hllByteArr
+			for i := 0; i < srcK; i++ {
+				srcV := srcArr[i]
+				tgtV := tgtArr[i]
+				tgtArr[i] = max(srcV, tgtV)
+			}
+		}
+	case 8, 9: //!HLL_8, srcLgK=tgtLgK, src=heap, tgt=heap/mem
+		{
+			tgtAbsHllArr := tgt.(*hllSketchImpl).sketch.(*hll8ArrayImpl)
+			if src.GetTgtHllType() == TgtHllType_HLL_4 {
+				src4 := src.(*hllSketchImpl).sketch.(*hll4ArrayImpl)
+				auxHashMap := src4.auxHashMap
+				curMin := src4.curMin
+				i := 0
+				j := 0
+				for j < srcK {
+					b := src4.hllByteArr[i]
+					i++
+					value := uint(b) & loNibbleMask
+					if value == auxToken {
+						v := auxHashMap.mustFindValueFor(j)
+						tgtAbsHllArr.updateSlotNoKxQ(j, v)
+					} else {
+						tgtAbsHllArr.updateSlotNoKxQ(j, int(value)+curMin)
+					}
+					j++
+					value = uint(b) >> 4
+					if value == auxToken {
+						v := auxHashMap.mustFindValueFor(j)
+						tgtAbsHllArr.updateSlotNoKxQ(j, v)
+					} else {
+						tgtAbsHllArr.updateSlotNoKxQ(j, int(value)+curMin)
+					}
+					j++
+				}
+			} else {
+				src6 := src.(*hllSketchImpl).sketch.(*hll6ArrayImpl)
+				i := 0
+				j := 0
+				for j < srcK {
+					b1 := src6.hllByteArr[i]
+					b2 := src6.hllByteArr[i+1]
+					b3 := src6.hllByteArr[i+2]
+					i += 3
+					value := uint(b1) & 0x3f
+					tgtAbsHllArr.updateSlotNoKxQ(j, int(value))
+					j++
+					value = uint(b1) >> 6
+					value |= (uint(b2) & 0x0f) << 2
+					tgtAbsHllArr.updateSlotNoKxQ(j, int(value))
+					j++
+					value = uint(b2) >> 4
+					value |= (uint(b3) & 3) << 4
+					tgtAbsHllArr.updateSlotNoKxQ(j, int(value))
+					j++
+					value = uint(b3) >> 2
+					tgtAbsHllArr.updateSlotNoKxQ(j, int(value))
+					j++
+				}
+			}
+		}
+		// TODO continue implementing
+	default:
+		panic("not implemented")
+	}
+	tgt.(*hllSketchImpl).sketch.putRebuildCurMinNumKxQFlag(true)
+}
diff --git a/hll/union_test.go b/hll/union_test.go
new file mode 100644
index 0000000..641e41c
--- /dev/null
+++ b/hll/union_test.go
@@ -0,0 +1,312 @@
+/*
+ * 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.
+ */
+
+package hll
+
+import (
+	"fmt"
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+var nArr = []int{1, 3, 10, 30, 100, 300, 1000, 3000, 10000, 30000}
+
+func TestUnions(t *testing.T) {
+	//HLL_4: t=0,  HLL_6: t=1, HLL_8: t=2
+	const (
+		t1 = 2 //type = HLL_8
+		t2 = 2
+		rt = 2 //result type
+	)
+
+	const (
+		lgK1   = 7
+		lgK2   = 7
+		lgMaxK = 7
+		n1     = 7
+		n2     = 7
+	)
+
+	checkBasicUnion(t, n1, n2, lgK1, lgK2, lgMaxK, t1, t2, rt)
+}
+
+func checkBasicUnion(t *testing.T, n1 int, n2 int, lgK1 int, lgK2 int, lgMaxK int, t1 int, t2 int, rt int) {
+	v := 0
+	tot := n1 + n2
+
+	type1 := TgtHllType(t1)
+	type2 := TgtHllType(t2)
+	resultType := TgtHllType(rt)
+
+	h1, err := NewHllSketch(lgK1, type1)
+	assert.NoError(t, err)
+	h2, err := NewHllSketch(lgK1, type2)
+	assert.NoError(t, err)
+
+	lgControlK := min(min(lgK1, lgK2), lgMaxK)
+	control, err := NewHllSketch(lgControlK, resultType)
+	assert.NoError(t, err)
+
+	for i := 0; i < n1; i++ {
+		h1.UpdateInt64(int64(v + i))
+		control.UpdateInt64(int64(v + i))
+	}
+	v += n1
+	for i := 0; i < n2; i++ {
+		h2.UpdateInt64(int64(v + i))
+		control.UpdateInt64(int64(v + i))
+	}
+	//v += n2
+
+	union, err := NewUnion(lgMaxK)
+	assert.NoError(t, err)
+	union.UpdateSketch(h1)
+	union.UpdateSketch(h2)
+	result, err := union.GetResult(resultType)
+	assert.NoError(t, err)
+
+	uEst := result.GetEstimate()
+	uUb, err := result.GetUpperBound(2)
+	assert.NoError(t, err)
+	uLb, err := result.GetLowerBound(2)
+	assert.NoError(t, err)
+	//rErr := ((uEst / float64(tot)) - 1.0) * 100
+	//
+	//mode1 := h1.GetCurMode()
+	//mode2 := h2.GetCurMode()
+	//modeR := result.GetCurMode()
+
+	// Control
+	controlEst := control.GetEstimate()
+	controlUb, err := control.GetUpperBound(2)
+	assert.NoError(t, err)
+	controlLb, err := control.GetLowerBound(2)
+	assert.NoError(t, err)
+	//h1ooo := h1.isOutOfOrderFlag()
+	//h1ooo := h2.isOutOfOrderFlag()
+	//resultooo := result.isOutOfOrderFlag()
+
+	assert.True(t, controlUb-controlEst >= 0)
+	assert.True(t, uUb-uEst >= 0)
+	assert.True(t, controlEst-controlLb >= 0)
+	assert.True(t, uEst-uLb >= 0)
+
+	assert.Equal(t, 7, result.GetLgConfigK())
+	est := result.GetEstimate()
+	assert.InDelta(t, tot, est, float64(tot)*0.03)
+}
+
+func TestToFromUnion1(t *testing.T) {
+	for i := 0; i < 10; i++ {
+		n := nArr[i]
+		for lgK := 4; lgK <= 13; lgK++ {
+			toFrom1(t, lgK, TgtHllType_HLL_4, n)
+			toFrom1(t, lgK, TgtHllType_HLL_6, n)
+			toFrom1(t, lgK, TgtHllType_HLL_8, n)
+		}
+	}
+}
+
+func toFrom1(t *testing.T, lgK int, tgtHllType TgtHllType, n int) {
+	srcU, err := NewUnion(lgK)
+	assert.NoError(t, err)
+	srcSk, err := NewHllSketch(lgK, tgtHllType)
+	assert.NoError(t, err)
+	for i := 0; i < n; i++ {
+		srcSk.UpdateInt64(int64(i))
+	}
+	srcU.UpdateSketch(srcSk)
+	fmt.Printf("n: %d, lgK: %d, type: %d\n", n, lgK, tgtHllType)
+
+	byteArr, err := srcU.ToCompactSlice()
+	assert.NoError(t, err)
+	dstU, _ := DeserializeUnion(byteArr)
+
+	dstUest := dstU.GetEstimate()
+	srcUest := srcU.GetEstimate()
+
+	assert.Equal(t, dstUest, srcUest)
+}
+
+func TestUnionCompositeEst(t *testing.T) {
+	u, err := NewUnionWithDefault()
+	assert.NoError(t, err)
+	est := u.GetCompositeEstimate()
+	assert.Equal(t, est, 0.0)
+	for i := 1; i <= 15; i++ {
+		u.UpdateInt64(int64(i))
+	}
+	est = u.GetCompositeEstimate()
+
+	assert.InDelta(t, est, 15.0, 15.0*0.03)
+	for i := 15; i <= 1000; i++ {
+		u.UpdateInt64(int64(i))
+	}
+	est = u.GetCompositeEstimate()
+	assert.InDelta(t, est, 1000.0, 1000.0*0.03)
+}
+
+func TestDeserialize1k(t *testing.T) {
+	u, err := NewUnion(16)
+	assert.NoError(t, err)
+	for i := 0; i < (1 << 10); i++ {
+		u.UpdateInt64(int64(i))
+	}
+	expected := u.GetEstimate()
+	byteArr, err := u.ToUpdatableSlice()
+	assert.NoError(t, err)
+	u2, e := DeserializeUnion(byteArr)
+	assert.NoError(t, e)
+	est := u2.GetEstimate()
+	assert.Equal(t, expected, est)
+}
+
+func TestDeserialize1M(t *testing.T) {
+	u, err := NewUnion(16)
+	assert.NoError(t, err)
+	for i := 0; i < (1 << 20); i++ {
+		u.UpdateInt64(int64(i))
+	}
+	expected := u.GetEstimate()
+	byteArr, err := u.ToUpdatableSlice()
+	assert.NoError(t, err)
+	u2, e := DeserializeUnion(byteArr)
+	assert.NoError(t, e)
+	est := u2.GetEstimate()
+	assert.Equal(t, expected, est)
+}
+
+func TestEmptyCouponMisc(t *testing.T) {
+	lgK := 8
+	u, err := NewUnion(lgK)
+	assert.NoError(t, err)
+	for i := 0; i < 20; i++ {
+		u.UpdateInt64(int64(i))
+	}
+	u.couponUpdate(0)
+	est := u.GetEstimate()
+	assert.InDelta(t, est, 20.0, 0.001)
+	assert.Equal(t, u.GetTgtHllType(), TgtHllType_HLL_8)
+	bytes := u.GetUpdatableSerializationBytes()
+	assert.True(t, bytes <= getMaxUpdatableSerializationBytes(lgK, TgtHllType_HLL_8))
+}
+
+func TestUnionWithWrap(t *testing.T) {
+	lgK := 4
+	type1 := TgtHllType_HLL_4
+	n := 2
+	sk, err := NewHllSketch(lgK, type1)
+	assert.NoError(t, err)
+	for i := 0; i < n; i++ {
+		sk.UpdateInt64(int64(i))
+	}
+	est := sk.GetEstimate()
+	skByteArr, err := sk.ToCompactSlice()
+	assert.NoError(t, err)
+
+	sk2, _ := DeserializeHllSketch(skByteArr, false)
+	est2 := sk2.GetEstimate()
+	assert.Equal(t, est2, est)
+
+	u, err := NewUnion(lgK)
+	assert.NoError(t, err)
+	u.UpdateSketch(sk2)
+	estU := u.GetEstimate()
+	assert.Equal(t, estU, est)
+}
+
+func TestUnionWithWrap2(t *testing.T) {
+	lgK := 10
+	n := 128
+	sk, err := NewHllSketchDefault(lgK)
+	assert.NoError(t, err)
+	for i := 0; i < n; i++ {
+		sk.UpdateInt64(int64(i))
+	}
+	est := sk.GetEstimate()
+	skByteArr, err := sk.ToCompactSlice()
+	assert.NoError(t, err)
+
+	sk2, _ := DeserializeHllSketch(skByteArr, false)
+	sk2Est := sk2.GetEstimate()
+	assert.Equal(t, sk2Est, est)
+
+	u, err := NewUnion(lgK)
+	assert.NoError(t, err)
+	u.UpdateSketch(sk2)
+	estU := u.GetEstimate()
+	assert.Equal(t, estU, est)
+}
+
+func TestConversions(t *testing.T) {
+	lgK := 4
+	sk1, err := NewHllSketch(lgK, TgtHllType_HLL_8)
+	assert.NoError(t, err)
+	sk2, err := NewHllSketch(lgK, TgtHllType_HLL_8)
+	assert.NoError(t, err)
+	u := 1 << 20
+	for i := 0; i < u; i++ {
+		sk1.UpdateInt64(int64(i))
+		sk2.UpdateInt64(int64(i + u))
+	}
+	union, err := NewUnion(lgK)
+	assert.NoError(t, err)
+	union.UpdateSketch(sk1)
+	union.UpdateSketch(sk2)
+	rsk1, err := union.GetResult(TgtHllType_HLL_8)
+	assert.NoError(t, err)
+	rsk2, err := union.GetResult(TgtHllType_HLL_6)
+	assert.NoError(t, err)
+	rsk3, err := union.GetResult(TgtHllType_HLL_4)
+	assert.NoError(t, err)
+	est1 := rsk1.GetEstimate()
+	est2 := rsk2.GetEstimate()
+	est3 := rsk3.GetEstimate()
+	assert.Equal(t, est2, est1)
+	assert.Equal(t, est3, est1)
+}
+
+func TestCheckUnionDeserializeRebuildAfterMerge(t *testing.T) {
+	lgK := 12
+	//Build 2 sketches in HLL (dense) mode.
+	u := 1 << (lgK - 3) //(lgK < 8) ? 16 : 1 << (lgK - 3) //allows changing lgK above
+	sk1, err := NewHllSketchDefault(lgK)
+	assert.NoError(t, err)
+	sk2, err := NewHllSketchDefault(lgK)
+	assert.NoError(t, err)
+	for i := 0; i < u; i++ {
+		sk1.UpdateInt64(int64(i))
+		sk2.UpdateInt64(int64(i + u))
+	}
+	union1, err := NewUnion(lgK)
+	assert.NoError(t, err)
+	union1.UpdateSketch(sk1)
+	union1.UpdateSketch(sk2) //oooFlag = Rebuild_KxQ = TRUE
+	rebuild := union1.(*unionImpl).gadget.(*hllSketchImpl).sketch.(*hll8ArrayImpl).isRebuildCurMinNumKxQFlag()
+	hipAccum := union1.(*unionImpl).gadget.(*hllSketchImpl).sketch.(*hll8ArrayImpl).hipAccum
+	assert.True(t, rebuild)
+	assert.Equal(t, hipAccum, 0.0)
+	//Deserialize byteArr as if it were a sketch, but it is actually a union!
+	sl, err := union1.ToUpdatableSlice() //forces rebuild
+	assert.NoError(t, err)
+	sk3, e := DeserializeHllSketch(sl, false) //rebuilds sk3
+	assert.NoError(t, e)
+	rebuild = sk3.(*hllSketchImpl).sketch.(*hll8ArrayImpl).isRebuildCurMinNumKxQFlag()
+	assert.False(t, rebuild)
+
+}
diff --git a/main.go b/main.go
new file mode 100644
index 0000000..066066e
--- /dev/null
+++ b/main.go
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */
+
+package datatasketches_go
diff --git a/thetacommon/theta_utils.go b/thetacommon/theta_utils.go
new file mode 100644
index 0000000..a952dd8
--- /dev/null
+++ b/thetacommon/theta_utils.go
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+package thetacommon
+
+const (
+	DEFAULT_UPDATE_SEED = uint32(9001)
+)


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