fix bookie decommission sleep timeout value is negative bug

when decommission a bookie, and the ledger size of the bookie is big enough, the thread timeout will get negative, and the decommission operation will give up by throw exceptions as follow
```
14:12:56.982 [main] INFO  org.apache.bookkeeper.client.BookKeeperAdmin - Count of Ledgers which need to be rereplicated: 272752
14:12:56.983 [main] ERROR org.apache.bookkeeper.bookie.BookieShell - Received exception in DecommissionBookieCmd 
java.lang.IllegalArgumentException: timeout value is negative
	at java.lang.Thread.sleep(Native Method) ~[?:?]
	at org.apache.bookkeeper.client.BookKeeperAdmin.waitForLedgersToBeReplicated(BookKeeperAdmin.java:1528) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
	at org.apache.bookkeeper.client.BookKeeperAdmin.decommissionBookie(BookKeeperAdmin.java:1500) ~[org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
	at org.apache.bookkeeper.bookie.BookieShell$DecommissionBookieCmd.runCmd(BookieShell.java:2664) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
	at org.apache.bookkeeper.bookie.BookieShell$MyCommand.runCmd(BookieShell.java:277) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
	at org.apache.bookkeeper.bookie.BookieShell.run(BookieShell.java:3081) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
	at org.apache.bookkeeper.bookie.BookieShell.main(BookieShell.java:3172) [org.apache.bookkeeper-bookkeeper-server-4.9.2.jar:4.9.2]
14:12:57.013 [main] INFO  org.apache.zookeeper.ZooKeeper - Session: 0x206189927840052 closed
```
The exception code is 
```
private void waitForLedgersToBeReplicated(Collection<Long> ledgers, BookieSocketAddress thisBookieAddress,
            LedgerManager ledgerManager) throws InterruptedException, TimeoutException {
        int maxSleepTimeInBetweenChecks = 10 * 60 * 1000; // 10 minutes
        int sleepTimePerLedger = 10 * 1000; // 10 secs
        Predicate<Long> validateBookieIsNotPartOfEnsemble = ledgerId -> !areEntriesOfLedgerStoredInTheBookie(ledgerId,
                thisBookieAddress, ledgerManager);
        while (!ledgers.isEmpty()) {
            LOG.info("Count of Ledgers which need to be rereplicated: {}", ledgers.size());
            int sleepTimeForThisCheck = ledgers.size() * sleepTimePerLedger > maxSleepTimeInBetweenChecks
                    ? maxSleepTimeInBetweenChecks : ledgers.size() * sleepTimePerLedger;
            Thread.sleep(sleepTimeForThisCheck);
            LOG.debug("Making sure following ledgers replication to be completed: {}", ledgers);
            ledgers.removeIf(validateBookieIsNotPartOfEnsemble);
        }
    }
```
the ledger size is `272752`, when computing sleepTimeForThisCheck, 
`ledgers.size() * sleepTimePerLedger` is `272752 * 10 * 1000 = 2727520000`, 
the value exceeds max int value `2147483647`, it will turn to `-1567447296`, then the sleepTimeForThisCheck will be `-1567447296`.
 Thread.sleep will throw `java.lang.IllegalArgumentException: timeout value is negative` exception

Reviewers: Enrico Olivelli <eolivelli@gmail.com>, Jia Zhai <zhaijia@apache.org>

This closes #2284 from hangc0276/bug_fix
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
index 88a7c08..cac1d9d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
@@ -1536,7 +1536,7 @@
                 thisBookieAddress, ledgerManager);
         while (!ledgers.isEmpty()) {
             LOG.info("Count of Ledgers which need to be rereplicated: {}", ledgers.size());
-            int sleepTimeForThisCheck = ledgers.size() * sleepTimePerLedger > maxSleepTimeInBetweenChecks
+            int sleepTimeForThisCheck = (long) ledgers.size() * sleepTimePerLedger > maxSleepTimeInBetweenChecks
                     ? maxSleepTimeInBetweenChecks : ledgers.size() * sleepTimePerLedger;
             Thread.sleep(sleepTimeForThisCheck);
             LOG.debug("Making sure following ledgers replication to be completed: {}", ledgers);