-
Notifications
You must be signed in to change notification settings - Fork 3.6k
/
SSTableMetadataViewer.java
583 lines (541 loc) · 26.9 KB
/
SSTableMetadataViewer.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
/*
* 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.tools;
import java.io.IOException;
import java.io.PrintStream;
import java.io.PrintWriter;
import java.nio.ByteBuffer;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import com.google.common.collect.MinMaxPriorityQueue;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.HelpFormatter;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.cli.PosixParser;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ClusteringComparator;
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.db.rows.Row;
import org.apache.cassandra.db.rows.Unfiltered;
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.io.compress.CompressionMetadata;
import org.apache.cassandra.io.compress.ICompressor;
import org.apache.cassandra.io.sstable.Descriptor;
import org.apache.cassandra.io.sstable.ISSTableScanner;
import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.sstable.format.StatsComponent;
import org.apache.cassandra.io.sstable.metadata.CompactionMetadata;
import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
import org.apache.cassandra.io.util.File;
import org.apache.cassandra.schema.TableMetadata;
import org.apache.cassandra.schema.TableMetadataRef;
import org.apache.cassandra.tools.Util.TermHistogram;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;
import static org.apache.cassandra.tools.Util.BLUE;
import static org.apache.cassandra.tools.Util.CYAN;
import static org.apache.cassandra.tools.Util.RESET;
import static org.apache.cassandra.tools.Util.WHITE;
import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis;
import static org.apache.commons.lang3.time.DurationFormatUtils.formatDurationWords;
/**
* Shows the contents of sstable metadata
*/
public class SSTableMetadataViewer
{
private static final Options options = new Options();
private static CommandLine cmd;
private static final String COLORS = "c";
private static final String UNICODE = "u";
private static final String GCGS_KEY = "g";
private static final String TIMESTAMP_UNIT = "t";
private static final String SCAN = "s";
private static final String HELP = "h";
private static final Comparator<ValuedByteBuffer> VCOMP = Comparator.comparingLong(ValuedByteBuffer::getValue).reversed();
static
{
DatabaseDescriptor.toolInitialization();
}
boolean color;
boolean unicode;
int gc;
PrintStream out;
String[] files;
TimeUnit tsUnit;
public SSTableMetadataViewer()
{
this(true, true, 0, TimeUnit.MICROSECONDS, System.out);
}
public SSTableMetadataViewer(boolean color, boolean unicode, int gc, TimeUnit tsUnit, PrintStream out)
{
this.color = color;
this.tsUnit = tsUnit;
this.unicode = unicode;
this.out = out;
this.gc = gc;
}
public static String deletion(long time)
{
if (time == 0 || time == Long.MAX_VALUE)
{
return "no tombstones";
}
return toDateString(time, TimeUnit.SECONDS);
}
public static String toDateString(long time, TimeUnit unit)
{
if (time == 0)
{
return null;
}
return new java.text.SimpleDateFormat("MM/dd/yyyy HH:mm:ss").format(new java.util.Date(unit.toMillis(time)));
}
public static String toDurationString(long duration, TimeUnit unit)
{
if (duration == 0)
{
return null;
}
else if (duration == Integer.MAX_VALUE)
{
return "never";
}
return formatDurationWords(unit.toMillis(duration), true, true);
}
public static String toByteString(long bytes)
{
if (bytes == 0)
return null;
else if (bytes < 1024)
return bytes + " B";
int exp = (int) (Math.log(bytes) / Math.log(1024));
char pre = "kMGTP".charAt(exp - 1);
return String.format("%.1f %sB", bytes / Math.pow(1024, exp), pre);
}
public String scannedOverviewOutput(String key, long value)
{
StringBuilder sb = new StringBuilder();
if (color) sb.append(CYAN);
sb.append('[');
if (color) sb.append(RESET);
sb.append(key);
if (color) sb.append(CYAN);
sb.append("] ");
if (color) sb.append(RESET);
sb.append(value);
return sb.toString();
}
private void printScannedOverview(Descriptor descriptor, StatsMetadata stats) throws IOException
{
TableMetadata cfm = Util.metadataFromSSTable(descriptor);
SSTableReader reader = SSTableReader.openNoValidation(null, descriptor, TableMetadataRef.forOfflineTools(cfm));
try (ISSTableScanner scanner = reader.getScanner())
{
long bytes = scanner.getLengthInBytes();
MinMaxPriorityQueue<ValuedByteBuffer> widestPartitions = MinMaxPriorityQueue
.orderedBy(VCOMP)
.maximumSize(5)
.create();
MinMaxPriorityQueue<ValuedByteBuffer> largestPartitions = MinMaxPriorityQueue
.orderedBy(VCOMP)
.maximumSize(5)
.create();
MinMaxPriorityQueue<ValuedByteBuffer> mostTombstones = MinMaxPriorityQueue
.orderedBy(VCOMP)
.maximumSize(5)
.create();
long partitionCount = 0;
long rowCount = 0;
long tombstoneCount = 0;
long cellCount = 0;
double totalCells = stats.totalColumnsSet;
int lastPercent = 0;
long lastPercentTime = 0;
while (scanner.hasNext())
{
try (UnfilteredRowIterator partition = scanner.next())
{
long psize = 0;
long pcount = 0;
int ptombcount = 0;
partitionCount++;
if (!partition.staticRow().isEmpty())
{
rowCount++;
pcount++;
psize += partition.staticRow().dataSize();
}
if (!partition.partitionLevelDeletion().isLive())
{
tombstoneCount++;
ptombcount++;
}
while (partition.hasNext())
{
Unfiltered unfiltered = partition.next();
switch (unfiltered.kind())
{
case ROW:
rowCount++;
Row row = (Row) unfiltered;
psize += row.dataSize();
pcount++;
for (org.apache.cassandra.db.rows.Cell<?> cell : row.cells())
{
cellCount++;
double percentComplete = Math.min(1.0, cellCount / totalCells);
if (lastPercent != (int) (percentComplete * 100) &&
(currentTimeMillis() - lastPercentTime) > 1000)
{
lastPercentTime = currentTimeMillis();
lastPercent = (int) (percentComplete * 100);
if (color)
out.printf("\r%sAnalyzing SSTable... %s%s %s(%%%s)", BLUE, CYAN,
Util.progress(percentComplete, 30, unicode),
RESET,
(int) (percentComplete * 100));
else
out.printf("\rAnalyzing SSTable... %s (%%%s)",
Util.progress(percentComplete, 30, unicode),
(int) (percentComplete * 100));
out.flush();
}
if (cell.isTombstone())
{
tombstoneCount++;
ptombcount++;
}
}
break;
case RANGE_TOMBSTONE_MARKER:
tombstoneCount++;
ptombcount++;
break;
}
}
widestPartitions.add(new ValuedByteBuffer(partition.partitionKey().getKey(), pcount));
largestPartitions.add(new ValuedByteBuffer(partition.partitionKey().getKey(), psize));
mostTombstones.add(new ValuedByteBuffer(partition.partitionKey().getKey(), ptombcount));
}
}
out.printf("\r%80s\r", " ");
field("Size", bytes);
field("Partitions", partitionCount);
field("Rows", rowCount);
field("Tombstones", tombstoneCount);
field("Cells", cellCount);
field("Widest Partitions", "");
Util.iterToStream(widestPartitions.iterator()).sorted(VCOMP).forEach(p ->
{
out.println(" " + scannedOverviewOutput(cfm.partitionKeyType.getString(p.buffer), p.value));
});
field("Largest Partitions", "");
Util.iterToStream(largestPartitions.iterator()).sorted(VCOMP).forEach(p ->
{
out.print(" ");
out.print(scannedOverviewOutput(cfm.partitionKeyType.getString(p.buffer), p.value));
if (color)
out.print(WHITE);
out.print(" (");
out.print(toByteString(p.value));
out.print(")");
if (color)
out.print(RESET);
out.println();
});
StringBuilder tleaders = new StringBuilder();
Util.iterToStream(mostTombstones.iterator()).sorted(VCOMP).forEach(p ->
{
if (p.value > 0)
{
tleaders.append(" ");
tleaders.append(scannedOverviewOutput(cfm.partitionKeyType.getString(p.buffer), p.value));
tleaders.append(System.lineSeparator());
}
});
String tombstoneLeaders = tleaders.toString();
if (tombstoneLeaders.length() > 10)
{
field("Tombstone Leaders", "");
out.print(tombstoneLeaders);
}
}
finally
{
reader.selfRef().ensureReleased();
}
}
private void printSStableMetadata(File file, boolean scan) throws IOException
{
Descriptor descriptor = Descriptor.fromFileWithComponent(file, false).left;
StatsComponent statsComponent = StatsComponent.load(descriptor);
ValidationMetadata validation = statsComponent.validationMetadata();
StatsMetadata stats = statsComponent.statsMetadata();
CompactionMetadata compaction = statsComponent.compactionMetadata();
SerializationHeader.Component header = statsComponent.serializationHeader();
Class<? extends ICompressor> compressorClass = null;
try (CompressionMetadata compression = CompressionInfoComponent.loadIfExists(descriptor))
{
compressorClass = compression != null ? compression.compressor().getClass() : null;
}
field("SSTable", descriptor);
if (scan && descriptor.version.version.compareTo("ma") >= 0)
{
printScannedOverview(descriptor, stats);
}
if (validation != null)
{
field("Partitioner", validation.partitioner);
field("Bloom Filter FP chance", validation.bloomFilterFPChance);
}
if (stats != null)
{
TimeUnit tsUnit = TimeUnit.MICROSECONDS;
field("Minimum timestamp", toDateString(stats.minTimestamp, tsUnit), Long.toString(stats.minTimestamp));
field("Maximum timestamp", toDateString(stats.maxTimestamp, tsUnit), Long.toString(stats.maxTimestamp));
field("Duration", durationString(stats.maxTimestamp - stats.minTimestamp));
field("SSTable min local deletion time", deletion(stats.minLocalDeletionTime), Long.toString(stats.minLocalDeletionTime));
field("SSTable max local deletion time", deletion(stats.maxLocalDeletionTime), Long.toString(stats.maxLocalDeletionTime));
field("Compressor", compressorClass != null ? compressorClass.getName() : "-");
if (compressorClass != null)
field("Compression ratio", stats.compressionRatio);
field("TTL min", stats.minTTL, toDurationString(stats.minTTL, TimeUnit.SECONDS));
field("TTL max", stats.maxTTL, toDurationString(stats.maxTTL, TimeUnit.SECONDS));
if (validation != null && header != null)
printMinMaxToken(descriptor, FBUtilities.newPartitioner(descriptor), header.getKeyType(), stats);
if (header != null)
{
ClusteringComparator comparator = new ClusteringComparator(header.getClusteringTypes());
field("Covered clusterings", stats.coveredClustering.toString(comparator));
}
field("Estimated droppable tombstones",
stats.getEstimatedDroppableTombstoneRatio((int) (currentTimeMillis() / 1000) - this.gc));
field("SSTable Level", stats.sstableLevel);
field("Repaired at", stats.repairedAt, toDateString(stats.repairedAt, TimeUnit.MILLISECONDS));
field("Originating host id", stats.originatingHostId);
field("Pending repair", stats.pendingRepair);
field("Replay positions covered", stats.commitLogIntervals);
field("totalColumnsSet", stats.totalColumnsSet);
field("totalRows", stats.totalRows);
field("Estimated tombstone drop times", "");
TermHistogram estDropped = new TermHistogram(stats.estimatedTombstoneDropTime,
"Drop Time",
offset -> String.format("%d %s",
offset,
Util.wrapQuiet(toDateString(offset, TimeUnit.SECONDS),
color)),
String::valueOf);
estDropped.printHistogram(out, color, unicode);
field("Partition Size", "");
TermHistogram rowSize = new TermHistogram(stats.estimatedPartitionSize,
"Size (bytes)",
offset -> String.format("%d %s",
offset,
Util.wrapQuiet(toByteString(offset), color)),
String::valueOf);
rowSize.printHistogram(out, color, unicode);
field("Column Count", "");
TermHistogram cellCount = new TermHistogram(stats.estimatedCellPerPartitionCount,
"Columns",
String::valueOf,
String::valueOf);
cellCount.printHistogram(out, color, unicode);
field("Local token space coverage", stats.tokenSpaceCoverage);
}
if (compaction != null)
{
field("Estimated cardinality", compaction.cardinalityEstimator.cardinality());
}
if (header != null)
{
EncodingStats encodingStats = header.getEncodingStats();
AbstractType<?> keyType = header.getKeyType();
List<AbstractType<?>> clusteringTypes = header.getClusteringTypes();
Map<ByteBuffer, AbstractType<?>> staticColumns = header.getStaticColumns();
Map<String, String> statics = staticColumns.entrySet().stream()
.collect(Collectors.toMap(e -> UTF8Type.instance.getString(e.getKey()),
e -> e.getValue().toString()));
Map<ByteBuffer, AbstractType<?>> regularColumns = header.getRegularColumns();
Map<String, String> regulars = regularColumns.entrySet().stream()
.collect(Collectors.toMap(e -> UTF8Type.instance.getString(e.getKey()),
e -> e.getValue().toString()));
field("EncodingStats minTTL", encodingStats.minTTL,
toDurationString(encodingStats.minTTL, TimeUnit.SECONDS));
field("EncodingStats minLocalDeletionTime", toDateString(encodingStats.minLocalDeletionTime,
TimeUnit.SECONDS), Long.toString(encodingStats.minLocalDeletionTime));
field("EncodingStats minTimestamp", toDateString(encodingStats.minTimestamp, tsUnit),
Long.toString(encodingStats.minTimestamp));
field("KeyType", keyType.toString());
field("ClusteringTypes", clusteringTypes.toString());
field("StaticColumns", FBUtilities.toString(statics));
field("RegularColumns", FBUtilities.toString(regulars));
if (stats != null)
field("IsTransient", stats.isTransient);
}
}
private void field(String field, Object value)
{
field(field, value, null);
}
private void field(String field, Object value, String comment)
{
StringBuilder sb = new StringBuilder();
if (color) sb.append(BLUE);
sb.append(field);
if (color) sb.append(CYAN);
sb.append(": ");
if (color) sb.append(RESET);
sb.append(value == null? "--" : value.toString());
if (comment != null)
{
if (color) sb.append(WHITE);
sb.append(" (");
sb.append(comment);
sb.append(')');
if (color) sb.append(RESET);
}
this.out.println(sb);
}
public static String durationString(Long value)
{
long seconds = TimeUnit.MICROSECONDS.toSeconds(value);
long day = TimeUnit.SECONDS.toDays(seconds);
long hours = TimeUnit.SECONDS.toHours(seconds) - (day * 24);
long minute = TimeUnit.SECONDS.toMinutes(seconds) - (TimeUnit.SECONDS.toHours(seconds) * 60);
long second = TimeUnit.SECONDS.toSeconds(seconds) - (TimeUnit.SECONDS.toMinutes(seconds) * 60);
return "Days: " + day + " Hours: " + hours + " Minutes: " + minute + " Seconds: " + second;
}
private static void printUsage()
{
try (PrintWriter errWriter = new PrintWriter(System.err, true))
{
HelpFormatter formatter = new HelpFormatter();
formatter.printHelp(errWriter, 120, "sstablemetadata <options> <sstable...>",
String.format("%nDump information about SSTable[s] for Apache Cassandra 3.x%nOptions:"),
options, 2, 1, "", true);
errWriter.println();
}
}
private void printMinMaxToken(Descriptor descriptor, IPartitioner partitioner, AbstractType<?> keyType, StatsMetadata statsMetadata)
throws IOException
{
if (descriptor.version.hasKeyRange())
{
if (statsMetadata.firstKey == null || statsMetadata.lastKey == null)
return;
field("First token", partitioner.getToken(statsMetadata.firstKey), keyType.getString(statsMetadata.firstKey));
field("Last token", partitioner.getToken(statsMetadata.lastKey), keyType.getString(statsMetadata.lastKey));
}
else
{
Pair<DecoratedKey, DecoratedKey> firstLast = descriptor.getFormat().getReaderFactory().readKeyRange(descriptor, partitioner);
if (firstLast == null)
return;
field("First token", firstLast.left.getToken(), keyType.getString(firstLast.left.getKey()));
field("Last token", firstLast.right.getToken(), keyType.getString(firstLast.right.getKey()));
}
}
/**
* @param args
* a list of sstables whose metadata we're interested in
*/
public static void main(String[] args) throws IOException
{
CommandLineParser parser = new PosixParser();
Option disableColors = new Option(COLORS, "colors", false, "Use ANSI color sequences");
disableColors.setOptionalArg(true);
options.addOption(disableColors);
Option unicode = new Option(UNICODE, "unicode", false, "Use unicode to draw histograms and progress bars");
unicode.setOptionalArg(true);
options.addOption(unicode);
Option gcgs = new Option(GCGS_KEY, "gc_grace_seconds", true, "Time to use when calculating droppable tombstones");
gcgs.setOptionalArg(true);
options.addOption(gcgs);
Option tsUnit = new Option(TIMESTAMP_UNIT, "timestamp_unit", true, "Time unit that cell timestamps are written with");
tsUnit.setOptionalArg(true);
options.addOption(tsUnit);
Option help = new Option(HELP, "help", false, "Help");
help.setOptionalArg(true);
options.addOption(help);
Option scanEnabled = new Option(SCAN, "scan", false,
"Full sstable scan for additional details. Only available in 3.0+ sstables. Defaults: false");
scanEnabled.setOptionalArg(true);
options.addOption(scanEnabled);
try
{
cmd = parser.parse(options, args);
}
catch (ParseException e1)
{
System.err.println(e1.getMessage());
printUsage();
System.exit(1);
}
if (cmd.getArgs().length < 1)
{
System.err.println("You must supply at least one sstable");
printUsage();
System.exit(1);
}
boolean enabledColors = cmd.hasOption(COLORS);
boolean enabledUnicode = cmd.hasOption(UNICODE);
boolean fullScan = cmd.hasOption(SCAN);
int gc = Integer.parseInt(cmd.getOptionValue(GCGS_KEY, "0"));
TimeUnit ts = TimeUnit.valueOf(cmd.getOptionValue(TIMESTAMP_UNIT, "MICROSECONDS"));
SSTableMetadataViewer metawriter = new SSTableMetadataViewer(enabledColors, enabledUnicode, gc, ts, System.out);
for (String fname : cmd.getArgs())
{
File sstable = new File(fname);
if (sstable.exists())
{
metawriter.printSStableMetadata(sstable, fullScan);
}
else
{
System.out.println("No such file: " + fname);
}
}
}
private static class ValuedByteBuffer
{
public long value;
public ByteBuffer buffer;
public ValuedByteBuffer(ByteBuffer buffer, long value)
{
this.value = value;
this.buffer = buffer;
}
public long getValue()
{
return value;
}
}
}