wchevreuil commented on code in PR #5179:
URL: https://github.com/apache/hbase/pull/5179#discussion_r1166518348


##########
hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerMigrationTool.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.hadoop.hbase.replication;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A tool for migrating replication peer data across different replication 
peer storages.
+ */
[email protected](HBaseInterfaceAudience.TOOLS)
+public class ReplicationPeerMigrationTool extends Configured implements Tool {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReplicationPeerMigrationTool.class);
+
+  public static final String NAME = "migratereppeer";
+
+  public ReplicationPeerMigrationTool(Configuration conf) {
+    super(conf);
+  }
+
+  private ReplicationPeerStorage create(String type, FileSystem fs, ZKWatcher 
zk) {
+    Configuration conf = getConf();
+    Configuration srcConf = new Configuration(conf);
+    srcConf.set(ReplicationStorageFactory.REPLICATION_PEER_STORAGE_IMPL, type);
+    return ReplicationStorageFactory.getReplicationPeerStorage(fs, zk, 
srcConf);
+  }
+
+  private ZKWatcher createZKWatcher() throws IOException {
+    return new ZKWatcher(getConf(), getClass().getSimpleName(), new 
Abortable() {
+
+      private volatile boolean aborted;
+
+      @Override
+      public boolean isAborted() {
+        return aborted;
+      }
+
+      @Override
+      public void abort(String why, Throwable e) {
+        aborted = true;
+        LOG.error(why, e);
+        System.exit(1);
+      }
+    });
+  }
+
+  private void migrate(ReplicationPeerStorage src, ReplicationPeerStorage dst)
+    throws ReplicationException {
+    LOG.info("Start migrating from {} to {}", src.getClass().getSimpleName(),
+      dst.getClass().getSimpleName());
+    for (String peerId : src.listPeerIds()) {
+      LOG.info("Going to migrate {}", peerId);
+      ReplicationPeerConfig peerConfig = src.getPeerConfig(peerId);
+      boolean enabled = src.isPeerEnabled(peerId);
+      SyncReplicationState syncState = src.getPeerSyncReplicationState(peerId);
+      SyncReplicationState newSyncState = 
src.getPeerNewSyncReplicationState(peerId);
+      if (newSyncState != SyncReplicationState.NONE) {
+        throw new IllegalStateException("Can not migrate peer " + peerId
+          + " as it is in an intermediate state, syncReplicationState is " + 
syncState
+          + " while newSyncReplicationState is " + newSyncState);
+      }
+      dst.addPeer(peerId, peerConfig, enabled, syncState);
+      LOG.info("Migrated peer {}, peerConfig = '{}', enabled = {}, 
syncReplicationState = {}",

Review Comment:
   Should we delete the peer from the src storage? Otherwise, rather than 
saying we migrated it, we should say we created/duplicated it in a new storage.



##########
hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerMigrationTool.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.hadoop.hbase.replication;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A tool for migrating replication peer data across different replication 
peer storages.
+ */
[email protected](HBaseInterfaceAudience.TOOLS)
+public class ReplicationPeerMigrationTool extends Configured implements Tool {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReplicationPeerMigrationTool.class);
+
+  public static final String NAME = "migratereppeer";
+
+  public ReplicationPeerMigrationTool(Configuration conf) {
+    super(conf);
+  }
+
+  private ReplicationPeerStorage create(String type, FileSystem fs, ZKWatcher 
zk) {
+    Configuration conf = getConf();
+    Configuration srcConf = new Configuration(conf);
+    srcConf.set(ReplicationStorageFactory.REPLICATION_PEER_STORAGE_IMPL, type);
+    return ReplicationStorageFactory.getReplicationPeerStorage(fs, zk, 
srcConf);
+  }
+
+  private ZKWatcher createZKWatcher() throws IOException {
+    return new ZKWatcher(getConf(), getClass().getSimpleName(), new 
Abortable() {
+
+      private volatile boolean aborted;
+
+      @Override
+      public boolean isAborted() {
+        return aborted;
+      }
+
+      @Override
+      public void abort(String why, Throwable e) {
+        aborted = true;
+        LOG.error(why, e);
+        System.exit(1);
+      }
+    });
+  }

Review Comment:
   Can we move this to ReplicationStorageFactory? Same for FS creation, that 
way we encapsulate the requirements of each impl in the factory, and 
ReplicationStorageFactory.getReplicationPeerStorage would need only the config 
as param.



##########
hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerMigrationTool.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.hadoop.hbase.replication;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A tool for migrating replication peer data across different replication 
peer storages.
+ */
[email protected](HBaseInterfaceAudience.TOOLS)
+public class ReplicationPeerMigrationTool extends Configured implements Tool {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ReplicationPeerMigrationTool.class);
+
+  public static final String NAME = "migratereppeer";
+
+  public ReplicationPeerMigrationTool(Configuration conf) {
+    super(conf);
+  }
+
+  private ReplicationPeerStorage create(String type, FileSystem fs, ZKWatcher 
zk) {
+    Configuration conf = getConf();
+    Configuration srcConf = new Configuration(conf);

Review Comment:
   nit: do `Configuration conf = new Configuration(getConf())`, then we don't 
need this extra `srcConf` var (which name is a bit misleading as we call this 
for both src and dst).



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to