Skip to content

Commit

Permalink
fix serialization error in getsstables --show-levels
Browse files Browse the repository at this point in the history
Patch by Jordan West; Reviewed by Brandon Williams, Cheng Wang for CASSANDRA-18140;
  • Loading branch information
jrwest committed Jan 11, 2023
1 parent 386bf78 commit e936b2c
Show file tree
Hide file tree
Showing 5 changed files with 18 additions and 11 deletions.
1 change: 1 addition & 0 deletions CHANGES.txt
@@ -1,4 +1,5 @@
4.2
* Fix serialization error in new getsstables --show-levels option (CASSANDRA-18140)
* Use checked casts when reading vints as ints (CASSANDRA-18099)
* Add Mutation Serialization Caching (CASSANDRA-17998)
* Only reload compaction strategies if disk boundaries change (CASSANDRA-17874)
Expand Down
19 changes: 13 additions & 6 deletions src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Expand Up @@ -64,12 +64,10 @@
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 @@ -1946,14 +1944,23 @@ public List<String> getSSTablesForKey(String key, boolean hexFormat)
return withSSTablesForKey(key, hexFormat, SSTableReader::getFilename);
}

public Map<Integer, Collection<String>> getSSTablesForKeyWithLevel(String key, boolean hexFormat)
public Map<Integer, Set<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();
HashMap<Integer, Set<String>> result = new HashMap<>();
for (Pair<Integer, String> sst : ssts)
result.put(sst.left, sst.right);
{
Set<String> perLevel = result.get(sst.left);
if (perLevel == null)
{
perLevel = new HashSet<>();
result.put(sst.left, perLevel);
}

return result.asMap();
perLevel.add(sst.right);
}

return result;
}

public <T> List<T> withSSTablesForKey(String key, boolean hexFormat, Function<SSTableReader, T> mapper)
Expand Down
Expand Up @@ -168,7 +168,7 @@ public interface ColumnFamilyStoreMBean
* @param hexFormat
* @return list of filenames and levels containing the key
*/
public Map<Integer, Collection<String>> getSSTablesForKeyWithLevel(String key, boolean hexFormat);
public Map<Integer, Set<String>> getSSTablesForKeyWithLevel(String key, boolean hexFormat);

/**
* Load new sstables from the given directory
Expand Down
3 changes: 1 addition & 2 deletions src/java/org/apache/cassandra/tools/NodeProbe.java
Expand Up @@ -30,7 +30,6 @@
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 @@ -1045,7 +1044,7 @@ 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)
public Map<Integer, Set<String>> getSSTablesWithLevel(String keyspace, String cf, String key, boolean hexFormat)
{
ColumnFamilyStoreMBean cfsProxy = getCfsProxy(keyspace, cf);
return cfsProxy.getSSTablesForKeyWithLevel(key, hexFormat);
Expand Down
4 changes: 2 additions & 2 deletions src/java/org/apache/cassandra/tools/nodetool/GetSSTables.java
Expand Up @@ -22,9 +22,9 @@
import io.airlift.airline.Command;

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

import io.airlift.airline.Option;
import org.apache.cassandra.tools.NodeProbe;
Expand Down Expand Up @@ -54,7 +54,7 @@ public void execute(NodeProbe probe)

if (showLevels && probe.isLeveledCompaction(ks, cf))
{
Map<Integer, Collection<String>> sstables = probe.getSSTablesWithLevel(ks, cf, key, hexFormat);
Map<Integer, Set<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);
Expand Down

0 comments on commit e936b2c

Please sign in to comment.