You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by clohfink <gi...@git.apache.org> on 2018/08/03 02:42:03 UTC
[GitHub] cassandra pull request #244: Refactor and add samplers for 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 <cl...@...>
Date: 2018-05-19T04:30:20Z
Refactor and add samplers for CASSANDRA-14436
----
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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<Sampler, TopKSampler<ByteBuffer>> samplers;
+ public final Map<SamplerType, Sampler<?>> samplers;
--- End diff --
This can be replaced with an `EnumMap`.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
Github user dineshjoshi commented on a diff in the pull request:
https://github.com/apache/cassandra/pull/244#discussion_r210487768
--- Diff: src/java/org/apache/cassandra/db/ReadExecutionController.java ---
@@ -132,6 +145,15 @@ public void close()
{
if (baseOp != null)
baseOp.close();
+
+ if (startTimeNanos != -1)
+ {
+ String cql = command.toCQLString();
+ int timeMicros = (int) Math.min(TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - startTimeNanos), Integer.MAX_VALUE);
--- End diff --
This should be `o.a.c.u.Clock`
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra issue #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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<T>
+{
+ 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<Runnable>(),
+ 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<Sample<T>> finishSampling(int count);
+
+ public abstract String toString(T value);
+
+ /**
+ * Represents the ranked items collected during a sample period
+ */
+ public static class Sample<S> implements Serializable
+ {
+
--- End diff --
Extra white space?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra issue #244: Refactor and add samplers for CASSANDRA-14436
Posted by clohfink <gi...@git.apache.org>.
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.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
Github user dineshjoshi commented on a diff in the pull request:
https://github.com/apache/cassandra/pull/244#discussion_r210487821
--- Diff: src/java/org/apache/cassandra/db/ReadExecutionController.java ---
@@ -90,11 +100,11 @@ static ReadExecutionController forCommand(ReadCommand command)
try
{
baseOp = baseCfs.readOrdering.start();
- indexController = new ReadExecutionController(indexCfs.readOrdering.start(), indexCfs.metadata(), null, null);
+ indexController = new ReadExecutionController(command, indexCfs.readOrdering.start(), indexCfs.metadata(), null, null);
// TODO: this should perhaps not open and maintain a writeOp for the full duration, but instead only *try* to delete stale entries, without blocking if there's no room
// as it stands, we open a writeOp and keep it open for the duration to ensure that should this CF get flushed to make room we don't block the reclamation of any room being made
writeContext = baseCfs.keyspace.getWriteHandler().createContextForRead();
- return new ReadExecutionController(baseOp, baseCfs.metadata(), indexController, writeContext);
+ result = new ReadExecutionController(command, baseOp, baseCfs.metadata(), indexController, writeContext);
--- End diff --
Nit: Extra space.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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
+ * <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 --
This allows the user of the class to enable the `FrequencySampler` without actually initializing the `summary` variable. This will cause an NPE.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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 = "<keyspace> <cfname> <duration>", description = "The keyspace, column family name, and duration in milliseconds")
+ private List<String> 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 = 10000;
+ 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<String> targets = Lists.newArrayList();
+ List<String> 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<String, List<CompositeData>> results;
+ try
+ {
+ if (keyspace == null)
+ {
--- End diff --
Braces are unnecessary.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by clohfink <gi...@git.apache.org>.
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
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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
+ * <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;
--- End diff --
`Collections.emptyList()` is safer.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
Github user dineshjoshi commented on a diff in the pull request:
https://github.com/apache/cassandra/pull/244#discussion_r210488658
--- Diff: src/java/org/apache/cassandra/metrics/MaxSampler.java ---
@@ -0,0 +1,59 @@
+package org.apache.cassandra.metrics;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import com.google.common.collect.MinMaxPriorityQueue;
+
+public abstract class MaxSampler<T> extends Sampler<T>
+{
+ private int capacity;
+ private MinMaxPriorityQueue<Sample<T>> queue;
+ private long endTimeMillis = -1;
+ private final Comparator<Sample<T>> comp = Collections.reverseOrder(Comparator.comparing(p -> p.count));
+
+ public boolean isEnabled()
+ {
+ return endTimeMillis != -1 && clock.currentTimeMillis() <= endTimeMillis;
+ }
+
--- End diff --
Nit: Extra space.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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`.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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<T>
--- End diff --
It would be nice to have a jmh benchmark for the new `Sampler`s
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org
[GitHub] cassandra pull request #244: Refactor and add samplers for CASSANDRA-14436
Posted by dineshjoshi <gi...@git.apache.org>.
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 = "<keyspace> <cfname> <duration>", description = "The keyspace, column family name, and duration in milliseconds")
+ private List<String> 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 = 10000;
--- End diff --
What is the unit for `duration`? It might be better to just use `java.time.Duration`?
---
---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org