jiangxin369 commented on code in PR #162:
URL: https://github.com/apache/flink-ml/pull/162#discussion_r1004123592


##########
flink-ml-core/src/main/java/org/apache/flink/ml/util/QuantileSummary.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.flink.ml.util;
+
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * Helper class to compute an approximate quantile summary. This 
implementation is based on the
+ * algorithm proposed in the paper: "Space-efficient Online Computation of 
Quantile Summaries" by
+ * Greenwald, Michael and Khanna, Sanjeev. 
(https://doi.org/10.1145/375663.375670)
+ */
+public class QuantileSummary implements Serializable {
+
+    /** The target relative error. */
+    private double relativeError;
+
+    /**
+     * The compression threshold. After the internal buffer of statistics 
crosses this size, it
+     * attempts to compress the statistics together.
+     */
+    private int compressThreshold;
+
+    /** The count of all the elements inserted to be calculated. */
+    private long count;
+
+    /** A buffer of quantile statistics. */
+    private List<StatsTuple> sampled;
+
+    /** The default size of head buffer. */
+    private static final int DEFAULT_HEAD_SIZE = 10000;
+
+    /** The default compression threshold. */
+    private static final int DEFAULT_COMPRESS_THRESHOLD = 10000;
+
+    /** A buffer of the latest samples seen so far. */
+    private List<Double> headBuffer = new ArrayList<>(DEFAULT_HEAD_SIZE);
+
+    /**
+     * QuantileSummary Constructor.
+     *
+     * @param relativeError The target relative error.
+     */
+    public QuantileSummary(double relativeError) {
+        this(relativeError, DEFAULT_COMPRESS_THRESHOLD);
+    }
+
+    /**
+     * QuantileSummaries Constructor.
+     *
+     * @param relativeError The target relative error.
+     * @param compressThreshold the compression threshold. After the internal 
buffer of statistics
+     *     crosses this size, it attempts to compress the statistics together.
+     */
+    public QuantileSummary(double relativeError, int compressThreshold) {
+        this(relativeError, compressThreshold, Collections.EMPTY_LIST, 0);
+    }
+
+    /**
+     * QuantileSummaries Constructor.
+     *
+     * @param relativeError The target relative error.
+     * @param compressThreshold the compression threshold.
+     * @param sampled A buffer of quantile statistics. See the G-K article for 
more details.
+     * @param count The count of all the elements inserted in the sampled 
buffer.
+     */
+    private QuantileSummary(
+            double relativeError, int compressThreshold, List<StatsTuple> 
sampled, long count) {
+        Preconditions.checkArgument(
+                relativeError > 0 && relativeError < 1,
+                "An appropriate relative error must lay between 0 and 1.");
+        Preconditions.checkArgument(
+                compressThreshold > 0, "An compress threshold must greater 
than 0.");
+        this.relativeError = relativeError;
+        this.compressThreshold = compressThreshold;
+        this.sampled = sampled;
+        this.count = count;
+    }
+
+    /**
+     * Insert a new observation to the summary.
+     *
+     * @param item The new observation to insert into the summary.
+     * @return A summary with the given observation inserted into the summary.
+     */
+    public QuantileSummary insert(Double item) {
+        headBuffer.add(item);
+        if (headBuffer.size() >= DEFAULT_HEAD_SIZE) {
+            QuantileSummary result = insertHeadBuffer();
+            if (result.sampled.size() >= compressThreshold) {
+                return result.compress();
+            } else {
+                return result;
+            }
+        } else {
+            return this;
+        }
+    }
+
+    /**
+     * This implements the COMPRESS function of the GK algorithm.
+     *
+     * @return A new summary that compresses the summary statistics and the 
head buffer.
+     */
+    public QuantileSummary compress() {
+        QuantileSummary inserted = insertHeadBuffer();
+        Preconditions.checkState(inserted.headBuffer.isEmpty());
+        Preconditions.checkState(inserted.count == count + headBuffer.size());
+
+        List<StatsTuple> compressed =
+                compressInternal(inserted.sampled, 2 * relativeError * 
inserted.count);
+        return new QuantileSummary(relativeError, compressThreshold, 
compressed, inserted.count);
+    }
+
+    /**
+     * Merges two compressed summaries together.
+     *
+     * @param other The summary to be merged.
+     * @return The merged summary.
+     */
+    public QuantileSummary merge(QuantileSummary other) {
+        Preconditions.checkState(
+                headBuffer.isEmpty(), "Current buffer needs to be compressed 
before merge");
+        Preconditions.checkState(
+                other.headBuffer.isEmpty(), "Other buffer needs to be 
compressed before merge");
+
+        if (other.count == 0) {
+            return shallowCopy();
+        } else if (count == 0) {
+            return other.shallowCopy();
+        } else {
+            List<StatsTuple> mergedSampled = new ArrayList<>();
+            double mergedRelativeError = Math.max(relativeError, 
other.relativeError);
+            long mergedCount = count + other.count;
+            long additionalSelfDelta =
+                    new Double(Math.floor(2 * other.relativeError * 
other.count)).longValue();
+            long additionalOtherDelta =
+                    new Double(Math.floor(2 * relativeError * 
count)).longValue();
+
+            int selfIdx = 0;
+            int otherIdx = 0;
+            while (selfIdx < sampled.size() && otherIdx < 
other.sampled.size()) {
+                StatsTuple selfSample = sampled.get(selfIdx);
+                StatsTuple otherSample = other.sampled.get(otherIdx);
+                StatsTuple nextSample;
+                long additionalDelta = 0;
+                if (selfSample.value < otherSample.value) {
+                    nextSample = selfSample;
+                    if (otherIdx > 0) {
+                        additionalDelta = additionalSelfDelta;
+                    }
+                    selfIdx++;
+                } else {
+                    nextSample = otherSample;
+                    if (selfIdx > 0) {
+                        additionalDelta = additionalOtherDelta;
+                    }
+                    otherIdx++;
+                }
+                nextSample = nextSample.shallowCopy();
+                nextSample.delta = nextSample.delta + additionalDelta;
+                mergedSampled.add(nextSample);
+            }
+            IntStream.range(selfIdx, sampled.size())
+                    .forEach(i -> mergedSampled.add(sampled.get(i)));
+            IntStream.range(otherIdx, other.sampled.size())
+                    .forEach(i -> mergedSampled.add(other.sampled.get(i)));
+
+            List<StatsTuple> comp =
+                    compressInternal(mergedSampled, 2 * mergedRelativeError * 
mergedCount);
+            return new QuantileSummary(mergedRelativeError, compressThreshold, 
comp, mergedCount);
+        }
+    }
+
+    /**
+     * Runs a query for a given percentile. The query can only be run on a 
compressed summary, you
+     * need to call compress() before using it.
+     *
+     * @param percentile The target percentile.
+     * @return The corresponding approximate quantile.
+     */
+    public double query(double percentile) {
+        double[] result = query(new double[] {percentile});
+        return result == null ? null : result[0];
+    }
+
+    /**
+     * Runs a query for a given sequence of percentiles. The query can only be 
run on a compressed
+     * summary, you need to call compress() before using it.
+     *
+     * @param percentiles A list of the target percentiles.
+     * @return A list of the corresponding approximate quantiles, in the same 
order as the input.
+     */
+    public double[] query(double[] percentiles) {
+        Arrays.stream(percentiles)
+                .forEach(
+                        x ->
+                                Preconditions.checkState(
+                                        x >= 0 && x <= 1.0,
+                                        "percentile should be in the range 
[0.0, 1.0]."));
+        Preconditions.checkState(
+                headBuffer.isEmpty(),
+                "Cannot operate on an uncompressed summary, call compress() 
first");
+        if (sampled == null || sampled.isEmpty()) {
+            return null;
+        }
+        double targetError = Long.MIN_VALUE;
+        for (StatsTuple tuple : sampled) {
+            targetError = Math.max(targetError, (tuple.delta + tuple.g));
+        }
+        targetError = targetError / 2;
+        Map<Double, Integer> zipWithIndex = new HashMap<>(percentiles.length);
+        IntStream.range(0, percentiles.length).forEach(i -> 
zipWithIndex.put(percentiles[i], i));
+
+        int index = 0;
+        long minRank = sampled.get(0).g;
+        double[] sorted = Arrays.stream(percentiles).sorted().toArray();
+        double[] result = new double[percentiles.length];
+
+        for (int i = 0; i < sorted.length; i++) {
+            int percentileIndex = zipWithIndex.get(sorted[i]);
+            if (sorted[i] <= relativeError) {
+                result[percentileIndex] = sampled.get(0).value;
+            } else if (sorted[i] >= 1 - relativeError) {
+                result[percentileIndex] = sampled.get(sampled.size() - 
1).value;
+            } else {
+                QueryResult queryResult =
+                        findApproximateQuantile(index, minRank, targetError, 
sorted[i]);
+                index = queryResult.index;
+                minRank = queryResult.minRankAtIndex;
+                result[percentileIndex] = queryResult.percentile;
+            }
+        }
+        return result;
+    }
+
+    private QuantileSummary insertHeadBuffer() {
+        if (headBuffer.isEmpty()) {
+            return this;
+        }
+
+        long newCount = count;
+        List<StatsTuple> newSamples = new ArrayList<>();
+        List<Double> sorted = 
headBuffer.stream().sorted().collect(Collectors.toList());
+
+        int cursor = 0;
+        for (int i = 0; i < sorted.size(); i++) {
+            while (cursor < sampled.size() && sampled.get(cursor).value <= 
sorted.get(i)) {
+                newSamples.add(sampled.get(cursor));
+                cursor++;
+            }
+
+            long delta = new Double(Math.floor(2.0 * relativeError * 
count)).longValue();
+            if (newSamples.isEmpty() || (cursor == sampled.size() && i == 
sorted.size() - 1)) {
+                delta = 0;
+            }
+            StatsTuple tuple = new StatsTuple(sorted.get(i), 1L, delta);
+            newSamples.add(tuple);
+            newCount++;
+        }
+
+        for (int i = cursor; i < sampled.size(); i++) {
+            newSamples.add(sampled.get(i));
+        }
+        return new QuantileSummary(relativeError, compressThreshold, 
newSamples, newCount);
+    }
+
+    private List<StatsTuple> compressInternal(
+            List<StatsTuple> currentSamples, double mergeThreshold) {
+        if (currentSamples.isEmpty()) {
+            return Collections.emptyList();
+        }
+        LinkedList<StatsTuple> result = new LinkedList<>();
+
+        StatsTuple head = currentSamples.get(currentSamples.size() - 1);
+        for (int i = currentSamples.size() - 2; i >= 1; i--) {
+            StatsTuple tuple = currentSamples.get(i);
+            if (tuple.g + head.g + head.delta < mergeThreshold) {
+                head = head.shallowCopy();
+                head.g = head.g + tuple.g;
+            } else {
+                result.addFirst(head);
+                head = tuple;
+            }
+        }
+        result.addFirst(head);
+
+        StatsTuple currHead = currentSamples.get(0);
+        if (currHead.value <= head.value && currentSamples.size() > 1) {
+            result.addFirst(currHead);
+        }
+        return new ArrayList<>(result);
+    }
+
+    private QueryResult findApproximateQuantile(
+            int index, long minRankAtIndex, double targetError, double 
percentile) {
+        StatsTuple curSample = sampled.get(index);
+        long rank = new Double(Math.ceil(percentile * count)).longValue();
+        long minRank = minRankAtIndex;
+
+        for (int i = index; i < sampled.size() - 1; ) {
+            long maxRank = minRank + curSample.delta;
+            if (maxRank - targetError < rank && rank <= minRank + targetError) 
{
+                return new QueryResult(i, minRank, curSample.value);
+            } else {
+                curSample = sampled.get(++i);
+                minRank += curSample.g;
+            }
+        }
+        return new QueryResult(sampled.size() - 1, 0, 
sampled.get(sampled.size() - 1).value);
+    }
+
+    public double getRelativeError() {
+        return relativeError;
+    }
+
+    public void setRelativeError(double relativeError) {
+        this.relativeError = relativeError;
+    }
+
+    private QuantileSummary shallowCopy() {
+        return new QuantileSummary(relativeError, compressThreshold, sampled, 
count);
+    }
+
+    /** Wrapper class to hold all information returned after querying. */
+    private static class QueryResult {
+        private int index;
+        private long minRankAtIndex;
+        private double percentile;
+
+        public QueryResult(int index, long minRankAtIndex, double percentile) {
+            this.index = index;
+            this.minRankAtIndex = minRankAtIndex;
+            this.percentile = percentile;
+        }
+    }
+
+    /**
+     * Wrapper class to hold all statistics from the Greenwald-Khanna paper. 
It contains the
+     * following information:
+     *
+     * <ul>
+     *   <li>value: the sampled value.
+     *   <li>g: the difference between the least rank of this element and the 
rank of the preceding

Review Comment:
   I prefer to keep it there because it shows that these items are all 
semantics in the GK paper.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to