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/03 06:18:56 UTC

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

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 bfbd6b0  Migrate command `decommission`
bfbd6b0 is described below

commit bfbd6b023b68ca0c23bf976274279ac8338181f5
Author: Yong Zhang <zh...@gmail.com>
AuthorDate: Wed Apr 3 14:18:52 2019 +0800

    Migrate command `decommission`
    
    Descriptions of the changes in this PR:
    
    #2049
    
    Reviewers: Sijie Guo <si...@apache.org>
    
    This closes #2050 from zymap/command-decommissionbookie
---
 .../org/apache/bookkeeper/bookie/BookieShell.java  |  41 ++-----
 .../cli/commands/bookies/DecommissionCommand.java  | 126 +++++++++++++++++++
 .../tools/cli/commands/BookiesCommandGroup.java    |   2 +
 .../commands/bookies/DecommissionCommandTest.java  | 135 +++++++++++++++++++++
 4 files changed, 270 insertions(+), 34 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 0d8dea5..870d31b 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
@@ -106,6 +106,7 @@ import org.apache.bookkeeper.tools.cli.commands.bookie.ReadJournalCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.ReadLogMetadataCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.SanityTestCommand;
+import org.apache.bookkeeper.tools.cli.commands.bookies.DecommissionCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookies.MetaFormatCommand;
@@ -2240,40 +2241,12 @@ public class BookieShell implements Tool {
 
         @Override
         public int runCmd(CommandLine cmdLine) throws Exception {
-            ClientConfiguration adminConf = new ClientConfiguration(bkConf);
-            BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
-            try {
-                final String remoteBookieidToDecommission = cmdLine.getOptionValue("bookieid");
-                final BookieSocketAddress bookieAddressToDecommission = (StringUtils
-                        .isBlank(remoteBookieidToDecommission) ? Bookie.getBookieAddress(bkConf)
-                                : new BookieSocketAddress(remoteBookieidToDecommission));
-                admin.decommissionBookie(bookieAddressToDecommission);
-                LOG.info("The ledgers stored in the given decommissioning bookie: {} are properly replicated",
-                        bookieAddressToDecommission);
-                runFunctionWithRegistrationManager(bkConf, rm -> {
-                    try {
-                        Versioned<Cookie> cookie = Cookie.readFromRegistrationManager(rm, bookieAddressToDecommission);
-                        cookie.getValue().deleteFromRegistrationManager(rm, bookieAddressToDecommission,
-                                cookie.getVersion());
-                    } catch (CookieNotFoundException nne) {
-                        LOG.warn("No cookie to remove for the decommissioning bookie: {}, it could be deleted already",
-                                bookieAddressToDecommission, nne);
-                    } catch (BookieException be) {
-                        throw new UncheckedExecutionException(be.getMessage(), be);
-                    }
-                    return 0;
-                });
-                LOG.info("Cookie of the decommissioned bookie: {} is deleted successfully",
-                        bookieAddressToDecommission);
-                return 0;
-            } catch (Exception e) {
-                LOG.error("Received exception in DecommissionBookieCmd ", e);
-                return -1;
-            } finally {
-                if (admin != null) {
-                    admin.close();
-                }
-            }
+            DecommissionCommand cmd = new DecommissionCommand();
+            DecommissionCommand.DecommissionFlags flags = new DecommissionCommand.DecommissionFlags();
+            final String remoteBookieidToDecommission = cmdLine.getOptionValue("bookieid");
+            flags.remoteBookieIdToDecommission(remoteBookieidToDecommission);
+            boolean result = cmd.apply(bkConf, flags);
+            return (result) ? 0 : -1;
         }
     }
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java
new file mode 100644
index 0000000..c7aa537
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommand.java
@@ -0,0 +1,126 @@
+/*
+ * 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.bookies;
+
+import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithRegistrationManager;
+
+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.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.Cookie;
+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.net.BookieSocketAddress;
+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.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Command to trigger AuditTask by resetting lostBookieRecoveryDelay and
+ * then make sure the ledgers stored in the bookie are properly replicated
+ * and Cookie of the decommissioned bookie should be deleted from metadata
+ * server.
+ */
+public class DecommissionCommand extends BookieCommand<DecommissionCommand.DecommissionFlags> {
+
+    static final Logger LOG = LoggerFactory.getLogger(DecommissionCommand.class);
+
+    private static final String NAME = "decommission";
+    private static final String DESC =
+        "Force trigger the Audittask and make sure all the ledgers stored in the decommissioning bookie"
+        + " are replicated and cookie of the decommissioned bookie is deleted from metadata server.";
+
+    public DecommissionCommand() {
+        this(new DecommissionFlags());
+    }
+
+    private DecommissionCommand(DecommissionFlags flags) {
+        super(CliSpec.<DecommissionFlags>newBuilder().withName(NAME).withDescription(DESC).withFlags(flags).build());
+    }
+
+    /**
+     * Flags for decommission command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class DecommissionFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieid" }, description = "Decommission a remote bookie")
+        private String remoteBookieIdToDecommission;
+
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, DecommissionFlags cmdFlags) {
+        try {
+            return decommission(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean decommission(ServerConfiguration conf, DecommissionFlags flags)
+        throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            final String remoteBookieidToDecommission = flags.remoteBookieIdToDecommission;
+            final BookieSocketAddress bookieAddressToDecommission = (StringUtils.isBlank(remoteBookieidToDecommission)
+                                                                         ? Bookie.getBookieAddress(conf)
+                                                                         : new BookieSocketAddress(
+                                                                             remoteBookieidToDecommission));
+            admin.decommissionBookie(bookieAddressToDecommission);
+            LOG.info("The ledgers stored in the given decommissioning bookie: {} are properly replicated",
+                     bookieAddressToDecommission);
+            runFunctionWithRegistrationManager(conf, rm -> {
+                try {
+                    Versioned<Cookie> cookie = Cookie.readFromRegistrationManager(rm, bookieAddressToDecommission);
+                    cookie.getValue().deleteFromRegistrationManager(rm, bookieAddressToDecommission,
+                                                                    cookie.getVersion());
+                } catch (BookieException.CookieNotFoundException nne) {
+                    LOG.warn("No cookie to remove for the decommissioning bookie: {}, it could be deleted already",
+                             bookieAddressToDecommission, nne);
+                } catch (BookieException be) {
+                    throw new UncheckedExecutionException(be.getMessage(), be);
+                }
+                return true;
+            });
+            LOG.info("Cookie of the decommissioned bookie: {} is deleted successfully",
+                     bookieAddressToDecommission);
+            return true;
+        } catch (Exception e) {
+            LOG.error("Received exception in DecommissionBookieCmd ", e);
+            return false;
+        } finally {
+            if (admin != null) {
+                admin.close();
+            }
+        }
+    }
+}
diff --git a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java
index 710a2fe..1f4d4cf 100644
--- a/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.java
+++ b/tools/ledger/src/main/java/org/apache/bookkeeper/tools/cli/commands/BookiesCommandGroup.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.bookies.DecommissionCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookies.InfoCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookies.InitCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookies.ListBookiesCommand;
@@ -47,6 +48,7 @@ public class BookiesCommandGroup extends CliCommandGroup<BKFlags> {
         .addCommand(new InfoCommand())
         .addCommand(new NukeExistingClusterCommand())
         .addCommand(new MetaFormatCommand())
+        .addCommand(new DecommissionCommand())
         .addCommand(new InitCommand())
         .build();
 
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java
new file mode 100644
index 0000000..7ce5a02
--- /dev/null
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookies/DecommissionCommandTest.java
@@ -0,0 +1,135 @@
+/*
+ * 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.bookies;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+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 java.util.function.Function;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.Cookie;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.tools.cli.helpers.BookieCommandTestBase;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+/**
+ * Unit test for {@link DecommissionCommand}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ DecommissionCommand.class, Bookie.class, MetadataDrivers.class, Cookie.class })
+public class DecommissionCommandTest extends BookieCommandTestBase {
+
+    @Mock
+    private ClientConfiguration clientConfiguration;
+
+    @Mock
+    private BookKeeperAdmin bookKeeperAdmin;
+
+    @Mock
+    private BookieSocketAddress bookieSocketAddress;
+
+    @Mock
+    private Versioned<Cookie> cookieVersioned;
+
+    @Mock
+    private Cookie cookie;
+
+    @Mock
+    private Version version;
+    public DecommissionCommandTest() {
+        super(3, 0);
+    }
+
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+
+        PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf);
+        PowerMockito.whenNew(ClientConfiguration.class).withArguments(eq(conf)).thenReturn(clientConfiguration);
+        PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(ClientConfiguration.class)
+                    .withArguments(eq(clientConfiguration)).thenReturn(bookKeeperAdmin);
+        PowerMockito.whenNew(BookieSocketAddress.class).withArguments(anyString()).thenReturn(bookieSocketAddress);
+        PowerMockito.mockStatic(Bookie.class);
+        PowerMockito.when(Bookie.getBookieAddress(any(ServerConfiguration.class))).thenReturn(bookieSocketAddress);
+        PowerMockito.doNothing().when(bookKeeperAdmin).decommissionBookie(eq(bookieSocketAddress));
+
+        RegistrationManager registrationManager = mock(RegistrationManager.class);
+        PowerMockito.mockStatic(MetadataDrivers.class);
+        PowerMockito.doAnswer(invocationOnMock -> {
+            Function<RegistrationManager, ?> f = invocationOnMock.getArgument(1);
+            f.apply(registrationManager);
+            return true;
+        }).when(MetadataDrivers.class, "runFunctionWithRegistrationManager", any(ServerConfiguration.class),
+                any(Function.class));
+
+        PowerMockito.mockStatic(Cookie.class);
+        PowerMockito.when(Cookie.readFromRegistrationManager(eq(registrationManager), eq(bookieSocketAddress)))
+                    .thenReturn(cookieVersioned);
+        when(cookieVersioned.getValue()).thenReturn(cookie);
+        when(cookieVersioned.getVersion()).thenReturn(version);
+        PowerMockito.doNothing().when(cookie)
+                    .deleteFromRegistrationManager(eq(registrationManager), eq(bookieSocketAddress), eq(version));
+    }
+
+    @Test
+    public void testWithoutBookieId() throws Exception {
+        DecommissionCommand cmd = new DecommissionCommand();
+        Assert.assertTrue(cmd.apply(bkFlags, new String[] { "" }));
+
+        verifyNew(ClientConfiguration.class, times(1)).withArguments(eq(conf));
+        verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(clientConfiguration));
+        verifyNew(BookieSocketAddress.class, never()).withArguments(anyString());
+        verify(bookKeeperAdmin, times(1)).decommissionBookie(eq(bookieSocketAddress));
+        verify(cookieVersioned, times(1)).getValue();
+        verify(cookieVersioned, times(1)).getVersion();
+    }
+
+    @Test
+    public void testWithBookieId() throws Exception {
+        DecommissionCommand cmd = new DecommissionCommand();
+        Assert.assertTrue(cmd.apply(bkFlags, new String[] { "-b", "1" }));
+
+        verifyNew(ClientConfiguration.class, times(1)).withArguments(eq(conf));
+        verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(clientConfiguration));
+        verifyNew(BookieSocketAddress.class, times(1)).withArguments(anyString());
+        verify(bookKeeperAdmin, times(1)).decommissionBookie(eq(bookieSocketAddress));
+        verify(cookieVersioned, times(1)).getValue();
+        verify(cookieVersioned, times(1)).getVersion();
+    }
+}