You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by zh...@apache.org on 2019/03/11 01:57:40 UTC

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

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

zhaijia 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 5a34488  Migrate command `bookieformat`
5a34488 is described below

commit 5a3448860be585972aa328aa1602b63ead177269
Author: Yong Zhang <zh...@gmail.com>
AuthorDate: Mon Mar 11 09:57:35 2019 +0800

    Migrate command `bookieformat`
    
    Descriptions of the changes in this PR:
    
    Migrate command `bookieformat`.
    
    ### Motivation
    
    #1974
    
    ### Changes
    
    - Add command `bookieformat` to bkctl
    - Replace command in shell
    
    
    Reviewers: Jia Zhai <zh...@apache.org>, Sijie Guo <si...@apache.org>
    
    This closes #1975 from zymap/command-bookieformat and squashes the following commits:
    
    90e8a1dde [Yong Zhang] Add doc
    e079383ac [Yong Zhang] Fix the way deal with exception
    eb78fe74f [Yong Zhang] Add unit test for `commandformat`
    a3bafe6d9 [Yong Zhang] Migrate command `bookieformate` to bkctl
---
 .../org/apache/bookkeeper/bookie/BookieShell.java  |  26 ++--
 .../tools/cli/commands/bookie/FormatCommand.java   | 109 +++++++++++++++++
 .../tools/cli/commands/BookieCommandGroup.java     |   2 +
 .../cli/commands/bookie/FormatCommandTest.java     | 136 +++++++++++++++++++++
 4 files changed, 256 insertions(+), 17 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 33f9f9a..628690e 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
@@ -109,6 +109,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.bookie.FormatCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand;
@@ -456,23 +457,14 @@ public class BookieShell implements Tool {
         int runCmd(CommandLine cmdLine) throws Exception {
             boolean interactive = (!cmdLine.hasOption("n"));
             boolean force = cmdLine.hasOption("f");
-
-            ServerConfiguration conf = new ServerConfiguration(bkConf);
-            boolean result = Bookie.format(conf, interactive, force);
-            // delete cookie
-            if (cmdLine.hasOption("d")) {
-                runFunctionWithRegistrationManager(bkConf, rm -> {
-                    try {
-                        Versioned<Cookie> cookie = Cookie.readFromRegistrationManager(rm, conf);
-                        cookie.getValue().deleteFromRegistrationManager(rm, conf, cookie.getVersion());
-                    } catch (CookieNotFoundException nne) {
-                        LOG.warn("No cookie to remove : ", nne);
-                    } catch (BookieException be) {
-                        throw new UncheckedExecutionException(be.getMessage(), be);
-                    }
-                    return null;
-                });
-            }
+            boolean deletecookie = cmdLine.hasOption("d");
+
+            FormatCommand.Flags flags = new FormatCommand.Flags()
+                .nonInteractive(interactive)
+                .force(force)
+                .deleteCookie(deletecookie);
+            FormatCommand command = new FormatCommand(flags);
+            boolean result = command.apply(bkConf, flags);
             return (result) ? 0 : 1;
         }
     }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommand.java
new file mode 100644
index 0000000..f213c17
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommand.java
@@ -0,0 +1,109 @@
+/*
+ * 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.bookie;
+
+import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager;
+
+import com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.concurrent.ExecutionException;
+import lombok.Setter;
+import lombok.experimental.Accessors;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.Cookie;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.exceptions.MetadataException;
+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.bookkeeper.versioning.Versioned;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Command to format the current server contents.
+ */
+public class FormatCommand extends BookieCommand<FormatCommand.Flags> {
+
+    static final Logger LOG = LoggerFactory.getLogger(FormatCommand.class);
+
+    private static final String NAME = "format";
+    private static final String DESC = "Format the current server contents.";
+
+    public FormatCommand() {
+        this(new Flags());
+    }
+
+    public FormatCommand(Flags flags) {
+        super(CliSpec.<Flags>newBuilder()
+            .withName(NAME)
+            .withDescription(DESC)
+            .withFlags(flags)
+            .build());
+    }
+
+    /**
+     * Flags for format bookie command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class Flags extends CliFlags {
+
+        @Parameter(names = {"-n", "--noninteractive"},
+            description = "Whether to confirm if old data exists?")
+        private boolean nonInteractive;
+
+        @Parameter(names = {"-f", "--force"},
+            description = "If [noninteractive] is specified, then whether"
+                + "to force delete the old data without prompt?")
+        private boolean force;
+
+        @Parameter(names = {"-d", "--deletecookie"},
+            description = "Delete its cookie on metadata store.")
+        private boolean deleteCookie;
+
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, Flags cmdFlags) {
+
+        ServerConfiguration bfconf = new ServerConfiguration(conf);
+        boolean result = Bookie.format(bfconf, cmdFlags.nonInteractive, cmdFlags.force);
+
+        // delete cookie
+        if (cmdFlags.deleteCookie) {
+            try {
+                runFunctionWithRegistrationManager(conf, rm -> {
+
+                    try {
+                        Versioned<Cookie> cookie = Cookie.readFromRegistrationManager(rm, bfconf);
+                        cookie.getValue().deleteFromRegistrationManager(rm, bfconf, cookie.getVersion());
+                    } catch (Exception e) {
+                        throw new UncheckedExecutionException(e.getMessage(), e);
+                    }
+
+                    return null;
+                });
+            } catch (MetadataException | ExecutionException e) {
+                throw new UncheckedExecutionException(e.getMessage(), e);
+            }
+        }
+        return result;
+    }
+}
diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java
index 2a5cf1f..cfa37e3 100644
--- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java
+++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookieCommandGroup.java
@@ -21,6 +21,7 @@ package org.apache.bookkeeper.tools.cli.commands;
 import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_INFRA_SERVICE;
 
 import org.apache.bookkeeper.tools.cli.BKCtl;
+import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
 import org.apache.bookkeeper.tools.common.BKFlags;
 import org.apache.bookkeeper.tools.framework.CliCommandGroup;
@@ -40,6 +41,7 @@ public class BookieCommandGroup extends CliCommandGroup<BKFlags> {
         .withParent(BKCtl.NAME)
         .withCategory(CATEGORY_INFRA_SERVICE)
         .addCommand(new LastMarkCommand())
+        .addCommand(new FormatCommand())
         .build();
 
     public BookieCommandGroup() {
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommandTest.java
new file mode 100644
index 0000000..f8fa984
--- /dev/null
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FormatCommandTest.java
@@ -0,0 +1,136 @@
+/*
+ * 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.bookie;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.function.Function;
+
+import org.apache.bookkeeper.bookie.Cookie;
+import org.apache.bookkeeper.conf.AbstractConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase;
+import org.apache.bookkeeper.versioning.LongVersion;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+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 {@link FormatCommand}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ FormatCommand.class, MetadataDrivers.class, Cookie.class })
+public class FormatCommandTest extends BookieCommandTestBase {
+
+    public FormatCommandTest() {
+        super(3, 0);
+    }
+
+    @SuppressWarnings("unchecked")
+    public void setup() throws Exception {
+        super.setup();
+
+        PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf);
+
+        PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class)
+                .withArguments(eq(conf)).thenReturn(conf);
+
+        PowerMockito.mockStatic(MetadataDrivers.class);
+        RegistrationManager rm = mock(RegistrationManager.class);
+        PowerMockito.doAnswer(invocationOnMock -> {
+            Function<RegistrationManager, ?> func = invocationOnMock.getArgument(1);
+            func.apply(rm);
+            return true;
+        }).when(MetadataDrivers.class, "runFunctionWithRegistrationManager", any(ServerConfiguration.class),
+                any(Function.class));
+
+        Versioned cookie = mock(Versioned.class);
+        PowerMockito.whenNew(Versioned.class).withParameterTypes(Object.class, Version.class)
+                .withArguments(any(Cookie.class), eq(new LongVersion(1L))).thenReturn(cookie);
+
+        PowerMockito.mockStatic(Cookie.class);
+        when(Cookie.readFromRegistrationManager(rm, conf)).thenReturn((Versioned<Cookie>) cookie);
+
+        when(cookie.getValue()).thenReturn(mock(Cookie.class));
+    }
+
+    /**
+     * Test different type of command flags.
+     */
+    @Test
+    public void testNonInteraction() {
+        testCommand("-n");
+    }
+
+    @Test
+    public void testNonInteractionLongArgs() {
+        testCommand("--noninteractive");
+    }
+
+    @Test
+    public void testForce() {
+        testCommand("-f");
+    }
+
+    @Test
+    public void testForceLongArgs() {
+        testCommand("--force");
+    }
+
+    @Test
+    public void testDeleteCookie() {
+        testCommand("-d");
+    }
+
+    @Test
+    public void testDeleteCookieLongArgs() {
+        testCommand("--deletecookie");
+    }
+
+    @Test
+    public void testAllCommand() {
+        testCommand("-n", "-f", "-d");
+    }
+
+    @Test
+    public void testAllCommandLongArgs() {
+        testCommand("--noninteractive", "--force", "--deletecookie");
+    }
+
+    private void testCommand(String... args) {
+        FormatCommand cmd = new FormatCommand();
+        try {
+            assertTrue(cmd.apply(bkFlags, args));
+        } catch (Exception e) {
+            fail("Should not throw any exception here");
+        }
+    }
+
+}