blob: cb5d4a29df1b1538fa21c9c75f6593d6b3f3dc99 [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.replication, class: CopyReplicationPeers">
<meta name="generator" content="javadoc/SourceToHTMLConverter">
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body class="source-page">
<main role="main">
<div class="source-container">
<pre><span class="source-line-no">001</span><span id="line-1">/*</span>
<span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span>
<span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span>
<span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span>
<span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span>
<span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span>
<span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span>
<span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span>
<span class="source-line-no">009</span><span id="line-9"> *</span>
<span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="source-line-no">011</span><span id="line-11"> *</span>
<span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span>
<span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span>
<span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span>
<span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span>
<span class="source-line-no">017</span><span id="line-17"> */</span>
<span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.replication;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import java.io.IOException;</span>
<span class="source-line-no">021</span><span id="line-21">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">022</span><span id="line-22">import org.apache.hadoop.conf.Configured;</span>
<span class="source-line-no">023</span><span id="line-23">import org.apache.hadoop.fs.FileSystem;</span>
<span class="source-line-no">024</span><span id="line-24">import org.apache.hadoop.hbase.Abortable;</span>
<span class="source-line-no">025</span><span id="line-25">import org.apache.hadoop.hbase.HBaseConfiguration;</span>
<span class="source-line-no">026</span><span id="line-26">import org.apache.hadoop.hbase.HBaseInterfaceAudience;</span>
<span class="source-line-no">027</span><span id="line-27">import org.apache.hadoop.hbase.zookeeper.ZKWatcher;</span>
<span class="source-line-no">028</span><span id="line-28">import org.apache.hadoop.util.Tool;</span>
<span class="source-line-no">029</span><span id="line-29">import org.apache.hadoop.util.ToolRunner;</span>
<span class="source-line-no">030</span><span id="line-30">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">031</span><span id="line-31">import org.slf4j.Logger;</span>
<span class="source-line-no">032</span><span id="line-32">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">033</span><span id="line-33"></span>
<span class="source-line-no">034</span><span id="line-34">/**</span>
<span class="source-line-no">035</span><span id="line-35"> * A tool for copying replication peer data across different replication peer storages.</span>
<span class="source-line-no">036</span><span id="line-36"> * &lt;p/&gt;</span>
<span class="source-line-no">037</span><span id="line-37"> * Notice that we will not delete the replication peer data from the source storage, as this tool</span>
<span class="source-line-no">038</span><span id="line-38"> * can also be used by online migration. See HBASE-27110 for the whole design.</span>
<span class="source-line-no">039</span><span id="line-39"> */</span>
<span class="source-line-no">040</span><span id="line-40">@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)</span>
<span class="source-line-no">041</span><span id="line-41">public class CopyReplicationPeers extends Configured implements Tool {</span>
<span class="source-line-no">042</span><span id="line-42"></span>
<span class="source-line-no">043</span><span id="line-43"> private static final Logger LOG = LoggerFactory.getLogger(CopyReplicationPeers.class);</span>
<span class="source-line-no">044</span><span id="line-44"></span>
<span class="source-line-no">045</span><span id="line-45"> public static final String NAME = "copyreppeers";</span>
<span class="source-line-no">046</span><span id="line-46"></span>
<span class="source-line-no">047</span><span id="line-47"> public CopyReplicationPeers(Configuration conf) {</span>
<span class="source-line-no">048</span><span id="line-48"> super(conf);</span>
<span class="source-line-no">049</span><span id="line-49"> }</span>
<span class="source-line-no">050</span><span id="line-50"></span>
<span class="source-line-no">051</span><span id="line-51"> private ReplicationPeerStorage create(String type, FileSystem fs, ZKWatcher zk) {</span>
<span class="source-line-no">052</span><span id="line-52"> Configuration conf = new Configuration(getConf());</span>
<span class="source-line-no">053</span><span id="line-53"> conf.set(ReplicationStorageFactory.REPLICATION_PEER_STORAGE_IMPL, type);</span>
<span class="source-line-no">054</span><span id="line-54"> return ReplicationStorageFactory.getReplicationPeerStorage(fs, zk, conf);</span>
<span class="source-line-no">055</span><span id="line-55"> }</span>
<span class="source-line-no">056</span><span id="line-56"></span>
<span class="source-line-no">057</span><span id="line-57"> private ZKWatcher createZKWatcher() throws IOException {</span>
<span class="source-line-no">058</span><span id="line-58"> return new ZKWatcher(getConf(), getClass().getSimpleName(), new Abortable() {</span>
<span class="source-line-no">059</span><span id="line-59"></span>
<span class="source-line-no">060</span><span id="line-60"> private volatile boolean aborted;</span>
<span class="source-line-no">061</span><span id="line-61"></span>
<span class="source-line-no">062</span><span id="line-62"> @Override</span>
<span class="source-line-no">063</span><span id="line-63"> public boolean isAborted() {</span>
<span class="source-line-no">064</span><span id="line-64"> return aborted;</span>
<span class="source-line-no">065</span><span id="line-65"> }</span>
<span class="source-line-no">066</span><span id="line-66"></span>
<span class="source-line-no">067</span><span id="line-67"> @Override</span>
<span class="source-line-no">068</span><span id="line-68"> public void abort(String why, Throwable e) {</span>
<span class="source-line-no">069</span><span id="line-69"> aborted = true;</span>
<span class="source-line-no">070</span><span id="line-70"> LOG.error(why, e);</span>
<span class="source-line-no">071</span><span id="line-71"> System.exit(1);</span>
<span class="source-line-no">072</span><span id="line-72"> }</span>
<span class="source-line-no">073</span><span id="line-73"> });</span>
<span class="source-line-no">074</span><span id="line-74"> }</span>
<span class="source-line-no">075</span><span id="line-75"></span>
<span class="source-line-no">076</span><span id="line-76"> private void migrate(ReplicationPeerStorage src, ReplicationPeerStorage dst)</span>
<span class="source-line-no">077</span><span id="line-77"> throws ReplicationException {</span>
<span class="source-line-no">078</span><span id="line-78"> LOG.info("Start migrating from {} to {}", src.getClass().getSimpleName(),</span>
<span class="source-line-no">079</span><span id="line-79"> dst.getClass().getSimpleName());</span>
<span class="source-line-no">080</span><span id="line-80"> for (String peerId : src.listPeerIds()) {</span>
<span class="source-line-no">081</span><span id="line-81"> LOG.info("Going to migrate {}", peerId);</span>
<span class="source-line-no">082</span><span id="line-82"> ReplicationPeerConfig peerConfig = src.getPeerConfig(peerId);</span>
<span class="source-line-no">083</span><span id="line-83"> boolean enabled = src.isPeerEnabled(peerId);</span>
<span class="source-line-no">084</span><span id="line-84"> SyncReplicationState syncState = src.getPeerSyncReplicationState(peerId);</span>
<span class="source-line-no">085</span><span id="line-85"> SyncReplicationState newSyncState = src.getPeerNewSyncReplicationState(peerId);</span>
<span class="source-line-no">086</span><span id="line-86"> if (newSyncState != SyncReplicationState.NONE) {</span>
<span class="source-line-no">087</span><span id="line-87"> throw new IllegalStateException("Can not migrate peer " + peerId</span>
<span class="source-line-no">088</span><span id="line-88"> + " as it is in an intermediate state, syncReplicationState is " + syncState</span>
<span class="source-line-no">089</span><span id="line-89"> + " while newSyncReplicationState is " + newSyncState);</span>
<span class="source-line-no">090</span><span id="line-90"> }</span>
<span class="source-line-no">091</span><span id="line-91"> dst.addPeer(peerId, peerConfig, enabled, syncState);</span>
<span class="source-line-no">092</span><span id="line-92"> LOG.info("Migrated peer {}, peerConfig = '{}', enabled = {}, syncReplicationState = {}",</span>
<span class="source-line-no">093</span><span id="line-93"> peerId, peerConfig, enabled, syncState);</span>
<span class="source-line-no">094</span><span id="line-94"> }</span>
<span class="source-line-no">095</span><span id="line-95"> }</span>
<span class="source-line-no">096</span><span id="line-96"></span>
<span class="source-line-no">097</span><span id="line-97"> @Override</span>
<span class="source-line-no">098</span><span id="line-98"> public int run(String[] args) throws Exception {</span>
<span class="source-line-no">099</span><span id="line-99"> if (args.length != 2) {</span>
<span class="source-line-no">100</span><span id="line-100"> System.err.println("Usage: bin/hbase " + NAME</span>
<span class="source-line-no">101</span><span id="line-101"> + " &lt;SRC_REPLICATION_PEER_STORAGE&gt; &lt;DST_REPLICATION_PEER_STORAGE&gt;");</span>
<span class="source-line-no">102</span><span id="line-102"> System.err.println("The possible values for replication storage type:");</span>
<span class="source-line-no">103</span><span id="line-103"> for (ReplicationPeerStorageType type : ReplicationPeerStorageType.values()) {</span>
<span class="source-line-no">104</span><span id="line-104"> System.err.println(" " + type.name().toLowerCase());</span>
<span class="source-line-no">105</span><span id="line-105"> }</span>
<span class="source-line-no">106</span><span id="line-106"> return -1;</span>
<span class="source-line-no">107</span><span id="line-107"> }</span>
<span class="source-line-no">108</span><span id="line-108"> FileSystem fs = FileSystem.get(getConf());</span>
<span class="source-line-no">109</span><span id="line-109"> try (ZKWatcher zk = createZKWatcher()) {</span>
<span class="source-line-no">110</span><span id="line-110"> ReplicationPeerStorage src = create(args[0], fs, zk);</span>
<span class="source-line-no">111</span><span id="line-111"> ReplicationPeerStorage dst = create(args[1], fs, zk);</span>
<span class="source-line-no">112</span><span id="line-112"> migrate(src, dst);</span>
<span class="source-line-no">113</span><span id="line-113"> }</span>
<span class="source-line-no">114</span><span id="line-114"> return 0;</span>
<span class="source-line-no">115</span><span id="line-115"> }</span>
<span class="source-line-no">116</span><span id="line-116"></span>
<span class="source-line-no">117</span><span id="line-117"> public static void main(String[] args) throws Exception {</span>
<span class="source-line-no">118</span><span id="line-118"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">119</span><span id="line-119"> int ret = ToolRunner.run(conf, new CopyReplicationPeers(conf), args);</span>
<span class="source-line-no">120</span><span id="line-120"> System.exit(ret);</span>
<span class="source-line-no">121</span><span id="line-121"> }</span>
<span class="source-line-no">122</span><span id="line-122">}</span>
</pre>
</div>
</main>
</body>
</html>