You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@datasketches.apache.org by le...@apache.org on 2020/08/25 18:44:22 UTC

[incubator-datasketches-java] 03/05: interim

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

leerho pushed a commit to branch RelativeErrorQuantiles
in repository https://gitbox.apache.org/repos/asf/incubator-datasketches-java.git

commit 42feebcaf8d6d6f776f6e97ad6b22cad715f36b6
Author: Lee Rhodes <le...@users.noreply.github.com>
AuthorDate: Sat Aug 22 17:10:37 2020 -0700

    interim
---
 .../apache/datasketches/kll/RelativeCompactor.java | 110 -------
 .../datasketches/kll/RelativeErrorSketch.java      | 184 -----------
 .../kll/RelativeErrorSketchBuilder.java            | 129 --------
 .../java/org/apache/datasketches/req/Buffer.java   | 343 +++++++++++++++++++++
 .../apache/datasketches/req/RelativeCompactor.java | 264 ++++++++++++++++
 .../datasketches/req/RelativeErrorSketch.java      | 264 ++++++++++++++++
 .../req/RelativeErrorSketchIterator.java           |  84 +++++
 .../package-info.java}                             |  14 +-
 .../datasketches/kll/RelativeErrorSketchTest.java} |  25 +-
 .../org/apache/datasketches/req/BufferTest.java    | 146 +++++++++
 .../datasketches/req/RelativeErrorSketchTest.java} |  25 +-
 11 files changed, 1130 insertions(+), 458 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/kll/RelativeCompactor.java b/src/main/java/org/apache/datasketches/kll/RelativeCompactor.java
deleted file mode 100644
index 9d1b4be..0000000
--- a/src/main/java/org/apache/datasketches/kll/RelativeCompactor.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * 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 org.apache.datasketches.kll;
-
-import static org.apache.datasketches.kll.RelativeErrorUtil.INIT_NUMBER_OF_SECTIONS;
-
-import java.util.Arrays;
-
-import org.apache.datasketches.kll.RelativeErrorUtil.Schedule;
-
-/**
- * @author Lee Rhodes
- */
-@SuppressWarnings({"javadoc","unused"})
-public class RelativeCompactor {
-  private int numCompactions = 0; //number of compaction operations performed
-  private int state = 0; //state of the deterministic compaction schedule
-  private int offset = 0; //0 or 1 uniformly at random in each compaction
-  private float[] buf;
-  private boolean sorted = false;
-  private int numValues = 0;
-
-  //extracted from constructor
-  private boolean alternate = true; //every other compaction has the opposite offset
-  private int sectionSize = 32;
-  private int numSections = INIT_NUMBER_OF_SECTIONS; //# of sections in the buffer
-  private int always = sectionSize;
-  private int never = sectionSize * numSections;
-  private boolean neverGrows = true;
-  private int height = 0;
-  private Schedule schedule = Schedule.DETERMINISTIC;
-
-  //derived
-  private float sectionSizeF = sectionSize;
-
-  //Empty Constructor, assume all defaults
-  public RelativeCompactor() { }
-
-  //Constructor
-  public RelativeCompactor(
-      final Schedule schedule,
-      final int sectionSize,
-      final int numSections,
-      final int always,
-      final int never,
-      final boolean neverGrows,
-      final int height,
-      final boolean alternate) {
-    this.schedule = schedule;
-    this.sectionSize = sectionSize;
-    this.numSections = numSections;
-    this.always = always;
-    this.never = never;
-    this.neverGrows = neverGrows;
-    this.height = height;
-    this.alternate = alternate;
-    final int cap = never + (numSections * sectionSize) + always;
-    buf = new float[cap];
-  }
-
-  public float[] getBuf() { return buf; }
-
-  public int getNumValues() { return numValues; }
-
-  public void incrementNumValus() { numValues++; }
-
-  public void compact() {
-    //assert
-  }
-
-  public int capacity() {
-    final int cap = never + (numSections * sectionSize) + always;
-    assert cap > 1;
-    return cap;
-  }
-
-  public int rank(final float value) { //one-based
-    sort();
-    int count = 0;
-    for (int i = 0; i < numValues; i++) {
-      if (buf[i] <= value) { count++; continue; }
-      break;
-    }
-    return count;
-  }
-
-  public void sort() {
-    if (!sorted) {
-      Arrays.sort(buf, 0, numValues);
-      sorted = true;
-    }
-  }
-}
diff --git a/src/main/java/org/apache/datasketches/kll/RelativeErrorSketch.java b/src/main/java/org/apache/datasketches/kll/RelativeErrorSketch.java
deleted file mode 100644
index 82addb2..0000000
--- a/src/main/java/org/apache/datasketches/kll/RelativeErrorSketch.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * 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 org.apache.datasketches.kll;
-
-import static java.lang.Math.ceil;
-import static java.lang.Math.sqrt;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.datasketches.kll.RelativeErrorUtil.Schedule;
-
-/**
- * Proof-of-concept code for paper "Relative Error Streaming Quantiles",
- * https://arxiv.org/abs/2004.01668.
- *
- * <p>This implementation differs from the algorithm described in the paper in the following:</p>
- * <ul><li>The algorithm requires no upper bound on the stream length (input size).
- * Instead, each relative-compactor (i.e. buffer) counts the number of compaction operations performed
- * so far (variable numCompactions). Initially, the relative-compactor starts with 2 buffer sections
- * and each time the numCompactions exceeds 2^{# of sections}, we double the number of sections
- * (variable numSections).</li>
- * <li>The size of each buffer section (variable sectionSize in the code and parameter k in the paper)
- * is initialized with a value set by the user via variable sectionSize (parameter -sec)
- * or via setting epsilon (parameter -eps). Setting the failure probability delta is not implememnted.
- * When the number of sections doubles, we decrease sectionSize by a factor of sqrt(2)
- * (for which we use a float variable sectionSizeF). As in item 1), this is applied
- * at each level separately.
- * Thus, when we double the number of section, the buffer size increases by a factor of sqrt(2)
- * (up to +-1 after rounding). For experimental purposes, the buffer consists of three parts:
- * <ul><li>a part that is never compacted (its size can be set by variable never),</li>
- * <li>numSections many sections of size sectionSize, and</li>
- * <li>a part that is always involved in a compaction (its size can be set by variable always).</li>
- * </ul></li>
- * <li>The merge operation here does not perform "special compactions", which are used in the paper
- * to allow for a tight analysis of the sketch.</li>
- * </ul>
- *
- * @author Edo Liberty
- * @author Pavel Vesely
- * @author Lee Rhodes
- */
-@SuppressWarnings("unused")
-public class RelativeErrorSketch {
-
-  static {
-    final double x = sqrt(2);
-    final double y = ceil(x);
-    final Random rand =  new Random();
-  }
-
-  //final class parameters
-  private final double eps; //default = DEFAULT_EPS = .01
-  private final Schedule schedule; //default = Schedule.DETERMINISTIC;
-  private final int initNumSections; //default = INIT_NUMBER_OF_SECTIONS;
-  private final boolean lazy; //default = true;
-  private final boolean alternate;//default = true;
-  private final boolean neverGrows; //default = false;
-
-  //class variables
-  private int always;
-  private int never; //the part that is never compacted. Default = sectionSize * numSections
-  private int sectionSize; //the number of sections & determined by eps
-
-  private List<RelativeCompactor> compactors = new ArrayList<>();
-  private int H = 0;
-  private int size = 0;
-  private int maxSize = 0;
-
-  /**
-   * Constructor.
-   * @param eps target error rate. Must be less than .1. Default = .01.
-   * @param sch schedule, whether deterministic or randomized. Default = deterministic.
-   * @param always the size of the buffer that is always compacted
-   * @param never the size of the buffer that is never compacted
-   * @param sectionSize the size of a section in the buffer
-   * @param initNumSections the initial number of sections. Default = 2.
-   * @param lazy if true, stop compressing after the first compressible candidate found.
-   * @param alternate if true, then random choice of odd/even done every other time.
-   * @param neverGrows if true, we do not let "never" grow.
-   */
-  RelativeErrorSketch(
-      final double eps,
-      final Schedule schedule,
-      final int always,
-      final int never,
-      final int sectionSize,
-      final int initNumSections,
-      final boolean lazy,
-      final boolean alternate,
-      final boolean neverGrows) {
-    this.eps = eps;
-    this.schedule = schedule;
-    this.always = always;
-    this.never = never;
-    this.sectionSize = sectionSize;
-    //an initial upper bound on log_2 of the number of compactions
-    this.initNumSections = initNumSections;
-    this.lazy = lazy;
-    this.alternate = alternate;
-    this.neverGrows = neverGrows;
-
-    H = 0;
-    size = 0;
-    maxSize = 0;
-    grow();
-  }
-
-  void grow() {
-    compactors.add(
-        new RelativeCompactor(schedule,
-                              sectionSize,
-                              initNumSections,
-                              always,
-                              never,
-                              neverGrows,
-                              H,
-                              alternate
-        ));
-    H = compactors.size();
-    updateMaxSize();
-  }
-
-  void updateMaxSize() {
-    int maxSize = 0;
-    for (RelativeCompactor c : compactors) { maxSize += c.capacity(); }
-    this.maxSize = maxSize;
-  }
-
-  void update(final float item) {
-    final RelativeCompactor c = compactors.get(0);
-
-  }
-
-  void compress(final boolean lazy) {
-
-  }
-
-  void mergeIntoSelf(final RelativeErrorSketch sketch) {
-
-  }
-
-  void merge(final RelativeErrorSketch sketch1, final RelativeErrorSketch sketch2) {
-
-  }
-
-  int rank(final float value) {
-    return 0;
-  }
-
-  Pair[] cdf() {
-    return null;
-  }
-
-  Pair[] ranks() {
-    return null;
-  }
-
-  class Pair {
-    float rank;
-    float value;
-  }
-
-}
-
-
diff --git a/src/main/java/org/apache/datasketches/kll/RelativeErrorSketchBuilder.java b/src/main/java/org/apache/datasketches/kll/RelativeErrorSketchBuilder.java
deleted file mode 100644
index e7601af..0000000
--- a/src/main/java/org/apache/datasketches/kll/RelativeErrorSketchBuilder.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * 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 org.apache.datasketches.kll;
-
-import static org.apache.datasketches.kll.RelativeErrorUtil.DEFAULT_EPS;
-import static org.apache.datasketches.kll.RelativeErrorUtil.EPS_UPPER_BOUND;
-import static org.apache.datasketches.kll.RelativeErrorUtil.INIT_NUMBER_OF_SECTIONS;
-import static org.apache.datasketches.kll.RelativeErrorUtil.NEVER_SIZE_SCALAR;
-import static org.apache.datasketches.kll.RelativeErrorUtil.SECTION_SIZE_SCALAR;
-
-import org.apache.datasketches.SketchesArgumentException;
-import org.apache.datasketches.kll.RelativeErrorUtil.Schedule;
-
-/**
- * @author Lee Rhodes
- */
-public class RelativeErrorSketchBuilder {
-  private double bEps = DEFAULT_EPS;
-  private Schedule bSchedule = Schedule.DETERMINISTIC;
-  private int bInitNumSections = INIT_NUMBER_OF_SECTIONS;
-  private boolean bLazy = true;
-  private boolean bAlternate = true;
-  private boolean bNeverGrows = false;
-  private int bAlways = -1;
-  private int bNever = -1;
-  private int bSectionSize = -1;
-
-  /**
-   * Default constructor
-   */
-  public RelativeErrorSketchBuilder() {}
-
-  /**
-   * Builds a RelativeErrorSketch
-   * @return a RelativeErrorSketch
-   */
-  public RelativeErrorSketch build() {
-    // default setting of sectionSize, always, and "never", according to eps
-    if (bSectionSize == -1) {
-      // ensured to be even and positive (thus >= 2)
-      bSectionSize = 2 * ((int)(SECTION_SIZE_SCALAR / bEps) + 1);
-    }
-    if (bAlways == -1) { bAlways = bSectionSize; }
-
-    bNeverGrows = false; //if never is set true by the user, then we do not let it grow
-    if (bNever == -1) {
-      bNever = (int)(NEVER_SIZE_SCALAR * bSectionSize * bInitNumSections);
-      bNeverGrows = true;
-    }
-    return new RelativeErrorSketch(bEps, bSchedule, bAlways, bNever, bSectionSize, bInitNumSections,
-        bLazy, bAlternate, bNeverGrows);
-  }
-
-  /**
-   * Set the target error rate. Must be less than .1. Default = .01.
-   * @param eps the target error rate
-   */
-  public void setEps(final double eps) {
-    if (eps > EPS_UPPER_BOUND) {
-      throw new SketchesArgumentException("eps must be at most " + EPS_UPPER_BOUND);
-    }
-    bEps = eps;
-  }
-
-  /**
-   * Set whether DETERMINISTIC, RANDOMIZED or RANDOMIZED_LINAR. Default = DETERMINISTIC.
-   * @param schedule whether DETERMINISTIC, RANDOMIZED or RANDOMIZED_LINAR.
-   */
-  public void setSchedule(final Schedule schedule) {
-    bSchedule = schedule;
-  }
-
-  /**
-   * Sets the initial number of sections. Default = 2.
-   * @param initNumSections the initial number of sections.
-   */
-  public void setInitNumSections(final int initNumSections) {
-    bInitNumSections = initNumSections;
-  }
-
-  /**
-   * Sets lazy.
-   * @param lazy if true, stop compressing after the first compressible candidate found.
-   */
-  public void setLazy(final boolean lazy) {
-    bLazy = lazy;
-  }
-
-  /**
-   * Sets alternate.
-   * @param alternate if true, then random choice of odd/even done every other time.
-   */
-  public void setAlternate(final boolean alternate) {
-    bAlternate = alternate;
-  }
-
-  /**
-   * Sets always
-   * @param always the size of the buffer that is always compacted
-   */
-  public void setAlways(final int always) {
-    bAlways = always;
-  }
-
-  /**
-   * Sets never
-   * @param never the size of the buffer that is never compacted
-   */
-  public void setNever(final int never) {
-    bNever = never;
-  }
-}
diff --git a/src/main/java/org/apache/datasketches/req/Buffer.java b/src/main/java/org/apache/datasketches/req/Buffer.java
new file mode 100755
index 0000000..52bc28e
--- /dev/null
+++ b/src/main/java/org/apache/datasketches/req/Buffer.java
@@ -0,0 +1,343 @@
+/*
+ * Copyright 2015, Yahoo! Inc.
+ * Licensed under the terms of the Apache License 2.0. See LICENSE file at the project root for terms.
+ */
+
+package org.apache.datasketches.req;
+
+import static java.lang.Math.max;
+
+import java.util.Arrays;
+
+import org.apache.datasketches.SketchesArgumentException;
+
+/**
+ * A special buffer of floats.
+ *
+ * @author Lee Rhodes
+ */
+class Buffer {
+  private float[] arr_;
+  private int count_;
+  private int delta_;
+  private int capacity_;
+  private boolean sorted_;
+
+  /**
+   * Constructs a new Buffer with a default size of 1024 items.
+   */
+  Buffer() {
+    this(1024, 256);
+  }
+
+  /**
+   * Constructs an empty Buffer with an initial capacity specified by
+   * the <code>capacity</code> argument.
+   *
+   * @param capacity the initial capacity.
+   * @param delta add space in increments of this size
+   */
+  Buffer(final int capacity, final int delta) {
+    arr_ = new float[capacity];
+    count_ = 0;
+    delta_ = delta;
+    capacity_ = capacity;
+    sorted_ = false;
+  }
+
+  /**
+   * Copy Constructor
+   * @param buf the Buffer to be copied into this one
+   */
+  Buffer(final Buffer buf) {
+    arr_ = buf.arr_.clone();
+    count_ = buf.count_;
+    capacity_ = buf.capacity_;
+    sorted_ = buf.sorted_;
+  }
+
+  /**
+   * Appends the given item to the end of the active array and increments length().
+   * This will expand the array if necessary. //TODO add delta?
+   * @param item the given item
+   * @return this
+   */
+  Buffer append(final float item) {
+    ensureSpace(1);
+    arr_[count_++] = item;
+    sorted_ = false;
+    return this;
+  }
+
+  /**
+   * Returns the current capacity of this Buffer. The capacity is the total amount of storage
+   * available.
+   *
+   * @return the current capacity
+   */
+  int capacity() {
+    return capacity_;
+  }
+
+  /**
+   * Returns count of items less-than the given value.
+   * @param value the given value
+   * @return count of items less-than the given value.
+   */
+  int countLessThan(final float value) {
+    int num = 0;
+    if (sorted_) {
+      for (int i = 0; i < count_; i++) {
+        if (arr_[i] < value) { num++; }
+        else { break; }
+      }
+    } else {
+      for (int i = 0; i < count_; i++) {
+        if (arr_[i] < value) { num++; }
+      }
+    }
+    return num;
+  }
+
+  /**
+   * Clears a range of the buffer
+   * @param start the start of the range, inclusive
+   * @param end the end of the range, exclusive
+   * @return this
+   */
+  Buffer clear(final int start, final int end) {
+    for (int i = start; i < end; i++) { arr_[i] = 0; }
+    return this;
+  }
+
+  /**
+   * Ensures that the capacity of this Buffer is at least newCapacity.
+   * If newCapacity &lt; capacity(), no action is taken.
+   * @return this
+   */
+  Buffer ensureCapacity(final int newCapacity) {
+    if (newCapacity > capacity_) {
+      arr_ = Arrays.copyOf(arr_, newCapacity);
+      capacity_ = newCapacity;
+    }
+    return this;
+  }
+
+  /**
+   * Ensures that the space remaining (capacity() - length()) is at least the given space.
+   * @param space the requested space remaining
+   * @return this
+   */
+  Buffer ensureSpace(final int space) {
+    if ((count_ + space) > arr_.length) {
+      ensureCapacity(count_ + max(space, delta_));
+    }
+    return this;
+  }
+
+  /**
+   * Extends the given item array starting at length(). This will expand the Buffer if necessary.
+   * @param floatArray the given item array
+   * @return this Buffer
+   */
+  Buffer extend(final float[] floatArray) {
+    final int len = floatArray.length;
+    ensureSpace(len);
+    System.arraycopy(floatArray, 0, arr_, count_, len); //TODO a merge sort instead!
+    count_ += len;
+    sorted_ = false;
+    return this;
+  }
+
+  /**
+   * Append other buffer to this buffer. Items beyond length() are ignored.
+   * @param other the other buffer
+   * @return this
+   */
+  Buffer extend(final Buffer other) { //may not need this
+    final int len = other.length();
+    ensureSpace(len);
+    System.arraycopy(other.getArray(), 0, arr_, length(), len);
+    count_ += len;
+    sorted_ = false;
+    return this;
+  }
+
+  /**
+   * Returns a reference to the internal item array.
+   * @return the internal item array.
+   */
+  float[] getArray() {
+    return arr_;
+  }
+
+  /**
+   * Returns an array of the even values from the range start (inclusive) to end (exclusive).
+   * The even values are with respect to the start index. If the starting index is odd with
+   * respect to the origin of the Buffer, then this will actually return the odd values.
+   * @param start the starting index
+   * @param end the end index, exclusive
+   * @return the selected evens from the range
+   */
+  float[] getEvens(final int start, final int end) {
+    final int range = end - start;
+    final int odd = range & 1;
+    final int len = odd + (range / 2);
+    final float[] out = new float[len];
+    for (int i = start, j = 0; i < end; i += 2, j++) {
+      out[j] = arr_[i];
+    }
+    return out;
+  }
+
+  float getItem(final int index) {
+    return arr_[index];
+  }
+
+  /**
+   * Returns an array of the odd values from the range start (inclusive) to end (exclusive).
+   * The odd values are with respect to the start index. If the starting index is odd with
+   * respect to the origin of the Buffer, then this will actually return the even values.
+   * @param start the starting index
+   * @param end the end index, exclusive
+   * @return the selected odds from the range
+   */
+  float[] getOdds(final int start, final int end) {
+    final int outLen = (end - start) / 2;
+    final float[] out = new float[outLen];
+    for (int i = start + 1, j = 0; i < end; i += 2, j++) {
+      out[j] = arr_[i];
+    }
+    return out;
+  }
+
+  boolean isEmpty() {
+    return count_ == 0;
+  }
+
+  /**
+   * Returns true if this Buffer is sorted.
+   * @return true if sorted
+   */
+  boolean isSorted() {
+    return sorted_;
+  }
+
+  /**
+   * Returns the length (item count).
+   *
+   * @return the number of active items currently in this array.
+   */
+  int length() {
+    return count_;
+  }
+
+  /**
+   * Merges the incoming sorted array into this sorted array.
+   * @param arrIn sorted array in
+   * @return this
+   */
+  Buffer mergeSortIn(final float[] arrIn) {
+    if (!sorted_) { throw new SketchesArgumentException("Must be sorted."); }
+    ensureSpace(arrIn.length);
+    int i = count_;
+    int j = arrIn.length;
+    for (int k = i-- + j--; k-- > 0; ) {
+      if ((i >= 0) && (j >= 0)) { //both valid
+        arr_[k] = (arr_[i] >= arrIn[j]) ? arr_[i--] : arrIn[j--];
+      } else if (i >= 0) { //i is valid
+        arr_[k] = arr_[i--];
+      } else if (j >= 0) { //j is valid
+        arr_[k] = arrIn[j--];
+      } else {
+        break;
+      }
+    }
+    count_ += arrIn.length;
+    setSorted(true);
+    return this;
+  }
+
+  /**
+   * Set the sorted state
+   * @param sortedState the sorted state
+   * @return the sorted state
+   */
+  Buffer setSorted(final boolean sortedState) {
+    sorted_ = sortedState;
+    return this;
+  }
+
+  /**
+   * Sorts this array from 0 to length();
+   * @return this
+   */
+  Buffer sort() {
+    Arrays.sort(arr_, 0, count_);
+    sorted_ = true;
+    return this;
+  }
+
+  /**
+   * Sorts this array from start to length;
+   * @param start starting index
+   * @param length number of items to sort
+   * @return this
+   */
+  Buffer sort(final int start, final int length) {
+    Arrays.sort(arr_, start, length);
+    return this;
+  }
+
+  /**
+   * Returns a new items array of all the active data.
+   * @return a new items array with data.
+   */
+  float[] toItemArray() {
+    return Arrays.copyOf(arr_, count_);
+  }
+
+  /**
+   * Returns a new item array of the active data specified by the offset and length.
+   *
+   * @param offset the zero-based offset into the array.
+   * @param length the number of items to copy to the new array.
+   * @return a new item array of all the active data specified by the offset and length.
+   */
+  float[] toItemArray(final int offset, final int length) {
+    if ((offset + length) > count_) {
+      throw new SketchesArgumentException("Sum of arguments exceed current length().");
+    }
+    return Arrays.copyOfRange(arr_, offset, offset + length);
+  }
+
+  /**
+   * Trims the capacity of this Buffer to length().
+   * @return this
+   */
+  Buffer trimCapacity() {
+    if (count_ < arr_.length) {
+      arr_ = Arrays.copyOf(arr_, count_);
+      capacity_ = count_;
+    }
+    return this;
+  }
+
+  /**
+   * Trims the length to newLength. If newLength &gt; length() this does nothing and returns. If
+   * newLength is &lt; length() this clears all values between newLength and length() and resets
+   * length() to the newLength.
+   *
+   * @param newLength the new length
+   * @return this
+   */
+  Buffer trimLength(final int newLength) {
+    if (newLength < count_) {
+      for (int i = newLength; i < count_; i++) {
+        arr_[i] = 0;
+      }
+      count_ = newLength;
+    }
+    return this;
+  }
+}
diff --git a/src/main/java/org/apache/datasketches/req/RelativeCompactor.java b/src/main/java/org/apache/datasketches/req/RelativeCompactor.java
new file mode 100644
index 0000000..8e97a0e
--- /dev/null
+++ b/src/main/java/org/apache/datasketches/req/RelativeCompactor.java
@@ -0,0 +1,264 @@
+/*
+ * 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 org.apache.datasketches.req;
+
+import static java.lang.Math.max;
+import static java.lang.Math.round;
+import static org.apache.datasketches.Util.numberOfTrailingOnes;
+import static org.apache.datasketches.req.RelativeErrorSketch.INIT_NUMBER_OF_SECTIONS;
+import static org.apache.datasketches.req.RelativeErrorSketch.MIN_K;
+import static org.apache.datasketches.req.RelativeErrorSketch.println;
+
+import java.util.Random;
+
+/**
+ * @author Lee Rhodes
+ */
+//@SuppressWarnings({"javadoc","unused"})
+public class RelativeCompactor {
+  private static final double SQRT2 = Math.sqrt(2.0);
+  private int numCompactions; //number of compaction operations performed
+  private int state; //state of the deterministic compaction schedule
+  //if there are no merge operations performed, state == numCompactions
+
+  private boolean coin; //true or false uniformly at random for each compaction
+  private int sectionSize; //k
+  private int numSections; //# of sections in the buffer, minimum 3
+  Buffer buf;
+  int lgWeight = 0;
+  private boolean debug;
+  Random rand = new Random();
+
+
+  /**
+   * Constructor
+   * @param sectionSize the value of k
+   * @param lgWeight this compactor's lgWeight
+   * @param debug optional
+   */
+  public RelativeCompactor(final int sectionSize, final int lgWeight, final boolean debug) {
+    this.sectionSize = sectionSize;
+    this.lgWeight = lgWeight;
+    this.debug = debug;
+
+    numCompactions = 0;
+    state = 0;
+    coin = false;
+    numSections = INIT_NUMBER_OF_SECTIONS;
+    final int cap = 2 * numSections * sectionSize; //cap is always even
+    buf = new Buffer(cap, cap / 4);
+  }
+
+  /**
+   * Copy Constuctor
+   * @param other the compactor to be copied into this one
+   */
+  public RelativeCompactor(final RelativeCompactor other) {
+    sectionSize = other.sectionSize;
+    lgWeight = other.lgWeight;
+    debug = other.debug;
+    numCompactions = other.numCompactions;
+    state = other.state;
+    coin = other.coin;
+    numSections = other.numSections;
+    buf = new Buffer(other.buf);
+  }
+
+  /**
+   * Append one item to this compactor
+   * @param item the given item
+   * @return this;
+   */
+  public RelativeCompactor append(final float item) {
+    buf.append(item);
+    return this;
+  }
+
+  /**
+   * Perform a compaction operation on this compactor
+   * @return the array of items to be promoted to the next level compactor
+   */
+  public float[] compact() {
+    if (debug) { println("Compactor " + lgWeight + " Compacting ..."); }
+    final int cap = capacity(); //ensures and gets
+    if (!buf.isSorted()) {
+      buf.sort(); //Footnote 1
+    }
+    //choose a part of the buffer to compac
+    final int compactionOffset;
+    if (sectionSize < MIN_K) {  //COMMENT: can be avoided by making sure sectionSize >= MIN_K
+      //too small sections => compact half of the buffer always
+      compactionOffset = cap / 2;  //COMMENT:  Not sure this makes sense and may be unneccesary
+    }
+    else { //Footnote 2
+      final int secsToCompact = numberOfTrailingOnes(state) + 1;
+      compactionOffset = (cap / 2) + ((numSections - secsToCompact) * sectionSize);
+
+      if (numCompactions >= (1 << (numSections - 1))) { //make numSections larger
+        numSections *= 2; //Footnote 3
+        sectionSize = max(nearestEven(sectionSize / SQRT2), MIN_K); //Footnote 4
+      }
+    }
+
+    //COMMENT: we can avoid this if we can guarantee that buf.length, compactionSize are even
+    //if (((buf.length() - compactionOffset) % 2) == 1) { //ensure compacted part has an even size
+    //  if (compactionOffset > 0) { compactionOffset--; }
+    //} else { compactionOffset++; }
+    assert compactionOffset <= (buf.length() - 2); //Footnote 5; This is easier to read!
+
+    if ((numCompactions % 2) == 1) { coin = !coin; } //invert coin; Footnote 6
+    else { coin = (rand.nextDouble() < 0.5); } //random coin flip
+
+    final float[] promote = (coin)
+        ? buf.getEvens(compactionOffset, buf.length())
+        : buf.getOdds(compactionOffset, buf.length());
+
+    //if (debug) { println("RelativeCompactor: Compacting..."); } //Footnote 7
+
+    buf.trimLength(compactionOffset);
+    numCompactions += 1;
+    state += 1;
+
+    if (debug) { println("Compactor: Done\n  Buf Length   :\t" + buf.length()); }
+    return promote;
+  } //End Compact
+
+  /**
+   * Sets the current maximum capacity of this compactor.
+   * @return the current maximum capacity of this compactor.
+   */
+  public int capacity() {
+    buf.ensureCapacity(2 * numSections * sectionSize);
+    return buf.capacity();
+  }
+
+  /**
+   * Extends this compactor with items
+   * @param items the given items
+   * @return this.
+   */
+  public RelativeCompactor extend(final float[] items) {
+    buf.extend(items);
+    return this;
+  }
+
+  /**
+   * Gets a reference to this compactor's internal Buffer
+   * @return a reference to this compactor's internal Buffer
+   */
+  Buffer getBuf() { return buf; }
+
+  /**
+   * Gets the current capacity of this compactor
+   * @return the current capacity of this compactor
+   */
+  public int getCapacity() {
+    return buf.capacity();
+  }
+
+  /**
+   * Gets the lgWeight of this buffer
+   * @return the lgWeight of this buffer
+   */
+  public int getLgWeight() {
+    return lgWeight;
+  }
+
+  /**
+   * Gets the length (number of retained values) in this compactor.
+   * @return the length of this compactor
+   */
+  public int length() { return buf.length(); }
+
+  /**
+   * Merge the other given compactor into this one
+   * @param other the other given compactor
+   * @return this
+   */
+  public RelativeCompactor mergeIntoSelf(final RelativeCompactor other) {
+    state |= other.state;
+    numCompactions += other.numCompactions;
+    buf.extend(other.getBuf());
+    buf.sort(); //TODO this wasn't in Pavel's code
+    return this;
+  }
+
+  /**
+   * Sort only the values in this compactor that are not already sorted.
+   * @return this
+   */
+  public RelativeCompactor optimizedSort() { //TODO not done
+    return this;
+  }
+
+  /**
+   * Gets the non-normalized rank of the given value.  This is equal to the number of values in
+   * this compactor that are &lt; the given value.
+   * @param value the given value
+   * @return the non-normalized rank of the given value
+   */
+  public int rank(final float value) { //one-based integer
+    return buf.countLessThan(value);
+  }
+
+  /**
+   * Sort all values in this compactor.
+   * @return this
+   */
+  public RelativeCompactor sort() {
+    if (!buf.isSorted()) { buf.sort(); }
+    return this;
+  }
+
+  @Override
+  public String toString() {
+    return null;
+  }
+
+  private static final int nearestEven(final double value) {
+    return ((int) round(value / 2.0)) << 1;
+  }
+
+  /* Footnotes:
+   * 1. Sort the items in the buffer; use self.sort(reverse=True) for a better accuracy for
+   *    higher-ranked items; TODO: test this reversed order.
+   *    Remark: it's actually not needed to sort the whole buffer, we just need to ensure that the
+   *    compacted part of the buffer is sorted and contains largest items
+   *
+   * 2. Choose according to the deterministic schedule, i.e., according to the number
+   *    of trailing zeros in binary representation of the state, which is the number of
+   *    compactions so far, unless there are merge operations.
+   *
+   * 3. This is, basically, a doubling strategy on log_2(number of compactions).
+   *    TODO replace doubling strategy by increments by 1?
+   *
+   * 4. Decreasing section size so that it equals roughly
+   *    initial size / sqrt(log_2 (number of compactions)
+   *
+   * 5. TODO ensure under merge operations: s >= cap / 2 - 1)
+   *    at least half of the buffer should remain unaffected by compaction.
+   *
+   * 6. Random offset for choosing odd/even items in the compacted part;
+   *    Random choice done every other time.
+   *
+   * 7. Possible debug outputs: compactionOffset, numCompactions, sectionsToCompact, length,
+   *    capacity, sectionSize, numSections
+   */
+}
diff --git a/src/main/java/org/apache/datasketches/req/RelativeErrorSketch.java b/src/main/java/org/apache/datasketches/req/RelativeErrorSketch.java
new file mode 100644
index 0000000..0ee325d
--- /dev/null
+++ b/src/main/java/org/apache/datasketches/req/RelativeErrorSketch.java
@@ -0,0 +1,264 @@
+/*
+ * 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 org.apache.datasketches.req;
+
+import static java.lang.Math.max;
+
+import java.util.ArrayList;
+import java.util.List;
+
+
+/**
+ * Proof-of-concept code for paper "Relative Error Streaming Quantiles",
+ * https://arxiv.org/abs/2004.01668.
+ *
+ * <p>This implementation differs from the algorithm described in the paper in the following:</p>
+ * <ul><li>The algorithm requires no upper bound on the stream length.
+ * Instead, each relative-compactor (i.e. buffer) counts the number of compaction operations performed
+ * so far (variable numCompactions). Initially, the relative-compactor starts with 2 buffer sections
+ * and each time the numCompactions exceeds 2^{# of sections}, we double the number of sections
+ * (variable numSections).
+ * </li>
+ * <li>The size of each buffer section (variable k and sectionSize in the code and parameter k in
+ * the paper) is initialized with a value set by the user via variable k.
+ * When the number of sections doubles, we decrease sectionSize by a factor of sqrt(2)
+ * (for which we use a float variable sectionSizeF). As above, this is applied
+ * at each level separately. Thus, when we double the number of sections, the buffer size
+ * increases by a factor of sqrt(2) (up to +-1 after rounding).</li>
+ * <li>The merge operation here does not perform "special compactions", which are used in the paper
+ * to allow for a tight analysis of the sketch.</li>
+ * </ul>
+ *
+ * @author Edo Liberty
+ * @author Pavel Vesely
+ * @author Lee Rhodes
+ */
+@SuppressWarnings("unused")
+public class RelativeErrorSketch {
+  //An initial upper bound on log_2 (number of compactions) + 1 COMMMENT: Huh?
+  final static int INIT_NUMBER_OF_SECTIONS = 3;
+  final static int MIN_K = 4;
+  //should be even; value of 50 roughly corresponds to 0.01-relative error guarantee wiTH
+  //constant probability (TODO determine confidence bounds)
+  final static int DEFAULT_K = 50;
+
+  private int k; //default
+  private boolean debug = false;
+  List<RelativeCompactor> compactors = new ArrayList<>();
+  //int levels; //number of compactors; was H
+  int size; //retained items
+  private int maxSize; //capacity
+  private long totalN; //total items offered to sketch
+
+  /**
+   * Constructor with default k = 50;
+   *
+   */
+  RelativeErrorSketch() {
+    this(DEFAULT_K, false);
+  }
+
+  /**
+   * Constructor
+   * @param k Controls the size and error of the sketch
+   */
+  RelativeErrorSketch(final int k) {
+    this(k, false);
+  }
+
+  /**
+   * Constructor.
+   * @param k Controls the size and error of the sketch. It must be even, if not, it will be
+   * rounded down by one.
+   * @param debug debug mode
+   */
+  RelativeErrorSketch(final int k, final boolean debug) {
+    this.k = max(k & -2, MIN_K);
+    this.debug = debug;
+    size = 0;
+    maxSize = 0;
+    totalN = 0;
+    grow();
+  }
+
+  /**
+   * Copy Constructor
+   * @param other the other sketch to be deep copied into this one.
+   */
+  RelativeErrorSketch(final RelativeErrorSketch other) {
+    k = other.k;
+    debug = other.debug;
+    for (int i = 0; i < other.levels(); i++) {
+      compactors.add(new RelativeCompactor(other.compactors.get(i)));
+    }
+    size = other.size;
+    maxSize = other.maxSize;
+    totalN = other.totalN;
+
+  }
+
+  void compress(final boolean lazy) {
+    if (debug) { println("Compression Start ..."); }
+    updateMaxSize();
+    if (size < maxSize) { return; }
+    for (int h = 0; h < compactors.size(); h++) { //# compactors
+      final RelativeCompactor c = compactors.get(h);
+      if (c.length() >= c.capacity()) {
+        if ((h + 1) >= levels()) { grow(); } //add a level
+        final float[] arr = c.compact();
+        compactors.get(h + 1).extend(arr);
+        size += arr.length;
+        if (lazy && (size < maxSize)) { break; }
+      }
+    }
+    if (debug) {
+      println("Compresssion Done:\n  RetainedItems:\t" + size + "\n  Capacity     :\t" + maxSize);
+    }
+  }
+
+  double[] getCDF(final float[] splitPoints) {
+    return getPmfOrCdf(splitPoints, true);
+  }
+
+  @SuppressWarnings("static-method")
+  private double[] getPmfOrCdf(final float[] splitPoints, final boolean isCdf) {
+    return null;
+  }
+
+  public double getQuantile(final double rank) {
+
+    return 0;
+  }
+
+  void grow() {
+    compactors.add( new RelativeCompactor(k, levels(), debug));
+    updateMaxSize();
+  }
+
+  /**
+   * Returns true if this sketch is empty.
+   * @return empty flag
+   */
+  public boolean isEmpty() {
+    return totalN == 0;
+  }
+
+  /**
+   * Returns true if this sketch is in estimation mode.
+   * @return estimation mode flag
+   */
+  public boolean isEstimationMode() {
+    return levels() > 1;
+  }
+
+  /**
+   * Returns an iterator for all the items in this sketch.
+   * @return an iterator for all the items in this sketch.
+   */
+  public RelativeErrorSketchIterator iterator() {
+    return new RelativeErrorSketchIterator(this);
+  }
+
+  int levels() {
+    return compactors.size();
+  }
+
+  /**
+   * Merge other sketch into this one. The other sketch is not modified.
+   * @param other sketch to be merged into this one.
+   */
+  RelativeErrorSketch mergeIntoSelf(final RelativeErrorSketch other) {
+    //Grow until self has at least as many compactors as other
+    while (levels() < other.levels()) { grow(); }
+    //Append the items in same height compactors
+    for (int i = 0; i < levels(); i++) {
+      compactors.get(i).mergeIntoSelf(other.compactors.get(i));
+    }
+    updateRetainedItems();
+    // After merging, we should not be lazy when compressing the sketch (as the maxSize bound may
+    // be exceeded on many levels)
+    if (size >= maxSize) { compress(false); }
+    assert size < maxSize;
+    return this;
+  }
+
+  class Pair {
+    float rank;
+    float value;
+  }
+
+  /**
+   * Computes the normalized rank of the given value in the stream.
+   * The normalized rank is the fraction of values less than the given value.
+   * @param value the given value
+   * @return the normalized rank of the given value in the stream.
+   */
+  double rank(final float value) {
+    int nnRank = 0;
+    for (int i = 0; i < levels(); i++) {
+      final RelativeCompactor c = compactors.get(i);
+      nnRank += c.rank(value) * (1 << c.lgWeight);
+    }
+    return (double)nnRank / totalN;
+  }
+
+  Pair[] ranks() { //debug
+    return null;
+  }
+
+  void update(final float item) {
+    final RelativeCompactor c = compactors.get(0).append(item);
+    size++;
+    if (size >= maxSize) { compress(true); }
+    totalN++;
+  }
+
+/**
+ * Computes a new bound for determining when to compress the sketch.
+ * @return this
+ */
+RelativeErrorSketch updateMaxSize() {
+  int cap = 0;
+  for (RelativeCompactor c : compactors) { cap += c.capacity(); } //get or set?
+  maxSize = cap;
+  return this;
+}
+
+/**
+ * Computes the size for the sketch.
+ * @return this
+ */
+RelativeErrorSketch updateRetainedItems() {
+  int count = 0;
+  for (RelativeCompactor c : compactors) { count += c.length(); }
+  size = count;
+  return this;
+}
+
+  //temporary
+  static final void printf(final String format, final Object ...args) {
+    System.out.printf(format, args);
+  }
+
+  static final void print(final Object o) { System.out.print(o.toString()); }
+
+  static final void println(final Object o) { System.out.println(o.toString()); }
+
+}
diff --git a/src/main/java/org/apache/datasketches/req/RelativeErrorSketchIterator.java b/src/main/java/org/apache/datasketches/req/RelativeErrorSketchIterator.java
new file mode 100644
index 0000000..5e1459b
--- /dev/null
+++ b/src/main/java/org/apache/datasketches/req/RelativeErrorSketchIterator.java
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.datasketches.req;
+
+import java.util.List;
+
+/**
+ * Iterator over KllFloatsSketch. The order is not defined.
+ *
+ * @author Lee Rhodes
+ */
+public class RelativeErrorSketchIterator {
+  private List<RelativeCompactor> compactors;
+  private int cIndex;
+  private int bIndex;
+  private int retainedItems;
+  private Buffer currentBuf;
+
+  RelativeErrorSketchIterator(final RelativeErrorSketch sketch) {
+    compactors = sketch.compactors;
+    retainedItems = sketch.size;
+    currentBuf = compactors.get(0).buf;
+    cIndex = 0;
+    bIndex = -1;
+  }
+
+  /**
+   * Advancing the iterator and checking existence of the next entry
+   * is combined here for efficiency. This results in an undefined
+   * state of the iterator before the first call of this method.
+   * @return true if the next element exists
+   */
+  public boolean next() {
+    if ((retainedItems == 0)
+        || ((cIndex == (compactors.size() - 1)) && (bIndex == currentBuf.length()))) {
+      return false;
+    }
+    if (bIndex == currentBuf.length()) {
+      cIndex++;
+      currentBuf = compactors.get(cIndex).buf;
+      bIndex = 0;
+    } else {
+      bIndex++;
+    }
+    return true;
+  }
+
+  /**
+   * Gets a value from the current entry in the sketch.
+   * Don't call this before calling next() for the first time
+   * or after getting false from next().
+   * @return value from the current entry
+   */
+  public float getValue() {
+    return currentBuf.getItem(bIndex);
+  }
+
+  /**
+   * Gets a weight for the value from the current entry in the sketch.
+   * Don't call this before calling next() for the first time
+   * or after getting false from next().
+   * @return weight for the value from the current entry
+   */
+  public long getWeight() {
+    return 1 << cIndex;
+  }
+}
diff --git a/src/main/java/org/apache/datasketches/kll/RelativeErrorUtil.java b/src/main/java/org/apache/datasketches/req/package-info.java
similarity index 61%
copy from src/main/java/org/apache/datasketches/kll/RelativeErrorUtil.java
copy to src/main/java/org/apache/datasketches/req/package-info.java
index 7efb4a8..5904f69 100644
--- a/src/main/java/org/apache/datasketches/kll/RelativeErrorUtil.java
+++ b/src/main/java/org/apache/datasketches/req/package-info.java
@@ -17,20 +17,8 @@
  * under the License.
  */
 
-package org.apache.datasketches.kll;
-
 /**
  * @author Lee Rhodes
  */
-@SuppressWarnings({"javadoc"})
-public class RelativeErrorUtil {
-  final static double SECTION_SIZE_SCALAR = 0.5;
-  final static double NEVER_SIZE_SCALAR = 0.5;
-  final static int INIT_NUMBER_OF_SECTIONS = 2;
-  final static int SMALLEST_MEANINGFUL_SECTION_SIZE = 4;
-  final static double DEFAULT_EPS = 0.01;
-  //the sketch gives rather bad results for eps > 0.1
-  final static double EPS_UPPER_BOUND = 0.1;
 
-  public enum Schedule { DETERMINISTIC, RANDOMIZED, RANDOMIZED_LINAR }
-}
+package org.apache.datasketches.req;
diff --git a/src/main/java/org/apache/datasketches/kll/RelativeErrorUtil.java b/src/test/java/org/apache/datasketches/kll/RelativeErrorSketchTest.java
similarity index 64%
copy from src/main/java/org/apache/datasketches/kll/RelativeErrorUtil.java
copy to src/test/java/org/apache/datasketches/kll/RelativeErrorSketchTest.java
index 7efb4a8..a4192c7 100644
--- a/src/main/java/org/apache/datasketches/kll/RelativeErrorUtil.java
+++ b/src/test/java/org/apache/datasketches/kll/RelativeErrorSketchTest.java
@@ -19,18 +19,21 @@
 
 package org.apache.datasketches.kll;
 
+import org.testng.annotations.Test;
+
 /**
  * @author Lee Rhodes
  */
-@SuppressWarnings({"javadoc"})
-public class RelativeErrorUtil {
-  final static double SECTION_SIZE_SCALAR = 0.5;
-  final static double NEVER_SIZE_SCALAR = 0.5;
-  final static int INIT_NUMBER_OF_SECTIONS = 2;
-  final static int SMALLEST_MEANINGFUL_SECTION_SIZE = 4;
-  final static double DEFAULT_EPS = 0.01;
-  //the sketch gives rather bad results for eps > 0.1
-  final static double EPS_UPPER_BOUND = 0.1;
-
-  public enum Schedule { DETERMINISTIC, RANDOMIZED, RANDOMIZED_LINAR }
+@SuppressWarnings("javadoc")
+public class RelativeErrorSketchTest {
+
+  @Test
+  public void check1() {
+
+  }
+
+
+  static void print(Object o) { System.out.print(o.toString()); }
+
+  static void println(Object o) { System.out.println(o.toString()); }
 }
diff --git a/src/test/java/org/apache/datasketches/req/BufferTest.java b/src/test/java/org/apache/datasketches/req/BufferTest.java
new file mode 100644
index 0000000..afbd64c
--- /dev/null
+++ b/src/test/java/org/apache/datasketches/req/BufferTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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 org.apache.datasketches.req;
+
+import static org.testng.Assert.assertEquals;
+
+import org.testng.annotations.Test;
+
+/**
+ * @author Lee Rhodes
+ */
+@SuppressWarnings("javadoc")
+public class BufferTest {
+
+  @Test
+  public void checkTrimLength() {
+    Buffer buf = new Buffer(16, 4);
+    for (int i = 0; i < 8; i++) { buf.append(i+1); }
+    assertEquals(buf.length(), 8);
+    buf.trimLength(4);
+    assertEquals(buf.length(), 4);
+  }
+
+  @Test
+  public void checkGetOdds() {
+    int cap = 16;
+    Buffer buf = new Buffer(cap, cap / 4);
+    for (int i = 0; i < buf.capacity(); i++) {
+      buf.append(i);
+    }
+    float[] out = buf.getOdds(0, cap);
+    println("");
+    for (int i = 0; i < out.length; i++) {
+      print((int)out[i] + " ");
+    }
+  }
+
+  @Test
+  public void checkGetEvens() {
+    int cap = 15;
+    Buffer buf = new Buffer(cap, cap / 4);
+    for (int i = 0; i < buf.capacity(); i++) {
+      buf.append(i);
+    }
+    float[] out = buf.getEvens(0, buf.capacity());
+    println("");
+    for (int i = 0; i < out.length; i++) {
+      print((int)out[i] + " ");
+    }
+  }
+
+  @Test
+  public void checkAppend() {
+    Buffer buf = new Buffer(2, 2);
+    buf.append(1);
+    assertEquals(buf.length(), 1);
+    buf.append(2);
+    assertEquals(buf.length(), 2);
+    buf.append(3);
+    assertEquals(buf.capacity(), 4);
+  }
+
+  @Test
+  public void checkCountLessThan() {
+    Buffer buf = new Buffer(16, 2);
+    buf.extend(new float[] {1,2,3,4,5,6,7,1});
+    buf.setSorted(true);
+    assertEquals(buf.countLessThan(4), 3);
+    buf.setSorted(false);
+    assertEquals(buf.countLessThan(4), 4);
+    buf.clear(4, 7);
+    assertEquals(buf.getItem(4), 0.0F);
+    assertEquals(buf.getItem(5), 0.0F);
+    assertEquals(buf.getItem(6), 0.0F);
+  }
+
+  @Test
+  public void checkExtendArray() {
+    Buffer buf = new Buffer(0, 2);
+    float[] arr1 = {1,2};
+    float[] arr2 = {3,4};
+    buf.extend(arr1);
+    buf.extend(arr2);
+    for (int i = 0; i < buf.length(); i++) {
+      println(buf.getItem(i));
+    }
+  }
+
+  @Test
+  public void checkExtendWithBuffer() {
+    Buffer buf = new Buffer(0, 2);
+    float[] arr1 = {1,2};
+    buf.extend(arr1);
+    Buffer buf2 = new Buffer(0, 2);
+    float[] arr2 = {3,4};
+    buf2.extend(arr2);
+    buf.extend(buf2);
+    float[] arr3 = buf.getArray();
+    assertEquals(arr3.length, 4);
+
+    float[] arr4 = buf.getOdds(0, 4);
+    for (int i = 0; i < arr4.length; i++) {
+      println(arr4[i]);
+    }
+    arr4 = buf.getEvens(0, 4);
+    for (int i = 0; i < arr4.length; i++) {
+      println(arr4[i]);
+    }
+    assertEquals(buf.isSorted(), false);
+    assertEquals(buf.sort().isSorted(), true);
+  }
+
+  @Test
+  public void checkMergeSortIn() {
+    Buffer buf = new Buffer(4,0);
+    float[] arr1 = {1,2,5,6};
+    float[] arr2 = {3,4,4,7};
+    buf.extend(arr1);
+    buf.sort();
+    buf.mergeSortIn(arr2);
+    int len = buf.length();
+    for (int i = 0; i < len; i++) { print(buf.getItem(i) + ", "); }
+    println("");
+  }
+
+  static void print(Object o) { System.out.print(o.toString()); }
+
+  static void println(Object o) { System.out.println(o.toString()); }
+}
diff --git a/src/main/java/org/apache/datasketches/kll/RelativeErrorUtil.java b/src/test/java/org/apache/datasketches/req/RelativeErrorSketchTest.java
similarity index 62%
rename from src/main/java/org/apache/datasketches/kll/RelativeErrorUtil.java
rename to src/test/java/org/apache/datasketches/req/RelativeErrorSketchTest.java
index 7efb4a8..5989e79 100644
--- a/src/main/java/org/apache/datasketches/kll/RelativeErrorUtil.java
+++ b/src/test/java/org/apache/datasketches/req/RelativeErrorSketchTest.java
@@ -17,20 +17,23 @@
  * under the License.
  */
 
-package org.apache.datasketches.kll;
+package org.apache.datasketches.req;
+
+import org.testng.annotations.Test;
 
 /**
  * @author Lee Rhodes
  */
-@SuppressWarnings({"javadoc"})
-public class RelativeErrorUtil {
-  final static double SECTION_SIZE_SCALAR = 0.5;
-  final static double NEVER_SIZE_SCALAR = 0.5;
-  final static int INIT_NUMBER_OF_SECTIONS = 2;
-  final static int SMALLEST_MEANINGFUL_SECTION_SIZE = 4;
-  final static double DEFAULT_EPS = 0.01;
-  //the sketch gives rather bad results for eps > 0.1
-  final static double EPS_UPPER_BOUND = 0.1;
+@SuppressWarnings("javadoc")
+public class RelativeErrorSketchTest {
+
+
+  @Test
+  public void test1() {
+    RelativeErrorSketch sk = new RelativeErrorSketch(4, true); //w debug
+    for (int i = 1; i < 100; i++) {
+      sk.update(i);
+    }
+  }
 
-  public enum Schedule { DETERMINISTIC, RANDOMIZED, RANDOMIZED_LINAR }
 }


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