You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2016/03/18 17:27:23 UTC

cassandra git commit: sstablemetadata should print min/max token for the sstable

Repository: cassandra
Updated Branches:
  refs/heads/trunk 6f067af51 -> 77b27f33e


sstablemetadata should print min/max token for the sstable

Patch by marcuse and Vladislav Sinjavin; reviewed by yukim for CASSANDRA-7159


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/77b27f33
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/77b27f33
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/77b27f33

Branch: refs/heads/trunk
Commit: 77b27f33e2bc4a99435fdddb78f887085c094ddd
Parents: 6f067af
Author: Marcus Eriksson <ma...@apache.org>
Authored: Thu Mar 17 15:30:55 2016 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Fri Mar 18 17:25:46 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/io/sstable/IndexSummary.java      | 23 ++++++++++++++++++
 .../cassandra/tools/SSTableMetadataViewer.java  | 25 ++++++++++++++++++++
 3 files changed, 49 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/77b27f33/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 173d14f..be9115e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.6
+ * sstablemetadata should print sstable min/max token (CASSANDRA-7159)
  * Do not wrap CassandraException in TriggerExecutor (CASSANDRA-9421)
  * COPY TO should have higher double precision (CASSANDRA-11255)
  * Stress should exit with non-zero status after failure (CASSANDRA-10340)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/77b27f33/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummary.java b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
index 371a243..6de3478 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
@@ -29,7 +29,9 @@ import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.PartitionPosition;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.concurrent.Ref;
 import org.apache.cassandra.utils.concurrent.WrappedSharedCloseable;
 import org.apache.cassandra.utils.memory.MemoryUtil;
@@ -347,5 +349,26 @@ public class IndexSummary extends WrappedSharedCloseable
                 offsets.setInt(i, (int) (offsets.getInt(i) - offsets.size()));
             return new IndexSummary(partitioner, offsets, offsetCount, entries, entries.size(), fullSamplingSummarySize, minIndexInterval, samplingLevel);
         }
+
+        /**
+         * Deserializes the first and last key stored in the summary
+         *
+         * Only for use by offline tools like SSTableMetadataViewer, otherwise SSTable.first/last should be used.
+         */
+        public Pair<DecoratedKey, DecoratedKey> deserializeFirstLastKey(DataInputStream in, IPartitioner partitioner, boolean haveSamplingLevel) throws IOException
+        {
+            in.skipBytes(4); // minIndexInterval
+            int offsetCount = in.readInt();
+            long offheapSize = in.readLong();
+            if (haveSamplingLevel)
+                in.skipBytes(8); // samplingLevel, fullSamplingSummarySize
+
+            in.skip(offsetCount * 4);
+            in.skip(offheapSize - offsetCount * 4);
+
+            DecoratedKey first = partitioner.decorateKey(ByteBufferUtil.readWithLength(in));
+            DecoratedKey last = partitioner.decorateKey(ByteBufferUtil.readWithLength(in));
+            return Pair.create(first, last);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/77b27f33/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 04e148a..d2e0513 100644
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@ -17,7 +17,9 @@
  */
 package org.apache.cassandra.tools;
 
+import java.io.DataInputStream;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.ByteBuffer;
@@ -27,15 +29,19 @@ import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 
+import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.rows.EncodingStats;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummary;
 import org.apache.cassandra.io.sstable.metadata.*;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
 
 /**
  * Shows the contents of sstable metadata
@@ -88,6 +94,10 @@ public class SSTableMetadataViewer
                         out.printf("Compression ratio: %s%n", stats.compressionRatio);
                     out.printf("TTL min: %s%n", stats.minTTL);
                     out.printf("TTL max: %s%n", stats.maxTTL);
+
+                    if (validation != null && header != null)
+                        printMinMaxToken(descriptor, FBUtilities.newPartitioner(validation.partitioner), header.getKeyType(), out);
+
                     if (header != null && header.getClusteringTypes().size() == stats.minClusteringValues.size())
                     {
                         List<AbstractType<?>> clusteringTypes = header.getClusteringTypes();
@@ -170,4 +180,19 @@ public class SSTableMetadataViewer
                                       (i < ecch.length ? ecch[i] : "")));
         }
     }
+
+    private static void printMinMaxToken(Descriptor descriptor, IPartitioner partitioner, AbstractType<?> keyType, PrintStream out) throws IOException
+    {
+        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        if (!summariesFile.exists())
+            return;
+
+        try (DataInputStream iStream = new DataInputStream(new FileInputStream(summariesFile)))
+        {
+            Pair<DecoratedKey, DecoratedKey> firstLast = new IndexSummary.IndexSummarySerializer().deserializeFirstLastKey(iStream, partitioner, descriptor.version.hasSamplingLevel());
+            out.printf("First token: %s (key=%s)%n", firstLast.left.getToken(), keyType.getString(firstLast.left.getKey()));
+            out.printf("Last token: %s (key=%s)%n", firstLast.right.getToken(), keyType.getString(firstLast.right.getKey()));
+        }
+    }
+
 }