[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16603348#comment-16603348 ] Dinesh Joshi commented on CASSANDRA-14436: -- Thank you for committing this, [~iamaleksey]! > 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 > Labels: 4.0-feature-freeze-review-requested, > pull-request-available > Fix For: 4.0 > > Time Spent: 0.5h > Remaining Estimate: 0h > > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16599037#comment-16599037 ] Aleksey Yeschenko commented on CASSANDRA-14436: --- Committed as [f31d1a05a1f6f85f64c9b965009db814960c4eca|https://github.com/apache/cassandra/commit/f31d1a05a1f6f85f64c9b965009db814960c4eca] to trunk. Mostly just looked at potential negative effects on the read path and found none, but cleaned up {{ReadExecutionController}} a little in the process. I trust Chris and Dinesh to have collectively done a good job at implementation and review of the rest. > 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 > Labels: 4.0-feature-freeze-review-requested, > pull-request-available > Fix For: 4.0 > > Time Spent: 0.5h > Remaining Estimate: 0h > > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16597722#comment-16597722 ] Jeff Jirsa commented on CASSANDRA-14436: Asked for second review from [~iamaleksey] because this touches some internals (notably the changes to the {{ReadExecutionController}} ) where someone with a lot of familiarity with that code makes sense - also anything in the hot path really deserves two reviewers at this point in the game. > 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 > Labels: pull-request-available > Fix For: 4.0.x > > Time Spent: 0.5h > Remaining Estimate: 0h > > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16584137#comment-16584137 ] Chris Lohfink commented on CASSANDRA-14436: --- Pushed requested changes. Having some issues with circleci and dtests Ill ask for some help with though |[units|https://circleci.com/gh/clohfink/cassandra/298]| > 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 > Labels: pull-request-available > Time Spent: 0.5h > Remaining Estimate: 0h > > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16582036#comment-16582036 ] Dinesh Joshi commented on CASSANDRA-14436: -- Hey [~cnlwsu], I have left a few more comments. I think other than those, this looks good. > 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 > Labels: pull-request-available > Time Spent: 0.5h > Remaining Estimate: 0h > > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16571162#comment-16571162 ] Chris Lohfink commented on CASSANDRA-14436: --- While I think we can do something like creating a concurrent set of Samplers for each SamplerType that we tie to a Sampler session and flag it to start at same time I dont think its necessary. The current use of top partitions has never had a reported issue with people trying to concurrently run profiling sessions so it can be a new feature to add in another ticket at sometime but I dont think its needed enough here. In meantime I added a strict restriction on a single at a time, raising an exception if someone tries to kick off a 2nd one. Also the sampling will timeout at the end of the duration so if the finish is never called it wont spin forever. I did write some basic jmh benchmarks but i didnt want to make insert() accessible and the {{.*microbench.*}} in build.xml makes default visibility not an option so... yeah. Ultimately (when on) its just ThreadExecuterPool.submit() on the addSample in read/write path which is pretty straight forward limitation on the contention on the queue but i saw 100-300nanosecond -ish. Going into the actual guts, the frequency sampler being a wrapper around the addthis StreamSummary - which there might be something better out there now but its seemed to do fine so far. In some worst case JMH benchmarks I was able to see this hit 3us or so, which could conceivably underperform writes which would cause a backup. The MaxSampler uses MinMaxPriorityQueue, which after PriorityQueue(comparator) becomes available (post java8) that can be replaced to be more performant, but that rarely breaks a microsecond even with top 1024. Just incase as a catchall I added the same as the trace executor - a throwaway loadshedding incase the sampler executor does get backed up. This includes some plumbing so its reported appropriately in metrics. > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 + * add("x", 10); and add("x", 20); will result in "x" = 30 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 + */ +public abstract class FrequencySampler extends Sampler +{ +private static final Logger logger = LoggerFactory.getLogger(FrequencySampler.class); +private boolean enabled = false; + +private StreamSummary 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(capacity); +enabled = true; +} +} + +/** + * Call to stop collecting samples, and gather the results + * @param count Number of most frequent items to return + */ +public synchronized List> finishSampling(int count) +{ +List> results = Collections.EMPTY_LIST; +if (enabled) +{ +enabled = false; +results = summary.topK(count) + .stream() + .map(c -> new Sample(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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569050#comment-16569050 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user clohfink commented on the issue: https://github.com/apache/cassandra/pull/244 I like idea of countdown latch, ill give that a shot. > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569021#comment-16569021 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on the issue: https://github.com/apache/cassandra/pull/244 Stepping back a bit, I see the samplers are stateful classes that are enabled and disabled. This means, if there is an exception in the rmi thread that is executing the `beginLocalSampling` and `finishLocalSampling`, the samplers will continue to run indefinitely and this might cause issues. It would be best to instantiate Samplers on demand with a specific Duration. Each sampler can stop accepting new samples once the duration expires. This would also mean that you no longer have to keep enabling disabling samplers - allowing you to get rid of `enabled` and other internal state variables could be made immutable for example in `FrequencySampler`, `StreamSummary` can be declared as `final` and initialized in the constructor. If you want all samplers to start sampling exactly at the same moment (not sure if that is a requirement) then you could potentially use a shared countdown latch. The thread instantiating the samplers can decrement it once it is done creating and initializing all samplers. WDYT? > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569010#comment-16569010 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207697381 --- 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 + * add("x", 10); and add("x", 20); will result in "x" = 30 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 + */ +public abstract class FrequencySampler extends Sampler +{ +private static final Logger logger = LoggerFactory.getLogger(FrequencySampler.class); +private boolean enabled = false; + +private StreamSummary 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(capacity); +enabled = true; +} +} + +/** + * Call to stop collecting samples, and gather the results + * @param count Number of most frequent items to return + */ +public synchronized List> finishSampling(int count) +{ +List> results = Collections.EMPTY_LIST; --- End diff -- `Collections.emptyList()` is safer. > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569002#comment-16569002 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207697017 --- Diff: src/java/org/apache/cassandra/metrics/TableMetrics.java --- @@ -281,7 +301,7 @@ public Long getValue() public final Meter readRepairRequests; public final Meter shortReadProtectionRequests; -public final Map> samplers; +public final Map> samplers; --- End diff -- This can be replaced with an `EnumMap`. > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569012#comment-16569012 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207697430 --- 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 + * add("x", 10); and add("x", 20); will result in "x" = 30 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 + */ +public abstract class FrequencySampler extends Sampler +{ +private static final Logger logger = LoggerFactory.getLogger(FrequencySampler.class); +private boolean enabled = false; + +private StreamSummary 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(capacity); +enabled = true; +} +} + +/** + * Call to stop collecting samples, and gather the results + * @param count Number of most frequent items to return + */ +public synchronized List> finishSampling(int count) +{ +List> results = Collections.EMPTY_LIST; +if (enabled) +{ +enabled = false; +results = summary.topK(count) + .stream() + .map(c -> new Sample(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 -- This allows the user of the class to enable the `FrequencySampler` without actually initializing the `summary` variable. This will cause an NPE. > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569004#comment-16569004 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207697244 --- Diff: src/java/org/apache/cassandra/db/ReadExecutionController.java --- @@ -113,6 +123,11 @@ static ReadExecutionController forCommand(ReadCommand command) throw e; } } +if (baseCfs.metric.topLocalReadQueryTime.isEnabled()) +{ +result.startTime = System.nanoTime(); --- End diff -- I prefer naming primitive variables with units for example - `startTimeNanos`. > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569007#comment-16569007 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207697297 --- Diff: src/java/org/apache/cassandra/db/ReadExecutionController.java --- @@ -132,6 +147,17 @@ public void close() { if (baseOp != null) baseOp.close(); + +if (startTime != -1) +{ +String cql = command.toCQLString(); +int time = (int) Math.min(TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - startTime), Integer.MAX_VALUE); +ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(baseMetadata.id); +if(cfs != null) +{ --- End diff -- You can skip braces for single line if conditions. > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569009#comment-16569009 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207697707 --- Diff: src/java/org/apache/cassandra/metrics/Sampler.java --- @@ -0,0 +1,67 @@ +package org.apache.cassandra.metrics; + +import java.io.Serializable; +import java.util.List; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor; +import org.apache.cassandra.concurrent.NamedThreadFactory; + +import com.google.common.annotations.VisibleForTesting; + +public abstract class Sampler --- End diff -- It would be nice to have a jmh benchmark for the new `Sampler`s > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569011#comment-16569011 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207697637 --- Diff: src/java/org/apache/cassandra/tools/nodetool/ProfileLoad.java --- @@ -0,0 +1,178 @@ +package org.apache.cassandra.tools.nodetool; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.commons.lang3.StringUtils.join; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +import javax.management.openmbean.CompositeData; +import javax.management.openmbean.OpenDataException; + +import org.apache.cassandra.metrics.Sampler.SamplerType; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.NodeTool.NodeToolCmd; +import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; +import org.apache.cassandra.utils.Pair; + +import com.google.common.collect.Lists; + +import io.airlift.airline.Arguments; +import io.airlift.airline.Command; +import io.airlift.airline.Option; + +@Command(name = "profileload", description = "Low footprint profiling of activity for a period of time") +public class ProfileLoad extends NodeToolCmd +{ +@Arguments(usage = " ", description = "The keyspace, column family name, and duration in milliseconds") +private List args = new ArrayList<>(); + +@Option(name = "-s", description = "Capacity of the sampler, higher for more accuracy (Default: 256)") +private int capacity = 256; + +@Option(name = "-k", description = "Number of the top samples to list (Default: 10)") +private int topCount = 10; + +@Option(name = "-a", description = "Comma separated list of samplers to use (Default: all)") +private String samplers = join(SamplerType.values(), ','); + +@Override +public void execute(NodeProbe probe) +{ +checkArgument(args.size() == 3 || args.size() == 1 || args.size() == 0, "Invalid arguments, either [keyspace table duration] or [duration] or no args"); +checkArgument(topCount < capacity, "TopK count (-k) option must be smaller then the summary capacity (-s)"); +String keyspace = null; +String table = null; +Integer duration = 1; --- End diff -- What is the unit for `duration`? It might be better to just use `java.time.Duration`? > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569008#comment-16569008 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207697487 --- Diff: src/java/org/apache/cassandra/metrics/Sampler.java --- @@ -0,0 +1,67 @@ +package org.apache.cassandra.metrics; + +import java.io.Serializable; +import java.util.List; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor; +import org.apache.cassandra.concurrent.NamedThreadFactory; + +import com.google.common.annotations.VisibleForTesting; + +public abstract class Sampler +{ +public enum SamplerType +{ +READS, WRITES, LOCAL_READ_TIME, WRITE_SIZE, CAS_CONTENTIONS +} + +@VisibleForTesting +static final ThreadPoolExecutor samplerExecutor = new JMXEnabledThreadPoolExecutor(1, 1, +TimeUnit.SECONDS, +new LinkedBlockingQueue(), +new NamedThreadFactory("Sampler"), +"internal"); + +public void addSample(final T item, final int value) +{ +if (isEnabled()) +samplerExecutor.execute(() -> insert(item, value)); +} + +protected abstract void insert(T item, long value); + +public abstract boolean isEnabled(); + +public abstract void beginSampling(int capacity); + +public abstract List> finishSampling(int count); + +public abstract String toString(T value); + +/** + * Represents the ranked items collected during a sample period + */ +public static class Sample implements Serializable +{ + --- End diff -- Extra white space? > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569006#comment-16569006 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207697651 --- Diff: src/java/org/apache/cassandra/tools/nodetool/ProfileLoad.java --- @@ -0,0 +1,178 @@ +package org.apache.cassandra.tools.nodetool; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.commons.lang3.StringUtils.join; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +import javax.management.openmbean.CompositeData; +import javax.management.openmbean.OpenDataException; + +import org.apache.cassandra.metrics.Sampler.SamplerType; +import org.apache.cassandra.tools.NodeProbe; +import org.apache.cassandra.tools.NodeTool.NodeToolCmd; +import org.apache.cassandra.tools.nodetool.formatter.TableBuilder; +import org.apache.cassandra.utils.Pair; + +import com.google.common.collect.Lists; + +import io.airlift.airline.Arguments; +import io.airlift.airline.Command; +import io.airlift.airline.Option; + +@Command(name = "profileload", description = "Low footprint profiling of activity for a period of time") +public class ProfileLoad extends NodeToolCmd +{ +@Arguments(usage = " ", description = "The keyspace, column family name, and duration in milliseconds") +private List args = new ArrayList<>(); + +@Option(name = "-s", description = "Capacity of the sampler, higher for more accuracy (Default: 256)") +private int capacity = 256; + +@Option(name = "-k", description = "Number of the top samples to list (Default: 10)") +private int topCount = 10; + +@Option(name = "-a", description = "Comma separated list of samplers to use (Default: all)") +private String samplers = join(SamplerType.values(), ','); + +@Override +public void execute(NodeProbe probe) +{ +checkArgument(args.size() == 3 || args.size() == 1 || args.size() == 0, "Invalid arguments, either [keyspace table duration] or [duration] or no args"); +checkArgument(topCount < capacity, "TopK count (-k) option must be smaller then the summary capacity (-s)"); +String keyspace = null; +String table = null; +Integer duration = 1; +if(args.size() == 3) +{ +keyspace = args.get(0); +table = args.get(1); +duration = Integer.valueOf(args.get(2)); +} +else if (args.size() == 1) +{ +duration = Integer.valueOf(args.get(0)); +} +// generate the list of samplers +List targets = Lists.newArrayList(); +List available = Arrays.stream(SamplerType.values()).map(Enum::toString).collect(Collectors.toList()); +for (String s : samplers.split(",")) +{ +String sampler = s.trim().toUpperCase(); +checkArgument(available.contains(sampler), String.format("'%s' sampler is not available from: %s", s, Arrays.toString(SamplerType.values(; +targets.add(sampler); +} + +Map> results; +try +{ +if (keyspace == null) +{ --- End diff -- Braces are unnecessary. > 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,
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569005#comment-16569005 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207697287 --- Diff: src/java/org/apache/cassandra/db/ReadExecutionController.java --- @@ -132,6 +147,17 @@ public void close() { if (baseOp != null) baseOp.close(); + +if (startTime != -1) +{ +String cql = command.toCQLString(); +int time = (int) Math.min(TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - startTime), Integer.MAX_VALUE); --- End diff -- `timeMillis` or `millis` for brevity? > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16569003#comment-16569003 ] ASF GitHub Bot commented on CASSANDRA-14436: Github user dineshjoshi commented on a diff in the pull request: https://github.com/apache/cassandra/pull/244#discussion_r207697217 --- Diff: src/java/org/apache/cassandra/db/ReadExecutionController.java --- @@ -113,6 +123,11 @@ static ReadExecutionController forCommand(ReadCommand command) throw e; } } +if (baseCfs.metric.topLocalReadQueryTime.isEnabled()) +{ --- End diff -- Single line if conditions don't need braces. > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16567696#comment-16567696 ] ASF GitHub Bot commented on CASSANDRA-14436: GitHub user clohfink opened a pull request: https://github.com/apache/cassandra/pull/244 Refactor and add samplers for CASSANDRA-14436 You can merge this pull request into a Git repository by running: $ git pull https://github.com/clohfink/cassandra 14436-trunk Alternatively you can review and apply these changes as the patch at: https://github.com/apache/cassandra/pull/244.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #244 commit cf1f26e985a2aa2eb6a683457bb6cd755d773d7c Author: Chris Lohfink Date: 2018-05-19T04:30:20Z Refactor and add samplers for CASSANDRA-14436 > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16481455#comment-16481455 ] Chris Lohfink commented on CASSANDRA-14436: --- ||branch||tests|| | [trunk|https://github.com/clohfink/cassandra/tree/14436-trunk]|[unit|https://circleci.com/gh/clohfink/cassandra/135]| > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16474788#comment-16474788 ] Chris Lohfink commented on CASSANDRA-14436: --- new samplers: cas contentiion, read query, mutation size > 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
[jira] [Commented] (CASSANDRA-14436) Add sampler for query time and expose with nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-14436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16474787#comment-16474787 ] Chris Lohfink commented on CASSANDRA-14436: --- Example output: {code:java} #> nodetool profileload Frequency of reads by partition: Table Partition Count +/- basic.wide row1 75424 0 basic.cas p1 656 0 system.paxos 7031 550 0 system.local local 2 0 Frequency of writes by partition: Table Partition Count +/- system.paxos 7031 585 0 basic.cas p1 112 0 basic.wide row4864 20 19 basic.wide row4870 20 19 basic.wide row4868 20 19 basic.wide row4871 20 19 basic.wide row4869 20 19 basic.wide row4866 20 19 basic.wide row4865 20 19 basic.wide row4867 20 19 Frequency of cas contentions by partition: Table Partition Count +/- basic.cas p1 76 0 Max mutation size by partition: Table Partition Bytes basic.wide row0 1056 basic.wide row7 1056 basic.wide row11 1056 basic.wide row59 1056 basic.wide row255 1056 basic.wide row254 1056 basic.wide row253 1056 basic.wide row251 1056 basic.wide row252 1056 basic.wide row250 1056 Longest read query times: Query Microseconds SELECT * FROM basic.wide WHERE key = row1 LIMIT 5000 25681 SELECT * FROM basic.wide WHERE key = row1 LIMIT 5000 16131 SELECT * FROM basic.wide WHERE key = row1 LIMIT 5000 14715 SELECT * FROM basic.wide WHERE key = row1 LIMIT 5000 14563 SELECT * FROM basic.wide WHERE key = row1 LIMIT 5000 10497 SELECT * FROM basic.wide WHERE key = row1 LIMIT 5000 10489 SELECT * FROM system_schema.columns 2784 SELECT * FROM system_schema.columns 2285 SELECT * FROM system_schema.tables 1553 SELECT * FROM system_schema.tables 1275 {code} > 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