|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.cassandra.tools.nodetool; |
| 19 | + |
| 20 | +import java.io.PrintStream; |
| 21 | +import java.util.concurrent.TimeUnit; |
| 22 | + |
| 23 | +import com.google.common.util.concurrent.Uninterruptibles; |
| 24 | + |
| 25 | +import org.apache.cassandra.db.compression.ICompressionDictionaryTrainer.TrainingStatus; |
| 26 | +import org.apache.cassandra.db.compression.TrainingState; |
| 27 | +import org.apache.cassandra.tools.NodeProbe; |
| 28 | +import org.apache.cassandra.utils.Clock; |
| 29 | +import picocli.CommandLine.Command; |
| 30 | +import picocli.CommandLine.Option; |
| 31 | +import picocli.CommandLine.Parameters; |
| 32 | + |
| 33 | +@Command(name = "compressiondictionary", |
| 34 | + description = "Manage compression dictionaries", |
| 35 | + subcommands = { CompressionDictionary.Train.class }) |
| 36 | +public class CompressionDictionary |
| 37 | +{ |
| 38 | + @Command(name = "train", |
| 39 | + description = "Manually trigger compression dictionary training for a table. If no SSTables are available, the memtable will be flushed first.") |
| 40 | + public static class Train extends AbstractCommand |
| 41 | + { |
| 42 | + @Parameters(index = "0", description = "The keyspace name", arity = "1") |
| 43 | + private String keyspace; |
| 44 | + |
| 45 | + @Parameters(index = "1", description = "The table name", arity = "1") |
| 46 | + private String table; |
| 47 | + |
| 48 | + @Option(names = { "-f", "--force" }, description = "Force the dictionary training even if there are not enough samples") |
| 49 | + private boolean force = false; |
| 50 | + |
| 51 | + @Override |
| 52 | + public void execute(NodeProbe probe) |
| 53 | + { |
| 54 | + PrintStream out = probe.output().out; |
| 55 | + PrintStream err = probe.output().err; |
| 56 | + |
| 57 | + try |
| 58 | + { |
| 59 | + out.printf("Starting compression dictionary training for %s.%s...%n", keyspace, table); |
| 60 | + out.printf("Training from existing SSTables (flushing first if needed)%n"); |
| 61 | + |
| 62 | + probe.trainCompressionDictionary(keyspace, table, force); |
| 63 | + |
| 64 | + // Wait for training completion (10 minutes timeout for SSTable-based training) |
| 65 | + out.println("Sampling from existing SSTables and training."); |
| 66 | + long maxWaitMillis = TimeUnit.MINUTES.toMillis(10); |
| 67 | + long startTime = Clock.Global.currentTimeMillis(); |
| 68 | + |
| 69 | + while (Clock.Global.currentTimeMillis() - startTime < maxWaitMillis) |
| 70 | + { |
| 71 | + TrainingState trainingState = probe.getCompressionDictionaryTrainingState(keyspace, table); |
| 72 | + TrainingStatus status = trainingState.getStatus(); |
| 73 | + displayProgress(trainingState, startTime, out, status); |
| 74 | + if (TrainingStatus.COMPLETED == status) |
| 75 | + { |
| 76 | + out.printf("%nTraining completed successfully for %s.%s%n", keyspace, table); |
| 77 | + return; |
| 78 | + } |
| 79 | + else if (TrainingStatus.FAILED == status) |
| 80 | + { |
| 81 | + err.printf("%nTraining failed for %s.%s%n", keyspace, table); |
| 82 | + try |
| 83 | + { |
| 84 | + String failureMessage = trainingState.getFailureMessage(); |
| 85 | + if (failureMessage != null && !failureMessage.isEmpty()) |
| 86 | + { |
| 87 | + err.printf("Reason: %s%n", failureMessage); |
| 88 | + } |
| 89 | + } |
| 90 | + catch (Exception e) |
| 91 | + { |
| 92 | + // If we can't get the failure message, just continue without it |
| 93 | + } |
| 94 | + System.exit(1); |
| 95 | + } |
| 96 | + |
| 97 | + Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS); |
| 98 | + } |
| 99 | + |
| 100 | + err.printf("%nTraining did not complete within expected timeframe (10 minutes).%n"); |
| 101 | + System.exit(1); |
| 102 | + } |
| 103 | + catch (Exception e) |
| 104 | + { |
| 105 | + err.printf("Failed to trigger training: %s%n", e.getMessage()); |
| 106 | + System.exit(1); |
| 107 | + } |
| 108 | + } |
| 109 | + |
| 110 | + private static void displayProgress(TrainingState trainingState, long startTime, PrintStream out, TrainingStatus status) |
| 111 | + { |
| 112 | + // Display meaningful statistics |
| 113 | + long sampleCount = trainingState.getSampleCount(); |
| 114 | + long totalSampleSize = trainingState.getTotalSampleSize(); |
| 115 | + long elapsedSeconds = (Clock.Global.currentTimeMillis() - startTime) / 1000; |
| 116 | + double sampleSizeMB = totalSampleSize / (1024.0 * 1024.0); |
| 117 | + |
| 118 | + out.printf("\rStatus: %s | Samples: %d | Size: %.2f MiB | Elapsed: %ds", |
| 119 | + status, sampleCount, sampleSizeMB, elapsedSeconds); |
| 120 | + } |
| 121 | + } |
| 122 | +} |
0 commit comments