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