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/10 01:42:09 UTC

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

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 06f2b6f  Migrate command `updateledgers`
06f2b6f is described below

commit 06f2b6f50ca2c6c1ccc630aa9d9fd761abf1becc
Author: Yong Zhang <zh...@gmail.com>
AuthorDate: Wed Apr 10 09:42:04 2019 +0800

    Migrate command `updateledgers`
    
    Descriptions of the changes in this PR:
    
    #2057
    
    ```
    Update bookie id in ledgers (this may take a long time).
    
    Usage:  bkctl ledger update [flags]
    
    Flags:
    
        -host, --hostname
            Expects configuration useHostNameAsBookieID=true as the optin value
    
        -l, --limit
            Maximum number of ledgers of ledgers to update (default: no limit)
    
        -p, --printprogress
            Print messages on every configured seconds if verbose turned on
            (default: 10 secs)
    
        -v, --verbose
            Print status of the ledger updation (default: false)
    
        -s, updatepersec
            Number of ledgers updating per second (default: 5 per sec)
    
    
        -h, --help
            Display help information
    ```
    
    
    Reviewers: Jia Zhai <zh...@apache.org>, Sijie Guo <si...@apache.org>
    
    This closes #2058 from zymap/command-updateledgers
---
 .../org/apache/bookkeeper/bookie/BookieShell.java  |  62 ++------
 .../cli/commands/bookie/FlipBookieIdCommand.java   | 165 +++++++++++++++++++++
 .../cli/commands/client/DeleteLedgerCommand.java   |   2 +-
 .../tools/cli/commands/BookieCommandGroup.java     |   2 +
 .../commands/bookie/FlipBookieIdCommandTest.java   | 108 ++++++++++++++
 5 files changed, 288 insertions(+), 51 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 4de4aa7..f3f83a3 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
@@ -46,15 +46,12 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 import org.apache.bookkeeper.bookie.BookieException.CookieNotFoundException;
 import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException;
 import org.apache.bookkeeper.bookie.storage.ldb.LocationsIndexRebuildOp;
-import org.apache.bookkeeper.client.BookKeeper;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
 import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.UpdateLedgerOp;
 import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.common.annotation.InterfaceAudience.Private;
 import org.apache.bookkeeper.conf.ClientConfiguration;
@@ -70,6 +67,7 @@ import org.apache.bookkeeper.tools.cli.commands.autorecovery.ListUnderReplicated
 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.FlipBookieIdCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
@@ -102,7 +100,6 @@ import org.apache.bookkeeper.util.BookKeeperConstants;
 import org.apache.bookkeeper.util.EntryFormatter;
 import org.apache.bookkeeper.util.IOUtils;
 import org.apache.bookkeeper.util.LedgerIdFormatter;
-import org.apache.bookkeeper.util.MathUtils;
 import org.apache.bookkeeper.util.Tool;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
@@ -1788,6 +1785,9 @@ public class BookieShell implements Tool {
 
         @Override
         int runCmd(CommandLine cmdLine) throws Exception {
+            FlipBookieIdCommand cmd = new FlipBookieIdCommand();
+            FlipBookieIdCommand.FlipBookieIdFlags flags = new FlipBookieIdCommand.FlipBookieIdFlags();
+
             final String bookieId = cmdLine.getOptionValue("bookieId");
             if (StringUtils.isBlank(bookieId)) {
                 LOG.error("Invalid argument list!");
@@ -1800,23 +1800,8 @@ public class BookieShell implements Tool {
                 return -1;
             }
             boolean useHostName = getOptionalValue(bookieId, "hostname");
-            if (!bkConf.getUseHostNameAsBookieID() && useHostName) {
-                LOG.error("Expects configuration useHostNameAsBookieID=true as the option value passed is 'hostname'");
-                return -1;
-            } else if (bkConf.getUseHostNameAsBookieID() && !useHostName) {
-                LOG.error("Expects configuration useHostNameAsBookieID=false as the option value passed is 'ip'");
-                return -1;
-            }
             final int rate = getOptionIntValue(cmdLine, "updatespersec", 5);
-            if (rate <= 0) {
-                LOG.error("Invalid updatespersec {}, should be > 0", rate);
-                return -1;
-            }
             final int limit = getOptionIntValue(cmdLine, "limit", Integer.MIN_VALUE);
-            if (limit <= 0 && limit != Integer.MIN_VALUE) {
-                LOG.error("Invalid limit {}, should be > 0", limit);
-                return -1;
-            }
             final boolean verbose = getOptionBooleanValue(cmdLine, "verbose", false);
             final long printprogress;
             if (!verbose) {
@@ -1828,37 +1813,14 @@ public class BookieShell implements Tool {
                 // defaulting to 10 seconds
                 printprogress = getOptionLongValue(cmdLine, "printprogress", 10);
             }
-            final ClientConfiguration conf = new ClientConfiguration();
-            conf.addConfiguration(bkConf);
-            final BookKeeper bk = new BookKeeper(conf);
-            final BookKeeperAdmin admin = new BookKeeperAdmin(conf);
-            final UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, admin);
-            final ServerConfiguration serverConf = new ServerConfiguration(bkConf);
-            final BookieSocketAddress newBookieId = Bookie.getBookieAddress(serverConf);
-            serverConf.setUseHostNameAsBookieID(!useHostName);
-            final BookieSocketAddress oldBookieId = Bookie.getBookieAddress(serverConf);
-
-            UpdateLedgerNotifier progressable = new UpdateLedgerNotifier() {
-                long lastReport = System.nanoTime();
-
-                @Override
-                public void progress(long updated, long issued) {
-                    if (printprogress <= 0) {
-                        return; // disabled
-                    }
-                    if (TimeUnit.MILLISECONDS.toSeconds(MathUtils.elapsedMSec(lastReport)) >= printprogress) {
-                        LOG.info("Number of ledgers issued={}, updated={}", issued, updated);
-                        lastReport = MathUtils.nowInNano();
-                    }
-                }
-            };
-            try {
-                updateLedgerOp.updateBookieIdInLedgers(oldBookieId, newBookieId, rate, limit, progressable);
-            } catch (IOException e) {
-                LOG.error("Failed to update ledger metadata", e);
-                return -1;
-            }
-            return 0;
+            flags.hostname(useHostName);
+            flags.printProgress(printprogress);
+            flags.limit(limit);
+            flags.updatePerSec(rate);
+            flags.verbose(verbose);
+
+            boolean result = cmd.apply(bkConf, flags);
+            return (result) ? 0 : -1;
         }
     }
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java
new file mode 100644
index 0000000..852379f
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommand.java
@@ -0,0 +1,165 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import lombok.Setter;
+import lombok.experimental.Accessors;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieShell;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.UpdateLedgerOp;
+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.util.MathUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Command to update ledger command.
+ */
+public class FlipBookieIdCommand extends BookieCommand<FlipBookieIdCommand.FlipBookieIdFlags> {
+
+    static final Logger LOG = LoggerFactory.getLogger(FlipBookieIdCommand.class);
+
+    private static final String NAME = "flip-bookie-id";
+    private static final String DESC = "Update bookie id in ledgers (this may take a long time).";
+
+    public FlipBookieIdCommand() {
+        this(new FlipBookieIdFlags());
+    }
+
+    private FlipBookieIdCommand(FlipBookieIdFlags flags) {
+        super(CliSpec.<FlipBookieIdFlags>newBuilder()
+                  .withName(NAME)
+                  .withDescription(DESC)
+                  .withFlags(flags)
+                  .build());
+    }
+
+    /**
+     * Flags for update ledger command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class FlipBookieIdFlags extends CliFlags {
+
+        @Parameter(names = { "-host", "--hostname" },
+            description = "Expects configuration useHostNameAsBookieID=true as the option value (default: ip address)")
+        private boolean hostname;
+
+        @Parameter(names = { "-s", "--updatepersec" },
+            description = "Number of ledgers updating per second (default: 5 per sec)")
+        private int updatePerSec = 5;
+
+        @Parameter(names = {"-l", "--limit"},
+            description = "Maximum number of ledgers of ledgers to update (default: no limit)")
+        private int limit = Integer.MIN_VALUE;
+
+        @Parameter(names = { "-v", "--verbose" }, description = "Print status of the ledger updation (default: false)")
+        private boolean verbose;
+
+        @Parameter(names = { "-p", "--printprogress" },
+            description = "Print messages on every configured seconds if verbose turned on (default: 10 secs)")
+        private long printProgress = 10;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, FlipBookieIdFlags cmdFlags) {
+        try {
+            return updateLedger(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean updateLedger(ServerConfiguration conf, FlipBookieIdFlags flags)
+        throws InterruptedException, BKException, IOException {
+
+        if (!conf.getUseHostNameAsBookieID() && flags.hostname) {
+            LOG.error("Expects configuration useHostNameAsBookieID=true as the option value");
+            return false;
+        } else if (conf.getUseHostNameAsBookieID() && !flags.hostname) {
+            LOG.error("Expects configuration useHostNameAsBookieID=false as the option value'");
+            return false;
+        }
+
+        final int rate = flags.updatePerSec;
+        if (rate <= 0) {
+            LOG.error("Invalid updatespersec {}, should be > 0", rate);
+            return false;
+        }
+
+        final int limit = flags.limit;
+        if (limit <= 0 && limit != Integer.MIN_VALUE) {
+            LOG.error("Invalid limit {}, should be > 0", limit);
+            return false;
+        }
+
+        final long printProgress;
+        if (flags.verbose) {
+            printProgress = 10;
+        } else {
+            printProgress = flags.printProgress;
+        }
+
+        final ClientConfiguration clientConfiguration = new ClientConfiguration();
+        clientConfiguration.addConfiguration(conf);
+        final BookKeeper bk = new BookKeeper(clientConfiguration);
+        final BookKeeperAdmin admin = new BookKeeperAdmin(bk);
+        final UpdateLedgerOp updateLedgerOp = new UpdateLedgerOp(bk, admin);
+        final ServerConfiguration serverConfiguration = new ServerConfiguration(conf);
+        final BookieSocketAddress newBookieId = Bookie.getBookieAddress(serverConfiguration);
+        serverConfiguration.setUseHostNameAsBookieID(!flags.hostname);
+        final BookieSocketAddress oldBookieId = Bookie.getBookieAddress(serverConfiguration);
+
+        BookieShell.UpdateLedgerNotifier progressable = new BookieShell.UpdateLedgerNotifier() {
+            long lastReport = System.nanoTime();
+
+            @Override
+            public void progress(long updated, long issued) {
+                if (printProgress <= 0) {
+                    return; // disabled
+                }
+                if (TimeUnit.MILLISECONDS.toSeconds(MathUtils.elapsedMSec(lastReport)) >= printProgress) {
+                    LOG.info("Number of ledgers issued={}, updated={}", issued, updated);
+                    lastReport = MathUtils.nowInNano();
+                }
+            }
+        };
+
+        try {
+            updateLedgerOp.updateBookieIdInLedgers(oldBookieId, newBookieId, rate, limit, progressable);
+        } catch (IOException e) {
+            LOG.error("Failed to update ledger metadata", e);
+            return false;
+        }
+
+        return true;
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java
index 70ffee6..410f47b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/client/DeleteLedgerCommand.java
@@ -38,7 +38,7 @@ import org.apache.bookkeeper.util.LedgerIdFormatter;
  */
 public class DeleteLedgerCommand extends BookieCommand<DeleteLedgerCommand.DeleteLedgerFlags> {
 
-    private static final String NAME = "deleteledger";
+    private static final String NAME = "delete";
     private static final String DESC = "Delete a ledger.";
     private static final String DEFAULT = "";
 
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 9e94cd2..d7eca51 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
@@ -23,6 +23,7 @@ import static org.apache.bookkeeper.tools.common.BKCommandCategories.CATEGORY_IN
 import org.apache.bookkeeper.tools.cli.BKCtl;
 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.FlipBookieIdCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.FormatCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.InitCommand;
 import org.apache.bookkeeper.tools.cli.commands.bookie.LastMarkCommand;
@@ -67,6 +68,7 @@ public class BookieCommandGroup extends CliCommandGroup<BKFlags> {
         .addCommand(new ReadLogCommand())
         .addCommand(new ReadLogMetadataCommand())
         .addCommand(new LocalConsistencyCheckCommand())
+        .addCommand(new FlipBookieIdCommand())
         .addCommand(new RegenerateInterleavedStorageIndexFileCommand())
         .build();
 
diff --git a/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java
new file mode 100644
index 0000000..a57a758
--- /dev/null
+++ b/tools/ledger/src/test/java/org/apache/bookkeeper/tools/cli/commands/bookie/FlipBookieIdCommandTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.verify;
+import static org.mockito.internal.verification.VerificationModeFactory.times;
+import static org.powermock.api.mockito.PowerMockito.verifyNew;
+
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.UpdateLedgerOp;
+import org.apache.bookkeeper.conf.AbstractConfiguration;
+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.BookieCommandTestBase;
+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 FlipBookieIdCommand}.
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({ FlipBookieIdCommand.class, Bookie.class, UpdateLedgerOp.class })
+public class FlipBookieIdCommandTest extends BookieCommandTestBase {
+
+    @Mock
+    private ClientConfiguration clientConfiguration;
+
+    @Mock
+    private BookKeeper bookKeeper;
+
+    @Mock
+    private BookKeeperAdmin bookKeeperAdmin;
+
+    @Mock
+    private UpdateLedgerOp updateLedgerOp;
+
+    @Mock
+    private ServerConfiguration serverConfiguration;
+
+    @Mock
+    private BookieSocketAddress bookieSocketAddress;
+
+    public FlipBookieIdCommandTest() {
+        super(3, 0);
+    }
+
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+
+        PowerMockito.whenNew(ServerConfiguration.class).withNoArguments().thenReturn(conf);
+        PowerMockito.whenNew(ClientConfiguration.class).withNoArguments().thenReturn(clientConfiguration);
+        PowerMockito.whenNew(BookKeeper.class).withParameterTypes(ClientConfiguration.class)
+                    .withArguments(eq(clientConfiguration)).thenReturn(bookKeeper);
+        PowerMockito.whenNew(BookKeeperAdmin.class).withParameterTypes(BookKeeper.class).withArguments(eq(bookKeeper))
+                    .thenReturn(bookKeeperAdmin);
+        PowerMockito.whenNew(UpdateLedgerOp.class).withArguments(eq(bookKeeper), eq(bookKeeperAdmin))
+                    .thenReturn(updateLedgerOp);
+        PowerMockito.whenNew(ServerConfiguration.class).withParameterTypes(AbstractConfiguration.class)
+                    .withArguments(eq(conf)).thenReturn(serverConfiguration);
+        PowerMockito.mockStatic(Bookie.class);
+        PowerMockito.when(Bookie.getBookieAddress(eq(serverConfiguration))).thenReturn(bookieSocketAddress);
+    }
+
+    @Test
+    public void testCommand() throws Exception {
+        FlipBookieIdCommand cmd = new FlipBookieIdCommand();
+        Assert.assertTrue(cmd.apply(bkFlags, new String[] { "" }));
+        verifyNew(ClientConfiguration.class, times(1)).withNoArguments();
+        verify(clientConfiguration, times(1)).addConfiguration(eq(conf));
+        verifyNew(BookKeeper.class, times(1)).withArguments(eq(clientConfiguration));
+        verifyNew(BookKeeperAdmin.class, times(1)).withArguments(eq(bookKeeper));
+        verifyNew(UpdateLedgerOp.class, times(1)).withArguments(eq(bookKeeper), eq(bookKeeperAdmin));
+        verifyNew(ServerConfiguration.class, times(1)).withArguments(eq(conf));
+        verify(serverConfiguration, times(1)).setUseHostNameAsBookieID(anyBoolean());
+        verify(updateLedgerOp, times(1))
+            .updateBookieIdInLedgers(eq(bookieSocketAddress), eq(bookieSocketAddress), anyInt(), anyInt(), any());
+    }
+
+}