zhipeng93 commented on code in PR #162: URL: https://github.com/apache/flink-ml/pull/162#discussion_r1007572419
########## flink-ml-lib/src/main/java/org/apache/flink/ml/common/util/QuantileSummary.java: ########## @@ -0,0 +1,400 @@ +/* + * 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.common.util; + +import org.apache.flink.annotation.Internal; +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) + */ +@Internal +public class QuantileSummary implements Serializable { + + /** The target relative error. */ + private final double relativeError; + + /** + * The compression threshold. After the internal buffer of statistics crosses this size, it + * attempts to compress the statistics together. + */ + private final int compressThreshold; + + /** The count of all the elements inserted to be calculated. */ + private final long count; + + /** A buffer of quantile statistics. */ + private final List<StatsTuple> sampled; + + /** The default size of head buffer. */ + private static final int DEFAULT_HEAD_SIZE = 50000; + + /** 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); Review Comment: nit: Do you think making it a `double[]` could be more memory efficient? I am also fine the keep the current implementation. ########## flink-ml-lib/src/main/java/org/apache/flink/ml/common/util/QuantileSummary.java: ########## @@ -0,0 +1,400 @@ +/* + * 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.common.util; + +import org.apache.flink.annotation.Internal; +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) + */ +@Internal +public class QuantileSummary implements Serializable { + + /** The target relative error. */ + private final double relativeError; + + /** + * The compression threshold. After the internal buffer of statistics crosses this size, it + * attempts to compress the statistics together. + */ + private final int compressThreshold; + + /** The count of all the elements inserted to be calculated. */ + private final long count; + + /** A buffer of quantile statistics. */ + private final List<StatsTuple> sampled; + + /** The default size of head buffer. */ + private static final int DEFAULT_HEAD_SIZE = 50000; + + /** 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); + } + + /** + * QuantileSummary 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); + } + + /** + * QuantileSummary 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; + } + } + + /** + * Returns a new summary that compresses the summary statistics and the head buffer. + * + * <p>This implements the COMPRESS function of the GK algorithm. + * + * @return The compressed summary. + */ + 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 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) { + return query(new double[] {percentile})[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."); + Preconditions.checkState( + sampled != null && !sampled.isEmpty(), + "Cannot query percentiles without any records inserted."); + 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; + } + + /** + * Check whether the QuantileSummary has inserted rows. Running query on an empty Review Comment: nit: `Checks` ########## flink-ml-lib/src/main/java/org/apache/flink/ml/common/util/QuantileSummary.java: ########## @@ -0,0 +1,400 @@ +/* + * 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.common.util; + +import org.apache.flink.annotation.Internal; +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) + */ +@Internal +public class QuantileSummary implements Serializable { + + /** The target relative error. */ + private final double relativeError; + + /** + * The compression threshold. After the internal buffer of statistics crosses this size, it + * attempts to compress the statistics together. + */ + private final int compressThreshold; + + /** The count of all the elements inserted to be calculated. */ + private final long count; + + /** A buffer of quantile statistics. */ + private final List<StatsTuple> sampled; + + /** The default size of head buffer. */ + private static final int DEFAULT_HEAD_SIZE = 50000; + + /** 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); + } + + /** + * QuantileSummary 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); + } + + /** + * QuantileSummary 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. Review Comment: nit: `Inserts a new observation into the summary` -- 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]
