blob: efda7ae9a31feac79b9238e418758c8ae8abf14b [file] [log] [blame]
<!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.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.ArrayList;<a name="line.21"></a>
<span class="sourceLineNo">022</span>import java.util.Collection;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import java.util.Collections;<a name="line.23"></a>
<span class="sourceLineNo">024</span>import java.util.HashMap;<a name="line.24"></a>
<span class="sourceLineNo">025</span>import java.util.HashSet;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import java.util.List;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import java.util.Map;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import java.util.NavigableMap;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import java.util.Set;<a name="line.29"></a>
<span class="sourceLineNo">030</span>import java.util.function.Supplier;<a name="line.30"></a>
<span class="sourceLineNo">031</span>import java.util.stream.Collectors;<a name="line.31"></a>
<span class="sourceLineNo">032</span>import org.apache.hadoop.fs.Path;<a name="line.32"></a>
<span class="sourceLineNo">033</span>import org.apache.hadoop.hbase.Cell;<a name="line.33"></a>
<span class="sourceLineNo">034</span>import org.apache.hadoop.hbase.CellScanner;<a name="line.34"></a>
<span class="sourceLineNo">035</span>import org.apache.hadoop.hbase.CellUtil;<a name="line.35"></a>
<span class="sourceLineNo">036</span>import org.apache.hadoop.hbase.CompareOperator;<a name="line.36"></a>
<span class="sourceLineNo">037</span>import org.apache.hadoop.hbase.HConstants;<a name="line.37"></a>
<span class="sourceLineNo">038</span>import org.apache.hadoop.hbase.ServerName;<a name="line.38"></a>
<span class="sourceLineNo">039</span>import org.apache.hadoop.hbase.TableName;<a name="line.39"></a>
<span class="sourceLineNo">040</span>import org.apache.hadoop.hbase.client.AsyncTable;<a name="line.40"></a>
<span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.client.Connection;<a name="line.41"></a>
<span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.client.Delete;<a name="line.42"></a>
<span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.client.Get;<a name="line.43"></a>
<span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.client.Put;<a name="line.44"></a>
<span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.client.Result;<a name="line.45"></a>
<span class="sourceLineNo">046</span>import org.apache.hadoop.hbase.client.ResultScanner;<a name="line.46"></a>
<span class="sourceLineNo">047</span>import org.apache.hadoop.hbase.client.Scan;<a name="line.47"></a>
<span class="sourceLineNo">048</span>import org.apache.hadoop.hbase.client.Scan.ReadType;<a name="line.48"></a>
<span class="sourceLineNo">049</span>import org.apache.hadoop.hbase.client.Table;<a name="line.49"></a>
<span class="sourceLineNo">050</span>import org.apache.hadoop.hbase.filter.KeyOnlyFilter;<a name="line.50"></a>
<span class="sourceLineNo">051</span>import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration.ZkLastPushedSeqId;<a name="line.51"></a>
<span class="sourceLineNo">052</span>import org.apache.hadoop.hbase.util.Bytes;<a name="line.52"></a>
<span class="sourceLineNo">053</span>import org.apache.hadoop.hbase.util.FutureUtils;<a name="line.53"></a>
<span class="sourceLineNo">054</span>import org.apache.hadoop.hbase.util.Pair;<a name="line.54"></a>
<span class="sourceLineNo">055</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.55"></a>
<span class="sourceLineNo">056</span><a name="line.56"></a>
<span class="sourceLineNo">057</span>import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;<a name="line.57"></a>
<span class="sourceLineNo">058</span><a name="line.58"></a>
<span class="sourceLineNo">059</span>import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;<a name="line.59"></a>
<span class="sourceLineNo">060</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;<a name="line.60"></a>
<span class="sourceLineNo">061</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;<a name="line.61"></a>
<span class="sourceLineNo">062</span>import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos;<a name="line.62"></a>
<span class="sourceLineNo">063</span><a name="line.63"></a>
<span class="sourceLineNo">064</span>/**<a name="line.64"></a>
<span class="sourceLineNo">065</span> * HBase table based replication queue storage.<a name="line.65"></a>
<span class="sourceLineNo">066</span> */<a name="line.66"></a>
<span class="sourceLineNo">067</span>@InterfaceAudience.Private<a name="line.67"></a>
<span class="sourceLineNo">068</span>public class TableReplicationQueueStorage implements ReplicationQueueStorage {<a name="line.68"></a>
<span class="sourceLineNo">069</span><a name="line.69"></a>
<span class="sourceLineNo">070</span> public static final byte[] QUEUE_FAMILY = Bytes.toBytes("queue");<a name="line.70"></a>
<span class="sourceLineNo">071</span><a name="line.71"></a>
<span class="sourceLineNo">072</span> public static final byte[] LAST_SEQUENCE_ID_FAMILY = Bytes.toBytes("sid");<a name="line.72"></a>
<span class="sourceLineNo">073</span><a name="line.73"></a>
<span class="sourceLineNo">074</span> public static final byte[] HFILE_REF_FAMILY = Bytes.toBytes("hfileref");<a name="line.74"></a>
<span class="sourceLineNo">075</span><a name="line.75"></a>
<span class="sourceLineNo">076</span> private final Connection conn;<a name="line.76"></a>
<span class="sourceLineNo">077</span><a name="line.77"></a>
<span class="sourceLineNo">078</span> private final TableName tableName;<a name="line.78"></a>
<span class="sourceLineNo">079</span><a name="line.79"></a>
<span class="sourceLineNo">080</span> public TableReplicationQueueStorage(Connection conn, TableName tableName) {<a name="line.80"></a>
<span class="sourceLineNo">081</span> this.conn = conn;<a name="line.81"></a>
<span class="sourceLineNo">082</span> this.tableName = tableName;<a name="line.82"></a>
<span class="sourceLineNo">083</span> }<a name="line.83"></a>
<span class="sourceLineNo">084</span><a name="line.84"></a>
<span class="sourceLineNo">085</span> private void addLastSeqIdsPut(MultiRowMutationProtos.MutateRowsRequest.Builder builder,<a name="line.85"></a>
<span class="sourceLineNo">086</span> String peerId, Map&lt;String, Long&gt; lastSeqIds, AsyncTable&lt;?&gt; table) throws IOException {<a name="line.86"></a>
<span class="sourceLineNo">087</span> // get the previous sequence ids first<a name="line.87"></a>
<span class="sourceLineNo">088</span> byte[] row = Bytes.toBytes(peerId);<a name="line.88"></a>
<span class="sourceLineNo">089</span> Get get = new Get(row);<a name="line.89"></a>
<span class="sourceLineNo">090</span> lastSeqIds.keySet().forEach(encodedRegionName -&gt; get.addColumn(LAST_SEQUENCE_ID_FAMILY,<a name="line.90"></a>
<span class="sourceLineNo">091</span> Bytes.toBytes(encodedRegionName)));<a name="line.91"></a>
<span class="sourceLineNo">092</span> Result result = FutureUtils.get(table.get(get));<a name="line.92"></a>
<span class="sourceLineNo">093</span> Put put = new Put(row);<a name="line.93"></a>
<span class="sourceLineNo">094</span> for (Map.Entry&lt;String, Long&gt; entry : lastSeqIds.entrySet()) {<a name="line.94"></a>
<span class="sourceLineNo">095</span> String encodedRegionName = entry.getKey();<a name="line.95"></a>
<span class="sourceLineNo">096</span> long lastSeqId = entry.getValue();<a name="line.96"></a>
<span class="sourceLineNo">097</span> byte[] encodedRegionNameAsBytes = Bytes.toBytes(encodedRegionName);<a name="line.97"></a>
<span class="sourceLineNo">098</span> byte[] previousLastSeqIdAsBytes =<a name="line.98"></a>
<span class="sourceLineNo">099</span> result.getValue(LAST_SEQUENCE_ID_FAMILY, encodedRegionNameAsBytes);<a name="line.99"></a>
<span class="sourceLineNo">100</span> if (previousLastSeqIdAsBytes != null) {<a name="line.100"></a>
<span class="sourceLineNo">101</span> long previousLastSeqId = Bytes.toLong(previousLastSeqIdAsBytes);<a name="line.101"></a>
<span class="sourceLineNo">102</span> if (lastSeqId &gt; previousLastSeqId) {<a name="line.102"></a>
<span class="sourceLineNo">103</span> // update last seq id when it is greater, and use CAS to make sure we do not overwrite<a name="line.103"></a>
<span class="sourceLineNo">104</span> // other's value.<a name="line.104"></a>
<span class="sourceLineNo">105</span> put.addColumn(LAST_SEQUENCE_ID_FAMILY, encodedRegionNameAsBytes,<a name="line.105"></a>
<span class="sourceLineNo">106</span> Bytes.toBytes(lastSeqId));<a name="line.106"></a>
<span class="sourceLineNo">107</span> builder.addCondition(ProtobufUtil.toCondition(row, LAST_SEQUENCE_ID_FAMILY,<a name="line.107"></a>
<span class="sourceLineNo">108</span> encodedRegionNameAsBytes, CompareOperator.EQUAL, previousLastSeqIdAsBytes, null));<a name="line.108"></a>
<span class="sourceLineNo">109</span> }<a name="line.109"></a>
<span class="sourceLineNo">110</span> } else {<a name="line.110"></a>
<span class="sourceLineNo">111</span> // also update last seq id when there is no value yet, and use CAS to make sure we do not<a name="line.111"></a>
<span class="sourceLineNo">112</span> // overwrite<a name="line.112"></a>
<span class="sourceLineNo">113</span> // other's value.<a name="line.113"></a>
<span class="sourceLineNo">114</span> put.addColumn(LAST_SEQUENCE_ID_FAMILY, encodedRegionNameAsBytes, Bytes.toBytes(lastSeqId));<a name="line.114"></a>
<span class="sourceLineNo">115</span> builder.addCondition(ProtobufUtil.toCondition(row, LAST_SEQUENCE_ID_FAMILY,<a name="line.115"></a>
<span class="sourceLineNo">116</span> encodedRegionNameAsBytes, CompareOperator.EQUAL, null, null));<a name="line.116"></a>
<span class="sourceLineNo">117</span> }<a name="line.117"></a>
<span class="sourceLineNo">118</span> }<a name="line.118"></a>
<span class="sourceLineNo">119</span> if (!put.isEmpty()) {<a name="line.119"></a>
<span class="sourceLineNo">120</span> builder.addMutationRequest(ProtobufUtil.toMutation(MutationType.PUT, put));<a name="line.120"></a>
<span class="sourceLineNo">121</span> }<a name="line.121"></a>
<span class="sourceLineNo">122</span> }<a name="line.122"></a>
<span class="sourceLineNo">123</span><a name="line.123"></a>
<span class="sourceLineNo">124</span> @Override<a name="line.124"></a>
<span class="sourceLineNo">125</span> public void setOffset(ReplicationQueueId queueId, String walGroup, ReplicationGroupOffset offset,<a name="line.125"></a>
<span class="sourceLineNo">126</span> Map&lt;String, Long&gt; lastSeqIds) throws ReplicationException {<a name="line.126"></a>
<span class="sourceLineNo">127</span> Put put = new Put(Bytes.toBytes(queueId.toString())).addColumn(QUEUE_FAMILY,<a name="line.127"></a>
<span class="sourceLineNo">128</span> Bytes.toBytes(walGroup), Bytes.toBytes(offset.toString()));<a name="line.128"></a>
<span class="sourceLineNo">129</span> AsyncTable&lt;?&gt; asyncTable = conn.toAsyncConnection().getTable(tableName);<a name="line.129"></a>
<span class="sourceLineNo">130</span> try {<a name="line.130"></a>
<span class="sourceLineNo">131</span> if (lastSeqIds.isEmpty()) {<a name="line.131"></a>
<span class="sourceLineNo">132</span> FutureUtils.get(asyncTable.put(put));<a name="line.132"></a>
<span class="sourceLineNo">133</span> } else {<a name="line.133"></a>
<span class="sourceLineNo">134</span> for (;;) {<a name="line.134"></a>
<span class="sourceLineNo">135</span> MultiRowMutationProtos.MutateRowsRequest.Builder builder =<a name="line.135"></a>
<span class="sourceLineNo">136</span> MultiRowMutationProtos.MutateRowsRequest.newBuilder();<a name="line.136"></a>
<span class="sourceLineNo">137</span> addLastSeqIdsPut(builder, queueId.getPeerId(), lastSeqIds, asyncTable);<a name="line.137"></a>
<span class="sourceLineNo">138</span> if (builder.getMutationRequestCount() &gt; 0) {<a name="line.138"></a>
<span class="sourceLineNo">139</span> // use MultiRowMutationService to atomically update offset and last sequence ids<a name="line.139"></a>
<span class="sourceLineNo">140</span> MultiRowMutationProtos.MutateRowsRequest request =<a name="line.140"></a>
<span class="sourceLineNo">141</span> builder.addMutationRequest(ProtobufUtil.toMutation(MutationType.PUT, put)).build();<a name="line.141"></a>
<span class="sourceLineNo">142</span> MultiRowMutationProtos.MutateRowsResponse responose =<a name="line.142"></a>
<span class="sourceLineNo">143</span> FutureUtils.get(asyncTable.&lt;MultiRowMutationProtos.MultiRowMutationService.Interface,<a name="line.143"></a>
<span class="sourceLineNo">144</span> MultiRowMutationProtos.MutateRowsResponse&gt; coprocessorService(<a name="line.144"></a>
<span class="sourceLineNo">145</span> MultiRowMutationProtos.MultiRowMutationService::newStub,<a name="line.145"></a>
<span class="sourceLineNo">146</span> (stub, controller, done) -&gt; stub.mutateRows(controller, request, done),<a name="line.146"></a>
<span class="sourceLineNo">147</span> put.getRow()));<a name="line.147"></a>
<span class="sourceLineNo">148</span> if (responose.getProcessed()) {<a name="line.148"></a>
<span class="sourceLineNo">149</span> break;<a name="line.149"></a>
<span class="sourceLineNo">150</span> }<a name="line.150"></a>
<span class="sourceLineNo">151</span> } else {<a name="line.151"></a>
<span class="sourceLineNo">152</span> // we do not need to update last seq id, fallback to single put<a name="line.152"></a>
<span class="sourceLineNo">153</span> FutureUtils.get(asyncTable.put(put));<a name="line.153"></a>
<span class="sourceLineNo">154</span> break;<a name="line.154"></a>
<span class="sourceLineNo">155</span> }<a name="line.155"></a>
<span class="sourceLineNo">156</span> }<a name="line.156"></a>
<span class="sourceLineNo">157</span> }<a name="line.157"></a>
<span class="sourceLineNo">158</span> } catch (IOException e) {<a name="line.158"></a>
<span class="sourceLineNo">159</span> throw new ReplicationException("failed to setOffset, queueId=" + queueId + ", walGroup="<a name="line.159"></a>
<span class="sourceLineNo">160</span> + walGroup + ", offset=" + offset + ", lastSeqIds=" + lastSeqIds, e);<a name="line.160"></a>
<span class="sourceLineNo">161</span> }<a name="line.161"></a>
<span class="sourceLineNo">162</span> }<a name="line.162"></a>
<span class="sourceLineNo">163</span><a name="line.163"></a>
<span class="sourceLineNo">164</span> private ImmutableMap&lt;String, ReplicationGroupOffset&gt; parseOffsets(Result result) {<a name="line.164"></a>
<span class="sourceLineNo">165</span> ImmutableMap.Builder&lt;String, ReplicationGroupOffset&gt; builder =<a name="line.165"></a>
<span class="sourceLineNo">166</span> ImmutableMap.builderWithExpectedSize(result.size());<a name="line.166"></a>
<span class="sourceLineNo">167</span> NavigableMap&lt;byte[], byte[]&gt; map = result.getFamilyMap(QUEUE_FAMILY);<a name="line.167"></a>
<span class="sourceLineNo">168</span> if (map != null) {<a name="line.168"></a>
<span class="sourceLineNo">169</span> map.forEach((k, v) -&gt; {<a name="line.169"></a>
<span class="sourceLineNo">170</span> String walGroup = Bytes.toString(k);<a name="line.170"></a>
<span class="sourceLineNo">171</span> ReplicationGroupOffset offset = ReplicationGroupOffset.parse(Bytes.toString(v));<a name="line.171"></a>
<span class="sourceLineNo">172</span> builder.put(walGroup, offset);<a name="line.172"></a>
<span class="sourceLineNo">173</span> });<a name="line.173"></a>
<span class="sourceLineNo">174</span> }<a name="line.174"></a>
<span class="sourceLineNo">175</span> return builder.build();<a name="line.175"></a>
<span class="sourceLineNo">176</span> }<a name="line.176"></a>
<span class="sourceLineNo">177</span><a name="line.177"></a>
<span class="sourceLineNo">178</span> private Map&lt;String, ReplicationGroupOffset&gt; getOffsets0(Table table, ReplicationQueueId queueId)<a name="line.178"></a>
<span class="sourceLineNo">179</span> throws IOException {<a name="line.179"></a>
<span class="sourceLineNo">180</span> Result result = table.get(new Get(Bytes.toBytes(queueId.toString())).addFamily(QUEUE_FAMILY));<a name="line.180"></a>
<span class="sourceLineNo">181</span> return parseOffsets(result);<a name="line.181"></a>
<span class="sourceLineNo">182</span> }<a name="line.182"></a>
<span class="sourceLineNo">183</span><a name="line.183"></a>
<span class="sourceLineNo">184</span> @Override<a name="line.184"></a>
<span class="sourceLineNo">185</span> public Map&lt;String, ReplicationGroupOffset&gt; getOffsets(ReplicationQueueId queueId)<a name="line.185"></a>
<span class="sourceLineNo">186</span> throws ReplicationException {<a name="line.186"></a>
<span class="sourceLineNo">187</span> try (Table table = conn.getTable(tableName)) {<a name="line.187"></a>
<span class="sourceLineNo">188</span> return getOffsets0(table, queueId);<a name="line.188"></a>
<span class="sourceLineNo">189</span> } catch (IOException e) {<a name="line.189"></a>
<span class="sourceLineNo">190</span> throw new ReplicationException("failed to getOffsets, queueId=" + queueId, e);<a name="line.190"></a>
<span class="sourceLineNo">191</span> }<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> private void listAllQueueIds(Table table, Scan scan, List&lt;ReplicationQueueId&gt; queueIds)<a name="line.194"></a>
<span class="sourceLineNo">195</span> throws IOException {<a name="line.195"></a>
<span class="sourceLineNo">196</span> try (ResultScanner scanner = table.getScanner(scan)) {<a name="line.196"></a>
<span class="sourceLineNo">197</span> for (;;) {<a name="line.197"></a>
<span class="sourceLineNo">198</span> Result result = scanner.next();<a name="line.198"></a>
<span class="sourceLineNo">199</span> if (result == null) {<a name="line.199"></a>
<span class="sourceLineNo">200</span> break;<a name="line.200"></a>
<span class="sourceLineNo">201</span> }<a name="line.201"></a>
<span class="sourceLineNo">202</span> ReplicationQueueId queueId = ReplicationQueueId.parse(Bytes.toString(result.getRow()));<a name="line.202"></a>
<span class="sourceLineNo">203</span> queueIds.add(queueId);<a name="line.203"></a>
<span class="sourceLineNo">204</span> }<a name="line.204"></a>
<span class="sourceLineNo">205</span> }<a name="line.205"></a>
<span class="sourceLineNo">206</span> }<a name="line.206"></a>
<span class="sourceLineNo">207</span><a name="line.207"></a>
<span class="sourceLineNo">208</span> private void listAllQueueIds(Table table, String peerId, ServerName serverName,<a name="line.208"></a>
<span class="sourceLineNo">209</span> List&lt;ReplicationQueueId&gt; queueIds) throws IOException {<a name="line.209"></a>
<span class="sourceLineNo">210</span> listAllQueueIds(table,<a name="line.210"></a>
<span class="sourceLineNo">211</span> new Scan().setStartStopRowForPrefixScan(ReplicationQueueId.getScanPrefix(serverName, peerId))<a name="line.211"></a>
<span class="sourceLineNo">212</span> .addFamily(QUEUE_FAMILY).setFilter(new KeyOnlyFilter()),<a name="line.212"></a>
<span class="sourceLineNo">213</span> queueIds);<a name="line.213"></a>
<span class="sourceLineNo">214</span> }<a name="line.214"></a>
<span class="sourceLineNo">215</span><a name="line.215"></a>
<span class="sourceLineNo">216</span> @Override<a name="line.216"></a>
<span class="sourceLineNo">217</span> public List&lt;ReplicationQueueId&gt; listAllQueueIds(String peerId) throws ReplicationException {<a name="line.217"></a>
<span class="sourceLineNo">218</span> Scan scan = new Scan().setStartStopRowForPrefixScan(ReplicationQueueId.getScanPrefix(peerId))<a name="line.218"></a>
<span class="sourceLineNo">219</span> .addFamily(QUEUE_FAMILY).setFilter(new KeyOnlyFilter());<a name="line.219"></a>
<span class="sourceLineNo">220</span> List&lt;ReplicationQueueId&gt; queueIds = new ArrayList&lt;&gt;();<a name="line.220"></a>
<span class="sourceLineNo">221</span> try (Table table = conn.getTable(tableName)) {<a name="line.221"></a>
<span class="sourceLineNo">222</span> listAllQueueIds(table, scan, queueIds);<a name="line.222"></a>
<span class="sourceLineNo">223</span> } catch (IOException e) {<a name="line.223"></a>
<span class="sourceLineNo">224</span> throw new ReplicationException("failed to listAllQueueIds, peerId=" + peerId, e);<a name="line.224"></a>
<span class="sourceLineNo">225</span> }<a name="line.225"></a>
<span class="sourceLineNo">226</span> return queueIds;<a name="line.226"></a>
<span class="sourceLineNo">227</span> }<a name="line.227"></a>
<span class="sourceLineNo">228</span><a name="line.228"></a>
<span class="sourceLineNo">229</span> @Override<a name="line.229"></a>
<span class="sourceLineNo">230</span> public List&lt;ReplicationQueueId&gt; listAllQueueIds(ServerName serverName)<a name="line.230"></a>
<span class="sourceLineNo">231</span> throws ReplicationException {<a name="line.231"></a>
<span class="sourceLineNo">232</span> List&lt;ReplicationQueueId&gt; queueIds = new ArrayList&lt;&gt;();<a name="line.232"></a>
<span class="sourceLineNo">233</span> try (Table table = conn.getTable(tableName)) {<a name="line.233"></a>
<span class="sourceLineNo">234</span> KeyOnlyFilter keyOnlyFilter = new KeyOnlyFilter();<a name="line.234"></a>
<span class="sourceLineNo">235</span> String previousPeerId = null;<a name="line.235"></a>
<span class="sourceLineNo">236</span> for (;;) {<a name="line.236"></a>
<span class="sourceLineNo">237</span> // first, get the next peerId<a name="line.237"></a>
<span class="sourceLineNo">238</span> Scan peerScan =<a name="line.238"></a>
<span class="sourceLineNo">239</span> new Scan().addFamily(QUEUE_FAMILY).setOneRowLimit().setFilter(keyOnlyFilter);<a name="line.239"></a>
<span class="sourceLineNo">240</span> if (previousPeerId != null) {<a name="line.240"></a>
<span class="sourceLineNo">241</span> peerScan.withStartRow(ReplicationQueueId.getScanStartRowForNextPeerId(previousPeerId));<a name="line.241"></a>
<span class="sourceLineNo">242</span> }<a name="line.242"></a>
<span class="sourceLineNo">243</span> String peerId;<a name="line.243"></a>
<span class="sourceLineNo">244</span> try (ResultScanner scanner = table.getScanner(peerScan)) {<a name="line.244"></a>
<span class="sourceLineNo">245</span> Result result = scanner.next();<a name="line.245"></a>
<span class="sourceLineNo">246</span> if (result == null) {<a name="line.246"></a>
<span class="sourceLineNo">247</span> // no more peers, break<a name="line.247"></a>
<span class="sourceLineNo">248</span> break;<a name="line.248"></a>
<span class="sourceLineNo">249</span> }<a name="line.249"></a>
<span class="sourceLineNo">250</span> peerId = ReplicationQueueId.getPeerId(Bytes.toString(result.getRow()));<a name="line.250"></a>
<span class="sourceLineNo">251</span> }<a name="line.251"></a>
<span class="sourceLineNo">252</span> listAllQueueIds(table, peerId, serverName, queueIds);<a name="line.252"></a>
<span class="sourceLineNo">253</span> previousPeerId = peerId;<a name="line.253"></a>
<span class="sourceLineNo">254</span> }<a name="line.254"></a>
<span class="sourceLineNo">255</span> } catch (IOException e) {<a name="line.255"></a>
<span class="sourceLineNo">256</span> throw new ReplicationException("failed to listAllQueueIds, serverName=" + serverName, e);<a name="line.256"></a>
<span class="sourceLineNo">257</span> }<a name="line.257"></a>
<span class="sourceLineNo">258</span> return queueIds;<a name="line.258"></a>
<span class="sourceLineNo">259</span> }<a name="line.259"></a>
<span class="sourceLineNo">260</span><a name="line.260"></a>
<span class="sourceLineNo">261</span> @Override<a name="line.261"></a>
<span class="sourceLineNo">262</span> public List&lt;ReplicationQueueId&gt; listAllQueueIds(String peerId, ServerName serverName)<a name="line.262"></a>
<span class="sourceLineNo">263</span> throws ReplicationException {<a name="line.263"></a>
<span class="sourceLineNo">264</span> List&lt;ReplicationQueueId&gt; queueIds = new ArrayList&lt;&gt;();<a name="line.264"></a>
<span class="sourceLineNo">265</span> try (Table table = conn.getTable(tableName)) {<a name="line.265"></a>
<span class="sourceLineNo">266</span> listAllQueueIds(table, peerId, serverName, queueIds);<a name="line.266"></a>
<span class="sourceLineNo">267</span> } catch (IOException e) {<a name="line.267"></a>
<span class="sourceLineNo">268</span> throw new ReplicationException(<a name="line.268"></a>
<span class="sourceLineNo">269</span> "failed to listAllQueueIds, peerId=" + peerId + ", serverName=" + serverName, e);<a name="line.269"></a>
<span class="sourceLineNo">270</span> }<a name="line.270"></a>
<span class="sourceLineNo">271</span> return queueIds;<a name="line.271"></a>
<span class="sourceLineNo">272</span> }<a name="line.272"></a>
<span class="sourceLineNo">273</span><a name="line.273"></a>
<span class="sourceLineNo">274</span> @Override<a name="line.274"></a>
<span class="sourceLineNo">275</span> public List&lt;ReplicationQueueData&gt; listAllQueues() throws ReplicationException {<a name="line.275"></a>
<span class="sourceLineNo">276</span> List&lt;ReplicationQueueData&gt; queues = new ArrayList&lt;&gt;();<a name="line.276"></a>
<span class="sourceLineNo">277</span> Scan scan = new Scan().addFamily(QUEUE_FAMILY).setReadType(ReadType.STREAM);<a name="line.277"></a>
<span class="sourceLineNo">278</span> try (Table table = conn.getTable(tableName); ResultScanner scanner = table.getScanner(scan)) {<a name="line.278"></a>
<span class="sourceLineNo">279</span> for (;;) {<a name="line.279"></a>
<span class="sourceLineNo">280</span> Result result = scanner.next();<a name="line.280"></a>
<span class="sourceLineNo">281</span> if (result == null) {<a name="line.281"></a>
<span class="sourceLineNo">282</span> break;<a name="line.282"></a>
<span class="sourceLineNo">283</span> }<a name="line.283"></a>
<span class="sourceLineNo">284</span> ReplicationQueueId queueId = ReplicationQueueId.parse(Bytes.toString(result.getRow()));<a name="line.284"></a>
<span class="sourceLineNo">285</span> ReplicationQueueData queueData = new ReplicationQueueData(queueId, parseOffsets(result));<a name="line.285"></a>
<span class="sourceLineNo">286</span> queues.add(queueData);<a name="line.286"></a>
<span class="sourceLineNo">287</span> }<a name="line.287"></a>
<span class="sourceLineNo">288</span> } catch (IOException e) {<a name="line.288"></a>
<span class="sourceLineNo">289</span> throw new ReplicationException("failed to listAllQueues", e);<a name="line.289"></a>
<span class="sourceLineNo">290</span> }<a name="line.290"></a>
<span class="sourceLineNo">291</span> return queues;<a name="line.291"></a>
<span class="sourceLineNo">292</span> }<a name="line.292"></a>
<span class="sourceLineNo">293</span><a name="line.293"></a>
<span class="sourceLineNo">294</span> @Override<a name="line.294"></a>
<span class="sourceLineNo">295</span> public List&lt;ServerName&gt; listAllReplicators() throws ReplicationException {<a name="line.295"></a>
<span class="sourceLineNo">296</span> Set&lt;ServerName&gt; replicators = new HashSet&lt;&gt;();<a name="line.296"></a>
<span class="sourceLineNo">297</span> Scan scan = new Scan().addFamily(QUEUE_FAMILY).setFilter(new KeyOnlyFilter())<a name="line.297"></a>
<span class="sourceLineNo">298</span> .setReadType(ReadType.STREAM);<a name="line.298"></a>
<span class="sourceLineNo">299</span> try (Table table = conn.getTable(tableName); ResultScanner scanner = table.getScanner(scan)) {<a name="line.299"></a>
<span class="sourceLineNo">300</span> for (;;) {<a name="line.300"></a>
<span class="sourceLineNo">301</span> Result result = scanner.next();<a name="line.301"></a>
<span class="sourceLineNo">302</span> if (result == null) {<a name="line.302"></a>
<span class="sourceLineNo">303</span> break;<a name="line.303"></a>
<span class="sourceLineNo">304</span> }<a name="line.304"></a>
<span class="sourceLineNo">305</span> ReplicationQueueId queueId = ReplicationQueueId.parse(Bytes.toString(result.getRow()));<a name="line.305"></a>
<span class="sourceLineNo">306</span> replicators.add(queueId.getServerName());<a name="line.306"></a>
<span class="sourceLineNo">307</span> }<a name="line.307"></a>
<span class="sourceLineNo">308</span> } catch (IOException e) {<a name="line.308"></a>
<span class="sourceLineNo">309</span> throw new ReplicationException("failed to listAllReplicators", e);<a name="line.309"></a>
<span class="sourceLineNo">310</span> }<a name="line.310"></a>
<span class="sourceLineNo">311</span> return new ArrayList&lt;&gt;(replicators);<a name="line.311"></a>
<span class="sourceLineNo">312</span> }<a name="line.312"></a>
<span class="sourceLineNo">313</span><a name="line.313"></a>
<span class="sourceLineNo">314</span> @Override<a name="line.314"></a>
<span class="sourceLineNo">315</span> public Map&lt;String, ReplicationGroupOffset&gt; claimQueue(ReplicationQueueId queueId,<a name="line.315"></a>
<span class="sourceLineNo">316</span> ServerName targetServerName) throws ReplicationException {<a name="line.316"></a>
<span class="sourceLineNo">317</span> ReplicationQueueId newQueueId = queueId.claim(targetServerName);<a name="line.317"></a>
<span class="sourceLineNo">318</span> byte[] coprocessorRow = ReplicationQueueId.getScanPrefix(queueId.getPeerId());<a name="line.318"></a>
<span class="sourceLineNo">319</span> AsyncTable&lt;?&gt; asyncTable = conn.toAsyncConnection().getTable(tableName);<a name="line.319"></a>
<span class="sourceLineNo">320</span> try (Table table = conn.getTable(tableName)) {<a name="line.320"></a>
<span class="sourceLineNo">321</span> for (;;) {<a name="line.321"></a>
<span class="sourceLineNo">322</span> Map&lt;String, ReplicationGroupOffset&gt; offsets = getOffsets0(table, queueId);<a name="line.322"></a>
<span class="sourceLineNo">323</span> if (offsets.isEmpty()) {<a name="line.323"></a>
<span class="sourceLineNo">324</span> return Collections.emptyMap();<a name="line.324"></a>
<span class="sourceLineNo">325</span> }<a name="line.325"></a>
<span class="sourceLineNo">326</span> Map.Entry&lt;String, ReplicationGroupOffset&gt; entry = offsets.entrySet().iterator().next();<a name="line.326"></a>
<span class="sourceLineNo">327</span> ClientProtos.Condition condition = ProtobufUtil.toCondition(<a name="line.327"></a>
<span class="sourceLineNo">328</span> Bytes.toBytes(queueId.toString()), QUEUE_FAMILY, Bytes.toBytes(entry.getKey()),<a name="line.328"></a>
<span class="sourceLineNo">329</span> CompareOperator.EQUAL, Bytes.toBytes(entry.getValue().toString()), null);<a name="line.329"></a>
<span class="sourceLineNo">330</span> Delete delete = new Delete(Bytes.toBytes(queueId.toString())).addFamily(QUEUE_FAMILY);<a name="line.330"></a>
<span class="sourceLineNo">331</span> Put put = new Put(Bytes.toBytes(newQueueId.toString()));<a name="line.331"></a>
<span class="sourceLineNo">332</span> offsets.forEach((walGroup, offset) -&gt; put.addColumn(QUEUE_FAMILY, Bytes.toBytes(walGroup),<a name="line.332"></a>
<span class="sourceLineNo">333</span> Bytes.toBytes(offset.toString())));<a name="line.333"></a>
<span class="sourceLineNo">334</span> MultiRowMutationProtos.MutateRowsRequest request =<a name="line.334"></a>
<span class="sourceLineNo">335</span> MultiRowMutationProtos.MutateRowsRequest.newBuilder().addCondition(condition)<a name="line.335"></a>
<span class="sourceLineNo">336</span> .addMutationRequest(ProtobufUtil.toMutation(MutationType.DELETE, delete))<a name="line.336"></a>
<span class="sourceLineNo">337</span> .addMutationRequest(ProtobufUtil.toMutation(MutationType.PUT, put)).build();<a name="line.337"></a>
<span class="sourceLineNo">338</span> MultiRowMutationProtos.MutateRowsResponse resp =<a name="line.338"></a>
<span class="sourceLineNo">339</span> FutureUtils.get(asyncTable.&lt;MultiRowMutationProtos.MultiRowMutationService.Interface,<a name="line.339"></a>
<span class="sourceLineNo">340</span> MultiRowMutationProtos.MutateRowsResponse&gt; coprocessorService(<a name="line.340"></a>
<span class="sourceLineNo">341</span> MultiRowMutationProtos.MultiRowMutationService::newStub,<a name="line.341"></a>
<span class="sourceLineNo">342</span> (stub, controller, done) -&gt; stub.mutateRows(controller, request, done),<a name="line.342"></a>
<span class="sourceLineNo">343</span> coprocessorRow));<a name="line.343"></a>
<span class="sourceLineNo">344</span> if (resp.getProcessed()) {<a name="line.344"></a>
<span class="sourceLineNo">345</span> return offsets;<a name="line.345"></a>
<span class="sourceLineNo">346</span> }<a name="line.346"></a>
<span class="sourceLineNo">347</span> // if the multi is not processed, which usually the queue has already been claimed by<a name="line.347"></a>
<span class="sourceLineNo">348</span> // others, for safety, let's try claiming again, usually the next get operation above will<a name="line.348"></a>
<span class="sourceLineNo">349</span> // return an empty map and we will quit the loop.<a name="line.349"></a>
<span class="sourceLineNo">350</span> }<a name="line.350"></a>
<span class="sourceLineNo">351</span> } catch (IOException e) {<a name="line.351"></a>
<span class="sourceLineNo">352</span> throw new ReplicationException(<a name="line.352"></a>
<span class="sourceLineNo">353</span> "failed to claimQueue, queueId=" + queueId + ", targetServerName=" + targetServerName, e);<a name="line.353"></a>
<span class="sourceLineNo">354</span> }<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> @Override<a name="line.357"></a>
<span class="sourceLineNo">358</span> public void removeQueue(ReplicationQueueId queueId) throws ReplicationException {<a name="line.358"></a>
<span class="sourceLineNo">359</span> try (Table table = conn.getTable(tableName)) {<a name="line.359"></a>
<span class="sourceLineNo">360</span> table.delete(new Delete(Bytes.toBytes(queueId.toString())).addFamily(QUEUE_FAMILY));<a name="line.360"></a>
<span class="sourceLineNo">361</span> } catch (IOException e) {<a name="line.361"></a>
<span class="sourceLineNo">362</span> throw new ReplicationException("failed to removeQueue, queueId=" + queueId, e);<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><a name="line.365"></a>
<span class="sourceLineNo">366</span> @Override<a name="line.366"></a>
<span class="sourceLineNo">367</span> public void removeAllQueues(String peerId) throws ReplicationException {<a name="line.367"></a>
<span class="sourceLineNo">368</span> Scan scan = new Scan().setStartStopRowForPrefixScan(ReplicationQueueId.getScanPrefix(peerId))<a name="line.368"></a>
<span class="sourceLineNo">369</span> .addFamily(QUEUE_FAMILY).setFilter(new KeyOnlyFilter());<a name="line.369"></a>
<span class="sourceLineNo">370</span> try (Table table = conn.getTable(tableName); ResultScanner scanner = table.getScanner(scan)) {<a name="line.370"></a>
<span class="sourceLineNo">371</span> for (;;) {<a name="line.371"></a>
<span class="sourceLineNo">372</span> Result result = scanner.next();<a name="line.372"></a>
<span class="sourceLineNo">373</span> if (result == null) {<a name="line.373"></a>
<span class="sourceLineNo">374</span> break;<a name="line.374"></a>
<span class="sourceLineNo">375</span> }<a name="line.375"></a>
<span class="sourceLineNo">376</span> table.delete(new Delete(result.getRow()));<a name="line.376"></a>
<span class="sourceLineNo">377</span> }<a name="line.377"></a>
<span class="sourceLineNo">378</span> } catch (IOException e) {<a name="line.378"></a>
<span class="sourceLineNo">379</span> throw new ReplicationException("failed to listAllQueueIds, peerId=" + peerId, e);<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><a name="line.382"></a>
<span class="sourceLineNo">383</span> @Override<a name="line.383"></a>
<span class="sourceLineNo">384</span> public long getLastSequenceId(String encodedRegionName, String peerId)<a name="line.384"></a>
<span class="sourceLineNo">385</span> throws ReplicationException {<a name="line.385"></a>
<span class="sourceLineNo">386</span> byte[] qual = Bytes.toBytes(encodedRegionName);<a name="line.386"></a>
<span class="sourceLineNo">387</span> try (Table table = conn.getTable(tableName)) {<a name="line.387"></a>
<span class="sourceLineNo">388</span> Result result =<a name="line.388"></a>
<span class="sourceLineNo">389</span> table.get(new Get(Bytes.toBytes(peerId)).addColumn(LAST_SEQUENCE_ID_FAMILY, qual));<a name="line.389"></a>
<span class="sourceLineNo">390</span> byte[] lastSeqId = result.getValue(LAST_SEQUENCE_ID_FAMILY, qual);<a name="line.390"></a>
<span class="sourceLineNo">391</span> return lastSeqId != null ? Bytes.toLong(lastSeqId) : HConstants.NO_SEQNUM;<a name="line.391"></a>
<span class="sourceLineNo">392</span> } catch (IOException e) {<a name="line.392"></a>
<span class="sourceLineNo">393</span> throw new ReplicationException("failed to getLastSequenceId, encodedRegionName="<a name="line.393"></a>
<span class="sourceLineNo">394</span> + encodedRegionName + ", peerId=" + peerId, e);<a name="line.394"></a>
<span class="sourceLineNo">395</span> }<a name="line.395"></a>
<span class="sourceLineNo">396</span> }<a name="line.396"></a>
<span class="sourceLineNo">397</span><a name="line.397"></a>
<span class="sourceLineNo">398</span> @Override<a name="line.398"></a>
<span class="sourceLineNo">399</span> public void setLastSequenceIds(String peerId, Map&lt;String, Long&gt; lastSeqIds)<a name="line.399"></a>
<span class="sourceLineNo">400</span> throws ReplicationException {<a name="line.400"></a>
<span class="sourceLineNo">401</span> // No need CAS and retry here, because it'll call setLastSequenceIds() for disabled peers<a name="line.401"></a>
<span class="sourceLineNo">402</span> // only, so no conflict happen.<a name="line.402"></a>
<span class="sourceLineNo">403</span> Put put = new Put(Bytes.toBytes(peerId));<a name="line.403"></a>
<span class="sourceLineNo">404</span> lastSeqIds.forEach((encodedRegionName, lastSeqId) -&gt; put.addColumn(LAST_SEQUENCE_ID_FAMILY,<a name="line.404"></a>
<span class="sourceLineNo">405</span> Bytes.toBytes(encodedRegionName), Bytes.toBytes(lastSeqId)));<a name="line.405"></a>
<span class="sourceLineNo">406</span> try (Table table = conn.getTable(tableName)) {<a name="line.406"></a>
<span class="sourceLineNo">407</span> table.put(put);<a name="line.407"></a>
<span class="sourceLineNo">408</span> } catch (IOException e) {<a name="line.408"></a>
<span class="sourceLineNo">409</span> throw new ReplicationException(<a name="line.409"></a>
<span class="sourceLineNo">410</span> "failed to setLastSequenceIds, peerId=" + peerId + ", lastSeqIds=" + lastSeqIds, e);<a name="line.410"></a>
<span class="sourceLineNo">411</span> }<a name="line.411"></a>
<span class="sourceLineNo">412</span> }<a name="line.412"></a>
<span class="sourceLineNo">413</span><a name="line.413"></a>
<span class="sourceLineNo">414</span> @Override<a name="line.414"></a>
<span class="sourceLineNo">415</span> public void removeLastSequenceIds(String peerId) throws ReplicationException {<a name="line.415"></a>
<span class="sourceLineNo">416</span> Delete delete = new Delete(Bytes.toBytes(peerId)).addFamily(LAST_SEQUENCE_ID_FAMILY);<a name="line.416"></a>
<span class="sourceLineNo">417</span> try (Table table = conn.getTable(tableName)) {<a name="line.417"></a>
<span class="sourceLineNo">418</span> table.delete(delete);<a name="line.418"></a>
<span class="sourceLineNo">419</span> } catch (IOException e) {<a name="line.419"></a>
<span class="sourceLineNo">420</span> throw new ReplicationException("failed to removeLastSequenceIds, peerId=" + peerId, e);<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><a name="line.423"></a>
<span class="sourceLineNo">424</span> @Override<a name="line.424"></a>
<span class="sourceLineNo">425</span> public void removeLastSequenceIds(String peerId, List&lt;String&gt; encodedRegionNames)<a name="line.425"></a>
<span class="sourceLineNo">426</span> throws ReplicationException {<a name="line.426"></a>
<span class="sourceLineNo">427</span> Delete delete = new Delete(Bytes.toBytes(peerId));<a name="line.427"></a>
<span class="sourceLineNo">428</span> encodedRegionNames.forEach(n -&gt; delete.addColumns(LAST_SEQUENCE_ID_FAMILY, Bytes.toBytes(n)));<a name="line.428"></a>
<span class="sourceLineNo">429</span> try (Table table = conn.getTable(tableName)) {<a name="line.429"></a>
<span class="sourceLineNo">430</span> table.delete(delete);<a name="line.430"></a>
<span class="sourceLineNo">431</span> } catch (IOException e) {<a name="line.431"></a>
<span class="sourceLineNo">432</span> throw new ReplicationException("failed to removeLastSequenceIds, peerId=" + peerId<a name="line.432"></a>
<span class="sourceLineNo">433</span> + ", encodedRegionNames=" + encodedRegionNames, e);<a name="line.433"></a>
<span class="sourceLineNo">434</span> }<a name="line.434"></a>
<span class="sourceLineNo">435</span> }<a name="line.435"></a>
<span class="sourceLineNo">436</span><a name="line.436"></a>
<span class="sourceLineNo">437</span> @Override<a name="line.437"></a>
<span class="sourceLineNo">438</span> public void removePeerFromHFileRefs(String peerId) throws ReplicationException {<a name="line.438"></a>
<span class="sourceLineNo">439</span> try (Table table = conn.getTable(tableName)) {<a name="line.439"></a>
<span class="sourceLineNo">440</span> table.delete(new Delete(Bytes.toBytes(peerId)).addFamily(HFILE_REF_FAMILY));<a name="line.440"></a>
<span class="sourceLineNo">441</span> } catch (IOException e) {<a name="line.441"></a>
<span class="sourceLineNo">442</span> throw new ReplicationException("failed to removePeerFromHFileRefs, peerId=" + peerId, e);<a name="line.442"></a>
<span class="sourceLineNo">443</span> }<a name="line.443"></a>
<span class="sourceLineNo">444</span> }<a name="line.444"></a>
<span class="sourceLineNo">445</span><a name="line.445"></a>
<span class="sourceLineNo">446</span> @Override<a name="line.446"></a>
<span class="sourceLineNo">447</span> public void addHFileRefs(String peerId, List&lt;Pair&lt;Path, Path&gt;&gt; pairs)<a name="line.447"></a>
<span class="sourceLineNo">448</span> throws ReplicationException {<a name="line.448"></a>
<span class="sourceLineNo">449</span> Put put = new Put(Bytes.toBytes(peerId));<a name="line.449"></a>
<span class="sourceLineNo">450</span> pairs.forEach(p -&gt; put.addColumn(HFILE_REF_FAMILY, Bytes.toBytes(p.getSecond().getName()),<a name="line.450"></a>
<span class="sourceLineNo">451</span> HConstants.EMPTY_BYTE_ARRAY));<a name="line.451"></a>
<span class="sourceLineNo">452</span> try (Table table = conn.getTable(tableName)) {<a name="line.452"></a>
<span class="sourceLineNo">453</span> table.put(put);<a name="line.453"></a>
<span class="sourceLineNo">454</span> } catch (IOException e) {<a name="line.454"></a>
<span class="sourceLineNo">455</span> throw new ReplicationException(<a name="line.455"></a>
<span class="sourceLineNo">456</span> "failed to addHFileRefs, peerId=" + peerId + ", pairs=" + pairs, e);<a name="line.456"></a>
<span class="sourceLineNo">457</span> }<a name="line.457"></a>
<span class="sourceLineNo">458</span> }<a name="line.458"></a>
<span class="sourceLineNo">459</span><a name="line.459"></a>
<span class="sourceLineNo">460</span> @Override<a name="line.460"></a>
<span class="sourceLineNo">461</span> public void removeHFileRefs(String peerId, List&lt;String&gt; files) throws ReplicationException {<a name="line.461"></a>
<span class="sourceLineNo">462</span> Delete delete = new Delete(Bytes.toBytes(peerId));<a name="line.462"></a>
<span class="sourceLineNo">463</span> files.forEach(f -&gt; delete.addColumns(HFILE_REF_FAMILY, Bytes.toBytes(f)));<a name="line.463"></a>
<span class="sourceLineNo">464</span> try (Table table = conn.getTable(tableName)) {<a name="line.464"></a>
<span class="sourceLineNo">465</span> table.delete(delete);<a name="line.465"></a>
<span class="sourceLineNo">466</span> } catch (IOException e) {<a name="line.466"></a>
<span class="sourceLineNo">467</span> throw new ReplicationException(<a name="line.467"></a>
<span class="sourceLineNo">468</span> "failed to removeHFileRefs, peerId=" + peerId + ", files=" + files, e);<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><a name="line.471"></a>
<span class="sourceLineNo">472</span> @Override<a name="line.472"></a>
<span class="sourceLineNo">473</span> public List&lt;String&gt; getAllPeersFromHFileRefsQueue() throws ReplicationException {<a name="line.473"></a>
<span class="sourceLineNo">474</span> List&lt;String&gt; peerIds = new ArrayList&lt;&gt;();<a name="line.474"></a>
<span class="sourceLineNo">475</span> Scan scan = new Scan().addFamily(HFILE_REF_FAMILY).setReadType(ReadType.STREAM)<a name="line.475"></a>
<span class="sourceLineNo">476</span> .setFilter(new KeyOnlyFilter());<a name="line.476"></a>
<span class="sourceLineNo">477</span> try (Table table = conn.getTable(tableName); ResultScanner scanner = table.getScanner(scan)) {<a name="line.477"></a>
<span class="sourceLineNo">478</span> for (;;) {<a name="line.478"></a>
<span class="sourceLineNo">479</span> Result result = scanner.next();<a name="line.479"></a>
<span class="sourceLineNo">480</span> if (result == null) {<a name="line.480"></a>
<span class="sourceLineNo">481</span> break;<a name="line.481"></a>
<span class="sourceLineNo">482</span> }<a name="line.482"></a>
<span class="sourceLineNo">483</span> peerIds.add(Bytes.toString(result.getRow()));<a name="line.483"></a>
<span class="sourceLineNo">484</span> }<a name="line.484"></a>
<span class="sourceLineNo">485</span> } catch (IOException e) {<a name="line.485"></a>
<span class="sourceLineNo">486</span> throw new ReplicationException("failed to getAllPeersFromHFileRefsQueue", e);<a name="line.486"></a>
<span class="sourceLineNo">487</span> }<a name="line.487"></a>
<span class="sourceLineNo">488</span> return peerIds;<a name="line.488"></a>
<span class="sourceLineNo">489</span> }<a name="line.489"></a>
<span class="sourceLineNo">490</span><a name="line.490"></a>
<span class="sourceLineNo">491</span> private &lt;T extends Collection&lt;String&gt;&gt; T scanHFiles(Scan scan, Supplier&lt;T&gt; creator)<a name="line.491"></a>
<span class="sourceLineNo">492</span> throws IOException {<a name="line.492"></a>
<span class="sourceLineNo">493</span> T files = creator.get();<a name="line.493"></a>
<span class="sourceLineNo">494</span> try (Table table = conn.getTable(tableName); ResultScanner scanner = table.getScanner(scan)) {<a name="line.494"></a>
<span class="sourceLineNo">495</span> for (;;) {<a name="line.495"></a>
<span class="sourceLineNo">496</span> Result result = scanner.next();<a name="line.496"></a>
<span class="sourceLineNo">497</span> if (result == null) {<a name="line.497"></a>
<span class="sourceLineNo">498</span> break;<a name="line.498"></a>
<span class="sourceLineNo">499</span> }<a name="line.499"></a>
<span class="sourceLineNo">500</span> CellScanner cellScanner = result.cellScanner();<a name="line.500"></a>
<span class="sourceLineNo">501</span> while (cellScanner.advance()) {<a name="line.501"></a>
<span class="sourceLineNo">502</span> Cell cell = cellScanner.current();<a name="line.502"></a>
<span class="sourceLineNo">503</span> files.add(Bytes.toString(CellUtil.cloneQualifier(cell)));<a name="line.503"></a>
<span class="sourceLineNo">504</span> }<a name="line.504"></a>
<span class="sourceLineNo">505</span> }<a name="line.505"></a>
<span class="sourceLineNo">506</span> }<a name="line.506"></a>
<span class="sourceLineNo">507</span> return files;<a name="line.507"></a>
<span class="sourceLineNo">508</span> }<a name="line.508"></a>
<span class="sourceLineNo">509</span><a name="line.509"></a>
<span class="sourceLineNo">510</span> @Override<a name="line.510"></a>
<span class="sourceLineNo">511</span> public List&lt;String&gt; getReplicableHFiles(String peerId) throws ReplicationException {<a name="line.511"></a>
<span class="sourceLineNo">512</span> // use scan to avoid getting a too large row one time, which may cause a very huge memory usage.<a name="line.512"></a>
<span class="sourceLineNo">513</span> Scan scan = new Scan().addFamily(HFILE_REF_FAMILY)<a name="line.513"></a>
<span class="sourceLineNo">514</span> .setStartStopRowForPrefixScan(Bytes.toBytes(peerId)).setAllowPartialResults(true);<a name="line.514"></a>
<span class="sourceLineNo">515</span> try {<a name="line.515"></a>
<span class="sourceLineNo">516</span> return scanHFiles(scan, ArrayList::new);<a name="line.516"></a>
<span class="sourceLineNo">517</span> } catch (IOException e) {<a name="line.517"></a>
<span class="sourceLineNo">518</span> throw new ReplicationException("failed to getReplicableHFiles, peerId=" + peerId, e);<a name="line.518"></a>
<span class="sourceLineNo">519</span> }<a name="line.519"></a>
<span class="sourceLineNo">520</span> }<a name="line.520"></a>
<span class="sourceLineNo">521</span><a name="line.521"></a>
<span class="sourceLineNo">522</span> @Override<a name="line.522"></a>
<span class="sourceLineNo">523</span> public Set&lt;String&gt; getAllHFileRefs() throws ReplicationException {<a name="line.523"></a>
<span class="sourceLineNo">524</span> Scan scan = new Scan().addFamily(HFILE_REF_FAMILY).setReadType(ReadType.STREAM)<a name="line.524"></a>
<span class="sourceLineNo">525</span> .setAllowPartialResults(true);<a name="line.525"></a>
<span class="sourceLineNo">526</span> try {<a name="line.526"></a>
<span class="sourceLineNo">527</span> return scanHFiles(scan, HashSet::new);<a name="line.527"></a>
<span class="sourceLineNo">528</span> } catch (IOException e) {<a name="line.528"></a>
<span class="sourceLineNo">529</span> throw new ReplicationException("failed to getAllHFileRefs", e);<a name="line.529"></a>
<span class="sourceLineNo">530</span> }<a name="line.530"></a>
<span class="sourceLineNo">531</span> }<a name="line.531"></a>
<span class="sourceLineNo">532</span><a name="line.532"></a>
<span class="sourceLineNo">533</span> @Override<a name="line.533"></a>
<span class="sourceLineNo">534</span> public boolean hasData() throws ReplicationException {<a name="line.534"></a>
<span class="sourceLineNo">535</span> try {<a name="line.535"></a>
<span class="sourceLineNo">536</span> return conn.getAdmin().tableExists(tableName);<a name="line.536"></a>
<span class="sourceLineNo">537</span> } catch (IOException e) {<a name="line.537"></a>
<span class="sourceLineNo">538</span> throw new ReplicationException("failed to get replication queue table", e);<a name="line.538"></a>
<span class="sourceLineNo">539</span> }<a name="line.539"></a>
<span class="sourceLineNo">540</span> }<a name="line.540"></a>
<span class="sourceLineNo">541</span><a name="line.541"></a>
<span class="sourceLineNo">542</span> @Override<a name="line.542"></a>
<span class="sourceLineNo">543</span> public void batchUpdateQueues(ServerName serverName, List&lt;ReplicationQueueData&gt; datas)<a name="line.543"></a>
<span class="sourceLineNo">544</span> throws ReplicationException {<a name="line.544"></a>
<span class="sourceLineNo">545</span> List&lt;Put&gt; puts = new ArrayList&lt;&gt;();<a name="line.545"></a>
<span class="sourceLineNo">546</span> for (ReplicationQueueData data : datas) {<a name="line.546"></a>
<span class="sourceLineNo">547</span> if (data.getOffsets().isEmpty()) {<a name="line.547"></a>
<span class="sourceLineNo">548</span> continue;<a name="line.548"></a>
<span class="sourceLineNo">549</span> }<a name="line.549"></a>
<span class="sourceLineNo">550</span> Put put = new Put(Bytes.toBytes(data.getId().toString()));<a name="line.550"></a>
<span class="sourceLineNo">551</span> data.getOffsets().forEach((walGroup, offset) -&gt; {<a name="line.551"></a>
<span class="sourceLineNo">552</span> put.addColumn(QUEUE_FAMILY, Bytes.toBytes(walGroup), Bytes.toBytes(offset.toString()));<a name="line.552"></a>
<span class="sourceLineNo">553</span> });<a name="line.553"></a>
<span class="sourceLineNo">554</span> puts.add(put);<a name="line.554"></a>
<span class="sourceLineNo">555</span> }<a name="line.555"></a>
<span class="sourceLineNo">556</span> try (Table table = conn.getTable(tableName)) {<a name="line.556"></a>
<span class="sourceLineNo">557</span> table.put(puts);<a name="line.557"></a>
<span class="sourceLineNo">558</span> } catch (IOException e) {<a name="line.558"></a>
<span class="sourceLineNo">559</span> throw new ReplicationException("failed to batch update queues", e);<a name="line.559"></a>
<span class="sourceLineNo">560</span> }<a name="line.560"></a>
<span class="sourceLineNo">561</span> }<a name="line.561"></a>
<span class="sourceLineNo">562</span><a name="line.562"></a>
<span class="sourceLineNo">563</span> @Override<a name="line.563"></a>
<span class="sourceLineNo">564</span> public void batchUpdateLastSequenceIds(List&lt;ZkLastPushedSeqId&gt; lastPushedSeqIds)<a name="line.564"></a>
<span class="sourceLineNo">565</span> throws ReplicationException {<a name="line.565"></a>
<span class="sourceLineNo">566</span> Map&lt;String, Put&gt; peerId2Put = new HashMap&lt;&gt;();<a name="line.566"></a>
<span class="sourceLineNo">567</span> for (ZkLastPushedSeqId lastPushedSeqId : lastPushedSeqIds) {<a name="line.567"></a>
<span class="sourceLineNo">568</span> peerId2Put<a name="line.568"></a>
<span class="sourceLineNo">569</span> .computeIfAbsent(lastPushedSeqId.getPeerId(), peerId -&gt; new Put(Bytes.toBytes(peerId)))<a name="line.569"></a>
<span class="sourceLineNo">570</span> .addColumn(LAST_SEQUENCE_ID_FAMILY, Bytes.toBytes(lastPushedSeqId.getEncodedRegionName()),<a name="line.570"></a>
<span class="sourceLineNo">571</span> Bytes.toBytes(lastPushedSeqId.getLastPushedSeqId()));<a name="line.571"></a>
<span class="sourceLineNo">572</span> }<a name="line.572"></a>
<span class="sourceLineNo">573</span> try (Table table = conn.getTable(tableName)) {<a name="line.573"></a>
<span class="sourceLineNo">574</span> table<a name="line.574"></a>
<span class="sourceLineNo">575</span> .put(peerId2Put.values().stream().filter(p -&gt; !p.isEmpty()).collect(Collectors.toList()));<a name="line.575"></a>
<span class="sourceLineNo">576</span> } catch (IOException e) {<a name="line.576"></a>
<span class="sourceLineNo">577</span> throw new ReplicationException("failed to batch update last pushed sequence ids", e);<a name="line.577"></a>
<span class="sourceLineNo">578</span> }<a name="line.578"></a>
<span class="sourceLineNo">579</span> }<a name="line.579"></a>
<span class="sourceLineNo">580</span><a name="line.580"></a>
<span class="sourceLineNo">581</span> @Override<a name="line.581"></a>
<span class="sourceLineNo">582</span> public void batchUpdateHFileRefs(String peerId, List&lt;String&gt; hfileRefs)<a name="line.582"></a>
<span class="sourceLineNo">583</span> throws ReplicationException {<a name="line.583"></a>
<span class="sourceLineNo">584</span> if (hfileRefs.isEmpty()) {<a name="line.584"></a>
<span class="sourceLineNo">585</span> return;<a name="line.585"></a>
<span class="sourceLineNo">586</span> }<a name="line.586"></a>
<span class="sourceLineNo">587</span> Put put = new Put(Bytes.toBytes(peerId));<a name="line.587"></a>
<span class="sourceLineNo">588</span> for (String ref : hfileRefs) {<a name="line.588"></a>
<span class="sourceLineNo">589</span> put.addColumn(HFILE_REF_FAMILY, Bytes.toBytes(ref), HConstants.EMPTY_BYTE_ARRAY);<a name="line.589"></a>
<span class="sourceLineNo">590</span> }<a name="line.590"></a>
<span class="sourceLineNo">591</span> try (Table table = conn.getTable(tableName)) {<a name="line.591"></a>
<span class="sourceLineNo">592</span> table.put(put);<a name="line.592"></a>
<span class="sourceLineNo">593</span> } catch (IOException e) {<a name="line.593"></a>
<span class="sourceLineNo">594</span> throw new ReplicationException("failed to batch update hfile references", e);<a name="line.594"></a>
<span class="sourceLineNo">595</span> }<a name="line.595"></a>
<span class="sourceLineNo">596</span> }<a name="line.596"></a>
<span class="sourceLineNo">597</span><a name="line.597"></a>
<span class="sourceLineNo">598</span> @Override<a name="line.598"></a>
<span class="sourceLineNo">599</span> public void removeLastSequenceIdsAndHFileRefsBefore(long ts) throws ReplicationException {<a name="line.599"></a>
<span class="sourceLineNo">600</span> try (Table table = conn.getTable(tableName);<a name="line.600"></a>
<span class="sourceLineNo">601</span> ResultScanner scanner = table.getScanner(new Scan().addFamily(LAST_SEQUENCE_ID_FAMILY)<a name="line.601"></a>
<span class="sourceLineNo">602</span> .addFamily(HFILE_REF_FAMILY).setFilter(new KeyOnlyFilter()))) {<a name="line.602"></a>
<span class="sourceLineNo">603</span> for (;;) {<a name="line.603"></a>
<span class="sourceLineNo">604</span> Result r = scanner.next();<a name="line.604"></a>
<span class="sourceLineNo">605</span> if (r == null) {<a name="line.605"></a>
<span class="sourceLineNo">606</span> break;<a name="line.606"></a>
<span class="sourceLineNo">607</span> }<a name="line.607"></a>
<span class="sourceLineNo">608</span> Delete delete = new Delete(r.getRow()).addFamily(LAST_SEQUENCE_ID_FAMILY, ts)<a name="line.608"></a>
<span class="sourceLineNo">609</span> .addFamily(HFILE_REF_FAMILY, ts);<a name="line.609"></a>
<span class="sourceLineNo">610</span> table.delete(delete);<a name="line.610"></a>
<span class="sourceLineNo">611</span> }<a name="line.611"></a>
<span class="sourceLineNo">612</span> } catch (IOException e) {<a name="line.612"></a>
<span class="sourceLineNo">613</span> throw new ReplicationException(<a name="line.613"></a>
<span class="sourceLineNo">614</span> "failed to remove last sequence ids and hfile references before timestamp " + ts, e);<a name="line.614"></a>
<span class="sourceLineNo">615</span> }<a name="line.615"></a>
<span class="sourceLineNo">616</span> }<a name="line.616"></a>
<span class="sourceLineNo">617</span>}<a name="line.617"></a>
</pre>
</div>
</body>
</html>