blob: 55a90a9a1dcb4d1a4e5fc2b35bf987491d18c273 [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.regionserver, class: DumpReplicationQueues">
<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.regionserver;</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.FileNotFoundException;</span>
<span class="source-line-no">021</span><span id="line-21">import java.io.IOException;</span>
<span class="source-line-no">022</span><span id="line-22">import java.net.URLEncoder;</span>
<span class="source-line-no">023</span><span id="line-23">import java.nio.charset.StandardCharsets;</span>
<span class="source-line-no">024</span><span id="line-24">import java.util.ArrayList;</span>
<span class="source-line-no">025</span><span id="line-25">import java.util.Arrays;</span>
<span class="source-line-no">026</span><span id="line-26">import java.util.Collections;</span>
<span class="source-line-no">027</span><span id="line-27">import java.util.Comparator;</span>
<span class="source-line-no">028</span><span id="line-28">import java.util.LinkedList;</span>
<span class="source-line-no">029</span><span id="line-29">import java.util.List;</span>
<span class="source-line-no">030</span><span id="line-30">import java.util.Map;</span>
<span class="source-line-no">031</span><span id="line-31">import java.util.Queue;</span>
<span class="source-line-no">032</span><span id="line-32">import java.util.Set;</span>
<span class="source-line-no">033</span><span id="line-33">import java.util.stream.Collectors;</span>
<span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">035</span><span id="line-35">import org.apache.hadoop.conf.Configured;</span>
<span class="source-line-no">036</span><span id="line-36">import org.apache.hadoop.fs.FileStatus;</span>
<span class="source-line-no">037</span><span id="line-37">import org.apache.hadoop.fs.FileSystem;</span>
<span class="source-line-no">038</span><span id="line-38">import org.apache.hadoop.fs.Path;</span>
<span class="source-line-no">039</span><span id="line-39">import org.apache.hadoop.hbase.HBaseConfiguration;</span>
<span class="source-line-no">040</span><span id="line-40">import org.apache.hadoop.hbase.ServerName;</span>
<span class="source-line-no">041</span><span id="line-41">import org.apache.hadoop.hbase.client.Admin;</span>
<span class="source-line-no">042</span><span id="line-42">import org.apache.hadoop.hbase.client.Connection;</span>
<span class="source-line-no">043</span><span id="line-43">import org.apache.hadoop.hbase.client.ConnectionFactory;</span>
<span class="source-line-no">044</span><span id="line-44">import org.apache.hadoop.hbase.client.replication.TableCFs;</span>
<span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.hbase.io.WALLink;</span>
<span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.hbase.procedure2.util.StringUtils;</span>
<span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.hbase.replication.ReplicationException;</span>
<span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.hbase.replication.ReplicationGroupOffset;</span>
<span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.hbase.replication.ReplicationOffsetUtil;</span>
<span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;</span>
<span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;</span>
<span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.replication.ReplicationQueueData;</span>
<span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.replication.ReplicationQueueId;</span>
<span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;</span>
<span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;</span>
<span class="source-line-no">056</span><span id="line-56">import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;</span>
<span class="source-line-no">057</span><span id="line-57">import org.apache.hadoop.util.Tool;</span>
<span class="source-line-no">058</span><span id="line-58">import org.apache.hadoop.util.ToolRunner;</span>
<span class="source-line-no">059</span><span id="line-59">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">060</span><span id="line-60">import org.slf4j.Logger;</span>
<span class="source-line-no">061</span><span id="line-61">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">062</span><span id="line-62"></span>
<span class="source-line-no">063</span><span id="line-63">import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;</span>
<span class="source-line-no">064</span><span id="line-64">import org.apache.hbase.thirdparty.com.google.common.util.concurrent.AtomicLongMap;</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"> * &lt;p/&gt;</span>
<span class="source-line-no">068</span><span id="line-68"> * Provides information about the existing states of replication, replication peers and queues.</span>
<span class="source-line-no">069</span><span id="line-69"> * Usage: hbase org.apache.hadoop.hbase.replication.regionserver.DumpReplicationQueues [args]</span>
<span class="source-line-no">070</span><span id="line-70"> * Arguments: --distributed Polls each RS to dump information about the queue --hdfs Reports HDFS</span>
<span class="source-line-no">071</span><span id="line-71"> * usage by the replication queues (note: can be overestimated). In the new version, we</span>
<span class="source-line-no">072</span><span id="line-72"> * reimplemented the DumpReplicationQueues tool to support obtaining information from replication</span>
<span class="source-line-no">073</span><span id="line-73"> * table.</span>
<span class="source-line-no">074</span><span id="line-74"> */</span>
<span class="source-line-no">075</span><span id="line-75">@InterfaceAudience.Private</span>
<span class="source-line-no">076</span><span id="line-76">public class DumpReplicationQueues extends Configured implements Tool {</span>
<span class="source-line-no">077</span><span id="line-77"></span>
<span class="source-line-no">078</span><span id="line-78"> private static final Logger LOG = LoggerFactory.getLogger(DumpReplicationQueues.class.getName());</span>
<span class="source-line-no">079</span><span id="line-79"></span>
<span class="source-line-no">080</span><span id="line-80"> private List&lt;String&gt; deadRegionServers;</span>
<span class="source-line-no">081</span><span id="line-81"> private List&lt;String&gt; deletedQueues;</span>
<span class="source-line-no">082</span><span id="line-82"> private AtomicLongMap&lt;String&gt; peersQueueSize;</span>
<span class="source-line-no">083</span><span id="line-83"> private long totalSizeOfWALs;</span>
<span class="source-line-no">084</span><span id="line-84"> private long numWalsNotFound;</span>
<span class="source-line-no">085</span><span id="line-85"></span>
<span class="source-line-no">086</span><span id="line-86"> public DumpReplicationQueues() {</span>
<span class="source-line-no">087</span><span id="line-87"> deadRegionServers = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">088</span><span id="line-88"> deletedQueues = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">089</span><span id="line-89"> peersQueueSize = AtomicLongMap.create();</span>
<span class="source-line-no">090</span><span id="line-90"> totalSizeOfWALs = 0;</span>
<span class="source-line-no">091</span><span id="line-91"> numWalsNotFound = 0;</span>
<span class="source-line-no">092</span><span id="line-92"> }</span>
<span class="source-line-no">093</span><span id="line-93"></span>
<span class="source-line-no">094</span><span id="line-94"> static class DumpOptions {</span>
<span class="source-line-no">095</span><span id="line-95"> boolean hdfs = false;</span>
<span class="source-line-no">096</span><span id="line-96"> boolean distributed = false;</span>
<span class="source-line-no">097</span><span id="line-97"></span>
<span class="source-line-no">098</span><span id="line-98"> public DumpOptions() {</span>
<span class="source-line-no">099</span><span id="line-99"> }</span>
<span class="source-line-no">100</span><span id="line-100"></span>
<span class="source-line-no">101</span><span id="line-101"> public DumpOptions(DumpOptions that) {</span>
<span class="source-line-no">102</span><span id="line-102"> this.hdfs = that.hdfs;</span>
<span class="source-line-no">103</span><span id="line-103"> this.distributed = that.distributed;</span>
<span class="source-line-no">104</span><span id="line-104"> }</span>
<span class="source-line-no">105</span><span id="line-105"></span>
<span class="source-line-no">106</span><span id="line-106"> boolean isHdfs() {</span>
<span class="source-line-no">107</span><span id="line-107"> return hdfs;</span>
<span class="source-line-no">108</span><span id="line-108"> }</span>
<span class="source-line-no">109</span><span id="line-109"></span>
<span class="source-line-no">110</span><span id="line-110"> boolean isDistributed() {</span>
<span class="source-line-no">111</span><span id="line-111"> return distributed;</span>
<span class="source-line-no">112</span><span id="line-112"> }</span>
<span class="source-line-no">113</span><span id="line-113"></span>
<span class="source-line-no">114</span><span id="line-114"> void setHdfs(boolean hdfs) {</span>
<span class="source-line-no">115</span><span id="line-115"> this.hdfs = hdfs;</span>
<span class="source-line-no">116</span><span id="line-116"> }</span>
<span class="source-line-no">117</span><span id="line-117"></span>
<span class="source-line-no">118</span><span id="line-118"> void setDistributed(boolean distributed) {</span>
<span class="source-line-no">119</span><span id="line-119"> this.distributed = distributed;</span>
<span class="source-line-no">120</span><span id="line-120"> }</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>
<span class="source-line-no">123</span><span id="line-123"> static DumpOptions parseOpts(Queue&lt;String&gt; args) {</span>
<span class="source-line-no">124</span><span id="line-124"> DumpOptions opts = new DumpOptions();</span>
<span class="source-line-no">125</span><span id="line-125"></span>
<span class="source-line-no">126</span><span id="line-126"> String cmd = null;</span>
<span class="source-line-no">127</span><span id="line-127"> while ((cmd = args.poll()) != null) {</span>
<span class="source-line-no">128</span><span id="line-128"> if (cmd.equals("-h") || cmd.equals("--h") || cmd.equals("--help")) {</span>
<span class="source-line-no">129</span><span id="line-129"> // place item back onto queue so that caller knows parsing was incomplete</span>
<span class="source-line-no">130</span><span id="line-130"> args.add(cmd);</span>
<span class="source-line-no">131</span><span id="line-131"> break;</span>
<span class="source-line-no">132</span><span id="line-132"> }</span>
<span class="source-line-no">133</span><span id="line-133"> final String hdfs = "--hdfs";</span>
<span class="source-line-no">134</span><span id="line-134"> if (cmd.equals(hdfs)) {</span>
<span class="source-line-no">135</span><span id="line-135"> opts.setHdfs(true);</span>
<span class="source-line-no">136</span><span id="line-136"> continue;</span>
<span class="source-line-no">137</span><span id="line-137"> }</span>
<span class="source-line-no">138</span><span id="line-138"> final String distributed = "--distributed";</span>
<span class="source-line-no">139</span><span id="line-139"> if (cmd.equals(distributed)) {</span>
<span class="source-line-no">140</span><span id="line-140"> opts.setDistributed(true);</span>
<span class="source-line-no">141</span><span id="line-141"> continue;</span>
<span class="source-line-no">142</span><span id="line-142"> } else {</span>
<span class="source-line-no">143</span><span id="line-143"> printUsageAndExit("ERROR: Unrecognized option/command: " + cmd, -1);</span>
<span class="source-line-no">144</span><span id="line-144"> }</span>
<span class="source-line-no">145</span><span id="line-145"> // check that --distributed is present when --hdfs is in the arguments</span>
<span class="source-line-no">146</span><span id="line-146"> if (!opts.isDistributed() &amp;&amp; opts.isHdfs()) {</span>
<span class="source-line-no">147</span><span id="line-147"> printUsageAndExit("ERROR: --hdfs option can only be used with --distributed: " + cmd, -1);</span>
<span class="source-line-no">148</span><span id="line-148"> }</span>
<span class="source-line-no">149</span><span id="line-149"> }</span>
<span class="source-line-no">150</span><span id="line-150"> return opts;</span>
<span class="source-line-no">151</span><span id="line-151"> }</span>
<span class="source-line-no">152</span><span id="line-152"></span>
<span class="source-line-no">153</span><span id="line-153"> /**</span>
<span class="source-line-no">154</span><span id="line-154"> * Main</span>
<span class="source-line-no">155</span><span id="line-155"> */</span>
<span class="source-line-no">156</span><span id="line-156"> public static void main(String[] args) throws Exception {</span>
<span class="source-line-no">157</span><span id="line-157"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">158</span><span id="line-158"> int ret = ToolRunner.run(conf, new DumpReplicationQueues(), args);</span>
<span class="source-line-no">159</span><span id="line-159"> System.exit(ret);</span>
<span class="source-line-no">160</span><span id="line-160"> }</span>
<span class="source-line-no">161</span><span id="line-161"></span>
<span class="source-line-no">162</span><span id="line-162"> @Override</span>
<span class="source-line-no">163</span><span id="line-163"> public int run(String[] args) throws Exception {</span>
<span class="source-line-no">164</span><span id="line-164"></span>
<span class="source-line-no">165</span><span id="line-165"> int errCode = -1;</span>
<span class="source-line-no">166</span><span id="line-166"> LinkedList&lt;String&gt; argv = new LinkedList&lt;&gt;();</span>
<span class="source-line-no">167</span><span id="line-167"> argv.addAll(Arrays.asList(args));</span>
<span class="source-line-no">168</span><span id="line-168"> DumpOptions opts = parseOpts(argv);</span>
<span class="source-line-no">169</span><span id="line-169"></span>
<span class="source-line-no">170</span><span id="line-170"> // args remaining, print help and exit</span>
<span class="source-line-no">171</span><span id="line-171"> if (!argv.isEmpty()) {</span>
<span class="source-line-no">172</span><span id="line-172"> errCode = 0;</span>
<span class="source-line-no">173</span><span id="line-173"> printUsage();</span>
<span class="source-line-no">174</span><span id="line-174"> return errCode;</span>
<span class="source-line-no">175</span><span id="line-175"> }</span>
<span class="source-line-no">176</span><span id="line-176"> return dumpReplicationQueues(opts);</span>
<span class="source-line-no">177</span><span id="line-177"> }</span>
<span class="source-line-no">178</span><span id="line-178"></span>
<span class="source-line-no">179</span><span id="line-179"> protected void printUsage() {</span>
<span class="source-line-no">180</span><span id="line-180"> printUsage(this.getClass().getName(), null);</span>
<span class="source-line-no">181</span><span id="line-181"> }</span>
<span class="source-line-no">182</span><span id="line-182"></span>
<span class="source-line-no">183</span><span id="line-183"> protected static void printUsage(final String message) {</span>
<span class="source-line-no">184</span><span id="line-184"> printUsage(DumpReplicationQueues.class.getName(), message);</span>
<span class="source-line-no">185</span><span id="line-185"> }</span>
<span class="source-line-no">186</span><span id="line-186"></span>
<span class="source-line-no">187</span><span id="line-187"> protected static void printUsage(final String className, final String message) {</span>
<span class="source-line-no">188</span><span id="line-188"> if (message != null &amp;&amp; message.length() &gt; 0) {</span>
<span class="source-line-no">189</span><span id="line-189"> System.err.println(message);</span>
<span class="source-line-no">190</span><span id="line-190"> }</span>
<span class="source-line-no">191</span><span id="line-191"> System.err.println("Usage: hbase " + className + " \\");</span>
<span class="source-line-no">192</span><span id="line-192"> System.err.println(" &lt;OPTIONS&gt; [-D&lt;property=value&gt;]*");</span>
<span class="source-line-no">193</span><span id="line-193"> System.err.println();</span>
<span class="source-line-no">194</span><span id="line-194"> System.err.println("General Options:");</span>
<span class="source-line-no">195</span><span id="line-195"> System.err.println(" -h|--h|--help Show this help and exit.");</span>
<span class="source-line-no">196</span><span id="line-196"> System.err.println(" --distributed Poll each RS and print its own replication queue. "</span>
<span class="source-line-no">197</span><span id="line-197"> + "Default only polls replication table.");</span>
<span class="source-line-no">198</span><span id="line-198"> System.err.println(" --hdfs Use HDFS to calculate usage of WALs by replication."</span>
<span class="source-line-no">199</span><span id="line-199"> + " It could be overestimated if replicating to multiple peers."</span>
<span class="source-line-no">200</span><span id="line-200"> + " --distributed flag is also needed.");</span>
<span class="source-line-no">201</span><span id="line-201"> }</span>
<span class="source-line-no">202</span><span id="line-202"></span>
<span class="source-line-no">203</span><span id="line-203"> protected static void printUsageAndExit(final String message, final int exitCode) {</span>
<span class="source-line-no">204</span><span id="line-204"> printUsage(message);</span>
<span class="source-line-no">205</span><span id="line-205"> System.exit(exitCode);</span>
<span class="source-line-no">206</span><span id="line-206"> }</span>
<span class="source-line-no">207</span><span id="line-207"></span>
<span class="source-line-no">208</span><span id="line-208"> private int dumpReplicationQueues(DumpOptions opts) throws Exception {</span>
<span class="source-line-no">209</span><span id="line-209"> Configuration conf = getConf();</span>
<span class="source-line-no">210</span><span id="line-210"> Connection connection = ConnectionFactory.createConnection(conf);</span>
<span class="source-line-no">211</span><span id="line-211"> Admin admin = connection.getAdmin();</span>
<span class="source-line-no">212</span><span id="line-212"></span>
<span class="source-line-no">213</span><span id="line-213"> try {</span>
<span class="source-line-no">214</span><span id="line-214"> List&lt;TableCFs&gt; replicatedTableCFs = admin.listReplicatedTableCFs();</span>
<span class="source-line-no">215</span><span id="line-215"> if (replicatedTableCFs.isEmpty()) {</span>
<span class="source-line-no">216</span><span id="line-216"> LOG.info("No tables with a configured replication peer were found.");</span>
<span class="source-line-no">217</span><span id="line-217"> return (0);</span>
<span class="source-line-no">218</span><span id="line-218"> } else {</span>
<span class="source-line-no">219</span><span id="line-219"> LOG.info("Replicated Tables: " + replicatedTableCFs);</span>
<span class="source-line-no">220</span><span id="line-220"> }</span>
<span class="source-line-no">221</span><span id="line-221"></span>
<span class="source-line-no">222</span><span id="line-222"> List&lt;ReplicationPeerDescription&gt; peers = admin.listReplicationPeers();</span>
<span class="source-line-no">223</span><span id="line-223"></span>
<span class="source-line-no">224</span><span id="line-224"> if (peers.isEmpty()) {</span>
<span class="source-line-no">225</span><span id="line-225"> LOG.info("Replication is enabled but no peer configuration was found.");</span>
<span class="source-line-no">226</span><span id="line-226"> }</span>
<span class="source-line-no">227</span><span id="line-227"></span>
<span class="source-line-no">228</span><span id="line-228"> System.out.println("Dumping replication peers and configurations:");</span>
<span class="source-line-no">229</span><span id="line-229"> System.out.println(dumpPeersState(peers));</span>
<span class="source-line-no">230</span><span id="line-230"></span>
<span class="source-line-no">231</span><span id="line-231"> if (opts.isDistributed()) {</span>
<span class="source-line-no">232</span><span id="line-232"> LOG.info("Found [--distributed], will poll each RegionServer.");</span>
<span class="source-line-no">233</span><span id="line-233"> Set&lt;String&gt; peerIds =</span>
<span class="source-line-no">234</span><span id="line-234"> peers.stream().map((peer) -&gt; peer.getPeerId()).collect(Collectors.toSet());</span>
<span class="source-line-no">235</span><span id="line-235"> System.out.println(dumpQueues(connection, peerIds, opts.isHdfs(), conf));</span>
<span class="source-line-no">236</span><span id="line-236"> System.out.println(dumpReplicationSummary());</span>
<span class="source-line-no">237</span><span id="line-237"> } else {</span>
<span class="source-line-no">238</span><span id="line-238"> // use replication table instead</span>
<span class="source-line-no">239</span><span id="line-239"> System.out.println("Dumping replication info via replication table.");</span>
<span class="source-line-no">240</span><span id="line-240"> System.out.println(dumpReplicationViaTable(connection, conf));</span>
<span class="source-line-no">241</span><span id="line-241"> }</span>
<span class="source-line-no">242</span><span id="line-242"> return (0);</span>
<span class="source-line-no">243</span><span id="line-243"> } catch (IOException e) {</span>
<span class="source-line-no">244</span><span id="line-244"> return (-1);</span>
<span class="source-line-no">245</span><span id="line-245"> } finally {</span>
<span class="source-line-no">246</span><span id="line-246"> connection.close();</span>
<span class="source-line-no">247</span><span id="line-247"> }</span>
<span class="source-line-no">248</span><span id="line-248"> }</span>
<span class="source-line-no">249</span><span id="line-249"></span>
<span class="source-line-no">250</span><span id="line-250"> public String dumpReplicationViaTable(Connection connection, Configuration conf)</span>
<span class="source-line-no">251</span><span id="line-251"> throws ReplicationException, IOException {</span>
<span class="source-line-no">252</span><span id="line-252"> StringBuilder sb = new StringBuilder();</span>
<span class="source-line-no">253</span><span id="line-253"> ReplicationQueueStorage queueStorage =</span>
<span class="source-line-no">254</span><span id="line-254"> ReplicationStorageFactory.getReplicationQueueStorage(connection, conf);</span>
<span class="source-line-no">255</span><span id="line-255"></span>
<span class="source-line-no">256</span><span id="line-256"> // The dump info format is as follows:</span>
<span class="source-line-no">257</span><span id="line-257"> // peers:</span>
<span class="source-line-no">258</span><span id="line-258"> // peers/1: zk1:2181:/hbase</span>
<span class="source-line-no">259</span><span id="line-259"> // peers/1/peer-state: ENABLED</span>
<span class="source-line-no">260</span><span id="line-260"> // rs:</span>
<span class="source-line-no">261</span><span id="line-261"> // rs/rs1,16020,1664092120094/1/rs1%2C16020%2C1664092120094.1664096778778: 123</span>
<span class="source-line-no">262</span><span id="line-262"> // rs/rs2,16020,1664092120094/2/rs1%2C16020%2C1664092120094.1664096778778: 321</span>
<span class="source-line-no">263</span><span id="line-263"> // hfile-refs:</span>
<span class="source-line-no">264</span><span id="line-264"> // hfile-refs/1/hfile1,hfile2</span>
<span class="source-line-no">265</span><span id="line-265"> // hfile-refs/2/hfile3,hfile4</span>
<span class="source-line-no">266</span><span id="line-266"> String peersKey = "peers";</span>
<span class="source-line-no">267</span><span id="line-267"> sb.append(peersKey).append(": ").append("\n");</span>
<span class="source-line-no">268</span><span id="line-268"> List&lt;ReplicationPeerDescription&gt; repPeerDescs = connection.getAdmin().listReplicationPeers();</span>
<span class="source-line-no">269</span><span id="line-269"> for (ReplicationPeerDescription repPeerDesc : repPeerDescs) {</span>
<span class="source-line-no">270</span><span id="line-270"> sb.append(peersKey).append("/").append(repPeerDesc.getPeerId()).append(": ")</span>
<span class="source-line-no">271</span><span id="line-271"> .append(repPeerDesc.getPeerConfig().getClusterKey()).append("\n");</span>
<span class="source-line-no">272</span><span id="line-272"> sb.append(peersKey).append("/").append(repPeerDesc.getPeerId()).append("/peer-state: ")</span>
<span class="source-line-no">273</span><span id="line-273"> .append(repPeerDesc.isEnabled() ? "ENABLED" : "DISABLED").append("\n");</span>
<span class="source-line-no">274</span><span id="line-274"> }</span>
<span class="source-line-no">275</span><span id="line-275"></span>
<span class="source-line-no">276</span><span id="line-276"> List&lt;ReplicationQueueData&gt; repQueueDataList = queueStorage.listAllQueues();</span>
<span class="source-line-no">277</span><span id="line-277"> String rsKey = "rs";</span>
<span class="source-line-no">278</span><span id="line-278"> sb.append(rsKey).append(": ").append("\n");</span>
<span class="source-line-no">279</span><span id="line-279"> for (ReplicationQueueData repQueueData : repQueueDataList) {</span>
<span class="source-line-no">280</span><span id="line-280"> String peerId = repQueueData.getId().getPeerId();</span>
<span class="source-line-no">281</span><span id="line-281"> for (ImmutableMap.Entry&lt;String, ReplicationGroupOffset&gt; entry : repQueueData.getOffsets()</span>
<span class="source-line-no">282</span><span id="line-282"> .entrySet()) {</span>
<span class="source-line-no">283</span><span id="line-283"> sb.append(rsKey).append("/").append(entry.getKey()).append("/").append(peerId).append("/")</span>
<span class="source-line-no">284</span><span id="line-284"> .append(entry.getValue().getWal()).append(": ").append(entry.getValue().getOffset())</span>
<span class="source-line-no">285</span><span id="line-285"> .append("\n");</span>
<span class="source-line-no">286</span><span id="line-286"> }</span>
<span class="source-line-no">287</span><span id="line-287"> }</span>
<span class="source-line-no">288</span><span id="line-288"></span>
<span class="source-line-no">289</span><span id="line-289"> List&lt;String&gt; peerIds = queueStorage.getAllPeersFromHFileRefsQueue();</span>
<span class="source-line-no">290</span><span id="line-290"> String hfileKey = "hfile-refs";</span>
<span class="source-line-no">291</span><span id="line-291"> sb.append(hfileKey).append(": ").append("\n");</span>
<span class="source-line-no">292</span><span id="line-292"> for (String peerId : peerIds) {</span>
<span class="source-line-no">293</span><span id="line-293"> List&lt;String&gt; hfiles = queueStorage.getReplicableHFiles(peerId);</span>
<span class="source-line-no">294</span><span id="line-294"> sb.append(hfileKey).append("/").append(peerId).append("/").append(String.join(",", hfiles))</span>
<span class="source-line-no">295</span><span id="line-295"> .append("\n");</span>
<span class="source-line-no">296</span><span id="line-296"> }</span>
<span class="source-line-no">297</span><span id="line-297"></span>
<span class="source-line-no">298</span><span id="line-298"> return sb.toString();</span>
<span class="source-line-no">299</span><span id="line-299"> }</span>
<span class="source-line-no">300</span><span id="line-300"></span>
<span class="source-line-no">301</span><span id="line-301"> public String dumpReplicationSummary() {</span>
<span class="source-line-no">302</span><span id="line-302"> StringBuilder sb = new StringBuilder();</span>
<span class="source-line-no">303</span><span id="line-303"> if (!deletedQueues.isEmpty()) {</span>
<span class="source-line-no">304</span><span id="line-304"> sb.append("Found " + deletedQueues.size() + " deleted queues"</span>
<span class="source-line-no">305</span><span id="line-305"> + ", run hbck -fixReplication in order to remove the deleted replication queues\n");</span>
<span class="source-line-no">306</span><span id="line-306"> for (String deletedQueue : deletedQueues) {</span>
<span class="source-line-no">307</span><span id="line-307"> sb.append(" " + deletedQueue + "\n");</span>
<span class="source-line-no">308</span><span id="line-308"> }</span>
<span class="source-line-no">309</span><span id="line-309"> }</span>
<span class="source-line-no">310</span><span id="line-310"> if (!deadRegionServers.isEmpty()) {</span>
<span class="source-line-no">311</span><span id="line-311"> sb.append("Found " + deadRegionServers.size() + " dead regionservers"</span>
<span class="source-line-no">312</span><span id="line-312"> + ", restart one regionserver to transfer the queues of dead regionservers\n");</span>
<span class="source-line-no">313</span><span id="line-313"> for (String deadRs : deadRegionServers) {</span>
<span class="source-line-no">314</span><span id="line-314"> sb.append(" " + deadRs + "\n");</span>
<span class="source-line-no">315</span><span id="line-315"> }</span>
<span class="source-line-no">316</span><span id="line-316"> }</span>
<span class="source-line-no">317</span><span id="line-317"> if (!peersQueueSize.isEmpty()) {</span>
<span class="source-line-no">318</span><span id="line-318"> sb.append("Dumping all peers's number of WALs in replication queue\n");</span>
<span class="source-line-no">319</span><span id="line-319"> for (Map.Entry&lt;String, Long&gt; entry : peersQueueSize.asMap().entrySet()) {</span>
<span class="source-line-no">320</span><span id="line-320"> sb.append(</span>
<span class="source-line-no">321</span><span id="line-321"> " PeerId: " + entry.getKey() + " , sizeOfLogQueue: " + entry.getValue() + "\n");</span>
<span class="source-line-no">322</span><span id="line-322"> }</span>
<span class="source-line-no">323</span><span id="line-323"> }</span>
<span class="source-line-no">324</span><span id="line-324"> sb.append(" Total size of WALs on HDFS: " + StringUtils.humanSize(totalSizeOfWALs) + "\n");</span>
<span class="source-line-no">325</span><span id="line-325"> if (numWalsNotFound &gt; 0) {</span>
<span class="source-line-no">326</span><span id="line-326"> sb.append(" ERROR: There are " + numWalsNotFound + " WALs not found!!!\n");</span>
<span class="source-line-no">327</span><span id="line-327"> }</span>
<span class="source-line-no">328</span><span id="line-328"> return sb.toString();</span>
<span class="source-line-no">329</span><span id="line-329"> }</span>
<span class="source-line-no">330</span><span id="line-330"></span>
<span class="source-line-no">331</span><span id="line-331"> public String dumpPeersState(List&lt;ReplicationPeerDescription&gt; peers) throws Exception {</span>
<span class="source-line-no">332</span><span id="line-332"> Map&lt;String, String&gt; currentConf;</span>
<span class="source-line-no">333</span><span id="line-333"> StringBuilder sb = new StringBuilder();</span>
<span class="source-line-no">334</span><span id="line-334"> for (ReplicationPeerDescription peer : peers) {</span>
<span class="source-line-no">335</span><span id="line-335"> ReplicationPeerConfig peerConfig = peer.getPeerConfig();</span>
<span class="source-line-no">336</span><span id="line-336"> sb.append("Peer: " + peer.getPeerId() + "\n");</span>
<span class="source-line-no">337</span><span id="line-337"> sb.append(" " + "State: " + (peer.isEnabled() ? "ENABLED" : "DISABLED") + "\n");</span>
<span class="source-line-no">338</span><span id="line-338"> sb.append(" " + "Cluster Name: " + peerConfig.getClusterKey() + "\n");</span>
<span class="source-line-no">339</span><span id="line-339"> sb.append(" " + "Replication Endpoint: " + peerConfig.getReplicationEndpointImpl() + "\n");</span>
<span class="source-line-no">340</span><span id="line-340"> currentConf = peerConfig.getConfiguration();</span>
<span class="source-line-no">341</span><span id="line-341"> // Only show when we have a custom configuration for the peer</span>
<span class="source-line-no">342</span><span id="line-342"> if (currentConf.size() &gt; 1) {</span>
<span class="source-line-no">343</span><span id="line-343"> sb.append(" " + "Peer Configuration: " + currentConf + "\n");</span>
<span class="source-line-no">344</span><span id="line-344"> }</span>
<span class="source-line-no">345</span><span id="line-345"> sb.append(" " + "Peer Table CFs: " + peerConfig.getTableCFsMap() + "\n");</span>
<span class="source-line-no">346</span><span id="line-346"> sb.append(" " + "Peer Namespaces: " + peerConfig.getNamespaces() + "\n");</span>
<span class="source-line-no">347</span><span id="line-347"> }</span>
<span class="source-line-no">348</span><span id="line-348"> return sb.toString();</span>
<span class="source-line-no">349</span><span id="line-349"> }</span>
<span class="source-line-no">350</span><span id="line-350"></span>
<span class="source-line-no">351</span><span id="line-351"> public String dumpQueues(Connection connection, Set&lt;String&gt; peerIds, boolean hdfs,</span>
<span class="source-line-no">352</span><span id="line-352"> Configuration conf) throws Exception {</span>
<span class="source-line-no">353</span><span id="line-353"> StringBuilder sb = new StringBuilder();</span>
<span class="source-line-no">354</span><span id="line-354"> ReplicationQueueStorage queueStorage =</span>
<span class="source-line-no">355</span><span id="line-355"> ReplicationStorageFactory.getReplicationQueueStorage(connection, conf);</span>
<span class="source-line-no">356</span><span id="line-356"></span>
<span class="source-line-no">357</span><span id="line-357"> Set&lt;ServerName&gt; liveRegionServers =</span>
<span class="source-line-no">358</span><span id="line-358"> connection.getAdmin().getClusterMetrics().getLiveServerMetrics().keySet();</span>
<span class="source-line-no">359</span><span id="line-359"></span>
<span class="source-line-no">360</span><span id="line-360"> List&lt;ServerName&gt; regionServers = queueStorage.listAllReplicators();</span>
<span class="source-line-no">361</span><span id="line-361"> if (regionServers == null || regionServers.isEmpty()) {</span>
<span class="source-line-no">362</span><span id="line-362"> return sb.toString();</span>
<span class="source-line-no">363</span><span id="line-363"> }</span>
<span class="source-line-no">364</span><span id="line-364"> for (ServerName regionServer : regionServers) {</span>
<span class="source-line-no">365</span><span id="line-365"> List&lt;ReplicationQueueId&gt; queueIds = queueStorage.listAllQueueIds(regionServer);</span>
<span class="source-line-no">366</span><span id="line-366"></span>
<span class="source-line-no">367</span><span id="line-367"> if (!liveRegionServers.contains(regionServer)) {</span>
<span class="source-line-no">368</span><span id="line-368"> deadRegionServers.add(regionServer.getServerName());</span>
<span class="source-line-no">369</span><span id="line-369"> }</span>
<span class="source-line-no">370</span><span id="line-370"> for (ReplicationQueueId queueId : queueIds) {</span>
<span class="source-line-no">371</span><span id="line-371"> List&lt;String&gt; tmpWals = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">372</span><span id="line-372"> // wals</span>
<span class="source-line-no">373</span><span id="line-373"> AbstractFSWALProvider</span>
<span class="source-line-no">374</span><span id="line-374"> .getWALFiles(connection.getConfiguration(), queueId.getServerWALsBelongTo()).stream()</span>
<span class="source-line-no">375</span><span id="line-375"> .map(Path::toString).forEach(tmpWals::add);</span>
<span class="source-line-no">376</span><span id="line-376"></span>
<span class="source-line-no">377</span><span id="line-377"> // old wals</span>
<span class="source-line-no">378</span><span id="line-378"> AbstractFSWALProvider.getArchivedWALFiles(connection.getConfiguration(),</span>
<span class="source-line-no">379</span><span id="line-379"> queueId.getServerWALsBelongTo(), URLEncoder</span>
<span class="source-line-no">380</span><span id="line-380"> .encode(queueId.getServerWALsBelongTo().toString(), StandardCharsets.UTF_8.name()))</span>
<span class="source-line-no">381</span><span id="line-381"> .stream().map(Path::toString).forEach(tmpWals::add);</span>
<span class="source-line-no">382</span><span id="line-382"></span>
<span class="source-line-no">383</span><span id="line-383"> Map&lt;String, ReplicationGroupOffset&gt; offsets = queueStorage.getOffsets(queueId);</span>
<span class="source-line-no">384</span><span id="line-384"> // filter out the wal files that should replicate</span>
<span class="source-line-no">385</span><span id="line-385"> List&lt;String&gt; wals = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">386</span><span id="line-386"> for (Map.Entry&lt;String, ReplicationGroupOffset&gt; entry : offsets.entrySet()) {</span>
<span class="source-line-no">387</span><span id="line-387"> ReplicationGroupOffset offset = entry.getValue();</span>
<span class="source-line-no">388</span><span id="line-388"> for (String wal : tmpWals) {</span>
<span class="source-line-no">389</span><span id="line-389"> if (ReplicationOffsetUtil.shouldReplicate(offset, wal)) {</span>
<span class="source-line-no">390</span><span id="line-390"> wals.add(wal);</span>
<span class="source-line-no">391</span><span id="line-391"> }</span>
<span class="source-line-no">392</span><span id="line-392"> }</span>
<span class="source-line-no">393</span><span id="line-393"> }</span>
<span class="source-line-no">394</span><span id="line-394"> Collections.sort(wals, Comparator.comparingLong(AbstractFSWALProvider::getTimestamp));</span>
<span class="source-line-no">395</span><span id="line-395"> if (!peerIds.contains(queueId.getPeerId())) {</span>
<span class="source-line-no">396</span><span id="line-396"> deletedQueues.add(regionServer + "/" + queueId);</span>
<span class="source-line-no">397</span><span id="line-397"> sb.append(formatQueue(regionServer, offsets, wals, queueId, true, hdfs));</span>
<span class="source-line-no">398</span><span id="line-398"> } else {</span>
<span class="source-line-no">399</span><span id="line-399"> sb.append(formatQueue(regionServer, offsets, wals, queueId, false, hdfs));</span>
<span class="source-line-no">400</span><span id="line-400"> }</span>
<span class="source-line-no">401</span><span id="line-401"> }</span>
<span class="source-line-no">402</span><span id="line-402"> }</span>
<span class="source-line-no">403</span><span id="line-403"> return sb.toString();</span>
<span class="source-line-no">404</span><span id="line-404"> }</span>
<span class="source-line-no">405</span><span id="line-405"></span>
<span class="source-line-no">406</span><span id="line-406"> private String formatQueue(ServerName regionServer, Map&lt;String, ReplicationGroupOffset&gt; offsets,</span>
<span class="source-line-no">407</span><span id="line-407"> List&lt;String&gt; wals, ReplicationQueueId queueId, boolean isDeleted, boolean hdfs)</span>
<span class="source-line-no">408</span><span id="line-408"> throws Exception {</span>
<span class="source-line-no">409</span><span id="line-409"> StringBuilder sb = new StringBuilder();</span>
<span class="source-line-no">410</span><span id="line-410"></span>
<span class="source-line-no">411</span><span id="line-411"> sb.append("Dumping replication queue info for RegionServer: [" + regionServer + "]" + "\n");</span>
<span class="source-line-no">412</span><span id="line-412"> sb.append(" Queue id: " + queueId + "\n");</span>
<span class="source-line-no">413</span><span id="line-413"> sb.append(" PeerID: " + queueId.getPeerId() + "\n");</span>
<span class="source-line-no">414</span><span id="line-414"> sb.append(" Recovered: " + queueId.isRecovered() + "\n");</span>
<span class="source-line-no">415</span><span id="line-415"> // In new version, we only record the first dead RegionServer in queueId.</span>
<span class="source-line-no">416</span><span id="line-416"> if (queueId.getSourceServerName().isPresent()) {</span>
<span class="source-line-no">417</span><span id="line-417"> sb.append(" Dead RegionServer: " + queueId.getSourceServerName().get() + "\n");</span>
<span class="source-line-no">418</span><span id="line-418"> } else {</span>
<span class="source-line-no">419</span><span id="line-419"> sb.append(" No dead RegionServer found in this queue." + "\n");</span>
<span class="source-line-no">420</span><span id="line-420"> }</span>
<span class="source-line-no">421</span><span id="line-421"> sb.append(" Was deleted: " + isDeleted + "\n");</span>
<span class="source-line-no">422</span><span id="line-422"> sb.append(" Number of WALs in replication queue: " + wals.size() + "\n");</span>
<span class="source-line-no">423</span><span id="line-423"> peersQueueSize.addAndGet(queueId.getPeerId(), wals.size());</span>
<span class="source-line-no">424</span><span id="line-424"></span>
<span class="source-line-no">425</span><span id="line-425"> for (Map.Entry&lt;String, ReplicationGroupOffset&gt; entry : offsets.entrySet()) {</span>
<span class="source-line-no">426</span><span id="line-426"> String walGroup = entry.getKey();</span>
<span class="source-line-no">427</span><span id="line-427"> ReplicationGroupOffset offset = entry.getValue();</span>
<span class="source-line-no">428</span><span id="line-428"> for (String wal : wals) {</span>
<span class="source-line-no">429</span><span id="line-429"> long position = 0;</span>
<span class="source-line-no">430</span><span id="line-430"> if (offset.getWal().equals(wal)) {</span>
<span class="source-line-no">431</span><span id="line-431"> position = offset.getOffset();</span>
<span class="source-line-no">432</span><span id="line-432"> }</span>
<span class="source-line-no">433</span><span id="line-433"> sb.append(</span>
<span class="source-line-no">434</span><span id="line-434"> " Replication position for " + (walGroup != null ? walGroup + "/" + wal : wal) + ": ");</span>
<span class="source-line-no">435</span><span id="line-435"> if (position == 0) {</span>
<span class="source-line-no">436</span><span id="line-436"> sb.append("0 (not started or nothing to replicate)");</span>
<span class="source-line-no">437</span><span id="line-437"> } else if (position &gt; 0) {</span>
<span class="source-line-no">438</span><span id="line-438"> sb.append(position);</span>
<span class="source-line-no">439</span><span id="line-439"> }</span>
<span class="source-line-no">440</span><span id="line-440"> sb.append("\n");</span>
<span class="source-line-no">441</span><span id="line-441"> }</span>
<span class="source-line-no">442</span><span id="line-442"> }</span>
<span class="source-line-no">443</span><span id="line-443"></span>
<span class="source-line-no">444</span><span id="line-444"> if (hdfs) {</span>
<span class="source-line-no">445</span><span id="line-445"> FileSystem fs = FileSystem.get(getConf());</span>
<span class="source-line-no">446</span><span id="line-446"> sb.append(" Total size of WALs on HDFS for this queue: "</span>
<span class="source-line-no">447</span><span id="line-447"> + StringUtils.humanSize(getTotalWALSize(fs, wals, regionServer)) + "\n");</span>
<span class="source-line-no">448</span><span id="line-448"> }</span>
<span class="source-line-no">449</span><span id="line-449"> return sb.toString();</span>
<span class="source-line-no">450</span><span id="line-450"> }</span>
<span class="source-line-no">451</span><span id="line-451"></span>
<span class="source-line-no">452</span><span id="line-452"> /**</span>
<span class="source-line-no">453</span><span id="line-453"> * return total size in bytes from a list of WALs</span>
<span class="source-line-no">454</span><span id="line-454"> */</span>
<span class="source-line-no">455</span><span id="line-455"> private long getTotalWALSize(FileSystem fs, List&lt;String&gt; wals, ServerName server) {</span>
<span class="source-line-no">456</span><span id="line-456"> long size = 0;</span>
<span class="source-line-no">457</span><span id="line-457"> FileStatus fileStatus;</span>
<span class="source-line-no">458</span><span id="line-458"></span>
<span class="source-line-no">459</span><span id="line-459"> for (String wal : wals) {</span>
<span class="source-line-no">460</span><span id="line-460"> try {</span>
<span class="source-line-no">461</span><span id="line-461"> fileStatus = (new WALLink(getConf(), server.getServerName(), wal)).getFileStatus(fs);</span>
<span class="source-line-no">462</span><span id="line-462"> } catch (IOException e) {</span>
<span class="source-line-no">463</span><span id="line-463"> if (e instanceof FileNotFoundException) {</span>
<span class="source-line-no">464</span><span id="line-464"> numWalsNotFound++;</span>
<span class="source-line-no">465</span><span id="line-465"> LOG.warn("WAL " + wal + " couldn't be found, skipping", e);</span>
<span class="source-line-no">466</span><span id="line-466"> } else {</span>
<span class="source-line-no">467</span><span id="line-467"> LOG.warn("Can't get file status of WAL " + wal + ", skipping", e);</span>
<span class="source-line-no">468</span><span id="line-468"> }</span>
<span class="source-line-no">469</span><span id="line-469"> continue;</span>
<span class="source-line-no">470</span><span id="line-470"> }</span>
<span class="source-line-no">471</span><span id="line-471"> size += fileStatus.getLen();</span>
<span class="source-line-no">472</span><span id="line-472"> }</span>
<span class="source-line-no">473</span><span id="line-473"></span>
<span class="source-line-no">474</span><span id="line-474"> totalSizeOfWALs += size;</span>
<span class="source-line-no">475</span><span id="line-475"> return size;</span>
<span class="source-line-no">476</span><span id="line-476"> }</span>
<span class="source-line-no">477</span><span id="line-477">}</span>
</pre>
</div>
</main>
</body>
</html>