Skip to content

Commit 1a18985

Browse files
authored
KAFKA-18947 Remove unused raftManager in metadataShell (apache#19169)
* Remove unused `raftManager` in `metadataShell` * Enhance error message when no snapshot provided. * Since `raftManager` is removed, make `snapshot` a required argument. Result when no snapshot is given ``` $ ./bin/kafka-metadata-shell.sh usage: kafka-metadata-shell [-h] --snapshot SNAPSHOT [command [command ...]] kafka-metadata-shell: error: argument --snapshot/-s is required ``` ``` $ ./bin/kafka-metadata-shell.sh --help usage: kafka-metadata-shell [-h] --snapshot SNAPSHOT [command [command ...]] The Apache Kafka metadata shell positional arguments: command The command to run. optional arguments: -h, --help show this help message and exit --snapshot SNAPSHOT, -s SNAPSHOT The metadata snapshot file to read. ``` Reviewers: Jhen-Yung Hsu <[email protected]>, TengYao Chi <[email protected]>, Chia-Ping Tsai <[email protected]>
1 parent 0ebc3e8 commit 1a18985

File tree

2 files changed

+7
-50
lines changed

2 files changed

+7
-50
lines changed

shell/src/main/java/org/apache/kafka/shell/MetadataShell.java

+6-49
Original file line numberDiff line numberDiff line change
@@ -17,14 +17,12 @@
1717

1818
package org.apache.kafka.shell;
1919

20-
import kafka.raft.KafkaRaftManager;
2120
import kafka.tools.TerseFailure;
2221

2322
import org.apache.kafka.common.utils.Exit;
2423
import org.apache.kafka.common.utils.Utils;
2524
import org.apache.kafka.image.loader.MetadataLoader;
2625
import org.apache.kafka.metadata.util.SnapshotFileReader;
27-
import org.apache.kafka.server.common.ApiMessageAndVersion;
2826
import org.apache.kafka.server.fault.FaultHandler;
2927
import org.apache.kafka.server.fault.LoggingFaultHandler;
3028
import org.apache.kafka.server.util.FileLock;
@@ -60,17 +58,9 @@ public final class MetadataShell {
6058
private static final Logger log = LoggerFactory.getLogger(MetadataShell.class);
6159

6260
public static class Builder {
63-
private KafkaRaftManager<ApiMessageAndVersion> raftManager = null;
6461
private String snapshotPath = null;
6562
private FaultHandler faultHandler = new LoggingFaultHandler("shell", () -> { });
6663

67-
// Note: we assume that we have already taken the lock on the log directory before calling
68-
// this method.
69-
public Builder setRaftManager(KafkaRaftManager<ApiMessageAndVersion> raftManager) {
70-
this.raftManager = raftManager;
71-
return this;
72-
}
73-
7464
public Builder setSnapshotPath(String snapshotPath) {
7565
this.snapshotPath = snapshotPath;
7666
return this;
@@ -82,9 +72,7 @@ public Builder setFaultHandler(FaultHandler faultHandler) {
8272
}
8373

8474
public MetadataShell build() {
85-
return new MetadataShell(raftManager,
86-
snapshotPath,
87-
faultHandler);
75+
return new MetadataShell(snapshotPath, faultHandler);
8876
}
8977
}
9078

@@ -96,7 +84,7 @@ static File parent(File file) {
9684
File parent = file.getParentFile();
9785
return parent == null ? file : parent;
9886
}
99-
87+
10088
static File parentParent(File file) {
10189
return parent(parent(file));
10290
}
@@ -136,8 +124,6 @@ static FileLock takeDirectoryLock(File directory) throws IOException {
136124

137125
private final MetadataShellState state;
138126

139-
private final KafkaRaftManager<ApiMessageAndVersion> raftManager;
140-
141127
private final String snapshotPath;
142128

143129
private final FaultHandler faultHandler;
@@ -151,29 +137,17 @@ static FileLock takeDirectoryLock(File directory) throws IOException {
151137
private MetadataLoader loader;
152138

153139
public MetadataShell(
154-
KafkaRaftManager<ApiMessageAndVersion> raftManager,
155140
String snapshotPath,
156141
FaultHandler faultHandler
157142
) {
158143
this.state = new MetadataShellState();
159-
this.raftManager = raftManager;
160144
this.snapshotPath = snapshotPath;
161145
this.faultHandler = faultHandler;
162146
this.publisher = new MetadataShellPublisher(state);
163147
this.fileLock = null;
164148
this.snapshotFileReader = null;
165149
}
166150

167-
private void initializeWithRaftManager() {
168-
raftManager.startup();
169-
this.loader = new MetadataLoader.Builder().
170-
setFaultHandler(faultHandler).
171-
setNodeId(-1).
172-
setHighWaterMarkAccessor(() -> raftManager.client().highWatermark()).
173-
build();
174-
raftManager.register(loader);
175-
}
176-
177151
private void initializeWithSnapshotFileReader() throws Exception {
178152
this.fileLock = takeDirectoryLockIfExists(parentParent(new File(snapshotPath)));
179153
this.loader = new MetadataLoader.Builder().
@@ -186,18 +160,7 @@ private void initializeWithSnapshotFileReader() throws Exception {
186160
}
187161

188162
public void run(List<String> args) throws Exception {
189-
if (raftManager != null) {
190-
if (snapshotPath != null) {
191-
throw new RuntimeException("Can't specify both a raft manager and " +
192-
"snapshot file reader.");
193-
}
194-
initializeWithRaftManager();
195-
} else if (snapshotPath != null) {
196-
initializeWithSnapshotFileReader();
197-
} else {
198-
throw new RuntimeException("You must specify either a raft manager or a " +
199-
"snapshot file reader.");
200-
}
163+
initializeWithSnapshotFileReader();
201164
loader.installPublishers(Collections.singletonList(publisher)).get(15, TimeUnit.MINUTES);
202165
if (args == null || args.isEmpty()) {
203166
// Interactive mode.
@@ -222,14 +185,7 @@ public void run(List<String> args) throws Exception {
222185

223186
public void close() {
224187
Utils.closeQuietly(loader, "loader");
225-
if (raftManager != null) {
226-
try {
227-
raftManager.shutdown();
228-
} catch (Exception e) {
229-
log.error("Error shutting down RaftManager", e);
230-
}
231-
}
232-
Utils.closeQuietly(snapshotFileReader, "raftManager");
188+
Utils.closeQuietly(snapshotFileReader, "snapshotFileReader");
233189
if (fileLock != null) {
234190
try {
235191
fileLock.destroy();
@@ -248,7 +204,8 @@ public static void main(String[] args) {
248204
.description("The Apache Kafka metadata shell");
249205
parser.addArgument("--snapshot", "-s")
250206
.type(String.class)
251-
.help("The snapshot file to read.");
207+
.required(true)
208+
.help("The metadata snapshot file to read.");
252209
parser.addArgument("command")
253210
.nargs("*")
254211
.help("The command to run.");

shell/src/test/java/org/apache/kafka/shell/MetadataShellIntegrationTest.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -84,7 +84,7 @@ public void close() {
8484
@ValueSource(booleans = {false, true})
8585
public void testLock(boolean canLock) throws Exception {
8686
try (IntegrationEnv env = new IntegrationEnv()) {
87-
env.shell = new MetadataShell(null,
87+
env.shell = new MetadataShell(
8888
new File(new File(env.tempDir, "__cluster_metadata-0"), "00000000000122906351-0000000226.checkpoint").getAbsolutePath(),
8989
env.faultHandler);
9090

0 commit comments

Comments
 (0)