You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2022/08/26 11:48:46 UTC

[GitHub] [bookkeeper] lordcheng10 opened a new pull request, #3457: Support non-stop bookie data migration and bookie offline

lordcheng10 opened a new pull request, #3457:
URL: https://github.com/apache/bookkeeper/pull/3457

   ### Motivation
   bookie offline steps:
   
   1. Log on to the bookie node, check if there are underreplicated ledgers.If there are, the decommission command will force them to be replicated: bin/bookkeeper shell listunderreplicated
   2. Stop the bookie : bin/bookkeeper-daemon.sh stop bookie
   3. Run the decommission command. If you have logged onto the node you wish to decommission, you don't need to provide -bookieid If you are running the decommission command for target bookie node from another bookie node you should mention the target bookie id in the arguments for -bookieid : bin/bookkeeper shell decommissionbookie or $ bin/bookkeeper shell decommissionbookie -bookieid <target bookieid>
   4. Validate that there are no ledgers on decommissioned bookie $ bin/bookkeeper shell listledgers -bookieid <target bookieid>
   
   **For the current bookie offline solution, need to stop the bookie first,execute the decommission command and wait for the ledger migration on the bookie to complete.**
   
   **It can be seen that it is very time-consuming to offline a bookie node. When we need to offline a lot of bookie nodes, the time-consuming of this solution will not be acceptable**
   
   **Therefore, we need a solution that can migrate data without stopping bookie, so that bookie nodes can be offlined in batches**
   
   
   ### Changes
   
   1. Implement a command to submit the bookie to be offline and the corresponding ledgers, for example:
   ```
   bin/bookkeeper shell decommissionbookie -offline_bookieids  bookieId1,bookieId2,bookieId3,...bookieIdn
   ```
      This command will write all ledgers on the offline bookie node to the zookeeper directory, for example:
   
    ` put ledgers/offline_replicas/ledgerId  bookId1,bookId2,...bookIdn;`
   
   2. Design a **OfflineReplicasWorker** class to perform the actual ledger replication: this class will obtain a ledger from the zookeeper directory ledgers/offline_replicas for replication. Before copying the data, it will first **filter out all the fragments containing the offline bookieId under the ledger,then copy these fragments**;
   
   In addition, define a concept of Replicas: the replica data of a ledger on one or more bookies.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1241416161

   > There's some checkstyle error,please update the code like this @lordcheng10
   > 
   > <img alt="image" width="2000" src="https://user-images.githubusercontent.com/42990025/189253818-32cc3430-96f6-473f-a02c-e9e2121e4a42.png">
   
   OK, I will fix!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242050181

   ### Motivation
   **Currently bookkeeper does not support data migration, only data recovery.**
   
   We have a scenario where a large number of bookie nodes are offline, and we find that the bookie's offline solution is very time-consuming.
   
   Bookie offline steps are as follows:
   1. Set the bookie node to be offline to readOnly;
   2. Wait for the Pulsar data on the Bookie node to be offline to expire and delete;
   3. When most of the data on these offline nodes is expired and cleaned up, there will still be some data that cannot be expired and deleted;
   4. Stop a bookie, and then use the decommission command to migrate the data that has not been expired and cleaned up to the new node:
   `bin/bookkeeper shell decommissionbookie -bookieid xx`
   5. When the data on one bookie node is migrated, continue to the next bookie node;
   
   Step 4 is very time-consuming. We found that waiting for a bookie data migration to complete, it takes about 1 hour, and we have 125 bookie nodes to be offline.
   
   In addition, step 2 is also very time-consuming, depending on the pulsar retain time, usually more than ten hours.
   
   ### Proposal
   To solve this problem, we developed a data migration tool.
   After having this tool, our offline steps are as follows:
   1. Execute the data migration command:
   `bin/bookkeeper shell replicasMigration --bookieIds bookie1,bookie2 --ledgerIds ALL --readOnly true`
   2. When the data migration is completed, stop all bookie nodes to be offline;
   
   In addition, this command can also migrate the replica data on some bookie nodes to other nodes, for example:
   
   `bin/bookkeeper shell replicasMigration --bookieIds bookie1,bookie2 --ledgerIds ledger1,ledger2,ledger3 --readOnly false`
   
   
   ### For example
   1. Migrate all ledger data on bookie1 and bookie2 to other bookie nodes:
   `sh bin/bookkeeper shell replicasMigration -bookieIds bookie1,bookie2 -ledgerIds ALL -readOnly true
   `
   2. Migrate ledger1 and ledger3 on bookie1 and bookie2 to other bookie nodes:
   `sh bin/bookkeeper shell replicasMigration -bookieIds bookie1,bookie2 -ledgerIds ledger1,ledger3 -readOnly false`
   
   
   ### Application scenarios:
   **1. The bookie node goes offline:**
   As mentioned above, after bookkeeper has this data migration tool, the offline steps of bookie are only two steps, and the time-consuming is greatly reduced:
   a.Execute the data migration command:
   bin/bookkeeper shell replicasMigration --bookieIds bookie1,bookie2 --ledgerIds ALL --readOnly true
   b. When the data migration is completed, stop all bookie nodes to be offline;
   
   **2. Expand the bookie node to improve the reading speed of historical data:**
   a. When the client consumes historical data a few days ago, it hopes to increase the reading speed of historical data by expanding the bookie node.
   b. When we expand the new bookie node to the cluster, the new node can only receive the read and write of new data, and cannot improve the reading speed of historical data.
   c. After the data migration tool of bookkeeper, we can migrate some historical data to the new node, let the new node provide some historical data reading, and improve the reading speed of historical data.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] Shoothzj commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
Shoothzj commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242913172

   Hi @lordcheng10 , could you please explain more **how you do the data migration** in the description?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970293663


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,

Review Comment:
   OK, I will  fix!
   <img width="1117" alt="image" src="https://user-images.githubusercontent.com/19296967/190059205-2ff05bfb-3a73-4c0d-aa32-d3b80d43d3d3.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1253133016

   > In general, for the "The bookie node goes offline" use case i'd prefer if this was done via autorecovery: introduce bookie state like "draining" - read only for the client point of view, down for auditor/autorecovery purposes and use regular AR.
   @dlg99 
   
   1. The offline of the bookie node is just one of the cases solved by this data migration tool.
   
   2.IMO, bookkeeper should have the ability to migrate data, because only data recovery is currently supported.
   
   3. The data migration tool not only supports migrating all the data of one or more bookie nodes, but also supports migrating part of the data on one bookie node to other nodes.
   
   4. There are two cases that we use this tool to solve:
   ①. The bookie node goes offline;
   
   ②. Expand the bookie node to improve the reading speed of historical data:
   a. When the client consumes historical data a few days ago, it hopes to increase the reading speed of historical data by expanding the bookie node.
   b. When we expand the new bookie node to the cluster, the new node can only receive the read and write of new data, and cannot improve the reading speed of historical data.
   c. After the data migration tool of bookkeeper, we can migrate some historical data to the new node, let the new node provide some historical data reading, and improve the reading speed of historical data.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970393759


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Set<BookieId> bookieIds = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+            Set<Long> ledgerIdsToMigrate = null;
+            if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                ledgerIdsToMigrate = parseLedgerIds(flags.ledgerIdsOfMigratedReplica);
+            }
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIds);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {

Review Comment:
   admin.listLedgers(bookieIds) needs to be called whether it is ALL or not. It is currently used to build the mapping structure of ledger and bookies: toMigratedLedgerAndBookieMap.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975973829


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {

Review Comment:
   @zymap 
   It is @hangc0276 's suggestion. When some bookie nodes fail to set readonly, they need to fall back to the previous state:
   <img width="914" alt="image" src="https://user-images.githubusercontent.com/19296967/191399590-b8ffcb5d-651d-4fa6-b4db-8fce5c7ca1a9.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r990795552


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,77 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            Option bookiesOpt = new Option("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            Option ledgersOpt = new Option("l", "ledgerIds", true,

Review Comment:
   Fixed! PTAL,thanks! @zymap 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {

Review Comment:
   Fixed! PTAL,thanks! @zymap 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975974336


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,50 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    /**
+     * Get the list of ledgers corresponding to the replica to be migrated.
+     * */
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();

Review Comment:
   OK, I will fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r1006325702


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/MigrationReplicasCommand.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MigrationManager;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class MigrationReplicasCommand extends BookieCommand<MigrationReplicasCommand.MigrationReplicasFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(MigrationReplicasCommand.class);
+    private static final String NAME = "migrationReplicas";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    public MigrationReplicasCommand() {
+        this(new MigrationReplicasFlags());
+    }
+    private MigrationReplicasCommand(MigrationReplicasFlags flags) {
+        super(CliSpec.<MigrationReplicasFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for migrationReplicas command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class MigrationReplicasFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, MigrationReplicasFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+                toSwitchBookieIdsSet.removeAll(readOnlyBookies);
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {
+                    LOG.warn("Some bookie nodes that fail to set readonly, "
+                            + "and the successful bookies fall back to the previous state!");
+                    Set<BookieId> fallbackSuccessfulBookies = new HashSet<>();
+                    if (!switchToReadonly(admin, switchedBookies, fallbackSuccessfulBookies, false)) {
+                        switchedBookies.removeAll(fallbackSuccessfulBookies);
+                        LOG.error(String.format("Fallback failed! Failed nodes:%s", switchedBookies));
+                    }
+                    return false;
+                }
+            }
+
+            return migrationReplicas(admin, conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean switchToReadonly(BookKeeperAdmin admin,

Review Comment:
   rename to "switchBookieStatus" 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r971452857


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);

Review Comment:
   OK, I will fix!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242830153

   @michaeljmarshall PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242029522

   rerun failure checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r968245622


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java:
##########
@@ -88,6 +89,13 @@ default CompletableFuture<Void> disableHealthCheck() {
         return result;
     }
 
+    default CompletableFuture<Void> submitToMigrateReplicas(long ledgerId, String bookieIds)
+            throws UnsupportedEncodingException {

Review Comment:
   Yes, you are right!
   fixed!



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            //1. Get a replica ledger to replicate
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (KeeperException e) {
+                LOG.error("getReplicasToMigrate throw exception!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            //2. Get the fragment containing the migration bookie

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r969780125


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java:
##########
@@ -289,6 +297,57 @@ public void processResult(int rc, String path, Object ctx, String name) {
         return createResult;
     }
 
+    private String getLedgerForReplicasMigrationPath(long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    public CompletableFuture<Void> submitToMigrateReplicas(long ledgerId, String bookieIds) {
+        CompletableFuture<Void> createResult = new CompletableFuture<>();
+        zk.create(getLedgerForReplicasMigrationPath(ledgerId), bookieIds.getBytes(StandardCharsets.UTF_8), acls,

Review Comment:
   We'd better use `zk.multi()` operation to reduce the ZK RPC times.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            // 1. Get a replica ledger to replicate
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (KeeperException e) {
+                LOG.error("getReplicasToMigrate throw exception!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            // 2. Get the fragment containing the migration bookie
+            try (LedgerHandle lh = admin.openLedgerNoRecovery(replicas.ledgerId)) {

Review Comment:
   We need to introduce throttle for the replicate to control the IO impact to the cluster.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Set<BookieId> bookieIds = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+            Set<Long> ledgerIdsToMigrate = null;
+            if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                ledgerIdsToMigrate = parseLedgerIds(flags.ledgerIdsOfMigratedReplica);
+            }
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIds);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {

Review Comment:
   We'd better check 'ALL' first to reduce the call of `admin.listLedgers()`



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);

Review Comment:
   We get the whole list for each time and pick one ledger from the earliest, it will cost a lot of resources. Would you consider to tune it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1245517092

   > we also need tests, especially tests about recovering a partial migration that was interrupted
   
   @eolivelli 
   
   In the process of migrating data, stop the bookie process responsible for the migration task? After stopping the corresponding bookie process, other bookie processes will grab the ledger replication task and continue the replication, so I need to verify whether the ledger migration task can be completed normally? For example, check the final number of copies of the ledger and the distribution of copies on the bookie node after migration?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r990579865


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {

Review Comment:
   OK, I will fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242936546

   > 
   
   If we want to migrate all ledger data on some bookies, we need to set these bookies to read only first, otherwise during the migration process, new data may be written to these bookies


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970276702


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;

Review Comment:
   OK , I will fix.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];

Review Comment:
   OK , I will fix.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970506440


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;

Review Comment:
   Fixed. 
   PTAL,thanks!  @hangc0276 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;

Review Comment:
   Fixed. 
   PTAL,thanks!  @hangc0276 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970506050


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;

Review Comment:
   Fixed. 
   PTAL,thanks!  @hangc0276 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;

Review Comment:
   Fixed. 
   PTAL,thanks!  @hangc0276 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;

Review Comment:
   Fixed. 
   PTAL,thanks!  @hangc0276 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970473282


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            // 1. Get a replica ledger to replicate
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (KeeperException e) {
+                LOG.error("getReplicasToMigrate throw exception!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            // 2. Get the fragment containing the migration bookie
+            try (LedgerHandle lh = admin.openLedgerNoRecovery(replicas.ledgerId)) {

Review Comment:
   We can control the speed of data migration by configuring throttle:
   <img width="1159" alt="image" src="https://user-images.githubusercontent.com/19296967/190098404-9bc53d04-8204-40db-8fe1-7c553e472b8a.png">
   <img width="812" alt="image" src="https://user-images.githubusercontent.com/19296967/190098793-5ca50781-89d1-47ef-a65a-fcb5ec93a09c.png">
   <img width="1056" alt="image" src="https://user-images.githubusercontent.com/19296967/190098877-3fd35e04-8cdc-48be-8df2-0699aca19b7b.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r1006331618


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,77 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class MigrationReplicasCmd extends MyCommand {
+
+        MigrationReplicasCmd() {
+            super(CMD_MIGRATIONREPLICAS);
+            Option bookiesOpt = new Option("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            Option ledgersOpt = new Option("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            Option readOnlyOpt = new Option("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");

Review Comment:
   Fixed.
   PTAL,thanks! @hangc0276 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r976010336


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,77 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            Option bookiesOpt = new Option("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            Option ledgersOpt = new Option("l", "ledgerIds", true,

Review Comment:
   Here ledgersOpt and readOnlyOpt are optional by default, and bookiesOpt is required:
   <img width="1102" alt="image" src="https://user-images.githubusercontent.com/19296967/191411023-65a8bf40-31e3-4e19-b54a-df0aeabfe2fa.png">
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r968247139


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/api/v1/bookie/state/readonly", httpAddress);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");

Review Comment:
   @Shoothzj PTAL,thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1241952479

   @hangc0276 @eolivelli @zymap PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r968245811


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            //1. Get a replica ledger to replicate
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (KeeperException e) {
+                LOG.error("getReplicasToMigrate throw exception!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            //2. Get the fragment containing the migration bookie
+            try (LedgerHandle lh = admin.openLedgerNoRecovery(replicas.ledgerId)) {
+                Set<LedgerFragment> fragments = getFragmentsOnMigrationBookies(lh, replicas.bookieIds);
+                if (fragments.size() < 1) {
+                    //3.The replication has been completed, delete the ledgerId directory and release lock
+                    releaseLock(replicas);
+                    metadataClientDriver.deleteZkPath(getLedgerForMigrationReplicasPath(replicasMigrationPath,
+                            replicas.ledgerId));
+                    LOG.info(String.format("Finish ledgerId %s migration!", replicas.ledgerId));
+                }
+
+                //4. Start replicate

Review Comment:
   fixed



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/api/v1/bookie/state/readonly", httpAddress);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Set<BookieId> bookieIds = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+            Set<Long> ledgerIdsToMigrate = null;
+            if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                ledgerIdsToMigrate = parseLedgerIds(flags.ledgerIdsOfMigratedReplica);
+            }
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIds);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                    ledgersOnBookieId.retainAll(ledgerIdsToMigrate);
+                }
+                for (Long ledgerId : ledgersOnBookieId) {
+                    Set<BookieId> bookieIdSet = toMigratedLedgerAndBookieMap.getOrDefault(ledgerId, new HashSet<>());
+                    bookieIdSet.add(bookieIdSetEntry.getKey());
+                    toMigratedLedgerAndBookieMap.put(ledgerId, bookieIdSet);
+                }
+            }
+
+            MetadataDrivers.runFunctionWithMetadataBookieDriver(conf, driver -> {
+                for (Map.Entry<Long, Set<BookieId>> entry : toMigratedLedgerAndBookieMap.entrySet()) {
+                    Long ledgerId = entry.getKey();
+                    String bookies = StringUtils.join(entry.getValue(), seperator);
+                    try {
+                        driver.submitToMigrateReplicas(ledgerId, bookies).get();
+                    } catch (InterruptedException e) {
+                        throw new RuntimeException(e);
+                    } catch (Throwable e) {
+                        LOG.warn(String.format("The migration task of Replicas(ledgerId=%s,bookieIds=%s) "
+                                + "submit failed!", ledgerId, bookies), e);
+                    }
+                }
+                return true;
+            });
+
+            return true;
+        } catch (Exception e) {
+            LOG.error("Received exception in ReplicasMigrationCommand ", e);
+            return false;
+        } finally {
+            if (admin != null) {

Review Comment:
   fixed



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/api/v1/bookie/state/readonly", httpAddress);

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r968246603


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/api/v1/bookie/state/readonly", httpAddress);

Review Comment:
   @Shoothzj PTAL,thanks!



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/api/v1/bookie/state/readonly", httpAddress);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Set<BookieId> bookieIds = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+            Set<Long> ledgerIdsToMigrate = null;
+            if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                ledgerIdsToMigrate = parseLedgerIds(flags.ledgerIdsOfMigratedReplica);
+            }
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIds);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                    ledgersOnBookieId.retainAll(ledgerIdsToMigrate);
+                }
+                for (Long ledgerId : ledgersOnBookieId) {
+                    Set<BookieId> bookieIdSet = toMigratedLedgerAndBookieMap.getOrDefault(ledgerId, new HashSet<>());
+                    bookieIdSet.add(bookieIdSetEntry.getKey());
+                    toMigratedLedgerAndBookieMap.put(ledgerId, bookieIdSet);
+                }
+            }
+
+            MetadataDrivers.runFunctionWithMetadataBookieDriver(conf, driver -> {
+                for (Map.Entry<Long, Set<BookieId>> entry : toMigratedLedgerAndBookieMap.entrySet()) {
+                    Long ledgerId = entry.getKey();
+                    String bookies = StringUtils.join(entry.getValue(), seperator);
+                    try {
+                        driver.submitToMigrateReplicas(ledgerId, bookies).get();
+                    } catch (InterruptedException e) {
+                        throw new RuntimeException(e);
+                    } catch (Throwable e) {
+                        LOG.warn(String.format("The migration task of Replicas(ledgerId=%s,bookieIds=%s) "
+                                + "submit failed!", ledgerId, bookies), e);
+                    }
+                }
+                return true;
+            });
+
+            return true;
+        } catch (Exception e) {
+            LOG.error("Received exception in ReplicasMigrationCommand ", e);
+            return false;
+        } finally {
+            if (admin != null) {

Review Comment:
   @Shoothzj PTAL,thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970393759


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Set<BookieId> bookieIds = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+            Set<Long> ledgerIdsToMigrate = null;
+            if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                ledgerIdsToMigrate = parseLedgerIds(flags.ledgerIdsOfMigratedReplica);
+            }
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIds);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {

Review Comment:
   admin.listLedgers(bookieIds) needs to be called whether it is ALL or not. It is currently used to build the mapping structure of ledger and bookies: toMigratedLedgerAndBookieMap.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r971458160


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);

Review Comment:
   Fixed.
   PTAL,thanks! @hangc0276



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970287244


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,

Review Comment:
   @hanggc0276
   The readOnly default value has been set to false here:
   <img width="1025" alt="image" src="https://user-images.githubusercontent.com/19296967/190057439-77a31577-67a5-4d15-b5d7-c12bb3c354c7.png">
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970505834


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");
+            opts.addOption("p", "httpPort", true, "http port,default is 8080");

Review Comment:
   Fixed. 
   PTAL,thanks!  @hangc0276 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,

Review Comment:
   Fixed. 
   PTAL,thanks!  @hangc0276 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970479928


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            // 1. Get a replica ledger to replicate
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (KeeperException e) {
+                LOG.error("getReplicasToMigrate throw exception!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            // 2. Get the fragment containing the migration bookie
+            try (LedgerHandle lh = admin.openLedgerNoRecovery(replicas.ledgerId)) {

Review Comment:
   The original speed limit function can be reused:
   
   <img width="903" alt="image" src="https://user-images.githubusercontent.com/19296967/190100441-b28162ba-8186-45f5-80b4-fa66c7f6060d.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1253136059

   > Overall I am ok with the approach but:
   > 
   > PR needs tests for the new feature
   > PR needs tests that show that GC won't consider ledger as over replicated and delete the data if this kicks in at the same time as migration
   > PR needs tests to show that Autorecovery can co-exist with this (e.g. ensemble size 5, migrating two bookies, plus one goes down -> AR starts rereplication, can this end up in inconsistent metadata?)
   > PR needs tests for migration of an open ledger (ledger created, still open, some data written, never closed, no new writes, try to migrate it)
   > possibly more tests for variety of AR/GC cases (replication worker crashes mid-migration, bookie restarts mid-migration etc)
   > http API for the bookie to switch it to read-only is optional (may be disabled), that should be ok.
   > the problem is that the read-only status set by the REST API is not persisted between bookie restarts IIRC (do double check me), so the bookie can restart mid-migration and become writable.
   > 
   > These are the areas where data loss can occur and extra attention is required.
   
   Thanks for your suggestion! Excellent!
   I will test these cases you mentioned.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r990580437


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {

Review Comment:
   If the server is not enabled, subsequent http requests will fail, and the command will be terminated:
   <img width="1378" alt="image" src="https://user-images.githubusercontent.com/19296967/194685505-66aa77af-3a13-46d7-8da7-8459a63d852c.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242575031

   @hangc0276 @eolivelli @zymap PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r969740929


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,27 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();
+    }
+
+    /**
+     * After obtaining a replica migration task,
+     * lock the replica task to prevent it from being executed by other workers.
+     * */
+    default void lockMigrationReplicas(String lockPath, String advertisedAddress)
+            throws InterruptedException, KeeperException {
+    }
+
+    default String getOwnerBookiesMigrationReplicas(String ledgerForMigrationReplicasPath)
+            throws InterruptedException, KeeperException, UnsupportedEncodingException {
+        return "";

Review Comment:
   fixed! @eolivelli 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,27 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();
+    }
+
+    /**
+     * After obtaining a replica migration task,
+     * lock the replica task to prevent it from being executed by other workers.
+     * */
+    default void lockMigrationReplicas(String lockPath, String advertisedAddress)
+            throws InterruptedException, KeeperException {
+    }
+
+    default String getOwnerBookiesMigrationReplicas(String ledgerForMigrationReplicasPath)
+            throws InterruptedException, KeeperException, UnsupportedEncodingException {
+        return "";
+    }
+
+    default void deleteZkPath(String migrationLedgerPath) throws InterruptedException, KeeperException {
+    }

Review Comment:
   fixed! @eolivelli 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r969710040


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;

Review Comment:
   @eolivelli 
   Do you suggest to get the http port of the corresponding bookie from zookeeper?
   
   But sometimes the bookie node may also be down.
   So it may be better to let the user input the http port, if you do not enter the http port, use the default port 8080



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r969710607


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,27 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();
+    }
+
+    /**
+     * After obtaining a replica migration task,
+     * lock the replica task to prevent it from being executed by other workers.
+     * */
+    default void lockMigrationReplicas(String lockPath, String advertisedAddress)
+            throws InterruptedException, KeeperException {
+    }
+
+    default String getOwnerBookiesMigrationReplicas(String ledgerForMigrationReplicasPath)
+            throws InterruptedException, KeeperException, UnsupportedEncodingException {
+        return "";

Review Comment:
   OK, I will fix!



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,27 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();
+    }
+
+    /**
+     * After obtaining a replica migration task,
+     * lock the replica task to prevent it from being executed by other workers.
+     * */
+    default void lockMigrationReplicas(String lockPath, String advertisedAddress)
+            throws InterruptedException, KeeperException {
+    }
+
+    default String getOwnerBookiesMigrationReplicas(String ledgerForMigrationReplicasPath)
+            throws InterruptedException, KeeperException, UnsupportedEncodingException {
+        return "";
+    }
+
+    default void deleteZkPath(String migrationLedgerPath) throws InterruptedException, KeeperException {
+    }

Review Comment:
   OK, I will fix!



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/util/HttpClientUtils.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.util;
+
+import java.io.IOException;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+
+/**
+ * http client utils.
+ * */
+public class HttpClientUtils {
+    private static final CloseableHttpClient httpClient = HttpClientBuilder.create().build();

Review Comment:
   OK, I will fix!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r969747288


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;

Review Comment:
   Use `List<String>` instead of "," separate String.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,

Review Comment:
   1. required = false?
   2. We'd better set the default readOnly flag to `false`. If the admin uses this command to decommission some bookies, which configured placement policy, it may lead to writing new entries failing due to not meeting placement policy.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;

Review Comment:
   Use `List<Long>` instead of `String`



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Set<BookieId> bookieIds = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+            Set<Long> ledgerIdsToMigrate = null;
+            if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                ledgerIdsToMigrate = parseLedgerIds(flags.ledgerIdsOfMigratedReplica);
+            }
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIds);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                    ledgersOnBookieId.retainAll(ledgerIdsToMigrate);

Review Comment:
   `ledgerIdsToMigrate ` maybe null



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;

Review Comment:
   If we want to decommission 3 bookies, <bookie0, bookie1, bookie2>, and <bookie0, bookie1> set to readOnly mode succeed, and bookie2 set to readOnly failed, do we need to rollback <bookie0, bookie1> to the previous mode?



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;

Review Comment:
   Please configure the default value to false.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;

Review Comment:
   The type is `int`?



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {

Review Comment:
   Suggest `try (BookKeeperAdmin admin = new BookKeeperAdmin(adminConf))`



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");
+            opts.addOption("p", "httpPort", true, "http port,default is 8080");

Review Comment:
   required = false?



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];

Review Comment:
   +1, we need to use `resolve` the ID



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242054084

   @Shoothzj PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242936588

   > > 1. sh bin/bookkeeper shell replicasMigration -bookieIds bookie1,bookie2 -ledgerIds ALL -readOnly true
   > 
   > @Shoothzj
   > 
   > The data migration steps are as follows:
   > 
   > 1. Submit the ledger copy to be offline to zookeeper through the ReplicasMigrationCommand command. The directory organization on zookeeper is as follows:
   > 
   > > `ledgers/replicasMigration/ledgerId1`
   > 
   > And record the bookie node where the ledger1 wants to migrate data on the zookeeper directory, for example:
   > 
   > > `set ledgers/replicasMigration/ledgerId1 bookie1,bookie3`
   > 
   > 2. Start the corresponding data replication service ReplicasMigrationWorker in AutoRecoveryMain. The ReplicasMigrationWorker service first obtains a ledger copy migration task, then finds the fragments stored by the ledger on the corresponding bookie node, and finally replicates these fragments through the replicateLedgerFragment method.
   > 3. When a ledger copy migration task is completed, the corresponding ledger directory on zookeeper will be deleted, for example:
   > 
   > > `delete ledgers/replicasMigration/ledgerId1`
   
   If we want to migrate all ledger data on some bookies, we need to set these bookies to read only first, otherwise during the migration process, new data may be written to these bookies


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r968246858


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java:
##########
@@ -88,6 +89,13 @@ default CompletableFuture<Void> disableHealthCheck() {
         return result;
     }
 
+    default CompletableFuture<Void> submitToMigrateReplicas(long ledgerId, String bookieIds)

Review Comment:
   @Shoothzj PTAL,thanks!



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java:
##########
@@ -88,6 +89,13 @@ default CompletableFuture<Void> disableHealthCheck() {
         return result;
     }
 
+    default CompletableFuture<Void> submitToMigrateReplicas(long ledgerId, String bookieIds)
+            throws UnsupportedEncodingException {

Review Comment:
   @Shoothzj PTAL,thanks!



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            //1. Get a replica ledger to replicate

Review Comment:
   @Shoothzj PTAL,thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r990795693


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java:
##########
@@ -289,6 +302,52 @@ public void processResult(int rc, String path, Object ctx, String name) {
         return createResult;
     }
 
+    private String getLedgerForReplicasMigrationPath(long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);

Review Comment:
   Fixed! PTAL,thanks! @zymap 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java:
##########
@@ -239,6 +251,7 @@ protected void initialize(AbstractConfiguration<?> conf,
         }
 
         disableHealthCheckPath = ledgersRootPath + "/" + DISABLE_HEALTH_CHECK;
+        replicasMigrationPath = ledgersRootPath + "/" + MIGRATION_REPLICAS;

Review Comment:
   Fixed! PTAL,thanks! @zymap 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975974800


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java:
##########
@@ -289,6 +302,52 @@ public void processResult(int rc, String path, Object ctx, String name) {
         return createResult;
     }
 
+    private String getLedgerForReplicasMigrationPath(long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);

Review Comment:
   OK, I will fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r1006305665


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/MigrationReplicasCommand.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MigrationManager;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class MigrationReplicasCommand extends BookieCommand<MigrationReplicasCommand.MigrationReplicasFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(MigrationReplicasCommand.class);
+    private static final String NAME = "migrationReplicas";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    public MigrationReplicasCommand() {
+        this(new MigrationReplicasFlags());
+    }
+    private MigrationReplicasCommand(MigrationReplicasFlags flags) {
+        super(CliSpec.<MigrationReplicasFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for migrationReplicas command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class MigrationReplicasFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, MigrationReplicasFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+                toSwitchBookieIdsSet.removeAll(readOnlyBookies);
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {
+                    LOG.warn("Some bookie nodes that fail to set readonly, "
+                            + "and the successful bookies fall back to the previous state!");
+                    Set<BookieId> fallbackSuccessfulBookies = new HashSet<>();
+                    if (!switchToReadonly(admin, switchedBookies, fallbackSuccessfulBookies, false)) {
+                        switchedBookies.removeAll(fallbackSuccessfulBookies);
+                        LOG.error(String.format("Fallback failed! Failed nodes:%s", switchedBookies));
+                    }
+                    return false;
+                }
+            }
+
+            return migrationReplicas(admin, conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean switchToReadonly(BookKeeperAdmin admin,

Review Comment:
   OK,I Will fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1248839028

   > discuss thread:https://lists.apache.org/thread/0zlsm7z5j34j23gtlhf7rf72lfq62jyk vote thread: https://lists.apache.org/thread/bw4b08bhposh3gk1xzfk4sskxkjdx53b
   
   @eolivelli @hangc0276 @StevenLuMT @Shoothzj PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1250731032

   @hangc0276 PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975973829


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {

Review Comment:
   It is @hangc0276 's suggestion. When some bookie nodes fail to set readonly, they need to fall back to the previous state:
   <img width="914" alt="image" src="https://user-images.githubusercontent.com/19296967/191399590-b8ffcb5d-651d-4fa6-b4db-8fce5c7ca1a9.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975972313


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {

Review Comment:
   OK , I will fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r990795656


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {
+                    LOG.warn("Some bookie nodes that fail to set readonly, "
+                            + "and the successful bookies fall back to the previous state!");
+                    Set<BookieId> fallbackSuccessfulBookies = new HashSet<>();
+                    switchedBookies.removeAll(readOnlyBookies);
+                    if (!switchToReadonly(admin, switchedBookies, fallbackSuccessfulBookies, false)) {

Review Comment:
   Fixed! PTAL,thanks! @zymap 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,50 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    /**
+     * Get the list of ledgers corresponding to the replica to be migrated.
+     * */
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();

Review Comment:
   Fixed! PTAL,thanks! @zymap 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242053667

   @dlg99 @merlimat PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242574951

   @lhotari @merlimat PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970504620


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,27 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();
+    }
+
+    /**
+     * After obtaining a replica migration task,
+     * lock the replica task to prevent it from being executed by other workers.
+     * */
+    default void lockMigrationReplicas(String lockPath, String advertisedAddress)
+            throws InterruptedException, KeeperException {
+    }
+
+    default String getOwnerBookiesMigrationReplicas(String ledgerForMigrationReplicasPath)
+            throws InterruptedException, KeeperException, UnsupportedEncodingException {
+        return "";
+    }
+
+    default void deleteZkPath(String migrationLedgerPath) throws InterruptedException, KeeperException {
+    }

Review Comment:
   fixed. PTAL,thanks! @eolivelli



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970505070


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,27 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();
+    }
+
+    /**
+     * After obtaining a replica migration task,
+     * lock the replica task to prevent it from being executed by other workers.
+     * */
+    default void lockMigrationReplicas(String lockPath, String advertisedAddress)
+            throws InterruptedException, KeeperException {
+    }
+
+    default String getOwnerBookiesMigrationReplicas(String ledgerForMigrationReplicasPath)
+            throws InterruptedException, KeeperException, UnsupportedEncodingException {
+        return "";

Review Comment:
   fixed. PTAL,thanks! @eolivelli



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1245554062

   > we also need tests, especially tests about recovering a partial migration that was interrupted
   
   @eolivelli 
   
   How do I need to verify that it is as expected?
   
   We have done some simple verification in the test cluster, by checking the number of replicas and bookie distribution after migration of ledger replicas.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1282760010

   @hangc0276 PTAL, thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r1006325965


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/MigrationReplicasCommand.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MigrationManager;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class MigrationReplicasCommand extends BookieCommand<MigrationReplicasCommand.MigrationReplicasFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(MigrationReplicasCommand.class);
+    private static final String NAME = "migrationReplicas";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    public MigrationReplicasCommand() {
+        this(new MigrationReplicasFlags());
+    }
+    private MigrationReplicasCommand(MigrationReplicasFlags flags) {
+        super(CliSpec.<MigrationReplicasFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for migrationReplicas command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class MigrationReplicasFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, MigrationReplicasFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+                toSwitchBookieIdsSet.removeAll(readOnlyBookies);
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {
+                    LOG.warn("Some bookie nodes that fail to set readonly, "
+                            + "and the successful bookies fall back to the previous state!");
+                    Set<BookieId> fallbackSuccessfulBookies = new HashSet<>();
+                    if (!switchToReadonly(admin, switchedBookies, fallbackSuccessfulBookies, false)) {
+                        switchedBookies.removeAll(fallbackSuccessfulBookies);
+                        LOG.error(String.format("Fallback failed! Failed nodes:%s", switchedBookies));
+                    }
+                    return false;
+                }
+            }
+
+            return migrationReplicas(admin, conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean switchToReadonly(BookKeeperAdmin admin,

Review Comment:
   Fixed.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r1006324566


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,77 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class MigrationReplicasCmd extends MyCommand {
+
+        MigrationReplicasCmd() {
+            super(CMD_MIGRATIONREPLICAS);
+            Option bookiesOpt = new Option("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            Option ledgersOpt = new Option("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            Option readOnlyOpt = new Option("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");

Review Comment:
   The true here is not the default value, the corresponding parameter name is hasArg:
   <img width="974" alt="image" src="https://user-images.githubusercontent.com/19296967/198170890-ed0ee963-9f06-4170-846b-95642e26f17a.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1241401976

   There's some checkstyle error,please update the code like this @lordcheng10 
   
   <img width="2175" alt="image" src="https://user-images.githubusercontent.com/42990025/189253818-32cc3430-96f6-473f-a02c-e9e2121e4a42.png">
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242167114

   rerun failure checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1244784802

   rerun failure checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970506635


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {

Review Comment:
   Fixed. 
   PTAL,thanks!  @hangc0276 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Set<BookieId> bookieIds = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+            Set<Long> ledgerIdsToMigrate = null;
+            if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                ledgerIdsToMigrate = parseLedgerIds(flags.ledgerIdsOfMigratedReplica);
+            }
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIds);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                    ledgersOnBookieId.retainAll(ledgerIdsToMigrate);

Review Comment:
   Fixed. 
   PTAL,thanks!  @hangc0276 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] zymap commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
zymap commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r974062764


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {

Review Comment:
   Please add test



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {

Review Comment:
   It is a server configuration, the client doesn't know whether the HTTP server is enabled or not.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,77 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            Option bookiesOpt = new Option("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            Option ledgersOpt = new Option("l", "ledgerIds", true,

Review Comment:
   IIUC, in most cases, you want to migrate all the data from one bookie to another bookie, not part of the data. So why do we require the ledgerIds? Shouldn't it be optional?



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {

Review Comment:
   We can remove the read-only bookies from the switching bookie list first, then switch the remaining to the read-only.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,237 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {

Review Comment:
   MigrationReplicas? Please be consistent with all the places.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {
+                    LOG.warn("Some bookie nodes that fail to set readonly, "
+                            + "and the successful bookies fall back to the previous state!");
+                    Set<BookieId> fallbackSuccessfulBookies = new HashSet<>();
+                    switchedBookies.removeAll(readOnlyBookies);
+                    if (!switchToReadonly(admin, switchedBookies, fallbackSuccessfulBookies, false)) {
+                        switchedBookies.removeAll(fallbackSuccessfulBookies);
+                        LOG.error(String.format("Fallback failed! Failed nodes:%s", switchedBookies));
+                    }
+                    return false;
+                }
+            }
+
+            return replicasMigration(admin, conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean switchToReadonly(BookKeeperAdmin admin,
+                                     Set<BookieId> toSwitchBookieIdsSet,
+                                     Set<BookieId> switchedBookieIds,
+                                     boolean toReadOnly) throws BKException {
+        for (BookieId bookieId : toSwitchBookieIdsSet) {
+            BookieServiceInfo bookieServiceInfo = admin.getBookieServiceInfo(bookieId);
+            for (BookieServiceInfo.Endpoint endpoint : bookieServiceInfo.getEndpoints()) {
+                if (endpoint.getProtocol().equals("http")) {
+                    String host = endpoint.getHost();
+                    int port = endpoint.getPort();
+                    String url = String.format("http://%s:%s/%s", host, port, HttpRouter.BOOKIE_STATE_READONLY);
+                    String readOnly = toReadOnly ? "true" : "false";
+                    CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+                    int responseCode = response.getStatusLine().getStatusCode();
+                    if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                        LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                                readOnly, bookieId, responseCode));
+                        return false;
+                    }
+
+                    switchedBookieIds.add(bookieId);
+                    LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                            readOnly, bookieId, responseCode));
+                }
+            }
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(BookKeeperAdmin admin, ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException {
+        try {
+            // 1. Build bookieIdsSet adn ledgerIdsToMigrate
+            Set<BookieId> bookieIdsSet = new HashSet<>();
+            for (String bookieIdStr : flags.bookieIdsOfMigratedReplica) {
+                bookieIdsSet.add(BookieId.parse(bookieIdStr));
+            }
+            HashSet<Long> ledgerIdsToMigrate = new HashSet<>();
+            for (Long ledgerId : flags.ledgerIdsOfMigratedReplica) {
+                ledgerIdsToMigrate.add(ledgerId);
+            }
+
+            // 2. Build toMigratedLedgerAndBookieMap
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIdsSet);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!ledgerIdsToMigrate.isEmpty()) {
+                    ledgersOnBookieId.retainAll(ledgerIdsToMigrate);
+                }
+                for (Long ledgerId : ledgersOnBookieId) {
+                    Set<BookieId> bookieIdSet = toMigratedLedgerAndBookieMap.getOrDefault(ledgerId, new HashSet<>());
+                    bookieIdSet.add(bookieIdSetEntry.getKey());
+                    toMigratedLedgerAndBookieMap.put(ledgerId, bookieIdSet);
+                }
+            }
+
+            // 3. Submit ledger replicas to be migrated
+            MetadataDrivers.runFunctionWithMetadataBookieDriver(conf, driver -> {
+                try {
+                    driver.submitToMigrateReplicas(toMigratedLedgerAndBookieMap);
+                } catch (Throwable e) {
+                    LOG.warn(String.format("The migration task of Replicas=%s submit failed!",
+                            toMigratedLedgerAndBookieMap), e);
+                }
+                return true;

Review Comment:
   This method looks like don't need returned value?



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,237 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate(List<String> ledgerIds) throws InterruptedException, KeeperException {
+        Iterator<String> iterator = ledgerIds.iterator();
+        while (iterator.hasNext()) {
+            try {
+                String ledgerId = iterator.next();
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(replicasMigrationPath,
+                        Long.parseLong(ledgerId));
+                if (!metadataClientDriver.exists(ledgerForMigrationReplicasPath)) {
+                    // The ledger migration is completed
+                    iterator.remove();
+                    continue;
+                }
+                String bookieIdStr = metadataClientDriver.
+                        getOwnerBookiesMigrationReplicas(ledgerForMigrationReplicasPath);
+
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(replicasMigrationPath,
+                        Long.parseLong(ledgerId), lock);
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have locked the ledger
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("The encoding is not supported!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        List<String> toMigrateLedgerIds = Collections.emptyList();
+        while (workerRunning) {
+            // 1. build migrating ledgers
+            try {
+                if (toMigrateLedgerIds.isEmpty()) {
+                    toMigrateLedgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (Throwable e) {
+                LOG.error("list migrating ledgers failed!", e);
+            }
+
+            // 2. Get a replica ledger to replicate or wait back off
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate(toMigrateLedgerIds);
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (Throwable e) {
+                LOG.error("Get migrating replicas failed!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            // 3. Get the fragment containing the migration bookie
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            try (LedgerHandle lh = admin.openLedgerNoRecovery(replicas.ledgerId)) {
+                Set<LedgerFragment> fragments = getFragmentsOnMigrationBookies(lh, replicas.bookieIds);
+                if (fragments.size() < 1) {
+                    //3.The replication has been completed, delete the ledgerId directory and release lock
+                    releaseLock(replicas);

Review Comment:
   It will release the lock twice.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/util/BookKeeperConstants.java:
##########
@@ -40,6 +40,7 @@ public class BookKeeperConstants {
     public static final String AVAILABLE_NODE = "available";
     public static final String COOKIE_NODE = "cookies";
     public static final String UNDER_REPLICATION_NODE = "underreplication";
+    public static final String MIGRATION_REPLICAS = "migrationReplicas";

Review Comment:
   ```suggestion
       public static final String MIGRATION_REPLICAS = "migratedReplicas";
   
   ```



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java:
##########
@@ -239,6 +251,7 @@ protected void initialize(AbstractConfiguration<?> conf,
         }
 
         disableHealthCheckPath = ledgersRootPath + "/" + DISABLE_HEALTH_CHECK;
+        replicasMigrationPath = ledgersRootPath + "/" + MIGRATION_REPLICAS;

Review Comment:
   Can we make the replicas migration as a single class like how underreplication does? We can create a MigrationManager to handle that.
   
   https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java#L88



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java:
##########
@@ -289,6 +302,52 @@ public void processResult(int rc, String path, Object ctx, String name) {
         return createResult;
     }
 
+    private String getLedgerForReplicasMigrationPath(long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);

Review Comment:
   Because it is similar to replication, we can refer to how replication located the ledger in the zookeeper. And we can consistent the ledgerId format with that.
   https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/ZkLedgerUnderreplicationManager.java#L252



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,50 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    /**
+     * Get the list of ledgers corresponding to the replica to be migrated.
+     * */
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();

Review Comment:
   I saw other methods throw an unsupported exception, we should do the same thing on this.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {
+                    LOG.warn("Some bookie nodes that fail to set readonly, "
+                            + "and the successful bookies fall back to the previous state!");
+                    Set<BookieId> fallbackSuccessfulBookies = new HashSet<>();
+                    switchedBookies.removeAll(readOnlyBookies);
+                    if (!switchToReadonly(admin, switchedBookies, fallbackSuccessfulBookies, false)) {

Review Comment:
   Is it necessary to retry it? Because it is a command line tool, rerunning the command should be ok?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975974260


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {
+                    LOG.warn("Some bookie nodes that fail to set readonly, "
+                            + "and the successful bookies fall back to the previous state!");
+                    Set<BookieId> fallbackSuccessfulBookies = new HashSet<>();
+                    switchedBookies.removeAll(readOnlyBookies);
+                    if (!switchToReadonly(admin, switchedBookies, fallbackSuccessfulBookies, false)) {

Review Comment:
   This is not a retry, but a fallback to the previous state



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,50 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    /**
+     * Get the list of ledgers corresponding to the replica to be migrated.
+     * */
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();

Review Comment:
   OK, I will fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975977623


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,237 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate(List<String> ledgerIds) throws InterruptedException, KeeperException {
+        Iterator<String> iterator = ledgerIds.iterator();
+        while (iterator.hasNext()) {
+            try {
+                String ledgerId = iterator.next();
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(replicasMigrationPath,
+                        Long.parseLong(ledgerId));
+                if (!metadataClientDriver.exists(ledgerForMigrationReplicasPath)) {
+                    // The ledger migration is completed
+                    iterator.remove();
+                    continue;
+                }
+                String bookieIdStr = metadataClientDriver.
+                        getOwnerBookiesMigrationReplicas(ledgerForMigrationReplicasPath);
+
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(replicasMigrationPath,
+                        Long.parseLong(ledgerId), lock);
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have locked the ledger
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("The encoding is not supported!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        List<String> toMigrateLedgerIds = Collections.emptyList();
+        while (workerRunning) {
+            // 1. build migrating ledgers
+            try {
+                if (toMigrateLedgerIds.isEmpty()) {
+                    toMigrateLedgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (Throwable e) {
+                LOG.error("list migrating ledgers failed!", e);
+            }
+
+            // 2. Get a replica ledger to replicate or wait back off
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate(toMigrateLedgerIds);
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (Throwable e) {
+                LOG.error("Get migrating replicas failed!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            // 3. Get the fragment containing the migration bookie
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            try (LedgerHandle lh = admin.openLedgerNoRecovery(replicas.ledgerId)) {
+                Set<LedgerFragment> fragments = getFragmentsOnMigrationBookies(lh, replicas.bookieIds);
+                if (fragments.size() < 1) {
+                    //3.The replication has been completed, delete the ledgerId directory and release lock
+                    releaseLock(replicas);

Review Comment:
   OK, I will fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r969741181


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/util/HttpClientUtils.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.util;
+
+import java.io.IOException;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+
+/**
+ * http client utils.
+ * */
+public class HttpClientUtils {
+    private static final CloseableHttpClient httpClient = HttpClientBuilder.create().build();

Review Comment:
   fixed! @eolivelli 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970409896


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;

Review Comment:
   remove port, resovle ID can get port



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1247547723

   rerun failure checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r969070057


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/api/v1/bookie/state/readonly", httpAddress);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");

Review Comment:
   The listbookies command is the zookeeper directory of the list, not the http request sent, so there is no way to reuse it!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1247500516

   @hangc0276 @eolivelli FIxed,PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r990795622


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {

Review Comment:
   Fixed! PTAL,thanks! @zymap 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {

Review Comment:
   Fixed! PTAL,thanks! @zymap 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970503163


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;

Review Comment:
   fixed.  PTAL,thanks! @eolivelli 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;

Review Comment:
   @eolivelli 
   Do you suggest to get the http port of the corresponding bookie from zookeeper?
   
   But sometimes the bookie node may also be down.
   So it may be better to let the user input the http port, if you do not enter the http port, use the default port 8080



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970479928


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            // 1. Get a replica ledger to replicate
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (KeeperException e) {
+                LOG.error("getReplicasToMigrate throw exception!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            // 2. Get the fragment containing the migration bookie
+            try (LedgerHandle lh = admin.openLedgerNoRecovery(replicas.ledgerId)) {

Review Comment:
   The original speed limit function can be reused:
   
   <img width="1116" alt="image" src="https://user-images.githubusercontent.com/19296967/190100122-a2139a05-6aa9-487c-81a4-ccdbdfe911b1.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970286197


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");
+            opts.addOption("p", "httpPort", true, "http port,default is 8080");

Review Comment:
   @hangc0276
    required default is false, I didn't set this parameter, I can set ths bookiesOpt  with true:
   <img width="1162" alt="image" src="https://user-images.githubusercontent.com/19296967/190057027-6fdd40bd-bd30-463b-844a-a0f132497da0.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1246130220

   rerun failure checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970523924


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Set<BookieId> bookieIds = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+            Set<Long> ledgerIdsToMigrate = null;
+            if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                ledgerIdsToMigrate = parseLedgerIds(flags.ledgerIdsOfMigratedReplica);
+            }
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIds);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {

Review Comment:
   Check 'ALL' first, does not reduce the call of admin.listLedgers()



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1247578526

   discuss thread:https://lists.apache.org/thread/0zlsm7z5j34j23gtlhf7rf72lfq62jyk
   vote thread:  https://lists.apache.org/thread/bw4b08bhposh3gk1xzfk4sskxkjdx53b


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970294108


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;

Review Comment:
   OK, I will fix!



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;

Review Comment:
   OK, I will fix!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975972660


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {

Review Comment:
   If it is not enabled, an exception will be thrown and the user will be notified.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975975734


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/util/BookKeeperConstants.java:
##########
@@ -40,6 +40,7 @@ public class BookKeeperConstants {
     public static final String AVAILABLE_NODE = "available";
     public static final String COOKIE_NODE = "cookies";
     public static final String UNDER_REPLICATION_NODE = "underreplication";
+    public static final String MIGRATION_REPLICAS = "migrationReplicas";

Review Comment:
   OK



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1241447069

   @StevenLuMT fixed,PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242935104

   > 1. sh bin/bookkeeper shell replicasMigration -bookieIds bookie1,bookie2 -ledgerIds ALL -readOnly true
   
   The data migration steps are as follows:
   1. Submit the ledger copy to be offline to zookeeper through the ReplicasMigrationCommand command. The directory organization on zookeeper is as follows:
   
   ledgers/replicasMigration/ledgerId1
   
   And record the bookie node where the ledger1 wants to migrate data on the zookeeper directory, for example:
   
   set ledgers/replicasMigration/ledgerId1 bookie1,bookie3
   
   2. Start the corresponding data replication service ReplicasMigrationWorker in AutoRecoveryMain. The ReplicasMigrationWorker service first obtains a ledger copy migration task, then finds the fragments stored by the ledger on the corresponding bookie node, and finally replicates these fragments through the replicateLedgerFragment method.
   
   3. When a ledger copy migration task is completed, the corresponding ledger directory on zookeeper will be deleted, for example:
   delete ledgers/replicasMigration/ledgerId1


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975977082


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {
+                    LOG.warn("Some bookie nodes that fail to set readonly, "
+                            + "and the successful bookies fall back to the previous state!");
+                    Set<BookieId> fallbackSuccessfulBookies = new HashSet<>();
+                    switchedBookies.removeAll(readOnlyBookies);
+                    if (!switchToReadonly(admin, switchedBookies, fallbackSuccessfulBookies, false)) {
+                        switchedBookies.removeAll(fallbackSuccessfulBookies);
+                        LOG.error(String.format("Fallback failed! Failed nodes:%s", switchedBookies));
+                    }
+                    return false;
+                }
+            }
+
+            return replicasMigration(admin, conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean switchToReadonly(BookKeeperAdmin admin,
+                                     Set<BookieId> toSwitchBookieIdsSet,
+                                     Set<BookieId> switchedBookieIds,
+                                     boolean toReadOnly) throws BKException {
+        for (BookieId bookieId : toSwitchBookieIdsSet) {
+            BookieServiceInfo bookieServiceInfo = admin.getBookieServiceInfo(bookieId);
+            for (BookieServiceInfo.Endpoint endpoint : bookieServiceInfo.getEndpoints()) {
+                if (endpoint.getProtocol().equals("http")) {
+                    String host = endpoint.getHost();
+                    int port = endpoint.getPort();
+                    String url = String.format("http://%s:%s/%s", host, port, HttpRouter.BOOKIE_STATE_READONLY);
+                    String readOnly = toReadOnly ? "true" : "false";
+                    CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+                    int responseCode = response.getStatusLine().getStatusCode();
+                    if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                        LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                                readOnly, bookieId, responseCode));
+                        return false;
+                    }
+
+                    switchedBookieIds.add(bookieId);
+                    LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                            readOnly, bookieId, responseCode));
+                }
+            }
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(BookKeeperAdmin admin, ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException {
+        try {
+            // 1. Build bookieIdsSet adn ledgerIdsToMigrate
+            Set<BookieId> bookieIdsSet = new HashSet<>();
+            for (String bookieIdStr : flags.bookieIdsOfMigratedReplica) {
+                bookieIdsSet.add(BookieId.parse(bookieIdStr));
+            }
+            HashSet<Long> ledgerIdsToMigrate = new HashSet<>();
+            for (Long ledgerId : flags.ledgerIdsOfMigratedReplica) {
+                ledgerIdsToMigrate.add(ledgerId);
+            }
+
+            // 2. Build toMigratedLedgerAndBookieMap
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIdsSet);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!ledgerIdsToMigrate.isEmpty()) {
+                    ledgersOnBookieId.retainAll(ledgerIdsToMigrate);
+                }
+                for (Long ledgerId : ledgersOnBookieId) {
+                    Set<BookieId> bookieIdSet = toMigratedLedgerAndBookieMap.getOrDefault(ledgerId, new HashSet<>());
+                    bookieIdSet.add(bookieIdSetEntry.getKey());
+                    toMigratedLedgerAndBookieMap.put(ledgerId, bookieIdSet);
+                }
+            }
+
+            // 3. Submit ledger replicas to be migrated
+            MetadataDrivers.runFunctionWithMetadataBookieDriver(conf, driver -> {
+                try {
+                    driver.submitToMigrateReplicas(toMigratedLedgerAndBookieMap);
+                } catch (Throwable e) {
+                    LOG.warn(String.format("The migration task of Replicas=%s submit failed!",
+                            toMigratedLedgerAndBookieMap), e);
+                }
+                return true;

Review Comment:
   If no value is returned, an error will be reported.
   Values are returned in many places, such as:
   <img width="1172" alt="image" src="https://user-images.githubusercontent.com/19296967/191400587-28d5bf7f-5010-4eae-9d95-d39c8b12d2e5.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r1006305326


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,77 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class MigrationReplicasCmd extends MyCommand {
+
+        MigrationReplicasCmd() {
+            super(CMD_MIGRATIONREPLICAS);
+            Option bookiesOpt = new Option("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            Option ledgersOpt = new Option("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            Option readOnlyOpt = new Option("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");

Review Comment:
   OK



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r990795742


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,237 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate(List<String> ledgerIds) throws InterruptedException, KeeperException {
+        Iterator<String> iterator = ledgerIds.iterator();
+        while (iterator.hasNext()) {
+            try {
+                String ledgerId = iterator.next();
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(replicasMigrationPath,
+                        Long.parseLong(ledgerId));
+                if (!metadataClientDriver.exists(ledgerForMigrationReplicasPath)) {
+                    // The ledger migration is completed
+                    iterator.remove();
+                    continue;
+                }
+                String bookieIdStr = metadataClientDriver.
+                        getOwnerBookiesMigrationReplicas(ledgerForMigrationReplicasPath);
+
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(replicasMigrationPath,
+                        Long.parseLong(ledgerId), lock);
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have locked the ledger
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("The encoding is not supported!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        List<String> toMigrateLedgerIds = Collections.emptyList();
+        while (workerRunning) {
+            // 1. build migrating ledgers
+            try {
+                if (toMigrateLedgerIds.isEmpty()) {
+                    toMigrateLedgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+                }
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (Throwable e) {
+                LOG.error("list migrating ledgers failed!", e);
+            }
+
+            // 2. Get a replica ledger to replicate or wait back off
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate(toMigrateLedgerIds);
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (Throwable e) {
+                LOG.error("Get migrating replicas failed!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            // 3. Get the fragment containing the migration bookie
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            try (LedgerHandle lh = admin.openLedgerNoRecovery(replicas.ledgerId)) {
+                Set<LedgerFragment> fragments = getFragmentsOnMigrationBookies(lh, replicas.bookieIds);
+                if (fragments.size() < 1) {
+                    //3.The replication has been completed, delete the ledgerId directory and release lock
+                    releaseLock(replicas);

Review Comment:
   Fixed! PTAL,thanks! @zymap 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1253119549

   > I think you need to add more details to the proposal to know how this work. Like when will the migration does, how it does migration, and what will happen when you trigger the migration.
   > 
   > I take the first look at this PR. I will take another look in the next few days.
   
   OK , I will do


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1242055577

   @nicoloboschi PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r969077348


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/api/v1/bookie/state/readonly", httpAddress);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");

Review Comment:
   @Shoothzj PTAL,thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r968245213


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java:
##########
@@ -88,6 +89,13 @@ default CompletableFuture<Void> disableHealthCheck() {
         return result;
     }
 
+    default CompletableFuture<Void> submitToMigrateReplicas(long ledgerId, String bookieIds)

Review Comment:
   fixed



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            //1. Get a replica ledger to replicate

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1244800837

   @hangc0276 PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r969704163


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];

Review Comment:
   @eolivelli 
   So this command is more appropriate to pass in the ip address? 
   --bookieIds renamed to --bookies,  for example:
   
   bin/bookkeeper shell replicasMigration --bookies IP1:Port,IP2:Port2 --ledgerIds ALL --readOnly true



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970286197


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");
+            opts.addOption("p", "httpPort", true, "http port,default is 8080");

Review Comment:
   @hangc0276
    required default is false, I didn't set this parameter, I can set ths bookiesOpt  with true:
   <img width="1162" alt="image" src="https://user-images.githubusercontent.com/19296967/190057027-6fdd40bd-bd30-463b-844a-a0f132497da0.png">
   



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");
+            opts.addOption("p", "httpPort", true, "http port,default is 8080");

Review Comment:
   @hangc0276
    required default is false, I didn't set this parameter, I can show that bookiesOpt is set to true:
   <img width="1162" alt="image" src="https://user-images.githubusercontent.com/19296967/190057027-6fdd40bd-bd30-463b-844a-a0f132497da0.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970409553


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;

Review Comment:
   I will fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975973829


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {

Review Comment:
   @zymap 
   It is @hangc0276 's suggestion. When some bookie nodes fail to set readonly, they need to fall back to the previous state:
   <img width="914" alt="image" src="https://user-images.githubusercontent.com/19296967/191399590-b8ffcb5d-651d-4fa6-b4db-8fce5c7ca1a9.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975977082


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {
+                    LOG.warn("Some bookie nodes that fail to set readonly, "
+                            + "and the successful bookies fall back to the previous state!");
+                    Set<BookieId> fallbackSuccessfulBookies = new HashSet<>();
+                    switchedBookies.removeAll(readOnlyBookies);
+                    if (!switchToReadonly(admin, switchedBookies, fallbackSuccessfulBookies, false)) {
+                        switchedBookies.removeAll(fallbackSuccessfulBookies);
+                        LOG.error(String.format("Fallback failed! Failed nodes:%s", switchedBookies));
+                    }
+                    return false;
+                }
+            }
+
+            return replicasMigration(admin, conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean switchToReadonly(BookKeeperAdmin admin,
+                                     Set<BookieId> toSwitchBookieIdsSet,
+                                     Set<BookieId> switchedBookieIds,
+                                     boolean toReadOnly) throws BKException {
+        for (BookieId bookieId : toSwitchBookieIdsSet) {
+            BookieServiceInfo bookieServiceInfo = admin.getBookieServiceInfo(bookieId);
+            for (BookieServiceInfo.Endpoint endpoint : bookieServiceInfo.getEndpoints()) {
+                if (endpoint.getProtocol().equals("http")) {
+                    String host = endpoint.getHost();
+                    int port = endpoint.getPort();
+                    String url = String.format("http://%s:%s/%s", host, port, HttpRouter.BOOKIE_STATE_READONLY);
+                    String readOnly = toReadOnly ? "true" : "false";
+                    CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+                    int responseCode = response.getStatusLine().getStatusCode();
+                    if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                        LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                                readOnly, bookieId, responseCode));
+                        return false;
+                    }
+
+                    switchedBookieIds.add(bookieId);
+                    LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                            readOnly, bookieId, responseCode));
+                }
+            }
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(BookKeeperAdmin admin, ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException {
+        try {
+            // 1. Build bookieIdsSet adn ledgerIdsToMigrate
+            Set<BookieId> bookieIdsSet = new HashSet<>();
+            for (String bookieIdStr : flags.bookieIdsOfMigratedReplica) {
+                bookieIdsSet.add(BookieId.parse(bookieIdStr));
+            }
+            HashSet<Long> ledgerIdsToMigrate = new HashSet<>();
+            for (Long ledgerId : flags.ledgerIdsOfMigratedReplica) {
+                ledgerIdsToMigrate.add(ledgerId);
+            }
+
+            // 2. Build toMigratedLedgerAndBookieMap
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIdsSet);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!ledgerIdsToMigrate.isEmpty()) {
+                    ledgersOnBookieId.retainAll(ledgerIdsToMigrate);
+                }
+                for (Long ledgerId : ledgersOnBookieId) {
+                    Set<BookieId> bookieIdSet = toMigratedLedgerAndBookieMap.getOrDefault(ledgerId, new HashSet<>());
+                    bookieIdSet.add(bookieIdSetEntry.getKey());
+                    toMigratedLedgerAndBookieMap.put(ledgerId, bookieIdSet);
+                }
+            }
+
+            // 3. Submit ledger replicas to be migrated
+            MetadataDrivers.runFunctionWithMetadataBookieDriver(conf, driver -> {
+                try {
+                    driver.submitToMigrateReplicas(toMigratedLedgerAndBookieMap);
+                } catch (Throwable e) {
+                    LOG.warn(String.format("The migration task of Replicas=%s submit failed!",
+                            toMigratedLedgerAndBookieMap), e);
+                }
+                return true;

Review Comment:
   If no value is returned, an error will be reported.
   <img width="908" alt="image" src="https://user-images.githubusercontent.com/19296967/194686646-169b6e94-13e3-4700-b05a-5e44c11188c4.png">
   
   Values are returned in many places, such as:
   <img width="1172" alt="image" src="https://user-images.githubusercontent.com/19296967/191400587-28d5bf7f-5010-4eae-9d95-d39c8b12d2e5.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r990795717


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,237 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {

Review Comment:
   Fixed! PTAL,thanks! @zymap 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/util/BookKeeperConstants.java:
##########
@@ -40,6 +40,7 @@ public class BookKeeperConstants {
     public static final String AVAILABLE_NODE = "available";
     public static final String COOKIE_NODE = "cookies";
     public static final String UNDER_REPLICATION_NODE = "underreplication";
+    public static final String MIGRATION_REPLICAS = "migrationReplicas";

Review Comment:
   Fixed! PTAL,thanks! @zymap 



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {
+                    LOG.warn("Some bookie nodes that fail to set readonly, "
+                            + "and the successful bookies fall back to the previous state!");
+                    Set<BookieId> fallbackSuccessfulBookies = new HashSet<>();
+                    switchedBookies.removeAll(readOnlyBookies);
+                    if (!switchToReadonly(admin, switchedBookies, fallbackSuccessfulBookies, false)) {
+                        switchedBookies.removeAll(fallbackSuccessfulBookies);
+                        LOG.error(String.format("Fallback failed! Failed nodes:%s", switchedBookies));
+                    }
+                    return false;
+                }
+            }
+
+            return replicasMigration(admin, conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean switchToReadonly(BookKeeperAdmin admin,
+                                     Set<BookieId> toSwitchBookieIdsSet,
+                                     Set<BookieId> switchedBookieIds,
+                                     boolean toReadOnly) throws BKException {
+        for (BookieId bookieId : toSwitchBookieIdsSet) {
+            BookieServiceInfo bookieServiceInfo = admin.getBookieServiceInfo(bookieId);
+            for (BookieServiceInfo.Endpoint endpoint : bookieServiceInfo.getEndpoints()) {
+                if (endpoint.getProtocol().equals("http")) {
+                    String host = endpoint.getHost();
+                    int port = endpoint.getPort();
+                    String url = String.format("http://%s:%s/%s", host, port, HttpRouter.BOOKIE_STATE_READONLY);
+                    String readOnly = toReadOnly ? "true" : "false";
+                    CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+                    int responseCode = response.getStatusLine().getStatusCode();
+                    if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                        LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                                readOnly, bookieId, responseCode));
+                        return false;
+                    }
+
+                    switchedBookieIds.add(bookieId);
+                    LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                            readOnly, bookieId, responseCode));
+                }
+            }
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(BookKeeperAdmin admin, ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException {
+        try {
+            // 1. Build bookieIdsSet adn ledgerIdsToMigrate
+            Set<BookieId> bookieIdsSet = new HashSet<>();
+            for (String bookieIdStr : flags.bookieIdsOfMigratedReplica) {
+                bookieIdsSet.add(BookieId.parse(bookieIdStr));
+            }
+            HashSet<Long> ledgerIdsToMigrate = new HashSet<>();
+            for (Long ledgerId : flags.ledgerIdsOfMigratedReplica) {
+                ledgerIdsToMigrate.add(ledgerId);
+            }
+
+            // 2. Build toMigratedLedgerAndBookieMap
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIdsSet);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!ledgerIdsToMigrate.isEmpty()) {
+                    ledgersOnBookieId.retainAll(ledgerIdsToMigrate);
+                }
+                for (Long ledgerId : ledgersOnBookieId) {
+                    Set<BookieId> bookieIdSet = toMigratedLedgerAndBookieMap.getOrDefault(ledgerId, new HashSet<>());
+                    bookieIdSet.add(bookieIdSetEntry.getKey());
+                    toMigratedLedgerAndBookieMap.put(ledgerId, bookieIdSet);
+                }
+            }
+
+            // 3. Submit ledger replicas to be migrated
+            MetadataDrivers.runFunctionWithMetadataBookieDriver(conf, driver -> {
+                try {
+                    driver.submitToMigrateReplicas(toMigratedLedgerAndBookieMap);
+                } catch (Throwable e) {
+                    LOG.warn(String.format("The migration task of Replicas=%s submit failed!",
+                            toMigratedLedgerAndBookieMap), e);
+                }
+                return true;

Review Comment:
   Fixed! PTAL,thanks! @zymap 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#issuecomment-1243117603

   @eolivelli PTAL,thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] eolivelli commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
eolivelli commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r969445823


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];

Review Comment:
   this doesn't work if your bookieId is not a public hostname.
   the bookieId is a opaque string, we have to "resolve" the ID



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/util/HttpClientUtils.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.util;
+
+import java.io.IOException;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+
+/**
+ * http client utils.
+ * */
+public class HttpClientUtils {
+    private static final CloseableHttpClient httpClient = HttpClientBuilder.create().build();

Review Comment:
   we should close this object
   using a static field is not good.
   
   we can open/close the client per each request, it is not a big deal



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,27 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();
+    }
+
+    /**
+     * After obtaining a replica migration task,
+     * lock the replica task to prevent it from being executed by other workers.
+     * */
+    default void lockMigrationReplicas(String lockPath, String advertisedAddress)
+            throws InterruptedException, KeeperException {
+    }
+
+    default String getOwnerBookiesMigrationReplicas(String ledgerForMigrationReplicasPath)
+            throws InterruptedException, KeeperException, UnsupportedEncodingException {
+        return "";
+    }
+
+    default void deleteZkPath(String migrationLedgerPath) throws InterruptedException, KeeperException {
+    }

Review Comment:
   I would throw some UnsupportedOperationException



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,27 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();
+    }
+
+    /**
+     * After obtaining a replica migration task,
+     * lock the replica task to prevent it from being executed by other workers.
+     * */
+    default void lockMigrationReplicas(String lockPath, String advertisedAddress)
+            throws InterruptedException, KeeperException {
+    }
+
+    default String getOwnerBookiesMigrationReplicas(String ledgerForMigrationReplicasPath)
+            throws InterruptedException, KeeperException, UnsupportedEncodingException {
+        return "";

Review Comment:
   I would throw some UnsupportedOperationException



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;

Review Comment:
   if the bookie is up and running it publishes on metadata the http endpoint.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r968247070


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            //1. Get a replica ledger to replicate
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (KeeperException e) {
+                LOG.error("getReplicasToMigrate throw exception!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            //2. Get the fragment containing the migration bookie
+            try (LedgerHandle lh = admin.openLedgerNoRecovery(replicas.ledgerId)) {
+                Set<LedgerFragment> fragments = getFragmentsOnMigrationBookies(lh, replicas.bookieIds);
+                if (fragments.size() < 1) {
+                    //3.The replication has been completed, delete the ledgerId directory and release lock
+                    releaseLock(replicas);
+                    metadataClientDriver.deleteZkPath(getLedgerForMigrationReplicasPath(replicasMigrationPath,
+                            replicas.ledgerId));
+                    LOG.info(String.format("Finish ledgerId %s migration!", replicas.ledgerId));
+                }
+
+                //4. Start replicate

Review Comment:
   @Shoothzj PTAL,thanks!



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/api/v1/bookie/state/readonly", httpAddress);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");

Review Comment:
   @Shoothzj PTAL,thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r969704163


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];

Review Comment:
   @eolivelli 
   So this command is more appropriate to pass in the ip address? 
   --bookieIds renamed to --bookies,  for example:
   
   bin/bookkeeper shell replicasMigration --bookies IP1:Port1,IP2:Port2 --ledgerIds ALL --readOnly true



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] Shoothzj commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
Shoothzj commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r968084246


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataClientDriver.java:
##########
@@ -106,6 +110,23 @@ interface SessionStateListener {
      */
     void setSessionStateListener(SessionStateListener sessionStateListener);
 
+    default List<String> listLedgersOfMigrationReplicas(String migrationReplicasPath)
+            throws InterruptedException, KeeperException {
+        return new ArrayList<>();
+    }
+
+    default void lockMigrationReplicas(String lockPath, String advertisedAddress)

Review Comment:
   method in interface needs javadoc



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java:
##########
@@ -88,6 +89,13 @@ default CompletableFuture<Void> disableHealthCheck() {
         return result;
     }
 
+    default CompletableFuture<Void> submitToMigrateReplicas(long ledgerId, String bookieIds)

Review Comment:
   This method needs javadoc



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            //1. Get a replica ledger to replicate
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (KeeperException e) {
+                LOG.error("getReplicasToMigrate throw exception!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            //2. Get the fragment containing the migration bookie
+            try (LedgerHandle lh = admin.openLedgerNoRecovery(replicas.ledgerId)) {
+                Set<LedgerFragment> fragments = getFragmentsOnMigrationBookies(lh, replicas.bookieIds);
+                if (fragments.size() < 1) {
+                    //3.The replication has been completed, delete the ledgerId directory and release lock
+                    releaseLock(replicas);
+                    metadataClientDriver.deleteZkPath(getLedgerForMigrationReplicasPath(replicasMigrationPath,
+                            replicas.ledgerId));
+                    LOG.info(String.format("Finish ledgerId %s migration!", replicas.ledgerId));
+                }
+
+                //4. Start replicate

Review Comment:
   ```suggestion
                   // 4. Start replicate
   ```



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            //1. Get a replica ledger to replicate
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (KeeperException e) {
+                LOG.error("getReplicasToMigrate throw exception!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            //2. Get the fragment containing the migration bookie

Review Comment:
   ```suggestion
               // 2. Get the fragment containing the migration bookie
   ```



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/MetadataBookieDriver.java:
##########
@@ -88,6 +89,13 @@ default CompletableFuture<Void> disableHealthCheck() {
         return result;
     }
 
+    default CompletableFuture<Void> submitToMigrateReplicas(long ledgerId, String bookieIds)
+            throws UnsupportedEncodingException {

Review Comment:
   why we need this exception? If we use **UTF-8**, I think we can remove this throw



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            //1. Get a replica ledger to replicate

Review Comment:
   ```suggestion
               // 1. Get a replica ledger to replicate
   ```



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/api/v1/bookie/state/readonly", httpAddress);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Set<BookieId> bookieIds = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+            Set<Long> ledgerIdsToMigrate = null;
+            if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                ledgerIdsToMigrate = parseLedgerIds(flags.ledgerIdsOfMigratedReplica);
+            }
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIds);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                    ledgersOnBookieId.retainAll(ledgerIdsToMigrate);
+                }
+                for (Long ledgerId : ledgersOnBookieId) {
+                    Set<BookieId> bookieIdSet = toMigratedLedgerAndBookieMap.getOrDefault(ledgerId, new HashSet<>());
+                    bookieIdSet.add(bookieIdSetEntry.getKey());
+                    toMigratedLedgerAndBookieMap.put(ledgerId, bookieIdSet);
+                }
+            }
+
+            MetadataDrivers.runFunctionWithMetadataBookieDriver(conf, driver -> {
+                for (Map.Entry<Long, Set<BookieId>> entry : toMigratedLedgerAndBookieMap.entrySet()) {
+                    Long ledgerId = entry.getKey();
+                    String bookies = StringUtils.join(entry.getValue(), seperator);
+                    try {
+                        driver.submitToMigrateReplicas(ledgerId, bookies).get();
+                    } catch (InterruptedException e) {
+                        throw new RuntimeException(e);
+                    } catch (Throwable e) {
+                        LOG.warn(String.format("The migration task of Replicas(ledgerId=%s,bookieIds=%s) "
+                                + "submit failed!", ledgerId, bookies), e);
+                    }
+                }
+                return true;
+            });
+
+            return true;
+        } catch (Exception e) {
+            LOG.error("Received exception in ReplicasMigrationCommand ", e);
+            return false;
+        } finally {
+            if (admin != null) {

Review Comment:
   admin can't be null here. Am I right?



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/api/v1/bookie/state/readonly", httpAddress);

Review Comment:
   we can use the constant in `HttpRouter.BOOKIE_STATE_READONLY`



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,216 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {
+    private static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationWorker.class);
+    private final BookKeeperAdmin admin;
+    private final BiConsumer<Long, Long> onReadEntryFailureCallback;
+    private final MetadataClientDriver metadataClientDriver;
+    private final String rootPath;
+    private final String replicasMigrationPath;
+    private final String lock = "locked";
+    private final LedgerChecker ledgerChecker;
+    private final Thread workerThread;
+    private final long backoffMs = 100;
+    private volatile boolean workerRunning = false;
+    private final String seperator = ",";
+    private final String advertisedAddress;
+
+    @StatsDoc(
+            name = NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION,
+            help = "the number of entries ReplicasMigrationWorker unable to read"
+    )
+    private final Counter numEntriesUnableToReadForMigration;
+
+    public ReplicasMigrationWorker(final ServerConfiguration conf, BookKeeper bkc, StatsLogger statsLogger) {
+        this.advertisedAddress = conf.getAdvertisedAddress();
+        this.rootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
+        this.replicasMigrationPath = getReplicasMigrationPath(rootPath);
+        this.ledgerChecker = new LedgerChecker(bkc);
+        this.admin = new BookKeeperAdmin(bkc, statsLogger, new ClientConfiguration(conf));
+        this.metadataClientDriver = bkc.getMetadataClientDriver();
+        this.numEntriesUnableToReadForMigration = statsLogger
+                .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION);
+        this.onReadEntryFailureCallback = (ledgerid, entryid) -> {
+            numEntriesUnableToReadForMigration.inc();
+        };
+        this.workerThread = new BookieThread(this, "ReplicasMigrationWorker");
+    }
+
+    public static String getReplicasMigrationPath(String rootPath) {
+        return String.format("%s/%s", rootPath, BookKeeperConstants.MIGRATION_REPLICAS);
+    }
+
+    public static String getLockForMigrationReplicasPath(String replicasMigrationPath, long ledgerId, String lock) {
+        return String.format("%s/%s/%s", replicasMigrationPath, ledgerId, lock);
+    }
+
+    public static String getLedgerForMigrationReplicasPath(String replicasMigrationPath, long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    static class Replicas {
+        long ledgerId;
+        Set<BookieId> bookieIds;
+
+        public Replicas(long ledgerId, Set<BookieId> bookieIds) {
+            this.ledgerId = ledgerId;
+            this.bookieIds = bookieIds;
+        }
+
+        @Override
+        public String toString() {
+            return String.format("<ledgerId=%s,bookies=<%s>", ledgerId, bookieIds);
+        }
+    }
+
+    private Replicas getReplicasToMigrate() throws InterruptedException, KeeperException {
+        List<String> ledgerIds = metadataClientDriver.listLedgersOfMigrationReplicas(replicasMigrationPath);
+        for (String ledgerId : ledgerIds) {
+            try {
+                String lockForMigrationReplicasPath = getLockForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId), lock);
+                String ledgerForMigrationReplicasPath = getLedgerForMigrationReplicasPath(
+                        replicasMigrationPath, Long.parseLong(ledgerId));
+                metadataClientDriver.lockMigrationReplicas(lockForMigrationReplicasPath, advertisedAddress);
+                String bookieIdStr = metadataClientDriver.getOwnerBookiesMigrationReplicas(
+                        ledgerForMigrationReplicasPath);
+                String[] migrationBookieIds = bookieIdStr.split(seperator);
+                Set<BookieId> bookieIds = new HashSet<>();
+                for (int i = 0; i < migrationBookieIds.length; i++) {
+                    bookieIds.add(BookieId.parse(migrationBookieIds[i]));
+                }
+                return new Replicas(Long.parseLong(ledgerId), bookieIds);
+            } catch (KeeperException.NodeExistsException nee) {
+                // do nothing, someone each could have created it
+            } catch (UnsupportedEncodingException e) {
+                LOG.error("getReplicasToMigrate failed!", e);
+            }
+        }
+        return null;
+    }
+
+    private Set<LedgerFragment> getFragmentsOnMigrationBookies(LedgerHandle lh, Set<BookieId> migrationBookieIds) {
+        return ledgerChecker.getFragmentsOnMigrationBookies(lh, migrationBookieIds);
+    }
+
+    private static void waitBackOffTime(long backoffMs) {
+        try {
+            Thread.sleep(backoffMs);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void releaseLock(Replicas replicas) {
+        try {
+            metadataClientDriver.deleteZkPath(getLockForMigrationReplicasPath(replicasMigrationPath,
+                    replicas.ledgerId, lock));
+            LOG.info(String.format("Release lock for ledgerId %s success!", replicas.ledgerId));
+        } catch (KeeperException.NoNodeException e) {
+            // do nothing,already release lock
+        } catch (Exception e) {
+            LOG.error(String.format("Release lock for ledgerId %s failed!", replicas.ledgerId));
+        }
+    }
+
+    /**
+     * Start the migration replicas worker.
+     */
+    public void start() {
+        this.workerThread.start();
+    }
+
+    @Override
+    public void run() {
+        workerRunning = true;
+        while (workerRunning) {
+            //1. Get a replica ledger to replicate
+            Replicas replicas = null;
+            try {
+                replicas = getReplicasToMigrate();
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            } catch (KeeperException e) {
+                LOG.error("getReplicasToMigrate throw exception!", e);
+            }
+            if (null == replicas) {
+                waitBackOffTime(backoffMs);
+                continue;
+            }
+
+            LOG.info(String.format("Start migrate replicas(%s)!", replicas));
+            //2. Get the fragment containing the migration bookie
+            try (LedgerHandle lh = admin.openLedgerNoRecovery(replicas.ledgerId)) {
+                Set<LedgerFragment> fragments = getFragmentsOnMigrationBookies(lh, replicas.bookieIds);
+                if (fragments.size() < 1) {
+                    //3.The replication has been completed, delete the ledgerId directory and release lock

Review Comment:
   ```suggestion
                       // 3.The replication has been completed, delete the ledgerId directory and release lock
   ```



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/api/v1/bookie/state/readonly", httpAddress);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");

Review Comment:
   can we reuse some code in `listbookies` command?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970286197


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");
+            opts.addOption("p", "httpPort", true, "http port,default is 8080");

Review Comment:
   @hangc0276
    required default is false, I didn't set this parameter, I can show that bookiesOpt is set to true:
   <img width="1162" alt="image" src="https://user-images.githubusercontent.com/19296967/190057027-6fdd40bd-bd30-463b-844a-a0f132497da0.png">
   



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");
+            opts.addOption("p", "httpPort", true, "http port,default is 8080");

Review Comment:
   @hangc0276
    required default is false, I didn't set this parameter, I can show that bookiesOpt is set to true:
   <img width="1162" alt="image" src="https://user-images.githubusercontent.com/19296967/190057027-6fdd40bd-bd30-463b-844a-a0f132497da0.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970410253


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;

Review Comment:
   OK , add rollback



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) throws IOException {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];
+            String httpAddress = bookieAddress + ":" + flags.port;
+            String url = String.format("http://%s/%s", httpAddress, HttpRouter.BOOKIE_STATE_READONLY);
+            String readOnly = flags.readOnly ? "true" : "false";
+            CloseableHttpResponse response = HttpClientUtils.doPutRequest(url, readOnly, "UTF-8");
+            int responseCode = response.getStatusLine().getStatusCode();
+            if (responseCode != HttpServer.StatusCode.OK.getValue()) {
+                LOG.error(String.format("Set readonly to %s failed! bookieId:%s, responseCode:%s ",
+                        readOnly, bookieId, responseCode));
+                return false;
+            }
+            LOG.info(String.format("Set readonly to %s success! bookieId:%s, responseCode:%s",
+                    readOnly, bookieId, responseCode));
+        }
+        return true;
+    }
+
+    private boolean replicasMigration(ServerConfiguration conf, ReplicasMigrationFlags flags)
+            throws BKException, InterruptedException, IOException {
+        ClientConfiguration adminConf = new ClientConfiguration(conf);
+        BookKeeperAdmin admin = new BookKeeperAdmin(adminConf);
+        try {
+            Map<Long, Set<BookieId>> toMigratedLedgerAndBookieMap = new HashMap<>();
+            Set<BookieId> bookieIds = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+            Set<Long> ledgerIdsToMigrate = null;
+            if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {
+                ledgerIdsToMigrate = parseLedgerIds(flags.ledgerIdsOfMigratedReplica);
+            }
+            Map<BookieId, Set<Long>> bookieLedgerMaps = admin.listLedgers(bookieIds);
+            for (Map.Entry<BookieId, Set<Long>> bookieIdSetEntry : bookieLedgerMaps.entrySet()) {
+                Set<Long> ledgersOnBookieId = bookieIdSetEntry.getValue();
+                if (!flags.ledgerIdsOfMigratedReplica.equals(ALL)) {

Review Comment:
   @hangc0276
   
   admin.listLedgers(bookieIdsSet) is for building toMigratedLedgerAndBookieMap.
   
   toMigratedLedgerAndBookieMap records the mapping of ledger and bookie for each replica to be migrated.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970293663


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,56 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class ReplicasMigrationCmd extends MyCommand {
+
+        ReplicasMigrationCmd() {
+            super(CMD_REPLICASMIGRATION);
+            opts.addOption("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            opts.addOption("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            opts.addOption("r", "readOnly", true,

Review Comment:
   OK, I will  fix!
   <img width="1117" alt="image" src="https://user-images.githubusercontent.com/19296967/190059205-2ff05bfb-3a73-4c0d-aa32-d3b80d43d3d3.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970503594


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/ReplicasMigrationCommand.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+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.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MetadataDrivers;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class ReplicasMigrationCommand extends BookieCommand<ReplicasMigrationCommand.ReplicasMigrationFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(ReplicasMigrationCommand.class);
+    private static final String NAME = "replicasMigration";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    private static final String ALL = "ALL";
+    private final String seperator = ",";
+    public ReplicasMigrationCommand() {
+        this(new ReplicasMigrationFlags());
+    }
+    private ReplicasMigrationCommand(ReplicasMigrationFlags flags) {
+        super(CliSpec.<ReplicasMigrationFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for replicasMigration command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class ReplicasMigrationFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private String bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is ALL, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private String ledgerIdsOfMigratedReplica;
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is true")
+        private boolean readOnly;
+
+        @Parameter(names = {"-p", "--httpPort"}, description = "http port,default is 8080")
+        private String port;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, ReplicasMigrationFlags cmdFlags) {
+        try {
+            return switchToReadonly(conf, cmdFlags) && replicasMigration(conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    public Set<BookieId> parseBookieIds(String bookieIdsOfMigratedReplica) {
+        Set<BookieId> bookieIdsSet = new HashSet<>();
+        String[] bookieIds = bookieIdsOfMigratedReplica.split(seperator);
+        for (String bookieId : bookieIds) {
+            bookieIdsSet.add(BookieId.parse(bookieId));
+        }
+        return bookieIdsSet;
+    }
+
+    public Set<Long> parseLedgerIds(String ledgerIdsOfMigratedReplica) {
+        Set<Long> ledgerIdSet = new HashSet<>();
+        String[] ledgerIds = ledgerIdsOfMigratedReplica.split(seperator);
+        for (String ledgerId : ledgerIds) {
+            ledgerIdSet.add(Long.parseLong(ledgerId));
+        }
+        return ledgerIdSet;
+    }
+
+    private boolean switchToReadonly(ServerConfiguration conf, ReplicasMigrationFlags flags) {
+        if (flags.readOnly && !conf.isHttpServerEnabled()) {
+            throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+        }
+
+        Set<BookieId> bookieIdSet = parseBookieIds(flags.bookieIdsOfMigratedReplica);
+        for (BookieId bookieId : bookieIdSet) {
+            String bookieAddress = bookieId.getId().split(":")[0];

Review Comment:
   fixed. PTAL,thanks! @eolivelli



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970504180


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/util/HttpClientUtils.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.util;
+
+import java.io.IOException;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+
+/**
+ * http client utils.
+ * */
+public class HttpClientUtils {
+    private static final CloseableHttpClient httpClient = HttpClientBuilder.create().build();

Review Comment:
   fixed. PTAL,thanks! @eolivelli



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/util/HttpClientUtils.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.util;
+
+import java.io.IOException;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClientBuilder;
+
+/**
+ * http client utils.
+ * */
+public class HttpClientUtils {
+    private static final CloseableHttpClient httpClient = HttpClientBuilder.create().build();

Review Comment:
   fixed! @eolivelli 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r970506919


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java:
##########
@@ -289,6 +297,57 @@ public void processResult(int rc, String path, Object ctx, String name) {
         return createResult;
     }
 
+    private String getLedgerForReplicasMigrationPath(long ledgerId) {
+        return String.format("%s/%s", replicasMigrationPath, ledgerId);
+    }
+
+    public CompletableFuture<Void> submitToMigrateReplicas(long ledgerId, String bookieIds) {
+        CompletableFuture<Void> createResult = new CompletableFuture<>();
+        zk.create(getLedgerForReplicasMigrationPath(ledgerId), bookieIds.getBytes(StandardCharsets.UTF_8), acls,

Review Comment:
   Fixed. 
   PTAL,thanks!  @hangc0276 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975975297


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicasMigrationWorker.java:
##########
@@ -0,0 +1,237 @@
+/**
+ * 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.replication;
+
+import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_MIGRATION;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.BiConsumer;
+import org.apache.bookkeeper.bookie.BookieThread;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerFragment;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MetadataClientDriver;
+import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.annotations.StatsDoc;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Migrate replica data to other bookie nodes.
+ */
+public class ReplicasMigrationWorker implements Runnable {

Review Comment:
   OK , I will fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r975975057


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/zk/ZKMetadataDriverBase.java:
##########
@@ -239,6 +251,7 @@ protected void initialize(AbstractConfiguration<?> conf,
         }
 
         disableHealthCheckPath = ledgersRootPath + "/" + DISABLE_HEALTH_CHECK;
+        replicasMigrationPath = ledgersRootPath + "/" + MIGRATION_REPLICAS;

Review Comment:
   OK, I will fix



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r1005877921


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookies/MigrationReplicasCommand.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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 com.beust.jcommander.Parameter;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+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.discover.BookieServiceInfo;
+import org.apache.bookkeeper.http.HttpRouter;
+import org.apache.bookkeeper.http.HttpServer;
+import org.apache.bookkeeper.meta.MigrationManager;
+import org.apache.bookkeeper.net.BookieId;
+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.HttpClientUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Migrate the data of the specified replica to other bookie nodes.
+ * */
+public class MigrationReplicasCommand extends BookieCommand<MigrationReplicasCommand.MigrationReplicasFlags> {
+    static final Logger LOG = LoggerFactory.getLogger(MigrationReplicasCommand.class);
+    private static final String NAME = "migrationReplicas";
+    private static final String DESC = "Migrate the data of the specified replica to other bookie nodes";
+    public MigrationReplicasCommand() {
+        this(new MigrationReplicasFlags());
+    }
+    private MigrationReplicasCommand(MigrationReplicasFlags flags) {
+        super(CliSpec.<MigrationReplicasFlags>newBuilder()
+                .withName(NAME)
+                .withDescription(DESC)
+                .withFlags(flags)
+                .build());
+    }
+
+    /**
+     * Flags for migrationReplicas command.
+     */
+    @Accessors(fluent = true)
+    @Setter
+    public static class MigrationReplicasFlags extends CliFlags {
+
+        @Parameter(names = { "-b", "--bookieIds" }, description = "Bookies corresponding to the migrated replica, "
+                + "eg: bookieId1,bookieId2,bookieId3")
+        private List<String> bookieIdsOfMigratedReplica;
+
+        @Parameter(names = { "-l", "--ledgerIds" }, description = "The ledgerIds corresponding to the migrated replica,"
+                + "eg: ledgerId1,ledgerId2,ledgerId3. The default is empty, which means that all replica data on these "
+                + "bookie nodes will be migrated to other bookie nodes")
+        private List<Long> ledgerIdsOfMigratedReplica = Collections.emptyList();
+
+        @Parameter(names = { "-r", "--readOnly" }, description = "Whether to set the bookie nodes of the migrated data"
+                + " to readOnly, the default is false")
+        private boolean readOnly = false;
+    }
+
+    @Override
+    public boolean apply(ServerConfiguration conf, MigrationReplicasFlags cmdFlags) {
+        try (BookKeeperAdmin admin = new BookKeeperAdmin(new ClientConfiguration(conf))) {
+            if (cmdFlags.readOnly) {
+                if (!conf.isHttpServerEnabled()) {
+                    throw new RuntimeException("Please enable http service first, config httpServerEnabled is false!");
+                }
+                Set<BookieId> toSwitchBookieIdsSet = new HashSet<>();
+                for (String bookieIdStr : cmdFlags.bookieIdsOfMigratedReplica) {
+                    toSwitchBookieIdsSet.add(BookieId.parse(bookieIdStr));
+                }
+
+                Collection<BookieId> readOnlyBookies = admin.getReadOnlyBookies();
+                toSwitchBookieIdsSet.removeAll(readOnlyBookies);
+                Set<BookieId> switchedBookies = new HashSet<>();
+                if (!switchToReadonly(admin, toSwitchBookieIdsSet, switchedBookies, true)) {
+                    LOG.warn("Some bookie nodes that fail to set readonly, "
+                            + "and the successful bookies fall back to the previous state!");
+                    Set<BookieId> fallbackSuccessfulBookies = new HashSet<>();
+                    if (!switchToReadonly(admin, switchedBookies, fallbackSuccessfulBookies, false)) {
+                        switchedBookies.removeAll(fallbackSuccessfulBookies);
+                        LOG.error(String.format("Fallback failed! Failed nodes:%s", switchedBookies));
+                    }
+                    return false;
+                }
+            }
+
+            return migrationReplicas(admin, conf, cmdFlags);
+        } catch (Exception e) {
+            throw new UncheckedExecutionException(e.getMessage(), e);
+        }
+    }
+
+    private boolean switchToReadonly(BookKeeperAdmin admin,

Review Comment:
   The method can switch bookie's mode between readWrite and readOnly, please rename this method.



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,77 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class MigrationReplicasCmd extends MyCommand {
+
+        MigrationReplicasCmd() {
+            super(CMD_MIGRATIONREPLICAS);
+            Option bookiesOpt = new Option("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            Option ledgersOpt = new Option("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            Option readOnlyOpt = new Option("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");

Review Comment:
   Default is `true`? Not sync with `MigrationReplicasFlags` default value.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] lordcheng10 commented on a diff in pull request #3457: BP-56: Support data migration

Posted by GitBox <gi...@apache.org>.
lordcheng10 commented on code in PR #3457:
URL: https://github.com/apache/bookkeeper/pull/3457#discussion_r1006324566


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieShell.java:
##########
@@ -1951,6 +1953,77 @@ public int runCmd(CommandLine cmdLine) throws Exception {
         }
     }
 
+    /**
+     * Migrate the specified ledger data on some bookie to other bookies.
+     * */
+    class MigrationReplicasCmd extends MyCommand {
+
+        MigrationReplicasCmd() {
+            super(CMD_MIGRATIONREPLICAS);
+            Option bookiesOpt = new Option("b", "bookieIds", true,
+                    "Bookies corresponding to the migrated replica");
+            Option ledgersOpt = new Option("l", "ledgerIds", true,
+                    "The ledgerIds corresponding to the migrated replica");
+            Option readOnlyOpt = new Option("r", "readOnly", true,
+                    "Whether to set the bookie nodes of the migrated data to readOnly, the default is true");

Review Comment:
   The true here is not the default value, the corresponding parameter name is hasArg:
   <img width="974" alt="image" src="https://user-images.githubusercontent.com/19296967/198170890-ed0ee963-9f06-4170-846b-95642e26f17a.png">
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org