diff --git a/src/main/java/org/apache/datasketches/kll/KllDirectLongsSketch.java b/src/main/java/org/apache/datasketches/kll/KllDirectLongsSketch.java new file mode 100644 index 000000000..bf91baabd --- /dev/null +++ b/src/main/java/org/apache/datasketches/kll/KllDirectLongsSketch.java @@ -0,0 +1,391 @@ +/* + * 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.common.ByteArrayUtil.copyBytes; +import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR; +import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_SINGLE_ITEM; +import static org.apache.datasketches.kll.KllPreambleUtil.getMemoryK; +import static org.apache.datasketches.kll.KllPreambleUtil.getMemoryLevelZeroSortedFlag; +import static org.apache.datasketches.kll.KllPreambleUtil.getMemoryM; +import static org.apache.datasketches.kll.KllPreambleUtil.getMemoryMinK; +import static org.apache.datasketches.kll.KllPreambleUtil.getMemoryN; +import static org.apache.datasketches.kll.KllPreambleUtil.getMemoryNumLevels; +import static org.apache.datasketches.kll.KllPreambleUtil.setMemoryFamilyID; +import static org.apache.datasketches.kll.KllPreambleUtil.setMemoryK; +import static org.apache.datasketches.kll.KllPreambleUtil.setMemoryLevelZeroSortedFlag; +import static org.apache.datasketches.kll.KllPreambleUtil.setMemoryM; +import static org.apache.datasketches.kll.KllPreambleUtil.setMemoryMinK; +import static org.apache.datasketches.kll.KllPreambleUtil.setMemoryN; +import static org.apache.datasketches.kll.KllPreambleUtil.setMemoryNumLevels; +import static org.apache.datasketches.kll.KllPreambleUtil.setMemoryPreInts; +import static org.apache.datasketches.kll.KllPreambleUtil.setMemorySerVer; +import static org.apache.datasketches.kll.KllSketch.SketchStructure.COMPACT_EMPTY; +import static org.apache.datasketches.kll.KllSketch.SketchStructure.COMPACT_FULL; +import static org.apache.datasketches.kll.KllSketch.SketchStructure.COMPACT_SINGLE; +import static org.apache.datasketches.kll.KllSketch.SketchStructure.UPDATABLE; +import static org.apache.datasketches.kll.KllSketch.SketchType.LONGS_SKETCH; + +import org.apache.datasketches.common.ByteArrayUtil; +import org.apache.datasketches.common.Family; +import org.apache.datasketches.common.SketchesArgumentException; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.memory.MemoryRequestServer; +import org.apache.datasketches.memory.WritableMemory; + +/** + * This class implements an off-heap, updatable KllLongsSketch using WritableMemory. + * + *
Please refer to the documentation in the package-info:
+ * {@link org.apache.datasketches.kll}
Please refer to the documentation in the package-info:
+ * {@link org.apache.datasketches.kll}
Here is what we do for each level:
+ *It can be proved that generalCompress returns a sketch that satisfies the space constraints + * no matter how much data is passed in. + * We are pretty sure that it works correctly when inBuf and outBuf are the same. + * All levels except for level zero must be sorted before calling this, and will still be + * sorted afterwards. + * Level zero is not required to be sorted before, and may not be sorted afterwards.
+ * + *This trashes inBuf and inLevels and modifies outBuf and outLevels.
+ * + * @param k The sketch parameter k + * @param m The minimum level size + * @param numLevelsIn provisional number of number of levels = max(this.numLevels, other.numLevels) + * @param inBuf work buffer of size = this.getNumRetained() + other.getNumRetainedAboveLevelZero(). + * This contains the long[] of the other sketch + * @param inLevels work levels array size = ubOnNumLevels(this.n + other.n) + 2 + * @param outBuf the same array as inBuf + * @param outLevels the same size as inLevels + * @param isLevelZeroSorted true if this.level 0 is sorted + * @param random instance of java.util.Random + * @return int array of: {numLevels, targetItemCount, currentItemCount) + */ + // + private static int[] generalLongsCompress( + final int k, + final int m, + final int numLevelsIn, + final long[] inBuf, + final int[] inLevels, + final long[] outBuf, + final int[] outLevels, + final boolean isLevelZeroSorted, + final Random random) { + assert numLevelsIn > 0; // things are too weird if zero levels are allowed + int numLevels = numLevelsIn; + int currentItemCount = inLevels[numLevels] - inLevels[0]; // decreases with each compaction + int targetItemCount = KllHelper.computeTotalItemCapacity(k, m, numLevels); // increases if we add levels + boolean doneYet = false; + outLevels[0] = 0; + int curLevel = -1; + while (!doneYet) { + curLevel++; // start out at level 0 + + // If we are at the current top level, add an empty level above it for convenience, + // but do not actually increment numLevels until later + if (curLevel == (numLevels - 1)) { + inLevels[curLevel + 2] = inLevels[curLevel + 1]; + } + + final int rawBeg = inLevels[curLevel]; + final int rawLim = inLevels[curLevel + 1]; + final int rawPop = rawLim - rawBeg; + + if ((currentItemCount < targetItemCount) || (rawPop < KllHelper.levelCapacity(k, numLevels, curLevel, m))) { + // copy level over as is + // because inBuf and outBuf could be the same, make sure we are not moving data upwards! + assert (rawBeg >= outLevels[curLevel]); + System.arraycopy(inBuf, rawBeg, outBuf, outLevels[curLevel], rawPop); + outLevels[curLevel + 1] = outLevels[curLevel] + rawPop; + } + else { + // The sketch is too full AND this level is too full, so we compact it + // Note: this can add a level and thus change the sketch's capacity + + final int popAbove = inLevels[curLevel + 2] - rawLim; + final boolean oddPop = isOdd(rawPop); + final int adjBeg = oddPop ? 1 + rawBeg : rawBeg; + final int adjPop = oddPop ? rawPop - 1 : rawPop; + final int halfAdjPop = adjPop / 2; + + if (oddPop) { // copy one guy over + outBuf[outLevels[curLevel]] = inBuf[rawBeg]; + outLevels[curLevel + 1] = outLevels[curLevel] + 1; + } else { // copy zero guys over + outLevels[curLevel + 1] = outLevels[curLevel]; + } + + // level zero might not be sorted, so we must sort it if we wish to compact it + if ((curLevel == 0) && !isLevelZeroSorted) { + Arrays.sort(inBuf, adjBeg, adjBeg + adjPop); + } + + if (popAbove == 0) { // Level above is empty, so halve up + randomlyHalveUpLongs(inBuf, adjBeg, adjPop, random); + } else { // Level above is nonempty, so halve down, then merge up + randomlyHalveDownLongs(inBuf, adjBeg, adjPop, random); + mergeSortedLongArrays(inBuf, adjBeg, halfAdjPop, inBuf, rawLim, popAbove, inBuf, adjBeg + halfAdjPop); + } + + // track the fact that we just eliminated some data + currentItemCount -= halfAdjPop; + + // Adjust the boundaries of the level above + inLevels[curLevel + 1] = inLevels[curLevel + 1] - halfAdjPop; + + // Increment numLevels if we just compacted the old top level + // This creates some more capacity (the size of the new bottom level) + if (curLevel == (numLevels - 1)) { + numLevels++; + targetItemCount += KllHelper.levelCapacity(k, numLevels, 0, m); + } + } // end of code for compacting a level + + // determine whether we have processed all levels yet (including any new levels that we created) + if (curLevel == (numLevels - 1)) { doneYet = true; } + } // end of loop over levels + + assert (outLevels[numLevels] - outLevels[0]) == currentItemCount; + return new int[] {numLevels, targetItemCount, currentItemCount}; + } + + private static void populateLongWorkArrays( //workBuf and workLevels are modified + final long[] workBuf, final int[] workLevels, final int provisionalNumLevels, + final int myCurNumLevels, final int[] myCurLevelsArr, final long[] myCurLongItemsArr, + final int otherNumLevels, final int[] otherLevelsArr, final long[] otherLongItemsArr) { + + workLevels[0] = 0; + + // Note: the level zero data from "other" was already inserted into "self". + // This copies into workbuf. + final int selfPopZero = KllHelper.currentLevelSizeItems(0, myCurNumLevels, myCurLevelsArr); + System.arraycopy(myCurLongItemsArr, myCurLevelsArr[0], workBuf, workLevels[0], selfPopZero); + workLevels[1] = workLevels[0] + selfPopZero; + + for (int lvl = 1; lvl < provisionalNumLevels; lvl++) { + final int selfPop = KllHelper.currentLevelSizeItems(lvl, myCurNumLevels, myCurLevelsArr); + final int otherPop = KllHelper.currentLevelSizeItems(lvl, otherNumLevels, otherLevelsArr); + workLevels[lvl + 1] = workLevels[lvl] + selfPop + otherPop; + assert selfPop >= 0 && otherPop >= 0; + if (selfPop == 0 && otherPop == 0) { continue; } + if (selfPop > 0 && otherPop == 0) { + System.arraycopy(myCurLongItemsArr, myCurLevelsArr[lvl], workBuf, workLevels[lvl], selfPop); + } + else if (selfPop == 0 && otherPop > 0) { + System.arraycopy(otherLongItemsArr, otherLevelsArr[lvl], workBuf, workLevels[lvl], otherPop); + } + else if (selfPop > 0 && otherPop > 0) { + mergeSortedLongArrays( //only workBuf is modified + myCurLongItemsArr, myCurLevelsArr[lvl], selfPop, + otherLongItemsArr, otherLevelsArr[lvl], otherPop, + workBuf, workLevels[lvl]); + } + } + } + + /* + * Validation Method. + * The following must be enabled for use with the KllDoublesValidationTest, + * which is only enabled for manual testing. In addition, two Validation Methods + * above need to be modified. + */ //NOTE Validation Method: Need to uncomment to use + // static int nextOffset = 0; + // + // private static int deterministicOffset() { + // final int result = nextOffset; + // nextOffset = 1 - nextOffset; + // return result; + // } + +} diff --git a/src/main/java/org/apache/datasketches/kll/KllLongsSketch.java b/src/main/java/org/apache/datasketches/kll/KllLongsSketch.java new file mode 100644 index 000000000..0f6fa7d89 --- /dev/null +++ b/src/main/java/org/apache/datasketches/kll/KllLongsSketch.java @@ -0,0 +1,669 @@ +/* + * 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.max; +import static java.lang.Math.min; +import static org.apache.datasketches.common.ByteArrayUtil.putLongLE; +import static org.apache.datasketches.kll.KllSketch.SketchStructure.UPDATABLE; +import static org.apache.datasketches.kll.KllSketch.SketchType.LONGS_SKETCH; + +import java.util.Arrays; +import java.util.Objects; + +import org.apache.datasketches.common.ArrayOfItemsSerDe; +import org.apache.datasketches.common.SketchesArgumentException; +import org.apache.datasketches.common.SuppressFBWarnings; +import org.apache.datasketches.kll.KllDirectLongsSketch.KllDirectCompactLongsSketch; +import org.apache.datasketches.memory.DefaultMemoryRequestServer; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.memory.MemoryRequestServer; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.datasketches.quantilescommon.LongsSketchSortedView; +import org.apache.datasketches.quantilescommon.QuantileSearchCriteria; +import org.apache.datasketches.quantilescommon.QuantilesLongsAPI; +import org.apache.datasketches.quantilescommon.QuantilesLongsSketchIterator; + +/** + * This variation of the KllSketch implements primitive longs. + * + * @see org.apache.datasketches.kll.KllSketch + */ +public abstract class KllLongsSketch extends KllSketch implements QuantilesLongsAPI { + private LongsSketchSortedView longsSV = null; + final static int ITEM_BYTES = Long.BYTES; + + KllLongsSketch( + final SketchStructure sketchStructure) { + super(SketchType.LONGS_SKETCH, sketchStructure); + } + + //Factories for new heap instances. + + /** + * Create a new heap instance of this sketch with the default k = 200. + * The default k = 200 results in a normalized rank error of about + * 1.65%. Larger K will have smaller error but the sketch will be larger (and slower). + * @return new KllLongsSketch on the Java heap. + */ + public static KllLongsSketch newHeapInstance() { + return newHeapInstance(DEFAULT_K); + } + + /** + * Create a new heap instance of this sketch with a given parameter k. + * k can be between 8, inclusive, and 65535, inclusive. + * The default k = 200 results in a normalized rank error of about + * 1.65%. Larger K will have smaller error but the sketch will be larger (and slower). + * @param k parameter that controls size of the sketch and accuracy of estimates. + * @return new KllLongsSketch on the Java heap. + */ + public static KllLongsSketch newHeapInstance(final int k) { + return new KllHeapLongsSketch(k, DEFAULT_M); + } + + //Factories for new direct instances. + + /** + * Create a new direct updatable instance of this sketch with the default k. + * The default k = 200 results in a normalized rank error of about + * 1.65%. Larger k will have smaller error but the sketch will be larger (and slower). + * @param dstMem the given destination WritableMemory object for use by the sketch + * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory + * @return a new direct instance of this sketch + */ + public static KllLongsSketch newDirectInstance( + final WritableMemory dstMem, + final MemoryRequestServer memReqSvr) { + return newDirectInstance(DEFAULT_K, dstMem, memReqSvr); + } + + /** + * Create a new direct updatable instance of this sketch with a given k. + * @param k parameter that controls size of the sketch and accuracy of estimates. + * @param dstMem the given destination WritableMemory object for use by the sketch + * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory + * @return a new direct instance of this sketch + */ + public static KllLongsSketch newDirectInstance( + final int k, + final WritableMemory dstMem, + final MemoryRequestServer memReqSvr) { + Objects.requireNonNull(dstMem, "Parameter 'dstMem' must not be null"); + Objects.requireNonNull(memReqSvr, "Parameter 'memReqSvr' must not be null"); + return KllDirectLongsSketch.newDirectUpdatableInstance(k, DEFAULT_M, dstMem, memReqSvr); + } + + //Factory to create an heap instance from a Memory image + + /** + * Factory heapify takes a compact sketch image in Memory and instantiates an on-heap sketch. + * The resulting sketch will not retain any link to the source Memory. + * @param srcMem a compact Memory image of a sketch serialized by this sketch. + * See Memory + * @return a heap-based sketch based on the given Memory. + */ + public static KllLongsSketch heapify(final Memory srcMem) { + Objects.requireNonNull(srcMem, "Parameter 'srcMem' must not be null"); + return KllHeapLongsSketch.heapifyImpl(srcMem); + } + + //Factory to wrap a Read-Only Memory + + /** + * Wrap a sketch around the given read only compact source Memory containing sketch data + * that originated from this sketch. + * @param srcMem the read only source Memory + * @return instance of this sketch + */ + public static KllLongsSketch wrap(final Memory srcMem) { + Objects.requireNonNull(srcMem, "Parameter 'srcMem' must not be null"); + final KllMemoryValidate memVal = new KllMemoryValidate(srcMem, LONGS_SKETCH, null); + if (memVal.sketchStructure == UPDATABLE) { + final MemoryRequestServer memReqSvr = new DefaultMemoryRequestServer(); //dummy + return new KllDirectLongsSketch(memVal.sketchStructure, (WritableMemory)srcMem, memReqSvr, memVal); + } else { + return new KllDirectCompactLongsSketch(memVal.sketchStructure, srcMem, memVal); + } + } + + //Factory to wrap a WritableMemory image + + /** + * Wrap a sketch around the given source Writable Memory containing sketch data + * that originated from this sketch. + * @param srcMem a WritableMemory that contains data. + * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory + * @return instance of this sketch + */ + public static KllLongsSketch writableWrap( + final WritableMemory srcMem, + final MemoryRequestServer memReqSvr) { + Objects.requireNonNull(srcMem, "Parameter 'srcMem' must not be null"); + Objects.requireNonNull(memReqSvr, "Parameter 'memReqSvr' must not be null"); + final KllMemoryValidate memVal = new KllMemoryValidate(srcMem, LONGS_SKETCH); + if (memVal.sketchStructure == UPDATABLE) { + return new KllDirectLongsSketch(UPDATABLE, srcMem, memReqSvr, memVal); + } else { + return new KllDirectCompactLongsSketch(memVal.sketchStructure, srcMem, memVal); + } + } + + //END of Constructors + + @Override + public double[] getCDF(final long[] splitPoints, final QuantileSearchCriteria searchCrit) { + if (isEmpty()) { throw new SketchesArgumentException(EMPTY_MSG); } + refreshSortedView(); + return longsSV.getCDF(splitPoints, searchCrit); + } + + @Override + public double[] getPMF(final long[] splitPoints, final QuantileSearchCriteria searchCrit) { + if (isEmpty()) { throw new SketchesArgumentException(EMPTY_MSG); } + refreshSortedView(); + return longsSV.getPMF(splitPoints, searchCrit); + } + + @Override + public long getQuantile(final double rank, final QuantileSearchCriteria searchCrit) { + if (isEmpty()) { throw new SketchesArgumentException(EMPTY_MSG); } + refreshSortedView(); + return longsSV.getQuantile(rank, searchCrit); + } + + @Override + public long[] getQuantiles(final double[] ranks, final QuantileSearchCriteria searchCrit) { + if (isEmpty()) { throw new SketchesArgumentException(EMPTY_MSG); } + refreshSortedView(); + final int len = ranks.length; + final long[] quantiles = new long[len]; + for (int i = 0; i < len; i++) { + quantiles[i] = longsSV.getQuantile(ranks[i], searchCrit); + } + return quantiles; + } + + /** + * {@inheritDoc} + * The approximate probability that the true quantile is within the confidence interval + * specified by the upper and lower quantile bounds for this sketch is 0.99. + */ + @Override + public long getQuantileLowerBound(final double rank) { + return getQuantile(max(0, rank - KllHelper.getNormalizedRankError(getMinK(), false))); + } + + /** + * {@inheritDoc} + * The approximate probability that the true quantile is within the confidence interval + * specified by the upper and lower quantile bounds for this sketch is 0.99. + */ + @Override + public long getQuantileUpperBound(final double rank) { + return getQuantile(min(1.0, rank + KllHelper.getNormalizedRankError(getMinK(), false))); + } + + @Override + public double getRank(final long quantile, final QuantileSearchCriteria searchCrit) { + if (isEmpty()) { throw new SketchesArgumentException(EMPTY_MSG); } + refreshSortedView(); + return longsSV.getRank(quantile, searchCrit); + } + + /** + * {@inheritDoc} + * The approximate probability that the true rank is within the confidence interval + * specified by the upper and lower rank bounds for this sketch is 0.99. + */ + @Override + public double getRankLowerBound(final double rank) { + return max(0.0, rank - KllHelper.getNormalizedRankError(getMinK(), false)); + } + + /** + * {@inheritDoc} + * The approximate probability that the true rank is within the confidence interval + * specified by the upper and lower rank bounds for this sketch is 0.99. + */ + @Override + public double getRankUpperBound(final double rank) { + return min(1.0, rank + KllHelper.getNormalizedRankError(getMinK(), false)); + } + + @Override + public double[] getRanks(final long[] quantiles, final QuantileSearchCriteria searchCrit) { + if (isEmpty()) { throw new SketchesArgumentException(EMPTY_MSG); } + refreshSortedView(); + final int len = quantiles.length; + final double[] ranks = new double[len]; + for (int i = 0; i < len; i++) { + ranks[i] = longsSV.getRank(quantiles[i], searchCrit); + } + return ranks; + } + + @Override + public QuantilesLongsSketchIterator iterator() { + return new KllLongsSketchIterator( + getLongItemsArray(), getLevelsArray(SketchStructure.UPDATABLE), getNumLevels()); + } + + @Override + public final void merge(final KllSketch other) { + if (readOnly || sketchStructure != UPDATABLE) { throw new SketchesArgumentException(TGT_IS_READ_ONLY_MSG); } + if (this == other) { throw new SketchesArgumentException(SELF_MERGE_MSG); } + final KllLongsSketch otherLngSk = (KllLongsSketch)other; + if (otherLngSk.isEmpty()) { return; } + KllLongsHelper.mergeLongsImpl(this, otherLngSk); + longsSV = null; + } + + /** + * {@inheritDoc} + *The parameter k will not change.
+ */ + @Override + public final void reset() { + if (readOnly) { throw new SketchesArgumentException(TGT_IS_READ_ONLY_MSG); } + final int k = getK(); + setN(0); + setMinK(k); + setNumLevels(1); + setLevelZeroSorted(false); + setLevelsArray(new int[] {k, k}); + setMinItem(Long.MAX_VALUE); + setMaxItem(Long.MIN_VALUE); + setLongItemsArray(new long[k]); + longsSV = null; + } + + @Override + public byte[] toByteArray() { + return KllHelper.toByteArray(this, false); + } + + @Override + public String toString(final boolean withLevels, final boolean withLevelsAndItems) { + KllSketch sketch = this; + if (withLevelsAndItems && sketchStructure != UPDATABLE) { + final Memory mem = getWritableMemory(); + assert mem != null; + sketch = KllLongsSketch.heapify(getWritableMemory()); + } + return KllHelper.toStringImpl(sketch, withLevels, withLevelsAndItems, getSerDe()); + } + + //SINGLE UPDATE + + @Override + public void update(final long item) { + // Align with KllDoublesSketch + if (readOnly) { throw new SketchesArgumentException(TGT_IS_READ_ONLY_MSG); } + updateLong(this, item); + longsSV = null; + } + + //Also Called from KllLongsHelper::merge + static void updateLong(final KllLongsSketch lngSk, final long item) { + lngSk.updateMinMax(item); + int freeSpace = lngSk.levelsArr[0]; + assert (freeSpace >= 0); + if (freeSpace == 0) { + KllLongsHelper.compressWhileUpdatingSketch(lngSk); + freeSpace = lngSk.levelsArr[0]; + assert (freeSpace > 0); + } + lngSk.incN(1); + lngSk.setLevelZeroSorted(false); + final int nextPos = freeSpace - 1; + lngSk.setLevelsArrayAt(0, nextPos); + lngSk.setLongItemsArrayAt(nextPos, item); + } + + /** + * Single update of min and max + * @param item the source item, it must not be a NaN. + */ + final void updateMinMax(final long item) { + if (isEmpty()) { + setMinItem(item); + setMaxItem(item); + } else { + setMinItem(min(getMinItemInternal(), item)); + setMaxItem(max(getMaxItemInternal(), item)); + } + } + + //WEIGHTED UPDATE + + /** + * Weighted update. Updates this sketch with the given item the number of times specified by the given integer weight. + * @param item the item to be repeated. NaNs are ignored. + * @param weight the number of times the update of item is to be repeated. It must be ≥ one. + */ + public void update(final long item, final long weight) { + // + if (readOnly) { throw new SketchesArgumentException(TGT_IS_READ_ONLY_MSG); } + if (weight < 1L) { throw new SketchesArgumentException("Weight is less than one."); } + if (weight == 1L) { updateLong(this, item); } + else { + if (weight < levelsArr[0]) { + for (int i = 0; i < (int)weight; i++) { updateLong(this, item); } + } else { + final KllHeapLongsSketch tmpSk = new KllHeapLongsSketch(getK(), DEFAULT_M, item, weight); + merge(tmpSk); + } + } + longsSV = null; + } + + // VECTOR UPDATE + + /** + * Vector update. Updates this sketch with the given array (vector) of items, starting at the items + * offset for a length number of items. This is not supported for direct sketches. + * @param items the vector of items + * @param offset the starting index of the items[] array + * @param length the number of items + */ + public void update(final long[] items, final int offset, final int length) { + if (readOnly) { throw new SketchesArgumentException(TGT_IS_READ_ONLY_MSG); } + if (length == 0) { return; } + updateLong(items, offset, length); + longsSV = null; + } + /* Align with KllDoublesSketch + + + + + + + + + + + + + */ + private void updateLong(final long[] srcItems, final int srcOffset, final int length) { + if (isEmpty()) { + setMinItem(srcItems[srcOffset]); //initialize with a real value + setMaxItem(srcItems[srcOffset]); + } + + int count = 0; + while (count < length) { + if (levelsArr[0] == 0) { + KllLongsHelper.compressWhileUpdatingSketch(this); + } + final int spaceNeeded = length - count; + final int freeSpace = levelsArr[0]; + assert (freeSpace > 0); + final int numItemsToCopy = min(spaceNeeded, freeSpace); + final int dstOffset = freeSpace - numItemsToCopy; + final int localSrcOffset = srcOffset + count; + setLongItemsArrayAt(dstOffset, srcItems, localSrcOffset, numItemsToCopy); + updateMinMax(srcItems, localSrcOffset, numItemsToCopy); + count += numItemsToCopy; + incN(numItemsToCopy); + setLevelsArrayAt(0, dstOffset); + } + setLevelZeroSorted(false); + } + + /** + * Vector update of min and max. + * @param srcItems the input source array of values, no NaNs allowed. + * @param srcOffset the starting offset in srcItems + * @param length the number of items to update min and max + */ + private void updateMinMax(final long[] srcItems, final int srcOffset, final int length) { + final int end = srcOffset + length; + for (int i = srcOffset; i < end; i++) { + setMinItem(min(getMinItemInternal(), srcItems[i])); + setMaxItem(max(getMaxItemInternal(), srcItems[i])); + } + } + /* Align with KllDoublesSketch + + + + + + + + + */ + // END ALL UPDATE METHODS + + /** + * @return full size of internal items array including empty space at bottom. + */ + abstract long[] getLongItemsArray(); + + /** + * @return items array of retained items. + */ + abstract long[] getLongRetainedItemsArray(); + + abstract long getLongSingleItem(); + + // Min & Max Methods + + abstract long getMaxItemInternal(); + + abstract void setMaxItem(long item); + + abstract long getMinItemInternal(); + + abstract void setMinItem(long item); + + @Override + abstract byte[] getMinMaxByteArr(); + + @Override + int getMinMaxSizeBytes() { + return Long.BYTES * 2; + } + + //END Min & Max Methods + + @Override + abstract byte[] getRetainedItemsByteArr(); + + @Override + int getRetainedItemsSizeBytes() { + return getNumRetained() * Long.BYTES; + } + + @Override + ArrayOfItemsSerDe> getSerDe() { return null; } + + @Override + final byte[] getSingleItemByteArr() { + final byte[] bytes = new byte[ITEM_BYTES]; + putLongLE(bytes, 0, getLongSingleItem()); + return bytes; + } + + @Override + int getSingleItemSizeBytes() { + return Long.BYTES; + } + + @Override + abstract byte[] getTotalItemsByteArr(); + + @Override + int getTotalItemsNumBytes() { + return levelsArr[getNumLevels()] * Long.BYTES; + } + + abstract void setLongItemsArray(long[] longItems); + + abstract void setLongItemsArrayAt(int index, long item); + + abstract void setLongItemsArrayAt(int dstIndex, long[] srcItems, int srcOffset, int length); + + // SORTED VIEW + + @Override + @SuppressFBWarnings(value = "EI_EXPOSE_REP", justification = "OK in this case.") + public LongsSketchSortedView getSortedView() { + refreshSortedView(); + return longsSV; + } + + private final LongsSketchSortedView refreshSortedView() { + if (longsSV == null) { + final CreateSortedView csv = new CreateSortedView(); + longsSV = csv.getSV(); + } + return longsSV; + } + + private final class CreateSortedView { + long[] quantiles; + long[] cumWeights; + + LongsSketchSortedView getSV() { + if (isEmpty()) { throw new SketchesArgumentException(EMPTY_MSG); } + final long[] srcQuantiles = getLongItemsArray(); + final int[] srcLevels = levelsArr; + final int srcNumLevels = getNumLevels(); + + if (!isLevelZeroSorted()) { + Arrays.sort(srcQuantiles, srcLevels[0], srcLevels[1]); + if (!hasMemory()) { setLevelZeroSorted(true); } + //we don't sort level0 in Memory, only our copy. + } + final int numQuantiles = getNumRetained(); + quantiles = new long[numQuantiles]; + cumWeights = new long[numQuantiles]; + populateFromSketch(srcQuantiles, srcLevels, srcNumLevels, numQuantiles); + return new LongsSketchSortedView( + quantiles, cumWeights, KllLongsSketch.this); + } + + private void populateFromSketch(final long[] srcQuantiles, final int[] srcLevels, + final int srcNumLevels, final int numItems) { + final int[] myLevels = new int[srcNumLevels + 1]; + final int offset = srcLevels[0]; + System.arraycopy(srcQuantiles, offset, quantiles, 0, numItems); + int srcLevel = 0; + int dstLevel = 0; + long weight = 1; + while (srcLevel < srcNumLevels) { + final int fromIndex = srcLevels[srcLevel] - offset; + final int toIndex = srcLevels[srcLevel + 1] - offset; // exclusive + if (fromIndex < toIndex) { // if equal, skip empty level + Arrays.fill(cumWeights, fromIndex, toIndex, weight); + myLevels[dstLevel] = fromIndex; + myLevels[dstLevel + 1] = toIndex; + dstLevel++; + } + srcLevel++; + weight *= 2; + } + final int numLevels = dstLevel; + blockyTandemMergeSort(quantiles, cumWeights, myLevels, numLevels); //create unit weights + KllHelper.convertToCumulative(cumWeights); + } + } //End of class CreateSortedView + + private static void blockyTandemMergeSort(final long[] quantiles, final long[] weights, + final int[] levels, final int numLevels) { + if (numLevels == 1) { return; } + + // duplicate the input in preparation for the "ping-pong" copy reduction strategy. + final long[] quantilesTmp = Arrays.copyOf(quantiles, quantiles.length); + final long[] weightsTmp = Arrays.copyOf(weights, quantiles.length); // don't need the extra one + + blockyTandemMergeSortRecursion(quantilesTmp, weightsTmp, quantiles, weights, levels, 0, numLevels); + } + + private static void blockyTandemMergeSortRecursion( + final long[] quantilesSrc, final long[] weightsSrc, + final long[] quantilesDst, final long[] weightsDst, + final int[] levels, final int startingLevel, final int numLevels) { + if (numLevels == 1) { return; } + final int numLevels1 = numLevels / 2; + final int numLevels2 = numLevels - numLevels1; + assert numLevels1 >= 1; + assert numLevels2 >= numLevels1; + final int startingLevel1 = startingLevel; + final int startingLevel2 = startingLevel + numLevels1; + // swap roles of src and dst + blockyTandemMergeSortRecursion( + quantilesDst, weightsDst, + quantilesSrc, weightsSrc, + levels, startingLevel1, numLevels1); + blockyTandemMergeSortRecursion( + quantilesDst, weightsDst, + quantilesSrc, weightsSrc, + levels, startingLevel2, numLevels2); + tandemMerge( + quantilesSrc, weightsSrc, + quantilesDst, weightsDst, + levels, + startingLevel1, numLevels1, + startingLevel2, numLevels2); + } + + private static void tandemMerge( + final long[] quantilesSrc, final long[] weightsSrc, + final long[] quantilesDst, final long[] weightsDst, + final int[] levelStarts, + final int startingLevel1, final int numLevels1, + final int startingLevel2, final int numLevels2) { + final int fromIndex1 = levelStarts[startingLevel1]; + final int toIndex1 = levelStarts[startingLevel1 + numLevels1]; // exclusive + final int fromIndex2 = levelStarts[startingLevel2]; + final int toIndex2 = levelStarts[startingLevel2 + numLevels2]; // exclusive + int iSrc1 = fromIndex1; + int iSrc2 = fromIndex2; + int iDst = fromIndex1; + + while (iSrc1 < toIndex1 && iSrc2 < toIndex2) { + if (quantilesSrc[iSrc1] < quantilesSrc[iSrc2]) { + quantilesDst[iDst] = quantilesSrc[iSrc1]; + weightsDst[iDst] = weightsSrc[iSrc1]; + iSrc1++; + } else { + quantilesDst[iDst] = quantilesSrc[iSrc2]; + weightsDst[iDst] = weightsSrc[iSrc2]; + iSrc2++; + } + iDst++; + } + if (iSrc1 < toIndex1) { + System.arraycopy(quantilesSrc, iSrc1, quantilesDst, iDst, toIndex1 - iSrc1); + System.arraycopy(weightsSrc, iSrc1, weightsDst, iDst, toIndex1 - iSrc1); + } else if (iSrc2 < toIndex2) { + System.arraycopy(quantilesSrc, iSrc2, quantilesDst, iDst, toIndex2 - iSrc2); + System.arraycopy(weightsSrc, iSrc2, weightsDst, iDst, toIndex2 - iSrc2); + } + } + + // END SORTED VIEW + +} diff --git a/src/main/java/org/apache/datasketches/kll/KllLongsSketchIterator.java b/src/main/java/org/apache/datasketches/kll/KllLongsSketchIterator.java new file mode 100644 index 000000000..8922ec70e --- /dev/null +++ b/src/main/java/org/apache/datasketches/kll/KllLongsSketchIterator.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.datasketches.kll; + +import org.apache.datasketches.quantilescommon.QuantilesLongsSketchIterator; + +/** + * Iterator over KllLongsSketch. The order is not defined. + */ +public final class KllLongsSketchIterator extends KllSketchIterator implements QuantilesLongsSketchIterator { + private final long[] quantiles; + + KllLongsSketchIterator(final long[] quantiles, final int[] levelsArr, final int numLevels) { + super(levelsArr, numLevels); + this.quantiles = quantiles; + } + + @Override + public long getQuantile() { + return quantiles[index]; + } + +} diff --git a/src/main/java/org/apache/datasketches/kll/KllPreambleUtil.java b/src/main/java/org/apache/datasketches/kll/KllPreambleUtil.java index 18e4de3f0..f2c3847ae 100644 --- a/src/main/java/org/apache/datasketches/kll/KllPreambleUtil.java +++ b/src/main/java/org/apache/datasketches/kll/KllPreambleUtil.java @@ -28,6 +28,7 @@ import static org.apache.datasketches.kll.KllSketch.SketchType.DOUBLES_SKETCH; import static org.apache.datasketches.kll.KllSketch.SketchType.FLOATS_SKETCH; import static org.apache.datasketches.kll.KllSketch.SketchType.ITEMS_SKETCH; +import static org.apache.datasketches.kll.KllSketch.SketchType.LONGS_SKETCH; import java.util.Objects; @@ -210,7 +211,7 @@ static String toString(final Memory mem, final SketchType sketchType, final bool * Used primarily in testing. * * @param mem the given Memory - * @param sketchType the sketch type: FLOATS_SKETCH, DOUBLES_SKETCH, or ITEMS_SKETCH. + * @param sketchType the sketch type: FLOATS_SKETCH, DOUBLES_SKETCH, LONGS_SKETCH, or ITEMS_SKETCH. * @param includeData if true, includes detail of retained data. * @param serDe must be supplied for KllItemsSketch, otherwise can be null. * @return the summary string. @@ -226,10 +227,9 @@ staticThe resulting approximations have a probabilistic guarantee that can be obtained from the + * getNormalizedRankError(false) function.
+ * + * @param splitPoints an array of m unique, monotonically increasing items + * (of the same type as the input items) + * that divide the item input domain into m+1 overlapping intervals. + * + *The start of each interval is below the lowest item retained by the sketch + * corresponding to a zero rank or zero probability, and the end of the interval + * is the rank or cumulative probability corresponding to the split point.
+ * + *The (m+1)th interval represents 100% of the distribution represented by the sketch + * and consistent with the definition of a cumulative probability distribution, thus the (m+1)th + * rank or probability in the returned array is always 1.0.
+ * + *If a split point exactly equals a retained item of the sketch and the search criterion is:
+ * + *It is not recommended to include either the minimum or maximum items of the input stream.
+ * + * @param searchCrit the desired search criteria. + * @return a discrete CDF array of m+1 double ranks (or cumulative probabilities) on the interval [0.0, 1.0]. + * @throws IllegalArgumentException if sketch is empty. + */ + default double[] getCDF(long[] splitPoints, QuantileSearchCriteria searchCrit) { + QuantilesUtil.checkLongsSplitPointsOrder(splitPoints); + final int len = splitPoints.length + 1; + final double[] buckets = new double[len]; + for (int i = 0; i < len - 1; i++) { + buckets[i] = getRank(splitPoints[i], searchCrit); + } + buckets[len - 1] = 1; + return buckets; + } + + /** + * Returns the maximum item of the stream. This may be distinct from the largest item retained by the + * sketch algorithm. + * + * @return the maximum item of the stream + * @throws IllegalArgumentException if sketch is empty. + */ + long getMaxItem(); + + /** + * Returns the minimum item of the stream. This may be distinct from the smallest item retained by the + * sketch algorithm. + * + * @return the minimum item of the stream + * @throws IllegalArgumentException if sketch is empty. + */ + long getMinItem(); + + /** + * Returns an approximation to the Probability Mass Function (PMF) of the input stream + * as an array of probability masses as doubles on the interval [0.0, 1.0], + * given a set of splitPoints. + * + *The resulting approximations have a probabilistic guarantee that can be obtained from the + * getNormalizedRankError(true) function.
+ * + * @param splitPoints an array of m unique, monotonically increasing items + * (of the same type as the input items) + * that divide the item input domain into m+1 consecutive, non-overlapping intervals. + * + *Each interval except for the end intervals starts with a split point and ends with the next split + * point in sequence.
+ * + *The first interval starts below the lowest item retained by the sketch + * corresponding to a zero rank or zero probability, and ends with the first split point
+ * + *The last (m+1)th interval starts with the last split point and ends after the last + * item retained by the sketch corresponding to a rank or probability of 1.0.
+ * + *The sum of the probability masses of all (m+1) intervals is 1.0.
+ * + *If the search criterion is:
+ * + *It is not recommended to include either the minimum or maximum items of the input stream.
+ * + * @param searchCrit the desired search criteria. + * @return a PMF array of m+1 probability masses as doubles on the interval [0.0, 1.0]. + * @throws IllegalArgumentException if sketch is empty. + */ + default double[] getPMF(long[] splitPoints, QuantileSearchCriteria searchCrit) { + final double[] buckets = getCDF(splitPoints, searchCrit); + final int len = buckets.length; + for (int i = len; i-- > 1; ) { + buckets[i] -= buckets[i - 1]; + } + return buckets; + } + + /** + * Gets the approximate quantile of the given normalized rank and the given search criterion. + * + * @param rank the given normalized rank, a double in the range [0.0, 1.0]. + * @param searchCrit If INCLUSIVE, the given rank includes all quantiles ≤ + * the quantile directly corresponding to the given rank. + * If EXCLUSIVE, he given rank includes all quantiles < + * the quantile directly corresponding to the given rank. + * @return the approximate quantile given the normalized rank. + * @throws IllegalArgumentException if sketch is empty. + * @see QuantileSearchCriteria + */ + long getQuantile(double rank, QuantileSearchCriteria searchCrit); + + /** + * Returns an array of all retained quantiles by the sketch. + * @return an array of all retained quantiles by the sketch. + */ + long[] getQuantiles(); + + /** + * Gets the normalized rank corresponding to the given a quantile. + * + * @param quantile the given quantile + * @param searchCrit if INCLUSIVE the given quantile is included into the rank. + * @return the normalized rank corresponding to the given quantile. + * @throws IllegalArgumentException if sketch is empty. + * @see QuantileSearchCriteria + */ + double getRank(long quantile, QuantileSearchCriteria searchCrit); + + @Override + LongsSortedViewIterator iterator(); + +} diff --git a/src/main/java/org/apache/datasketches/quantilescommon/LongsSortedViewIterator.java b/src/main/java/org/apache/datasketches/quantilescommon/LongsSortedViewIterator.java new file mode 100644 index 000000000..77510cd5a --- /dev/null +++ b/src/main/java/org/apache/datasketches/quantilescommon/LongsSortedViewIterator.java @@ -0,0 +1,51 @@ +/* + * 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.quantilescommon; + +/** + * Iterator over quantile sketches of primitive type long. + */ +public final class LongsSortedViewIterator extends SortedViewIterator { + private final long[] quantiles; + + /** + * Constructor. + * @param quantiles the given array of quantiles, which must be ordered. + * @param cumWeights the given array of cumulative weights, which must be ordered, start with the value one, and + * the last value must be equal to N, the total number of items updated to the sketch. + */ + public LongsSortedViewIterator(final long[] quantiles, final long[] cumWeights) { + super(cumWeights); + this.quantiles = quantiles; //SpotBugs EI_EXPOSE_REP2 suppressed by FindBugsExcludeFilter + } + + /** + * Gets the quantile at the current index. + * + *Don't call this before calling next() for the first time + * or after getting false from next().
+ * + * @return the quantile at the current index. + */ + public long getQuantile() { + return quantiles[index]; + } + +} diff --git a/src/main/java/org/apache/datasketches/quantilescommon/QuantilesLongsAPI.java b/src/main/java/org/apache/datasketches/quantilescommon/QuantilesLongsAPI.java new file mode 100644 index 000000000..2b542a68f --- /dev/null +++ b/src/main/java/org/apache/datasketches/quantilescommon/QuantilesLongsAPI.java @@ -0,0 +1,300 @@ +/* + * 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.quantilescommon; + +import static org.apache.datasketches.quantilescommon.QuantileSearchCriteria.INCLUSIVE; + +/** + * The Quantiles API for item type long. + * @see QuantilesAPI + * @author Lee Rhodes + * @author Zac Blanco + */ +public interface QuantilesLongsAPI extends QuantilesAPI { + + /** + * This is equivalent to {@link #getCDF(long[], QuantileSearchCriteria) getCDF(splitPoints, INCLUSIVE)} + * @param splitPoints an array of m unique, monotonically increasing items. + * @return a discrete CDF array of m+1 double ranks (or cumulative probabilities) on the interval [0.0, 1.0]. + * @throws IllegalArgumentException if sketch is empty. + */ + default double[] getCDF(long[] splitPoints) { + return getCDF(splitPoints, INCLUSIVE); + } + + /** + * Returns an approximation to the Cumulative Distribution Function (CDF) of the input stream + * as a monotonically increasing array of double ranks (or cumulative probabilities) on the interval [0.0, 1.0], + * given a set of splitPoints. + * + *The resulting approximations have a probabilistic guarantee that can be obtained from the + * getNormalizedRankError(false) function.
+ * + * @param splitPoints an array of m unique, monotonically increasing items + * (of the same type as the input items) + * that divide the item input domain into m+1 overlapping intervals. + * + *The start of each interval is below the lowest item retained by the sketch + * corresponding to a zero rank or zero probability, and the end of the interval + * is the rank or cumulative probability corresponding to the split point.
+ * + *The (m+1)th interval represents 100% of the distribution represented by the sketch + * and consistent with the definition of a cumulative probability distribution, thus the (m+1)th + * rank or probability in the returned array is always 1.0.
+ * + *If a split point exactly equals a retained item of the sketch and the search criterion is:
+ * + *It is not recommended to include either the minimum or maximum items of the input stream.
+ * + * @param searchCrit the desired search criteria. + * @return a discrete CDF array of m+1 double ranks (or cumulative probabilities) on the interval [0.0, 1.0]. + * @throws IllegalArgumentException if sketch is empty. + */ + double[] getCDF(long[] splitPoints, QuantileSearchCriteria searchCrit); + + /** + * Returns the maximum item of the stream. This is provided for convenience and may be different from the + * item returned by getQuantile(1.0). + * + * @return the maximum item of the stream + * @throws IllegalArgumentException if sketch is empty. + */ + long getMaxItem(); + + /** + * Returns the minimum item of the stream. This is provided for convenience and may be different from the + * item returned by getQuantile(0.0). + * + * @return the minimum item of the stream + * @throws IllegalArgumentException if sketch is empty. + */ + long getMinItem(); + + /** + * This is equivalent to {@link #getPMF(long[], QuantileSearchCriteria) getPMF(splitPoints, INCLUSIVE)} + * @param splitPoints an array of m unique, monotonically increasing items. + * @return a PMF array of m+1 probability masses as doubles on the interval [0.0, 1.0]. + * @throws IllegalArgumentException if sketch is empty. + */ + default double[] getPMF(long[] splitPoints) { + return getPMF(splitPoints, INCLUSIVE); + } + + /** + * Returns an approximation to the Probability Mass Function (PMF) of the input stream + * as an array of probability masses as doubles on the interval [0.0, 1.0], + * given a set of splitPoints. + * + *The resulting approximations have a probabilistic guarantee that can be obtained from the + * getNormalizedRankError(true) function.
+ * + * @param splitPoints an array of m unique, monotonically increasing items + * (of the same type as the input items) + * that divide the item input domain into m+1 consecutive, non-overlapping intervals. + * + *Each interval except for the end intervals starts with a split point and ends with the next split + * point in sequence.
+ * + *The first interval starts below the lowest item retained by the sketch + * corresponding to a zero rank or zero probability, and ends with the first split point
+ * + *The last (m+1)th interval starts with the last split point and ends after the last + * item retained by the sketch corresponding to a rank or probability of 1.0.
+ * + *The sum of the probability masses of all (m+1) intervals is 1.0.
+ * + *If the search criterion is:
+ * + *It is not recommended to include either the minimum or maximum items of the input stream.
+ * + * @param searchCrit the desired search criteria. + * @return a PMF array of m+1 probability masses as doubles on the interval [0.0, 1.0]. + * @throws IllegalArgumentException if sketch is empty. + */ + double[] getPMF(long[] splitPoints, QuantileSearchCriteria searchCrit); + + /** + * This is equivalent to {@link #getQuantile(double, QuantileSearchCriteria) getQuantile(rank, INCLUSIVE)} + * @param rank the given normalized rank, a double in the range [0.0, 1.0]. + * @return the approximate quantile given the normalized rank. + * @throws IllegalArgumentException if sketch is empty. + */ + default long getQuantile(double rank) { + return getQuantile(rank, INCLUSIVE); + } + + /** + * Gets the approximate quantile of the given normalized rank and the given search criterion. + * + * @param rank the given normalized rank, a double in the range [0.0, 1.0]. + * @param searchCrit If INCLUSIVE, the given rank includes all quantiles ≤ + * the quantile directly corresponding to the given rank. + * If EXCLUSIVE, he given rank includes all quantiles < + * the quantile directly corresponding to the given rank. + * @return the approximate quantile given the normalized rank. + * @throws IllegalArgumentException if sketch is empty. + * @see org.apache.datasketches.quantilescommon.QuantileSearchCriteria + */ + long getQuantile(double rank, QuantileSearchCriteria searchCrit); + + /** + * Gets the lower bound of the quantile confidence interval in which the quantile of the + * given rank exists. + * + *Although it is possible to estimate the probability that the true quantile + * exists within the quantile confidence interval specified by the upper and lower quantile bounds, + * it is not possible to guarantee the width of the quantile confidence interval + * as an additive or multiplicative percent of the true quantile.
+ * + * @param rank the given normalized rank + * @return the lower bound of the quantile confidence interval in which the quantile of the + * given rank exists. + * @throws IllegalArgumentException if sketch is empty. + */ + long getQuantileLowerBound(double rank); + + /** + * Gets the upper bound of the quantile confidence interval in which the true quantile of the + * given rank exists. + * + *Although it is possible to estimate the probability that the true quantile + * exists within the quantile confidence interval specified by the upper and lower quantile bounds, + * it is not possible to guarantee the width of the quantile interval + * as an additive or multiplicative percent of the true quantile.
+ * + * @param rank the given normalized rank + * @return the upper bound of the quantile confidence interval in which the true quantile of the + * given rank exists. + * @throws IllegalArgumentException if sketch is empty. + */ + long getQuantileUpperBound(double rank); + + /** + * This is equivalent to {@link #getQuantiles(double[], QuantileSearchCriteria) getQuantiles(ranks, INCLUSIVE)} + * @param ranks the given array of normalized ranks, each of which must be + * in the interval [0.0,1.0]. + * @return an array of quantiles corresponding to the given array of normalized ranks. + * @throws IllegalArgumentException if sketch is empty. + */ + default long[] getQuantiles(double[] ranks) { + return getQuantiles(ranks, INCLUSIVE); + } + + /** + * Gets an array of quantiles from the given array of normalized ranks. + * + * @param ranks the given array of normalized ranks, each of which must be + * in the interval [0.0,1.0]. + * @param searchCrit if INCLUSIVE, the given ranks include all quantiles ≤ + * the quantile directly corresponding to each rank. + * @return an array of quantiles corresponding to the given array of normalized ranks. + * @throws IllegalArgumentException if sketch is empty. + * @see org.apache.datasketches.quantilescommon.QuantileSearchCriteria + */ + long[] getQuantiles(double[] ranks, QuantileSearchCriteria searchCrit); + + /** + * This is equivalent to {@link #getRank(long, QuantileSearchCriteria) getRank(quantile, INCLUSIVE)} + * @param quantile the given quantile + * @return the normalized rank corresponding to the given quantile + * @throws IllegalArgumentException if sketch is empty. + */ + default double getRank(long quantile) { + return getRank(quantile, INCLUSIVE); + } + + /** + * Gets the normalized rank corresponding to the given a quantile. + * + * @param quantile the given quantile + * @param searchCrit if INCLUSIVE the given quantile is included into the rank. + * @return the normalized rank corresponding to the given quantile + * @throws IllegalArgumentException if sketch is empty. + * @see org.apache.datasketches.quantilescommon.QuantileSearchCriteria + */ + double getRank(long quantile, QuantileSearchCriteria searchCrit); + + /** + * This is equivalent to {@link #getRanks(long[], QuantileSearchCriteria) getRanks(quantiles, INCLUSIVE)} + * @param quantiles the given array of quantiles + * @return an array of normalized ranks corresponding to the given array of quantiles. + * @throws IllegalArgumentException if sketch is empty. + */ + default double[] getRanks(long[] quantiles) { + return getRanks(quantiles, INCLUSIVE); + } + + /** + * Gets an array of normalized ranks corresponding to the given array of quantiles and the given + * search criterion. + * + * @param quantiles the given array of quantiles + * @param searchCrit if INCLUSIVE, the given quantiles include the rank directly corresponding to each quantile. + * @return an array of normalized ranks corresponding to the given array of quantiles. + * @throws IllegalArgumentException if sketch is empty. + * @see org.apache.datasketches.quantilescommon.QuantileSearchCriteria + */ + double[] getRanks(long[] quantiles, QuantileSearchCriteria searchCrit); + + /** + * Returns the current number of bytes this Sketch would require if serialized. + * @return the number of bytes this sketch would require if serialized. + */ + int getSerializedSizeBytes(); + + /** + * Gets the sorted view of this sketch + * @return the sorted view of this sketch + */ + LongsSortedView getSortedView(); + + /** + * Gets the iterator for this sketch, which is not sorted. + * @return the iterator for this sketch + */ + QuantilesLongsSketchIterator iterator(); + + /** + * Returns a byte array representation of this sketch. + * @return a byte array representation of this sketch. + */ + byte[] toByteArray(); + + /** + * Updates this sketch with the given item. + * @param item from a stream of items. NaNs are ignored. + */ + void update(long item); + +} diff --git a/src/main/java/org/apache/datasketches/quantilescommon/QuantilesLongsSketchIterator.java b/src/main/java/org/apache/datasketches/quantilescommon/QuantilesLongsSketchIterator.java new file mode 100644 index 000000000..7ed0d9805 --- /dev/null +++ b/src/main/java/org/apache/datasketches/quantilescommon/QuantilesLongsSketchIterator.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.datasketches.quantilescommon; + +/** + * The quantiles sketch iterator for primitive type long. + * @see QuantilesSketchIterator + * @author Zac Blanco + */ +public interface QuantilesLongsSketchIterator extends QuantilesSketchIterator { + + /** + * Gets the long quantile at the current index. + * + *Don't call this before calling next() for the first time + * or after getting false from next().
+ * + * @return the long quantile at the current index. + */ + long getQuantile(); + +} + diff --git a/src/main/java/org/apache/datasketches/quantilescommon/QuantilesUtil.java b/src/main/java/org/apache/datasketches/quantilescommon/QuantilesUtil.java index 75798c20f..529fd386d 100644 --- a/src/main/java/org/apache/datasketches/quantilescommon/QuantilesUtil.java +++ b/src/main/java/org/apache/datasketches/quantilescommon/QuantilesUtil.java @@ -67,6 +67,21 @@ public static final void checkDoublesSplitPointsOrder(final double[] values) { } } + /** + * Checks the sequential validity of the given array of double values. + * They must be unique and monotonically increasing. + * @param values the given array of double values + */ + public static void checkLongsSplitPointsOrder(final long[] values) { + Objects.requireNonNull(values); + final int len = values.length; + for (int j = 0; j < len - 1; j++) { + if (values[j] < values[j + 1]) { continue; } + throw new SketchesArgumentException( + "Values must be unique and monotonically increasing."); + } + } + /** * Checks the sequential validity of the given array of float values. * They must be unique, monotonically increasing and not NaN. diff --git a/src/test/java/org/apache/datasketches/kll/KllCrossLanguageTest.java b/src/test/java/org/apache/datasketches/kll/KllCrossLanguageTest.java index 078f3503b..53b422b7c 100644 --- a/src/test/java/org/apache/datasketches/kll/KllCrossLanguageTest.java +++ b/src/test/java/org/apache/datasketches/kll/KllCrossLanguageTest.java @@ -67,6 +67,16 @@ public void generateKllFloatsSketchBinaries() throws IOException { } } + @Test(groups = {GENERATE_JAVA_FILES}) + public void generateKllLongsSketchBinaries() throws IOException { + final int[] nArr = {0, 1, 10, 100, 1_000, 10_000, 100_000, 1_000_000}; + for (int n: nArr) { + final KllLongsSketch sk = KllLongsSketch.newHeapInstance(); + for (int i = 1; i <= n; i++) { sk.update(i); } + Files.newOutputStream(javaPath.resolve("kll_long_n" + n + "_java.sk")).write(sk.toByteArray()); + } + } + @Test(groups = {GENERATE_JAVA_FILES}) public void generateKllItemsSketchBinaries() throws IOException { final int[] nArr = {0, 1, 10, 100, 1_000, 10_000, 100_000, 1_000_000}; diff --git a/src/test/java/org/apache/datasketches/kll/KllDirectCompactDoublesSketchTest.java b/src/test/java/org/apache/datasketches/kll/KllDirectCompactDoublesSketchTest.java index 9831c2f57..7a4d061ad 100644 --- a/src/test/java/org/apache/datasketches/kll/KllDirectCompactDoublesSketchTest.java +++ b/src/test/java/org/apache/datasketches/kll/KllDirectCompactDoublesSketchTest.java @@ -110,13 +110,13 @@ public void checkDirectCompactGetDoubleItemsArray() { KllDoublesSketch sk2 = KllDoublesSketch.wrap(Memory.wrap(sk.toByteArray())); double[] itemsArr = sk2.getDoubleItemsArray(); - for (int i = 0; i < 20; i++) { assertEquals(itemsArr[i], 0F); } + for (int i = 0; i < 20; i++) { assertEquals(itemsArr[i], 0.0); } sk.update(1); sk2 = KllDoublesSketch.wrap(Memory.wrap(sk.toByteArray())); itemsArr = sk2.getDoubleItemsArray(); - for (int i = 0; i < 19; i++) { assertEquals(itemsArr[i], 0F); } - assertEquals(itemsArr[19], 1F); + for (int i = 0; i < 19; i++) { assertEquals(itemsArr[i], 0.0); } + assertEquals(itemsArr[19], 1.0); for (int i = 2; i <= 21; i++) { sk.update(i); } sk2 = KllDoublesSketch.wrap(Memory.wrap(sk.toByteArray())); @@ -169,12 +169,12 @@ public void checkMinAndMax() { try { sk2.getMaxItem(); fail(); } catch (SketchesArgumentException e) {} sk.update(1); sk2 = KllDoublesSketch.wrap(Memory.wrap(sk.toByteArray())); - assertEquals(sk2.getMaxItem(),1.0F); - assertEquals(sk2.getMinItem(),1.0F); + assertEquals(sk2.getMaxItem(),1.0); + assertEquals(sk2.getMinItem(),1.0); for (int i = 2; i <= 21; i++) { sk.update(i); } sk2 = KllDoublesSketch.wrap(Memory.wrap(sk.toByteArray())); - assertEquals(sk2.getMaxItem(),21.0F); - assertEquals(sk2.getMinItem(),1.0F); + assertEquals(sk2.getMaxItem(),21.0); + assertEquals(sk2.getMinItem(),1.0); } @Test diff --git a/src/test/java/org/apache/datasketches/kll/KllDirectCompactLongsSketchTest.java b/src/test/java/org/apache/datasketches/kll/KllDirectCompactLongsSketchTest.java new file mode 100644 index 000000000..6b57fccac --- /dev/null +++ b/src/test/java/org/apache/datasketches/kll/KllDirectCompactLongsSketchTest.java @@ -0,0 +1,224 @@ +/* + * 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.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +import org.apache.datasketches.common.SketchesArgumentException; +import org.apache.datasketches.kll.KllDirectLongsSketch.KllDirectCompactLongsSketch; +import org.apache.datasketches.memory.DefaultMemoryRequestServer; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.memory.WritableMemory; +import org.testng.annotations.Test; + +public class KllDirectCompactLongsSketchTest { + private static final DefaultMemoryRequestServer memReqSvr = new DefaultMemoryRequestServer(); + + @Test + public void checkRODirectUpdatable_ROandWritable() { + int k = 20; + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= k + 1; i++) { sk.update(i); } + byte[] byteArr = KllHelper.toByteArray(sk, true); //request updatable + Memory srcMem = Memory.wrap(byteArr); //cast to Memory -> read only + KllLongsSketch sk2 = KllLongsSketch.wrap(srcMem); + assertTrue(sk2 instanceof KllDirectLongsSketch); + + assertTrue(sk2.isMemoryUpdatableFormat()); + assertTrue(sk2.isReadOnly()); + assertEquals(sk2.getMinItem(), 1L); + assertEquals(sk2.getMaxItem(), 21L); + + WritableMemory srcWmem = WritableMemory.writableWrap(byteArr); + KllLongsSketch sk3 = KllLongsSketch.writableWrap(srcWmem, memReqSvr); + assertTrue(sk3 instanceof KllDirectLongsSketch); + println(sk3.toString(true, false)); + assertFalse(sk3.isReadOnly()); + sk3.update(22); + assertEquals(sk2.getMinItem(), 1L); + assertEquals(sk2.getMaxItem(), 22L); + } + + @Test + public void checkRODirectCompact() { + int k = 20; + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= k + 1; i++) { sk.update(i); } + Memory srcMem = Memory.wrap(sk.toByteArray()); //compact RO fmt + KllLongsSketch sk2 = KllLongsSketch.wrap(srcMem); + assertTrue(sk2 instanceof KllDirectCompactLongsSketch); + //println(sk2.toString(true, false)); + assertFalse(sk2.isMemoryUpdatableFormat()); + assertTrue(sk2.isReadOnly()); + assertEquals(sk2.getMinItem(), 1L); + assertEquals(sk2.getMaxItem(), 21L); + Memory srcMem2 = Memory.wrap(sk2.toByteArray()); + KllLongsSketch sk3 = KllLongsSketch.writableWrap((WritableMemory)srcMem2, memReqSvr); + assertTrue(sk3 instanceof KllDirectCompactLongsSketch); + assertFalse(sk2.isMemoryUpdatableFormat()); + //println(sk3.toString(true, false)); + assertTrue(sk3.isReadOnly()); + assertEquals(sk3.getMinItem(), 1L); + assertEquals(sk3.getMaxItem(), 21L); + } + + @Test + public void checkDirectCompactSingleItem() { + int k = 20; + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + + sk.update(1); + KllLongsSketch sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + assertTrue(sk2 instanceof KllDirectCompactLongsSketch); + //println(sk2.toString(true, false)); + assertTrue(sk2.isReadOnly()); + assertEquals(sk2.getLongSingleItem(), 1L); + + sk.update(2); + sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + assertEquals(sk2.getN(), 2); + try { + sk2.getLongSingleItem(); + fail(); + } catch (SketchesArgumentException e) { } + } + + @Test + public void checkDirectCompactGetLongItemsArray() { + int k = 20; + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + + KllLongsSketch sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + long[] itemsArr = sk2.getLongItemsArray(); + for (int i = 0; i < 20; i++) { assertEquals(itemsArr[i], 0); } + + sk.update(1); + sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + itemsArr = sk2.getLongItemsArray(); + for (int i = 0; i < 19; i++) { assertEquals(itemsArr[i], 0); } + assertEquals(itemsArr[19], 1L); + + for (int i = 2; i <= 21; i++) { sk.update(i); } + sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + itemsArr = sk2.getLongItemsArray(); + assertEquals(itemsArr.length, 33); + assertEquals(itemsArr[22], 21); + } + + @Test + public void checkHeapAndDirectCompactGetRetainedItemsArray() { + int k = 20; + + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + long[] retArr = sk.getLongRetainedItemsArray(); + assertEquals(retArr.length, 0); + + KllLongsSketch sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + retArr = sk2.getLongRetainedItemsArray(); + assertEquals(retArr.length, sk.getNumRetained()); + assertEquals(retArr.length, 0); + + sk.update(1); + retArr = sk.getLongRetainedItemsArray(); + assertEquals(retArr.length, sk.getNumRetained()); + assertEquals(retArr.length, 1); + assertEquals(retArr[0], 1L); + + sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + retArr = sk2.getLongRetainedItemsArray(); + assertEquals(retArr.length, sk.getNumRetained()); + assertEquals(retArr.length, 1); + assertEquals(retArr[0], 1L); + + for (int i = 2; i <= 21; i++) { sk.update(i); } + retArr = sk.getLongRetainedItemsArray(); + assertEquals(retArr.length, sk.getNumRetained()); + assertEquals(retArr.length, 11); + + sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + assertEquals(retArr.length, sk2.getNumRetained()); + assertEquals(retArr.length, 11); + } + + @Test + public void checkMinAndMax() { + int k = 20; + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + KllLongsSketch sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + try { sk2.getMinItem(); fail(); } catch (SketchesArgumentException e) {} + try { sk2.getMaxItem(); fail(); } catch (SketchesArgumentException e) {} + sk.update(1); + sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + assertEquals(sk2.getMaxItem(),1L); + assertEquals(sk2.getMinItem(),1L); + for (int i = 2; i <= 21; i++) { sk.update(i); } + sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + assertEquals(sk2.getMaxItem(),21L); + assertEquals(sk2.getMinItem(),1L); + } + + @Test + public void checkQuantile() { + KllLongsSketch sk1 = KllLongsSketch.newHeapInstance(); + for (int i = 1; i <= 1000; i++) { sk1.update(i); } + KllLongsSketch sk2 = KllLongsSketch.wrap(Memory.wrap(sk1.toByteArray())); + long med2 = sk2.getQuantile(0.5); + long med1 = sk1.getQuantile(0.5); + assertEquals(med1, med2); + println("Med1: " + med1); + println("Med2: " + med2); + } + + @Test + public void checkCompactSingleItemMerge() { + int k = 20; + KllLongsSketch skH1 = KllLongsSketch.newHeapInstance(k); //Heap with 1 (single) + skH1.update(21); + KllLongsSketch skDC1 = KllLongsSketch.wrap(Memory.wrap(skH1.toByteArray())); //Direct Compact with 1 (single) + KllLongsSketch skH20 = KllLongsSketch.newHeapInstance(k); //Heap with 20 + for (int i = 1; i <= 20; i++) { skH20.update(i); } + skH20.merge(skDC1); + assertEquals(skH20.getN(), 21); + + WritableMemory wmem = WritableMemory.allocate(1000); + KllLongsSketch skDU20 = KllLongsSketch.newDirectInstance(k, wmem, memReqSvr);//Direct Updatable with 21 + for (int i = 1; i <= 20; i++) { skDU20.update(i); } + skDU20.merge(skDC1); + assertEquals(skDU20.getN(), 21); + } + + @Test + public void printlnTest() { + println("PRINTING: " + this.getClass().getName()); + } + + private final static boolean enablePrinting = false; + + /** + * @param o the Object to println + */ + private static final void println(final Object o) { + if (enablePrinting) { System.out.println(o.toString()); } + } + +} diff --git a/src/test/java/org/apache/datasketches/kll/KllDirectDoublesSketchIteratorTest.java b/src/test/java/org/apache/datasketches/kll/KllDirectDoublesSketchIteratorTest.java index 78a3b9cd5..4bfdfa4fc 100644 --- a/src/test/java/org/apache/datasketches/kll/KllDirectDoublesSketchIteratorTest.java +++ b/src/test/java/org/apache/datasketches/kll/KllDirectDoublesSketchIteratorTest.java @@ -41,7 +41,7 @@ public void oneItemSketch() { sketch.update(0); QuantilesDoublesSketchIterator it = sketch.iterator(); Assert.assertTrue(it.next()); - Assert.assertEquals(it.getQuantile(), 0f); + Assert.assertEquals(it.getQuantile(), 0); Assert.assertEquals(it.getWeight(), 1); Assert.assertFalse(it.next()); } diff --git a/src/test/java/org/apache/datasketches/kll/KllDirectDoublesSketchTest.java b/src/test/java/org/apache/datasketches/kll/KllDirectDoublesSketchTest.java index 33219a806..6342ac33d 100644 --- a/src/test/java/org/apache/datasketches/kll/KllDirectDoublesSketchTest.java +++ b/src/test/java/org/apache/datasketches/kll/KllDirectDoublesSketchTest.java @@ -189,11 +189,11 @@ public void mergeLowerK() { sketch2.update(2 * n - i - 1); } - assertEquals(sketch1.getMinItem(), 0.0f); - assertEquals(sketch1.getMaxItem(), n - 1f); + assertEquals(sketch1.getMinItem(), 0.0); + assertEquals(sketch1.getMaxItem(), n - 1.0); assertEquals(sketch2.getMinItem(), n); - assertEquals(sketch2.getMaxItem(), 2f * n - 1f); + assertEquals(sketch2.getMaxItem(), 2.0 * n - 1.0); assertTrue(sketch1.getNormalizedRankError(false) < sketch2.getNormalizedRankError(false)); assertTrue(sketch1.getNormalizedRankError(true) < sketch2.getNormalizedRankError(true)); @@ -613,7 +613,7 @@ public void checkWritableWrapOfCompactForm() { public void checkReadOnlyExceptions() { int k = 20; double[] dblArr = new double[0]; - double dblV = 1.0f; + double dblV = 1.0; int idx = 1; boolean bool = true; KllDoublesSketch sk = KllDoublesSketch.newHeapInstance(k); diff --git a/src/test/java/org/apache/datasketches/kll/KllDirectLongsSketchIteratorTest.java b/src/test/java/org/apache/datasketches/kll/KllDirectLongsSketchIteratorTest.java new file mode 100644 index 000000000..8be509f10 --- /dev/null +++ b/src/test/java/org/apache/datasketches/kll/KllDirectLongsSketchIteratorTest.java @@ -0,0 +1,78 @@ +/* + * 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 org.apache.datasketches.memory.DefaultMemoryRequestServer; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.datasketches.quantilescommon.QuantilesLongsSketchIterator; +import org.testng.Assert; +import org.testng.annotations.Test; + +public class KllDirectLongsSketchIteratorTest { + private static final DefaultMemoryRequestServer memReqSvr = new DefaultMemoryRequestServer(); + + @Test + public void emptySketch() { + final KllLongsSketch sketch = getDLSketch(200, 0); + QuantilesLongsSketchIterator it = sketch.iterator(); + Assert.assertFalse(it.next()); + } + + @Test + public void oneItemSketch() { + final KllLongsSketch sketch = getDLSketch(200, 0); + sketch.update(0); + QuantilesLongsSketchIterator it = sketch.iterator(); + Assert.assertTrue(it.next()); + Assert.assertEquals(it.getQuantile(), 0); + Assert.assertEquals(it.getWeight(), 1); + Assert.assertFalse(it.next()); + } + + @Test + public void bigSketches() { + for (int n = 1000; n < 100000; n += 2000) { + final KllLongsSketch sketch = getDLSketch(200, 0); + for (int i = 0; i < n; i++) { + sketch.update(i); + } + QuantilesLongsSketchIterator it = sketch.iterator(); + int count = 0; + int weight = 0; + while (it.next()) { + count++; + weight += (int)it.getWeight(); + } + Assert.assertEquals(count, sketch.getNumRetained()); + Assert.assertEquals(weight, n); + } + } + + private static KllLongsSketch getDLSketch(final int k, final int n) { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= n; i++) { sk.update(i); } + byte[] byteArr = KllHelper.toByteArray(sk, true); + WritableMemory wmem = WritableMemory.writableWrap(byteArr); + + KllLongsSketch dlsk = KllLongsSketch.writableWrap(wmem, memReqSvr); + return dlsk; + } + +} diff --git a/src/test/java/org/apache/datasketches/kll/KllDirectLongsSketchTest.java b/src/test/java/org/apache/datasketches/kll/KllDirectLongsSketchTest.java new file mode 100644 index 000000000..f1784b7ce --- /dev/null +++ b/src/test/java/org/apache/datasketches/kll/KllDirectLongsSketchTest.java @@ -0,0 +1,686 @@ +/* + * 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 org.apache.datasketches.common.SketchesArgumentException; +import org.apache.datasketches.kll.KllSketch.SketchStructure; +import org.apache.datasketches.memory.DefaultMemoryRequestServer; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.memory.WritableMemory; +import org.testng.annotations.Test; + +import static org.apache.datasketches.kll.KllSketch.SketchType.LONGS_SKETCH; +import static org.apache.datasketches.quantilescommon.QuantileSearchCriteria.EXCLUSIVE; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +public class KllDirectLongsSketchTest { + + private static final double PMF_EPS_FOR_K_8 = 0.35; // PMF rank error (epsilon) for k=8 + private static final double PMF_EPS_FOR_K_128 = 0.025; // PMF rank error (epsilon) for k=128 + private static final double PMF_EPS_FOR_K_256 = 0.013; // PMF rank error (epsilon) for k=256 + private static final double NUMERIC_NOISE_TOLERANCE = 1E-6; + private static final DefaultMemoryRequestServer memReqSvr = new DefaultMemoryRequestServer(); + + @Test + public void empty() { + final KllLongsSketch sketch = getUpdatableDirectLongSketch(200, 0); + assertTrue(sketch.isEmpty()); + assertEquals(sketch.getN(), 0); + assertEquals(sketch.getNumRetained(), 0); + try { sketch.getRank(0); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getMinItem(); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getMaxItem(); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getQuantile(0.5); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getQuantiles(new double[] {0.0, 1.0}); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getPMF(new long[] {0}); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getCDF(new long[0]); fail(); } catch (SketchesArgumentException e) {} + assertNotNull(sketch.toString(true, true)); + assertNotNull(sketch.toString()); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void getQuantileInvalidArg() { + final KllLongsSketch sketch = getUpdatableDirectLongSketch(200, 0); + sketch.update(1); + sketch.getQuantile(-1.0); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void getQuantilesInvalidArg() { + final KllLongsSketch sketch = getUpdatableDirectLongSketch(200, 0); + sketch.update(1); + sketch.getQuantiles(new double[] {2.0}); + } + + @Test + public void oneValue() { + final KllLongsSketch sketch = getUpdatableDirectLongSketch(200, 0); + sketch.update(1); + assertFalse(sketch.isEmpty()); + assertEquals(sketch.getN(), 1); + assertEquals(sketch.getNumRetained(), 1); + assertEquals(sketch.getRank(1, EXCLUSIVE), 0.0); + assertEquals(sketch.getRank(2, EXCLUSIVE), 1.0); + assertEquals(sketch.getMinItem(), 1L); + assertEquals(sketch.getMaxItem(), 1L); + assertEquals(sketch.getQuantile(0.5, EXCLUSIVE), 1L); + } + + @Test + public void manyValuesEstimationMode() { + final KllLongsSketch sketch = getUpdatableDirectLongSketch(200, 0); + final int n = 1_000_000; + + for (int i = 0; i < n; i++) { + sketch.update(i); + } + assertEquals(sketch.getN(), n); + + // test getRank + for (int i = 0; i < n; i++) { + final double trueRank = (double) i / n; + assertEquals(sketch.getRank(i), trueRank, PMF_EPS_FOR_K_256, "for value " + i); + } + + // test getPMF + final double[] pmf = sketch.getPMF(new long[] {n / 2}); // split at median + assertEquals(pmf.length, 2); + assertEquals(pmf[0], 0.5, PMF_EPS_FOR_K_256); + assertEquals(pmf[1], 0.5, PMF_EPS_FOR_K_256); + assertEquals(sketch.getMinItem(), 0); // min value is exact + assertEquals(sketch.getMaxItem(), n - 1L); // max value is exact + + // check at every 0.1 percentage point + final double[] ranks = new double[1001]; + final double[] reverseRanks = new double[1001]; // check that ordering doesn't matter + for (int i = 0; i <= 1000; i++) { + ranks[i] = (double) i / 1000; + reverseRanks[1000 - i] = ranks[i]; + } + final long[] quantiles = sketch.getQuantiles(ranks); + final long[] reverseQuantiles = sketch.getQuantiles(reverseRanks); + long previousQuantile = 0; + for (int i = 0; i <= 1000; i++) { + final long quantile = sketch.getQuantile(ranks[i]); + assertEquals(quantile, quantiles[i]); + assertEquals(quantile, reverseQuantiles[1000 - i]); + assertTrue(previousQuantile <= quantile); + previousQuantile = quantile; + } + } + + @Test + public void getRankGetCdfGetPmfConsistency() { + final KllLongsSketch sketch = getUpdatableDirectLongSketch(200, 0); + final int n = 1000; + final long[] values = new long[n]; + for (int i = 0; i < n; i++) { + sketch.update(i); + values[i] = i; + } + final double[] ranks = sketch.getCDF(values); + final double[] pmf = sketch.getPMF(values); + double sumPmf = 0; + for (int i = 0; i < n; i++) { + assertEquals(ranks[i], sketch.getRank(values[i]), NUMERIC_NOISE_TOLERANCE, + "rank vs CDF for value " + i); + sumPmf += pmf[i]; + assertEquals(ranks[i], sumPmf, NUMERIC_NOISE_TOLERANCE, "CDF vs PMF for value " + i); + } + sumPmf += pmf[n]; + assertEquals(sumPmf, 1.0, NUMERIC_NOISE_TOLERANCE); + assertEquals(ranks[n], 1.0, NUMERIC_NOISE_TOLERANCE); + } + + @Test + public void merge() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(200, 0); + final KllLongsSketch sketch2 = getUpdatableDirectLongSketch(200, 0); + final int n = 10_000; + for (int i = 0; i < n; i++) { + sketch1.update(i); + sketch2.update(2 * n - i - 1); + } + + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), n - 1); + + assertEquals(sketch2.getMinItem(), n); + assertEquals(sketch2.getMaxItem(), 2 * n - 1); + + sketch1.merge(sketch2); + + assertFalse(sketch1.isEmpty()); + assertEquals(sketch1.getN(), 2L * n); + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), 2 * n - 1L); + assertEquals(sketch1.getQuantile(0.5), n, n * PMF_EPS_FOR_K_256); + } + + @Test + public void mergeLowerK() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(256, 0); + final KllLongsSketch sketch2 = getUpdatableDirectLongSketch(128, 0); + final int n = 10_000; + for (int i = 0; i < n; i++) { + sketch1.update(i); + sketch2.update(2 * n - i - 1); + } + + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), n - 1); + + assertEquals(sketch2.getMinItem(), n); + assertEquals(sketch2.getMaxItem(), 2 * n - 1); + + assertTrue(sketch1.getNormalizedRankError(false) < sketch2.getNormalizedRankError(false)); + assertTrue(sketch1.getNormalizedRankError(true) < sketch2.getNormalizedRankError(true)); + sketch1.merge(sketch2); + + // sketch1 must get "contaminated" by the lower K in sketch2 + assertEquals(sketch1.getNormalizedRankError(false), sketch2.getNormalizedRankError(false)); + assertEquals(sketch1.getNormalizedRankError(true), sketch2.getNormalizedRankError(true)); + + assertFalse(sketch1.isEmpty()); + assertEquals(sketch1.getN(), 2 * n); + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), 2 * n - 1); + assertEquals(sketch1.getQuantile(0.5), n, n * PMF_EPS_FOR_K_128); + } + + @Test + public void mergeEmptyLowerK() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(256, 0); + final KllLongsSketch sketch2 = getUpdatableDirectLongSketch(128, 0); + final int n = 10_000; + for (int i = 0; i < n; i++) { + sketch1.update(i); + } + + // rank error should not be affected by a merge with an empty sketch with lower K + final double rankErrorBeforeMerge = sketch1.getNormalizedRankError(true); + sketch1.merge(sketch2); + assertEquals(sketch1.getNormalizedRankError(true), rankErrorBeforeMerge); + + assertFalse(sketch1.isEmpty()); + assertEquals(sketch1.getN(), n); + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), n - 1); + assertEquals(sketch1.getQuantile(0.5), n / 2, n / 2 * PMF_EPS_FOR_K_256); + + //merge the other way + sketch2.merge(sketch1); + assertFalse(sketch1.isEmpty()); + assertEquals(sketch1.getN(), n); + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), n - 1); + assertEquals(sketch1.getQuantile(0.5), n / 2, n / 2 * PMF_EPS_FOR_K_256); + } + + @Test + public void mergeExactModeLowerK() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(256, 0); + final KllLongsSketch sketch2 = getUpdatableDirectLongSketch(128, 0); + final int n = 10000; + for (int i = 0; i < n; i++) { + sketch1.update(i); + } + sketch2.update(1); + + // rank error should not be affected by a merge with a sketch in exact mode with lower K + final double rankErrorBeforeMerge = sketch1.getNormalizedRankError(true); + sketch1.merge(sketch2); + assertEquals(sketch1.getNormalizedRankError(true), rankErrorBeforeMerge); + } + + @Test + public void mergeMinMinValueFromOther() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(200, 0); + final KllLongsSketch sketch2 = getUpdatableDirectLongSketch(200, 0); + sketch1.update(1); + sketch2.update(2); + sketch2.merge(sketch1); + assertEquals(sketch2.getMinItem(), 1L); + } + + @Test + public void mergeMinAndMaxFromOther() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(200, 0); + final KllLongsSketch sketch2 = getUpdatableDirectLongSketch(200, 0); + int n = 1_000_000; + for (int i = 1; i <= n; i++) { + sketch1.update(i); + } + sketch2.merge(sketch1); + assertEquals(sketch2.getMinItem(), 1L); + assertEquals(sketch2.getMaxItem(), 1_000_000L); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void kTooSmall() { + getUpdatableDirectLongSketch(KllSketch.DEFAULT_M - 1, 0); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void kTooLarge() { + getUpdatableDirectLongSketch(KllSketch.MAX_K + 1, 0); + } + + @Test + public void minK() { + final KllLongsSketch sketch = getUpdatableDirectLongSketch(KllSketch.DEFAULT_M, 0); + for (int i = 0; i < 1000; i++) { + sketch.update(i); + } + assertEquals(sketch.getK(), KllSketch.DEFAULT_M); + assertEquals(sketch.getQuantile(0.5), 500, 500 * PMF_EPS_FOR_K_8); + } + + @Test + public void maxK() { + final KllLongsSketch sketch = getUpdatableDirectLongSketch(KllSketch.MAX_K, 0); + for (int i = 0; i < 1000; i++) { + sketch.update(i); + } + assertEquals(sketch.getK(), KllSketch.MAX_K); + assertEquals(sketch.getQuantile(0.5), 500, 500 * PMF_EPS_FOR_K_256); + } + + @Test + public void serializeDeserializeEmptyViaCompactHeapify() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(200, 0); + final byte[] bytes = sketch1.toByteArray(); //compact + final KllLongsSketch sketch2 = KllLongsSketch.heapify(Memory.wrap(bytes)); + assertEquals(bytes.length, sketch1.currentSerializedSizeBytes(false)); + assertTrue(sketch2.isEmpty()); + assertEquals(sketch2.getNumRetained(), sketch1.getNumRetained()); + assertEquals(sketch2.getN(), sketch1.getN()); + assertEquals(sketch2.getNormalizedRankError(false), sketch1.getNormalizedRankError(false)); + try { sketch2.getMinItem(); fail(); } catch (SketchesArgumentException e) {} + try { sketch2.getMaxItem(); fail(); } catch (SketchesArgumentException e) {} + assertEquals(sketch2.currentSerializedSizeBytes(false), + sketch1.currentSerializedSizeBytes(false)); + } + + @Test + public void serializeDeserializeEmptyViaUpdatableWritableWrap() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(200, 0); + final byte[] bytes = KllHelper.toByteArray(sketch1, true); + final KllLongsSketch sketch2 = + KllLongsSketch.writableWrap(WritableMemory.writableWrap(bytes),memReqSvr); + assertEquals(bytes.length, sketch1.currentSerializedSizeBytes(true)); + assertTrue(sketch2.isEmpty()); + assertEquals(sketch2.getNumRetained(), sketch1.getNumRetained()); + assertEquals(sketch2.getN(), sketch1.getN()); + assertEquals(sketch2.getNormalizedRankError(false), sketch1.getNormalizedRankError(false)); + try { sketch2.getMinItem(); fail(); } catch (SketchesArgumentException e) {} + try { sketch2.getMaxItem(); fail(); } catch (SketchesArgumentException e) {} + assertEquals(sketch2.currentSerializedSizeBytes(true), + sketch1.currentSerializedSizeBytes(true)); + } + + @Test + public void serializeDeserializeOneValueViaCompactHeapify() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(200, 0); + sketch1.update(1); + final byte[] bytes = sketch1.toByteArray(); + final KllLongsSketch sketch2 = KllLongsSketch.heapify(Memory.wrap(bytes)); + assertEquals(bytes.length, sketch1.currentSerializedSizeBytes(false)); + assertFalse(sketch2.isEmpty()); + assertEquals(sketch2.getNumRetained(), 1); + assertEquals(sketch2.getN(), 1); + assertEquals(sketch2.getNormalizedRankError(false), sketch1.getNormalizedRankError(false)); + assertTrue(sketch2.getMinItem() < Long.MAX_VALUE); + assertTrue(sketch2.getMaxItem() > Long.MIN_VALUE); + assertEquals(sketch2.currentSerializedSizeBytes(false), 8 + Long.BYTES); + } + + @Test + public void serializeDeserializeOneValueViaUpdatableWritableWrap() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(200, 0); + sketch1.update(1); + final byte[] bytes = KllHelper.toByteArray(sketch1, true); + final KllLongsSketch sketch2 = + KllLongsSketch.writableWrap(WritableMemory.writableWrap(bytes),memReqSvr); + assertEquals(bytes.length, sketch1.currentSerializedSizeBytes(true)); + assertFalse(sketch2.isEmpty()); + assertEquals(sketch2.getNumRetained(), 1); + assertEquals(sketch2.getN(), 1); + assertEquals(sketch2.getNormalizedRankError(false), sketch1.getNormalizedRankError(false)); + assertEquals(sketch2.getMinItem(), 1L); + assertEquals(sketch2.getMaxItem(), 1L); + assertEquals(sketch2.currentSerializedSizeBytes(false), 8 + Long.BYTES); + assertEquals(sketch2.currentSerializedSizeBytes(true), bytes.length); + } + + @Test + public void serializeDeserializeFullViaCompactHeapify() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(200, 1000); + final byte[] byteArr1 = sketch1.toByteArray(); //compact + final KllLongsSketch sketch2 = KllLongsSketch.heapify(Memory.wrap(byteArr1)); + assertEquals(byteArr1.length, sketch1.currentSerializedSizeBytes(false)); + assertFalse(sketch2.isEmpty()); + assertEquals(sketch2.getNumRetained(), sketch1.getNumRetained()); + assertEquals(sketch2.getN(), sketch1.getN()); + assertEquals(sketch2.getNormalizedRankError(false), sketch1.getNormalizedRankError(false)); + assertEquals(sketch2.getMinItem(), sketch1.getMinItem()); + assertEquals(sketch2.getMaxItem(), sketch1.getMaxItem()); + assertEquals(sketch2.currentSerializedSizeBytes(false), sketch1.currentSerializedSizeBytes(false)); + } + + @Test + public void serializeDeserializeFullViaUpdatableWritableWrap() { + final KllLongsSketch sketch1 = getUpdatableDirectLongSketch(200, 0); + final int n = 1000; + for (int i = 1; i <= n; i++) { + sketch1.update(i); + } + final byte[] bytes = KllHelper.toByteArray(sketch1, true); //updatable + final KllLongsSketch sketch2 = + KllLongsSketch.writableWrap(WritableMemory.writableWrap(bytes), memReqSvr); + assertEquals(bytes.length, sketch1.currentSerializedSizeBytes(true)); + assertFalse(sketch2.isEmpty()); + assertEquals(sketch2.getNumRetained(), sketch1.getNumRetained()); + assertEquals(sketch2.getN(), sketch1.getN()); + assertEquals(sketch2.getNormalizedRankError(false), sketch1.getNormalizedRankError(false)); + assertEquals(sketch2.getMinItem(), sketch1.getMinItem()); + assertEquals(sketch2.getMaxItem(), sketch1.getMaxItem()); + assertEquals(sketch2.currentSerializedSizeBytes(true), sketch1.currentSerializedSizeBytes(true)); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void outOfOrderSplitPoints() { + final KllLongsSketch sketch = getUpdatableDirectLongSketch(200, 0); + sketch.update(0); + sketch.getCDF(new long[] {1, 0}); + } + + @Test + public void checkSimpleMergeDirect() { //used for troubleshooting + int k = 20; + int n1 = 21; + int n2 = 43; + KllLongsSketch sk1 = KllLongsSketch.newHeapInstance(k); + KllLongsSketch sk2 = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= n1; i++) { + sk1.update(i); + } + for (int i = 1; i <= n2; i++) { + sk2.update(i + 100); + } + println("SK1:"); + println(sk1.toString(true, true)); + println("SK2:"); + println(sk2.toString(true, true)); + WritableMemory wmem1 = WritableMemory.writableWrap(KllHelper.toByteArray(sk1, true)); + WritableMemory wmem2 = WritableMemory.writableWrap(KllHelper.toByteArray(sk2, true)); + KllLongsSketch dsk1 = KllLongsSketch.writableWrap(wmem1, memReqSvr); + KllLongsSketch dsk2 = KllLongsSketch.writableWrap(wmem2, memReqSvr); + println("BEFORE MERGE"); + println(dsk1.toString(true, true)); + dsk1.merge(dsk2); + println("AFTER MERGE"); + println(dsk1.toString(true, true)); + } + + @Test + public void checkSketchInitializeDirectLongUpdatableMem() { + int k = 20; //don't change this + KllLongsSketch sk; + KllLongsSketch sk2; + byte[] compBytes; + WritableMemory wmem; + + println("#### CASE: LONG FULL DIRECT FROM UPDATABLE"); + sk2 = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= k + 1; i++) { sk2.update(i); } + //println(sk2.toString(true, true)); + compBytes = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(compBytes); + println(KllPreambleUtil.toString(compBytes, LONGS_SKETCH, true)); + sk = KllLongsSketch.writableWrap(wmem, memReqSvr); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), k + 1); + assertEquals(sk.getNumRetained(), 11); + assertFalse(sk.isEmpty()); + assertTrue(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 33); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 3); + assertEquals(sk.getMaxItem(), 21); + assertEquals(sk.getMinItem(), 1); + assertEquals(sk.getNumLevels(), 2); + assertFalse(sk.isLevelZeroSorted()); + + println("#### CASE: LONG EMPTY HEAPIFIED FROM UPDATABLE"); + sk2 = KllLongsSketch.newHeapInstance(k); + //println(sk.toString(true, true)); + compBytes = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(compBytes); + println(KllPreambleUtil.toString(compBytes, LONGS_SKETCH, true)); + sk = KllLongsSketch.writableWrap(wmem, memReqSvr); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 0); + assertEquals(sk.getNumRetained(), 0); + assertTrue(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + try { sk.getMaxItem(); fail(); } catch (SketchesArgumentException e) {} + try { sk.getMinItem(); fail(); } catch (SketchesArgumentException e) {} + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + + println("#### CASE: LONG SINGLE HEAPIFIED FROM UPDATABLE"); + sk2 = KllLongsSketch.newHeapInstance(k); + sk2.update(1); + //println(sk.toString(true, true)); + compBytes = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(compBytes); + println(KllPreambleUtil.toString(compBytes, LONGS_SKETCH, true)); + sk = KllLongsSketch.writableWrap(wmem, memReqSvr); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 1); + assertEquals(sk.getNumRetained(), 1); + assertFalse(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + assertEquals(sk.getMaxItem(), 1L); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + } + + @Test + public void checkGetWritableMemory() { + final KllLongsSketch sketch = getUpdatableDirectLongSketch(200, 200); + assertEquals(sketch.getK(), 200); + assertEquals(sketch.getN(), 200); + assertFalse(sketch.isEmpty()); + assertTrue(sketch.isMemoryUpdatableFormat()); + assertFalse(sketch.isEstimationMode()); + assertTrue(sketch.isLongsSketch()); + assertFalse(sketch.isLevelZeroSorted()); + assertFalse(sketch.isDoublesSketch()); + + final WritableMemory wmem = sketch.getWritableMemory(); + final KllLongsSketch sk = KllHeapLongsSketch.heapifyImpl(wmem); + assertEquals(sk.getK(), 200); + assertEquals(sk.getN(), 200); + assertFalse(sk.isEmpty()); + assertFalse(sk.isMemoryUpdatableFormat()); + assertFalse(sk.isEstimationMode()); + assertTrue(sk.isLongsSketch()); + assertFalse(sk.isLevelZeroSorted()); + assertFalse(sk.isDoublesSketch()); + } + + @Test + public void checkReset() { + WritableMemory dstMem = WritableMemory.allocate(3000); + KllLongsSketch sk = KllLongsSketch.newDirectInstance(20, dstMem, memReqSvr); + for (int i = 1; i <= 100; i++) { sk.update(i); } + long n1 = sk.getN(); + long min1 = sk.getMinItem(); + long max1 = sk.getMaxItem(); + sk.reset(); + for (int i = 1; i <= 100; i++) { sk.update(i); } + long n2 = sk.getN(); + long min2 = sk.getMinItem(); + long max2 = sk.getMaxItem(); + assertEquals(n2, n1); + assertEquals(min2, min1); + assertEquals(max2, max1); + } + + @Test + public void checkHeapify() { + WritableMemory dstMem = WritableMemory.allocate(6000); + KllLongsSketch sk = KllLongsSketch.newDirectInstance(20, dstMem, memReqSvr); + for (int i = 1; i <= 100; i++) { sk.update(i); } + KllLongsSketch sk2 = KllHeapLongsSketch.heapifyImpl(dstMem); + assertEquals(sk2.getMinItem(), 1L); + assertEquals(sk2.getMaxItem(), 100L); + } + + @Test + public void checkMergeKllLongsSketch() { + WritableMemory dstMem = WritableMemory.allocate(6000); + KllLongsSketch sk = KllLongsSketch.newDirectInstance(20, dstMem, memReqSvr); + for (int i = 1; i <= 21; i++) { sk.update(i); } + KllLongsSketch sk2 = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 21; i++ ) { sk2.update(i + 100); } + sk.merge(sk2); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getMaxItem(), 121L); + } + + @Test + public void checkReverseMergeKllLongsSketch() { + WritableMemory dstMem = WritableMemory.allocate(6000); + KllLongsSketch sk = KllLongsSketch.newDirectInstance(20, dstMem, memReqSvr); + for (int i = 1; i <= 21; i++) { sk.update(i); } + KllLongsSketch sk2 = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 21; i++ ) { sk2.update(i + 100); } + sk2.merge(sk); + assertEquals(sk2.getMinItem(), 1L); + assertEquals(sk2.getMaxItem(), 121L); + } + + @Test + public void checkWritableWrapOfCompactForm() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 21; i++ ) { sk.update(i); } + WritableMemory srcMem = WritableMemory.writableWrap(sk.toByteArray()); + KllLongsSketch sk2 = KllLongsSketch.writableWrap(srcMem, memReqSvr); + assertEquals(sk2.getMinItem(), 1L); + assertEquals(sk2.getMaxItem(), 21L); + } + + @Test + public void checkReadOnlyExceptions() { + int k = 20; + long[] fltArr = new long[0]; + long fltV = 1; + int idx = 1; + boolean bool = true; + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + KllLongsSketch sk2 = KllLongsSketch.wrap(Memory.wrap(sk.toByteArray())); + try { sk2.incN(1); fail(); } catch (SketchesArgumentException e) { } + try { sk2.incNumLevels(); fail(); } catch (SketchesArgumentException e) { } + try { sk2.setLongItemsArray(fltArr); fail(); } catch (SketchesArgumentException e) { } + try { sk2.setLongItemsArrayAt(idx, fltV); fail(); } catch (SketchesArgumentException e) { } + try { sk2.setLevelZeroSorted(bool); fail(); } catch (SketchesArgumentException e) { } + try { sk2.setMaxItem(fltV); fail(); } catch (SketchesArgumentException e) { } + try { sk2.setMinItem(fltV); fail(); } catch (SketchesArgumentException e) { } + try { sk2.setMinK(idx); fail(); } catch (SketchesArgumentException e) { } + try { sk2.setN(idx); fail(); } catch (SketchesArgumentException e) { } + try { sk2.setNumLevels(idx); fail(); } catch (SketchesArgumentException e) { } + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void checkMergeExceptions() { + KllLongsSketch sk1 = KllLongsSketch.newHeapInstance(20); + WritableMemory srcMem1 = WritableMemory.writableWrap(sk1.toByteArray()); + KllLongsSketch sk2 = KllLongsSketch.writableWrap(srcMem1, memReqSvr); + sk2.merge(sk1); + } + + @Test + public void checkVectorUpdate() { + WritableMemory dstMem = WritableMemory.allocate(6000); + KllLongsSketch sk = KllLongsSketch.newDirectInstance(20, dstMem, memReqSvr); + long[] v = new long[21]; + for (int i = 0; i < 21; i++) { v[i] = i + 1; } + sk.update(v, 0, 21); + println(sk.toString(true, true)); + int[] levelsArr = sk.getLevelsArray(SketchStructure.UPDATABLE); + assertEquals(levelsArr[0], 22); + long[] longsArr = sk.getLongItemsArray(); + assertEquals(longsArr[22], 21); + } + + @Test + public void checkWeightedUpdate() { + WritableMemory dstMem = WritableMemory.allocate(6000); + KllLongsSketch sk = KllLongsSketch.newDirectInstance(8, dstMem, memReqSvr); + for (int i = 0; i < 16; i++) { + sk.update(i + 1, 16); + } + println(sk.toString(true, true)); + assertEquals(sk.getN(), 256); + assertEquals(sk.getMaxItem(), 16L); + assertEquals(sk.getMinItem(), 1L); + } + + private static KllLongsSketch getUpdatableDirectLongSketch(int k, int n) { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= n; i++) { sk.update(i); } + byte[] byteArr = KllHelper.toByteArray(sk, true); + WritableMemory wmem = WritableMemory.writableWrap(byteArr); + KllLongsSketch dfsk = KllLongsSketch.writableWrap(wmem, memReqSvr); + return dfsk; + } + + @Test + public void checkMergeExceptionsWrongType() { + KllLongsSketch sk1 = KllLongsSketch.newHeapInstance(20); + KllDoublesSketch sk2 = KllDoublesSketch.newHeapInstance(20); + try { sk1.merge(sk2); fail(); } catch (ClassCastException e) { } + try { sk2.merge(sk1); fail(); } catch (ClassCastException e) { } + } + + private final static boolean enablePrinting = false; + + /** + * @param o the Object to println + */ + private static final void println(final Object o) { + if (enablePrinting) { System.out.println(o.toString()); } + } + +} diff --git a/src/test/java/org/apache/datasketches/kll/KllDoublesSketchSerDeTest.java b/src/test/java/org/apache/datasketches/kll/KllDoublesSketchSerDeTest.java index e07a395da..cca51bb44 100644 --- a/src/test/java/org/apache/datasketches/kll/KllDoublesSketchSerDeTest.java +++ b/src/test/java/org/apache/datasketches/kll/KllDoublesSketchSerDeTest.java @@ -74,8 +74,8 @@ public void serializeDeserializeOneValue() { assertEquals(sk2.getNumRetained(), 1); assertEquals(sk2.getN(), 1); assertEquals(sk2.getNormalizedRankError(false), sk1.getNormalizedRankError(false)); - assertEquals(sk2.getMinItem(), 1.0); - assertEquals(sk2.getMaxItem(), 1.0); + assertEquals(sk2.getMinItem(), 1L); + assertEquals(sk2.getMaxItem(), 1L); assertEquals(sk2.getSerializedSizeBytes(), Long.BYTES + Double.BYTES); //from heap -> byte[] -> off heap @@ -84,8 +84,8 @@ public void serializeDeserializeOneValue() { assertEquals(sk3.getNumRetained(), 1); assertEquals(sk3.getN(), 1); assertEquals(sk3.getNormalizedRankError(false), sk1.getNormalizedRankError(false)); - assertEquals(sk3.getMinItem(), 1.0); - assertEquals(sk3.getMaxItem(), 1.0); + assertEquals(sk3.getMinItem(), 1L); + assertEquals(sk3.getMaxItem(), 1L); assertEquals(sk3.getSerializedSizeBytes(), sk1.getSerializedSizeBytes()); //from heap -> byte[] -> off heap -> byte[] -> compare byte[] final byte[] bytes2 = sk3.toByteArray(); @@ -99,8 +99,8 @@ public void serializeDeserializeMultipleValues() { for (int i = 0; i < n; i++) { sk1.update(i); } - assertEquals(sk1.getMinItem(), 0.0); - assertEquals(sk1.getMaxItem(), 999.0); + assertEquals(sk1.getMinItem(), 0); + assertEquals(sk1.getMaxItem(), 999L); //from heap -> byte[] -> heap final byte[] bytes = sk1.toByteArray(); diff --git a/src/test/java/org/apache/datasketches/kll/KllDoublesSketchTest.java b/src/test/java/org/apache/datasketches/kll/KllDoublesSketchTest.java index 0b3818f1f..e143577f4 100644 --- a/src/test/java/org/apache/datasketches/kll/KllDoublesSketchTest.java +++ b/src/test/java/org/apache/datasketches/kll/KllDoublesSketchTest.java @@ -165,8 +165,8 @@ public void manyValuesEstimationMode() { assertEquals(pmf[0], 0.5, PMF_EPS_FOR_K_256); assertEquals(pmf[1], 0.5, PMF_EPS_FOR_K_256); - assertEquals(sketch.getMinItem(), 0f); // min value is exact - assertEquals(sketch.getMaxItem(), n - 1f); // max value is exact + assertEquals(sketch.getMinItem(), 0.0); // min value is exact + assertEquals(sketch.getMaxItem(), n - 1.0); // max value is exact // check at every 0.1 percentage point final double[] fractions = new double[1001]; @@ -261,11 +261,11 @@ public void mergeLowerK() { sketch2.update(2 * n - i - 1); } - assertEquals(sketch1.getMinItem(), 0.0f); - assertEquals(sketch1.getMaxItem(), n - 1f); + assertEquals(sketch1.getMinItem(), 0.0); + assertEquals(sketch1.getMaxItem(), n - 1); assertEquals(sketch2.getMinItem(), n); - assertEquals(sketch2.getMaxItem(), 2f * n - 1.0); + assertEquals(sketch2.getMaxItem(), 2.0 * n - 1.0); assertTrue(sketch1.getNormalizedRankError(false) < sketch2.getNormalizedRankError(false)); assertTrue(sketch1.getNormalizedRankError(true) < sketch2.getNormalizedRankError(true)); @@ -306,7 +306,7 @@ public void mergeEmptyLowerK() { sketch2.merge(sketch1); assertFalse(sketch1.isEmpty()); assertEquals(sketch1.getN(), n); - assertEquals(sketch1.getMinItem(), 0f); + assertEquals(sketch1.getMinItem(), 0.0); assertEquals(sketch1.getMaxItem(), n - 1.0); assertEquals(sketch1.getQuantile(0.5), n / 2.0, n * PMF_EPS_FOR_K_256); } @@ -424,7 +424,7 @@ public void checkNewDirectInstanceAndSize() { KllDoublesSketch.newDirectInstance(wmem, memReqSvr); try { KllDoublesSketch.newDirectInstance(null, memReqSvr); fail(); } catch (NullPointerException e) { } - try { KllFloatsSketch.newDirectInstance(wmem, null); fail(); } + try { KllDoublesSketch.newDirectInstance(wmem, null); fail(); } catch (NullPointerException e) { } int updateSize = KllSketch.getMaxSerializedSizeBytes(200, 0, DOUBLES_SKETCH, true); int compactSize = KllSketch.getMaxSerializedSizeBytes(200, 0, DOUBLES_SKETCH, false); diff --git a/src/test/java/org/apache/datasketches/kll/KllDoublesValidationTest.java b/src/test/java/org/apache/datasketches/kll/KllDoublesValidationTest.java index 42802ba65..f647b43f4 100644 --- a/src/test/java/org/apache/datasketches/kll/KllDoublesValidationTest.java +++ b/src/test/java/org/apache/datasketches/kll/KllDoublesValidationTest.java @@ -160,7 +160,7 @@ public class KllDoublesValidationTest { private static int[] makeInputArray(int n, int stride) { assert isOdd(stride); - int mask = (1 << 23) - 1; // because library items are single-precision floats + int mask = (1 << 23) - 1; // because library items are single-precision floats //TODO ? int cur = 0; int[] arr = new int[n]; for (int i = 0; i < n; i++) { diff --git a/src/test/java/org/apache/datasketches/kll/KllLongsSketchIteratorTest.java b/src/test/java/org/apache/datasketches/kll/KllLongsSketchIteratorTest.java new file mode 100644 index 000000000..a98c32c9e --- /dev/null +++ b/src/test/java/org/apache/datasketches/kll/KllLongsSketchIteratorTest.java @@ -0,0 +1,114 @@ +/* + * 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 org.apache.datasketches.quantilescommon.LongsSortedViewIterator; +import org.apache.datasketches.quantilescommon.QuantilesLongsSketchIterator; +import org.testng.Assert; +import org.testng.annotations.Test; + +import static org.apache.datasketches.quantilescommon.QuantileSearchCriteria.EXCLUSIVE; +import static org.apache.datasketches.quantilescommon.QuantileSearchCriteria.INCLUSIVE; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +public class KllLongsSketchIteratorTest { + + @Test + public void emptySketch() { + KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + QuantilesLongsSketchIterator it = sketch.iterator(); + Assert.assertFalse(it.next()); + } + + @Test + public void oneItemSketch() { + KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + sketch.update(1); + QuantilesLongsSketchIterator it = sketch.iterator(); + Assert.assertTrue(it.next()); + Assert.assertEquals(it.getQuantile(), 1L); + Assert.assertEquals(it.getWeight(), 1); + Assert.assertFalse(it.next()); + } + + @Test + public void twoItemSketchForIterator() { + KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + sketch.update(1); + sketch.update(2); + QuantilesLongsSketchIterator itr = sketch.iterator(); + assertTrue(itr.next()); + + assertEquals(itr.getQuantile(), 2L); + assertEquals(itr.getWeight(), 1); + + assertTrue(itr.next()); + + assertEquals(itr.getQuantile(), 1L); + assertEquals(itr.getWeight(), 1); + } + + @Test + public void twoItemSketchForSortedViewIterator() { + KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + sketch.update(1); + sketch.update(2); + LongsSortedViewIterator itr = sketch.getSortedView().iterator(); + + assertTrue(itr.next()); + + assertEquals(itr.getQuantile(), 1L); + assertEquals(itr.getWeight(), 1); + assertEquals(itr.getNaturalRank(EXCLUSIVE), 0); + assertEquals(itr.getNaturalRank(INCLUSIVE), 1); + assertEquals(itr.getNormalizedRank(EXCLUSIVE), 0); + assertEquals(itr.getNormalizedRank(INCLUSIVE), 0.5); + + assertTrue(itr.next()); + + assertEquals(itr.getQuantile(), 2L); + assertEquals(itr.getWeight(), 1); + assertEquals(itr.getNaturalRank(EXCLUSIVE), 1); + assertEquals(itr.getNaturalRank(INCLUSIVE), 2); + assertEquals(itr.getNormalizedRank(EXCLUSIVE), 0.5); + assertEquals(itr.getNormalizedRank(INCLUSIVE), 1.0); + } + + @Test + public void bigSketches() { + for (int n = 1000; n < 100000; n += 2000) { + KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + for (int i = 0; i < n; i++) { + sketch.update(i); + } + QuantilesLongsSketchIterator it = sketch.iterator(); + int count = 0; + int weight = 0; + while (it.next()) { + count++; + weight += (int)it.getWeight(); + } + Assert.assertEquals(count, sketch.getNumRetained()); + Assert.assertEquals(weight, n); + } + } + +} diff --git a/src/test/java/org/apache/datasketches/kll/KllLongsSketchSerDeTest.java b/src/test/java/org/apache/datasketches/kll/KllLongsSketchSerDeTest.java new file mode 100644 index 000000000..b9b0f800d --- /dev/null +++ b/src/test/java/org/apache/datasketches/kll/KllLongsSketchSerDeTest.java @@ -0,0 +1,131 @@ +/* + * 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.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +import org.apache.datasketches.common.SketchesArgumentException; +import org.apache.datasketches.memory.Memory; +import org.testng.annotations.Test; + +public class KllLongsSketchSerDeTest { + + @Test + public void serializeDeserializeEmpty() { + final int N = 20; + + final KllLongsSketch sk1 = KllLongsSketch.newHeapInstance(N); + //Empty: from heap -> byte[] -> heap + final byte[] bytes = sk1.toByteArray(); + final KllLongsSketch sk2 = KllLongsSketch.heapify(Memory.wrap(bytes)); + assertEquals(bytes.length, sk1.getSerializedSizeBytes()); + assertTrue(sk2.isEmpty()); + assertEquals(sk2.getNumRetained(), sk1.getNumRetained()); + assertEquals(sk2.getN(), sk1.getN()); + assertEquals(sk2.getNormalizedRankError(false), sk1.getNormalizedRankError(false)); + try { sk2.getMinItem(); fail(); } catch (SketchesArgumentException e) {} + try { sk2.getMaxItem(); fail(); } catch (SketchesArgumentException e) {} + assertEquals(sk2.getSerializedSizeBytes(), sk1.getSerializedSizeBytes()); + + //Empty: from heap -> byte[] -> off heap + final KllLongsSketch sk3 = KllLongsSketch.wrap(Memory.wrap(bytes)); + assertTrue(sk3.isEmpty()); + assertEquals(sk3.getNumRetained(), sk1.getNumRetained()); + assertEquals(sk3.getN(), sk1.getN()); + assertEquals(sk3.getNormalizedRankError(false), sk1.getNormalizedRankError(false)); + try { sk3.getMinItem(); fail(); } catch (SketchesArgumentException e) {} + try { sk3.getMaxItem(); fail(); } catch (SketchesArgumentException e) {} + assertEquals(sk3.getSerializedSizeBytes(), sk1.getSerializedSizeBytes()); + //from heap -> byte[] -> off heap -> byte[] -> compare byte[] + final byte[] bytes2 = sk3.toByteArray(); + assertEquals(bytes, bytes2); + } + + @Test + public void serializeDeserializeOneValue() { + final KllLongsSketch sk1 = KllLongsSketch.newHeapInstance(); + sk1.update(1); + + //from heap -> byte[] -> heap + final byte[] bytes = sk1.toByteArray(); + final KllLongsSketch sk2 = KllLongsSketch.heapify(Memory.wrap(bytes)); + assertEquals(bytes.length, sk1.getSerializedSizeBytes()); + assertFalse(sk2.isEmpty()); + assertEquals(sk2.getNumRetained(), 1); + assertEquals(sk2.getN(), 1); + assertEquals(sk2.getNormalizedRankError(false), sk1.getNormalizedRankError(false)); + assertEquals(sk2.getMinItem(), 1L); + assertEquals(sk2.getMaxItem(), 1L); + assertEquals(sk2.getSerializedSizeBytes(), Long.BYTES + Long.BYTES); + + //from heap -> byte[] -> off heap + final KllLongsSketch sk3 = KllLongsSketch.wrap(Memory.wrap(bytes)); + assertFalse(sk3.isEmpty()); + assertEquals(sk3.getNumRetained(), 1); + assertEquals(sk3.getN(), 1); + assertEquals(sk3.getNormalizedRankError(false), sk1.getNormalizedRankError(false)); + assertEquals(sk3.getMinItem(), 1L); + assertEquals(sk3.getMaxItem(), 1L); + assertEquals(sk3.getSerializedSizeBytes(), sk1.getSerializedSizeBytes()); + //from heap -> byte[] -> off heap -> byte[] -> compare byte[] + final byte[] bytes2 = sk3.toByteArray(); + assertEquals(bytes, bytes2); + } + + @Test + public void serializeDeserializeMultipleValues() { + final KllLongsSketch sk1 = KllLongsSketch.newHeapInstance(); + final int n = 1000; + for (int i = 0; i < n; i++) { + sk1.update(i); + } + assertEquals(sk1.getMinItem(), 0); + assertEquals(sk1.getMaxItem(), 999L); + + //from heap -> byte[] -> heap + final byte[] bytes = sk1.toByteArray(); + final KllLongsSketch sk2 = KllLongsSketch.heapify(Memory.wrap(bytes)); + assertEquals(bytes.length, sk1.getSerializedSizeBytes()); + assertFalse(sk2.isEmpty()); + assertEquals(sk2.getNumRetained(), sk1.getNumRetained()); + assertEquals(sk2.getN(), sk1.getN()); + assertEquals(sk2.getNormalizedRankError(false), sk1.getNormalizedRankError(false)); + assertEquals(sk2.getMinItem(), sk1.getMinItem()); + assertEquals(sk2.getMaxItem(), sk1.getMaxItem()); + assertEquals(sk2.getSerializedSizeBytes(), sk1.getSerializedSizeBytes()); + + //from heap -> byte[] -> off heap + final KllLongsSketch sk3 = KllLongsSketch.wrap(Memory.wrap(bytes)); + assertFalse(sk3.isEmpty()); + assertEquals(sk3.getNumRetained(), sk1.getNumRetained()); + assertEquals(sk3.getN(), sk1.getN()); + assertEquals(sk3.getNormalizedRankError(false), sk1.getNormalizedRankError(false)); + assertEquals(sk3.getMinItem(), sk1.getMinItem()); + assertEquals(sk3.getMaxItem(), sk1.getMaxItem()); + assertEquals(sk3.getSerializedSizeBytes(), sk1.getSerializedSizeBytes()); + //from heap -> byte[] -> off heap -> byte[] -> compare byte[] + final byte[] bytes2 = sk3.toByteArray(); + assertEquals(bytes, bytes2); + } + +} diff --git a/src/test/java/org/apache/datasketches/kll/KllLongsSketchTest.java b/src/test/java/org/apache/datasketches/kll/KllLongsSketchTest.java new file mode 100644 index 000000000..1e4c1004a --- /dev/null +++ b/src/test/java/org/apache/datasketches/kll/KllLongsSketchTest.java @@ -0,0 +1,719 @@ +/* + * 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.min; +import static org.apache.datasketches.kll.KllSketch.SketchType.LONGS_SKETCH; +import static org.apache.datasketches.quantilescommon.QuantileSearchCriteria.EXCLUSIVE; +import static org.apache.datasketches.quantilescommon.QuantileSearchCriteria.INCLUSIVE; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +import org.apache.datasketches.common.SketchesArgumentException; +import org.apache.datasketches.memory.DefaultMemoryRequestServer; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.datasketches.quantilescommon.LongsSortedView; +import org.apache.datasketches.quantilescommon.LongsSortedViewIterator; +import org.testng.annotations.Test; + +public class KllLongsSketchTest { + private static final String LS = System.getProperty("line.separator"); + private static final double PMF_EPS_FOR_K_8 = KllSketch.getNormalizedRankError(8, true); + private static final double PMF_EPS_FOR_K_128 = KllSketch.getNormalizedRankError(128, true); + private static final double PMF_EPS_FOR_K_256 = KllSketch.getNormalizedRankError(256, true); + private static final double NUMERIC_NOISE_TOLERANCE = 1E-6; + private static final DefaultMemoryRequestServer memReqSvr = new DefaultMemoryRequestServer(); + + @Test + public void empty() { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + assertTrue(sketch.isEmpty()); + assertEquals(sketch.getN(), 0); + assertEquals(sketch.getNumRetained(), 0); + try { sketch.getRank(0); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getMinItem(); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getMaxItem(); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getQuantile(0.5); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getQuantiles(new double[] {0}); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getPMF(new long[] {0}); fail(); } catch (SketchesArgumentException e) {} + try { sketch.getCDF(new long[] {0}); fail(); } catch (SketchesArgumentException e) {} + assertNotNull(sketch.toString(true, true)); + assertNotNull(sketch.toString()); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void getQuantileInvalidArg() { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + sketch.update(1); + sketch.getQuantile(-1.0); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void getQuantilesInvalidArg() { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + sketch.update(1); + sketch.getQuantiles(new double[] {2.0}); + } + + @Test + public void oneValue() { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + sketch.update(1); + assertFalse(sketch.isEmpty()); + assertEquals(sketch.getN(), 1); + assertEquals(sketch.getNumRetained(), 1); + assertEquals(sketch.getRank(0L, EXCLUSIVE), 0.0); + assertEquals(sketch.getRank(1L, EXCLUSIVE), 0.0); + assertEquals(sketch.getRank(2L, EXCLUSIVE), 1.0); + assertEquals(sketch.getRank(0L, INCLUSIVE), 0.0); + assertEquals(sketch.getRank(1L, INCLUSIVE), 1.0); + assertEquals(sketch.getRank(2L, INCLUSIVE), 1.0); + assertEquals(sketch.getMinItem(), 1L); + assertEquals(sketch.getMaxItem(), 1L); + assertEquals(sketch.getQuantile(0.5, EXCLUSIVE), 1L); + assertEquals(sketch.getQuantile(0.5, INCLUSIVE), 1L); + } + + @Test + public void tenValues() { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 10; i++) { sketch.update(i); } + assertFalse(sketch.isEmpty()); + assertEquals(sketch.getN(), 10); + assertEquals(sketch.getNumRetained(), 10); + for (int i = 1; i <= 10; i++) { + assertEquals(sketch.getRank(i, EXCLUSIVE), (i - 1) / 10.0); + assertEquals(sketch.getRank(i, INCLUSIVE), i / 10.0); + } + final long[] qArr = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; + double[] rOut = sketch.getRanks(qArr); //inclusive + for (int i = 0; i < qArr.length; i++) { + assertEquals(rOut[i], (i + 1) / 10.0); + } + rOut = sketch.getRanks(qArr, EXCLUSIVE); //exclusive + for (int i = 0; i < qArr.length; i++) { + assertEquals(rOut[i], i / 10.0); + } + + for (int i = 0; i >= 10; i++) { + double rank = i/10.0; + double q = rank == 1.0 ? i : i + 1; + assertEquals(sketch.getQuantile(rank, EXCLUSIVE), q); + q = rank == 0 ? i + 1 : i; + assertEquals(sketch.getQuantile(rank, INCLUSIVE), q); + } + + { + // getQuantile() and getQuantiles() equivalence EXCLUSIVE + final long[] quantiles = + sketch.getQuantiles(new double[] {0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0}, EXCLUSIVE); + for (int i = 0; i <= 10; i++) { + assertEquals(sketch.getQuantile(i / 10.0, EXCLUSIVE), quantiles[i]); + } + } + { + // getQuantile() and getQuantiles() equivalence INCLUSIVE + final long[] quantiles = + sketch.getQuantiles(new double[] {0, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0}, INCLUSIVE); + for (int i = 0; i <= 10; i++) { + assertEquals(sketch.getQuantile(i / 10.0, INCLUSIVE), quantiles[i]); + } + } + } + + @Test + public void manyValuesEstimationMode() { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + final int n = 1_000_000; + + for (int i = 0; i < n; i++) { + sketch.update(i); + } + assertEquals(sketch.getN(), n); + + // test getRank + for (int i = 0; i < n; i++) { + final double trueRank = (double) i / n; + assertEquals(sketch.getRank(i), trueRank, PMF_EPS_FOR_K_256, "for value " + i); + } + + // test getPMF + final double[] pmf = sketch.getPMF(new long[] {n / 2}); // split at median + assertEquals(pmf.length, 2); + assertEquals(pmf[0], 0.5, PMF_EPS_FOR_K_256); + assertEquals(pmf[1], 0.5, PMF_EPS_FOR_K_256); + + assertEquals(sketch.getMinItem(), 0); // min value is exact + assertEquals(sketch.getMaxItem(), n - 1); // max value is exact + + // check at every 0.1 percentage point + final double[] fractions = new double[1001]; + final double[] reverseFractions = new double[1001]; // check that ordering doesn't matter + for (int i = 0; i <= 1000; i++) { + fractions[i] = (double) i / 1000; + reverseFractions[1000 - i] = fractions[i]; + } + final long[] quantiles = sketch.getQuantiles(fractions); + final long[] reverseQuantiles = sketch.getQuantiles(reverseFractions); + double previousQuantile = 0; + for (int i = 0; i <= 1000; i++) { + final double quantile = sketch.getQuantile(fractions[i]); + assertEquals(quantile, quantiles[i]); + assertEquals(quantile, reverseQuantiles[1000 - i]); + assertTrue(previousQuantile <= quantile); + previousQuantile = quantile; + } + } + + @Test + public void getRankGetCdfGetPmfConsistency() { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + final int n = 1000; + final long[] values = new long[n]; + for (int i = 0; i < n; i++) { + sketch.update(i); + values[i] = i; + } + { // inclusive = false (default) + final double[] ranks = sketch.getCDF(values); + final double[] pmf = sketch.getPMF(values); + double sumPmf = 0; + for (int i = 0; i < n; i++) { + assertEquals(ranks[i], sketch.getRank(values[i]), NUMERIC_NOISE_TOLERANCE, + "rank vs CDF for value " + i); + sumPmf += pmf[i]; + assertEquals(ranks[i], sumPmf, NUMERIC_NOISE_TOLERANCE, "CDF vs PMF for value " + i); + } + sumPmf += pmf[n]; + assertEquals(sumPmf, 1.0, NUMERIC_NOISE_TOLERANCE); + assertEquals(ranks[n], 1.0, NUMERIC_NOISE_TOLERANCE); + } + { // inclusive = true + final double[] ranks = sketch.getCDF(values, INCLUSIVE); + final double[] pmf = sketch.getPMF(values, INCLUSIVE); + double sumPmf = 0; + for (int i = 0; i < n; i++) { + assertEquals(ranks[i], sketch.getRank(values[i], INCLUSIVE), NUMERIC_NOISE_TOLERANCE, + "rank vs CDF for value " + i); + sumPmf += pmf[i]; + assertEquals(ranks[i], sumPmf, NUMERIC_NOISE_TOLERANCE, "CDF vs PMF for value " + i); + } + sumPmf += pmf[n]; + assertEquals(sumPmf, 1.0, NUMERIC_NOISE_TOLERANCE); + assertEquals(ranks[n], 1.0, NUMERIC_NOISE_TOLERANCE); + } + } + + @Test + public void merge() { + final KllLongsSketch sketch1 = KllLongsSketch.newHeapInstance(); + final KllLongsSketch sketch2 = KllLongsSketch.newHeapInstance(); + final int n = 10000; + for (int i = 0; i < n; i++) { + sketch1.update(i); + sketch2.update(2 * n - i - 1); + } + + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), (n - 1)); + + assertEquals(sketch2.getMinItem(), n); + assertEquals(sketch2.getMaxItem(), (2 * n - 1)); + + sketch1.merge(sketch2); + + assertFalse(sketch1.isEmpty()); + assertEquals(sketch1.getN(), 2L * n); + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), (2 * n - 1)); + assertEquals(sketch1.getQuantile(0.5), n, 2 * n * PMF_EPS_FOR_K_256); + } + + @Test + public void mergeLowerK() { + final KllLongsSketch sketch1 = KllLongsSketch.newHeapInstance(256); + final KllLongsSketch sketch2 = KllLongsSketch.newHeapInstance(128); + final int n = 10000; + for (int i = 0; i < n; i++) { + sketch1.update(i); + sketch2.update(2 * n - i - 1); + } + + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), n - 1L); + + assertEquals(sketch2.getMinItem(), n); + assertEquals(sketch2.getMaxItem(), 2L * n - 1L); + + assertTrue(sketch1.getNormalizedRankError(false) < sketch2.getNormalizedRankError(false)); + assertTrue(sketch1.getNormalizedRankError(true) < sketch2.getNormalizedRankError(true)); + sketch1.merge(sketch2); + + // sketch1 must get "contaminated" by the lower K in sketch2 + assertEquals(sketch1.getNormalizedRankError(false), sketch2.getNormalizedRankError(false)); + assertEquals(sketch1.getNormalizedRankError(true), sketch2.getNormalizedRankError(true)); + + assertFalse(sketch1.isEmpty()); + assertEquals(sketch1.getN(), 2 * n); + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), 2L * n - 1L); + assertEquals(sketch1.getQuantile(0.5), n, 2L * n * PMF_EPS_FOR_K_128); + } + + @Test + public void mergeEmptyLowerK() { + final KllLongsSketch sketch1 = KllLongsSketch.newHeapInstance(256); + final KllLongsSketch sketch2 = KllLongsSketch.newHeapInstance(128); + final int n = 10_000; + for (int i = 0; i < n; i++) { + sketch1.update(i); + } + + // rank error should not be affected by a merge with an empty sketch with lower K + final double rankErrorBeforeMerge = sketch1.getNormalizedRankError(true); + sketch1.merge(sketch2); + assertEquals(sketch1.getNormalizedRankError(true), rankErrorBeforeMerge); + + assertFalse(sketch1.isEmpty()); + assertEquals(sketch1.getN(), n); + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), n - 1); + assertEquals(sketch1.getQuantile(0.5), n / 2, n * PMF_EPS_FOR_K_256); + + //merge the other way + sketch2.merge(sketch1); + assertFalse(sketch1.isEmpty()); + assertEquals(sketch1.getN(), n); + assertEquals(sketch1.getMinItem(), 0); + assertEquals(sketch1.getMaxItem(), n - 1); + assertEquals(sketch1.getQuantile(0.5), n / 2, n * PMF_EPS_FOR_K_256); + } + + @Test + public void mergeExactModeLowerK() { + final KllLongsSketch sketch1 = KllLongsSketch.newHeapInstance(256); + final KllLongsSketch sketch2 = KllLongsSketch.newHeapInstance(128); + final int n = 10000; + for (int i = 0; i < n; i++) { + sketch1.update(i); + } + sketch2.update(1); + + // rank error should not be affected by a merge with a sketch in exact mode with lower K + final double rankErrorBeforeMerge = sketch1.getNormalizedRankError(true); + sketch1.merge(sketch2); + assertEquals(sketch1.getNormalizedRankError(true), rankErrorBeforeMerge); + } + + @Test + public void mergeMinMinValueFromOther() { + final KllLongsSketch sketch1 = KllLongsSketch.newHeapInstance(); + final KllLongsSketch sketch2 = KllLongsSketch.newHeapInstance(); + sketch1.update(1); + sketch2.update(2); + sketch2.merge(sketch1); + assertEquals(sketch2.getMinItem(), 1); + } + + @Test + public void mergeMinAndMaxFromOther() { + final KllLongsSketch sketch1 = KllLongsSketch.newHeapInstance(); + for (int i = 1; i <= 1_000_000; i++) { + sketch1.update(i); + } + final KllLongsSketch sketch2 = KllLongsSketch.newHeapInstance(10); + sketch2.merge(sketch1); + assertEquals(sketch2.getMinItem(), 1); + assertEquals(sketch2.getMaxItem(), 1_000_000); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void kTooSmall() { + KllLongsSketch.newHeapInstance(KllSketch.DEFAULT_M - 1); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void kTooLarge() { + KllLongsSketch.newHeapInstance(KllSketch.MAX_K + 1); + } + + @Test + public void minK() { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(KllSketch.DEFAULT_M); + for (int i = 0; i < 1000; i++) { + sketch.update(i); + } + assertEquals(sketch.getK(), KllSketch.DEFAULT_M); + assertEquals(sketch.getQuantile(0.5), 500, 1000 * PMF_EPS_FOR_K_8); + } + + @Test + public void maxK() { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(KllSketch.MAX_K); + for (int i = 0; i < 1000; i++) { + sketch.update(i); + } + assertEquals(sketch.getK(), KllSketch.MAX_K); + assertEquals(sketch.getQuantile(0.5), 500, 1000 * PMF_EPS_FOR_K_256); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void outOfOrderSplitPoints() { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + sketch.update(0); + sketch.getCDF(new long[] {1L, 0L}); + } + + @Test + public void checkReset() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 100; i++) { sk.update(i); } + long n1 = sk.getN(); + double min1 = sk.getMinItem(); + double max1 = sk.getMaxItem(); + sk.reset(); + for (int i = 1; i <= 100; i++) { sk.update(i); } + long n2 = sk.getN(); + double min2 = sk.getMinItem(); + double max2 = sk.getMaxItem(); + assertEquals(n2, n1); + assertEquals(min2, min1); + assertEquals(max2, max1); + } + + @Test + public void checkReadOnlyUpdate() { + KllLongsSketch sk1 = KllLongsSketch.newHeapInstance(20); + Memory mem = Memory.wrap(sk1.toByteArray()); + KllLongsSketch sk2 = KllLongsSketch.wrap(mem); + try { sk2.update(1); fail(); } catch (SketchesArgumentException e) { } + } + + @Test + public void checkNewDirectInstanceAndSize() { + WritableMemory wmem = WritableMemory.allocate(3000); + KllLongsSketch.newDirectInstance(wmem, memReqSvr); + try { KllLongsSketch.newDirectInstance(null, memReqSvr); fail(); } + catch (NullPointerException e) { } + try { KllLongsSketch.newDirectInstance(wmem, null); fail(); } + catch (NullPointerException e) { } + int updateSize = KllSketch.getMaxSerializedSizeBytes(200, 0, LONGS_SKETCH, true); + int compactSize = KllSketch.getMaxSerializedSizeBytes(200, 0, LONGS_SKETCH, false); + assertTrue(compactSize < updateSize); + } + + @Test + public void sortedView() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(); + sk.update(3); + sk.update(1); + sk.update(2); + + LongsSortedView view = sk.getSortedView(); + LongsSortedViewIterator itr = view.iterator(); + assertEquals(itr.next(), true); + assertEquals(itr.getQuantile(), 1); + assertEquals(itr.getWeight(), 1); + assertEquals(itr.getNaturalRank(EXCLUSIVE), 0); + assertEquals(itr.getNaturalRank(INCLUSIVE), 1); + assertEquals(itr.next(), true); + assertEquals(itr.getQuantile(), 2); + assertEquals(itr.getWeight(), 1); + assertEquals(itr.getNaturalRank(EXCLUSIVE), 1); + assertEquals(itr.getNaturalRank(INCLUSIVE), 2); + assertEquals(itr.next(), true); + assertEquals(itr.getQuantile(), 3); + assertEquals(itr.getWeight(), 1); + assertEquals(itr.getNaturalRank(EXCLUSIVE), 2); + assertEquals(itr.getNaturalRank(INCLUSIVE), 3); + assertEquals(itr.next(), false); + } + + @Test //also visual + public void checkCDF_PDF() { + final double[] cdfI = {.25, .50, .75, 1.0, 1.0 }; + final double[] cdfE = {0.0, .25, .50, .75, 1.0 }; + final double[] pmfI = {.25, .25, .25, .25, 0.0 }; + final double[] pmfE = {0.0, .25, .25, .25, .25 }; + final double toll = 1E-10; + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(); + final long[] doublesIn = {10, 20, 30, 40}; + for (int i = 0; i < doublesIn.length; i++) { sketch.update(doublesIn[i]); } + long[] sp = new long[] { 10, 20, 30, 40 }; + println("SplitPoints:"); + for (int i = 0; i < sp.length; i++) { + printf("%10d", sp[i]); + } + println(""); + println("INCLUSIVE:"); + double[] cdf = sketch.getCDF(sp, INCLUSIVE); + double[] pmf = sketch.getPMF(sp, INCLUSIVE); + printf("%10s%10s" + LS, "CDF", "PMF"); + for (int i = 0; i < cdf.length; i++) { + printf("%10.2f%10.2f" + LS, cdf[i], pmf[i]); + assertEquals(cdf[i], cdfI[i], toll); + assertEquals(pmf[i], pmfI[i], toll); + } + println("EXCLUSIVE"); + cdf = sketch.getCDF(sp, EXCLUSIVE); + pmf = sketch.getPMF(sp, EXCLUSIVE); + printf("%10s%10s" + LS, "CDF", "PMF"); + for (int i = 0; i < cdf.length; i++) { + printf("%10.2f%10.2f" + LS, cdf[i], pmf[i]); + assertEquals(cdf[i], cdfE[i], toll); + assertEquals(pmf[i], pmfE[i], toll); + } + } + + @Test + public void checkWrapCase1Doubles() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 21; i++) { sk.update(i); } + + Memory mem = Memory.wrap(sk.toByteArray()); + KllLongsSketch sk2 = KllLongsSketch.wrap(mem); + + assertTrue(mem.isReadOnly()); + assertTrue(sk2.isReadOnly()); + assertFalse(sk2.isDirect()); + } + + @Test + public void checkWritableWrapCase6And2Doubles() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 21; i++) { sk.update(i); } + + WritableMemory wmem = WritableMemory.writableWrap(KllHelper.toByteArray(sk, true)); + KllLongsSketch sk2 = KllLongsSketch.writableWrap(wmem, memReqSvr); + + assertFalse(wmem.isReadOnly()); + assertFalse(sk2.isReadOnly()); + assertFalse(sk2.isDirect()); + } + + @Test + public void checkKllSketchCase5Doubles() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 21; i++) { sk.update(i); } + + WritableMemory wmem = WritableMemory.writableWrap(sk.toByteArray()); + KllLongsSketch sk2 = KllLongsSketch.writableWrap(wmem, memReqSvr); + + assertFalse(wmem.isReadOnly()); + assertTrue(sk2.isReadOnly()); + assertFalse(sk2.isDirect()); + } + + @Test + public void checkKllSketchCase3Doubles() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 21; i++) { sk.update(i); } + + Memory mem = Memory.wrap(KllHelper.toByteArray(sk, true)); + WritableMemory wmem = (WritableMemory) mem; + KllLongsSketch sk2 = KllLongsSketch.writableWrap(wmem, memReqSvr); + + assertTrue(wmem.isReadOnly()); + assertTrue(sk2.isReadOnly()); + assertFalse(sk2.isDirect()); + } + + @Test + public void checkKllSketchCase7Doubles() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 21; i++) { sk.update(i); } + + Memory mem = Memory.wrap(KllHelper.toByteArray(sk, true)); + WritableMemory wmem = (WritableMemory) mem; + KllLongsSketch sk2 = KllLongsSketch.writableWrap(wmem, memReqSvr); + + assertTrue(wmem.isReadOnly()); + assertTrue(sk2.isReadOnly()); + assertFalse(sk2.isDirect()); + } + + @Test + public void checkReadOnlyExceptions() { + int[] intArr = new int[0]; + int intV = 2; + int idx = 1; + KllLongsSketch sk1 = KllLongsSketch.newHeapInstance(20); + Memory mem = Memory.wrap(sk1.toByteArray()); + KllLongsSketch sk2 = KllLongsSketch.wrap(mem); + try { sk2.setLevelsArray(intArr); fail(); } catch (SketchesArgumentException e) { } + try { sk2.setLevelsArrayAt(idx,intV); fail(); } catch (SketchesArgumentException e) { } + } + + @Test + public void checkIsSameResource() { + int cap = 128; + WritableMemory wmem = WritableMemory.allocate(cap); + WritableMemory reg1 = wmem.writableRegion(0, 64); + WritableMemory reg2 = wmem.writableRegion(64, 64); + assertFalse(reg1 == reg2); + assertFalse(reg1.isSameResource(reg2)); + + WritableMemory reg3 = wmem.writableRegion(0, 64); + assertFalse(reg1 == reg3); + assertTrue(reg1.isSameResource(reg3)); + + byte[] byteArr1 = KllLongsSketch.newHeapInstance(20).toByteArray(); + reg1.putByteArray(0, byteArr1, 0, byteArr1.length); + KllLongsSketch sk1 = KllLongsSketch.wrap(reg1); + + byte[] byteArr2 = KllLongsSketch.newHeapInstance(20).toByteArray(); + reg2.putByteArray(0, byteArr2, 0, byteArr2.length); + assertFalse(sk1.isSameResource(reg2)); + + byte[] byteArr3 = KllLongsSketch.newHeapInstance(20).toByteArray(); + reg3.putByteArray(0, byteArr3, 0, byteArr3.length); + assertTrue(sk1.isSameResource(reg3)); + } + + @Test + public void checkSortedViewAfterReset() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(20); + sk.update(1L); + LongsSortedView sv = sk.getSortedView(); + long dsv = sv.getQuantile(1.0, INCLUSIVE); + assertEquals(dsv, 1L); + sk.reset(); + try { sk.getSortedView(); fail(); } catch (SketchesArgumentException e) { } + } + + @Test + public void checkVectorUpdate() { + boolean withLevels = false; + boolean withLevelsAndItems = true; + int k = 20; + int n = 108; + int maxVsz = 40; //max vector size + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + int j = 1; + int rem; + while ((rem = n - j + 1) > 0) { + int vecSz = min(rem, maxVsz); + long[] v = new long[vecSz]; + for (int i = 0; i < vecSz; i++) { v[i] = j++; } + sk.update(v, 0, vecSz); + } + println(LS + "#<<< END STATE # >>>"); + println(sk.toString(withLevels, withLevelsAndItems)); + println(""); + assertEquals(sk.getN(), 108); + assertEquals(sk.getMaxItem(), 108L); + assertEquals(sk.getMinItem(), 1L); + } + + @Test + public void vectorizedUpdates() { + final int trials = 1; + final int M = 1; //number of vectors + final int N = 1000; //vector size + final int K = 256; + final long[] values = new long[N]; + long vIn = 1L; + long totN = 0; + final long startTime = System.nanoTime(); + for (int t = 0; t < trials; t++) { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(K); + for (int m = 0; m < M; m++) { + for (int n = 0; n < N; n++) { + values[n] = vIn++; //fill vector + } + sketch.update(values, 0, N); //vector input + } + totN = sketch.getN(); + assertEquals(totN, M * N); + assertEquals(sketch.getMinItem(), 1L); + assertEquals(sketch.getMaxItem(), totN); + assertEquals(sketch.getQuantile(0.5), totN / 2, totN * PMF_EPS_FOR_K_256 * 2.0); //wider tolerance + } + final long runTime = System.nanoTime() - startTime; + println("Vectorized Updates"); + printf(" Vector size : %,12d" + LS, N); + printf(" Num Vectors : %,12d" + LS, M); + printf(" Total Input : %,12d" + LS, totN); + printf(" Run Time mS : %,12.3f" + LS, runTime / 1e6); + final double trialTime = runTime / (1e6 * trials); + printf(" mS / Trial : %,12.3f" + LS, trialTime); + final double updateTime = runTime / (1.0 * totN * trials); + printf(" nS / Update : %,12.3f" + LS, updateTime); + } + + @Test + public void nonVectorizedUpdates() { + final int trials = 1; + final int M = 1; //number of vectors + final int N = 1000; //vector size + final int K = 256; + final long[] values = new long[N]; + long vIn = 1L; + long totN = 0; + final long startTime = System.nanoTime(); + for (int t = 0; t < trials; t++) { + final KllLongsSketch sketch = KllLongsSketch.newHeapInstance(K); + for (int m = 0; m < M; m++) { + for (int n = 0; n < N; n++) { + values[n] = vIn++; //fill vector + } + for (int i = 0; i < N; i++) { + sketch.update(values[i]); //single item input + } + } + totN = sketch.getN(); + assertEquals(totN, M * N); + assertEquals(sketch.getMinItem(), 1L); + assertEquals(sketch.getMaxItem(), totN); + assertEquals(sketch.getQuantile(0.5), totN / 2, totN * PMF_EPS_FOR_K_256 * 2.0); //wider tolerance + } + final long runTime = System.nanoTime() - startTime; + println("Vectorized Updates"); + printf(" Vector size : %,12d" + LS, N); + printf(" Num Vectors : %,12d" + LS, M); + printf(" Total Input : %,12d" + LS, totN); + printf(" Run Time mS : %,12.3f" + LS, runTime / 1e6); + final double trialTime = runTime / (1e6 * trials); + printf(" mS / Trial : %,12.3f" + LS, trialTime); + final double updateTime = runTime / (1.0 * totN * trials); + printf(" nS / Update : %,12.3f" + LS, updateTime); + } + + private final static boolean enablePrinting = false; + + /** + * @param format the format + * @param args the args + */ + private static final void printf(final String format, final Object ...args) { + if (enablePrinting) { System.out.printf(format, args); } + } + + /** + * @param o the Object to println + */ + private static final void println(final Object o) { + if (enablePrinting) { System.out.println(o.toString()); } + } +} diff --git a/src/test/java/org/apache/datasketches/kll/KllMiscDirectLongsTest.java b/src/test/java/org/apache/datasketches/kll/KllMiscDirectLongsTest.java new file mode 100644 index 000000000..366f93f7d --- /dev/null +++ b/src/test/java/org/apache/datasketches/kll/KllMiscDirectLongsTest.java @@ -0,0 +1,459 @@ +/* + * 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.KllSketch.SketchType.LONGS_SKETCH; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +import org.apache.datasketches.common.SketchesArgumentException; +import org.apache.datasketches.memory.DefaultMemoryRequestServer; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.datasketches.quantilescommon.LongsSortedView; +import org.apache.datasketches.quantilescommon.LongsSortedViewIterator; +import org.testng.annotations.Test; + +public class KllMiscDirectLongsTest { + static final String LS = System.getProperty("line.separator"); + private static final DefaultMemoryRequestServer memReqSvr = new DefaultMemoryRequestServer(); + + @Test + public void checkBounds() { + final KllLongsSketch kll = getDirectLongsSketch(200, 0); + for (int i = 0; i < 1000; i++) { + kll.update(i); + } + final double eps = kll.getNormalizedRankError(false); + final long est = kll.getQuantile(0.5); + final long ub = kll.getQuantileUpperBound(0.5); + final long lb = kll.getQuantileLowerBound(0.5); + assertEquals(ub, kll.getQuantile(.5 + eps)); + assertEquals(lb, kll.getQuantile(0.5 - eps)); + println("Ext : " + est); + println("UB : " + ub); + println("LB : " + lb); + final double rest = kll.getRank(est); + final double restUB = kll.getRankUpperBound(rest); + final double restLB = kll.getRankLowerBound(rest); + assertTrue(restUB - rest < (2 * eps)); + assertTrue(rest - restLB < (2 * eps)); + } + + //@Test //enable static println(..) for visual checking + public void visualCheckToString() { + final int k = 20; + final KllLongsSketch sk = getDirectLongsSketch(k, 0); + for (int i = 0; i < 10; i++) { sk.update(i + 1); } + println(sk.toString(true, true)); + + final KllLongsSketch sk2 = getDirectLongsSketch(k, 0); + for (int i = 0; i < 400; i++) { sk2.update(i + 1); } + println("\n" + sk2.toString(true, true)); + + sk2.merge(sk); + final String s2 = sk2.toString(true, true); + println(LS + s2); + } + + @Test + public void viewDirectCompactions() { + int k = 20; + int u = 108; + KllLongsSketch sk = getDirectLongsSketch(k, 0); + for (int i = 1; i <= u; i++) { + sk.update(i); + if (sk.levelsArr[0] == 0) { + println(sk.toString(true, true)); + sk.update(++i); + println(sk.toString(true, true)); + assertEquals(sk.getLongItemsArray()[sk.levelsArr[0]], i); + } + } + } + + @Test + public void viewCompactionAndSortedView() { + int k = 20; + KllLongsSketch sk = getDirectLongsSketch(k, 0); + show(sk, 20); + LongsSortedView sv = sk.getSortedView(); + LongsSortedViewIterator itr = sv.iterator(); + printf("%12s%12s\n", "Value", "CumWeight"); + while (itr.next()) { + long v = itr.getQuantile(); + long wt = itr.getWeight(); + printf("%12d%12d\n", v, wt); + } + } + + private static void show(final KllLongsSketch sk, int limit) { + int i = (int) sk.getN(); + for ( ; i < limit; i++) { sk.update(i + 1); } + println(sk.toString(true, true)); + } + + @Test + public void checkSketchInitializeLongHeap() { + int k = 20; //don't change this + KllLongsSketch sk; + + //println("#### CASE: LONG FULL HEAP"); + sk = getDirectLongsSketch(k, 0); + for (int i = 1; i <= k + 1; i++) { sk.update(i); } + //println(sk.toString(true, true)); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), k + 1); + assertEquals(sk.getNumRetained(), 11); + assertFalse(sk.isEmpty()); + assertTrue(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 33); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 3); + assertEquals(sk.getMaxItem(), 21L); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getNumLevels(), 2); + assertFalse(sk.isLevelZeroSorted()); + + //println("#### CASE: LONG HEAP EMPTY"); + sk = getDirectLongsSketch(k, 0); + //println(sk.toString(true, true)); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 0); + assertEquals(sk.getNumRetained(), 0); + assertTrue(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + try { sk.getMaxItem(); fail(); } catch (SketchesArgumentException e) { } + try { sk.getMinItem(); fail(); } catch (SketchesArgumentException e) { } + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + + //println("#### CASE: LONG HEAP SINGLE"); + sk = getDirectLongsSketch(k, 0); + sk.update(1); + //println(sk.toString(true, true)); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 1); + assertEquals(sk.getNumRetained(), 1); + assertFalse(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + assertEquals(sk.getMaxItem(), 1L); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + } + + @Test + public void checkSketchInitializeLongHeapifyCompactMem() { + int k = 20; //don't change this + KllLongsSketch sk; + KllLongsSketch sk2; + byte[] compBytes; + WritableMemory wmem; + + //println("#### CASE: LONG FULL HEAPIFIED FROM COMPACT"); + sk2 = getDirectLongsSketch(k, 0); + for (int i = 1; i <= k + 1; i++) { sk2.update(i); } + //println(sk.toString(true, true)); + compBytes = sk2.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes); + //println(KllPreambleUtil.toString(wmem)); + sk = KllLongsSketch.heapify(wmem); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), k + 1); + assertEquals(sk.getNumRetained(), 11); + assertFalse(sk.isEmpty()); + assertTrue(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 33); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 3); + assertEquals(sk.getMaxItem(), 21L); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getNumLevels(), 2); + assertFalse(sk.isLevelZeroSorted()); + + //println("#### CASE: LONG EMPTY HEAPIFIED FROM COMPACT"); + sk2 = getDirectLongsSketch(k, 0); + //println(sk.toString(true, true)); + compBytes = sk2.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes); + //println(KllPreambleUtil.toString(wmem)); + sk = KllLongsSketch.heapify(wmem); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 0); + assertEquals(sk.getNumRetained(), 0); + assertTrue(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + try { sk.getMaxItem(); fail(); } catch (SketchesArgumentException e) { } + try { sk.getMinItem(); fail(); } catch (SketchesArgumentException e) { } + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + + //println("#### CASE: LONG SINGLE HEAPIFIED FROM COMPACT"); + sk2 = getDirectLongsSketch(k, 0); + sk2.update(1); + //println(sk2.toString(true, true)); + compBytes = sk2.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes); + //println(KllPreambleUtil.toString(wmem)); + sk = KllLongsSketch.heapify(wmem); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 1); + assertEquals(sk.getNumRetained(), 1); + assertFalse(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + assertEquals(sk.getMaxItem(), 1L); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + } + + @Test + public void checkSketchInitializeLongHeapifyUpdatableMem() { + int k = 20; //don't change this + KllLongsSketch sk; + KllLongsSketch sk2; + byte[] compBytes; + WritableMemory wmem; + + //println("#### CASE: LONG FULL HEAPIFIED FROM UPDATABLE"); + sk2 = getDirectLongsSketch(k, 0); + for (int i = 1; i <= k + 1; i++) { sk2.update(i); } + //println(sk2.toString(true, true)); + compBytes = KllHelper.toByteArray(sk2,true); + wmem = WritableMemory.writableWrap(compBytes); + sk = KllHeapLongsSketch.heapifyImpl(wmem); + //println(sk.toString(true, true)); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), k + 1); + assertEquals(sk.getNumRetained(), 11); + assertFalse(sk.isEmpty()); + assertTrue(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 33); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 3); + assertEquals(sk.getMaxItem(), 21L); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getNumLevels(), 2); + assertFalse(sk.isLevelZeroSorted()); + + // println("#### CASE: LONG EMPTY HEAPIFIED FROM UPDATABLE"); + sk2 = getDirectLongsSketch(k, 0); + //println(sk.toString(true, true)); + compBytes = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(compBytes); + //println(KllPreambleUtil.toString(wmem)); + sk = KllHeapLongsSketch.heapifyImpl(wmem); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 0); + assertEquals(sk.getNumRetained(), 0); + assertTrue(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + try { sk.getMaxItem(); fail(); } catch (SketchesArgumentException e) { } + try { sk.getMinItem(); fail(); } catch (SketchesArgumentException e) { } + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + + //println("#### CASE: LONG SINGLE HEAPIFIED FROM UPDATABLE"); + sk2 = getDirectLongsSketch(k, 0); + sk2.update(1); + //println(sk.toString(true, true)); + compBytes = KllHelper.toByteArray(sk2,true); + wmem = WritableMemory.writableWrap(compBytes); + //println(KllPreambleUtil.toString(wmem)); + sk = KllHeapLongsSketch.heapifyImpl(wmem); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 1); + assertEquals(sk.getNumRetained(), 1); + assertFalse(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + assertEquals(sk.getMaxItem(), 1L); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + } + + @Test + public void checkMemoryToStringLongUpdatable() { + int k = 20; //don't change this + KllLongsSketch sk; + KllLongsSketch sk2; + byte[] upBytes; + byte[] upBytes2; + WritableMemory wmem; + String s; + + println("#### CASE: LONG FULL UPDATABLE"); + sk = getDirectLongsSketch(k, 0); + for (int i = 1; i <= k + 1; i++) { sk.update(i); } + upBytes = KllHelper.toByteArray(sk, true); + wmem = WritableMemory.writableWrap(upBytes); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 1: sketch to byte[]/memory & analyze memory"); + println(s); + sk2 = KllLongsSketch.writableWrap(wmem, memReqSvr); + upBytes2 = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(upBytes2); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 2: memory to heap sketch, to byte[]/memory & analyze memory. Should match above"); + println(s); + assertEquals(upBytes, upBytes2); + + println("#### CASE: LONG EMPTY UPDATABLE"); + sk = getDirectLongsSketch(k, 0); + upBytes = KllHelper.toByteArray(sk, true); + wmem = WritableMemory.writableWrap(upBytes); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 1: sketch to byte[]/memory & analyze memory"); + println(s); + sk2 = KllLongsSketch.writableWrap(wmem, memReqSvr); + upBytes2 = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(upBytes2); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 2: memory to heap sketch, to byte[]/memory & analyze memory. Should match above"); + println(s); + assertEquals(upBytes, upBytes2); + + println("#### CASE: LONG SINGLE UPDATABL"); + sk = getDirectLongsSketch(k, 0); + sk.update(1); + upBytes = KllHelper.toByteArray(sk, true); + wmem = WritableMemory.writableWrap(upBytes); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 1: sketch to byte[]/memory & analyze memory"); + println(s); + sk2 = KllLongsSketch.writableWrap(wmem, memReqSvr); + upBytes2 = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(upBytes2); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 2: memory to heap sketch, to byte[]/memory & analyze memory. Should match above"); + println(s); + assertEquals(upBytes, upBytes2); + } + + @Test + public void checkSimpleMerge() { + int k = 20; + int n1 = 21; + int n2 = 21; + KllLongsSketch sk1 = getDirectLongsSketch(k, 0); + KllLongsSketch sk2 = getDirectLongsSketch(k, 0); + for (int i = 1; i <= n1; i++) { + sk1.update(i); + } + for (int i = 1; i <= n2; i++) { + sk2.update(i + 100); + } + println(sk1.toString(true, true)); + println(sk2.toString(true, true)); + sk1.merge(sk2); + println(sk1.toString(true, true)); + assertEquals(sk1.getMaxItem(), 121L); + assertEquals(sk1.getMinItem(), 1L); + } + + @Test + public void checkSizes() { + KllLongsSketch sk = getDirectLongsSketch(20, 0); + for (int i = 1; i <= 21; i++) { sk.update(i); } + //println(sk.toString(true, true)); + byte[] byteArr1 = KllHelper.toByteArray(sk, true); + int size1 = sk.currentSerializedSizeBytes(true); + assertEquals(size1, byteArr1.length); + byte[] byteArr2 = sk.toByteArray(); + int size2 = sk.currentSerializedSizeBytes(false); + assertEquals(size2, byteArr2.length); + } + + @Test + public void checkNewInstance() { + int k = 200; + WritableMemory dstMem = WritableMemory.allocate(3000); + KllLongsSketch sk = KllLongsSketch.newDirectInstance(k, dstMem, memReqSvr); + for (int i = 1; i <= 10_000; i++) {sk.update(i); } + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getMaxItem(), 10000L); + //println(sk.toString(true, true)); + } + + @Test + public void checkDifferentM() { + int k = 20; + int m = 4; + WritableMemory dstMem = WritableMemory.allocate(1000); + KllLongsSketch sk = KllDirectLongsSketch.newDirectUpdatableInstance(k, m, dstMem, memReqSvr); + for (int i = 1; i <= 200; i++) {sk.update(i); } + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getMaxItem(), 200L); + } + + private static KllLongsSketch getDirectLongsSketch(final int k, final int n) { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= n; i++) { sk.update(i); } + byte[] byteArr = KllHelper.toByteArray(sk, true); + WritableMemory wmem = WritableMemory.writableWrap(byteArr); + KllLongsSketch dfsk = KllLongsSketch.writableWrap(wmem, memReqSvr); + return dfsk; + } + + @Test + public void printlnTest() { + String s = "PRINTING: printf in " + this.getClass().getName(); + println(s); + printf("%s\n", s); + } + + private final static boolean enablePrinting = false; + + /** + * @param format the format + * @param args the args + */ + private static final void printf(final String format, final Object ...args) { + if (enablePrinting) { System.out.printf(format, args); } + } + + /** + * @param o the Object to println + */ + private static final void println(final Object o) { + if (enablePrinting) { System.out.println(o.toString()); } + } + +} diff --git a/src/test/java/org/apache/datasketches/kll/KllMiscDoublesTest.java b/src/test/java/org/apache/datasketches/kll/KllMiscDoublesTest.java index e58c27419..4ce988d22 100644 --- a/src/test/java/org/apache/datasketches/kll/KllMiscDoublesTest.java +++ b/src/test/java/org/apache/datasketches/kll/KllMiscDoublesTest.java @@ -100,8 +100,8 @@ public void checkHeapifyExceptions2() { @Test(expectedExceptions = SketchesArgumentException.class) public void checkHeapifyExceptions3() { KllDoublesSketch sk = KllDoublesSketch.newHeapInstance(); - sk.update(1.0f); - sk.update(2.0f); + sk.update(1.0); + sk.update(2.0); WritableMemory wmem = WritableMemory.writableWrap(sk.toByteArray()); wmem.putByte(0, (byte) 1); //corrupt preamble ints, should be 5 KllDoublesSketch.heapify(wmem); diff --git a/src/test/java/org/apache/datasketches/kll/KllMiscLongsTest.java b/src/test/java/org/apache/datasketches/kll/KllMiscLongsTest.java new file mode 100644 index 000000000..018ad91db --- /dev/null +++ b/src/test/java/org/apache/datasketches/kll/KllMiscLongsTest.java @@ -0,0 +1,790 @@ +/* + * 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 org.apache.datasketches.common.SketchesArgumentException; +import org.apache.datasketches.kll.KllDirectLongsSketch.KllDirectCompactLongsSketch; +import org.apache.datasketches.memory.DefaultMemoryRequestServer; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.memory.MemoryRequestServer; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.datasketches.quantilescommon.LongsSortedView; +import org.apache.datasketches.quantilescommon.LongsSortedViewIterator; +import org.testng.annotations.Test; + +import static org.apache.datasketches.common.Util.LS; +import static org.apache.datasketches.common.Util.bitAt; +import static org.apache.datasketches.kll.KllSketch.SketchType.LONGS_SKETCH; +import static org.apache.datasketches.quantilescommon.QuantileSearchCriteria.INCLUSIVE; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +/** + * @author Lee Rhodes + */ +public class KllMiscLongsTest { + private final MemoryRequestServer memReqSvr = new DefaultMemoryRequestServer(); + + @Test + public void checkSortedViewConstruction() { + final KllLongsSketch kll = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 20; i++) { kll.update(i); } + LongsSortedView fsv = kll.getSortedView(); + long[] cumWeights = fsv.getCumulativeWeights(); + long[] values = fsv.getQuantiles(); + assertEquals(cumWeights.length, 20); + assertEquals(values.length, 20); + for (int i = 0; i < 20; i++) { + assertEquals(cumWeights[i], i + 1); + assertEquals(values[i], i + 1); + } + } + + @Test //set static enablePrinting = true for visual checking + public void checkBounds() { + final KllLongsSketch kll = KllLongsSketch.newHeapInstance(); //default k = 200 + for (int i = 0; i < 1000; i++) { + kll.update(i); + } + final double eps = kll.getNormalizedRankError(false); + final long est = kll.getQuantile(0.5); + final long ub = kll.getQuantileUpperBound(0.5); + final long lb = kll.getQuantileLowerBound(0.5); + assertEquals(ub, kll.getQuantile(.5 + eps)); + assertEquals(lb, kll.getQuantile(0.5 - eps)); + println("Ext : " + est); + println("UB : " + ub); + println("LB : " + lb); + final double rest = kll.getRank(est); + final double restUB = kll.getRankUpperBound(rest); + final double restLB = kll.getRankLowerBound(rest); + assertTrue(restUB - rest < (2 * eps)); + assertTrue(rest - restLB < (2 * eps)); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void checkHeapifyExceptions1() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(); + WritableMemory wmem = WritableMemory.writableWrap(sk.toByteArray()); + wmem.putByte(6, (byte) 3); //corrupt with odd M + KllLongsSketch.heapify(wmem); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void checkHeapifyExceptions2() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(); + WritableMemory wmem = WritableMemory.writableWrap(sk.toByteArray()); + wmem.putByte(0, (byte) 1); //corrupt preamble ints, should be 2 + KllLongsSketch.heapify(wmem); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void checkHeapifyExceptions3() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(); + sk.update(1); + sk.update(2); + WritableMemory wmem = WritableMemory.writableWrap(sk.toByteArray()); + wmem.putByte(0, (byte) 1); //corrupt preamble ints, should be 5 + KllLongsSketch.heapify(wmem); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void checkHeapifyExceptions4() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(); + WritableMemory wmem = WritableMemory.writableWrap(sk.toByteArray()); + wmem.putByte(1, (byte) 0); //corrupt SerVer, should be 1 or 2 + KllLongsSketch.heapify(wmem); + } + + @Test(expectedExceptions = SketchesArgumentException.class) + public void checkHeapifyExceptions5() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(); + WritableMemory wmem = WritableMemory.writableWrap(sk.toByteArray()); + wmem.putByte(2, (byte) 0); //corrupt FamilyID, should be 15 + KllLongsSketch.heapify(wmem); + } + + @Test //set static enablePrinting = true for visual checking + public void checkMisc() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(8); + try { sk.getMaxItem(); fail(); } catch (SketchesArgumentException e) {} //empty + println(sk.toString(true, true)); + for (int i = 0; i < 20; i++) { sk.update(i); } + println(sk.toString(true, true)); + sk.toByteArray(); + final long[] items = sk.getLongItemsArray(); + assertEquals(items.length, 16); + final int[] levels = sk.getLevelsArray(sk.sketchStructure); + assertEquals(levels.length, 3); + assertEquals(sk.getNumLevels(), 2); + } + + @Test //set static enablePrinting = true for visual checking + public void visualCheckToString() { + final KllLongsSketch sk = KllLongsSketch.newHeapInstance(20); + int n = 21; + for (int i = 1; i <= n; i++) { sk.update(i); } + println(sk.toString(true, true)); + assertEquals(sk.getNumLevels(), 2); + assertEquals(sk.getMinItem(), 1); + assertEquals(sk.getMaxItem(), 21); + assertEquals(sk.getNumRetained(), 11); + + final KllLongsSketch sk2 = KllLongsSketch.newHeapInstance(20); + n = 400; + for (int i = 101; i <= n + 100; i++) { sk2.update(i); } + println(LS + sk2.toString(true, true)); + assertEquals(sk2.getNumLevels(), 5); + assertEquals(sk2.getMinItem(), 101); + assertEquals(sk2.getMaxItem(), 500); + assertEquals(sk2.getNumRetained(), 52); + + sk2.merge(sk); + println(LS + sk2.toString(true, true)); + assertEquals(sk2.getNumLevels(), 5); + assertEquals(sk2.getMinItem(), 1); + assertEquals(sk2.getMaxItem(), 500); + assertEquals(sk2.getNumRetained(), 56); + } + + @Test //set static enablePrinting = true for visual checking + public void viewHeapCompactions() { + int k = 20; + int n = 108; + boolean withLevels = false; + boolean withLevelsAndItems = true; + int compaction = 0; + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= n; i++) { + sk.update(i); + if (sk.levelsArr[0] == 0) { + println(LS + "#<<< BEFORE COMPACTION # " + (++compaction) + " >>>"); + println(sk.toString(withLevels, withLevelsAndItems)); + sk.update(++i); + println(LS + "#<<< AFTER COMPACTION # " + (compaction) + " >>>"); + println(sk.toString(withLevels, withLevelsAndItems)); + assertEquals(sk.getLongItemsArray()[sk.levelsArr[0]], i); + } + } + println(LS + "#<<< END STATE # >>>"); + println(sk.toString(withLevels, withLevelsAndItems)); + println(""); + } + + @Test //set static enablePrinting = true for visual checking + public void viewDirectCompactions() { + int k = 20; + int n = 108; + boolean withLevels = false; + boolean withLevelsAndItems = true; + int compaction = 0; + int sizeBytes = KllSketch.getMaxSerializedSizeBytes(k, n, LONGS_SKETCH, true); + WritableMemory wmem = WritableMemory.allocate(sizeBytes); + KllLongsSketch sk = KllLongsSketch.newDirectInstance(k, wmem, memReqSvr); + for (int i = 1; i <= n; i++) { + sk.update(i); + if (sk.levelsArr[0] == 0) { + println(LS + "#<<< BEFORE COMPACTION # " + (++compaction) + " >>>"); + println(sk.toString(withLevels, withLevelsAndItems)); + sk.update(++i); + println(LS + "#<<< AFTER COMPACTION # " + (compaction) + " >>>"); + println(sk.toString(withLevels, withLevelsAndItems)); + assertEquals(sk.getLongItemsArray()[sk.levelsArr[0]], i); + } + } + println(LS + "#<<< END STATE # >>>"); + println(sk.toString(withLevels, withLevelsAndItems)); + println(""); + } + + @Test //set static enablePrinting = true for visual checking + public void viewCompactionAndSortedView() { + int n = 43; + KllLongsSketch sk = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= n; i++) { sk.update(i); } + println(sk.toString(true, true)); + LongsSortedView sv = sk.getSortedView(); + LongsSortedViewIterator itr = sv.iterator(); + println("### SORTED VIEW"); + printf("%6s %12s %12s" + LS, "Idx", "Value", "Weight"); + int i = 0; + while (itr.next()) { + long v = itr.getQuantile(); + long wt = itr.getWeight(); + printf("%6d %12d %12d" + LS, i, v, wt); + i++; + } + assertEquals(sv.getMinItem(), 1L); + assertEquals(sv.getMaxItem(), n); + } + + @Test //set static enablePrinting = true for visual checking + public void checkWeightedUpdates1() { + int k = 20; + int weight = 127; + long item = 10; + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + println(sk.toString(true, true)); + sk.update(item, weight); + println(sk.toString(true, true)); + assertEquals(sk.getNumRetained(), 7); + assertEquals(sk.getN(), weight); + sk.update(item, weight); + println(sk.toString(true, true)); + assertEquals(sk.getNumRetained(), 14); + assertEquals(sk.getN(), 254); + } + + @Test //set static enablePrinting = true for visual checking + public void checkWeightedUpdates2() { + int k = 20; + int initial = 1000; + int weight = 127; + long item = 10; + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= initial; i++) { sk.update(i + 1000); } + println(sk.toString(true, true)); + sk.update(item, weight); + println(sk.toString(true, true)); + assertEquals(sk.getNumRetained(), 65); + assertEquals(sk.getN(), 1127); + + LongsSortedViewIterator itr = sk.getSortedView().iterator(); + println("### SORTED VIEW"); + printf("%12s %12s %12s" + LS, "Value", "Weight", "NaturalRank"); + long cumWt = 0; + while (itr.next()) { + long v = itr.getQuantile(); + long wt = itr.getWeight(); + long natRank = itr.getNaturalRank(INCLUSIVE); + cumWt += wt; + assertEquals(cumWt, natRank); + printf("%12d %12d %12d" + LS, v, wt, natRank); + } + assertEquals(cumWt, sk.getN()); + } + + @Test //set static enablePrinting = true for visual checking + public void checkCreateItemsArray() { //used with weighted updates + long item = 10; + int weight = 108; + long[] itemsArr = KllLongsHelper.createItemsArray(item, weight); + assertEquals(itemsArr.length, 4); + for (int i = 0; i < itemsArr.length; i++) { itemsArr[i] = item; } + outputItems(itemsArr); + } + + private static void outputItems(long[] itemsArr) { + String[] hdr2 = {"Index", "Value"}; + String hdr2fmt = "%6s %15s" + LS; + String d2fmt = "%6d %15d" + LS; + println("ItemsArr"); + printf(hdr2fmt, (Object[]) hdr2); + for (int i = 0; i < itemsArr.length; i++) { + printf(d2fmt, i, itemsArr[i]); + } + println(""); + } + + @Test //set static enablePrinting = true for visual checking + public void checkCreateLevelsArray() { //used with weighted updates + int weight = 108; + int[] levelsArr = KllHelper.createLevelsArray(weight); + assertEquals(levelsArr.length, 8); + int[] correct = {0,0,0,1,2,2,3,4}; + for (int i = 0; i < levelsArr.length; i++) { + assertEquals(levelsArr[i], correct[i]); + } + outputLevels(weight, levelsArr); + } + + private static void outputLevels(int weight, int[] levelsArr) { + String[] hdr = {"Lvl", "StartAdr", "BitPattern", "Weight"}; + String hdrfmt = "%3s %9s %10s %s" + LS; + String dfmt = "%3d %9d %10d %d" + LS; + String dfmt_2 = "%3d %9d %s" + LS; + println("Count = " + weight + " => " + (Integer.toBinaryString(weight))); + println("LevelsArr"); + printf(hdrfmt, (Object[]) hdr); + for (int i = 0; i < levelsArr.length; i++) { + if (i == levelsArr.length - 1) { printf(dfmt_2, i, levelsArr[i], "ItemsArr.length"); } + else { + int j = bitAt(weight, i); + printf(dfmt, i, levelsArr[i], j, 1 << (i)); + } + } + println(""); + } + + @Test + public void viewMemorySketchData() { + int k = 20; + int n = 109; + boolean withLevels = true; + boolean withLevelsAndItems = true; + KllLongsSketch sk = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= n; i++) { sk.update(i); } + byte[] byteArr = sk.toByteArray(); + Memory mem = Memory.wrap(byteArr); + KllLongsSketch fltSk = KllLongsSketch.wrap(mem); + println(fltSk.toString(withLevels, withLevelsAndItems)); + assertEquals(fltSk.getN(), n); + } + + @Test //set static enablePrinting = true for visual checking + public void checkIntCapAux() { + String[] hdr = {"level", "depth", "wt", "cap", "(end)", "MaxN"}; + String hdrFmt = "%6s %6s %28s %10s %10s %34s" + LS; + String dataFmt = "%6d %6d %,28d %,10d %,10d %,34d" + LS; + int k = 1000; + int m = 8; + int numLevels = 20; + println("k=" + k + ", m=" + m + ", numLevels=" + numLevels); + printf(hdrFmt, (Object[]) hdr); + long maxN = 0; + long[] correct = {0,1,1,2,2,3,5,8,12,17,26,39,59,88,132,198,296,444,667,1000}; + for (int i = 0; i < numLevels; i++) { + int depth = numLevels - i - 1; + long cap = KllHelper.intCapAux(k, depth); + long end = Math.max(m, cap); + long wt = 1L << i; + maxN += wt * end; + printf(dataFmt, i, depth, wt, cap, end, maxN); + assertEquals(cap, correct[i]); + } + } + + @Test //set static enablePrinting = true for visual checking + public void checkIntCapAuxAux() { + String[] hdr = {"d","twoK","2k*2^d","3^d","tmp=2k*2^d/3^d","(tmp + 1)/2", "(end)"}; + String hdrFmt = "%6s %10s %20s %20s %15s %12s %10s" + LS; + String dataFmt = "%6d %10d %,20d %,20d %15d %12d %10d" + LS; + long k = (1L << 16) - 1L; + long m = 8; + println("k = " + k + ", m = " + m); + printf(hdrFmt, (Object[]) hdr); + long[] correct = + {65535,43690,29127,19418,12945,8630,5753,3836,2557,1705,1136,758,505,337,224,150,100,67,44,30,20,13,9,6,4,3,2,1,1,1,0}; + for (int i = 0; i < 31; i++) { + long twoK = k << 1; + long twoKxtwoD = twoK << i; + long threeToD = KllHelper.powersOfThree[i]; + long tmp = twoKxtwoD / threeToD; + long result = (tmp + 1L) >>> 1; + long end = Math.max(m, result); //performed later + printf(dataFmt, i, twoK, twoKxtwoD, threeToD, tmp, result, end); + assertEquals(result,correct[i]); + assertEquals(result, KllHelper.intCapAuxAux(k, i)); + } + } + + @Test + public void checkGrowLevels() { + KllLongsSketch sk = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= 21; i++) { sk.update(i); } + assertEquals(sk.getNumLevels(), 2); + assertEquals(sk.getLongItemsArray().length, 33); + assertEquals(sk.getLevelsArray(sk.sketchStructure)[2], 33); + } + + @Test //set static enablePrinting = true for visual checking + public void checkSketchInitializeLongHeap() { + int k = 20; //don't change this + KllLongsSketch sk; + + println("#### CASE: LONG FULL HEAP"); + sk = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= k + 1; i++) { sk.update(i); } + println(sk.toString(true, true)); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), k + 1); + assertEquals(sk.getNumRetained(), 11); + assertFalse(sk.isEmpty()); + assertTrue(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 33); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 3); + assertEquals(sk.getMaxItem(), 21L); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getNumLevels(), 2); + assertFalse(sk.isLevelZeroSorted()); + + println("#### CASE: LONG HEAP EMPTY"); + sk = KllLongsSketch.newHeapInstance(k); + println(sk.toString(true, true)); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 0); + assertEquals(sk.getNumRetained(), 0); + assertTrue(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + try { sk.getMaxItem(); fail(); } catch (SketchesArgumentException e) { } + try { sk.getMinItem(); fail(); } catch (SketchesArgumentException e) { } + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + + println("#### CASE: LONG HEAP SINGLE"); + sk = KllLongsSketch.newHeapInstance(k); + sk.update(1); + println(sk.toString(true, true)); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 1); + assertEquals(sk.getNumRetained(), 1); + assertFalse(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + assertEquals(sk.getMaxItem(), 1L); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + } + + @Test //set static enablePrinting = true for visual checking + public void checkSketchInitializeLongHeapifyCompactMem() { + int k = 20; //don't change this + KllLongsSketch sk; + KllLongsSketch sk2; + byte[] compBytes; + WritableMemory wmem; + + println("#### CASE: LONG FULL HEAPIFIED FROM COMPACT"); + sk2 = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= k + 1; i++) { sk2.update(i); } + //println(sk2.toString(true, true)); + compBytes = sk2.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes); + println(KllPreambleUtil.toString(wmem, LONGS_SKETCH, true)); + sk = KllLongsSketch.heapify(wmem); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), k + 1); + assertEquals(sk.getNumRetained(), 11); + assertFalse(sk.isEmpty()); + assertTrue(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 33); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 3); + assertEquals(sk.getMaxItem(), 21); + assertEquals(sk.getMinItem(), 1); + assertEquals(sk.getNumLevels(), 2); + assertFalse(sk.isLevelZeroSorted()); + + println("#### CASE: LONG EMPTY HEAPIFIED FROM COMPACT"); + sk2 = KllLongsSketch.newHeapInstance(k); + //println(sk.toString(true, true)); + compBytes = sk2.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes); + //println(KllPreambleUtil.toString(wmem, LONGS_SKETCH, true)); + sk = KllLongsSketch.heapify(wmem); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 0); + assertEquals(sk.getNumRetained(), 0); + assertTrue(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + try { sk.getMaxItem(); fail(); } catch (SketchesArgumentException e) { } + try { sk.getMinItem(); fail(); } catch (SketchesArgumentException e) { } + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + + println("#### CASE: LONG SINGLE HEAPIFIED FROM COMPACT"); + sk2 = KllLongsSketch.newHeapInstance(k); + sk2.update(1); + //println(sk2.toString(true, true)); + compBytes = sk2.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes); + //println(KllPreambleUtil.toString(wmem, LONGS_SKETCH, true)); + sk = KllLongsSketch.heapify(wmem); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 1); + assertEquals(sk.getNumRetained(), 1); + assertFalse(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + assertEquals(sk.getMaxItem(), 1L); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + } + + @Test //set static enablePrinting = true for visual checking + public void checkSketchInitializeLongHeapifyUpdatableMem() { + int k = 20; //don't change this + KllLongsSketch sk; + KllLongsSketch sk2; + byte[] compBytes; + WritableMemory wmem; + + println("#### CASE: LONG FULL HEAPIFIED FROM UPDATABLE"); + sk2 = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= k + 1; i++) { sk2.update(i); } + compBytes = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(compBytes); + println(KllPreambleUtil.toString(wmem, LONGS_SKETCH, true)); + sk = KllHeapLongsSketch.heapifyImpl(wmem); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), k + 1); + assertEquals(sk.getNumRetained(), 11); + assertFalse(sk.isEmpty()); + assertTrue(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 33); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 3); + assertEquals(sk.getMaxItem(), 21); + assertEquals(sk.getMinItem(), 1); + assertEquals(sk.getNumLevels(), 2); + assertFalse(sk.isLevelZeroSorted()); + + println("#### CASE: LONG EMPTY HEAPIFIED FROM UPDATABLE"); + sk2 = KllLongsSketch.newHeapInstance(k); + //println(sk.toString(true, true)); + compBytes = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(compBytes); + //println(KllPreambleUtil.toString(wmem, LONGS_SKETCH, true)); + sk = KllHeapLongsSketch.heapifyImpl(wmem); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 0); + assertEquals(sk.getNumRetained(), 0); + assertTrue(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + try { sk.getMaxItem(); fail(); } catch (SketchesArgumentException e) { } + try { sk.getMinItem(); fail(); } catch (SketchesArgumentException e) { } + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + + println("#### CASE: LONG SINGLE HEAPIFIED FROM UPDATABLE"); + sk2 = KllLongsSketch.newHeapInstance(k); + sk2.update(1); + //println(sk.toString(true, true)); + compBytes = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(compBytes); + //println(KllPreambleUtil.toString(wmem, LONGS_SKETCH, true)); + sk = KllHeapLongsSketch.heapifyImpl(wmem); + assertEquals(sk.getK(), k); + assertEquals(sk.getN(), 1); + assertEquals(sk.getNumRetained(), 1); + assertFalse(sk.isEmpty()); + assertFalse(sk.isEstimationMode()); + assertEquals(sk.getMinK(), k); + assertEquals(sk.getLongItemsArray().length, 20); + assertEquals(sk.getLevelsArray(sk.sketchStructure).length, 2); + assertEquals(sk.getMaxItem(), 1L); + assertEquals(sk.getMinItem(), 1L); + assertEquals(sk.getNumLevels(), 1); + assertFalse(sk.isLevelZeroSorted()); + } + + @Test //set static enablePrinting = true for visual checking + public void checkMemoryToStringLongCompact() { + int k = 20; //don't change this + KllLongsSketch sk; + KllLongsSketch sk2; + byte[] compBytes; + byte[] compBytes2; + WritableMemory wmem; + String s; + + println("#### CASE: LONG FULL COMPACT"); + sk = KllLongsSketch.newHeapInstance(k); + for (int i = 1; i <= k + 1; i++) { sk.update(i); } + compBytes = sk.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 1: sketch to byte[]/memory & analyze memory"); + println(s); + sk2 = KllLongsSketch.heapify(wmem); + compBytes2 = sk2.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes2); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 2: memory to heap sketch, to byte[]/memory & analyze memory. Should match above"); + println(s); + assertEquals(compBytes, compBytes2); + + println("#### CASE: LONG EMPTY COMPACT"); + sk = KllLongsSketch.newHeapInstance(k); + compBytes = sk.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 1: sketch to byte[]/memory & analyze memory"); + println(s); + sk2 = KllLongsSketch.heapify(wmem); + compBytes2 = sk2.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes2); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 2: memory to heap sketch, to byte[]/memory & analyze memory. Should match above"); + println(s); + assertEquals(compBytes, compBytes2); + + println("#### CASE: LONG SINGLE COMPACT"); + sk = KllLongsSketch.newHeapInstance(k); + sk.update(1); + compBytes = sk.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 1: sketch to byte[]/memory & analyze memory"); + println(s); + sk2 = KllLongsSketch.heapify(wmem); + compBytes2 = sk2.toByteArray(); + wmem = WritableMemory.writableWrap(compBytes2); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 2: memory to heap sketch, to byte[]/memory & analyze memory. Should match above"); + println(s); + assertEquals(compBytes, compBytes2); + } + + @Test //set static enablePrinting = true for visual checking + public void checkMemoryToStringLongUpdatable() { + int k = 20; //don't change this + KllLongsSketch sk; + KllLongsSketch sk2; + byte[] upBytes; + byte[] upBytes2; + WritableMemory wmem; + String s; + + println("#### CASE: LONG FULL UPDATABLE"); + sk = KllLongsSketch.newHeapInstance(20); + for (int i = 1; i <= k + 1; i++) { sk.update(i); } + upBytes = KllHelper.toByteArray(sk, true); + wmem = WritableMemory.writableWrap(upBytes); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 1: sketch to byte[]/memory & analyze memory"); + println(s); + sk2 = KllHeapLongsSketch.heapifyImpl(wmem); + upBytes2 = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(upBytes2); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 2: memory to heap sketch, to byte[]/memory & analyze memory. Should match above"); + println(s); //note: heapify does not copy free space, while toUpdatableByteArray does + assertEquals(sk.getN(), sk2.getN()); + assertEquals(sk.getMinItem(), sk2.getMinItem()); + assertEquals(sk.getMaxItem(), sk2.getMaxItem()); + assertEquals(sk.getNumRetained(), sk2.getNumRetained()); + + println("#### CASE: LONG EMPTY UPDATABLE"); + sk = KllLongsSketch.newHeapInstance(k); + upBytes = KllHelper.toByteArray(sk, true); + wmem = WritableMemory.writableWrap(upBytes); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 1: sketch to byte[]/memory & analyze memory"); + println(s); + sk2 = KllHeapLongsSketch.heapifyImpl(wmem); + upBytes2 = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(upBytes2); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 2: memory to heap sketch, to byte[]/memory & analyze memory. Should match above"); + println(s); + assertEquals(upBytes, upBytes2); + + println("#### CASE: LONG SINGLE UPDATABLE"); + sk = KllLongsSketch.newHeapInstance(k); + sk.update(1); + upBytes = KllHelper.toByteArray(sk, true); + wmem = WritableMemory.writableWrap(upBytes); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 1: sketch to byte[]/memory & analyze memory"); + println(s); + sk2 = KllHeapLongsSketch.heapifyImpl(wmem); + upBytes2 = KllHelper.toByteArray(sk2, true); + wmem = WritableMemory.writableWrap(upBytes2); + s = KllPreambleUtil.toString(wmem, LONGS_SKETCH, true); + println("step 2: memory to heap sketch, to byte[]/memory & analyze memory. Should match above"); + println(s); + assertEquals(upBytes, upBytes2); + } + + @Test + public void checkSimpleMerge() { + int k = 20; + int m = 8; + int n1 = 21; + int n2 = 43; + WritableMemory wmem = WritableMemory.allocate(3000); + WritableMemory wmem2 = WritableMemory.allocate(3000); + + KllLongsSketch sk1 = KllDirectLongsSketch.newDirectUpdatableInstance(k, m, wmem, memReqSvr); + KllLongsSketch sk2 = KllDirectLongsSketch.newDirectUpdatableInstance(k, m, wmem2, memReqSvr); + for (int i = 1; i <= n1; i++) { + sk1.update(i); + } + for (int i = 1; i <= n2; i++) { + sk2.update(i + 100); + } + sk1.merge(sk2); + assertEquals(sk1.getMinItem(), 1L); + assertEquals(sk1.getMaxItem(), 143L); + } + + @Test + public void checkGetSingleItem() { + int k = 20; + KllLongsSketch skHeap = KllLongsSketch.newHeapInstance(k); + skHeap.update(1); + assertTrue(skHeap instanceof KllHeapLongsSketch); + assertEquals(skHeap.getLongSingleItem(), 1L); + + WritableMemory srcMem = WritableMemory.writableWrap(KllHelper.toByteArray(skHeap, true)); + KllLongsSketch skDirect = KllLongsSketch.writableWrap(srcMem, memReqSvr); + assertTrue(skDirect instanceof KllDirectLongsSketch); + assertEquals(skDirect.getLongSingleItem(), 1L); + + Memory srcMem2 = Memory.wrap(skHeap.toByteArray()); + KllLongsSketch skCompact = KllLongsSketch.wrap(srcMem2); + assertTrue(skCompact instanceof KllDirectCompactLongsSketch); + assertEquals(skCompact.getLongSingleItem(), 1L); + } + + @Test + public void printlnTest() { + String s = "PRINTING: printf in " + this.getClass().getName(); + println(s); + printf("%s" + LS, s); + } + + private final static boolean enablePrinting = false; + + /** + * @param format the format + * @param args the args + */ + private static final void printf(final String format, final Object ... args) { + if (enablePrinting) { System.out.printf(format, args); } + } + + /** + * @param o the Object to println + */ + private static final void println(final Object o) { + if (enablePrinting) { System.out.println(o.toString()); } + } + +}