Skip to content

Commit

Permalink
Add option to print level with getsstables output
Browse files Browse the repository at this point in the history
Patch by Jordan West; reviewed by Brandon Williams and Cheng Wang for CASSANDRA-18023
  • Loading branch information
jrwest committed Dec 3, 2022
1 parent 797b969 commit 279f284
Show file tree
Hide file tree
Showing 6 changed files with 84 additions and 6 deletions.
1 change: 1 addition & 0 deletions CHANGES.txt
@@ -1,4 +1,5 @@
4.2
* Add option to print level in nodetool getsstables output (CASSANDRA-18023)
* Implement a guardrail for not having zero default_time_to_live on tables with TWCS (CASSANDRA-18042)
* Add CQL scalar functions for collection aggregation (CASSANDRA-18060)
* Make cassandra.replayList property for CommitLogReplayer possible to react on keyspaces only (CASSANDRA-18044)
Expand Down
30 changes: 27 additions & 3 deletions src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Expand Up @@ -64,10 +64,12 @@
import com.google.common.base.Predicates;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Multimap;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.RateLimiter;
import org.slf4j.Logger;
Expand Down Expand Up @@ -167,6 +169,7 @@
import org.apache.cassandra.utils.JVMStabilityInspector;
import org.apache.cassandra.utils.MBeanWrapper;
import org.apache.cassandra.utils.NoSpamLogger;
import org.apache.cassandra.utils.Pair;
import org.apache.cassandra.utils.TimeUUID;
import org.apache.cassandra.utils.WrappedRunnable;
import org.apache.cassandra.utils.concurrent.CountDownLatch;
Expand Down Expand Up @@ -1939,19 +1942,34 @@ public List<String> getSSTablesForKey(String key)
}

public List<String> getSSTablesForKey(String key, boolean hexFormat)
{
return withSSTablesForKey(key, hexFormat, SSTableReader::getFilename);
}

public Map<Integer, Collection<String>> getSSTablesForKeyWithLevel(String key, boolean hexFormat)
{
List<Pair<Integer, String>> ssts = withSSTablesForKey(key, hexFormat, sstr -> Pair.create(sstr.getSSTableLevel(), sstr.getFilename()));
Multimap<Integer, String> result = HashMultimap.create();
for (Pair<Integer, String> sst : ssts)
result.put(sst.left, sst.right);

return result.asMap();
}

public <T> List<T> withSSTablesForKey(String key, boolean hexFormat, Function<SSTableReader, T> mapper)
{
ByteBuffer keyBuffer = hexFormat ? ByteBufferUtil.hexToBytes(key) : metadata().partitionKeyType.fromString(key);
DecoratedKey dk = decorateKey(keyBuffer);
try (OpOrder.Group op = readOrdering.start())
{
List<String> files = new ArrayList<>();
List<T> mapped = new ArrayList<>();
for (SSTableReader sstr : select(View.select(SSTableSet.LIVE, dk)).sstables)
{
// check if the key actually exists in this sstable, without updating cache and stats
if (sstr.getPosition(dk, SSTableReader.Operator.EQ, false) != null)
files.add(sstr.getFilename());
mapped.add(mapper.apply(sstr));
}
return files;
return mapped;
}
}

Expand Down Expand Up @@ -3061,6 +3079,12 @@ public long[] getPerLevelSizeBytes()
return compactionStrategyManager.getPerLevelSizeBytes();
}

@Override
public boolean isLeveledCompaction()
{
return compactionStrategyManager.isLeveledCompaction();
}

@Override
public int[] getSSTableCountPerTWCSBucket()
{
Expand Down
14 changes: 14 additions & 0 deletions src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
Expand Up @@ -161,6 +161,15 @@ public interface ColumnFamilyStoreMBean
*/
public List<String> getSSTablesForKey(String key, boolean hexFormat);

/**
* Returns a list of filenames that contain the given key and which level they belong to.
* Requires table to be compacted with {@link org.apache.cassandra.db.compaction.LeveledCompactionStrategy}
* @param key
* @param hexFormat
* @return list of filenames and levels containing the key
*/
public Map<Integer, Collection<String>> getSSTablesForKeyWithLevel(String key, boolean hexFormat);

/**
* Load new sstables from the given directory
*
Expand Down Expand Up @@ -225,6 +234,11 @@ public List<String> importNewSSTables(Set<String> srcPaths,
*/
public long[] getPerLevelSizeBytes();

/**
* @return true if the table is using LeveledCompactionStrategy. false otherwise.
*/
public boolean isLeveledCompaction();

/**
* @return sstable count for each bucket in TWCS. null unless time window compaction is used.
* array index corresponds to bucket(int[0] is for most recent, ...).
Expand Down
Expand Up @@ -617,6 +617,18 @@ public long[] getPerLevelSizeBytes()
}
}

public boolean isLeveledCompaction()
{
readLock.lock();
try
{
return repaired.first() instanceof LeveledCompactionStrategy;
} finally
{
readLock.unlock();
}
}

public int[] getSSTableCountPerTWCSBucket()
{
readLock.lock();
Expand Down
13 changes: 13 additions & 0 deletions src/java/org/apache/cassandra/tools/NodeProbe.java
Expand Up @@ -30,6 +30,7 @@
import java.rmi.server.RMISocketFactory;
import java.util.AbstractMap;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
Expand Down Expand Up @@ -1044,6 +1045,18 @@ public List<String> getSSTables(String keyspace, String cf, String key, boolean
return cfsProxy.getSSTablesForKey(key, hexFormat);
}

public Map<Integer, Collection<String>> getSSTablesWithLevel(String keyspace, String cf, String key, boolean hexFormat)
{
ColumnFamilyStoreMBean cfsProxy = getCfsProxy(keyspace, cf);
return cfsProxy.getSSTablesForKeyWithLevel(key, hexFormat);
}

public boolean isLeveledCompaction(String keyspace, String cf)
{
ColumnFamilyStoreMBean cfsProxy = getCfsProxy(keyspace, cf);
return cfsProxy.isLeveledCompaction();
}

public Set<StreamState> getStreamStatus()
{
return Sets.newHashSet(Iterables.transform(streamProxy.getCurrentStreams(), new Function<CompositeData, StreamState>()
Expand Down
20 changes: 17 additions & 3 deletions src/java/org/apache/cassandra/tools/nodetool/GetSSTables.java
Expand Up @@ -22,7 +22,9 @@
import io.airlift.airline.Command;

import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;

import io.airlift.airline.Option;
import org.apache.cassandra.tools.NodeProbe;
Expand All @@ -36,6 +38,9 @@ public class GetSSTables extends NodeToolCmd
description = "Specify the key in hexadecimal string format")
private boolean hexFormat = false;

@Option(name={"-l", "--show-levels"}, description="If the table is using leveled compaction the level of each sstable will be included in the output (Default: false)")
private boolean showLevels = false;

@Arguments(usage = "<keyspace> <cfname> <key>", description = "The keyspace, the column family, and the key")
private List<String> args = new ArrayList<>();

Expand All @@ -47,10 +52,19 @@ public void execute(NodeProbe probe)
String cf = args.get(1);
String key = args.get(2);

List<String> sstables = probe.getSSTables(ks, cf, key, hexFormat);
for (String sstable : sstables)
if (showLevels && probe.isLeveledCompaction(ks, cf))
{
Map<Integer, Collection<String>> sstables = probe.getSSTablesWithLevel(ks, cf, key, hexFormat);
for (Integer level : sstables.keySet())
for (String sstable : sstables.get(level))
probe.output().out.println(level + ": " + sstable);
} else
{
probe.output().out.println(sstable);
List<String> sstables = probe.getSSTables(ks, cf, key, hexFormat);
for (String sstable : sstables)
{
probe.output().out.println(sstable);
}
}
}
}

0 comments on commit 279f284

Please sign in to comment.