| <!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd"> |
| <html lang="en"> |
| <head> |
| <title>Source code</title> |
| <link rel="stylesheet" type="text/css" href="../../../../../../../stylesheet.css" title="Style"> |
| </head> |
| <body> |
| <div class="sourceContainer"> |
| <pre><span class="sourceLineNo">001</span>/*<a name="line.1"></a> |
| <span class="sourceLineNo">002</span> * Licensed to the Apache Software Foundation (ASF) under one<a name="line.2"></a> |
| <span class="sourceLineNo">003</span> * or more contributor license agreements. See the NOTICE file<a name="line.3"></a> |
| <span class="sourceLineNo">004</span> * distributed with this work for additional information<a name="line.4"></a> |
| <span class="sourceLineNo">005</span> * regarding copyright ownership. The ASF licenses this file<a name="line.5"></a> |
| <span class="sourceLineNo">006</span> * to you under the Apache License, Version 2.0 (the<a name="line.6"></a> |
| <span class="sourceLineNo">007</span> * "License"); you may not use this file except in compliance<a name="line.7"></a> |
| <span class="sourceLineNo">008</span> * with the License. You may obtain a copy of the License at<a name="line.8"></a> |
| <span class="sourceLineNo">009</span> *<a name="line.9"></a> |
| <span class="sourceLineNo">010</span> * http://www.apache.org/licenses/LICENSE-2.0<a name="line.10"></a> |
| <span class="sourceLineNo">011</span> *<a name="line.11"></a> |
| <span class="sourceLineNo">012</span> * Unless required by applicable law or agreed to in writing, software<a name="line.12"></a> |
| <span class="sourceLineNo">013</span> * distributed under the License is distributed on an "AS IS" BASIS,<a name="line.13"></a> |
| <span class="sourceLineNo">014</span> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.<a name="line.14"></a> |
| <span class="sourceLineNo">015</span> * See the License for the specific language governing permissions and<a name="line.15"></a> |
| <span class="sourceLineNo">016</span> * limitations under the License.<a name="line.16"></a> |
| <span class="sourceLineNo">017</span> */<a name="line.17"></a> |
| <span class="sourceLineNo">018</span>package org.apache.hadoop.hbase.mapreduce.replication;<a name="line.18"></a> |
| <span class="sourceLineNo">019</span><a name="line.19"></a> |
| <span class="sourceLineNo">020</span>import java.io.IOException;<a name="line.20"></a> |
| <span class="sourceLineNo">021</span>import java.util.Arrays;<a name="line.21"></a> |
| <span class="sourceLineNo">022</span>import java.util.List;<a name="line.22"></a> |
| <span class="sourceLineNo">023</span>import java.util.UUID;<a name="line.23"></a> |
| <span class="sourceLineNo">024</span>import java.util.concurrent.SynchronousQueue;<a name="line.24"></a> |
| <span class="sourceLineNo">025</span>import java.util.concurrent.ThreadPoolExecutor;<a name="line.25"></a> |
| <span class="sourceLineNo">026</span>import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy;<a name="line.26"></a> |
| <span class="sourceLineNo">027</span>import java.util.concurrent.TimeUnit;<a name="line.27"></a> |
| <span class="sourceLineNo">028</span>import org.apache.hadoop.conf.Configuration;<a name="line.28"></a> |
| <span class="sourceLineNo">029</span>import org.apache.hadoop.conf.Configured;<a name="line.29"></a> |
| <span class="sourceLineNo">030</span>import org.apache.hadoop.fs.FileSystem;<a name="line.30"></a> |
| <span class="sourceLineNo">031</span>import org.apache.hadoop.fs.Path;<a name="line.31"></a> |
| <span class="sourceLineNo">032</span>import org.apache.hadoop.hbase.Abortable;<a name="line.32"></a> |
| <span class="sourceLineNo">033</span>import org.apache.hadoop.hbase.HBaseConfiguration;<a name="line.33"></a> |
| <span class="sourceLineNo">034</span>import org.apache.hadoop.hbase.HConstants;<a name="line.34"></a> |
| <span class="sourceLineNo">035</span>import org.apache.hadoop.hbase.TableName;<a name="line.35"></a> |
| <span class="sourceLineNo">036</span>import org.apache.hadoop.hbase.client.Connection;<a name="line.36"></a> |
| <span class="sourceLineNo">037</span>import org.apache.hadoop.hbase.client.ConnectionFactory;<a name="line.37"></a> |
| <span class="sourceLineNo">038</span>import org.apache.hadoop.hbase.client.Put;<a name="line.38"></a> |
| <span class="sourceLineNo">039</span>import org.apache.hadoop.hbase.client.Result;<a name="line.39"></a> |
| <span class="sourceLineNo">040</span>import org.apache.hadoop.hbase.client.ResultScanner;<a name="line.40"></a> |
| <span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.client.Scan;<a name="line.41"></a> |
| <span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.client.Table;<a name="line.42"></a> |
| <span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.client.TableSnapshotScanner;<a name="line.43"></a> |
| <span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.filter.Filter;<a name="line.44"></a> |
| <span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.filter.FilterList;<a name="line.45"></a> |
| <span class="sourceLineNo">046</span>import org.apache.hadoop.hbase.filter.PrefixFilter;<a name="line.46"></a> |
| <span class="sourceLineNo">047</span>import org.apache.hadoop.hbase.io.ImmutableBytesWritable;<a name="line.47"></a> |
| <span class="sourceLineNo">048</span>import org.apache.hadoop.hbase.mapreduce.TableInputFormat;<a name="line.48"></a> |
| <span class="sourceLineNo">049</span>import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;<a name="line.49"></a> |
| <span class="sourceLineNo">050</span>import org.apache.hadoop.hbase.mapreduce.TableMapper;<a name="line.50"></a> |
| <span class="sourceLineNo">051</span>import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat;<a name="line.51"></a> |
| <span class="sourceLineNo">052</span>import org.apache.hadoop.hbase.mapreduce.TableSplit;<a name="line.52"></a> |
| <span class="sourceLineNo">053</span>import org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication.Verifier.Counters;<a name="line.53"></a> |
| <span class="sourceLineNo">054</span>import org.apache.hadoop.hbase.replication.ReplicationException;<a name="line.54"></a> |
| <span class="sourceLineNo">055</span>import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;<a name="line.55"></a> |
| <span class="sourceLineNo">056</span>import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;<a name="line.56"></a> |
| <span class="sourceLineNo">057</span>import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;<a name="line.57"></a> |
| <span class="sourceLineNo">058</span>import org.apache.hadoop.hbase.replication.ReplicationUtils;<a name="line.58"></a> |
| <span class="sourceLineNo">059</span>import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper;<a name="line.59"></a> |
| <span class="sourceLineNo">060</span>import org.apache.hadoop.hbase.util.Bytes;<a name="line.60"></a> |
| <span class="sourceLineNo">061</span>import org.apache.hadoop.hbase.util.CommonFSUtils;<a name="line.61"></a> |
| <span class="sourceLineNo">062</span>import org.apache.hadoop.hbase.util.Pair;<a name="line.62"></a> |
| <span class="sourceLineNo">063</span>import org.apache.hadoop.hbase.zookeeper.ZKConfig;<a name="line.63"></a> |
| <span class="sourceLineNo">064</span>import org.apache.hadoop.hbase.zookeeper.ZKWatcher;<a name="line.64"></a> |
| <span class="sourceLineNo">065</span>import org.apache.hadoop.mapreduce.InputSplit;<a name="line.65"></a> |
| <span class="sourceLineNo">066</span>import org.apache.hadoop.mapreduce.Job;<a name="line.66"></a> |
| <span class="sourceLineNo">067</span>import org.apache.hadoop.mapreduce.MRJobConfig;<a name="line.67"></a> |
| <span class="sourceLineNo">068</span>import org.apache.hadoop.mapreduce.Mapper;<a name="line.68"></a> |
| <span class="sourceLineNo">069</span>import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;<a name="line.69"></a> |
| <span class="sourceLineNo">070</span>import org.apache.hadoop.util.Tool;<a name="line.70"></a> |
| <span class="sourceLineNo">071</span>import org.apache.hadoop.util.ToolRunner;<a name="line.71"></a> |
| <span class="sourceLineNo">072</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.72"></a> |
| <span class="sourceLineNo">073</span>import org.slf4j.Logger;<a name="line.73"></a> |
| <span class="sourceLineNo">074</span>import org.slf4j.LoggerFactory;<a name="line.74"></a> |
| <span class="sourceLineNo">075</span><a name="line.75"></a> |
| <span class="sourceLineNo">076</span>/**<a name="line.76"></a> |
| <span class="sourceLineNo">077</span> * This map-only job compares the data from a local table with a remote one. Every cell is compared<a name="line.77"></a> |
| <span class="sourceLineNo">078</span> * and must have exactly the same keys (even timestamp) as well as same value. It is possible to<a name="line.78"></a> |
| <span class="sourceLineNo">079</span> * restrict the job by time range and families. The peer id that's provided must match the one given<a name="line.79"></a> |
| <span class="sourceLineNo">080</span> * when the replication stream was setup.<a name="line.80"></a> |
| <span class="sourceLineNo">081</span> * <p><a name="line.81"></a> |
| <span class="sourceLineNo">082</span> * Two counters are provided, Verifier.Counters.GOODROWS and BADROWS. The reason for a why a row is<a name="line.82"></a> |
| <span class="sourceLineNo">083</span> * different is shown in the map's log.<a name="line.83"></a> |
| <span class="sourceLineNo">084</span> */<a name="line.84"></a> |
| <span class="sourceLineNo">085</span>@InterfaceAudience.Private<a name="line.85"></a> |
| <span class="sourceLineNo">086</span>public class VerifyReplication extends Configured implements Tool {<a name="line.86"></a> |
| <span class="sourceLineNo">087</span><a name="line.87"></a> |
| <span class="sourceLineNo">088</span> private static final Logger LOG = LoggerFactory.getLogger(VerifyReplication.class);<a name="line.88"></a> |
| <span class="sourceLineNo">089</span><a name="line.89"></a> |
| <span class="sourceLineNo">090</span> public final static String NAME = "verifyrep";<a name="line.90"></a> |
| <span class="sourceLineNo">091</span> private final static String PEER_CONFIG_PREFIX = NAME + ".peer.";<a name="line.91"></a> |
| <span class="sourceLineNo">092</span> private static ThreadPoolExecutor reCompareExecutor = null;<a name="line.92"></a> |
| <span class="sourceLineNo">093</span> int reCompareTries = 0;<a name="line.93"></a> |
| <span class="sourceLineNo">094</span> int reCompareBackoffExponent = 0;<a name="line.94"></a> |
| <span class="sourceLineNo">095</span> int reCompareThreads = 0;<a name="line.95"></a> |
| <span class="sourceLineNo">096</span> int sleepMsBeforeReCompare = 0;<a name="line.96"></a> |
| <span class="sourceLineNo">097</span> long startTime = 0;<a name="line.97"></a> |
| <span class="sourceLineNo">098</span> long endTime = Long.MAX_VALUE;<a name="line.98"></a> |
| <span class="sourceLineNo">099</span> int batch = -1;<a name="line.99"></a> |
| <span class="sourceLineNo">100</span> int versions = -1;<a name="line.100"></a> |
| <span class="sourceLineNo">101</span> String tableName = null;<a name="line.101"></a> |
| <span class="sourceLineNo">102</span> String families = null;<a name="line.102"></a> |
| <span class="sourceLineNo">103</span> String delimiter = "";<a name="line.103"></a> |
| <span class="sourceLineNo">104</span> String peerId = null;<a name="line.104"></a> |
| <span class="sourceLineNo">105</span> String peerQuorumAddress = null;<a name="line.105"></a> |
| <span class="sourceLineNo">106</span> String rowPrefixes = null;<a name="line.106"></a> |
| <span class="sourceLineNo">107</span> boolean verbose = false;<a name="line.107"></a> |
| <span class="sourceLineNo">108</span> boolean includeDeletedCells = false;<a name="line.108"></a> |
| <span class="sourceLineNo">109</span> // Source table snapshot name<a name="line.109"></a> |
| <span class="sourceLineNo">110</span> String sourceSnapshotName = null;<a name="line.110"></a> |
| <span class="sourceLineNo">111</span> // Temp location in source cluster to restore source snapshot<a name="line.111"></a> |
| <span class="sourceLineNo">112</span> String sourceSnapshotTmpDir = null;<a name="line.112"></a> |
| <span class="sourceLineNo">113</span> // Peer table snapshot name<a name="line.113"></a> |
| <span class="sourceLineNo">114</span> String peerSnapshotName = null;<a name="line.114"></a> |
| <span class="sourceLineNo">115</span> // Temp location in peer cluster to restore peer snapshot<a name="line.115"></a> |
| <span class="sourceLineNo">116</span> String peerSnapshotTmpDir = null;<a name="line.116"></a> |
| <span class="sourceLineNo">117</span> // Peer cluster Hadoop FS address<a name="line.117"></a> |
| <span class="sourceLineNo">118</span> String peerFSAddress = null;<a name="line.118"></a> |
| <span class="sourceLineNo">119</span> // Peer cluster HBase root dir location<a name="line.119"></a> |
| <span class="sourceLineNo">120</span> String peerHBaseRootAddress = null;<a name="line.120"></a> |
| <span class="sourceLineNo">121</span> // Peer Table Name<a name="line.121"></a> |
| <span class="sourceLineNo">122</span> String peerTableName = null;<a name="line.122"></a> |
| <span class="sourceLineNo">123</span><a name="line.123"></a> |
| <span class="sourceLineNo">124</span> private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";<a name="line.124"></a> |
| <span class="sourceLineNo">125</span><a name="line.125"></a> |
| <span class="sourceLineNo">126</span> /**<a name="line.126"></a> |
| <span class="sourceLineNo">127</span> * Map-only comparator for 2 tables<a name="line.127"></a> |
| <span class="sourceLineNo">128</span> */<a name="line.128"></a> |
| <span class="sourceLineNo">129</span> public static class Verifier extends TableMapper<ImmutableBytesWritable, Put> {<a name="line.129"></a> |
| <span class="sourceLineNo">130</span><a name="line.130"></a> |
| <span class="sourceLineNo">131</span> public enum Counters {<a name="line.131"></a> |
| <span class="sourceLineNo">132</span> GOODROWS,<a name="line.132"></a> |
| <span class="sourceLineNo">133</span> BADROWS,<a name="line.133"></a> |
| <span class="sourceLineNo">134</span> ONLY_IN_SOURCE_TABLE_ROWS,<a name="line.134"></a> |
| <span class="sourceLineNo">135</span> ONLY_IN_PEER_TABLE_ROWS,<a name="line.135"></a> |
| <span class="sourceLineNo">136</span> CONTENT_DIFFERENT_ROWS,<a name="line.136"></a> |
| <span class="sourceLineNo">137</span> RECOMPARES,<a name="line.137"></a> |
| <span class="sourceLineNo">138</span> MAIN_THREAD_RECOMPARES,<a name="line.138"></a> |
| <span class="sourceLineNo">139</span> SOURCE_ROW_CHANGED,<a name="line.139"></a> |
| <span class="sourceLineNo">140</span> PEER_ROW_CHANGED,<a name="line.140"></a> |
| <span class="sourceLineNo">141</span> FAILED_RECOMPARE<a name="line.141"></a> |
| <span class="sourceLineNo">142</span> }<a name="line.142"></a> |
| <span class="sourceLineNo">143</span><a name="line.143"></a> |
| <span class="sourceLineNo">144</span> private Connection sourceConnection;<a name="line.144"></a> |
| <span class="sourceLineNo">145</span> private Table sourceTable;<a name="line.145"></a> |
| <span class="sourceLineNo">146</span> private Connection replicatedConnection;<a name="line.146"></a> |
| <span class="sourceLineNo">147</span> private Table replicatedTable;<a name="line.147"></a> |
| <span class="sourceLineNo">148</span> private ResultScanner replicatedScanner;<a name="line.148"></a> |
| <span class="sourceLineNo">149</span> private Result currentCompareRowInPeerTable;<a name="line.149"></a> |
| <span class="sourceLineNo">150</span> private Scan tableScan;<a name="line.150"></a> |
| <span class="sourceLineNo">151</span> private int reCompareTries;<a name="line.151"></a> |
| <span class="sourceLineNo">152</span> private int reCompareBackoffExponent;<a name="line.152"></a> |
| <span class="sourceLineNo">153</span> private int sleepMsBeforeReCompare;<a name="line.153"></a> |
| <span class="sourceLineNo">154</span> private String delimiter = "";<a name="line.154"></a> |
| <span class="sourceLineNo">155</span> private boolean verbose = false;<a name="line.155"></a> |
| <span class="sourceLineNo">156</span> private int batch = -1;<a name="line.156"></a> |
| <span class="sourceLineNo">157</span><a name="line.157"></a> |
| <span class="sourceLineNo">158</span> /**<a name="line.158"></a> |
| <span class="sourceLineNo">159</span> * Map method that compares every scanned row with the equivalent from a distant cluster.<a name="line.159"></a> |
| <span class="sourceLineNo">160</span> * @param row The current table row key.<a name="line.160"></a> |
| <span class="sourceLineNo">161</span> * @param value The columns.<a name="line.161"></a> |
| <span class="sourceLineNo">162</span> * @param context The current context.<a name="line.162"></a> |
| <span class="sourceLineNo">163</span> * @throws IOException When something is broken with the data.<a name="line.163"></a> |
| <span class="sourceLineNo">164</span> */<a name="line.164"></a> |
| <span class="sourceLineNo">165</span> @Override<a name="line.165"></a> |
| <span class="sourceLineNo">166</span> public void map(ImmutableBytesWritable row, final Result value, Context context)<a name="line.166"></a> |
| <span class="sourceLineNo">167</span> throws IOException {<a name="line.167"></a> |
| <span class="sourceLineNo">168</span> if (replicatedScanner == null) {<a name="line.168"></a> |
| <span class="sourceLineNo">169</span> Configuration conf = context.getConfiguration();<a name="line.169"></a> |
| <span class="sourceLineNo">170</span> reCompareTries = conf.getInt(NAME + ".recompareTries", 0);<a name="line.170"></a> |
| <span class="sourceLineNo">171</span> reCompareBackoffExponent = conf.getInt(NAME + ".recompareBackoffExponent", 1);<a name="line.171"></a> |
| <span class="sourceLineNo">172</span> sleepMsBeforeReCompare = conf.getInt(NAME + ".sleepMsBeforeReCompare", 0);<a name="line.172"></a> |
| <span class="sourceLineNo">173</span> if (sleepMsBeforeReCompare > 0) {<a name="line.173"></a> |
| <span class="sourceLineNo">174</span> reCompareTries = Math.max(reCompareTries, 1);<a name="line.174"></a> |
| <span class="sourceLineNo">175</span> }<a name="line.175"></a> |
| <span class="sourceLineNo">176</span> delimiter = conf.get(NAME + ".delimiter", "");<a name="line.176"></a> |
| <span class="sourceLineNo">177</span> verbose = conf.getBoolean(NAME + ".verbose", false);<a name="line.177"></a> |
| <span class="sourceLineNo">178</span> batch = conf.getInt(NAME + ".batch", -1);<a name="line.178"></a> |
| <span class="sourceLineNo">179</span> final Scan scan = new Scan();<a name="line.179"></a> |
| <span class="sourceLineNo">180</span> if (batch > 0) {<a name="line.180"></a> |
| <span class="sourceLineNo">181</span> scan.setBatch(batch);<a name="line.181"></a> |
| <span class="sourceLineNo">182</span> }<a name="line.182"></a> |
| <span class="sourceLineNo">183</span> scan.setCacheBlocks(false);<a name="line.183"></a> |
| <span class="sourceLineNo">184</span> scan.setCaching(conf.getInt(TableInputFormat.SCAN_CACHEDROWS, 1));<a name="line.184"></a> |
| <span class="sourceLineNo">185</span> long startTime = conf.getLong(NAME + ".startTime", 0);<a name="line.185"></a> |
| <span class="sourceLineNo">186</span> long endTime = conf.getLong(NAME + ".endTime", Long.MAX_VALUE);<a name="line.186"></a> |
| <span class="sourceLineNo">187</span> String families = conf.get(NAME + ".families", null);<a name="line.187"></a> |
| <span class="sourceLineNo">188</span> if (families != null) {<a name="line.188"></a> |
| <span class="sourceLineNo">189</span> String[] fams = families.split(",");<a name="line.189"></a> |
| <span class="sourceLineNo">190</span> for (String fam : fams) {<a name="line.190"></a> |
| <span class="sourceLineNo">191</span> scan.addFamily(Bytes.toBytes(fam));<a name="line.191"></a> |
| <span class="sourceLineNo">192</span> }<a name="line.192"></a> |
| <span class="sourceLineNo">193</span> }<a name="line.193"></a> |
| <span class="sourceLineNo">194</span> boolean includeDeletedCells = conf.getBoolean(NAME + ".includeDeletedCells", false);<a name="line.194"></a> |
| <span class="sourceLineNo">195</span> scan.setRaw(includeDeletedCells);<a name="line.195"></a> |
| <span class="sourceLineNo">196</span> String rowPrefixes = conf.get(NAME + ".rowPrefixes", null);<a name="line.196"></a> |
| <span class="sourceLineNo">197</span> setRowPrefixFilter(scan, rowPrefixes);<a name="line.197"></a> |
| <span class="sourceLineNo">198</span> scan.setTimeRange(startTime, endTime);<a name="line.198"></a> |
| <span class="sourceLineNo">199</span> int versions = conf.getInt(NAME + ".versions", -1);<a name="line.199"></a> |
| <span class="sourceLineNo">200</span> LOG.info("Setting number of version inside map as: " + versions);<a name="line.200"></a> |
| <span class="sourceLineNo">201</span> if (versions >= 0) {<a name="line.201"></a> |
| <span class="sourceLineNo">202</span> scan.readVersions(versions);<a name="line.202"></a> |
| <span class="sourceLineNo">203</span> }<a name="line.203"></a> |
| <span class="sourceLineNo">204</span> int reCompareThreads = conf.getInt(NAME + ".recompareThreads", 0);<a name="line.204"></a> |
| <span class="sourceLineNo">205</span> reCompareExecutor = buildReCompareExecutor(reCompareThreads, context);<a name="line.205"></a> |
| <span class="sourceLineNo">206</span> TableName tableName = TableName.valueOf(conf.get(NAME + ".tableName"));<a name="line.206"></a> |
| <span class="sourceLineNo">207</span> sourceConnection = ConnectionFactory.createConnection(conf);<a name="line.207"></a> |
| <span class="sourceLineNo">208</span> sourceTable = sourceConnection.getTable(tableName);<a name="line.208"></a> |
| <span class="sourceLineNo">209</span> tableScan = scan;<a name="line.209"></a> |
| <span class="sourceLineNo">210</span><a name="line.210"></a> |
| <span class="sourceLineNo">211</span> final InputSplit tableSplit = context.getInputSplit();<a name="line.211"></a> |
| <span class="sourceLineNo">212</span><a name="line.212"></a> |
| <span class="sourceLineNo">213</span> String zkClusterKey = conf.get(NAME + ".peerQuorumAddress");<a name="line.213"></a> |
| <span class="sourceLineNo">214</span> Configuration peerConf =<a name="line.214"></a> |
| <span class="sourceLineNo">215</span> HBaseConfiguration.createClusterConf(conf, zkClusterKey, PEER_CONFIG_PREFIX);<a name="line.215"></a> |
| <span class="sourceLineNo">216</span><a name="line.216"></a> |
| <span class="sourceLineNo">217</span> String peerName = peerConf.get(NAME + ".peerTableName", tableName.getNameAsString());<a name="line.217"></a> |
| <span class="sourceLineNo">218</span> TableName peerTableName = TableName.valueOf(peerName);<a name="line.218"></a> |
| <span class="sourceLineNo">219</span> replicatedConnection = ConnectionFactory.createConnection(peerConf);<a name="line.219"></a> |
| <span class="sourceLineNo">220</span> replicatedTable = replicatedConnection.getTable(peerTableName);<a name="line.220"></a> |
| <span class="sourceLineNo">221</span> scan.withStartRow(value.getRow());<a name="line.221"></a> |
| <span class="sourceLineNo">222</span><a name="line.222"></a> |
| <span class="sourceLineNo">223</span> byte[] endRow = null;<a name="line.223"></a> |
| <span class="sourceLineNo">224</span> if (tableSplit instanceof TableSnapshotInputFormat.TableSnapshotRegionSplit) {<a name="line.224"></a> |
| <span class="sourceLineNo">225</span> endRow = ((TableSnapshotInputFormat.TableSnapshotRegionSplit) tableSplit).getRegion()<a name="line.225"></a> |
| <span class="sourceLineNo">226</span> .getEndKey();<a name="line.226"></a> |
| <span class="sourceLineNo">227</span> } else {<a name="line.227"></a> |
| <span class="sourceLineNo">228</span> endRow = ((TableSplit) tableSplit).getEndRow();<a name="line.228"></a> |
| <span class="sourceLineNo">229</span> }<a name="line.229"></a> |
| <span class="sourceLineNo">230</span><a name="line.230"></a> |
| <span class="sourceLineNo">231</span> scan.withStopRow(endRow);<a name="line.231"></a> |
| <span class="sourceLineNo">232</span><a name="line.232"></a> |
| <span class="sourceLineNo">233</span> String peerSnapshotName = conf.get(NAME + ".peerSnapshotName", null);<a name="line.233"></a> |
| <span class="sourceLineNo">234</span> if (peerSnapshotName != null) {<a name="line.234"></a> |
| <span class="sourceLineNo">235</span> String peerSnapshotTmpDir = conf.get(NAME + ".peerSnapshotTmpDir", null);<a name="line.235"></a> |
| <span class="sourceLineNo">236</span> String peerFSAddress = conf.get(NAME + ".peerFSAddress", null);<a name="line.236"></a> |
| <span class="sourceLineNo">237</span> String peerHBaseRootAddress = conf.get(NAME + ".peerHBaseRootAddress", null);<a name="line.237"></a> |
| <span class="sourceLineNo">238</span> FileSystem.setDefaultUri(peerConf, peerFSAddress);<a name="line.238"></a> |
| <span class="sourceLineNo">239</span> CommonFSUtils.setRootDir(peerConf, new Path(peerHBaseRootAddress));<a name="line.239"></a> |
| <span class="sourceLineNo">240</span> LOG.info("Using peer snapshot:" + peerSnapshotName + " with temp dir:"<a name="line.240"></a> |
| <span class="sourceLineNo">241</span> + peerSnapshotTmpDir + " peer root uri:" + CommonFSUtils.getRootDir(peerConf)<a name="line.241"></a> |
| <span class="sourceLineNo">242</span> + " peerFSAddress:" + peerFSAddress);<a name="line.242"></a> |
| <span class="sourceLineNo">243</span><a name="line.243"></a> |
| <span class="sourceLineNo">244</span> replicatedScanner = new TableSnapshotScanner(peerConf, CommonFSUtils.getRootDir(peerConf),<a name="line.244"></a> |
| <span class="sourceLineNo">245</span> new Path(peerFSAddress, peerSnapshotTmpDir), peerSnapshotName, scan, true);<a name="line.245"></a> |
| <span class="sourceLineNo">246</span> } else {<a name="line.246"></a> |
| <span class="sourceLineNo">247</span> replicatedScanner = replicatedTable.getScanner(scan);<a name="line.247"></a> |
| <span class="sourceLineNo">248</span> }<a name="line.248"></a> |
| <span class="sourceLineNo">249</span> currentCompareRowInPeerTable = replicatedScanner.next();<a name="line.249"></a> |
| <span class="sourceLineNo">250</span> }<a name="line.250"></a> |
| <span class="sourceLineNo">251</span> while (true) {<a name="line.251"></a> |
| <span class="sourceLineNo">252</span> if (currentCompareRowInPeerTable == null) {<a name="line.252"></a> |
| <span class="sourceLineNo">253</span> // reach the region end of peer table, row only in source table<a name="line.253"></a> |
| <span class="sourceLineNo">254</span> logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_SOURCE_TABLE_ROWS, value, null);<a name="line.254"></a> |
| <span class="sourceLineNo">255</span> break;<a name="line.255"></a> |
| <span class="sourceLineNo">256</span> }<a name="line.256"></a> |
| <span class="sourceLineNo">257</span> int rowCmpRet = Bytes.compareTo(value.getRow(), currentCompareRowInPeerTable.getRow());<a name="line.257"></a> |
| <span class="sourceLineNo">258</span> if (rowCmpRet == 0) {<a name="line.258"></a> |
| <span class="sourceLineNo">259</span> // rowkey is same, need to compare the content of the row<a name="line.259"></a> |
| <span class="sourceLineNo">260</span> try {<a name="line.260"></a> |
| <span class="sourceLineNo">261</span> Result.compareResults(value, currentCompareRowInPeerTable, false);<a name="line.261"></a> |
| <span class="sourceLineNo">262</span> context.getCounter(Counters.GOODROWS).increment(1);<a name="line.262"></a> |
| <span class="sourceLineNo">263</span> if (verbose) {<a name="line.263"></a> |
| <span class="sourceLineNo">264</span> LOG.info(<a name="line.264"></a> |
| <span class="sourceLineNo">265</span> "Good row key: " + delimiter + Bytes.toStringBinary(value.getRow()) + delimiter);<a name="line.265"></a> |
| <span class="sourceLineNo">266</span> }<a name="line.266"></a> |
| <span class="sourceLineNo">267</span> } catch (Exception e) {<a name="line.267"></a> |
| <span class="sourceLineNo">268</span> logFailRowAndIncreaseCounter(context, Counters.CONTENT_DIFFERENT_ROWS, value,<a name="line.268"></a> |
| <span class="sourceLineNo">269</span> currentCompareRowInPeerTable);<a name="line.269"></a> |
| <span class="sourceLineNo">270</span> }<a name="line.270"></a> |
| <span class="sourceLineNo">271</span> currentCompareRowInPeerTable = replicatedScanner.next();<a name="line.271"></a> |
| <span class="sourceLineNo">272</span> break;<a name="line.272"></a> |
| <span class="sourceLineNo">273</span> } else if (rowCmpRet < 0) {<a name="line.273"></a> |
| <span class="sourceLineNo">274</span> // row only exists in source table<a name="line.274"></a> |
| <span class="sourceLineNo">275</span> logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_SOURCE_TABLE_ROWS, value, null);<a name="line.275"></a> |
| <span class="sourceLineNo">276</span> break;<a name="line.276"></a> |
| <span class="sourceLineNo">277</span> } else {<a name="line.277"></a> |
| <span class="sourceLineNo">278</span> // row only exists in peer table<a name="line.278"></a> |
| <span class="sourceLineNo">279</span> logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_PEER_TABLE_ROWS, null,<a name="line.279"></a> |
| <span class="sourceLineNo">280</span> currentCompareRowInPeerTable);<a name="line.280"></a> |
| <span class="sourceLineNo">281</span> currentCompareRowInPeerTable = replicatedScanner.next();<a name="line.281"></a> |
| <span class="sourceLineNo">282</span> }<a name="line.282"></a> |
| <span class="sourceLineNo">283</span> }<a name="line.283"></a> |
| <span class="sourceLineNo">284</span> }<a name="line.284"></a> |
| <span class="sourceLineNo">285</span><a name="line.285"></a> |
| <span class="sourceLineNo">286</span> @SuppressWarnings("FutureReturnValueIgnored")<a name="line.286"></a> |
| <span class="sourceLineNo">287</span> private void logFailRowAndIncreaseCounter(Context context, Counters counter, Result row,<a name="line.287"></a> |
| <span class="sourceLineNo">288</span> Result replicatedRow) {<a name="line.288"></a> |
| <span class="sourceLineNo">289</span> byte[] rowKey = getRow(row, replicatedRow);<a name="line.289"></a> |
| <span class="sourceLineNo">290</span> if (reCompareTries == 0) {<a name="line.290"></a> |
| <span class="sourceLineNo">291</span> context.getCounter(counter).increment(1);<a name="line.291"></a> |
| <span class="sourceLineNo">292</span> context.getCounter(Counters.BADROWS).increment(1);<a name="line.292"></a> |
| <span class="sourceLineNo">293</span> LOG.error("{}, rowkey={}{}{}", counter, delimiter, Bytes.toStringBinary(rowKey), delimiter);<a name="line.293"></a> |
| <span class="sourceLineNo">294</span> return;<a name="line.294"></a> |
| <span class="sourceLineNo">295</span> }<a name="line.295"></a> |
| <span class="sourceLineNo">296</span><a name="line.296"></a> |
| <span class="sourceLineNo">297</span> VerifyReplicationRecompareRunnable runnable = new VerifyReplicationRecompareRunnable(context,<a name="line.297"></a> |
| <span class="sourceLineNo">298</span> row, replicatedRow, counter, delimiter, tableScan, sourceTable, replicatedTable,<a name="line.298"></a> |
| <span class="sourceLineNo">299</span> reCompareTries, sleepMsBeforeReCompare, reCompareBackoffExponent, verbose);<a name="line.299"></a> |
| <span class="sourceLineNo">300</span><a name="line.300"></a> |
| <span class="sourceLineNo">301</span> if (reCompareExecutor == null) {<a name="line.301"></a> |
| <span class="sourceLineNo">302</span> runnable.run();<a name="line.302"></a> |
| <span class="sourceLineNo">303</span> return;<a name="line.303"></a> |
| <span class="sourceLineNo">304</span> }<a name="line.304"></a> |
| <span class="sourceLineNo">305</span><a name="line.305"></a> |
| <span class="sourceLineNo">306</span> reCompareExecutor.submit(runnable);<a name="line.306"></a> |
| <span class="sourceLineNo">307</span> }<a name="line.307"></a> |
| <span class="sourceLineNo">308</span><a name="line.308"></a> |
| <span class="sourceLineNo">309</span> @Override<a name="line.309"></a> |
| <span class="sourceLineNo">310</span> protected void cleanup(Context context) {<a name="line.310"></a> |
| <span class="sourceLineNo">311</span> if (reCompareExecutor != null && !reCompareExecutor.isShutdown()) {<a name="line.311"></a> |
| <span class="sourceLineNo">312</span> reCompareExecutor.shutdown();<a name="line.312"></a> |
| <span class="sourceLineNo">313</span> try {<a name="line.313"></a> |
| <span class="sourceLineNo">314</span> boolean terminated = reCompareExecutor.awaitTermination(1, TimeUnit.MINUTES);<a name="line.314"></a> |
| <span class="sourceLineNo">315</span> if (!terminated) {<a name="line.315"></a> |
| <span class="sourceLineNo">316</span> List<Runnable> queue = reCompareExecutor.shutdownNow();<a name="line.316"></a> |
| <span class="sourceLineNo">317</span> for (Runnable runnable : queue) {<a name="line.317"></a> |
| <span class="sourceLineNo">318</span> ((VerifyReplicationRecompareRunnable) runnable).fail();<a name="line.318"></a> |
| <span class="sourceLineNo">319</span> }<a name="line.319"></a> |
| <span class="sourceLineNo">320</span><a name="line.320"></a> |
| <span class="sourceLineNo">321</span> terminated = reCompareExecutor.awaitTermination(1, TimeUnit.MINUTES);<a name="line.321"></a> |
| <span class="sourceLineNo">322</span><a name="line.322"></a> |
| <span class="sourceLineNo">323</span> if (!terminated) {<a name="line.323"></a> |
| <span class="sourceLineNo">324</span> int activeCount = Math.max(1, reCompareExecutor.getActiveCount());<a name="line.324"></a> |
| <span class="sourceLineNo">325</span> LOG.warn("Found {} possible recompares still running in the executable"<a name="line.325"></a> |
| <span class="sourceLineNo">326</span> + " incrementing BADROWS and FAILED_RECOMPARE", activeCount);<a name="line.326"></a> |
| <span class="sourceLineNo">327</span> context.getCounter(Counters.BADROWS).increment(activeCount);<a name="line.327"></a> |
| <span class="sourceLineNo">328</span> context.getCounter(Counters.FAILED_RECOMPARE).increment(activeCount);<a name="line.328"></a> |
| <span class="sourceLineNo">329</span> }<a name="line.329"></a> |
| <span class="sourceLineNo">330</span> }<a name="line.330"></a> |
| <span class="sourceLineNo">331</span> } catch (InterruptedException e) {<a name="line.331"></a> |
| <span class="sourceLineNo">332</span> throw new RuntimeException("Failed to await executor termination in cleanup", e);<a name="line.332"></a> |
| <span class="sourceLineNo">333</span> }<a name="line.333"></a> |
| <span class="sourceLineNo">334</span> }<a name="line.334"></a> |
| <span class="sourceLineNo">335</span> if (replicatedScanner != null) {<a name="line.335"></a> |
| <span class="sourceLineNo">336</span> try {<a name="line.336"></a> |
| <span class="sourceLineNo">337</span> while (currentCompareRowInPeerTable != null) {<a name="line.337"></a> |
| <span class="sourceLineNo">338</span> logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_PEER_TABLE_ROWS, null,<a name="line.338"></a> |
| <span class="sourceLineNo">339</span> currentCompareRowInPeerTable);<a name="line.339"></a> |
| <span class="sourceLineNo">340</span> currentCompareRowInPeerTable = replicatedScanner.next();<a name="line.340"></a> |
| <span class="sourceLineNo">341</span> }<a name="line.341"></a> |
| <span class="sourceLineNo">342</span> } catch (Exception e) {<a name="line.342"></a> |
| <span class="sourceLineNo">343</span> LOG.error("fail to scan peer table in cleanup", e);<a name="line.343"></a> |
| <span class="sourceLineNo">344</span> } finally {<a name="line.344"></a> |
| <span class="sourceLineNo">345</span> replicatedScanner.close();<a name="line.345"></a> |
| <span class="sourceLineNo">346</span> replicatedScanner = null;<a name="line.346"></a> |
| <span class="sourceLineNo">347</span> }<a name="line.347"></a> |
| <span class="sourceLineNo">348</span> }<a name="line.348"></a> |
| <span class="sourceLineNo">349</span><a name="line.349"></a> |
| <span class="sourceLineNo">350</span> if (sourceTable != null) {<a name="line.350"></a> |
| <span class="sourceLineNo">351</span> try {<a name="line.351"></a> |
| <span class="sourceLineNo">352</span> sourceTable.close();<a name="line.352"></a> |
| <span class="sourceLineNo">353</span> } catch (IOException e) {<a name="line.353"></a> |
| <span class="sourceLineNo">354</span> LOG.error("fail to close source table in cleanup", e);<a name="line.354"></a> |
| <span class="sourceLineNo">355</span> }<a name="line.355"></a> |
| <span class="sourceLineNo">356</span> }<a name="line.356"></a> |
| <span class="sourceLineNo">357</span> if (sourceConnection != null) {<a name="line.357"></a> |
| <span class="sourceLineNo">358</span> try {<a name="line.358"></a> |
| <span class="sourceLineNo">359</span> sourceConnection.close();<a name="line.359"></a> |
| <span class="sourceLineNo">360</span> } catch (Exception e) {<a name="line.360"></a> |
| <span class="sourceLineNo">361</span> LOG.error("fail to close source connection in cleanup", e);<a name="line.361"></a> |
| <span class="sourceLineNo">362</span> }<a name="line.362"></a> |
| <span class="sourceLineNo">363</span> }<a name="line.363"></a> |
| <span class="sourceLineNo">364</span><a name="line.364"></a> |
| <span class="sourceLineNo">365</span> if (replicatedTable != null) {<a name="line.365"></a> |
| <span class="sourceLineNo">366</span> try {<a name="line.366"></a> |
| <span class="sourceLineNo">367</span> replicatedTable.close();<a name="line.367"></a> |
| <span class="sourceLineNo">368</span> } catch (Exception e) {<a name="line.368"></a> |
| <span class="sourceLineNo">369</span> LOG.error("fail to close replicated table in cleanup", e);<a name="line.369"></a> |
| <span class="sourceLineNo">370</span> }<a name="line.370"></a> |
| <span class="sourceLineNo">371</span> }<a name="line.371"></a> |
| <span class="sourceLineNo">372</span> if (replicatedConnection != null) {<a name="line.372"></a> |
| <span class="sourceLineNo">373</span> try {<a name="line.373"></a> |
| <span class="sourceLineNo">374</span> replicatedConnection.close();<a name="line.374"></a> |
| <span class="sourceLineNo">375</span> } catch (Exception e) {<a name="line.375"></a> |
| <span class="sourceLineNo">376</span> LOG.error("fail to close replicated connection in cleanup", e);<a name="line.376"></a> |
| <span class="sourceLineNo">377</span> }<a name="line.377"></a> |
| <span class="sourceLineNo">378</span> }<a name="line.378"></a> |
| <span class="sourceLineNo">379</span> }<a name="line.379"></a> |
| <span class="sourceLineNo">380</span> }<a name="line.380"></a> |
| <span class="sourceLineNo">381</span><a name="line.381"></a> |
| <span class="sourceLineNo">382</span> private static Pair<ReplicationPeerConfig, Configuration><a name="line.382"></a> |
| <span class="sourceLineNo">383</span> getPeerQuorumConfig(final Configuration conf, String peerId) throws IOException {<a name="line.383"></a> |
| <span class="sourceLineNo">384</span> ZKWatcher localZKW = null;<a name="line.384"></a> |
| <span class="sourceLineNo">385</span> try {<a name="line.385"></a> |
| <span class="sourceLineNo">386</span> localZKW = new ZKWatcher(conf, "VerifyReplication", new Abortable() {<a name="line.386"></a> |
| <span class="sourceLineNo">387</span> @Override<a name="line.387"></a> |
| <span class="sourceLineNo">388</span> public void abort(String why, Throwable e) {<a name="line.388"></a> |
| <span class="sourceLineNo">389</span> }<a name="line.389"></a> |
| <span class="sourceLineNo">390</span><a name="line.390"></a> |
| <span class="sourceLineNo">391</span> @Override<a name="line.391"></a> |
| <span class="sourceLineNo">392</span> public boolean isAborted() {<a name="line.392"></a> |
| <span class="sourceLineNo">393</span> return false;<a name="line.393"></a> |
| <span class="sourceLineNo">394</span> }<a name="line.394"></a> |
| <span class="sourceLineNo">395</span> });<a name="line.395"></a> |
| <span class="sourceLineNo">396</span> ReplicationPeerStorage storage =<a name="line.396"></a> |
| <span class="sourceLineNo">397</span> ReplicationStorageFactory.getReplicationPeerStorage(FileSystem.get(conf), localZKW, conf);<a name="line.397"></a> |
| <span class="sourceLineNo">398</span> ReplicationPeerConfig peerConfig = storage.getPeerConfig(peerId);<a name="line.398"></a> |
| <span class="sourceLineNo">399</span> return Pair.newPair(peerConfig,<a name="line.399"></a> |
| <span class="sourceLineNo">400</span> ReplicationUtils.getPeerClusterConfiguration(peerConfig, conf));<a name="line.400"></a> |
| <span class="sourceLineNo">401</span> } catch (ReplicationException e) {<a name="line.401"></a> |
| <span class="sourceLineNo">402</span> throw new IOException("An error occurred while trying to connect to the remote peer cluster",<a name="line.402"></a> |
| <span class="sourceLineNo">403</span> e);<a name="line.403"></a> |
| <span class="sourceLineNo">404</span> } finally {<a name="line.404"></a> |
| <span class="sourceLineNo">405</span> if (localZKW != null) {<a name="line.405"></a> |
| <span class="sourceLineNo">406</span> localZKW.close();<a name="line.406"></a> |
| <span class="sourceLineNo">407</span> }<a name="line.407"></a> |
| <span class="sourceLineNo">408</span> }<a name="line.408"></a> |
| <span class="sourceLineNo">409</span> }<a name="line.409"></a> |
| <span class="sourceLineNo">410</span><a name="line.410"></a> |
| <span class="sourceLineNo">411</span> private void restoreSnapshotForPeerCluster(Configuration conf, String peerQuorumAddress)<a name="line.411"></a> |
| <span class="sourceLineNo">412</span> throws IOException {<a name="line.412"></a> |
| <span class="sourceLineNo">413</span> Configuration peerConf =<a name="line.413"></a> |
| <span class="sourceLineNo">414</span> HBaseConfiguration.createClusterConf(conf, peerQuorumAddress, PEER_CONFIG_PREFIX);<a name="line.414"></a> |
| <span class="sourceLineNo">415</span> FileSystem.setDefaultUri(peerConf, peerFSAddress);<a name="line.415"></a> |
| <span class="sourceLineNo">416</span> CommonFSUtils.setRootDir(peerConf, new Path(peerFSAddress, peerHBaseRootAddress));<a name="line.416"></a> |
| <span class="sourceLineNo">417</span> FileSystem fs = FileSystem.get(peerConf);<a name="line.417"></a> |
| <span class="sourceLineNo">418</span> RestoreSnapshotHelper.copySnapshotForScanner(peerConf, fs, CommonFSUtils.getRootDir(peerConf),<a name="line.418"></a> |
| <span class="sourceLineNo">419</span> new Path(peerFSAddress, peerSnapshotTmpDir), peerSnapshotName);<a name="line.419"></a> |
| <span class="sourceLineNo">420</span> }<a name="line.420"></a> |
| <span class="sourceLineNo">421</span><a name="line.421"></a> |
| <span class="sourceLineNo">422</span> /**<a name="line.422"></a> |
| <span class="sourceLineNo">423</span> * Sets up the actual job.<a name="line.423"></a> |
| <span class="sourceLineNo">424</span> * @param conf The current configuration.<a name="line.424"></a> |
| <span class="sourceLineNo">425</span> * @param args The command line parameters.<a name="line.425"></a> |
| <span class="sourceLineNo">426</span> * @return The newly created job.<a name="line.426"></a> |
| <span class="sourceLineNo">427</span> * @throws java.io.IOException When setting up the job fails.<a name="line.427"></a> |
| <span class="sourceLineNo">428</span> */<a name="line.428"></a> |
| <span class="sourceLineNo">429</span> public Job createSubmittableJob(Configuration conf, String[] args) throws IOException {<a name="line.429"></a> |
| <span class="sourceLineNo">430</span> if (!doCommandLine(args)) {<a name="line.430"></a> |
| <span class="sourceLineNo">431</span> return null;<a name="line.431"></a> |
| <span class="sourceLineNo">432</span> }<a name="line.432"></a> |
| <span class="sourceLineNo">433</span> conf.set(NAME + ".tableName", tableName);<a name="line.433"></a> |
| <span class="sourceLineNo">434</span> conf.setLong(NAME + ".startTime", startTime);<a name="line.434"></a> |
| <span class="sourceLineNo">435</span> conf.setLong(NAME + ".endTime", endTime);<a name="line.435"></a> |
| <span class="sourceLineNo">436</span> conf.setInt(NAME + ".sleepMsBeforeReCompare", sleepMsBeforeReCompare);<a name="line.436"></a> |
| <span class="sourceLineNo">437</span> conf.set(NAME + ".delimiter", delimiter);<a name="line.437"></a> |
| <span class="sourceLineNo">438</span> conf.setInt(NAME + ".batch", batch);<a name="line.438"></a> |
| <span class="sourceLineNo">439</span> conf.setBoolean(NAME + ".verbose", verbose);<a name="line.439"></a> |
| <span class="sourceLineNo">440</span> conf.setBoolean(NAME + ".includeDeletedCells", includeDeletedCells);<a name="line.440"></a> |
| <span class="sourceLineNo">441</span> if (families != null) {<a name="line.441"></a> |
| <span class="sourceLineNo">442</span> conf.set(NAME + ".families", families);<a name="line.442"></a> |
| <span class="sourceLineNo">443</span> }<a name="line.443"></a> |
| <span class="sourceLineNo">444</span> if (rowPrefixes != null) {<a name="line.444"></a> |
| <span class="sourceLineNo">445</span> conf.set(NAME + ".rowPrefixes", rowPrefixes);<a name="line.445"></a> |
| <span class="sourceLineNo">446</span> }<a name="line.446"></a> |
| <span class="sourceLineNo">447</span><a name="line.447"></a> |
| <span class="sourceLineNo">448</span> String peerQuorumAddress;<a name="line.448"></a> |
| <span class="sourceLineNo">449</span> Pair<ReplicationPeerConfig, Configuration> peerConfigPair = null;<a name="line.449"></a> |
| <span class="sourceLineNo">450</span> if (peerId != null) {<a name="line.450"></a> |
| <span class="sourceLineNo">451</span> peerConfigPair = getPeerQuorumConfig(conf, peerId);<a name="line.451"></a> |
| <span class="sourceLineNo">452</span> ReplicationPeerConfig peerConfig = peerConfigPair.getFirst();<a name="line.452"></a> |
| <span class="sourceLineNo">453</span> peerQuorumAddress = peerConfig.getClusterKey();<a name="line.453"></a> |
| <span class="sourceLineNo">454</span> LOG.info("Peer Quorum Address: " + peerQuorumAddress + ", Peer Configuration: "<a name="line.454"></a> |
| <span class="sourceLineNo">455</span> + peerConfig.getConfiguration());<a name="line.455"></a> |
| <span class="sourceLineNo">456</span> conf.set(NAME + ".peerQuorumAddress", peerQuorumAddress);<a name="line.456"></a> |
| <span class="sourceLineNo">457</span> HBaseConfiguration.setWithPrefix(conf, PEER_CONFIG_PREFIX,<a name="line.457"></a> |
| <span class="sourceLineNo">458</span> peerConfig.getConfiguration().entrySet());<a name="line.458"></a> |
| <span class="sourceLineNo">459</span> } else {<a name="line.459"></a> |
| <span class="sourceLineNo">460</span> assert this.peerQuorumAddress != null;<a name="line.460"></a> |
| <span class="sourceLineNo">461</span> peerQuorumAddress = this.peerQuorumAddress;<a name="line.461"></a> |
| <span class="sourceLineNo">462</span> LOG.info("Peer Quorum Address: " + peerQuorumAddress);<a name="line.462"></a> |
| <span class="sourceLineNo">463</span> conf.set(NAME + ".peerQuorumAddress", peerQuorumAddress);<a name="line.463"></a> |
| <span class="sourceLineNo">464</span> }<a name="line.464"></a> |
| <span class="sourceLineNo">465</span><a name="line.465"></a> |
| <span class="sourceLineNo">466</span> if (peerTableName != null) {<a name="line.466"></a> |
| <span class="sourceLineNo">467</span> LOG.info("Peer Table Name: " + peerTableName);<a name="line.467"></a> |
| <span class="sourceLineNo">468</span> conf.set(NAME + ".peerTableName", peerTableName);<a name="line.468"></a> |
| <span class="sourceLineNo">469</span> }<a name="line.469"></a> |
| <span class="sourceLineNo">470</span><a name="line.470"></a> |
| <span class="sourceLineNo">471</span> conf.setInt(NAME + ".versions", versions);<a name="line.471"></a> |
| <span class="sourceLineNo">472</span> LOG.info("Number of version: " + versions);<a name="line.472"></a> |
| <span class="sourceLineNo">473</span><a name="line.473"></a> |
| <span class="sourceLineNo">474</span> conf.setInt(NAME + ".recompareTries", reCompareTries);<a name="line.474"></a> |
| <span class="sourceLineNo">475</span> conf.setInt(NAME + ".recompareBackoffExponent", reCompareBackoffExponent);<a name="line.475"></a> |
| <span class="sourceLineNo">476</span> conf.setInt(NAME + ".recompareThreads", reCompareThreads);<a name="line.476"></a> |
| <span class="sourceLineNo">477</span><a name="line.477"></a> |
| <span class="sourceLineNo">478</span> // Set Snapshot specific parameters<a name="line.478"></a> |
| <span class="sourceLineNo">479</span> if (peerSnapshotName != null) {<a name="line.479"></a> |
| <span class="sourceLineNo">480</span> conf.set(NAME + ".peerSnapshotName", peerSnapshotName);<a name="line.480"></a> |
| <span class="sourceLineNo">481</span><a name="line.481"></a> |
| <span class="sourceLineNo">482</span> // for verifyRep by snapshot, choose a unique sub-directory under peerSnapshotTmpDir to<a name="line.482"></a> |
| <span class="sourceLineNo">483</span> // restore snapshot.<a name="line.483"></a> |
| <span class="sourceLineNo">484</span> Path restoreDir = new Path(peerSnapshotTmpDir, UUID.randomUUID().toString());<a name="line.484"></a> |
| <span class="sourceLineNo">485</span> peerSnapshotTmpDir = restoreDir.toString();<a name="line.485"></a> |
| <span class="sourceLineNo">486</span> conf.set(NAME + ".peerSnapshotTmpDir", peerSnapshotTmpDir);<a name="line.486"></a> |
| <span class="sourceLineNo">487</span><a name="line.487"></a> |
| <span class="sourceLineNo">488</span> conf.set(NAME + ".peerFSAddress", peerFSAddress);<a name="line.488"></a> |
| <span class="sourceLineNo">489</span> conf.set(NAME + ".peerHBaseRootAddress", peerHBaseRootAddress);<a name="line.489"></a> |
| <span class="sourceLineNo">490</span><a name="line.490"></a> |
| <span class="sourceLineNo">491</span> // This is to create HDFS delegation token for peer cluster in case of secured<a name="line.491"></a> |
| <span class="sourceLineNo">492</span> conf.setStrings(MRJobConfig.JOB_NAMENODES, peerFSAddress, conf.get(HConstants.HBASE_DIR));<a name="line.492"></a> |
| <span class="sourceLineNo">493</span> }<a name="line.493"></a> |
| <span class="sourceLineNo">494</span><a name="line.494"></a> |
| <span class="sourceLineNo">495</span> Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));<a name="line.495"></a> |
| <span class="sourceLineNo">496</span> job.setJarByClass(VerifyReplication.class);<a name="line.496"></a> |
| <span class="sourceLineNo">497</span><a name="line.497"></a> |
| <span class="sourceLineNo">498</span> Scan scan = new Scan();<a name="line.498"></a> |
| <span class="sourceLineNo">499</span> scan.setTimeRange(startTime, endTime);<a name="line.499"></a> |
| <span class="sourceLineNo">500</span> scan.setRaw(includeDeletedCells);<a name="line.500"></a> |
| <span class="sourceLineNo">501</span> scan.setCacheBlocks(false);<a name="line.501"></a> |
| <span class="sourceLineNo">502</span> if (batch > 0) {<a name="line.502"></a> |
| <span class="sourceLineNo">503</span> scan.setBatch(batch);<a name="line.503"></a> |
| <span class="sourceLineNo">504</span> }<a name="line.504"></a> |
| <span class="sourceLineNo">505</span> if (versions >= 0) {<a name="line.505"></a> |
| <span class="sourceLineNo">506</span> scan.readVersions(versions);<a name="line.506"></a> |
| <span class="sourceLineNo">507</span> LOG.info("Number of versions set to " + versions);<a name="line.507"></a> |
| <span class="sourceLineNo">508</span> }<a name="line.508"></a> |
| <span class="sourceLineNo">509</span> if (families != null) {<a name="line.509"></a> |
| <span class="sourceLineNo">510</span> String[] fams = families.split(",");<a name="line.510"></a> |
| <span class="sourceLineNo">511</span> for (String fam : fams) {<a name="line.511"></a> |
| <span class="sourceLineNo">512</span> scan.addFamily(Bytes.toBytes(fam));<a name="line.512"></a> |
| <span class="sourceLineNo">513</span> }<a name="line.513"></a> |
| <span class="sourceLineNo">514</span> }<a name="line.514"></a> |
| <span class="sourceLineNo">515</span><a name="line.515"></a> |
| <span class="sourceLineNo">516</span> setRowPrefixFilter(scan, rowPrefixes);<a name="line.516"></a> |
| <span class="sourceLineNo">517</span><a name="line.517"></a> |
| <span class="sourceLineNo">518</span> if (sourceSnapshotName != null) {<a name="line.518"></a> |
| <span class="sourceLineNo">519</span> Path snapshotTempPath = new Path(sourceSnapshotTmpDir);<a name="line.519"></a> |
| <span class="sourceLineNo">520</span> LOG.info(<a name="line.520"></a> |
| <span class="sourceLineNo">521</span> "Using source snapshot-" + sourceSnapshotName + " with temp dir:" + sourceSnapshotTmpDir);<a name="line.521"></a> |
| <span class="sourceLineNo">522</span> TableMapReduceUtil.initTableSnapshotMapperJob(sourceSnapshotName, scan, Verifier.class, null,<a name="line.522"></a> |
| <span class="sourceLineNo">523</span> null, job, true, snapshotTempPath);<a name="line.523"></a> |
| <span class="sourceLineNo">524</span> restoreSnapshotForPeerCluster(conf, peerQuorumAddress);<a name="line.524"></a> |
| <span class="sourceLineNo">525</span> } else {<a name="line.525"></a> |
| <span class="sourceLineNo">526</span> TableMapReduceUtil.initTableMapperJob(tableName, scan, Verifier.class, null, null, job);<a name="line.526"></a> |
| <span class="sourceLineNo">527</span> }<a name="line.527"></a> |
| <span class="sourceLineNo">528</span><a name="line.528"></a> |
| <span class="sourceLineNo">529</span> Configuration peerClusterConf;<a name="line.529"></a> |
| <span class="sourceLineNo">530</span> if (peerId != null) {<a name="line.530"></a> |
| <span class="sourceLineNo">531</span> assert peerConfigPair != null;<a name="line.531"></a> |
| <span class="sourceLineNo">532</span> peerClusterConf = peerConfigPair.getSecond();<a name="line.532"></a> |
| <span class="sourceLineNo">533</span> } else {<a name="line.533"></a> |
| <span class="sourceLineNo">534</span> peerClusterConf =<a name="line.534"></a> |
| <span class="sourceLineNo">535</span> HBaseConfiguration.createClusterConf(conf, peerQuorumAddress, PEER_CONFIG_PREFIX);<a name="line.535"></a> |
| <span class="sourceLineNo">536</span> }<a name="line.536"></a> |
| <span class="sourceLineNo">537</span> // Obtain the auth token from peer cluster<a name="line.537"></a> |
| <span class="sourceLineNo">538</span> TableMapReduceUtil.initCredentialsForCluster(job, peerClusterConf);<a name="line.538"></a> |
| <span class="sourceLineNo">539</span><a name="line.539"></a> |
| <span class="sourceLineNo">540</span> job.setOutputFormatClass(NullOutputFormat.class);<a name="line.540"></a> |
| <span class="sourceLineNo">541</span> job.setNumReduceTasks(0);<a name="line.541"></a> |
| <span class="sourceLineNo">542</span> return job;<a name="line.542"></a> |
| <span class="sourceLineNo">543</span> }<a name="line.543"></a> |
| <span class="sourceLineNo">544</span><a name="line.544"></a> |
| <span class="sourceLineNo">545</span> protected static byte[] getRow(Result sourceResult, Result replicatedResult) {<a name="line.545"></a> |
| <span class="sourceLineNo">546</span> if (sourceResult != null) {<a name="line.546"></a> |
| <span class="sourceLineNo">547</span> return sourceResult.getRow();<a name="line.547"></a> |
| <span class="sourceLineNo">548</span> } else if (replicatedResult != null) {<a name="line.548"></a> |
| <span class="sourceLineNo">549</span> return replicatedResult.getRow();<a name="line.549"></a> |
| <span class="sourceLineNo">550</span> }<a name="line.550"></a> |
| <span class="sourceLineNo">551</span> throw new RuntimeException("Both sourceResult and replicatedResult are null!");<a name="line.551"></a> |
| <span class="sourceLineNo">552</span> }<a name="line.552"></a> |
| <span class="sourceLineNo">553</span><a name="line.553"></a> |
| <span class="sourceLineNo">554</span> private static void setRowPrefixFilter(Scan scan, String rowPrefixes) {<a name="line.554"></a> |
| <span class="sourceLineNo">555</span> if (rowPrefixes != null && !rowPrefixes.isEmpty()) {<a name="line.555"></a> |
| <span class="sourceLineNo">556</span> String[] rowPrefixArray = rowPrefixes.split(",");<a name="line.556"></a> |
| <span class="sourceLineNo">557</span> Arrays.sort(rowPrefixArray);<a name="line.557"></a> |
| <span class="sourceLineNo">558</span> FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);<a name="line.558"></a> |
| <span class="sourceLineNo">559</span> for (String prefix : rowPrefixArray) {<a name="line.559"></a> |
| <span class="sourceLineNo">560</span> Filter filter = new PrefixFilter(Bytes.toBytes(prefix));<a name="line.560"></a> |
| <span class="sourceLineNo">561</span> filterList.addFilter(filter);<a name="line.561"></a> |
| <span class="sourceLineNo">562</span> }<a name="line.562"></a> |
| <span class="sourceLineNo">563</span> scan.setFilter(filterList);<a name="line.563"></a> |
| <span class="sourceLineNo">564</span> byte[] startPrefixRow = Bytes.toBytes(rowPrefixArray[0]);<a name="line.564"></a> |
| <span class="sourceLineNo">565</span> byte[] lastPrefixRow = Bytes.toBytes(rowPrefixArray[rowPrefixArray.length - 1]);<a name="line.565"></a> |
| <span class="sourceLineNo">566</span> setStartAndStopRows(scan, startPrefixRow, lastPrefixRow);<a name="line.566"></a> |
| <span class="sourceLineNo">567</span> }<a name="line.567"></a> |
| <span class="sourceLineNo">568</span> }<a name="line.568"></a> |
| <span class="sourceLineNo">569</span><a name="line.569"></a> |
| <span class="sourceLineNo">570</span> private static void setStartAndStopRows(Scan scan, byte[] startPrefixRow, byte[] lastPrefixRow) {<a name="line.570"></a> |
| <span class="sourceLineNo">571</span> scan.withStartRow(startPrefixRow);<a name="line.571"></a> |
| <span class="sourceLineNo">572</span> byte[] stopRow = Bytes.add(Bytes.head(lastPrefixRow, lastPrefixRow.length - 1),<a name="line.572"></a> |
| <span class="sourceLineNo">573</span> new byte[] { (byte) (lastPrefixRow[lastPrefixRow.length - 1] + 1) });<a name="line.573"></a> |
| <span class="sourceLineNo">574</span> scan.withStopRow(stopRow);<a name="line.574"></a> |
| <span class="sourceLineNo">575</span> }<a name="line.575"></a> |
| <span class="sourceLineNo">576</span><a name="line.576"></a> |
| <span class="sourceLineNo">577</span> public boolean doCommandLine(final String[] args) {<a name="line.577"></a> |
| <span class="sourceLineNo">578</span> if (args.length < 2) {<a name="line.578"></a> |
| <span class="sourceLineNo">579</span> printUsage(null);<a name="line.579"></a> |
| <span class="sourceLineNo">580</span> return false;<a name="line.580"></a> |
| <span class="sourceLineNo">581</span> }<a name="line.581"></a> |
| <span class="sourceLineNo">582</span> try {<a name="line.582"></a> |
| <span class="sourceLineNo">583</span> for (int i = 0; i < args.length; i++) {<a name="line.583"></a> |
| <span class="sourceLineNo">584</span> String cmd = args[i];<a name="line.584"></a> |
| <span class="sourceLineNo">585</span> if (cmd.equals("-h") || cmd.startsWith("--h")) {<a name="line.585"></a> |
| <span class="sourceLineNo">586</span> printUsage(null);<a name="line.586"></a> |
| <span class="sourceLineNo">587</span> return false;<a name="line.587"></a> |
| <span class="sourceLineNo">588</span> }<a name="line.588"></a> |
| <span class="sourceLineNo">589</span><a name="line.589"></a> |
| <span class="sourceLineNo">590</span> final String startTimeArgKey = "--starttime=";<a name="line.590"></a> |
| <span class="sourceLineNo">591</span> if (cmd.startsWith(startTimeArgKey)) {<a name="line.591"></a> |
| <span class="sourceLineNo">592</span> startTime = Long.parseLong(cmd.substring(startTimeArgKey.length()));<a name="line.592"></a> |
| <span class="sourceLineNo">593</span> continue;<a name="line.593"></a> |
| <span class="sourceLineNo">594</span> }<a name="line.594"></a> |
| <span class="sourceLineNo">595</span><a name="line.595"></a> |
| <span class="sourceLineNo">596</span> final String endTimeArgKey = "--endtime=";<a name="line.596"></a> |
| <span class="sourceLineNo">597</span> if (cmd.startsWith(endTimeArgKey)) {<a name="line.597"></a> |
| <span class="sourceLineNo">598</span> endTime = Long.parseLong(cmd.substring(endTimeArgKey.length()));<a name="line.598"></a> |
| <span class="sourceLineNo">599</span> continue;<a name="line.599"></a> |
| <span class="sourceLineNo">600</span> }<a name="line.600"></a> |
| <span class="sourceLineNo">601</span><a name="line.601"></a> |
| <span class="sourceLineNo">602</span> final String includeDeletedCellsArgKey = "--raw";<a name="line.602"></a> |
| <span class="sourceLineNo">603</span> if (cmd.equals(includeDeletedCellsArgKey)) {<a name="line.603"></a> |
| <span class="sourceLineNo">604</span> includeDeletedCells = true;<a name="line.604"></a> |
| <span class="sourceLineNo">605</span> continue;<a name="line.605"></a> |
| <span class="sourceLineNo">606</span> }<a name="line.606"></a> |
| <span class="sourceLineNo">607</span><a name="line.607"></a> |
| <span class="sourceLineNo">608</span> final String versionsArgKey = "--versions=";<a name="line.608"></a> |
| <span class="sourceLineNo">609</span> if (cmd.startsWith(versionsArgKey)) {<a name="line.609"></a> |
| <span class="sourceLineNo">610</span> versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));<a name="line.610"></a> |
| <span class="sourceLineNo">611</span> continue;<a name="line.611"></a> |
| <span class="sourceLineNo">612</span> }<a name="line.612"></a> |
| <span class="sourceLineNo">613</span><a name="line.613"></a> |
| <span class="sourceLineNo">614</span> final String batchArgKey = "--batch=";<a name="line.614"></a> |
| <span class="sourceLineNo">615</span> if (cmd.startsWith(batchArgKey)) {<a name="line.615"></a> |
| <span class="sourceLineNo">616</span> batch = Integer.parseInt(cmd.substring(batchArgKey.length()));<a name="line.616"></a> |
| <span class="sourceLineNo">617</span> continue;<a name="line.617"></a> |
| <span class="sourceLineNo">618</span> }<a name="line.618"></a> |
| <span class="sourceLineNo">619</span><a name="line.619"></a> |
| <span class="sourceLineNo">620</span> final String familiesArgKey = "--families=";<a name="line.620"></a> |
| <span class="sourceLineNo">621</span> if (cmd.startsWith(familiesArgKey)) {<a name="line.621"></a> |
| <span class="sourceLineNo">622</span> families = cmd.substring(familiesArgKey.length());<a name="line.622"></a> |
| <span class="sourceLineNo">623</span> continue;<a name="line.623"></a> |
| <span class="sourceLineNo">624</span> }<a name="line.624"></a> |
| <span class="sourceLineNo">625</span><a name="line.625"></a> |
| <span class="sourceLineNo">626</span> final String rowPrefixesKey = "--row-prefixes=";<a name="line.626"></a> |
| <span class="sourceLineNo">627</span> if (cmd.startsWith(rowPrefixesKey)) {<a name="line.627"></a> |
| <span class="sourceLineNo">628</span> rowPrefixes = cmd.substring(rowPrefixesKey.length());<a name="line.628"></a> |
| <span class="sourceLineNo">629</span> continue;<a name="line.629"></a> |
| <span class="sourceLineNo">630</span> }<a name="line.630"></a> |
| <span class="sourceLineNo">631</span><a name="line.631"></a> |
| <span class="sourceLineNo">632</span> final String delimiterArgKey = "--delimiter=";<a name="line.632"></a> |
| <span class="sourceLineNo">633</span> if (cmd.startsWith(delimiterArgKey)) {<a name="line.633"></a> |
| <span class="sourceLineNo">634</span> delimiter = cmd.substring(delimiterArgKey.length());<a name="line.634"></a> |
| <span class="sourceLineNo">635</span> continue;<a name="line.635"></a> |
| <span class="sourceLineNo">636</span> }<a name="line.636"></a> |
| <span class="sourceLineNo">637</span><a name="line.637"></a> |
| <span class="sourceLineNo">638</span> final String deprecatedSleepToReCompareKey = "--recomparesleep=";<a name="line.638"></a> |
| <span class="sourceLineNo">639</span> final String sleepToReCompareKey = "--recompareSleep=";<a name="line.639"></a> |
| <span class="sourceLineNo">640</span> if (cmd.startsWith(deprecatedSleepToReCompareKey)) {<a name="line.640"></a> |
| <span class="sourceLineNo">641</span> LOG.warn("--recomparesleep is deprecated and will be removed in 4.0.0."<a name="line.641"></a> |
| <span class="sourceLineNo">642</span> + " Use --recompareSleep instead.");<a name="line.642"></a> |
| <span class="sourceLineNo">643</span> sleepMsBeforeReCompare =<a name="line.643"></a> |
| <span class="sourceLineNo">644</span> Integer.parseInt(cmd.substring(deprecatedSleepToReCompareKey.length()));<a name="line.644"></a> |
| <span class="sourceLineNo">645</span> continue;<a name="line.645"></a> |
| <span class="sourceLineNo">646</span> }<a name="line.646"></a> |
| <span class="sourceLineNo">647</span> if (cmd.startsWith(sleepToReCompareKey)) {<a name="line.647"></a> |
| <span class="sourceLineNo">648</span> sleepMsBeforeReCompare = Integer.parseInt(cmd.substring(sleepToReCompareKey.length()));<a name="line.648"></a> |
| <span class="sourceLineNo">649</span> continue;<a name="line.649"></a> |
| <span class="sourceLineNo">650</span> }<a name="line.650"></a> |
| <span class="sourceLineNo">651</span><a name="line.651"></a> |
| <span class="sourceLineNo">652</span> final String verboseKey = "--verbose";<a name="line.652"></a> |
| <span class="sourceLineNo">653</span> if (cmd.startsWith(verboseKey)) {<a name="line.653"></a> |
| <span class="sourceLineNo">654</span> verbose = true;<a name="line.654"></a> |
| <span class="sourceLineNo">655</span> continue;<a name="line.655"></a> |
| <span class="sourceLineNo">656</span> }<a name="line.656"></a> |
| <span class="sourceLineNo">657</span><a name="line.657"></a> |
| <span class="sourceLineNo">658</span> final String sourceSnapshotNameArgKey = "--sourceSnapshotName=";<a name="line.658"></a> |
| <span class="sourceLineNo">659</span> if (cmd.startsWith(sourceSnapshotNameArgKey)) {<a name="line.659"></a> |
| <span class="sourceLineNo">660</span> sourceSnapshotName = cmd.substring(sourceSnapshotNameArgKey.length());<a name="line.660"></a> |
| <span class="sourceLineNo">661</span> continue;<a name="line.661"></a> |
| <span class="sourceLineNo">662</span> }<a name="line.662"></a> |
| <span class="sourceLineNo">663</span><a name="line.663"></a> |
| <span class="sourceLineNo">664</span> final String sourceSnapshotTmpDirArgKey = "--sourceSnapshotTmpDir=";<a name="line.664"></a> |
| <span class="sourceLineNo">665</span> if (cmd.startsWith(sourceSnapshotTmpDirArgKey)) {<a name="line.665"></a> |
| <span class="sourceLineNo">666</span> sourceSnapshotTmpDir = cmd.substring(sourceSnapshotTmpDirArgKey.length());<a name="line.666"></a> |
| <span class="sourceLineNo">667</span> continue;<a name="line.667"></a> |
| <span class="sourceLineNo">668</span> }<a name="line.668"></a> |
| <span class="sourceLineNo">669</span><a name="line.669"></a> |
| <span class="sourceLineNo">670</span> final String peerSnapshotNameArgKey = "--peerSnapshotName=";<a name="line.670"></a> |
| <span class="sourceLineNo">671</span> if (cmd.startsWith(peerSnapshotNameArgKey)) {<a name="line.671"></a> |
| <span class="sourceLineNo">672</span> peerSnapshotName = cmd.substring(peerSnapshotNameArgKey.length());<a name="line.672"></a> |
| <span class="sourceLineNo">673</span> continue;<a name="line.673"></a> |
| <span class="sourceLineNo">674</span> }<a name="line.674"></a> |
| <span class="sourceLineNo">675</span><a name="line.675"></a> |
| <span class="sourceLineNo">676</span> final String peerSnapshotTmpDirArgKey = "--peerSnapshotTmpDir=";<a name="line.676"></a> |
| <span class="sourceLineNo">677</span> if (cmd.startsWith(peerSnapshotTmpDirArgKey)) {<a name="line.677"></a> |
| <span class="sourceLineNo">678</span> peerSnapshotTmpDir = cmd.substring(peerSnapshotTmpDirArgKey.length());<a name="line.678"></a> |
| <span class="sourceLineNo">679</span> continue;<a name="line.679"></a> |
| <span class="sourceLineNo">680</span> }<a name="line.680"></a> |
| <span class="sourceLineNo">681</span><a name="line.681"></a> |
| <span class="sourceLineNo">682</span> final String peerFSAddressArgKey = "--peerFSAddress=";<a name="line.682"></a> |
| <span class="sourceLineNo">683</span> if (cmd.startsWith(peerFSAddressArgKey)) {<a name="line.683"></a> |
| <span class="sourceLineNo">684</span> peerFSAddress = cmd.substring(peerFSAddressArgKey.length());<a name="line.684"></a> |
| <span class="sourceLineNo">685</span> continue;<a name="line.685"></a> |
| <span class="sourceLineNo">686</span> }<a name="line.686"></a> |
| <span class="sourceLineNo">687</span><a name="line.687"></a> |
| <span class="sourceLineNo">688</span> final String peerHBaseRootAddressArgKey = "--peerHBaseRootAddress=";<a name="line.688"></a> |
| <span class="sourceLineNo">689</span> if (cmd.startsWith(peerHBaseRootAddressArgKey)) {<a name="line.689"></a> |
| <span class="sourceLineNo">690</span> peerHBaseRootAddress = cmd.substring(peerHBaseRootAddressArgKey.length());<a name="line.690"></a> |
| <span class="sourceLineNo">691</span> continue;<a name="line.691"></a> |
| <span class="sourceLineNo">692</span> }<a name="line.692"></a> |
| <span class="sourceLineNo">693</span><a name="line.693"></a> |
| <span class="sourceLineNo">694</span> final String peerTableNameArgKey = "--peerTableName=";<a name="line.694"></a> |
| <span class="sourceLineNo">695</span> if (cmd.startsWith(peerTableNameArgKey)) {<a name="line.695"></a> |
| <span class="sourceLineNo">696</span> peerTableName = cmd.substring(peerTableNameArgKey.length());<a name="line.696"></a> |
| <span class="sourceLineNo">697</span> continue;<a name="line.697"></a> |
| <span class="sourceLineNo">698</span> }<a name="line.698"></a> |
| <span class="sourceLineNo">699</span><a name="line.699"></a> |
| <span class="sourceLineNo">700</span> final String reCompareThreadArgs = "--recompareThreads=";<a name="line.700"></a> |
| <span class="sourceLineNo">701</span> if (cmd.startsWith(reCompareThreadArgs)) {<a name="line.701"></a> |
| <span class="sourceLineNo">702</span> reCompareThreads = Integer.parseInt(cmd.substring(reCompareThreadArgs.length()));<a name="line.702"></a> |
| <span class="sourceLineNo">703</span> continue;<a name="line.703"></a> |
| <span class="sourceLineNo">704</span> }<a name="line.704"></a> |
| <span class="sourceLineNo">705</span><a name="line.705"></a> |
| <span class="sourceLineNo">706</span> final String reCompareTriesKey = "--recompareTries=";<a name="line.706"></a> |
| <span class="sourceLineNo">707</span> if (cmd.startsWith(reCompareTriesKey)) {<a name="line.707"></a> |
| <span class="sourceLineNo">708</span> reCompareTries = Integer.parseInt(cmd.substring(reCompareTriesKey.length()));<a name="line.708"></a> |
| <span class="sourceLineNo">709</span> continue;<a name="line.709"></a> |
| <span class="sourceLineNo">710</span> }<a name="line.710"></a> |
| <span class="sourceLineNo">711</span><a name="line.711"></a> |
| <span class="sourceLineNo">712</span> final String reCompareBackoffExponentKey = "--recompareBackoffExponent=";<a name="line.712"></a> |
| <span class="sourceLineNo">713</span> if (cmd.startsWith(reCompareBackoffExponentKey)) {<a name="line.713"></a> |
| <span class="sourceLineNo">714</span> reCompareBackoffExponent =<a name="line.714"></a> |
| <span class="sourceLineNo">715</span> Integer.parseInt(cmd.substring(reCompareBackoffExponentKey.length()));<a name="line.715"></a> |
| <span class="sourceLineNo">716</span> continue;<a name="line.716"></a> |
| <span class="sourceLineNo">717</span> }<a name="line.717"></a> |
| <span class="sourceLineNo">718</span><a name="line.718"></a> |
| <span class="sourceLineNo">719</span> if (cmd.startsWith("--")) {<a name="line.719"></a> |
| <span class="sourceLineNo">720</span> printUsage("Invalid argument '" + cmd + "'");<a name="line.720"></a> |
| <span class="sourceLineNo">721</span> return false;<a name="line.721"></a> |
| <span class="sourceLineNo">722</span> }<a name="line.722"></a> |
| <span class="sourceLineNo">723</span><a name="line.723"></a> |
| <span class="sourceLineNo">724</span> if (i == args.length - 2) {<a name="line.724"></a> |
| <span class="sourceLineNo">725</span> if (isPeerQuorumAddress(cmd)) {<a name="line.725"></a> |
| <span class="sourceLineNo">726</span> peerQuorumAddress = cmd;<a name="line.726"></a> |
| <span class="sourceLineNo">727</span> } else {<a name="line.727"></a> |
| <span class="sourceLineNo">728</span> peerId = cmd;<a name="line.728"></a> |
| <span class="sourceLineNo">729</span> }<a name="line.729"></a> |
| <span class="sourceLineNo">730</span> }<a name="line.730"></a> |
| <span class="sourceLineNo">731</span><a name="line.731"></a> |
| <span class="sourceLineNo">732</span> if (i == args.length - 1) {<a name="line.732"></a> |
| <span class="sourceLineNo">733</span> tableName = cmd;<a name="line.733"></a> |
| <span class="sourceLineNo">734</span> }<a name="line.734"></a> |
| <span class="sourceLineNo">735</span> }<a name="line.735"></a> |
| <span class="sourceLineNo">736</span><a name="line.736"></a> |
| <span class="sourceLineNo">737</span> if (<a name="line.737"></a> |
| <span class="sourceLineNo">738</span> (sourceSnapshotName != null && sourceSnapshotTmpDir == null)<a name="line.738"></a> |
| <span class="sourceLineNo">739</span> || (sourceSnapshotName == null && sourceSnapshotTmpDir != null)<a name="line.739"></a> |
| <span class="sourceLineNo">740</span> ) {<a name="line.740"></a> |
| <span class="sourceLineNo">741</span> printUsage("Source snapshot name and snapshot temp location should be provided"<a name="line.741"></a> |
| <span class="sourceLineNo">742</span> + " to use snapshots in source cluster");<a name="line.742"></a> |
| <span class="sourceLineNo">743</span> return false;<a name="line.743"></a> |
| <span class="sourceLineNo">744</span> }<a name="line.744"></a> |
| <span class="sourceLineNo">745</span><a name="line.745"></a> |
| <span class="sourceLineNo">746</span> if (<a name="line.746"></a> |
| <span class="sourceLineNo">747</span> peerSnapshotName != null || peerSnapshotTmpDir != null || peerFSAddress != null<a name="line.747"></a> |
| <span class="sourceLineNo">748</span> || peerHBaseRootAddress != null<a name="line.748"></a> |
| <span class="sourceLineNo">749</span> ) {<a name="line.749"></a> |
| <span class="sourceLineNo">750</span> if (<a name="line.750"></a> |
| <span class="sourceLineNo">751</span> peerSnapshotName == null || peerSnapshotTmpDir == null || peerFSAddress == null<a name="line.751"></a> |
| <span class="sourceLineNo">752</span> || peerHBaseRootAddress == null<a name="line.752"></a> |
| <span class="sourceLineNo">753</span> ) {<a name="line.753"></a> |
| <span class="sourceLineNo">754</span> printUsage(<a name="line.754"></a> |
| <span class="sourceLineNo">755</span> "Peer snapshot name, peer snapshot temp location, Peer HBase root address and "<a name="line.755"></a> |
| <span class="sourceLineNo">756</span> + "peer FSAddress should be provided to use snapshots in peer cluster");<a name="line.756"></a> |
| <span class="sourceLineNo">757</span> return false;<a name="line.757"></a> |
| <span class="sourceLineNo">758</span> }<a name="line.758"></a> |
| <span class="sourceLineNo">759</span> }<a name="line.759"></a> |
| <span class="sourceLineNo">760</span><a name="line.760"></a> |
| <span class="sourceLineNo">761</span> // This is to avoid making recompare calls to source/peer tables when snapshots are used<a name="line.761"></a> |
| <span class="sourceLineNo">762</span> if ((sourceSnapshotName != null || peerSnapshotName != null) && sleepMsBeforeReCompare > 0) {<a name="line.762"></a> |
| <span class="sourceLineNo">763</span> printUsage(<a name="line.763"></a> |
| <span class="sourceLineNo">764</span> "Using sleepMsBeforeReCompare along with snapshots is not allowed as snapshots are"<a name="line.764"></a> |
| <span class="sourceLineNo">765</span> + " immutable");<a name="line.765"></a> |
| <span class="sourceLineNo">766</span> return false;<a name="line.766"></a> |
| <span class="sourceLineNo">767</span> }<a name="line.767"></a> |
| <span class="sourceLineNo">768</span><a name="line.768"></a> |
| <span class="sourceLineNo">769</span> } catch (Exception e) {<a name="line.769"></a> |
| <span class="sourceLineNo">770</span> LOG.error("Failed to parse commandLine arguments", e);<a name="line.770"></a> |
| <span class="sourceLineNo">771</span> printUsage("Can't start because " + e.getMessage());<a name="line.771"></a> |
| <span class="sourceLineNo">772</span> return false;<a name="line.772"></a> |
| <span class="sourceLineNo">773</span> }<a name="line.773"></a> |
| <span class="sourceLineNo">774</span> return true;<a name="line.774"></a> |
| <span class="sourceLineNo">775</span> }<a name="line.775"></a> |
| <span class="sourceLineNo">776</span><a name="line.776"></a> |
| <span class="sourceLineNo">777</span> private boolean isPeerQuorumAddress(String cmd) {<a name="line.777"></a> |
| <span class="sourceLineNo">778</span> try {<a name="line.778"></a> |
| <span class="sourceLineNo">779</span> ZKConfig.validateClusterKey(cmd);<a name="line.779"></a> |
| <span class="sourceLineNo">780</span> } catch (IOException e) {<a name="line.780"></a> |
| <span class="sourceLineNo">781</span> // not a quorum address<a name="line.781"></a> |
| <span class="sourceLineNo">782</span> return false;<a name="line.782"></a> |
| <span class="sourceLineNo">783</span> }<a name="line.783"></a> |
| <span class="sourceLineNo">784</span> return true;<a name="line.784"></a> |
| <span class="sourceLineNo">785</span> }<a name="line.785"></a> |
| <span class="sourceLineNo">786</span><a name="line.786"></a> |
| <span class="sourceLineNo">787</span> /*<a name="line.787"></a> |
| <span class="sourceLineNo">788</span> * @param errorMsg Error message. Can be null.<a name="line.788"></a> |
| <span class="sourceLineNo">789</span> */<a name="line.789"></a> |
| <span class="sourceLineNo">790</span> private static void printUsage(final String errorMsg) {<a name="line.790"></a> |
| <span class="sourceLineNo">791</span> if (errorMsg != null && errorMsg.length() > 0) {<a name="line.791"></a> |
| <span class="sourceLineNo">792</span> System.err.println("ERROR: " + errorMsg);<a name="line.792"></a> |
| <span class="sourceLineNo">793</span> }<a name="line.793"></a> |
| <span class="sourceLineNo">794</span> System.err.println("Usage: verifyrep [--starttime=X]"<a name="line.794"></a> |
| <span class="sourceLineNo">795</span> + " [--endtime=Y] [--families=A] [--row-prefixes=B] [--delimiter=] [--recompareSleep=] "<a name="line.795"></a> |
| <span class="sourceLineNo">796</span> + "[--recompareThreads=] [--recompareTries=] [--recompareBackoffExponent=]"<a name="line.796"></a> |
| <span class="sourceLineNo">797</span> + "[--batch=] [--verbose] [--peerTableName=] [--sourceSnapshotName=P] "<a name="line.797"></a> |
| <span class="sourceLineNo">798</span> + "[--sourceSnapshotTmpDir=Q] [--peerSnapshotName=R] [--peerSnapshotTmpDir=S] "<a name="line.798"></a> |
| <span class="sourceLineNo">799</span> + "[--peerFSAddress=T] [--peerHBaseRootAddress=U] <peerid|peerQuorumAddress> <tablename>");<a name="line.799"></a> |
| <span class="sourceLineNo">800</span> System.err.println();<a name="line.800"></a> |
| <span class="sourceLineNo">801</span> System.err.println("Options:");<a name="line.801"></a> |
| <span class="sourceLineNo">802</span> System.err.println(" starttime beginning of the time range");<a name="line.802"></a> |
| <span class="sourceLineNo">803</span> System.err.println(" without endtime means from starttime to forever");<a name="line.803"></a> |
| <span class="sourceLineNo">804</span> System.err.println(" endtime end of the time range");<a name="line.804"></a> |
| <span class="sourceLineNo">805</span> System.err.println(" versions number of cell versions to verify");<a name="line.805"></a> |
| <span class="sourceLineNo">806</span> System.err.println(" batch batch count for scan, note that"<a name="line.806"></a> |
| <span class="sourceLineNo">807</span> + " result row counts will no longer be actual number of rows when you use this option");<a name="line.807"></a> |
| <span class="sourceLineNo">808</span> System.err.println(" raw includes raw scan if given in options");<a name="line.808"></a> |
| <span class="sourceLineNo">809</span> System.err.println(" families comma-separated list of families to copy");<a name="line.809"></a> |
| <span class="sourceLineNo">810</span> System.err.println(" row-prefixes comma-separated list of row key prefixes to filter on ");<a name="line.810"></a> |
| <span class="sourceLineNo">811</span> System.err.println(" delimiter the delimiter used in display around rowkey");<a name="line.811"></a> |
| <span class="sourceLineNo">812</span> System.err.println(" recompareSleep milliseconds to sleep before recompare row, "<a name="line.812"></a> |
| <span class="sourceLineNo">813</span> + "default value is 0 which disables the recompare.");<a name="line.813"></a> |
| <span class="sourceLineNo">814</span> System.err.println(" recompareThreads number of threads to run recompares in");<a name="line.814"></a> |
| <span class="sourceLineNo">815</span> System.err.println(" recompareTries number of recompare attempts before incrementing "<a name="line.815"></a> |
| <span class="sourceLineNo">816</span> + "the BADROWS counter. Defaults to 1 recompare");<a name="line.816"></a> |
| <span class="sourceLineNo">817</span> System.out.println(" recompareBackoffExponent exponential multiplier to increase "<a name="line.817"></a> |
| <span class="sourceLineNo">818</span> + "recompareSleep after each recompare attempt, "<a name="line.818"></a> |
| <span class="sourceLineNo">819</span> + "default value is 0 which results in a constant sleep time");<a name="line.819"></a> |
| <span class="sourceLineNo">820</span> System.err.println(" verbose logs row keys of good rows");<a name="line.820"></a> |
| <span class="sourceLineNo">821</span> System.err.println(" peerTableName Peer Table Name");<a name="line.821"></a> |
| <span class="sourceLineNo">822</span> System.err.println(" sourceSnapshotName Source Snapshot Name");<a name="line.822"></a> |
| <span class="sourceLineNo">823</span> System.err.println(" sourceSnapshotTmpDir Tmp location to restore source table snapshot");<a name="line.823"></a> |
| <span class="sourceLineNo">824</span> System.err.println(" peerSnapshotName Peer Snapshot Name");<a name="line.824"></a> |
| <span class="sourceLineNo">825</span> System.err.println(" peerSnapshotTmpDir Tmp location to restore peer table snapshot");<a name="line.825"></a> |
| <span class="sourceLineNo">826</span> System.err.println(" peerFSAddress Peer cluster Hadoop FS address");<a name="line.826"></a> |
| <span class="sourceLineNo">827</span> System.err.println(" peerHBaseRootAddress Peer cluster HBase root location");<a name="line.827"></a> |
| <span class="sourceLineNo">828</span> System.err.println();<a name="line.828"></a> |
| <span class="sourceLineNo">829</span> System.err.println("Args:");<a name="line.829"></a> |
| <span class="sourceLineNo">830</span> System.err.println(" peerid Id of the peer used for verification,"<a name="line.830"></a> |
| <span class="sourceLineNo">831</span> + " must match the one given for replication");<a name="line.831"></a> |
| <span class="sourceLineNo">832</span> System.err.println(" peerQuorumAddress quorumAdress of the peer used for verification. The "<a name="line.832"></a> |
| <span class="sourceLineNo">833</span> + "format is zk_quorum:zk_port:zk_hbase_path");<a name="line.833"></a> |
| <span class="sourceLineNo">834</span> System.err.println(" tablename Name of the table to verify");<a name="line.834"></a> |
| <span class="sourceLineNo">835</span> System.err.println();<a name="line.835"></a> |
| <span class="sourceLineNo">836</span> System.err.println("Examples:");<a name="line.836"></a> |
| <span class="sourceLineNo">837</span> System.err<a name="line.837"></a> |
| <span class="sourceLineNo">838</span> .println(" To verify the data replicated from TestTable for a 1 hour window with peer #5 ");<a name="line.838"></a> |
| <span class="sourceLineNo">839</span> System.err<a name="line.839"></a> |
| <span class="sourceLineNo">840</span> .println(" $ hbase " + "org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication"<a name="line.840"></a> |
| <span class="sourceLineNo">841</span> + " --starttime=1265875194289 --endtime=1265878794289 5 TestTable ");<a name="line.841"></a> |
| <span class="sourceLineNo">842</span> System.err.println();<a name="line.842"></a> |
| <span class="sourceLineNo">843</span> System.err.println(<a name="line.843"></a> |
| <span class="sourceLineNo">844</span> " To verify the data in TestTable between the cluster runs VerifyReplication and cluster-b");<a name="line.844"></a> |
| <span class="sourceLineNo">845</span> System.err.println(" Assume quorum address for cluster-b is"<a name="line.845"></a> |
| <span class="sourceLineNo">846</span> + " cluster-b-1.example.com,cluster-b-2.example.com,cluster-b-3.example.com:2181:/cluster-b");<a name="line.846"></a> |
| <span class="sourceLineNo">847</span> System.err<a name="line.847"></a> |
| <span class="sourceLineNo">848</span> .println(" $ hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication \\\n"<a name="line.848"></a> |
| <span class="sourceLineNo">849</span> + " cluster-b-1.example.com,cluster-b-2.example.com,cluster-b-3.example.com:"<a name="line.849"></a> |
| <span class="sourceLineNo">850</span> + "2181:/cluster-b \\\n" + " TestTable");<a name="line.850"></a> |
| <span class="sourceLineNo">851</span> System.err.println();<a name="line.851"></a> |
| <span class="sourceLineNo">852</span> System.err<a name="line.852"></a> |
| <span class="sourceLineNo">853</span> .println(" To verify the data in TestTable between the secured cluster runs VerifyReplication"<a name="line.853"></a> |
| <span class="sourceLineNo">854</span> + " and insecure cluster-b");<a name="line.854"></a> |
| <span class="sourceLineNo">855</span> System.err<a name="line.855"></a> |
| <span class="sourceLineNo">856</span> .println(" $ hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication \\\n"<a name="line.856"></a> |
| <span class="sourceLineNo">857</span> + " -D verifyrep.peer.hbase.security.authentication=simple \\\n"<a name="line.857"></a> |
| <span class="sourceLineNo">858</span> + " cluster-b-1.example.com,cluster-b-2.example.com,cluster-b-3.example.com:"<a name="line.858"></a> |
| <span class="sourceLineNo">859</span> + "2181:/cluster-b \\\n" + " TestTable");<a name="line.859"></a> |
| <span class="sourceLineNo">860</span> System.err.println();<a name="line.860"></a> |
| <span class="sourceLineNo">861</span> System.err.println(" To verify the data in TestTable between"<a name="line.861"></a> |
| <span class="sourceLineNo">862</span> + " the secured cluster runs VerifyReplication and secured cluster-b");<a name="line.862"></a> |
| <span class="sourceLineNo">863</span> System.err.println(" Assume cluster-b uses different kerberos principal, cluster-b/_HOST@E"<a name="line.863"></a> |
| <span class="sourceLineNo">864</span> + ", for master and regionserver kerberos principal from another cluster");<a name="line.864"></a> |
| <span class="sourceLineNo">865</span> System.err<a name="line.865"></a> |
| <span class="sourceLineNo">866</span> .println(" $ hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication \\\n"<a name="line.866"></a> |
| <span class="sourceLineNo">867</span> + " -D verifyrep.peer.hbase.regionserver.kerberos.principal="<a name="line.867"></a> |
| <span class="sourceLineNo">868</span> + "cluster-b/_HOST@EXAMPLE.COM \\\n"<a name="line.868"></a> |
| <span class="sourceLineNo">869</span> + " -D verifyrep.peer.hbase.master.kerberos.principal=cluster-b/_HOST@EXAMPLE.COM \\\n"<a name="line.869"></a> |
| <span class="sourceLineNo">870</span> + " cluster-b-1.example.com,cluster-b-2.example.com,cluster-b-3.example.com:"<a name="line.870"></a> |
| <span class="sourceLineNo">871</span> + "2181:/cluster-b \\\n" + " TestTable");<a name="line.871"></a> |
| <span class="sourceLineNo">872</span> System.err.println();<a name="line.872"></a> |
| <span class="sourceLineNo">873</span> System.err.println(<a name="line.873"></a> |
| <span class="sourceLineNo">874</span> " To verify the data in TestTable between the insecure cluster runs VerifyReplication"<a name="line.874"></a> |
| <span class="sourceLineNo">875</span> + " and secured cluster-b");<a name="line.875"></a> |
| <span class="sourceLineNo">876</span> System.err<a name="line.876"></a> |
| <span class="sourceLineNo">877</span> .println(" $ hbase org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication \\\n"<a name="line.877"></a> |
| <span class="sourceLineNo">878</span> + " -D verifyrep.peer.hbase.security.authentication=kerberos \\\n"<a name="line.878"></a> |
| <span class="sourceLineNo">879</span> + " -D verifyrep.peer.hbase.regionserver.kerberos.principal="<a name="line.879"></a> |
| <span class="sourceLineNo">880</span> + "cluster-b/_HOST@EXAMPLE.COM \\\n"<a name="line.880"></a> |
| <span class="sourceLineNo">881</span> + " -D verifyrep.peer.hbase.master.kerberos.principal=cluster-b/_HOST@EXAMPLE.COM \\\n"<a name="line.881"></a> |
| <span class="sourceLineNo">882</span> + " cluster-b-1.example.com,cluster-b-2.example.com,cluster-b-3.example.com:"<a name="line.882"></a> |
| <span class="sourceLineNo">883</span> + "2181:/cluster-b \\\n" + " TestTable");<a name="line.883"></a> |
| <span class="sourceLineNo">884</span> }<a name="line.884"></a> |
| <span class="sourceLineNo">885</span><a name="line.885"></a> |
| <span class="sourceLineNo">886</span> private static ThreadPoolExecutor buildReCompareExecutor(int maxThreads, Mapper.Context context) {<a name="line.886"></a> |
| <span class="sourceLineNo">887</span> if (maxThreads == 0) {<a name="line.887"></a> |
| <span class="sourceLineNo">888</span> return null;<a name="line.888"></a> |
| <span class="sourceLineNo">889</span> }<a name="line.889"></a> |
| <span class="sourceLineNo">890</span><a name="line.890"></a> |
| <span class="sourceLineNo">891</span> return new ThreadPoolExecutor(0, maxThreads, 1L, TimeUnit.SECONDS, new SynchronousQueue<>(),<a name="line.891"></a> |
| <span class="sourceLineNo">892</span> buildRejectedReComparePolicy(context));<a name="line.892"></a> |
| <span class="sourceLineNo">893</span> }<a name="line.893"></a> |
| <span class="sourceLineNo">894</span><a name="line.894"></a> |
| <span class="sourceLineNo">895</span> private static CallerRunsPolicy buildRejectedReComparePolicy(Mapper.Context context) {<a name="line.895"></a> |
| <span class="sourceLineNo">896</span> return new CallerRunsPolicy() {<a name="line.896"></a> |
| <span class="sourceLineNo">897</span> @Override<a name="line.897"></a> |
| <span class="sourceLineNo">898</span> public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) {<a name="line.898"></a> |
| <span class="sourceLineNo">899</span> LOG.debug("Re-comparison execution rejected. Running in main thread.");<a name="line.899"></a> |
| <span class="sourceLineNo">900</span> context.getCounter(Counters.MAIN_THREAD_RECOMPARES).increment(1);<a name="line.900"></a> |
| <span class="sourceLineNo">901</span> // will run in the current thread<a name="line.901"></a> |
| <span class="sourceLineNo">902</span> super.rejectedExecution(runnable, e);<a name="line.902"></a> |
| <span class="sourceLineNo">903</span> }<a name="line.903"></a> |
| <span class="sourceLineNo">904</span> };<a name="line.904"></a> |
| <span class="sourceLineNo">905</span> }<a name="line.905"></a> |
| <span class="sourceLineNo">906</span><a name="line.906"></a> |
| <span class="sourceLineNo">907</span> @Override<a name="line.907"></a> |
| <span class="sourceLineNo">908</span> public int run(String[] args) throws Exception {<a name="line.908"></a> |
| <span class="sourceLineNo">909</span> Configuration conf = this.getConf();<a name="line.909"></a> |
| <span class="sourceLineNo">910</span> Job job = createSubmittableJob(conf, args);<a name="line.910"></a> |
| <span class="sourceLineNo">911</span> if (job != null) {<a name="line.911"></a> |
| <span class="sourceLineNo">912</span> return job.waitForCompletion(true) ? 0 : 1;<a name="line.912"></a> |
| <span class="sourceLineNo">913</span> }<a name="line.913"></a> |
| <span class="sourceLineNo">914</span> return 1;<a name="line.914"></a> |
| <span class="sourceLineNo">915</span> }<a name="line.915"></a> |
| <span class="sourceLineNo">916</span><a name="line.916"></a> |
| <span class="sourceLineNo">917</span> /**<a name="line.917"></a> |
| <span class="sourceLineNo">918</span> * Main entry point.<a name="line.918"></a> |
| <span class="sourceLineNo">919</span> * @param args The command line parameters.<a name="line.919"></a> |
| <span class="sourceLineNo">920</span> * @throws Exception When running the job fails.<a name="line.920"></a> |
| <span class="sourceLineNo">921</span> */<a name="line.921"></a> |
| <span class="sourceLineNo">922</span> public static void main(String[] args) throws Exception {<a name="line.922"></a> |
| <span class="sourceLineNo">923</span> int res = ToolRunner.run(HBaseConfiguration.create(), new VerifyReplication(), args);<a name="line.923"></a> |
| <span class="sourceLineNo">924</span> System.exit(res);<a name="line.924"></a> |
| <span class="sourceLineNo">925</span> }<a name="line.925"></a> |
| <span class="sourceLineNo">926</span>}<a name="line.926"></a> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </body> |
| </html> |