[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16569158#comment-16569158 ]
ASF GitHub Bot commented on CASSANDRA-14436: -------------------------------------------- Github user clohfink commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207707285 --- Diff: src/java/org/apache/cassandra/metrics/FrequencySampler.java --- @@ -0,0 +1,105 @@ +/* + * 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.cassandra.metrics; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.clearspring.analytics.stream.StreamSummary; + +/** + * Find the most frequent sample. A sample adds to the sum of its key ie + * <p>add("x", 10); and add("x", 20); will result in "x" = 30</p> This uses StreamSummary to only store the + * approximate cardinality (capacity) of keys. If the number of distinct keys exceed the capacity, the error of the + * sample may increase depending on distribution of keys among the total set. + * + * @param <T> + */ +public abstract class FrequencySampler<T> extends Sampler<T> +{ + private static final Logger logger = LoggerFactory.getLogger(FrequencySampler.class); + private boolean enabled = false; + + private StreamSummary<T> summary; + + /** + * Start to record samples + * + * @param capacity + * Number of sample items to keep in memory, the lower this is + * the less accurate results are. For best results use value + * close to cardinality, but understand the memory trade offs. + */ + public synchronized void beginSampling(int capacity) + { + if (!enabled) + { + summary = new StreamSummary<T>(capacity); + enabled = true; + } + } + + /** + * Call to stop collecting samples, and gather the results + * @param count Number of most frequent items to return + */ + public synchronized List<Sample<T>> finishSampling(int count) + { + List<Sample<T>> results = Collections.EMPTY_LIST; + if (enabled) + { + enabled = false; + results = summary.topK(count) + .stream() + .map(c -> new Sample<T>(c.getItem(), c.getCount(), c.getError())) + .collect(Collectors.toList()); + } + return results; + } + + protected synchronized void insert(final T item, final long value) + { + // samplerExecutor is single threaded but still need + // synchronization against jmx calls to finishSampling + if (enabled && value > 0) + { + try + { + summary.offer(item, (int) Math.min(value, Integer.MAX_VALUE)); + } catch (Exception e) + { + logger.trace("Failure to offer sample", e); + } + } + } + + public boolean isEnabled() + { + return enabled; + } + + public void setEnabled(boolean enabled) + { + this.enabled = enabled; --- End diff -- That method was unnecessary so I just deleted it > Add sampler for query time and expose with nodetool > --------------------------------------------------- > > Key: CASSANDRA-14436 > URL: https://issues.apache.org/jira/browse/CASSANDRA-14436 > Project: Cassandra > Issue Type: Improvement > Reporter: Chris Lohfink > Assignee: Chris Lohfink > Priority: Major > > Create a new {{nodetool profileload}} that functions just like toppartitions > but with more data, returning the slowest local reads and writes on the host > during a given duration and highest frequency touched partitions (same as > {{nodetool toppartitions}}). Refactor included to extend use of the sampler > for uses outside of top frequency (max instead of total sample values). > Future work to this is to include top cpu and allocations by query and > possibly tasks/cpu/allocations by stage during time window. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org