You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2019/04/02 08:39:09 UTC

[bookkeeper] branch master updated: Migrate command `lostbookierecoverydelay`

This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 120d677  Migrate command `lostbookierecoverydelay`
120d677 is described below

commit 120d67737f894c8c9ce5d7954c697015f8e7b1e2
Author: Yong Zhang <zh...@gmail.com>
AuthorDate: Tue Apr 2 16:39:03 2019 +0800

    Migrate command `lostbookierecoverydelay`
    
    Descriptions of the changes in this PR:
    
    - Using `bkctl` run command `lostbookierecoverydelay`
    
    ### Motivation
    
    #2015
    
    Reviewers: Sijie Guo <si...@apache.org>
    
    This closes #2016 from zymap/command-lostbookierecoverydelay
---
 .../org/apache/bookkeeper/bookie/BookieShell.java  |  33 ++----
 .../LostBookieRecoveryDelayCommand.java            | 119 +++++++++++++++++++++
 .../cli/commands/autorecovery/package-info.java    |  23 ++++
 .../cli/commands/AutoRecoveryCommandGroup.java     |  47 ++++++++
 tools/ledger/src/main/resources/commands           |   1 +
 .../LostBookieRecoveryDelayCommandTets.java        | 101 +++++++++++++++++
 6 files changed, 301 insertions(+), 23 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
index 8ec97c7..a656f38 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java
@@ -98,6 +98,7 @@ import org.apache.bookkeeper.replication.ReplicationException;
 import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException;
 import org.apache.bookkeeper.replication.ReplicationException.UnavailableException;
 import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToDBStorageCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ConvertToInterleavedStorageCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand;
@@ -1679,30 +1680,16 @@ public class BookieShell implements Tool {
         int runCmd(CommandLine cmdLine) throws Exception {
             boolean getter = cmdLine.hasOption("g");
             boolean setter = cmdLine.hasOption("s");
-
-            if ((!getter && !setter) || (getter && setter)) {
-                LOG.error("One and only one of -get and -set must be specified");
-                printUsage();
-                return 1;
+            int set = 0;
+            if (setter) {
+                set = Integer.parseInt(cmdLine.getOptionValue("set"));
             }
-            ClientConfiguration adminConf = new ClientConfiguration(bkConf);
-            BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
-            try {
-                if (getter) {
-                    int lostBookieRecoveryDelay = admin.getLostBookieRecoveryDelay();
-                    LOG.info("LostBookieRecoveryDelay value in ZK: {}", String.valueOf(lostBookieRecoveryDelay));
-                } else {
-                    int lostBookieRecoveryDelay = Integer.parseInt(cmdLine.getOptionValue("set"));
-                    admin.setLostBookieRecoveryDelay(lostBookieRecoveryDelay);
-                    LOG.info("Successfully set LostBookieRecoveryDelay value in ZK: {}",
-                            String.valueOf(lostBookieRecoveryDelay));
-                }
-            } finally {
-                if (admin != null) {
-                    admin.close();
-                }
-            }
-            return 0;
+
+            LostBookieRecoveryDelayCommand.LBRDFlags flags = new LostBookieRecoveryDelayCommand.LBRDFlags()
+                .get(getter).set(set);
+            LostBookieRecoveryDelayCommand cmd = new LostBookieRecoveryDelayCommand();
+            boolean result = cmd.apply(bkConf, flags);
+            return result ? 0 : 1;
         }
     }
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommand.java
new file mode 100644
index 0000000..12a4564
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommand.java
@@ -0,0 +1,119 @@
+/*
+ * 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.bookkeeper.tools.cli.commands.autorecovery;
+
+import com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import lombok.Setter;
+import lombok.experimental.Accessors;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.replication.ReplicationException;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommand;
+import org.apache.bookkeeper.tools.framework.CliFlags;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Command to Setter and Getter for LostBookieRecoveryDelay value (in seconds) in metadata store.
+ */
+public class LostBookieRecoveryDelayCommand extends BookieCommand<LostBookieRecoveryDelayCommand.LBRDFlags> {
+
+    static final Logger LOG = LoggerFactory.getLogger(LostBookieRecoveryDelayCommand.class);
+
+    private static final String NAME = "lostbookierecoverydelay";
+    private static final String DESC =
+        "Setter and Getter for LostBookieRecoveryDelay value (in seconds) in metadata store";
+
+    private static final int DEFAULT = 0;
+
+    public LostBookieRecoveryDelayCommand() {
+        this(new LBRDFlags());
+    }
+
+    private LostBookieRecoveryDelayCommand(LBRDFlags flags) {
+        super(CliSpec.<LostBookieRecoveryDelayCommand.LBRDFlags>newBuilder()
+                  .withName(NAME)
+                  .withDescription(DESC)
+                  .withFlags(flags)
+                  .build());
+    }
+
+    /**
+     * Flags for command LostBookieRecoveryDelay.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class LBRDFlags extends CliFlags{
+
+        @Parameter(names = { "-g", "--get" }, description = "Get LostBookieRecoveryDelay value (in seconds)")
+        private boolean get;
+
+        @Parameter(names = { "-s", "--set" }, description = "Set LostBookieRecoveryDelay value (in seconds)")
+        private int set = DEFAULT;
+
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, LBRDFlags cmdFlags) {
+        try {
+            return handler(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public boolean handler(ServerConfiguration conf, LBRDFlags flags)
+        throws InterruptedException, BKException, IOException, ReplicationException.UnavailableException,
+               ReplicationException.CompatibilityException, KeeperException {
+        boolean getter = flags.get;
+        boolean setter = false;
+        if (flags.set != DEFAULT) {
+            setter = true;
+        }
+
+        if ((!getter && !setter) || (getter && setter)) {
+            LOG.error("One and only one of -get and -set must be specified");
+            return false;
+        }
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            if (getter) {
+                int lostBookieRecoveryDelay = admin.getLostBookieRecoveryDelay();
+                LOG.info("LostBookieRecoveryDelay value in ZK: {}", String.valueOf(lostBookieRecoveryDelay));
+            } else {
+                int lostBookieRecoveryDelay = flags.set;
+                admin.setLostBookieRecoveryDelay(lostBookieRecoveryDelay);
+                LOG.info("Successfully set LostBookieRecoveryDelay value in ZK: {}",
+                         String.valueOf(lostBookieRecoveryDelay));
+            }
+        } finally {
+            if (admin != null) {
+                admin.close();
+            }
+        }
+        return true;
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java
new file mode 100644
index 0000000..fa1bbbf
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package provides all toggle commands.
+ */
+package org.apache.bookkeeper.tools.cli.commands.autorecovery;
\ No newline at end of file
diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java
new file mode 100644
index 0000000..d59412b
--- /dev/null
+++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/AutoRecoveryCommandGroup.java
@@ -0,0 +1,47 @@
+/*
+ * 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.bookkeeper.tools.cli.commands;
+
+import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE;
+
+import org.apache.bookkeeper.tools.cli.commands.autorecovery.LostBookieRecoveryDelayCommand;
+import org.apache.bookkeeper.tools.common.BKFlags;
+import org.apache.bookkeeper.tools.framework.CliCommandGroup;
+import org.apache.bookkeeper.tools.framework.CliSpec;
+
+/**
+ * Commands on some specific operation.
+ */
+public class AutoRecoveryCommandGroup extends CliCommandGroup<BKFlags> {
+
+    private static final String NAME = "autorecovery";
+    private static final String DESC = "Command on some specific operation.";
+
+    private static final CliSpec<BKFlags> spec = CliSpec.<BKFlags>newBuilder()
+                                                     .withName(NAME)
+                                                     .withDescription(DESC)
+                                                     .withCategory(CATEGORY_INFRA_SERVICE)
+                                                     .addCommand(new LostBookieRecoveryDelayCommand())
+                                                     .build();
+
+    public AutoRecoveryCommandGroup() {
+        super(spec);
+    }
+}
+
diff --git a/tools/ledger/src/main/resources/commands b/tools/ledger/src/main/resources/commands
index c9af7a1..c4d6412 100644
--- a/tools/ledger/src/main/resources/commands
+++ b/tools/ledger/src/main/resources/commands
@@ -21,3 +21,4 @@ org.apache.bookkeeper.tools.cli.commands.BookieIdCommandGroup
 org.apache.bookkeeper.tools.cli.commands.BookiesCommandGroup
 org.apache.bookkeeper.tools.cli.commands.CookieCommandGroup
 org.apache.bookkeeper.tools.cli.commands.LedgerCommandGroup
+org.apache.bookkeeper.tools.cli.commands.AutoRecoveryCommandGroup
\ No newline at end of file
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommandTets.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommandTets.java
new file mode 100644
index 0000000..861c523
--- /dev/null
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/autorecovery/LostBookieRecoveryDelayCommandTets.java
@@ -0,0 +1,101 @@
+/*
+ * 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.bookkeeper.tools.cli.commands.autorecovery;
+
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.powermock.api.mockito.PowerMockito.doNothing;
+import static org.powermock.api.mockito.PowerMockito.mock;
+import static org.powermock.api.mockito.PowerMockito.verifyNew;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.conf.AbstractConfiguration;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * Unit test for {@link LostBookieRecoveryDelayCommand}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ LostBookieRecoveryDelayCommand.class, ClientConfiguration.class, BookKeeperAdmin.class })
+public class LostBookieRecoveryDelayCommandTets extends BookieCommandTestBase {
+
+    private ClientConfiguration clientConfiguration;
+    private BookKeeperAdmin admin;
+
+    public LostBookieRecoveryDelayCommandTets() {
+        super(3, 0);
+    }
+
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+
+        PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf);
+
+        clientConfiguration = mock(ClientConfiguration.class);
+        PowerMockito.whenNew(ClientConfiguration.class).withParameterTypes(AbstractConfiguration.class)
+                    .withArguments(eq(conf)).thenReturn(clientConfiguration);
+
+        admin = mock(BookKeeperAdmin.class);
+        PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(ClientConfiguration.class)
+                    .withArguments(eq(clientConfiguration)).thenReturn(admin);
+
+        when(admin.getLostBookieRecoveryDelay()).thenReturn(1);
+        doNothing().when(admin).setLostBookieRecoveryDelay(anyInt());
+    }
+
+    @Test
+    public void testWithoutArgs() {
+        LostBookieRecoveryDelayCommand cmd = new LostBookieRecoveryDelayCommand();
+        Assert.assertFalse(cmd.apply(bkFlags, new String[] { "" }));
+    }
+
+    @Test
+    public void testWithSet() throws Exception {
+        testCommand("-s", "1");
+        verifyNew(ClientConfiguration.class, times(1)).withArguments(conf);
+        verifyNew(BookKeeperAdmin.class, times(1)).withArguments(clientConfiguration);
+        verify(admin, times(1)).setLostBookieRecoveryDelay(1);
+    }
+
+    @Test
+    public void testWithGet() throws Exception {
+        testCommand("-g");
+        verifyNew(ClientConfiguration.class, times(1)).withArguments(conf);
+        verifyNew(BookKeeperAdmin.class, times(1)).withArguments(clientConfiguration);
+        verify(admin, times(1)).getLostBookieRecoveryDelay();
+    }
+
+
+    private void testCommand(String... args) {
+        LostBookieRecoveryDelayCommand cmd = new LostBookieRecoveryDelayCommand();
+        Assert.assertTrue(cmd.apply(bkFlags, args));
+    }
+}