blob: 780cd70a79465aa77c5a0befab5f11729948d906 [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, class: MetaTableAccessor">
<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;</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 edu.umd.cs.findbugs.annotations.NonNull;</span>
<span class="source-line-no">021</span><span id="line-21">import edu.umd.cs.findbugs.annotations.Nullable;</span>
<span class="source-line-no">022</span><span id="line-22">import java.io.Closeable;</span>
<span class="source-line-no">023</span><span id="line-23">import java.io.IOException;</span>
<span class="source-line-no">024</span><span id="line-24">import java.util.ArrayList;</span>
<span class="source-line-no">025</span><span id="line-25">import java.util.Collections;</span>
<span class="source-line-no">026</span><span id="line-26">import java.util.LinkedHashMap;</span>
<span class="source-line-no">027</span><span id="line-27">import java.util.List;</span>
<span class="source-line-no">028</span><span id="line-28">import java.util.Map;</span>
<span class="source-line-no">029</span><span id="line-29">import java.util.Objects;</span>
<span class="source-line-no">030</span><span id="line-30">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">031</span><span id="line-31">import org.apache.hadoop.hbase.Cell.Type;</span>
<span class="source-line-no">032</span><span id="line-32">import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;</span>
<span class="source-line-no">033</span><span id="line-33">import org.apache.hadoop.hbase.client.Connection;</span>
<span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.hbase.client.Consistency;</span>
<span class="source-line-no">035</span><span id="line-35">import org.apache.hadoop.hbase.client.Delete;</span>
<span class="source-line-no">036</span><span id="line-36">import org.apache.hadoop.hbase.client.Get;</span>
<span class="source-line-no">037</span><span id="line-37">import org.apache.hadoop.hbase.client.Mutation;</span>
<span class="source-line-no">038</span><span id="line-38">import org.apache.hadoop.hbase.client.Put;</span>
<span class="source-line-no">039</span><span id="line-39">import org.apache.hadoop.hbase.client.RegionInfo;</span>
<span class="source-line-no">040</span><span id="line-40">import org.apache.hadoop.hbase.client.RegionReplicaUtil;</span>
<span class="source-line-no">041</span><span id="line-41">import org.apache.hadoop.hbase.client.Result;</span>
<span class="source-line-no">042</span><span id="line-42">import org.apache.hadoop.hbase.client.ResultScanner;</span>
<span class="source-line-no">043</span><span id="line-43">import org.apache.hadoop.hbase.client.Scan;</span>
<span class="source-line-no">044</span><span id="line-44">import org.apache.hadoop.hbase.client.Table;</span>
<span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.hbase.client.TableState;</span>
<span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.hbase.filter.Filter;</span>
<span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.hbase.filter.RowFilter;</span>
<span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.hbase.filter.SubstringComparator;</span>
<span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.hbase.master.RegionState;</span>
<span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span>
<span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.util.ExceptionUtil;</span>
<span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.util.Pair;</span>
<span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.util.PairOfSameType;</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">import org.slf4j.Logger;</span>
<span class="source-line-no">057</span><span id="line-57">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">058</span><span id="line-58"></span>
<span class="source-line-no">059</span><span id="line-59">/**</span>
<span class="source-line-no">060</span><span id="line-60"> * Read/write operations on &lt;code&gt;hbase:meta&lt;/code&gt; region as well as assignment information stored</span>
<span class="source-line-no">061</span><span id="line-61"> * to &lt;code&gt;hbase:meta&lt;/code&gt;.</span>
<span class="source-line-no">062</span><span id="line-62"> * &lt;p/&gt;</span>
<span class="source-line-no">063</span><span id="line-63"> * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is</span>
<span class="source-line-no">064</span><span id="line-64"> * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection</span>
<span class="source-line-no">065</span><span id="line-65"> * (opened before each operation, closed right after), while when used on HM or HRS (like in</span>
<span class="source-line-no">066</span><span id="line-66"> * AssignmentManager) we want permanent connection.</span>
<span class="source-line-no">067</span><span id="line-67"> * &lt;p/&gt;</span>
<span class="source-line-no">068</span><span id="line-68"> * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table</span>
<span class="source-line-no">069</span><span id="line-69"> * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is</span>
<span class="source-line-no">070</span><span id="line-70"> * called default replica.</span>
<span class="source-line-no">071</span><span id="line-71"> * &lt;p/&gt;</span>
<span class="source-line-no">072</span><span id="line-72"> * &lt;h2&gt;Meta layout&lt;/h2&gt; For each table there is single row named for the table with a 'table' column</span>
<span class="source-line-no">073</span><span id="line-73"> * family. The column family currently has one column in it, the 'state' column:</span>
<span class="source-line-no">074</span><span id="line-74"> *</span>
<span class="source-line-no">075</span><span id="line-75"> * &lt;pre&gt;</span>
<span class="source-line-no">076</span><span id="line-76"> * table:state =&amp;gt; contains table state</span>
<span class="source-line-no">077</span><span id="line-77"> * &lt;/pre&gt;</span>
<span class="source-line-no">078</span><span id="line-78"> *</span>
<span class="source-line-no">079</span><span id="line-79"> * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.</span>
<span class="source-line-no">080</span><span id="line-80"> * &lt;p/&gt;</span>
<span class="source-line-no">081</span><span id="line-81"> * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.</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"> * The actual layout of meta should be encapsulated inside MetaTableAccessor methods, and should not</span>
<span class="source-line-no">084</span><span id="line-84"> * leak out of it (through Result objects, etc)</span>
<span class="source-line-no">085</span><span id="line-85"> * @see CatalogFamilyFormat</span>
<span class="source-line-no">086</span><span id="line-86"> * @see ClientMetaTableAccessor</span>
<span class="source-line-no">087</span><span id="line-87"> */</span>
<span class="source-line-no">088</span><span id="line-88">@InterfaceAudience.Private</span>
<span class="source-line-no">089</span><span id="line-89">public final class MetaTableAccessor {</span>
<span class="source-line-no">090</span><span id="line-90"></span>
<span class="source-line-no">091</span><span id="line-91"> private static final Logger LOG = LoggerFactory.getLogger(MetaTableAccessor.class);</span>
<span class="source-line-no">092</span><span id="line-92"> private static final Logger METALOG = LoggerFactory.getLogger("org.apache.hadoop.hbase.META");</span>
<span class="source-line-no">093</span><span id="line-93"></span>
<span class="source-line-no">094</span><span id="line-94"> private MetaTableAccessor() {</span>
<span class="source-line-no">095</span><span id="line-95"> }</span>
<span class="source-line-no">096</span><span id="line-96"></span>
<span class="source-line-no">097</span><span id="line-97"> ////////////////////////</span>
<span class="source-line-no">098</span><span id="line-98"> // Reading operations //</span>
<span class="source-line-no">099</span><span id="line-99"> ////////////////////////</span>
<span class="source-line-no">100</span><span id="line-100"></span>
<span class="source-line-no">101</span><span id="line-101"> /**</span>
<span class="source-line-no">102</span><span id="line-102"> * Performs a full scan of &lt;code&gt;hbase:meta&lt;/code&gt; for regions.</span>
<span class="source-line-no">103</span><span id="line-103"> * @param connection connection we're using</span>
<span class="source-line-no">104</span><span id="line-104"> * @param visitor Visitor invoked against each row in regions family.</span>
<span class="source-line-no">105</span><span id="line-105"> */</span>
<span class="source-line-no">106</span><span id="line-106"> public static void fullScanRegions(Connection connection,</span>
<span class="source-line-no">107</span><span id="line-107"> final ClientMetaTableAccessor.Visitor visitor) throws IOException {</span>
<span class="source-line-no">108</span><span id="line-108"> scanMeta(connection, null, null, QueryType.REGION, visitor);</span>
<span class="source-line-no">109</span><span id="line-109"> }</span>
<span class="source-line-no">110</span><span id="line-110"></span>
<span class="source-line-no">111</span><span id="line-111"> /**</span>
<span class="source-line-no">112</span><span id="line-112"> * Performs a full scan of &lt;code&gt;hbase:meta&lt;/code&gt; for regions.</span>
<span class="source-line-no">113</span><span id="line-113"> * @param connection connection we're using</span>
<span class="source-line-no">114</span><span id="line-114"> */</span>
<span class="source-line-no">115</span><span id="line-115"> public static List&lt;Result&gt; fullScanRegions(Connection connection) throws IOException {</span>
<span class="source-line-no">116</span><span id="line-116"> return fullScan(connection, QueryType.REGION);</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"> /**</span>
<span class="source-line-no">120</span><span id="line-120"> * Performs a full scan of &lt;code&gt;hbase:meta&lt;/code&gt; for tables.</span>
<span class="source-line-no">121</span><span id="line-121"> * @param connection connection we're using</span>
<span class="source-line-no">122</span><span id="line-122"> * @param visitor Visitor invoked against each row in tables family.</span>
<span class="source-line-no">123</span><span id="line-123"> */</span>
<span class="source-line-no">124</span><span id="line-124"> public static void fullScanTables(Connection connection,</span>
<span class="source-line-no">125</span><span id="line-125"> final ClientMetaTableAccessor.Visitor visitor) throws IOException {</span>
<span class="source-line-no">126</span><span id="line-126"> scanMeta(connection, null, null, QueryType.TABLE, visitor);</span>
<span class="source-line-no">127</span><span id="line-127"> }</span>
<span class="source-line-no">128</span><span id="line-128"></span>
<span class="source-line-no">129</span><span id="line-129"> /**</span>
<span class="source-line-no">130</span><span id="line-130"> * Performs a full scan of &lt;code&gt;hbase:meta&lt;/code&gt;.</span>
<span class="source-line-no">131</span><span id="line-131"> * @param connection connection we're using</span>
<span class="source-line-no">132</span><span id="line-132"> * @param type scanned part of meta</span>
<span class="source-line-no">133</span><span id="line-133"> * @return List of {@link Result}</span>
<span class="source-line-no">134</span><span id="line-134"> */</span>
<span class="source-line-no">135</span><span id="line-135"> private static List&lt;Result&gt; fullScan(Connection connection, QueryType type) throws IOException {</span>
<span class="source-line-no">136</span><span id="line-136"> ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();</span>
<span class="source-line-no">137</span><span id="line-137"> scanMeta(connection, null, null, type, v);</span>
<span class="source-line-no">138</span><span id="line-138"> return v.getResults();</span>
<span class="source-line-no">139</span><span id="line-139"> }</span>
<span class="source-line-no">140</span><span id="line-140"></span>
<span class="source-line-no">141</span><span id="line-141"> /**</span>
<span class="source-line-no">142</span><span id="line-142"> * Callers should call close on the returned {@link Table} instance.</span>
<span class="source-line-no">143</span><span id="line-143"> * @param connection connection we're using to access Meta</span>
<span class="source-line-no">144</span><span id="line-144"> * @return An {@link Table} for &lt;code&gt;hbase:meta&lt;/code&gt;</span>
<span class="source-line-no">145</span><span id="line-145"> * @throws NullPointerException if {@code connection} is {@code null}</span>
<span class="source-line-no">146</span><span id="line-146"> */</span>
<span class="source-line-no">147</span><span id="line-147"> public static Table getMetaHTable(final Connection connection) throws IOException {</span>
<span class="source-line-no">148</span><span id="line-148"> // We used to pass whole CatalogTracker in here, now we just pass in Connection</span>
<span class="source-line-no">149</span><span id="line-149"> Objects.requireNonNull(connection, "Connection cannot be null");</span>
<span class="source-line-no">150</span><span id="line-150"> if (connection.isClosed()) {</span>
<span class="source-line-no">151</span><span id="line-151"> throw new IOException("connection is closed");</span>
<span class="source-line-no">152</span><span id="line-152"> }</span>
<span class="source-line-no">153</span><span id="line-153"> return connection.getTable(TableName.META_TABLE_NAME);</span>
<span class="source-line-no">154</span><span id="line-154"> }</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"> * Gets the region info and assignment for the specified region.</span>
<span class="source-line-no">158</span><span id="line-158"> * @param connection connection we're using</span>
<span class="source-line-no">159</span><span id="line-159"> * @param regionName Region to lookup.</span>
<span class="source-line-no">160</span><span id="line-160"> * @return Location and RegionInfo for &lt;code&gt;regionName&lt;/code&gt;</span>
<span class="source-line-no">161</span><span id="line-161"> * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead</span>
<span class="source-line-no">162</span><span id="line-162"> */</span>
<span class="source-line-no">163</span><span id="line-163"> @Deprecated</span>
<span class="source-line-no">164</span><span id="line-164"> public static Pair&lt;RegionInfo, ServerName&gt; getRegion(Connection connection, byte[] regionName)</span>
<span class="source-line-no">165</span><span id="line-165"> throws IOException {</span>
<span class="source-line-no">166</span><span id="line-166"> HRegionLocation location = getRegionLocation(connection, regionName);</span>
<span class="source-line-no">167</span><span id="line-167"> return location == null ? null : new Pair&lt;&gt;(location.getRegion(), location.getServerName());</span>
<span class="source-line-no">168</span><span id="line-168"> }</span>
<span class="source-line-no">169</span><span id="line-169"></span>
<span class="source-line-no">170</span><span id="line-170"> /**</span>
<span class="source-line-no">171</span><span id="line-171"> * Returns the HRegionLocation from meta for the given region</span>
<span class="source-line-no">172</span><span id="line-172"> * @param connection connection we're using</span>
<span class="source-line-no">173</span><span id="line-173"> * @param regionName region we're looking for</span>
<span class="source-line-no">174</span><span id="line-174"> * @return HRegionLocation for the given region</span>
<span class="source-line-no">175</span><span id="line-175"> */</span>
<span class="source-line-no">176</span><span id="line-176"> public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)</span>
<span class="source-line-no">177</span><span id="line-177"> throws IOException {</span>
<span class="source-line-no">178</span><span id="line-178"> byte[] row = regionName;</span>
<span class="source-line-no">179</span><span id="line-179"> RegionInfo parsedInfo = null;</span>
<span class="source-line-no">180</span><span id="line-180"> try {</span>
<span class="source-line-no">181</span><span id="line-181"> parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);</span>
<span class="source-line-no">182</span><span id="line-182"> row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);</span>
<span class="source-line-no">183</span><span id="line-183"> } catch (Exception parseEx) {</span>
<span class="source-line-no">184</span><span id="line-184"> // Ignore. This is used with tableName passed as regionName.</span>
<span class="source-line-no">185</span><span id="line-185"> }</span>
<span class="source-line-no">186</span><span id="line-186"> Get get = new Get(row);</span>
<span class="source-line-no">187</span><span id="line-187"> get.addFamily(HConstants.CATALOG_FAMILY);</span>
<span class="source-line-no">188</span><span id="line-188"> Result r;</span>
<span class="source-line-no">189</span><span id="line-189"> try (Table t = getMetaHTable(connection)) {</span>
<span class="source-line-no">190</span><span id="line-190"> r = t.get(get);</span>
<span class="source-line-no">191</span><span id="line-191"> }</span>
<span class="source-line-no">192</span><span id="line-192"> RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);</span>
<span class="source-line-no">193</span><span id="line-193"> return locations == null</span>
<span class="source-line-no">194</span><span id="line-194"> ? null</span>
<span class="source-line-no">195</span><span id="line-195"> : locations.getRegionLocation(</span>
<span class="source-line-no">196</span><span id="line-196"> parsedInfo == null ? RegionInfo.DEFAULT_REPLICA_ID : parsedInfo.getReplicaId());</span>
<span class="source-line-no">197</span><span id="line-197"> }</span>
<span class="source-line-no">198</span><span id="line-198"></span>
<span class="source-line-no">199</span><span id="line-199"> /**</span>
<span class="source-line-no">200</span><span id="line-200"> * Returns the HRegionLocation from meta for the given region</span>
<span class="source-line-no">201</span><span id="line-201"> * @param connection connection we're using</span>
<span class="source-line-no">202</span><span id="line-202"> * @param regionInfo region information</span>
<span class="source-line-no">203</span><span id="line-203"> * @return HRegionLocation for the given region</span>
<span class="source-line-no">204</span><span id="line-204"> */</span>
<span class="source-line-no">205</span><span id="line-205"> public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)</span>
<span class="source-line-no">206</span><span id="line-206"> throws IOException {</span>
<span class="source-line-no">207</span><span id="line-207"> return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),</span>
<span class="source-line-no">208</span><span id="line-208"> regionInfo, regionInfo.getReplicaId());</span>
<span class="source-line-no">209</span><span id="line-209"> }</span>
<span class="source-line-no">210</span><span id="line-210"></span>
<span class="source-line-no">211</span><span id="line-211"> /** Returns Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta. */</span>
<span class="source-line-no">212</span><span id="line-212"> public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)</span>
<span class="source-line-no">213</span><span id="line-213"> throws IOException {</span>
<span class="source-line-no">214</span><span id="line-214"> Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));</span>
<span class="source-line-no">215</span><span id="line-215"> get.addFamily(HConstants.CATALOG_FAMILY);</span>
<span class="source-line-no">216</span><span id="line-216"> try (Table t = getMetaHTable(connection)) {</span>
<span class="source-line-no">217</span><span id="line-217"> return t.get(get);</span>
<span class="source-line-no">218</span><span id="line-218"> }</span>
<span class="source-line-no">219</span><span id="line-219"> }</span>
<span class="source-line-no">220</span><span id="line-220"></span>
<span class="source-line-no">221</span><span id="line-221"> /**</span>
<span class="source-line-no">222</span><span id="line-222"> * Gets the result in hbase:meta for the specified region.</span>
<span class="source-line-no">223</span><span id="line-223"> * @param connection connection we're using</span>
<span class="source-line-no">224</span><span id="line-224"> * @param regionInfo region we're looking for</span>
<span class="source-line-no">225</span><span id="line-225"> * @return result of the specified region</span>
<span class="source-line-no">226</span><span id="line-226"> */</span>
<span class="source-line-no">227</span><span id="line-227"> public static Result getRegionResult(Connection connection, RegionInfo regionInfo)</span>
<span class="source-line-no">228</span><span id="line-228"> throws IOException {</span>
<span class="source-line-no">229</span><span id="line-229"> Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(regionInfo));</span>
<span class="source-line-no">230</span><span id="line-230"> get.addFamily(HConstants.CATALOG_FAMILY);</span>
<span class="source-line-no">231</span><span id="line-231"> try (Table t = getMetaHTable(connection)) {</span>
<span class="source-line-no">232</span><span id="line-232"> return t.get(get);</span>
<span class="source-line-no">233</span><span id="line-233"> }</span>
<span class="source-line-no">234</span><span id="line-234"> }</span>
<span class="source-line-no">235</span><span id="line-235"></span>
<span class="source-line-no">236</span><span id="line-236"> /**</span>
<span class="source-line-no">237</span><span id="line-237"> * Scans META table for a row whose key contains the specified &lt;B&gt;regionEncodedName&lt;/B&gt;, returning</span>
<span class="source-line-no">238</span><span id="line-238"> * a single related &lt;code&gt;Result&lt;/code&gt; instance if any row is found, null otherwise.</span>
<span class="source-line-no">239</span><span id="line-239"> * @param connection the connection to query META table.</span>
<span class="source-line-no">240</span><span id="line-240"> * @param regionEncodedName the region encoded name to look for at META.</span>
<span class="source-line-no">241</span><span id="line-241"> * @return &lt;code&gt;Result&lt;/code&gt; instance with the row related info in META, null otherwise.</span>
<span class="source-line-no">242</span><span id="line-242"> * @throws IOException if any errors occur while querying META.</span>
<span class="source-line-no">243</span><span id="line-243"> */</span>
<span class="source-line-no">244</span><span id="line-244"> public static Result scanByRegionEncodedName(Connection connection, String regionEncodedName)</span>
<span class="source-line-no">245</span><span id="line-245"> throws IOException {</span>
<span class="source-line-no">246</span><span id="line-246"> RowFilter rowFilter =</span>
<span class="source-line-no">247</span><span id="line-247"> new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName));</span>
<span class="source-line-no">248</span><span id="line-248"> Scan scan = getMetaScan(connection.getConfiguration(), 1);</span>
<span class="source-line-no">249</span><span id="line-249"> scan.setFilter(rowFilter);</span>
<span class="source-line-no">250</span><span id="line-250"> try (Table table = getMetaHTable(connection);</span>
<span class="source-line-no">251</span><span id="line-251"> ResultScanner resultScanner = table.getScanner(scan)) {</span>
<span class="source-line-no">252</span><span id="line-252"> return resultScanner.next();</span>
<span class="source-line-no">253</span><span id="line-253"> }</span>
<span class="source-line-no">254</span><span id="line-254"> }</span>
<span class="source-line-no">255</span><span id="line-255"></span>
<span class="source-line-no">256</span><span id="line-256"> /**</span>
<span class="source-line-no">257</span><span id="line-257"> * Lists all of the regions currently in META.</span>
<span class="source-line-no">258</span><span id="line-258"> * @param connection to connect with</span>
<span class="source-line-no">259</span><span id="line-259"> * @param excludeOfflinedSplitParents False if we are to include offlined/splitparents regions,</span>
<span class="source-line-no">260</span><span id="line-260"> * true and we'll leave out offlined regions from returned list</span>
<span class="source-line-no">261</span><span id="line-261"> * @return List of all user-space regions.</span>
<span class="source-line-no">262</span><span id="line-262"> */</span>
<span class="source-line-no">263</span><span id="line-263"> public static List&lt;RegionInfo&gt; getAllRegions(Connection connection,</span>
<span class="source-line-no">264</span><span id="line-264"> boolean excludeOfflinedSplitParents) throws IOException {</span>
<span class="source-line-no">265</span><span id="line-265"> List&lt;Pair&lt;RegionInfo, ServerName&gt;&gt; result;</span>
<span class="source-line-no">266</span><span id="line-266"></span>
<span class="source-line-no">267</span><span id="line-267"> result = getTableRegionsAndLocations(connection, null, excludeOfflinedSplitParents);</span>
<span class="source-line-no">268</span><span id="line-268"></span>
<span class="source-line-no">269</span><span id="line-269"> return getListOfRegionInfos(result);</span>
<span class="source-line-no">270</span><span id="line-270"></span>
<span class="source-line-no">271</span><span id="line-271"> }</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"> * Gets all of the regions of the specified table. Do not use this method to get meta table</span>
<span class="source-line-no">275</span><span id="line-275"> * regions, use methods in MetaTableLocator instead.</span>
<span class="source-line-no">276</span><span id="line-276"> * @param connection connection we're using</span>
<span class="source-line-no">277</span><span id="line-277"> * @param tableName table we're looking for</span>
<span class="source-line-no">278</span><span id="line-278"> * @return Ordered list of {@link RegionInfo}.</span>
<span class="source-line-no">279</span><span id="line-279"> */</span>
<span class="source-line-no">280</span><span id="line-280"> public static List&lt;RegionInfo&gt; getTableRegions(Connection connection, TableName tableName)</span>
<span class="source-line-no">281</span><span id="line-281"> throws IOException {</span>
<span class="source-line-no">282</span><span id="line-282"> return getTableRegions(connection, tableName, false);</span>
<span class="source-line-no">283</span><span id="line-283"> }</span>
<span class="source-line-no">284</span><span id="line-284"></span>
<span class="source-line-no">285</span><span id="line-285"> /**</span>
<span class="source-line-no">286</span><span id="line-286"> * Gets all of the regions of the specified table. Do not use this method to get meta table</span>
<span class="source-line-no">287</span><span id="line-287"> * regions, use methods in MetaTableLocator instead.</span>
<span class="source-line-no">288</span><span id="line-288"> * @param connection connection we're using</span>
<span class="source-line-no">289</span><span id="line-289"> * @param tableName table we're looking for</span>
<span class="source-line-no">290</span><span id="line-290"> * @param excludeOfflinedSplitParents If true, do not include offlined split parents in the</span>
<span class="source-line-no">291</span><span id="line-291"> * return.</span>
<span class="source-line-no">292</span><span id="line-292"> * @return Ordered list of {@link RegionInfo}.</span>
<span class="source-line-no">293</span><span id="line-293"> */</span>
<span class="source-line-no">294</span><span id="line-294"> public static List&lt;RegionInfo&gt; getTableRegions(Connection connection, TableName tableName,</span>
<span class="source-line-no">295</span><span id="line-295"> final boolean excludeOfflinedSplitParents) throws IOException {</span>
<span class="source-line-no">296</span><span id="line-296"> List&lt;Pair&lt;RegionInfo, ServerName&gt;&gt; result =</span>
<span class="source-line-no">297</span><span id="line-297"> getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);</span>
<span class="source-line-no">298</span><span id="line-298"> return getListOfRegionInfos(result);</span>
<span class="source-line-no">299</span><span id="line-299"> }</span>
<span class="source-line-no">300</span><span id="line-300"></span>
<span class="source-line-no">301</span><span id="line-301"> private static List&lt;RegionInfo&gt;</span>
<span class="source-line-no">302</span><span id="line-302"> getListOfRegionInfos(final List&lt;Pair&lt;RegionInfo, ServerName&gt;&gt; pairs) {</span>
<span class="source-line-no">303</span><span id="line-303"> if (pairs == null || pairs.isEmpty()) {</span>
<span class="source-line-no">304</span><span id="line-304"> return Collections.emptyList();</span>
<span class="source-line-no">305</span><span id="line-305"> }</span>
<span class="source-line-no">306</span><span id="line-306"> List&lt;RegionInfo&gt; result = new ArrayList&lt;&gt;(pairs.size());</span>
<span class="source-line-no">307</span><span id="line-307"> for (Pair&lt;RegionInfo, ServerName&gt; pair : pairs) {</span>
<span class="source-line-no">308</span><span id="line-308"> result.add(pair.getFirst());</span>
<span class="source-line-no">309</span><span id="line-309"> }</span>
<span class="source-line-no">310</span><span id="line-310"> return result;</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"> /**</span>
<span class="source-line-no">314</span><span id="line-314"> * This method creates a Scan object that will only scan catalog rows that belong to the specified</span>
<span class="source-line-no">315</span><span id="line-315"> * table. It doesn't specify any columns. This is a better alternative to just using a start row</span>
<span class="source-line-no">316</span><span id="line-316"> * and scan until it hits a new table since that requires parsing the HRI to get the table name.</span>
<span class="source-line-no">317</span><span id="line-317"> * @param tableName bytes of table's name</span>
<span class="source-line-no">318</span><span id="line-318"> * @return configured Scan object</span>
<span class="source-line-no">319</span><span id="line-319"> */</span>
<span class="source-line-no">320</span><span id="line-320"> public static Scan getScanForTableName(Configuration conf, TableName tableName) {</span>
<span class="source-line-no">321</span><span id="line-321"> // Start key is just the table name with delimiters</span>
<span class="source-line-no">322</span><span id="line-322"> byte[] startKey = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);</span>
<span class="source-line-no">323</span><span id="line-323"> // Stop key appends the smallest possible char to the table name</span>
<span class="source-line-no">324</span><span id="line-324"> byte[] stopKey = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);</span>
<span class="source-line-no">325</span><span id="line-325"></span>
<span class="source-line-no">326</span><span id="line-326"> Scan scan = getMetaScan(conf, -1);</span>
<span class="source-line-no">327</span><span id="line-327"> scan.withStartRow(startKey);</span>
<span class="source-line-no">328</span><span id="line-328"> scan.withStopRow(stopKey);</span>
<span class="source-line-no">329</span><span id="line-329"> return scan;</span>
<span class="source-line-no">330</span><span id="line-330"> }</span>
<span class="source-line-no">331</span><span id="line-331"></span>
<span class="source-line-no">332</span><span id="line-332"> private static Scan getMetaScan(Configuration conf, int rowUpperLimit) {</span>
<span class="source-line-no">333</span><span id="line-333"> Scan scan = new Scan();</span>
<span class="source-line-no">334</span><span id="line-334"> int scannerCaching = conf.getInt(HConstants.HBASE_META_SCANNER_CACHING,</span>
<span class="source-line-no">335</span><span id="line-335"> HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);</span>
<span class="source-line-no">336</span><span id="line-336"> if (conf.getBoolean(HConstants.USE_META_REPLICAS, HConstants.DEFAULT_USE_META_REPLICAS)) {</span>
<span class="source-line-no">337</span><span id="line-337"> scan.setConsistency(Consistency.TIMELINE);</span>
<span class="source-line-no">338</span><span id="line-338"> }</span>
<span class="source-line-no">339</span><span id="line-339"> if (rowUpperLimit &gt; 0) {</span>
<span class="source-line-no">340</span><span id="line-340"> scan.setLimit(rowUpperLimit);</span>
<span class="source-line-no">341</span><span id="line-341"> scan.setReadType(Scan.ReadType.PREAD);</span>
<span class="source-line-no">342</span><span id="line-342"> }</span>
<span class="source-line-no">343</span><span id="line-343"> scan.setCaching(scannerCaching);</span>
<span class="source-line-no">344</span><span id="line-344"> return scan;</span>
<span class="source-line-no">345</span><span id="line-345"> }</span>
<span class="source-line-no">346</span><span id="line-346"></span>
<span class="source-line-no">347</span><span id="line-347"> /**</span>
<span class="source-line-no">348</span><span id="line-348"> * Do not use this method to get meta table regions, use methods in MetaTableLocator instead.</span>
<span class="source-line-no">349</span><span id="line-349"> * @param connection connection we're using</span>
<span class="source-line-no">350</span><span id="line-350"> * @param tableName table we're looking for</span>
<span class="source-line-no">351</span><span id="line-351"> * @return Return list of regioninfos and server.</span>
<span class="source-line-no">352</span><span id="line-352"> */</span>
<span class="source-line-no">353</span><span id="line-353"> public static List&lt;Pair&lt;RegionInfo, ServerName&gt;&gt;</span>
<span class="source-line-no">354</span><span id="line-354"> getTableRegionsAndLocations(Connection connection, TableName tableName) throws IOException {</span>
<span class="source-line-no">355</span><span id="line-355"> return getTableRegionsAndLocations(connection, tableName, true);</span>
<span class="source-line-no">356</span><span id="line-356"> }</span>
<span class="source-line-no">357</span><span id="line-357"></span>
<span class="source-line-no">358</span><span id="line-358"> /**</span>
<span class="source-line-no">359</span><span id="line-359"> * Do not use this method to get meta table regions, use methods in MetaTableLocator instead.</span>
<span class="source-line-no">360</span><span id="line-360"> * @param connection connection we're using</span>
<span class="source-line-no">361</span><span id="line-361"> * @param tableName table to work with, can be null for getting all regions</span>
<span class="source-line-no">362</span><span id="line-362"> * @param excludeOfflinedSplitParents don't return split parents</span>
<span class="source-line-no">363</span><span id="line-363"> * @return Return list of regioninfos and server addresses.</span>
<span class="source-line-no">364</span><span id="line-364"> */</span>
<span class="source-line-no">365</span><span id="line-365"> // What happens here when 1M regions in hbase:meta? This won't scale?</span>
<span class="source-line-no">366</span><span id="line-366"> public static List&lt;Pair&lt;RegionInfo, ServerName&gt;&gt; getTableRegionsAndLocations(</span>
<span class="source-line-no">367</span><span id="line-367"> Connection connection, @Nullable final TableName tableName,</span>
<span class="source-line-no">368</span><span id="line-368"> final boolean excludeOfflinedSplitParents) throws IOException {</span>
<span class="source-line-no">369</span><span id="line-369"> if (tableName != null &amp;&amp; tableName.equals(TableName.META_TABLE_NAME)) {</span>
<span class="source-line-no">370</span><span id="line-370"> throw new IOException(</span>
<span class="source-line-no">371</span><span id="line-371"> "This method can't be used to locate meta regions;" + " use MetaTableLocator instead");</span>
<span class="source-line-no">372</span><span id="line-372"> }</span>
<span class="source-line-no">373</span><span id="line-373"> // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress</span>
<span class="source-line-no">374</span><span id="line-374"> ClientMetaTableAccessor.CollectRegionLocationsVisitor visitor =</span>
<span class="source-line-no">375</span><span id="line-375"> new ClientMetaTableAccessor.CollectRegionLocationsVisitor(excludeOfflinedSplitParents);</span>
<span class="source-line-no">376</span><span id="line-376"> scanMeta(connection,</span>
<span class="source-line-no">377</span><span id="line-377"> ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION),</span>
<span class="source-line-no">378</span><span id="line-378"> ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION), QueryType.REGION,</span>
<span class="source-line-no">379</span><span id="line-379"> visitor);</span>
<span class="source-line-no">380</span><span id="line-380"> return visitor.getResults();</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"> public static void fullScanMetaAndPrint(Connection connection) throws IOException {</span>
<span class="source-line-no">384</span><span id="line-384"> ClientMetaTableAccessor.Visitor v = r -&gt; {</span>
<span class="source-line-no">385</span><span id="line-385"> if (r == null || r.isEmpty()) {</span>
<span class="source-line-no">386</span><span id="line-386"> return true;</span>
<span class="source-line-no">387</span><span id="line-387"> }</span>
<span class="source-line-no">388</span><span id="line-388"> LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);</span>
<span class="source-line-no">389</span><span id="line-389"> TableState state = CatalogFamilyFormat.getTableState(r);</span>
<span class="source-line-no">390</span><span id="line-390"> if (state != null) {</span>
<span class="source-line-no">391</span><span id="line-391"> LOG.info("fullScanMetaAndPrint.Table State={}" + state);</span>
<span class="source-line-no">392</span><span id="line-392"> } else {</span>
<span class="source-line-no">393</span><span id="line-393"> RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);</span>
<span class="source-line-no">394</span><span id="line-394"> if (locations == null) {</span>
<span class="source-line-no">395</span><span id="line-395"> return true;</span>
<span class="source-line-no">396</span><span id="line-396"> }</span>
<span class="source-line-no">397</span><span id="line-397"> for (HRegionLocation loc : locations.getRegionLocations()) {</span>
<span class="source-line-no">398</span><span id="line-398"> if (loc != null) {</span>
<span class="source-line-no">399</span><span id="line-399"> LOG.info("fullScanMetaAndPrint.HRI Print={}", loc.getRegion());</span>
<span class="source-line-no">400</span><span id="line-400"> }</span>
<span class="source-line-no">401</span><span id="line-401"> }</span>
<span class="source-line-no">402</span><span id="line-402"> }</span>
<span class="source-line-no">403</span><span id="line-403"> return true;</span>
<span class="source-line-no">404</span><span id="line-404"> };</span>
<span class="source-line-no">405</span><span id="line-405"> scanMeta(connection, null, null, QueryType.ALL, v);</span>
<span class="source-line-no">406</span><span id="line-406"> }</span>
<span class="source-line-no">407</span><span id="line-407"></span>
<span class="source-line-no">408</span><span id="line-408"> public static void scanMetaForTableRegions(Connection connection,</span>
<span class="source-line-no">409</span><span id="line-409"> ClientMetaTableAccessor.Visitor visitor, TableName tableName) throws IOException {</span>
<span class="source-line-no">410</span><span id="line-410"> scanMeta(connection, tableName, QueryType.REGION, Integer.MAX_VALUE, visitor);</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"> private static void scanMeta(Connection connection, TableName table, QueryType type, int maxRows,</span>
<span class="source-line-no">414</span><span id="line-414"> final ClientMetaTableAccessor.Visitor visitor) throws IOException {</span>
<span class="source-line-no">415</span><span id="line-415"> scanMeta(connection, ClientMetaTableAccessor.getTableStartRowForMeta(table, type),</span>
<span class="source-line-no">416</span><span id="line-416"> ClientMetaTableAccessor.getTableStopRowForMeta(table, type), type, maxRows, visitor);</span>
<span class="source-line-no">417</span><span id="line-417"> }</span>
<span class="source-line-no">418</span><span id="line-418"></span>
<span class="source-line-no">419</span><span id="line-419"> public static void scanMeta(Connection connection, @Nullable final byte[] startRow,</span>
<span class="source-line-no">420</span><span id="line-420"> @Nullable final byte[] stopRow, QueryType type, final ClientMetaTableAccessor.Visitor visitor)</span>
<span class="source-line-no">421</span><span id="line-421"> throws IOException {</span>
<span class="source-line-no">422</span><span id="line-422"> scanMeta(connection, startRow, stopRow, type, Integer.MAX_VALUE, visitor);</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"> /**</span>
<span class="source-line-no">426</span><span id="line-426"> * Performs a scan of META table for given table starting from given row.</span>
<span class="source-line-no">427</span><span id="line-427"> * @param connection connection we're using</span>
<span class="source-line-no">428</span><span id="line-428"> * @param visitor visitor to call</span>
<span class="source-line-no">429</span><span id="line-429"> * @param tableName table withing we scan</span>
<span class="source-line-no">430</span><span id="line-430"> * @param row start scan from this row</span>
<span class="source-line-no">431</span><span id="line-431"> * @param rowLimit max number of rows to return</span>
<span class="source-line-no">432</span><span id="line-432"> */</span>
<span class="source-line-no">433</span><span id="line-433"> public static void scanMeta(Connection connection, final ClientMetaTableAccessor.Visitor visitor,</span>
<span class="source-line-no">434</span><span id="line-434"> final TableName tableName, final byte[] row, final int rowLimit) throws IOException {</span>
<span class="source-line-no">435</span><span id="line-435"> byte[] startRow = null;</span>
<span class="source-line-no">436</span><span id="line-436"> byte[] stopRow = null;</span>
<span class="source-line-no">437</span><span id="line-437"> if (tableName != null) {</span>
<span class="source-line-no">438</span><span id="line-438"> startRow = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);</span>
<span class="source-line-no">439</span><span id="line-439"> if (row != null) {</span>
<span class="source-line-no">440</span><span id="line-440"> RegionInfo closestRi = getClosestRegionInfo(connection, tableName, row);</span>
<span class="source-line-no">441</span><span id="line-441"> startRow =</span>
<span class="source-line-no">442</span><span id="line-442"> RegionInfo.createRegionName(tableName, closestRi.getStartKey(), HConstants.ZEROES, false);</span>
<span class="source-line-no">443</span><span id="line-443"> }</span>
<span class="source-line-no">444</span><span id="line-444"> stopRow = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);</span>
<span class="source-line-no">445</span><span id="line-445"> }</span>
<span class="source-line-no">446</span><span id="line-446"> scanMeta(connection, startRow, stopRow, QueryType.REGION, rowLimit, visitor);</span>
<span class="source-line-no">447</span><span id="line-447"> }</span>
<span class="source-line-no">448</span><span id="line-448"></span>
<span class="source-line-no">449</span><span id="line-449"> /**</span>
<span class="source-line-no">450</span><span id="line-450"> * Performs a scan of META table.</span>
<span class="source-line-no">451</span><span id="line-451"> * @param connection connection we're using</span>
<span class="source-line-no">452</span><span id="line-452"> * @param startRow Where to start the scan. Pass null if want to begin scan at first row.</span>
<span class="source-line-no">453</span><span id="line-453"> * @param stopRow Where to stop the scan. Pass null if want to scan all rows from the start one</span>
<span class="source-line-no">454</span><span id="line-454"> * @param type scanned part of meta</span>
<span class="source-line-no">455</span><span id="line-455"> * @param maxRows maximum rows to return</span>
<span class="source-line-no">456</span><span id="line-456"> * @param visitor Visitor invoked against each row.</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 static void scanMeta(Connection connection, @Nullable final byte[] startRow,</span>
<span class="source-line-no">459</span><span id="line-459"> @Nullable final byte[] stopRow, QueryType type, int maxRows,</span>
<span class="source-line-no">460</span><span id="line-460"> final ClientMetaTableAccessor.Visitor visitor) throws IOException {</span>
<span class="source-line-no">461</span><span id="line-461"> scanMeta(connection, startRow, stopRow, type, null, maxRows, visitor);</span>
<span class="source-line-no">462</span><span id="line-462"> }</span>
<span class="source-line-no">463</span><span id="line-463"></span>
<span class="source-line-no">464</span><span id="line-464"> public static void scanMeta(Connection connection, @Nullable final byte[] startRow,</span>
<span class="source-line-no">465</span><span id="line-465"> @Nullable final byte[] stopRow, QueryType type, @Nullable Filter filter, int maxRows,</span>
<span class="source-line-no">466</span><span id="line-466"> final ClientMetaTableAccessor.Visitor visitor) throws IOException {</span>
<span class="source-line-no">467</span><span id="line-467"> int rowUpperLimit = maxRows &gt; 0 ? maxRows : Integer.MAX_VALUE;</span>
<span class="source-line-no">468</span><span id="line-468"> Scan scan = getMetaScan(connection.getConfiguration(), rowUpperLimit);</span>
<span class="source-line-no">469</span><span id="line-469"></span>
<span class="source-line-no">470</span><span id="line-470"> for (byte[] family : type.getFamilies()) {</span>
<span class="source-line-no">471</span><span id="line-471"> scan.addFamily(family);</span>
<span class="source-line-no">472</span><span id="line-472"> }</span>
<span class="source-line-no">473</span><span id="line-473"> if (startRow != null) {</span>
<span class="source-line-no">474</span><span id="line-474"> scan.withStartRow(startRow);</span>
<span class="source-line-no">475</span><span id="line-475"> }</span>
<span class="source-line-no">476</span><span id="line-476"> if (stopRow != null) {</span>
<span class="source-line-no">477</span><span id="line-477"> scan.withStopRow(stopRow);</span>
<span class="source-line-no">478</span><span id="line-478"> }</span>
<span class="source-line-no">479</span><span id="line-479"> if (filter != null) {</span>
<span class="source-line-no">480</span><span id="line-480"> scan.setFilter(filter);</span>
<span class="source-line-no">481</span><span id="line-481"> }</span>
<span class="source-line-no">482</span><span id="line-482"></span>
<span class="source-line-no">483</span><span id="line-483"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">484</span><span id="line-484"> LOG.trace("Scanning META" + " starting at row=" + Bytes.toStringBinary(startRow)</span>
<span class="source-line-no">485</span><span id="line-485"> + " stopping at row=" + Bytes.toStringBinary(stopRow) + " for max=" + rowUpperLimit</span>
<span class="source-line-no">486</span><span id="line-486"> + " with caching=" + scan.getCaching());</span>
<span class="source-line-no">487</span><span id="line-487"> }</span>
<span class="source-line-no">488</span><span id="line-488"></span>
<span class="source-line-no">489</span><span id="line-489"> int currentRow = 0;</span>
<span class="source-line-no">490</span><span id="line-490"> try (Table metaTable = getMetaHTable(connection)) {</span>
<span class="source-line-no">491</span><span id="line-491"> try (ResultScanner scanner = metaTable.getScanner(scan)) {</span>
<span class="source-line-no">492</span><span id="line-492"> Result data;</span>
<span class="source-line-no">493</span><span id="line-493"> while ((data = scanner.next()) != null) {</span>
<span class="source-line-no">494</span><span id="line-494"> if (data.isEmpty()) {</span>
<span class="source-line-no">495</span><span id="line-495"> continue;</span>
<span class="source-line-no">496</span><span id="line-496"> }</span>
<span class="source-line-no">497</span><span id="line-497"> // Break if visit returns false.</span>
<span class="source-line-no">498</span><span id="line-498"> if (!visitor.visit(data)) {</span>
<span class="source-line-no">499</span><span id="line-499"> break;</span>
<span class="source-line-no">500</span><span id="line-500"> }</span>
<span class="source-line-no">501</span><span id="line-501"> if (++currentRow &gt;= rowUpperLimit) {</span>
<span class="source-line-no">502</span><span id="line-502"> break;</span>
<span class="source-line-no">503</span><span id="line-503"> }</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"> if (visitor instanceof Closeable) {</span>
<span class="source-line-no">508</span><span id="line-508"> try {</span>
<span class="source-line-no">509</span><span id="line-509"> ((Closeable) visitor).close();</span>
<span class="source-line-no">510</span><span id="line-510"> } catch (Throwable t) {</span>
<span class="source-line-no">511</span><span id="line-511"> ExceptionUtil.rethrowIfInterrupt(t);</span>
<span class="source-line-no">512</span><span id="line-512"> LOG.debug("Got exception in closing the meta scanner visitor", t);</span>
<span class="source-line-no">513</span><span id="line-513"> }</span>
<span class="source-line-no">514</span><span id="line-514"> }</span>
<span class="source-line-no">515</span><span id="line-515"> }</span>
<span class="source-line-no">516</span><span id="line-516"></span>
<span class="source-line-no">517</span><span id="line-517"> /** Returns Get closest metatable region row to passed &lt;code&gt;row&lt;/code&gt; */</span>
<span class="source-line-no">518</span><span id="line-518"> @NonNull</span>
<span class="source-line-no">519</span><span id="line-519"> private static RegionInfo getClosestRegionInfo(Connection connection,</span>
<span class="source-line-no">520</span><span id="line-520"> @NonNull final TableName tableName, @NonNull final byte[] row) throws IOException {</span>
<span class="source-line-no">521</span><span id="line-521"> byte[] searchRow = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);</span>
<span class="source-line-no">522</span><span id="line-522"> Scan scan = getMetaScan(connection.getConfiguration(), 1);</span>
<span class="source-line-no">523</span><span id="line-523"> scan.setReversed(true);</span>
<span class="source-line-no">524</span><span id="line-524"> scan.withStartRow(searchRow);</span>
<span class="source-line-no">525</span><span id="line-525"> try (ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan)) {</span>
<span class="source-line-no">526</span><span id="line-526"> Result result = resultScanner.next();</span>
<span class="source-line-no">527</span><span id="line-527"> if (result == null) {</span>
<span class="source-line-no">528</span><span id="line-528"> throw new TableNotFoundException("Cannot find row in META " + " for table: " + tableName</span>
<span class="source-line-no">529</span><span id="line-529"> + ", row=" + Bytes.toStringBinary(row));</span>
<span class="source-line-no">530</span><span id="line-530"> }</span>
<span class="source-line-no">531</span><span id="line-531"> RegionInfo regionInfo = CatalogFamilyFormat.getRegionInfo(result);</span>
<span class="source-line-no">532</span><span id="line-532"> if (regionInfo == null) {</span>
<span class="source-line-no">533</span><span id="line-533"> throw new IOException("RegionInfo was null or empty in Meta for " + tableName + ", row="</span>
<span class="source-line-no">534</span><span id="line-534"> + Bytes.toStringBinary(row));</span>
<span class="source-line-no">535</span><span id="line-535"> }</span>
<span class="source-line-no">536</span><span id="line-536"> return regionInfo;</span>
<span class="source-line-no">537</span><span id="line-537"> }</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"> * Returns the {@link ServerName} from catalog table {@link Result} where the region is</span>
<span class="source-line-no">542</span><span id="line-542"> * transitioning on. It should be the same as</span>
<span class="source-line-no">543</span><span id="line-543"> * {@link CatalogFamilyFormat#getServerName(Result,int)} if the server is at OPEN state.</span>
<span class="source-line-no">544</span><span id="line-544"> * @param r Result to pull the transitioning server name from</span>
<span class="source-line-no">545</span><span id="line-545"> * @return A ServerName instance or {@link CatalogFamilyFormat#getServerName(Result,int)} if</span>
<span class="source-line-no">546</span><span id="line-546"> * necessary fields not found or empty.</span>
<span class="source-line-no">547</span><span id="line-547"> */</span>
<span class="source-line-no">548</span><span id="line-548"> @Nullable</span>
<span class="source-line-no">549</span><span id="line-549"> public static ServerName getTargetServerName(final Result r, final int replicaId) {</span>
<span class="source-line-no">550</span><span id="line-550"> final Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY,</span>
<span class="source-line-no">551</span><span id="line-551"> CatalogFamilyFormat.getServerNameColumn(replicaId));</span>
<span class="source-line-no">552</span><span id="line-552"> if (cell == null || cell.getValueLength() == 0) {</span>
<span class="source-line-no">553</span><span id="line-553"> RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);</span>
<span class="source-line-no">554</span><span id="line-554"> if (locations != null) {</span>
<span class="source-line-no">555</span><span id="line-555"> HRegionLocation location = locations.getRegionLocation(replicaId);</span>
<span class="source-line-no">556</span><span id="line-556"> if (location != null) {</span>
<span class="source-line-no">557</span><span id="line-557"> return location.getServerName();</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>
<span class="source-line-no">560</span><span id="line-560"> return null;</span>
<span class="source-line-no">561</span><span id="line-561"> }</span>
<span class="source-line-no">562</span><span id="line-562"> return ServerName.parseServerName(</span>
<span class="source-line-no">563</span><span id="line-563"> Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));</span>
<span class="source-line-no">564</span><span id="line-564"> }</span>
<span class="source-line-no">565</span><span id="line-565"></span>
<span class="source-line-no">566</span><span id="line-566"> /**</span>
<span class="source-line-no">567</span><span id="line-567"> * Returns the daughter regions by reading the corresponding columns of the catalog table Result.</span>
<span class="source-line-no">568</span><span id="line-568"> * @param data a Result object from the catalog table scan</span>
<span class="source-line-no">569</span><span id="line-569"> * @return pair of RegionInfo or PairOfSameType(null, null) if region is not a split parent</span>
<span class="source-line-no">570</span><span id="line-570"> */</span>
<span class="source-line-no">571</span><span id="line-571"> public static PairOfSameType&lt;RegionInfo&gt; getDaughterRegions(Result data) {</span>
<span class="source-line-no">572</span><span id="line-572"> RegionInfo splitA = CatalogFamilyFormat.getRegionInfo(data, HConstants.SPLITA_QUALIFIER);</span>
<span class="source-line-no">573</span><span id="line-573"> RegionInfo splitB = CatalogFamilyFormat.getRegionInfo(data, HConstants.SPLITB_QUALIFIER);</span>
<span class="source-line-no">574</span><span id="line-574"> return new PairOfSameType&lt;&gt;(splitA, splitB);</span>
<span class="source-line-no">575</span><span id="line-575"> }</span>
<span class="source-line-no">576</span><span id="line-576"></span>
<span class="source-line-no">577</span><span id="line-577"> /**</span>
<span class="source-line-no">578</span><span id="line-578"> * Fetch table state for given table from META table</span>
<span class="source-line-no">579</span><span id="line-579"> * @param conn connection to use</span>
<span class="source-line-no">580</span><span id="line-580"> * @param tableName table to fetch state for</span>
<span class="source-line-no">581</span><span id="line-581"> */</span>
<span class="source-line-no">582</span><span id="line-582"> @Nullable</span>
<span class="source-line-no">583</span><span id="line-583"> public static TableState getTableState(Connection conn, TableName tableName) throws IOException {</span>
<span class="source-line-no">584</span><span id="line-584"> if (tableName.equals(TableName.META_TABLE_NAME)) {</span>
<span class="source-line-no">585</span><span id="line-585"> return new TableState(tableName, TableState.State.ENABLED);</span>
<span class="source-line-no">586</span><span id="line-586"> }</span>
<span class="source-line-no">587</span><span id="line-587"> Table metaHTable = getMetaHTable(conn);</span>
<span class="source-line-no">588</span><span id="line-588"> Get get = new Get(tableName.getName()).addColumn(HConstants.TABLE_FAMILY,</span>
<span class="source-line-no">589</span><span id="line-589"> HConstants.TABLE_STATE_QUALIFIER);</span>
<span class="source-line-no">590</span><span id="line-590"> Result result = metaHTable.get(get);</span>
<span class="source-line-no">591</span><span id="line-591"> return CatalogFamilyFormat.getTableState(result);</span>
<span class="source-line-no">592</span><span id="line-592"> }</span>
<span class="source-line-no">593</span><span id="line-593"></span>
<span class="source-line-no">594</span><span id="line-594"> /**</span>
<span class="source-line-no">595</span><span id="line-595"> * Fetch table states from META table</span>
<span class="source-line-no">596</span><span id="line-596"> * @param conn connection to use</span>
<span class="source-line-no">597</span><span id="line-597"> * @return map {tableName -&amp;gt; state}</span>
<span class="source-line-no">598</span><span id="line-598"> */</span>
<span class="source-line-no">599</span><span id="line-599"> public static Map&lt;TableName, TableState&gt; getTableStates(Connection conn) throws IOException {</span>
<span class="source-line-no">600</span><span id="line-600"> final Map&lt;TableName, TableState&gt; states = new LinkedHashMap&lt;&gt;();</span>
<span class="source-line-no">601</span><span id="line-601"> ClientMetaTableAccessor.Visitor collector = r -&gt; {</span>
<span class="source-line-no">602</span><span id="line-602"> TableState state = CatalogFamilyFormat.getTableState(r);</span>
<span class="source-line-no">603</span><span id="line-603"> if (state != null) {</span>
<span class="source-line-no">604</span><span id="line-604"> states.put(state.getTableName(), state);</span>
<span class="source-line-no">605</span><span id="line-605"> }</span>
<span class="source-line-no">606</span><span id="line-606"> return true;</span>
<span class="source-line-no">607</span><span id="line-607"> };</span>
<span class="source-line-no">608</span><span id="line-608"> fullScanTables(conn, collector);</span>
<span class="source-line-no">609</span><span id="line-609"> return states;</span>
<span class="source-line-no">610</span><span id="line-610"> }</span>
<span class="source-line-no">611</span><span id="line-611"></span>
<span class="source-line-no">612</span><span id="line-612"> /**</span>
<span class="source-line-no">613</span><span id="line-613"> * Updates state in META Do not use. For internal use only.</span>
<span class="source-line-no">614</span><span id="line-614"> * @param conn connection to use</span>
<span class="source-line-no">615</span><span id="line-615"> * @param tableName table to look for</span>
<span class="source-line-no">616</span><span id="line-616"> */</span>
<span class="source-line-no">617</span><span id="line-617"> public static void updateTableState(Connection conn, TableName tableName, TableState.State actual)</span>
<span class="source-line-no">618</span><span id="line-618"> throws IOException {</span>
<span class="source-line-no">619</span><span id="line-619"> updateTableState(conn, new TableState(tableName, actual));</span>
<span class="source-line-no">620</span><span id="line-620"> }</span>
<span class="source-line-no">621</span><span id="line-621"></span>
<span class="source-line-no">622</span><span id="line-622"> ////////////////////////</span>
<span class="source-line-no">623</span><span id="line-623"> // Editing operations //</span>
<span class="source-line-no">624</span><span id="line-624"> ////////////////////////</span>
<span class="source-line-no">625</span><span id="line-625"></span>
<span class="source-line-no">626</span><span id="line-626"> /**</span>
<span class="source-line-no">627</span><span id="line-627"> * Generates and returns a {@link Put} containing the {@link RegionInfo} for the catalog table.</span>
<span class="source-line-no">628</span><span id="line-628"> * @throws IllegalArgumentException when the provided RegionInfo is not the default replica.</span>
<span class="source-line-no">629</span><span id="line-629"> */</span>
<span class="source-line-no">630</span><span id="line-630"> public static Put makePutFromRegionInfo(RegionInfo regionInfo) throws IOException {</span>
<span class="source-line-no">631</span><span id="line-631"> return makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());</span>
<span class="source-line-no">632</span><span id="line-632"> }</span>
<span class="source-line-no">633</span><span id="line-633"></span>
<span class="source-line-no">634</span><span id="line-634"> /**</span>
<span class="source-line-no">635</span><span id="line-635"> * Generates and returns a {@link Put} containing the {@link RegionInfo} for the catalog table.</span>
<span class="source-line-no">636</span><span id="line-636"> * @throws IllegalArgumentException when the provided RegionInfo is not the default replica.</span>
<span class="source-line-no">637</span><span id="line-637"> */</span>
<span class="source-line-no">638</span><span id="line-638"> public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts) throws IOException {</span>
<span class="source-line-no">639</span><span id="line-639"> return addRegionInfo(new Put(CatalogFamilyFormat.getMetaKeyForRegion(regionInfo), ts),</span>
<span class="source-line-no">640</span><span id="line-640"> regionInfo);</span>
<span class="source-line-no">641</span><span id="line-641"> }</span>
<span class="source-line-no">642</span><span id="line-642"></span>
<span class="source-line-no">643</span><span id="line-643"> /**</span>
<span class="source-line-no">644</span><span id="line-644"> * Generates and returns a Delete containing the region info for the catalog table</span>
<span class="source-line-no">645</span><span id="line-645"> */</span>
<span class="source-line-no">646</span><span id="line-646"> public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo, long ts) {</span>
<span class="source-line-no">647</span><span id="line-647"> if (regionInfo == null) {</span>
<span class="source-line-no">648</span><span id="line-648"> throw new IllegalArgumentException("Can't make a delete for null region");</span>
<span class="source-line-no">649</span><span id="line-649"> }</span>
<span class="source-line-no">650</span><span id="line-650"> if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {</span>
<span class="source-line-no">651</span><span id="line-651"> throw new IllegalArgumentException(</span>
<span class="source-line-no">652</span><span id="line-652"> "Can't make delete for a replica region. Operate on the primary");</span>
<span class="source-line-no">653</span><span id="line-653"> }</span>
<span class="source-line-no">654</span><span id="line-654"> Delete delete = new Delete(CatalogFamilyFormat.getMetaKeyForRegion(regionInfo));</span>
<span class="source-line-no">655</span><span id="line-655"> delete.addFamily(HConstants.CATALOG_FAMILY, ts);</span>
<span class="source-line-no">656</span><span id="line-656"> return delete;</span>
<span class="source-line-no">657</span><span id="line-657"> }</span>
<span class="source-line-no">658</span><span id="line-658"></span>
<span class="source-line-no">659</span><span id="line-659"> /**</span>
<span class="source-line-no">660</span><span id="line-660"> * Adds split daughters to the Put</span>
<span class="source-line-no">661</span><span id="line-661"> */</span>
<span class="source-line-no">662</span><span id="line-662"> public static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB)</span>
<span class="source-line-no">663</span><span id="line-663"> throws IOException {</span>
<span class="source-line-no">664</span><span id="line-664"> if (splitA != null) {</span>
<span class="source-line-no">665</span><span id="line-665"> put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())</span>
<span class="source-line-no">666</span><span id="line-666"> .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.SPLITA_QUALIFIER)</span>
<span class="source-line-no">667</span><span id="line-667"> .setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(RegionInfo.toByteArray(splitA))</span>
<span class="source-line-no">668</span><span id="line-668"> .build());</span>
<span class="source-line-no">669</span><span id="line-669"> }</span>
<span class="source-line-no">670</span><span id="line-670"> if (splitB != null) {</span>
<span class="source-line-no">671</span><span id="line-671"> put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())</span>
<span class="source-line-no">672</span><span id="line-672"> .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.SPLITB_QUALIFIER)</span>
<span class="source-line-no">673</span><span id="line-673"> .setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(RegionInfo.toByteArray(splitB))</span>
<span class="source-line-no">674</span><span id="line-674"> .build());</span>
<span class="source-line-no">675</span><span id="line-675"> }</span>
<span class="source-line-no">676</span><span id="line-676"> return put;</span>
<span class="source-line-no">677</span><span id="line-677"> }</span>
<span class="source-line-no">678</span><span id="line-678"></span>
<span class="source-line-no">679</span><span id="line-679"> /**</span>
<span class="source-line-no">680</span><span id="line-680"> * Put the passed &lt;code&gt;p&lt;/code&gt; to the &lt;code&gt;hbase:meta&lt;/code&gt; table.</span>
<span class="source-line-no">681</span><span id="line-681"> * @param connection connection we're using</span>
<span class="source-line-no">682</span><span id="line-682"> * @param p Put to add to hbase:meta</span>
<span class="source-line-no">683</span><span id="line-683"> */</span>
<span class="source-line-no">684</span><span id="line-684"> private static void putToMetaTable(Connection connection, Put p) throws IOException {</span>
<span class="source-line-no">685</span><span id="line-685"> try (Table table = getMetaHTable(connection)) {</span>
<span class="source-line-no">686</span><span id="line-686"> put(table, p);</span>
<span class="source-line-no">687</span><span id="line-687"> }</span>
<span class="source-line-no">688</span><span id="line-688"> }</span>
<span class="source-line-no">689</span><span id="line-689"></span>
<span class="source-line-no">690</span><span id="line-690"> /**</span>
<span class="source-line-no">691</span><span id="line-691"> * @param t Table to use</span>
<span class="source-line-no">692</span><span id="line-692"> * @param p put to make</span>
<span class="source-line-no">693</span><span id="line-693"> */</span>
<span class="source-line-no">694</span><span id="line-694"> private static void put(Table t, Put p) throws IOException {</span>
<span class="source-line-no">695</span><span id="line-695"> debugLogMutation(p);</span>
<span class="source-line-no">696</span><span id="line-696"> t.put(p);</span>
<span class="source-line-no">697</span><span id="line-697"> }</span>
<span class="source-line-no">698</span><span id="line-698"></span>
<span class="source-line-no">699</span><span id="line-699"> /**</span>
<span class="source-line-no">700</span><span id="line-700"> * Put the passed &lt;code&gt;ps&lt;/code&gt; to the &lt;code&gt;hbase:meta&lt;/code&gt; table.</span>
<span class="source-line-no">701</span><span id="line-701"> * @param connection connection we're using</span>
<span class="source-line-no">702</span><span id="line-702"> * @param ps Put to add to hbase:meta</span>
<span class="source-line-no">703</span><span id="line-703"> */</span>
<span class="source-line-no">704</span><span id="line-704"> public static void putsToMetaTable(final Connection connection, final List&lt;Put&gt; ps)</span>
<span class="source-line-no">705</span><span id="line-705"> throws IOException {</span>
<span class="source-line-no">706</span><span id="line-706"> if (ps.isEmpty()) {</span>
<span class="source-line-no">707</span><span id="line-707"> return;</span>
<span class="source-line-no">708</span><span id="line-708"> }</span>
<span class="source-line-no">709</span><span id="line-709"> try (Table t = getMetaHTable(connection)) {</span>
<span class="source-line-no">710</span><span id="line-710"> debugLogMutations(ps);</span>
<span class="source-line-no">711</span><span id="line-711"> // the implementation for putting a single Put is much simpler so here we do a check first.</span>
<span class="source-line-no">712</span><span id="line-712"> if (ps.size() == 1) {</span>
<span class="source-line-no">713</span><span id="line-713"> t.put(ps.get(0));</span>
<span class="source-line-no">714</span><span id="line-714"> } else {</span>
<span class="source-line-no">715</span><span id="line-715"> t.put(ps);</span>
<span class="source-line-no">716</span><span id="line-716"> }</span>
<span class="source-line-no">717</span><span id="line-717"> }</span>
<span class="source-line-no">718</span><span id="line-718"> }</span>
<span class="source-line-no">719</span><span id="line-719"></span>
<span class="source-line-no">720</span><span id="line-720"> /**</span>
<span class="source-line-no">721</span><span id="line-721"> * Delete the passed &lt;code&gt;d&lt;/code&gt; from the &lt;code&gt;hbase:meta&lt;/code&gt; table.</span>
<span class="source-line-no">722</span><span id="line-722"> * @param connection connection we're using</span>
<span class="source-line-no">723</span><span id="line-723"> * @param d Delete to add to hbase:meta</span>
<span class="source-line-no">724</span><span id="line-724"> */</span>
<span class="source-line-no">725</span><span id="line-725"> private static void deleteFromMetaTable(final Connection connection, final Delete d)</span>
<span class="source-line-no">726</span><span id="line-726"> throws IOException {</span>
<span class="source-line-no">727</span><span id="line-727"> List&lt;Delete&gt; dels = new ArrayList&lt;&gt;(1);</span>
<span class="source-line-no">728</span><span id="line-728"> dels.add(d);</span>
<span class="source-line-no">729</span><span id="line-729"> deleteFromMetaTable(connection, dels);</span>
<span class="source-line-no">730</span><span id="line-730"> }</span>
<span class="source-line-no">731</span><span id="line-731"></span>
<span class="source-line-no">732</span><span id="line-732"> /**</span>
<span class="source-line-no">733</span><span id="line-733"> * Delete the passed &lt;code&gt;deletes&lt;/code&gt; from the &lt;code&gt;hbase:meta&lt;/code&gt; table.</span>
<span class="source-line-no">734</span><span id="line-734"> * @param connection connection we're using</span>
<span class="source-line-no">735</span><span id="line-735"> * @param deletes Deletes to add to hbase:meta This list should support #remove.</span>
<span class="source-line-no">736</span><span id="line-736"> */</span>
<span class="source-line-no">737</span><span id="line-737"> private static void deleteFromMetaTable(final Connection connection, final List&lt;Delete&gt; deletes)</span>
<span class="source-line-no">738</span><span id="line-738"> throws IOException {</span>
<span class="source-line-no">739</span><span id="line-739"> try (Table t = getMetaHTable(connection)) {</span>
<span class="source-line-no">740</span><span id="line-740"> debugLogMutations(deletes);</span>
<span class="source-line-no">741</span><span id="line-741"> t.delete(deletes);</span>
<span class="source-line-no">742</span><span id="line-742"> }</span>
<span class="source-line-no">743</span><span id="line-743"> }</span>
<span class="source-line-no">744</span><span id="line-744"></span>
<span class="source-line-no">745</span><span id="line-745"> /**</span>
<span class="source-line-no">746</span><span id="line-746"> * Set the column value corresponding to this {@code replicaId}'s {@link RegionState} to the</span>
<span class="source-line-no">747</span><span id="line-747"> * provided {@code state}. Mutates the provided {@link Put}.</span>
<span class="source-line-no">748</span><span id="line-748"> */</span>
<span class="source-line-no">749</span><span id="line-749"> public static Put addRegionStateToPut(Put put, int replicaId, RegionState.State state)</span>
<span class="source-line-no">750</span><span id="line-750"> throws IOException {</span>
<span class="source-line-no">751</span><span id="line-751"> put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())</span>
<span class="source-line-no">752</span><span id="line-752"> .setFamily(HConstants.CATALOG_FAMILY)</span>
<span class="source-line-no">753</span><span id="line-753"> .setQualifier(CatalogFamilyFormat.getRegionStateColumn(replicaId))</span>
<span class="source-line-no">754</span><span id="line-754"> .setTimestamp(put.getTimestamp()).setType(Cell.Type.Put).setValue(Bytes.toBytes(state.name()))</span>
<span class="source-line-no">755</span><span id="line-755"> .build());</span>
<span class="source-line-no">756</span><span id="line-756"> return put;</span>
<span class="source-line-no">757</span><span id="line-757"> }</span>
<span class="source-line-no">758</span><span id="line-758"></span>
<span class="source-line-no">759</span><span id="line-759"> /**</span>
<span class="source-line-no">760</span><span id="line-760"> * Update state column in hbase:meta.</span>
<span class="source-line-no">761</span><span id="line-761"> */</span>
<span class="source-line-no">762</span><span id="line-762"> public static void updateRegionState(Connection connection, RegionInfo ri,</span>
<span class="source-line-no">763</span><span id="line-763"> RegionState.State state) throws IOException {</span>
<span class="source-line-no">764</span><span id="line-764"> final Put put = makePutFromRegionInfo(ri);</span>
<span class="source-line-no">765</span><span id="line-765"> addRegionStateToPut(put, ri.getReplicaId(), state);</span>
<span class="source-line-no">766</span><span id="line-766"> putsToMetaTable(connection, Collections.singletonList(put));</span>
<span class="source-line-no">767</span><span id="line-767"> }</span>
<span class="source-line-no">768</span><span id="line-768"></span>
<span class="source-line-no">769</span><span id="line-769"> /**</span>
<span class="source-line-no">770</span><span id="line-770"> * Adds daughter region infos to hbase:meta row for the specified region.</span>
<span class="source-line-no">771</span><span id="line-771"> * &lt;p/&gt;</span>
<span class="source-line-no">772</span><span id="line-772"> * Note that this does not add its daughter's as different rows, but adds information about the</span>
<span class="source-line-no">773</span><span id="line-773"> * daughters in the same row as the parent. Now only used in snapshot. Use</span>
<span class="source-line-no">774</span><span id="line-774"> * {@link org.apache.hadoop.hbase.master.assignment.RegionStateStore} if you want to split a</span>
<span class="source-line-no">775</span><span id="line-775"> * region.</span>
<span class="source-line-no">776</span><span id="line-776"> * @param connection connection we're using</span>
<span class="source-line-no">777</span><span id="line-777"> * @param regionInfo RegionInfo of parent region</span>
<span class="source-line-no">778</span><span id="line-778"> * @param splitA first split daughter of the parent regionInfo</span>
<span class="source-line-no">779</span><span id="line-779"> * @param splitB second split daughter of the parent regionInfo</span>
<span class="source-line-no">780</span><span id="line-780"> * @throws IOException if problem connecting or updating meta</span>
<span class="source-line-no">781</span><span id="line-781"> */</span>
<span class="source-line-no">782</span><span id="line-782"> public static void addSplitsToParent(Connection connection, RegionInfo regionInfo,</span>
<span class="source-line-no">783</span><span id="line-783"> RegionInfo splitA, RegionInfo splitB) throws IOException {</span>
<span class="source-line-no">784</span><span id="line-784"> try (Table meta = getMetaHTable(connection)) {</span>
<span class="source-line-no">785</span><span id="line-785"> Put put = makePutFromRegionInfo(regionInfo);</span>
<span class="source-line-no">786</span><span id="line-786"> addDaughtersToPut(put, splitA, splitB);</span>
<span class="source-line-no">787</span><span id="line-787"> meta.put(put);</span>
<span class="source-line-no">788</span><span id="line-788"> debugLogMutation(put);</span>
<span class="source-line-no">789</span><span id="line-789"> LOG.debug("Added region {}", regionInfo.getRegionNameAsString());</span>
<span class="source-line-no">790</span><span id="line-790"> }</span>
<span class="source-line-no">791</span><span id="line-791"> }</span>
<span class="source-line-no">792</span><span id="line-792"></span>
<span class="source-line-no">793</span><span id="line-793"> /**</span>
<span class="source-line-no">794</span><span id="line-794"> * Adds a hbase:meta row for each of the specified new regions. Initial state for new regions is</span>
<span class="source-line-no">795</span><span id="line-795"> * CLOSED.</span>
<span class="source-line-no">796</span><span id="line-796"> * @param connection connection we're using</span>
<span class="source-line-no">797</span><span id="line-797"> * @param regionInfos region information list</span>
<span class="source-line-no">798</span><span id="line-798"> * @throws IOException if problem connecting or updating meta</span>
<span class="source-line-no">799</span><span id="line-799"> */</span>
<span class="source-line-no">800</span><span id="line-800"> public static void addRegionsToMeta(Connection connection, List&lt;RegionInfo&gt; regionInfos,</span>
<span class="source-line-no">801</span><span id="line-801"> int regionReplication) throws IOException {</span>
<span class="source-line-no">802</span><span id="line-802"> addRegionsToMeta(connection, regionInfos, regionReplication,</span>
<span class="source-line-no">803</span><span id="line-803"> EnvironmentEdgeManager.currentTime());</span>
<span class="source-line-no">804</span><span id="line-804"> }</span>
<span class="source-line-no">805</span><span id="line-805"></span>
<span class="source-line-no">806</span><span id="line-806"> /**</span>
<span class="source-line-no">807</span><span id="line-807"> * Adds a hbase:meta row for each of the specified new regions. Initial state for new regions is</span>
<span class="source-line-no">808</span><span id="line-808"> * CLOSED.</span>
<span class="source-line-no">809</span><span id="line-809"> * @param connection connection we're using</span>
<span class="source-line-no">810</span><span id="line-810"> * @param regionInfos region information list</span>
<span class="source-line-no">811</span><span id="line-811"> * @param ts desired timestamp</span>
<span class="source-line-no">812</span><span id="line-812"> * @throws IOException if problem connecting or updating meta</span>
<span class="source-line-no">813</span><span id="line-813"> */</span>
<span class="source-line-no">814</span><span id="line-814"> public static void addRegionsToMeta(Connection connection, List&lt;RegionInfo&gt; regionInfos,</span>
<span class="source-line-no">815</span><span id="line-815"> int regionReplication, long ts) throws IOException {</span>
<span class="source-line-no">816</span><span id="line-816"> List&lt;Put&gt; puts = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">817</span><span id="line-817"> for (RegionInfo regionInfo : regionInfos) {</span>
<span class="source-line-no">818</span><span id="line-818"> if (!RegionReplicaUtil.isDefaultReplica(regionInfo)) {</span>
<span class="source-line-no">819</span><span id="line-819"> continue;</span>
<span class="source-line-no">820</span><span id="line-820"> }</span>
<span class="source-line-no">821</span><span id="line-821"> Put put = makePutFromRegionInfo(regionInfo, ts);</span>
<span class="source-line-no">822</span><span id="line-822"> // New regions are added with initial state of CLOSED.</span>
<span class="source-line-no">823</span><span id="line-823"> addRegionStateToPut(put, regionInfo.getReplicaId(), RegionState.State.CLOSED);</span>
<span class="source-line-no">824</span><span id="line-824"> // Add empty locations for region replicas so that number of replicas can be cached</span>
<span class="source-line-no">825</span><span id="line-825"> // whenever the primary region is looked up from meta</span>
<span class="source-line-no">826</span><span id="line-826"> for (int i = 1; i &lt; regionReplication; i++) {</span>
<span class="source-line-no">827</span><span id="line-827"> addEmptyLocation(put, i);</span>
<span class="source-line-no">828</span><span id="line-828"> }</span>
<span class="source-line-no">829</span><span id="line-829"> puts.add(put);</span>
<span class="source-line-no">830</span><span id="line-830"> }</span>
<span class="source-line-no">831</span><span id="line-831"> putsToMetaTable(connection, puts);</span>
<span class="source-line-no">832</span><span id="line-832"> LOG.info("Added {} regions to meta.", puts.size());</span>
<span class="source-line-no">833</span><span id="line-833"> }</span>
<span class="source-line-no">834</span><span id="line-834"></span>
<span class="source-line-no">835</span><span id="line-835"> /**</span>
<span class="source-line-no">836</span><span id="line-836"> * Update state of the table in meta.</span>
<span class="source-line-no">837</span><span id="line-837"> * @param connection what we use for update</span>
<span class="source-line-no">838</span><span id="line-838"> * @param state new state</span>
<span class="source-line-no">839</span><span id="line-839"> */</span>
<span class="source-line-no">840</span><span id="line-840"> private static void updateTableState(Connection connection, TableState state) throws IOException {</span>
<span class="source-line-no">841</span><span id="line-841"> Put put = makePutFromTableState(state, EnvironmentEdgeManager.currentTime());</span>
<span class="source-line-no">842</span><span id="line-842"> putToMetaTable(connection, put);</span>
<span class="source-line-no">843</span><span id="line-843"> LOG.info("Updated {} in hbase:meta", state);</span>
<span class="source-line-no">844</span><span id="line-844"> }</span>
<span class="source-line-no">845</span><span id="line-845"></span>
<span class="source-line-no">846</span><span id="line-846"> /**</span>
<span class="source-line-no">847</span><span id="line-847"> * Construct PUT for given state</span>
<span class="source-line-no">848</span><span id="line-848"> * @param state new state</span>
<span class="source-line-no">849</span><span id="line-849"> */</span>
<span class="source-line-no">850</span><span id="line-850"> public static Put makePutFromTableState(TableState state, long ts) {</span>
<span class="source-line-no">851</span><span id="line-851"> Put put = new Put(state.getTableName().getName(), ts);</span>
<span class="source-line-no">852</span><span id="line-852"> put.addColumn(HConstants.TABLE_FAMILY, HConstants.TABLE_STATE_QUALIFIER,</span>
<span class="source-line-no">853</span><span id="line-853"> state.convert().toByteArray());</span>
<span class="source-line-no">854</span><span id="line-854"> return put;</span>
<span class="source-line-no">855</span><span id="line-855"> }</span>
<span class="source-line-no">856</span><span id="line-856"></span>
<span class="source-line-no">857</span><span id="line-857"> /**</span>
<span class="source-line-no">858</span><span id="line-858"> * Remove state for table from meta</span>
<span class="source-line-no">859</span><span id="line-859"> * @param connection to use for deletion</span>
<span class="source-line-no">860</span><span id="line-860"> * @param table to delete state for</span>
<span class="source-line-no">861</span><span id="line-861"> */</span>
<span class="source-line-no">862</span><span id="line-862"> public static void deleteTableState(Connection connection, TableName table) throws IOException {</span>
<span class="source-line-no">863</span><span id="line-863"> long time = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">864</span><span id="line-864"> Delete delete = new Delete(table.getName());</span>
<span class="source-line-no">865</span><span id="line-865"> delete.addColumns(HConstants.TABLE_FAMILY, HConstants.TABLE_STATE_QUALIFIER, time);</span>
<span class="source-line-no">866</span><span id="line-866"> deleteFromMetaTable(connection, delete);</span>
<span class="source-line-no">867</span><span id="line-867"> LOG.info("Deleted table " + table + " state from META");</span>
<span class="source-line-no">868</span><span id="line-868"> }</span>
<span class="source-line-no">869</span><span id="line-869"></span>
<span class="source-line-no">870</span><span id="line-870"> /**</span>
<span class="source-line-no">871</span><span id="line-871"> * Updates the location of the specified region in hbase:meta to be the specified server hostname</span>
<span class="source-line-no">872</span><span id="line-872"> * and startcode.</span>
<span class="source-line-no">873</span><span id="line-873"> * &lt;p&gt;</span>
<span class="source-line-no">874</span><span id="line-874"> * Uses passed catalog tracker to get a connection to the server hosting hbase:meta and makes</span>
<span class="source-line-no">875</span><span id="line-875"> * edits to that region.</span>
<span class="source-line-no">876</span><span id="line-876"> * @param connection connection we're using</span>
<span class="source-line-no">877</span><span id="line-877"> * @param regionInfo region to update location of</span>
<span class="source-line-no">878</span><span id="line-878"> * @param openSeqNum the latest sequence number obtained when the region was open</span>
<span class="source-line-no">879</span><span id="line-879"> * @param sn Server name</span>
<span class="source-line-no">880</span><span id="line-880"> * @param masterSystemTime wall clock time from master if passed in the open region RPC</span>
<span class="source-line-no">881</span><span id="line-881"> */</span>
<span class="source-line-no">882</span><span id="line-882"> public static void updateRegionLocation(Connection connection, RegionInfo regionInfo,</span>
<span class="source-line-no">883</span><span id="line-883"> ServerName sn, long openSeqNum, long masterSystemTime) throws IOException {</span>
<span class="source-line-no">884</span><span id="line-884"> updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime);</span>
<span class="source-line-no">885</span><span id="line-885"> }</span>
<span class="source-line-no">886</span><span id="line-886"></span>
<span class="source-line-no">887</span><span id="line-887"> /**</span>
<span class="source-line-no">888</span><span id="line-888"> * Updates the location of the specified region to be the specified server.</span>
<span class="source-line-no">889</span><span id="line-889"> * &lt;p&gt;</span>
<span class="source-line-no">890</span><span id="line-890"> * Connects to the specified server which should be hosting the specified catalog region name to</span>
<span class="source-line-no">891</span><span id="line-891"> * perform the edit.</span>
<span class="source-line-no">892</span><span id="line-892"> * @param connection connection we're using</span>
<span class="source-line-no">893</span><span id="line-893"> * @param regionInfo region to update location of</span>
<span class="source-line-no">894</span><span id="line-894"> * @param sn Server name</span>
<span class="source-line-no">895</span><span id="line-895"> * @param openSeqNum the latest sequence number obtained when the region was open</span>
<span class="source-line-no">896</span><span id="line-896"> * @param masterSystemTime wall clock time from master if passed in the open region RPC</span>
<span class="source-line-no">897</span><span id="line-897"> * @throws IOException In particular could throw {@link java.net.ConnectException} if the server</span>
<span class="source-line-no">898</span><span id="line-898"> * is down on other end.</span>
<span class="source-line-no">899</span><span id="line-899"> */</span>
<span class="source-line-no">900</span><span id="line-900"> private static void updateLocation(Connection connection, RegionInfo regionInfo, ServerName sn,</span>
<span class="source-line-no">901</span><span id="line-901"> long openSeqNum, long masterSystemTime) throws IOException {</span>
<span class="source-line-no">902</span><span id="line-902"> // region replicas are kept in the primary region's row</span>
<span class="source-line-no">903</span><span id="line-903"> Put put = new Put(CatalogFamilyFormat.getMetaKeyForRegion(regionInfo), masterSystemTime);</span>
<span class="source-line-no">904</span><span id="line-904"> addRegionInfo(put, regionInfo);</span>
<span class="source-line-no">905</span><span id="line-905"> addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());</span>
<span class="source-line-no">906</span><span id="line-906"> putToMetaTable(connection, put);</span>
<span class="source-line-no">907</span><span id="line-907"> LOG.info("Updated row {} with server=", regionInfo.getRegionNameAsString(), sn);</span>
<span class="source-line-no">908</span><span id="line-908"> }</span>
<span class="source-line-no">909</span><span id="line-909"></span>
<span class="source-line-no">910</span><span id="line-910"> public static Put addRegionInfo(final Put p, final RegionInfo hri) throws IOException {</span>
<span class="source-line-no">911</span><span id="line-911"> p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(p.getRow())</span>
<span class="source-line-no">912</span><span id="line-912"> .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.REGIONINFO_QUALIFIER)</span>
<span class="source-line-no">913</span><span id="line-913"> .setTimestamp(p.getTimestamp()).setType(Type.Put)</span>
<span class="source-line-no">914</span><span id="line-914"> // Serialize the Default Replica HRI otherwise scan of hbase:meta</span>
<span class="source-line-no">915</span><span id="line-915"> // shows an info:regioninfo value with encoded name and region</span>
<span class="source-line-no">916</span><span id="line-916"> // name that differs from that of the hbase;meta row.</span>
<span class="source-line-no">917</span><span id="line-917"> .setValue(RegionInfo.toByteArray(RegionReplicaUtil.getRegionInfoForDefaultReplica(hri)))</span>
<span class="source-line-no">918</span><span id="line-918"> .build());</span>
<span class="source-line-no">919</span><span id="line-919"> return p;</span>
<span class="source-line-no">920</span><span id="line-920"> }</span>
<span class="source-line-no">921</span><span id="line-921"></span>
<span class="source-line-no">922</span><span id="line-922"> public static Put addLocation(Put p, ServerName sn, long openSeqNum, int replicaId)</span>
<span class="source-line-no">923</span><span id="line-923"> throws IOException {</span>
<span class="source-line-no">924</span><span id="line-924"> CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);</span>
<span class="source-line-no">925</span><span id="line-925"> return p</span>
<span class="source-line-no">926</span><span id="line-926"> .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)</span>
<span class="source-line-no">927</span><span id="line-927"> .setQualifier(CatalogFamilyFormat.getServerColumn(replicaId)).setTimestamp(p.getTimestamp())</span>
<span class="source-line-no">928</span><span id="line-928"> .setType(Cell.Type.Put).setValue(Bytes.toBytes(sn.getAddress().toString())).build())</span>
<span class="source-line-no">929</span><span id="line-929"> .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)</span>
<span class="source-line-no">930</span><span id="line-930"> .setQualifier(CatalogFamilyFormat.getStartCodeColumn(replicaId))</span>
<span class="source-line-no">931</span><span id="line-931"> .setTimestamp(p.getTimestamp()).setType(Cell.Type.Put)</span>
<span class="source-line-no">932</span><span id="line-932"> .setValue(Bytes.toBytes(sn.getStartcode())).build())</span>
<span class="source-line-no">933</span><span id="line-933"> .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)</span>
<span class="source-line-no">934</span><span id="line-934"> .setQualifier(CatalogFamilyFormat.getSeqNumColumn(replicaId)).setTimestamp(p.getTimestamp())</span>
<span class="source-line-no">935</span><span id="line-935"> .setType(Type.Put).setValue(Bytes.toBytes(openSeqNum)).build());</span>
<span class="source-line-no">936</span><span id="line-936"> }</span>
<span class="source-line-no">937</span><span id="line-937"></span>
<span class="source-line-no">938</span><span id="line-938"> public static Put addEmptyLocation(Put p, int replicaId) throws IOException {</span>
<span class="source-line-no">939</span><span id="line-939"> CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);</span>
<span class="source-line-no">940</span><span id="line-940"> return p</span>
<span class="source-line-no">941</span><span id="line-941"> .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)</span>
<span class="source-line-no">942</span><span id="line-942"> .setQualifier(CatalogFamilyFormat.getServerColumn(replicaId)).setTimestamp(p.getTimestamp())</span>
<span class="source-line-no">943</span><span id="line-943"> .setType(Type.Put).build())</span>
<span class="source-line-no">944</span><span id="line-944"> .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)</span>
<span class="source-line-no">945</span><span id="line-945"> .setQualifier(CatalogFamilyFormat.getStartCodeColumn(replicaId))</span>
<span class="source-line-no">946</span><span id="line-946"> .setTimestamp(p.getTimestamp()).setType(Cell.Type.Put).build())</span>
<span class="source-line-no">947</span><span id="line-947"> .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)</span>
<span class="source-line-no">948</span><span id="line-948"> .setQualifier(CatalogFamilyFormat.getSeqNumColumn(replicaId)).setTimestamp(p.getTimestamp())</span>
<span class="source-line-no">949</span><span id="line-949"> .setType(Cell.Type.Put).build());</span>
<span class="source-line-no">950</span><span id="line-950"> }</span>
<span class="source-line-no">951</span><span id="line-951"></span>
<span class="source-line-no">952</span><span id="line-952"> private static void debugLogMutations(List&lt;? extends Mutation&gt; mutations) throws IOException {</span>
<span class="source-line-no">953</span><span id="line-953"> if (!METALOG.isDebugEnabled()) {</span>
<span class="source-line-no">954</span><span id="line-954"> return;</span>
<span class="source-line-no">955</span><span id="line-955"> }</span>
<span class="source-line-no">956</span><span id="line-956"> // Logging each mutation in separate line makes it easier to see diff between them visually</span>
<span class="source-line-no">957</span><span id="line-957"> // because of common starting indentation.</span>
<span class="source-line-no">958</span><span id="line-958"> for (Mutation mutation : mutations) {</span>
<span class="source-line-no">959</span><span id="line-959"> debugLogMutation(mutation);</span>
<span class="source-line-no">960</span><span id="line-960"> }</span>
<span class="source-line-no">961</span><span id="line-961"> }</span>
<span class="source-line-no">962</span><span id="line-962"></span>
<span class="source-line-no">963</span><span id="line-963"> private static void debugLogMutation(Mutation p) throws IOException {</span>
<span class="source-line-no">964</span><span id="line-964"> METALOG.debug("{} {}", p.getClass().getSimpleName(), p.toJSON());</span>
<span class="source-line-no">965</span><span id="line-965"> }</span>
<span class="source-line-no">966</span><span id="line-966">}</span>
</pre>
</div>
</main>
</body>
</html>