blob: 39c3b11645f96829e84bfe1be66649a2efe5e515 [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.replication.regionserver, class: ReplicationSink">
<meta name="generator" content="javadoc/SourceToHTMLConverter">
<link rel="stylesheet" type="text/css" href="../../../../../../../stylesheet.css" title="Style">
</head>
<body class="source-page">
<main role="main">
<div class="source-container">
<pre><span class="source-line-no">001</span><span id="line-1">/*</span>
<span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span>
<span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span>
<span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span>
<span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span>
<span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span>
<span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span>
<span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span>
<span class="source-line-no">009</span><span id="line-9"> *</span>
<span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="source-line-no">011</span><span id="line-11"> *</span>
<span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span>
<span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span>
<span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span>
<span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span>
<span class="source-line-no">017</span><span id="line-17"> */</span>
<span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.replication.regionserver;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import static org.apache.hadoop.hbase.replication.master.ReplicationSinkTrackerTableCreator.OFFSET_COLUMN;</span>
<span class="source-line-no">021</span><span id="line-21">import static org.apache.hadoop.hbase.replication.master.ReplicationSinkTrackerTableCreator.REPLICATION_SINK_TRACKER_ENABLED_DEFAULT;</span>
<span class="source-line-no">022</span><span id="line-22">import static org.apache.hadoop.hbase.replication.master.ReplicationSinkTrackerTableCreator.REPLICATION_SINK_TRACKER_ENABLED_KEY;</span>
<span class="source-line-no">023</span><span id="line-23">import static org.apache.hadoop.hbase.replication.master.ReplicationSinkTrackerTableCreator.REPLICATION_SINK_TRACKER_INFO_FAMILY;</span>
<span class="source-line-no">024</span><span id="line-24">import static org.apache.hadoop.hbase.replication.master.ReplicationSinkTrackerTableCreator.REPLICATION_SINK_TRACKER_TABLE_NAME;</span>
<span class="source-line-no">025</span><span id="line-25">import static org.apache.hadoop.hbase.replication.master.ReplicationSinkTrackerTableCreator.RS_COLUMN;</span>
<span class="source-line-no">026</span><span id="line-26">import static org.apache.hadoop.hbase.replication.master.ReplicationSinkTrackerTableCreator.TIMESTAMP_COLUMN;</span>
<span class="source-line-no">027</span><span id="line-27">import static org.apache.hadoop.hbase.replication.master.ReplicationSinkTrackerTableCreator.WAL_NAME_COLUMN;</span>
<span class="source-line-no">028</span><span id="line-28"></span>
<span class="source-line-no">029</span><span id="line-29">import java.io.ByteArrayInputStream;</span>
<span class="source-line-no">030</span><span id="line-30">import java.io.IOException;</span>
<span class="source-line-no">031</span><span id="line-31">import java.util.ArrayList;</span>
<span class="source-line-no">032</span><span id="line-32">import java.util.Collection;</span>
<span class="source-line-no">033</span><span id="line-33">import java.util.Collections;</span>
<span class="source-line-no">034</span><span id="line-34">import java.util.HashMap;</span>
<span class="source-line-no">035</span><span id="line-35">import java.util.List;</span>
<span class="source-line-no">036</span><span id="line-36">import java.util.Map;</span>
<span class="source-line-no">037</span><span id="line-37">import java.util.Map.Entry;</span>
<span class="source-line-no">038</span><span id="line-38">import java.util.TreeMap;</span>
<span class="source-line-no">039</span><span id="line-39">import java.util.UUID;</span>
<span class="source-line-no">040</span><span id="line-40">import java.util.concurrent.Future;</span>
<span class="source-line-no">041</span><span id="line-41">import java.util.concurrent.atomic.AtomicLong;</span>
<span class="source-line-no">042</span><span id="line-42">import java.util.stream.Collectors;</span>
<span class="source-line-no">043</span><span id="line-43">import org.apache.commons.lang3.StringUtils;</span>
<span class="source-line-no">044</span><span id="line-44">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.fs.Path;</span>
<span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.hbase.Cell;</span>
<span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.hbase.CellUtil;</span>
<span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.hbase.ExtendedCell;</span>
<span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.hbase.ExtendedCellScanner;</span>
<span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.hbase.HBaseConfiguration;</span>
<span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.HConstants;</span>
<span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.TableName;</span>
<span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.TableNotFoundException;</span>
<span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.client.AsyncClusterConnection;</span>
<span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.hbase.client.AsyncTable;</span>
<span class="source-line-no">056</span><span id="line-56">import org.apache.hadoop.hbase.client.ClusterConnectionFactory;</span>
<span class="source-line-no">057</span><span id="line-57">import org.apache.hadoop.hbase.client.Delete;</span>
<span class="source-line-no">058</span><span id="line-58">import org.apache.hadoop.hbase.client.Mutation;</span>
<span class="source-line-no">059</span><span id="line-59">import org.apache.hadoop.hbase.client.Put;</span>
<span class="source-line-no">060</span><span id="line-60">import org.apache.hadoop.hbase.client.RetriesExhaustedException;</span>
<span class="source-line-no">061</span><span id="line-61">import org.apache.hadoop.hbase.client.Row;</span>
<span class="source-line-no">062</span><span id="line-62">import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;</span>
<span class="source-line-no">063</span><span id="line-63">import org.apache.hadoop.hbase.replication.ReplicationUtils;</span>
<span class="source-line-no">064</span><span id="line-64">import org.apache.hadoop.hbase.security.UserProvider;</span>
<span class="source-line-no">065</span><span id="line-65">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">066</span><span id="line-66">import org.apache.hadoop.hbase.util.FutureUtils;</span>
<span class="source-line-no">067</span><span id="line-67">import org.apache.hadoop.hbase.util.Pair;</span>
<span class="source-line-no">068</span><span id="line-68">import org.apache.hadoop.hbase.wal.WALEdit;</span>
<span class="source-line-no">069</span><span id="line-69">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">070</span><span id="line-70">import org.slf4j.Logger;</span>
<span class="source-line-no">071</span><span id="line-71">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">072</span><span id="line-72"></span>
<span class="source-line-no">073</span><span id="line-73">import org.apache.hbase.thirdparty.com.google.common.collect.Lists;</span>
<span class="source-line-no">074</span><span id="line-74"></span>
<span class="source-line-no">075</span><span id="line-75">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;</span>
<span class="source-line-no">076</span><span id="line-76">import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;</span>
<span class="source-line-no">077</span><span id="line-77">import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;</span>
<span class="source-line-no">078</span><span id="line-78">import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescriptor;</span>
<span class="source-line-no">079</span><span id="line-79">import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;</span>
<span class="source-line-no">080</span><span id="line-80"></span>
<span class="source-line-no">081</span><span id="line-81">/**</span>
<span class="source-line-no">082</span><span id="line-82"> * &lt;p&gt;</span>
<span class="source-line-no">083</span><span id="line-83"> * This class is responsible for replicating the edits coming from another cluster.</span>
<span class="source-line-no">084</span><span id="line-84"> * &lt;/p&gt;</span>
<span class="source-line-no">085</span><span id="line-85"> * &lt;p&gt;</span>
<span class="source-line-no">086</span><span id="line-86"> * This replication process is currently waiting for the edits to be applied before the method can</span>
<span class="source-line-no">087</span><span id="line-87"> * return. This means that the replication of edits is synchronized (after reading from WALs in</span>
<span class="source-line-no">088</span><span id="line-88"> * ReplicationSource) and that a single region server cannot receive edits from two sources at the</span>
<span class="source-line-no">089</span><span id="line-89"> * same time</span>
<span class="source-line-no">090</span><span id="line-90"> * &lt;/p&gt;</span>
<span class="source-line-no">091</span><span id="line-91"> * &lt;p&gt;</span>
<span class="source-line-no">092</span><span id="line-92"> * This class uses the native HBase client in order to replicate entries.</span>
<span class="source-line-no">093</span><span id="line-93"> * &lt;/p&gt;</span>
<span class="source-line-no">094</span><span id="line-94"> * TODO make this class more like ReplicationSource wrt log handling</span>
<span class="source-line-no">095</span><span id="line-95"> */</span>
<span class="source-line-no">096</span><span id="line-96">@InterfaceAudience.Private</span>
<span class="source-line-no">097</span><span id="line-97">public class ReplicationSink {</span>
<span class="source-line-no">098</span><span id="line-98"></span>
<span class="source-line-no">099</span><span id="line-99"> private static final Logger LOG = LoggerFactory.getLogger(ReplicationSink.class);</span>
<span class="source-line-no">100</span><span id="line-100"> private final Configuration conf;</span>
<span class="source-line-no">101</span><span id="line-101"> // Volatile because of note in here -- look for double-checked locking:</span>
<span class="source-line-no">102</span><span id="line-102"> // http://www.oracle.com/technetwork/articles/javase/bloch-effective-08-qa-140880.html</span>
<span class="source-line-no">103</span><span id="line-103"> private volatile AsyncClusterConnection sharedConn;</span>
<span class="source-line-no">104</span><span id="line-104"> private final MetricsSink metrics;</span>
<span class="source-line-no">105</span><span id="line-105"> private final AtomicLong totalReplicatedEdits = new AtomicLong();</span>
<span class="source-line-no">106</span><span id="line-106"> private final Object sharedConnLock = new Object();</span>
<span class="source-line-no">107</span><span id="line-107"> // Number of hfiles that we successfully replicated</span>
<span class="source-line-no">108</span><span id="line-108"> private long hfilesReplicated = 0;</span>
<span class="source-line-no">109</span><span id="line-109"> private SourceFSConfigurationProvider provider;</span>
<span class="source-line-no">110</span><span id="line-110"> private WALEntrySinkFilter walEntrySinkFilter;</span>
<span class="source-line-no">111</span><span id="line-111"></span>
<span class="source-line-no">112</span><span id="line-112"> /**</span>
<span class="source-line-no">113</span><span id="line-113"> * Row size threshold for multi requests above which a warning is logged</span>
<span class="source-line-no">114</span><span id="line-114"> */</span>
<span class="source-line-no">115</span><span id="line-115"> private final int rowSizeWarnThreshold;</span>
<span class="source-line-no">116</span><span id="line-116"> private boolean replicationSinkTrackerEnabled;</span>
<span class="source-line-no">117</span><span id="line-117"></span>
<span class="source-line-no">118</span><span id="line-118"> private final RegionServerCoprocessorHost rsServerHost;</span>
<span class="source-line-no">119</span><span id="line-119"></span>
<span class="source-line-no">120</span><span id="line-120"> /**</span>
<span class="source-line-no">121</span><span id="line-121"> * Create a sink for replication</span>
<span class="source-line-no">122</span><span id="line-122"> * @param conf conf object</span>
<span class="source-line-no">123</span><span id="line-123"> * @throws IOException thrown when HDFS goes bad or bad file name</span>
<span class="source-line-no">124</span><span id="line-124"> */</span>
<span class="source-line-no">125</span><span id="line-125"> public ReplicationSink(Configuration conf, RegionServerCoprocessorHost rsServerHost)</span>
<span class="source-line-no">126</span><span id="line-126"> throws IOException {</span>
<span class="source-line-no">127</span><span id="line-127"> this.conf = HBaseConfiguration.create(conf);</span>
<span class="source-line-no">128</span><span id="line-128"> this.rsServerHost = rsServerHost;</span>
<span class="source-line-no">129</span><span id="line-129"> rowSizeWarnThreshold =</span>
<span class="source-line-no">130</span><span id="line-130"> conf.getInt(HConstants.BATCH_ROWS_THRESHOLD_NAME, HConstants.BATCH_ROWS_THRESHOLD_DEFAULT);</span>
<span class="source-line-no">131</span><span id="line-131"> replicationSinkTrackerEnabled = conf.getBoolean(REPLICATION_SINK_TRACKER_ENABLED_KEY,</span>
<span class="source-line-no">132</span><span id="line-132"> REPLICATION_SINK_TRACKER_ENABLED_DEFAULT);</span>
<span class="source-line-no">133</span><span id="line-133"> decorateConf();</span>
<span class="source-line-no">134</span><span id="line-134"> this.metrics = new MetricsSink();</span>
<span class="source-line-no">135</span><span id="line-135"> this.walEntrySinkFilter = setupWALEntrySinkFilter();</span>
<span class="source-line-no">136</span><span id="line-136"> String className = conf.get("hbase.replication.source.fs.conf.provider",</span>
<span class="source-line-no">137</span><span id="line-137"> DefaultSourceFSConfigurationProvider.class.getCanonicalName());</span>
<span class="source-line-no">138</span><span id="line-138"> try {</span>
<span class="source-line-no">139</span><span id="line-139"> Class&lt;? extends SourceFSConfigurationProvider&gt; c =</span>
<span class="source-line-no">140</span><span id="line-140"> Class.forName(className).asSubclass(SourceFSConfigurationProvider.class);</span>
<span class="source-line-no">141</span><span id="line-141"> this.provider = c.getDeclaredConstructor().newInstance();</span>
<span class="source-line-no">142</span><span id="line-142"> } catch (Exception e) {</span>
<span class="source-line-no">143</span><span id="line-143"> throw new IllegalArgumentException(</span>
<span class="source-line-no">144</span><span id="line-144"> "Configured source fs configuration provider class " + className + " throws error.", e);</span>
<span class="source-line-no">145</span><span id="line-145"> }</span>
<span class="source-line-no">146</span><span id="line-146"> }</span>
<span class="source-line-no">147</span><span id="line-147"></span>
<span class="source-line-no">148</span><span id="line-148"> private WALEntrySinkFilter setupWALEntrySinkFilter() throws IOException {</span>
<span class="source-line-no">149</span><span id="line-149"> Class&lt;?&gt; walEntryFilterClass =</span>
<span class="source-line-no">150</span><span id="line-150"> this.conf.getClass(WALEntrySinkFilter.WAL_ENTRY_FILTER_KEY, null);</span>
<span class="source-line-no">151</span><span id="line-151"> WALEntrySinkFilter filter = null;</span>
<span class="source-line-no">152</span><span id="line-152"> try {</span>
<span class="source-line-no">153</span><span id="line-153"> filter = walEntryFilterClass == null</span>
<span class="source-line-no">154</span><span id="line-154"> ? null</span>
<span class="source-line-no">155</span><span id="line-155"> : (WALEntrySinkFilter) walEntryFilterClass.getDeclaredConstructor().newInstance();</span>
<span class="source-line-no">156</span><span id="line-156"> } catch (Exception e) {</span>
<span class="source-line-no">157</span><span id="line-157"> LOG.warn("Failed to instantiate " + walEntryFilterClass);</span>
<span class="source-line-no">158</span><span id="line-158"> }</span>
<span class="source-line-no">159</span><span id="line-159"> if (filter != null) {</span>
<span class="source-line-no">160</span><span id="line-160"> filter.init(getConnection());</span>
<span class="source-line-no">161</span><span id="line-161"> }</span>
<span class="source-line-no">162</span><span id="line-162"> return filter;</span>
<span class="source-line-no">163</span><span id="line-163"> }</span>
<span class="source-line-no">164</span><span id="line-164"></span>
<span class="source-line-no">165</span><span id="line-165"> /**</span>
<span class="source-line-no">166</span><span id="line-166"> * decorate the Configuration object to make replication more receptive to delays: lessen the</span>
<span class="source-line-no">167</span><span id="line-167"> * timeout and numTries.</span>
<span class="source-line-no">168</span><span id="line-168"> */</span>
<span class="source-line-no">169</span><span id="line-169"> private void decorateConf() {</span>
<span class="source-line-no">170</span><span id="line-170"> this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,</span>
<span class="source-line-no">171</span><span id="line-171"> this.conf.getInt("replication.sink.client.retries.number", 4));</span>
<span class="source-line-no">172</span><span id="line-172"> this.conf.setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,</span>
<span class="source-line-no">173</span><span id="line-173"> this.conf.getInt("replication.sink.client.ops.timeout", 10000));</span>
<span class="source-line-no">174</span><span id="line-174"> String replicationCodec = this.conf.get(HConstants.REPLICATION_CODEC_CONF_KEY);</span>
<span class="source-line-no">175</span><span id="line-175"> if (StringUtils.isNotEmpty(replicationCodec)) {</span>
<span class="source-line-no">176</span><span id="line-176"> this.conf.set(HConstants.RPC_CODEC_CONF_KEY, replicationCodec);</span>
<span class="source-line-no">177</span><span id="line-177"> }</span>
<span class="source-line-no">178</span><span id="line-178"> // use server ZK cluster for replication, so we unset the client ZK related properties if any</span>
<span class="source-line-no">179</span><span id="line-179"> if (this.conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM) != null) {</span>
<span class="source-line-no">180</span><span id="line-180"> this.conf.unset(HConstants.CLIENT_ZOOKEEPER_QUORUM);</span>
<span class="source-line-no">181</span><span id="line-181"> }</span>
<span class="source-line-no">182</span><span id="line-182"> }</span>
<span class="source-line-no">183</span><span id="line-183"></span>
<span class="source-line-no">184</span><span id="line-184"> /**</span>
<span class="source-line-no">185</span><span id="line-185"> * Replicate this array of entries directly into the local cluster using the native client. Only</span>
<span class="source-line-no">186</span><span id="line-186"> * operates against raw protobuf type saving on a conversion from pb to pojo.</span>
<span class="source-line-no">187</span><span id="line-187"> * @param entries WAL entries to be replicated.</span>
<span class="source-line-no">188</span><span id="line-188"> * @param cells cell scanner for iteration.</span>
<span class="source-line-no">189</span><span id="line-189"> * @param replicationClusterId Id which will uniquely identify source cluster FS client</span>
<span class="source-line-no">190</span><span id="line-190"> * configurations in the replication configuration directory</span>
<span class="source-line-no">191</span><span id="line-191"> * @param sourceBaseNamespaceDirPath Path that point to the source cluster base namespace</span>
<span class="source-line-no">192</span><span id="line-192"> * directory</span>
<span class="source-line-no">193</span><span id="line-193"> * @param sourceHFileArchiveDirPath Path that point to the source cluster hfile archive directory</span>
<span class="source-line-no">194</span><span id="line-194"> * @throws IOException If failed to replicate the data</span>
<span class="source-line-no">195</span><span id="line-195"> */</span>
<span class="source-line-no">196</span><span id="line-196"> public void replicateEntries(List&lt;WALEntry&gt; entries, final ExtendedCellScanner cells,</span>
<span class="source-line-no">197</span><span id="line-197"> String replicationClusterId, String sourceBaseNamespaceDirPath,</span>
<span class="source-line-no">198</span><span id="line-198"> String sourceHFileArchiveDirPath) throws IOException {</span>
<span class="source-line-no">199</span><span id="line-199"> if (entries.isEmpty()) {</span>
<span class="source-line-no">200</span><span id="line-200"> return;</span>
<span class="source-line-no">201</span><span id="line-201"> }</span>
<span class="source-line-no">202</span><span id="line-202"> // Very simple optimization where we batch sequences of rows going</span>
<span class="source-line-no">203</span><span id="line-203"> // to the same table.</span>
<span class="source-line-no">204</span><span id="line-204"> try {</span>
<span class="source-line-no">205</span><span id="line-205"> long totalReplicated = 0;</span>
<span class="source-line-no">206</span><span id="line-206"> // Map of table =&gt; list of Rows, grouped by cluster id, we only want to flushCommits once per</span>
<span class="source-line-no">207</span><span id="line-207"> // invocation of this method per table and cluster id.</span>
<span class="source-line-no">208</span><span id="line-208"> Map&lt;TableName, Map&lt;List&lt;UUID&gt;, List&lt;Row&gt;&gt;&gt; rowMap = new TreeMap&lt;&gt;();</span>
<span class="source-line-no">209</span><span id="line-209"></span>
<span class="source-line-no">210</span><span id="line-210"> Map&lt;List&lt;String&gt;, Map&lt;String, List&lt;Pair&lt;byte[], List&lt;String&gt;&gt;&gt;&gt;&gt; bulkLoadsPerClusters = null;</span>
<span class="source-line-no">211</span><span id="line-211"> Pair&lt;List&lt;Mutation&gt;, List&lt;WALEntry&gt;&gt; mutationsToWalEntriesPairs =</span>
<span class="source-line-no">212</span><span id="line-212"> new Pair&lt;&gt;(new ArrayList&lt;&gt;(), new ArrayList&lt;&gt;());</span>
<span class="source-line-no">213</span><span id="line-213"> for (WALEntry entry : entries) {</span>
<span class="source-line-no">214</span><span id="line-214"> TableName table = TableName.valueOf(entry.getKey().getTableName().toByteArray());</span>
<span class="source-line-no">215</span><span id="line-215"> if (this.walEntrySinkFilter != null) {</span>
<span class="source-line-no">216</span><span id="line-216"> if (this.walEntrySinkFilter.filter(table, entry.getKey().getWriteTime())) {</span>
<span class="source-line-no">217</span><span id="line-217"> // Skip Cells in CellScanner associated with this entry.</span>
<span class="source-line-no">218</span><span id="line-218"> int count = entry.getAssociatedCellCount();</span>
<span class="source-line-no">219</span><span id="line-219"> for (int i = 0; i &lt; count; i++) {</span>
<span class="source-line-no">220</span><span id="line-220"> // Throw index out of bounds if our cell count is off</span>
<span class="source-line-no">221</span><span id="line-221"> if (!cells.advance()) {</span>
<span class="source-line-no">222</span><span id="line-222"> this.metrics.incrementFailedBatches();</span>
<span class="source-line-no">223</span><span id="line-223"> throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);</span>
<span class="source-line-no">224</span><span id="line-224"> }</span>
<span class="source-line-no">225</span><span id="line-225"> }</span>
<span class="source-line-no">226</span><span id="line-226"> continue;</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"> ExtendedCell previousCell = null;</span>
<span class="source-line-no">230</span><span id="line-230"> Mutation mutation = null;</span>
<span class="source-line-no">231</span><span id="line-231"> int count = entry.getAssociatedCellCount();</span>
<span class="source-line-no">232</span><span id="line-232"> for (int i = 0; i &lt; count; i++) {</span>
<span class="source-line-no">233</span><span id="line-233"> // Throw index out of bounds if our cell count is off</span>
<span class="source-line-no">234</span><span id="line-234"> if (!cells.advance()) {</span>
<span class="source-line-no">235</span><span id="line-235"> this.metrics.incrementFailedBatches();</span>
<span class="source-line-no">236</span><span id="line-236"> throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);</span>
<span class="source-line-no">237</span><span id="line-237"> }</span>
<span class="source-line-no">238</span><span id="line-238"> ExtendedCell cell = cells.current();</span>
<span class="source-line-no">239</span><span id="line-239"> // Handle bulk load hfiles replication</span>
<span class="source-line-no">240</span><span id="line-240"> if (CellUtil.matchingQualifier(cell, WALEdit.BULK_LOAD)) {</span>
<span class="source-line-no">241</span><span id="line-241"> BulkLoadDescriptor bld = WALEdit.getBulkLoadDescriptor(cell);</span>
<span class="source-line-no">242</span><span id="line-242"> if (bld.getReplicate()) {</span>
<span class="source-line-no">243</span><span id="line-243"> if (bulkLoadsPerClusters == null) {</span>
<span class="source-line-no">244</span><span id="line-244"> bulkLoadsPerClusters = new HashMap&lt;&gt;();</span>
<span class="source-line-no">245</span><span id="line-245"> }</span>
<span class="source-line-no">246</span><span id="line-246"> // Map of table name Vs list of pair of family and list of</span>
<span class="source-line-no">247</span><span id="line-247"> // hfile paths from its namespace</span>
<span class="source-line-no">248</span><span id="line-248"> Map&lt;String, List&lt;Pair&lt;byte[], List&lt;String&gt;&gt;&gt;&gt; bulkLoadHFileMap =</span>
<span class="source-line-no">249</span><span id="line-249"> bulkLoadsPerClusters.computeIfAbsent(bld.getClusterIdsList(), k -&gt; new HashMap&lt;&gt;());</span>
<span class="source-line-no">250</span><span id="line-250"> buildBulkLoadHFileMap(bulkLoadHFileMap, table, bld);</span>
<span class="source-line-no">251</span><span id="line-251"> }</span>
<span class="source-line-no">252</span><span id="line-252"> } else if (CellUtil.matchingQualifier(cell, WALEdit.REPLICATION_MARKER)) {</span>
<span class="source-line-no">253</span><span id="line-253"> Mutation put = processReplicationMarkerEntry(cell);</span>
<span class="source-line-no">254</span><span id="line-254"> if (put == null) {</span>
<span class="source-line-no">255</span><span id="line-255"> continue;</span>
<span class="source-line-no">256</span><span id="line-256"> }</span>
<span class="source-line-no">257</span><span id="line-257"> table = REPLICATION_SINK_TRACKER_TABLE_NAME;</span>
<span class="source-line-no">258</span><span id="line-258"> List&lt;UUID&gt; clusterIds = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">259</span><span id="line-259"> for (HBaseProtos.UUID clusterId : entry.getKey().getClusterIdsList()) {</span>
<span class="source-line-no">260</span><span id="line-260"> clusterIds.add(toUUID(clusterId));</span>
<span class="source-line-no">261</span><span id="line-261"> }</span>
<span class="source-line-no">262</span><span id="line-262"> put.setClusterIds(clusterIds);</span>
<span class="source-line-no">263</span><span id="line-263"> addToHashMultiMap(rowMap, table, clusterIds, put);</span>
<span class="source-line-no">264</span><span id="line-264"> } else {</span>
<span class="source-line-no">265</span><span id="line-265"> // Handle wal replication</span>
<span class="source-line-no">266</span><span id="line-266"> if (isNewRowOrType(previousCell, cell)) {</span>
<span class="source-line-no">267</span><span id="line-267"> // Create new mutation</span>
<span class="source-line-no">268</span><span id="line-268"> mutation = CellUtil.isDelete(cell)</span>
<span class="source-line-no">269</span><span id="line-269"> ? new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())</span>
<span class="source-line-no">270</span><span id="line-270"> : new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());</span>
<span class="source-line-no">271</span><span id="line-271"> List&lt;UUID&gt; clusterIds = new ArrayList&lt;&gt;(entry.getKey().getClusterIdsList().size());</span>
<span class="source-line-no">272</span><span id="line-272"> for (HBaseProtos.UUID clusterId : entry.getKey().getClusterIdsList()) {</span>
<span class="source-line-no">273</span><span id="line-273"> clusterIds.add(toUUID(clusterId));</span>
<span class="source-line-no">274</span><span id="line-274"> }</span>
<span class="source-line-no">275</span><span id="line-275"> mutation.setClusterIds(clusterIds);</span>
<span class="source-line-no">276</span><span id="line-276"> mutation.setAttribute(ReplicationUtils.REPLICATION_ATTR_NAME,</span>
<span class="source-line-no">277</span><span id="line-277"> HConstants.EMPTY_BYTE_ARRAY);</span>
<span class="source-line-no">278</span><span id="line-278"> if (rsServerHost != null) {</span>
<span class="source-line-no">279</span><span id="line-279"> rsServerHost.preReplicationSinkBatchMutate(entry, mutation);</span>
<span class="source-line-no">280</span><span id="line-280"> mutationsToWalEntriesPairs.getFirst().add(mutation);</span>
<span class="source-line-no">281</span><span id="line-281"> mutationsToWalEntriesPairs.getSecond().add(entry);</span>
<span class="source-line-no">282</span><span id="line-282"> }</span>
<span class="source-line-no">283</span><span id="line-283"> addToHashMultiMap(rowMap, table, clusterIds, mutation);</span>
<span class="source-line-no">284</span><span id="line-284"> }</span>
<span class="source-line-no">285</span><span id="line-285"> if (CellUtil.isDelete(cell)) {</span>
<span class="source-line-no">286</span><span id="line-286"> ((Delete) mutation).add(cell);</span>
<span class="source-line-no">287</span><span id="line-287"> } else {</span>
<span class="source-line-no">288</span><span id="line-288"> ((Put) mutation).add(cell);</span>
<span class="source-line-no">289</span><span id="line-289"> }</span>
<span class="source-line-no">290</span><span id="line-290"> previousCell = cell;</span>
<span class="source-line-no">291</span><span id="line-291"> }</span>
<span class="source-line-no">292</span><span id="line-292"> }</span>
<span class="source-line-no">293</span><span id="line-293"> totalReplicated++;</span>
<span class="source-line-no">294</span><span id="line-294"> }</span>
<span class="source-line-no">295</span><span id="line-295"></span>
<span class="source-line-no">296</span><span id="line-296"> // TODO Replicating mutations and bulk loaded data can be made parallel</span>
<span class="source-line-no">297</span><span id="line-297"> if (!rowMap.isEmpty()) {</span>
<span class="source-line-no">298</span><span id="line-298"> LOG.debug("Started replicating mutations.");</span>
<span class="source-line-no">299</span><span id="line-299"> for (Entry&lt;TableName, Map&lt;List&lt;UUID&gt;, List&lt;Row&gt;&gt;&gt; entry : rowMap.entrySet()) {</span>
<span class="source-line-no">300</span><span id="line-300"> batch(entry.getKey(), entry.getValue().values(), rowSizeWarnThreshold);</span>
<span class="source-line-no">301</span><span id="line-301"> }</span>
<span class="source-line-no">302</span><span id="line-302"> LOG.debug("Finished replicating mutations.");</span>
<span class="source-line-no">303</span><span id="line-303"> }</span>
<span class="source-line-no">304</span><span id="line-304"></span>
<span class="source-line-no">305</span><span id="line-305"> if (rsServerHost != null) {</span>
<span class="source-line-no">306</span><span id="line-306"> List&lt;Mutation&gt; mutations = mutationsToWalEntriesPairs.getFirst();</span>
<span class="source-line-no">307</span><span id="line-307"> List&lt;WALEntry&gt; walEntries = mutationsToWalEntriesPairs.getSecond();</span>
<span class="source-line-no">308</span><span id="line-308"> for (int i = 0; i &lt; mutations.size(); i++) {</span>
<span class="source-line-no">309</span><span id="line-309"> rsServerHost.postReplicationSinkBatchMutate(walEntries.get(i), mutations.get(i));</span>
<span class="source-line-no">310</span><span id="line-310"> }</span>
<span class="source-line-no">311</span><span id="line-311"> }</span>
<span class="source-line-no">312</span><span id="line-312"></span>
<span class="source-line-no">313</span><span id="line-313"> if (bulkLoadsPerClusters != null) {</span>
<span class="source-line-no">314</span><span id="line-314"> for (Entry&lt;List&lt;String&gt;,</span>
<span class="source-line-no">315</span><span id="line-315"> Map&lt;String, List&lt;Pair&lt;byte[], List&lt;String&gt;&gt;&gt;&gt;&gt; entry : bulkLoadsPerClusters.entrySet()) {</span>
<span class="source-line-no">316</span><span id="line-316"> Map&lt;String, List&lt;Pair&lt;byte[], List&lt;String&gt;&gt;&gt;&gt; bulkLoadHFileMap = entry.getValue();</span>
<span class="source-line-no">317</span><span id="line-317"> if (bulkLoadHFileMap != null &amp;&amp; !bulkLoadHFileMap.isEmpty()) {</span>
<span class="source-line-no">318</span><span id="line-318"> LOG.debug("Replicating {} bulk loaded data", entry.getKey().toString());</span>
<span class="source-line-no">319</span><span id="line-319"> Configuration providerConf = this.provider.getConf(this.conf, replicationClusterId);</span>
<span class="source-line-no">320</span><span id="line-320"> try (HFileReplicator hFileReplicator = new HFileReplicator(providerConf,</span>
<span class="source-line-no">321</span><span id="line-321"> sourceBaseNamespaceDirPath, sourceHFileArchiveDirPath, bulkLoadHFileMap, conf,</span>
<span class="source-line-no">322</span><span id="line-322"> getConnection(), entry.getKey())) {</span>
<span class="source-line-no">323</span><span id="line-323"> hFileReplicator.replicate();</span>
<span class="source-line-no">324</span><span id="line-324"> LOG.debug("Finished replicating {} bulk loaded data", entry.getKey().toString());</span>
<span class="source-line-no">325</span><span id="line-325"> }</span>
<span class="source-line-no">326</span><span id="line-326"> }</span>
<span class="source-line-no">327</span><span id="line-327"> }</span>
<span class="source-line-no">328</span><span id="line-328"> }</span>
<span class="source-line-no">329</span><span id="line-329"></span>
<span class="source-line-no">330</span><span id="line-330"> int size = entries.size();</span>
<span class="source-line-no">331</span><span id="line-331"> this.metrics.setAgeOfLastAppliedOp(entries.get(size - 1).getKey().getWriteTime());</span>
<span class="source-line-no">332</span><span id="line-332"> this.metrics.applyBatch(size + hfilesReplicated, hfilesReplicated);</span>
<span class="source-line-no">333</span><span id="line-333"> this.totalReplicatedEdits.addAndGet(totalReplicated);</span>
<span class="source-line-no">334</span><span id="line-334"> } catch (IOException ex) {</span>
<span class="source-line-no">335</span><span id="line-335"> LOG.error("Unable to accept edit because:", ex);</span>
<span class="source-line-no">336</span><span id="line-336"> this.metrics.incrementFailedBatches();</span>
<span class="source-line-no">337</span><span id="line-337"> throw ex;</span>
<span class="source-line-no">338</span><span id="line-338"> }</span>
<span class="source-line-no">339</span><span id="line-339"> }</span>
<span class="source-line-no">340</span><span id="line-340"></span>
<span class="source-line-no">341</span><span id="line-341"> /*</span>
<span class="source-line-no">342</span><span id="line-342"> * First check if config key hbase.regionserver.replication.sink.tracker.enabled is true or not.</span>
<span class="source-line-no">343</span><span id="line-343"> * If false, then ignore this cell. If set to true, de-serialize value into</span>
<span class="source-line-no">344</span><span id="line-344"> * ReplicationTrackerDescriptor. Create a Put mutation with regionserver name, walname, offset and</span>
<span class="source-line-no">345</span><span id="line-345"> * timestamp from ReplicationMarkerDescriptor.</span>
<span class="source-line-no">346</span><span id="line-346"> */</span>
<span class="source-line-no">347</span><span id="line-347"> private Put processReplicationMarkerEntry(Cell cell) throws IOException {</span>
<span class="source-line-no">348</span><span id="line-348"> // If source is emitting replication marker rows but sink is not accepting them,</span>
<span class="source-line-no">349</span><span id="line-349"> // ignore the edits.</span>
<span class="source-line-no">350</span><span id="line-350"> if (!replicationSinkTrackerEnabled) {</span>
<span class="source-line-no">351</span><span id="line-351"> return null;</span>
<span class="source-line-no">352</span><span id="line-352"> }</span>
<span class="source-line-no">353</span><span id="line-353"> WALProtos.ReplicationMarkerDescriptor descriptor =</span>
<span class="source-line-no">354</span><span id="line-354"> WALProtos.ReplicationMarkerDescriptor.parseFrom(new ByteArrayInputStream(cell.getValueArray(),</span>
<span class="source-line-no">355</span><span id="line-355"> cell.getValueOffset(), cell.getValueLength()));</span>
<span class="source-line-no">356</span><span id="line-356"> Put put = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());</span>
<span class="source-line-no">357</span><span id="line-357"> put.addColumn(REPLICATION_SINK_TRACKER_INFO_FAMILY, RS_COLUMN, cell.getTimestamp(),</span>
<span class="source-line-no">358</span><span id="line-358"> (Bytes.toBytes(descriptor.getRegionServerName())));</span>
<span class="source-line-no">359</span><span id="line-359"> put.addColumn(REPLICATION_SINK_TRACKER_INFO_FAMILY, WAL_NAME_COLUMN, cell.getTimestamp(),</span>
<span class="source-line-no">360</span><span id="line-360"> Bytes.toBytes(descriptor.getWalName()));</span>
<span class="source-line-no">361</span><span id="line-361"> put.addColumn(REPLICATION_SINK_TRACKER_INFO_FAMILY, TIMESTAMP_COLUMN, cell.getTimestamp(),</span>
<span class="source-line-no">362</span><span id="line-362"> Bytes.toBytes(cell.getTimestamp()));</span>
<span class="source-line-no">363</span><span id="line-363"> put.addColumn(REPLICATION_SINK_TRACKER_INFO_FAMILY, OFFSET_COLUMN, cell.getTimestamp(),</span>
<span class="source-line-no">364</span><span id="line-364"> Bytes.toBytes(descriptor.getOffset()));</span>
<span class="source-line-no">365</span><span id="line-365"> return put;</span>
<span class="source-line-no">366</span><span id="line-366"> }</span>
<span class="source-line-no">367</span><span id="line-367"></span>
<span class="source-line-no">368</span><span id="line-368"> private void buildBulkLoadHFileMap(</span>
<span class="source-line-no">369</span><span id="line-369"> final Map&lt;String, List&lt;Pair&lt;byte[], List&lt;String&gt;&gt;&gt;&gt; bulkLoadHFileMap, TableName table,</span>
<span class="source-line-no">370</span><span id="line-370"> BulkLoadDescriptor bld) throws IOException {</span>
<span class="source-line-no">371</span><span id="line-371"> List&lt;StoreDescriptor&gt; storesList = bld.getStoresList();</span>
<span class="source-line-no">372</span><span id="line-372"> int storesSize = storesList.size();</span>
<span class="source-line-no">373</span><span id="line-373"> for (int j = 0; j &lt; storesSize; j++) {</span>
<span class="source-line-no">374</span><span id="line-374"> StoreDescriptor storeDescriptor = storesList.get(j);</span>
<span class="source-line-no">375</span><span id="line-375"> List&lt;String&gt; storeFileList = storeDescriptor.getStoreFileList();</span>
<span class="source-line-no">376</span><span id="line-376"> int storeFilesSize = storeFileList.size();</span>
<span class="source-line-no">377</span><span id="line-377"> hfilesReplicated += storeFilesSize;</span>
<span class="source-line-no">378</span><span id="line-378"> for (int k = 0; k &lt; storeFilesSize; k++) {</span>
<span class="source-line-no">379</span><span id="line-379"> byte[] family = storeDescriptor.getFamilyName().toByteArray();</span>
<span class="source-line-no">380</span><span id="line-380"></span>
<span class="source-line-no">381</span><span id="line-381"> // Build hfile relative path from its namespace</span>
<span class="source-line-no">382</span><span id="line-382"> String pathToHfileFromNS = getHFilePath(table, bld, storeFileList.get(k), family);</span>
<span class="source-line-no">383</span><span id="line-383"> String tableName = table.getNameWithNamespaceInclAsString();</span>
<span class="source-line-no">384</span><span id="line-384"> List&lt;Pair&lt;byte[], List&lt;String&gt;&gt;&gt; familyHFilePathsList = bulkLoadHFileMap.get(tableName);</span>
<span class="source-line-no">385</span><span id="line-385"> if (familyHFilePathsList != null) {</span>
<span class="source-line-no">386</span><span id="line-386"> boolean foundFamily = false;</span>
<span class="source-line-no">387</span><span id="line-387"> for (Pair&lt;byte[], List&lt;String&gt;&gt; familyHFilePathsPair : familyHFilePathsList) {</span>
<span class="source-line-no">388</span><span id="line-388"> if (Bytes.equals(familyHFilePathsPair.getFirst(), family)) {</span>
<span class="source-line-no">389</span><span id="line-389"> // Found family already present, just add the path to the existing list</span>
<span class="source-line-no">390</span><span id="line-390"> familyHFilePathsPair.getSecond().add(pathToHfileFromNS);</span>
<span class="source-line-no">391</span><span id="line-391"> foundFamily = true;</span>
<span class="source-line-no">392</span><span id="line-392"> break;</span>
<span class="source-line-no">393</span><span id="line-393"> }</span>
<span class="source-line-no">394</span><span id="line-394"> }</span>
<span class="source-line-no">395</span><span id="line-395"> if (!foundFamily) {</span>
<span class="source-line-no">396</span><span id="line-396"> // Family not found, add this family and its hfile paths pair to the list</span>
<span class="source-line-no">397</span><span id="line-397"> addFamilyAndItsHFilePathToTableInMap(family, pathToHfileFromNS, familyHFilePathsList);</span>
<span class="source-line-no">398</span><span id="line-398"> }</span>
<span class="source-line-no">399</span><span id="line-399"> } else {</span>
<span class="source-line-no">400</span><span id="line-400"> // Add this table entry into the map</span>
<span class="source-line-no">401</span><span id="line-401"> addNewTableEntryInMap(bulkLoadHFileMap, family, pathToHfileFromNS, tableName);</span>
<span class="source-line-no">402</span><span id="line-402"> }</span>
<span class="source-line-no">403</span><span id="line-403"> }</span>
<span class="source-line-no">404</span><span id="line-404"> }</span>
<span class="source-line-no">405</span><span id="line-405"> }</span>
<span class="source-line-no">406</span><span id="line-406"></span>
<span class="source-line-no">407</span><span id="line-407"> private void addFamilyAndItsHFilePathToTableInMap(byte[] family, String pathToHfileFromNS,</span>
<span class="source-line-no">408</span><span id="line-408"> List&lt;Pair&lt;byte[], List&lt;String&gt;&gt;&gt; familyHFilePathsList) {</span>
<span class="source-line-no">409</span><span id="line-409"> List&lt;String&gt; hfilePaths = new ArrayList&lt;&gt;(1);</span>
<span class="source-line-no">410</span><span id="line-410"> hfilePaths.add(pathToHfileFromNS);</span>
<span class="source-line-no">411</span><span id="line-411"> familyHFilePathsList.add(new Pair&lt;&gt;(family, hfilePaths));</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"> private void addNewTableEntryInMap(</span>
<span class="source-line-no">415</span><span id="line-415"> final Map&lt;String, List&lt;Pair&lt;byte[], List&lt;String&gt;&gt;&gt;&gt; bulkLoadHFileMap, byte[] family,</span>
<span class="source-line-no">416</span><span id="line-416"> String pathToHfileFromNS, String tableName) {</span>
<span class="source-line-no">417</span><span id="line-417"> List&lt;String&gt; hfilePaths = new ArrayList&lt;&gt;(1);</span>
<span class="source-line-no">418</span><span id="line-418"> hfilePaths.add(pathToHfileFromNS);</span>
<span class="source-line-no">419</span><span id="line-419"> Pair&lt;byte[], List&lt;String&gt;&gt; newFamilyHFilePathsPair = new Pair&lt;&gt;(family, hfilePaths);</span>
<span class="source-line-no">420</span><span id="line-420"> List&lt;Pair&lt;byte[], List&lt;String&gt;&gt;&gt; newFamilyHFilePathsList = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">421</span><span id="line-421"> newFamilyHFilePathsList.add(newFamilyHFilePathsPair);</span>
<span class="source-line-no">422</span><span id="line-422"> bulkLoadHFileMap.put(tableName, newFamilyHFilePathsList);</span>
<span class="source-line-no">423</span><span id="line-423"> }</span>
<span class="source-line-no">424</span><span id="line-424"></span>
<span class="source-line-no">425</span><span id="line-425"> private String getHFilePath(TableName table, BulkLoadDescriptor bld, String storeFile,</span>
<span class="source-line-no">426</span><span id="line-426"> byte[] family) {</span>
<span class="source-line-no">427</span><span id="line-427"> return new StringBuilder(100).append(table.getNamespaceAsString()).append(Path.SEPARATOR)</span>
<span class="source-line-no">428</span><span id="line-428"> .append(table.getQualifierAsString()).append(Path.SEPARATOR)</span>
<span class="source-line-no">429</span><span id="line-429"> .append(Bytes.toString(bld.getEncodedRegionName().toByteArray())).append(Path.SEPARATOR)</span>
<span class="source-line-no">430</span><span id="line-430"> .append(Bytes.toString(family)).append(Path.SEPARATOR).append(storeFile).toString();</span>
<span class="source-line-no">431</span><span id="line-431"> }</span>
<span class="source-line-no">432</span><span id="line-432"></span>
<span class="source-line-no">433</span><span id="line-433"> /** Returns True if we have crossed over onto a new row or type */</span>
<span class="source-line-no">434</span><span id="line-434"> private boolean isNewRowOrType(final ExtendedCell previousCell, final ExtendedCell cell) {</span>
<span class="source-line-no">435</span><span id="line-435"> return previousCell == null || previousCell.getTypeByte() != cell.getTypeByte()</span>
<span class="source-line-no">436</span><span id="line-436"> || !CellUtil.matchingRows(previousCell, cell);</span>
<span class="source-line-no">437</span><span id="line-437"> }</span>
<span class="source-line-no">438</span><span id="line-438"></span>
<span class="source-line-no">439</span><span id="line-439"> private java.util.UUID toUUID(final HBaseProtos.UUID uuid) {</span>
<span class="source-line-no">440</span><span id="line-440"> return new java.util.UUID(uuid.getMostSigBits(), uuid.getLeastSigBits());</span>
<span class="source-line-no">441</span><span id="line-441"> }</span>
<span class="source-line-no">442</span><span id="line-442"></span>
<span class="source-line-no">443</span><span id="line-443"> /**</span>
<span class="source-line-no">444</span><span id="line-444"> * Simple helper to a map from key to (a list of) values TODO: Make a general utility method</span>
<span class="source-line-no">445</span><span id="line-445"> * @return the list of values corresponding to key1 and key2</span>
<span class="source-line-no">446</span><span id="line-446"> */</span>
<span class="source-line-no">447</span><span id="line-447"> private &lt;K1, K2, V&gt; List&lt;V&gt; addToHashMultiMap(Map&lt;K1, Map&lt;K2, List&lt;V&gt;&gt;&gt; map, K1 key1, K2 key2,</span>
<span class="source-line-no">448</span><span id="line-448"> V value) {</span>
<span class="source-line-no">449</span><span id="line-449"> Map&lt;K2, List&lt;V&gt;&gt; innerMap = map.computeIfAbsent(key1, k -&gt; new HashMap&lt;&gt;());</span>
<span class="source-line-no">450</span><span id="line-450"> List&lt;V&gt; values = innerMap.computeIfAbsent(key2, k -&gt; new ArrayList&lt;&gt;());</span>
<span class="source-line-no">451</span><span id="line-451"> values.add(value);</span>
<span class="source-line-no">452</span><span id="line-452"> return values;</span>
<span class="source-line-no">453</span><span id="line-453"> }</span>
<span class="source-line-no">454</span><span id="line-454"></span>
<span class="source-line-no">455</span><span id="line-455"> /**</span>
<span class="source-line-no">456</span><span id="line-456"> * stop the thread pool executor. It is called when the regionserver is stopped.</span>
<span class="source-line-no">457</span><span id="line-457"> */</span>
<span class="source-line-no">458</span><span id="line-458"> public void stopReplicationSinkServices() {</span>
<span class="source-line-no">459</span><span id="line-459"> try {</span>
<span class="source-line-no">460</span><span id="line-460"> if (this.sharedConn != null) {</span>
<span class="source-line-no">461</span><span id="line-461"> synchronized (sharedConnLock) {</span>
<span class="source-line-no">462</span><span id="line-462"> if (this.sharedConn != null) {</span>
<span class="source-line-no">463</span><span id="line-463"> this.sharedConn.close();</span>
<span class="source-line-no">464</span><span id="line-464"> this.sharedConn = null;</span>
<span class="source-line-no">465</span><span id="line-465"> }</span>
<span class="source-line-no">466</span><span id="line-466"> }</span>
<span class="source-line-no">467</span><span id="line-467"> }</span>
<span class="source-line-no">468</span><span id="line-468"> } catch (IOException e) {</span>
<span class="source-line-no">469</span><span id="line-469"> LOG.warn("IOException while closing the connection", e); // ignoring as we are closing.</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"></span>
<span class="source-line-no">473</span><span id="line-473"> /**</span>
<span class="source-line-no">474</span><span id="line-474"> * Do the changes and handle the pool</span>
<span class="source-line-no">475</span><span id="line-475"> * @param tableName table to insert into</span>
<span class="source-line-no">476</span><span id="line-476"> * @param allRows list of actions</span>
<span class="source-line-no">477</span><span id="line-477"> * @param batchRowSizeThreshold rowSize threshold for batch mutation</span>
<span class="source-line-no">478</span><span id="line-478"> */</span>
<span class="source-line-no">479</span><span id="line-479"> private void batch(TableName tableName, Collection&lt;List&lt;Row&gt;&gt; allRows, int batchRowSizeThreshold)</span>
<span class="source-line-no">480</span><span id="line-480"> throws IOException {</span>
<span class="source-line-no">481</span><span id="line-481"> if (allRows.isEmpty()) {</span>
<span class="source-line-no">482</span><span id="line-482"> return;</span>
<span class="source-line-no">483</span><span id="line-483"> }</span>
<span class="source-line-no">484</span><span id="line-484"> AsyncTable&lt;?&gt; table = getConnection().getTable(tableName);</span>
<span class="source-line-no">485</span><span id="line-485"> List&lt;Future&lt;?&gt;&gt; futures = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">486</span><span id="line-486"> for (List&lt;Row&gt; rows : allRows) {</span>
<span class="source-line-no">487</span><span id="line-487"> List&lt;List&lt;Row&gt;&gt; batchRows;</span>
<span class="source-line-no">488</span><span id="line-488"> if (rows.size() &gt; batchRowSizeThreshold) {</span>
<span class="source-line-no">489</span><span id="line-489"> batchRows = Lists.partition(rows, batchRowSizeThreshold);</span>
<span class="source-line-no">490</span><span id="line-490"> } else {</span>
<span class="source-line-no">491</span><span id="line-491"> batchRows = Collections.singletonList(rows);</span>
<span class="source-line-no">492</span><span id="line-492"> }</span>
<span class="source-line-no">493</span><span id="line-493"> futures.addAll(batchRows.stream().map(table::batchAll).collect(Collectors.toList()));</span>
<span class="source-line-no">494</span><span id="line-494"> }</span>
<span class="source-line-no">495</span><span id="line-495"> // Here we will always wait until all futures are finished, even if there are failures when</span>
<span class="source-line-no">496</span><span id="line-496"> // getting from a future in the middle. This is because this method may be called in a rpc call,</span>
<span class="source-line-no">497</span><span id="line-497"> // so the batch operations may reference some off heap cells(through CellScanner). If we return</span>
<span class="source-line-no">498</span><span id="line-498"> // earlier here, the rpc call may be finished and they will release the off heap cells before</span>
<span class="source-line-no">499</span><span id="line-499"> // some of the batch operations finish, and then cause corrupt data or even crash the region</span>
<span class="source-line-no">500</span><span id="line-500"> // server. See HBASE-28584 and HBASE-28850 for more details.</span>
<span class="source-line-no">501</span><span id="line-501"> IOException error = null;</span>
<span class="source-line-no">502</span><span id="line-502"> for (Future&lt;?&gt; future : futures) {</span>
<span class="source-line-no">503</span><span id="line-503"> try {</span>
<span class="source-line-no">504</span><span id="line-504"> FutureUtils.get(future);</span>
<span class="source-line-no">505</span><span id="line-505"> } catch (RetriesExhaustedException e) {</span>
<span class="source-line-no">506</span><span id="line-506"> IOException ioe;</span>
<span class="source-line-no">507</span><span id="line-507"> if (e.getCause() instanceof TableNotFoundException) {</span>
<span class="source-line-no">508</span><span id="line-508"> ioe = new TableNotFoundException("'" + tableName + "'");</span>
<span class="source-line-no">509</span><span id="line-509"> } else {</span>
<span class="source-line-no">510</span><span id="line-510"> ioe = e;</span>
<span class="source-line-no">511</span><span id="line-511"> }</span>
<span class="source-line-no">512</span><span id="line-512"> if (error == null) {</span>
<span class="source-line-no">513</span><span id="line-513"> error = ioe;</span>
<span class="source-line-no">514</span><span id="line-514"> } else {</span>
<span class="source-line-no">515</span><span id="line-515"> error.addSuppressed(ioe);</span>
<span class="source-line-no">516</span><span id="line-516"> }</span>
<span class="source-line-no">517</span><span id="line-517"> }</span>
<span class="source-line-no">518</span><span id="line-518"> }</span>
<span class="source-line-no">519</span><span id="line-519"> if (error != null) {</span>
<span class="source-line-no">520</span><span id="line-520"> throw error;</span>
<span class="source-line-no">521</span><span id="line-521"> }</span>
<span class="source-line-no">522</span><span id="line-522"> }</span>
<span class="source-line-no">523</span><span id="line-523"></span>
<span class="source-line-no">524</span><span id="line-524"> private AsyncClusterConnection getConnection() throws IOException {</span>
<span class="source-line-no">525</span><span id="line-525"> // See https://en.wikipedia.org/wiki/Double-checked_locking</span>
<span class="source-line-no">526</span><span id="line-526"> AsyncClusterConnection connection = sharedConn;</span>
<span class="source-line-no">527</span><span id="line-527"> if (connection == null) {</span>
<span class="source-line-no">528</span><span id="line-528"> synchronized (sharedConnLock) {</span>
<span class="source-line-no">529</span><span id="line-529"> connection = sharedConn;</span>
<span class="source-line-no">530</span><span id="line-530"> if (connection == null) {</span>
<span class="source-line-no">531</span><span id="line-531"> connection = ClusterConnectionFactory.createAsyncClusterConnection(conf, null,</span>
<span class="source-line-no">532</span><span id="line-532"> UserProvider.instantiate(conf).getCurrent());</span>
<span class="source-line-no">533</span><span id="line-533"> sharedConn = connection;</span>
<span class="source-line-no">534</span><span id="line-534"> }</span>
<span class="source-line-no">535</span><span id="line-535"> }</span>
<span class="source-line-no">536</span><span id="line-536"> }</span>
<span class="source-line-no">537</span><span id="line-537"> return connection;</span>
<span class="source-line-no">538</span><span id="line-538"> }</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"> * Get a string representation of this sink's metrics</span>
<span class="source-line-no">542</span><span id="line-542"> * @return string with the total replicated edits count and the date of the last edit that was</span>
<span class="source-line-no">543</span><span id="line-543"> * applied</span>
<span class="source-line-no">544</span><span id="line-544"> */</span>
<span class="source-line-no">545</span><span id="line-545"> public String getStats() {</span>
<span class="source-line-no">546</span><span id="line-546"> long total = this.totalReplicatedEdits.get();</span>
<span class="source-line-no">547</span><span id="line-547"> return total == 0</span>
<span class="source-line-no">548</span><span id="line-548"> ? ""</span>
<span class="source-line-no">549</span><span id="line-549"> : "Sink: " + "age in ms of last applied edit: " + this.metrics.refreshAgeOfLastAppliedOp()</span>
<span class="source-line-no">550</span><span id="line-550"> + ", total replicated edits: " + total;</span>
<span class="source-line-no">551</span><span id="line-551"> }</span>
<span class="source-line-no">552</span><span id="line-552"></span>
<span class="source-line-no">553</span><span id="line-553"> /**</span>
<span class="source-line-no">554</span><span id="line-554"> * Get replication Sink Metrics</span>
<span class="source-line-no">555</span><span id="line-555"> */</span>
<span class="source-line-no">556</span><span id="line-556"> public MetricsSink getSinkMetrics() {</span>
<span class="source-line-no">557</span><span id="line-557"> return this.metrics;</span>
<span class="source-line-no">558</span><span id="line-558"> }</span>
<span class="source-line-no">559</span><span id="line-559">}</span>
</pre>
</div>
</main>
</body>
</html>